From 73c2daa68a1a29c53b23137a5cf0012805ffeda8 Mon Sep 17 00:00:00 2001 From: hemanthsavasere Date: Sun, 15 Mar 2026 18:27:30 +0000 Subject: [PATCH 1/2] feat: add end-to-end ROW (nested struct) column serialization support - Add `Datum::Row(Box)` variant with `as_row()` accessor - Add `get_row()` to `InternalRow` trait with default error impl - Implement `GenericRow::get_row()` and `CompactedRow::get_row()` delegation - Implement `ColumnarRow::get_row()` with Arrow StructArray extraction + OnceLock caching - Add `InnerValueWriter::Row(RowType)` and write path via nested CompactedRowWriter - Add `DataType::Row` arm in `CompactedRowDeserializer` for eager nested decode - Add `InnerFieldGetter::Row` and hook up FieldGetter/ValueWriter pipeline - Handle `Datum::Row` in `resolve_row_types` (C++ bindings) - Add round-trip tests: simple nesting, deep nesting, nullable fields, ROW as primary key Wire format matches Java: varint-length-prefixed blob of a complete CompactedRow. --- bindings/cpp/src/types.rs | 1 + crates/fluss/src/row/binary/binary_writer.rs | 25 +- crates/fluss/src/row/column.rs | 300 +++++++++++++++++- .../fluss/src/row/compacted/compacted_row.rs | 4 + .../src/row/compacted/compacted_row_reader.rs | 151 +++++++++ crates/fluss/src/row/datum.rs | 15 + .../src/row/encode/compacted_key_encoder.rs | 48 ++- crates/fluss/src/row/field_getter.rs | 12 +- crates/fluss/src/row/mod.rs | 19 +- 9 files changed, 563 insertions(+), 12 deletions(-) diff --git a/bindings/cpp/src/types.rs b/bindings/cpp/src/types.rs index f61711b5..19aa9a36 100644 --- a/bindings/cpp/src/types.rs +++ b/bindings/cpp/src/types.rs @@ -529,6 +529,7 @@ pub fn resolve_row_types( Datum::TimestampNtz(ts) => Datum::TimestampNtz(*ts), Datum::TimestampLtz(ts) => Datum::TimestampLtz(*ts), Datum::Array(a) => Datum::Array(a.clone()), + Datum::Row(r) => Datum::Row(Box::new(resolve_row_types(r, None)?)), }; out.set_field(idx, resolved); } diff --git a/crates/fluss/src/row/binary/binary_writer.rs b/crates/fluss/src/row/binary/binary_writer.rs index f51a6e80..fd7da4c9 100644 --- a/crates/fluss/src/row/binary/binary_writer.rs +++ b/crates/fluss/src/row/binary/binary_writer.rs @@ -17,7 +17,7 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; -use crate::metadata::DataType; +use crate::metadata::{DataType, RowType}; use crate::row::Datum; use crate::row::binary::BinaryRowFormat; @@ -136,7 +136,7 @@ pub enum InnerValueWriter { TimestampNtz(u32), // precision TimestampLtz(u32), // precision Array, - // TODO Row + Row(RowType), } /// Accessor for writing the fields/elements of a binary writer during runtime, the @@ -176,6 +176,7 @@ impl InnerValueWriter { Ok(InnerValueWriter::TimestampLtz(t.precision())) } DataType::Array(_) => Ok(InnerValueWriter::Array), + DataType::Row(row_type) => Ok(InnerValueWriter::Row(row_type.clone())), _ => unimplemented!( "ValueWriter for DataType {:?} is currently not implemented", data_type @@ -241,6 +242,26 @@ impl InnerValueWriter { (InnerValueWriter::Array, Datum::Array(arr)) => { writer.write_array(arr.as_bytes()); } + (InnerValueWriter::Row(row_type), Datum::Row(inner_row)) => { + use crate::row::compacted::CompactedRowWriter; + let field_count = row_type.fields().len(); + let mut nested = CompactedRowWriter::new(field_count); + for (i, field) in row_type.fields().iter().enumerate() { + let datum = &inner_row.values[i]; + if datum.is_null() { + if field.data_type.is_nullable() { + nested.set_null_at(i); + } + } else { + let vw = + InnerValueWriter::create_inner_value_writer(&field.data_type, None) + .expect("create_inner_value_writer failed for nested row field"); + vw.write_value(&mut nested, i, datum) + .expect("write_value failed for nested row field"); + } + } + writer.write_bytes(nested.buffer()); + } _ => { return Err(IllegalArgument { message: format!("{self:?} used to write value {value:?}"), diff --git a/crates/fluss/src/row/column.rs b/crates/fluss/src/row/column.rs index be5b850d..95cd3395 100644 --- a/crates/fluss/src/row/column.rs +++ b/crates/fluss/src/row/column.rs @@ -17,8 +17,8 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; -use crate::row::InternalRow; -use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; +use crate::row::{GenericRow, InternalRow}; +use crate::row::datum::{Date, Datum, Time, TimestampLtz, TimestampNtz}; use arrow::array::{ Array, AsArray, BinaryArray, BooleanArray, FixedSizeBinaryArray, ListArray, RecordBatch, StringArray, @@ -35,25 +35,33 @@ use std::sync::Arc; pub struct ColumnarRow { record_batch: Arc, row_id: usize, + nested_rows: Vec>>, } impl ColumnarRow { pub fn new(batch: Arc) -> Self { + let num_cols = batch.num_columns(); ColumnarRow { record_batch: batch, row_id: 0, + nested_rows: (0..num_cols).map(|_| std::sync::OnceLock::new()).collect(), } } pub fn new_with_row_id(bach: Arc, row_id: usize) -> Self { + let num_cols = bach.num_columns(); ColumnarRow { record_batch: bach, row_id, + nested_rows: (0..num_cols).map(|_| std::sync::OnceLock::new()).collect(), } } pub fn set_row_id(&mut self, row_id: usize) { - self.row_id = row_id + self.row_id = row_id; + for lock in &mut self.nested_rows { + *lock = std::sync::OnceLock::new(); + } } pub fn get_row_id(&self) -> usize { @@ -212,6 +220,168 @@ impl ColumnarRow { }), } } + + /// Extract a `GenericRow<'static>` from a column in the RecordBatch at the given row_id. + fn extract_struct_at( + batch: &RecordBatch, + pos: usize, + row_id: usize, + ) -> Result> { + let col = batch.column(pos); + Self::extract_struct_from_array(col.as_ref(), row_id) + } + + /// Recursively extract a `GenericRow<'static>` from a `StructArray` at row_id. + fn extract_struct_from_array(array: &dyn Array, row_id: usize) -> Result> { + use arrow::array::StructArray; + let sa = array + .as_any() + .downcast_ref::() + .ok_or_else(|| IllegalArgument { + message: format!("expected StructArray, got {:?}", array.data_type()), + })?; + let mut values = Vec::with_capacity(sa.num_columns()); + for i in 0..sa.num_columns() { + let child = sa.column(i); + values.push(Self::arrow_value_to_datum(child.as_ref(), row_id)?); + } + Ok(GenericRow { values }) + } + + /// Convert a single element at `row_id` in an Arrow array to a `Datum<'static>`. + fn arrow_value_to_datum(array: &dyn Array, row_id: usize) -> Result> { + use arrow::array::{ + BooleanArray, Decimal128Array, Float32Array, Float64Array, Int8Array, Int16Array, + Int32Array, Int64Array, Time32MillisecondArray, Time32SecondArray, + Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, + TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, + }; + use crate::row::Decimal; + + if array.is_null(row_id) { + return Ok(Datum::Null); + } + + match array.data_type() { + ArrowDataType::Boolean => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Bool(a.value(row_id))) + } + ArrowDataType::Int8 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Int8(a.value(row_id))) + } + ArrowDataType::Int16 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Int16(a.value(row_id))) + } + ArrowDataType::Int32 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Int32(a.value(row_id))) + } + ArrowDataType::Int64 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Int64(a.value(row_id))) + } + ArrowDataType::Float32 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Float32(a.value(row_id).into())) + } + ArrowDataType::Float64 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Float64(a.value(row_id).into())) + } + ArrowDataType::Utf8 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::String(std::borrow::Cow::Owned(a.value(row_id).to_owned()))) + } + ArrowDataType::Binary => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Blob(std::borrow::Cow::Owned(a.value(row_id).to_vec()))) + } + ArrowDataType::Decimal128(p, s) => { + let (p, s) = (*p, *s); + let a = array.as_any().downcast_ref::().unwrap(); + let i128_val = a.value(row_id); + Ok(Datum::Decimal(Decimal::from_arrow_decimal128( + i128_val, + s as i64, + p as u32, + s as u32, + )?)) + } + ArrowDataType::Date32 => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Date(Date::new(a.value(row_id)))) + } + ArrowDataType::Time32(TimeUnit::Second) => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Time(Time::new(a.value(row_id) * 1000))) + } + ArrowDataType::Time32(TimeUnit::Millisecond) => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Time(Time::new(a.value(row_id)))) + } + ArrowDataType::Time64(TimeUnit::Microsecond) => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Time(Time::new((a.value(row_id) / 1000) as i32))) + } + ArrowDataType::Time64(TimeUnit::Nanosecond) => { + let a = array.as_any().downcast_ref::().unwrap(); + Ok(Datum::Time(Time::new((a.value(row_id) / 1_000_000) as i32))) + } + ArrowDataType::Timestamp(time_unit, tz) => { + let value: i64 = match time_unit { + TimeUnit::Second => { + array.as_any().downcast_ref::().unwrap().value(row_id) + } + TimeUnit::Millisecond => { + array.as_any().downcast_ref::().unwrap().value(row_id) + } + TimeUnit::Microsecond => { + array.as_any().downcast_ref::().unwrap().value(row_id) + } + TimeUnit::Nanosecond => { + array.as_any().downcast_ref::().unwrap().value(row_id) + } + }; + let (millis, nanos) = match time_unit { + TimeUnit::Second => (value * 1000, 0i32), + TimeUnit::Millisecond => (value, 0i32), + TimeUnit::Microsecond => { + let millis = value.div_euclid(1000); + let nanos = (value.rem_euclid(1000) * 1000) as i32; + (millis, nanos) + } + TimeUnit::Nanosecond => { + let millis = value.div_euclid(1_000_000); + let nanos = value.rem_euclid(1_000_000) as i32; + (millis, nanos) + } + }; + if tz.is_some() { + if nanos == 0 { + Ok(Datum::TimestampLtz(TimestampLtz::new(millis))) + } else { + Ok(Datum::TimestampLtz(TimestampLtz::from_millis_nanos(millis, nanos)?)) + } + } else if nanos == 0 { + Ok(Datum::TimestampNtz(TimestampNtz::new(millis))) + } else { + Ok(Datum::TimestampNtz(TimestampNtz::from_millis_nanos(millis, nanos)?)) + } + } + ArrowDataType::Struct(_) => { + let nested = Self::extract_struct_from_array(array, row_id)?; + Ok(Datum::Row(Box::new(nested))) + } + other => Err(IllegalArgument { + message: format!( + "unsupported Arrow data type for nested row extraction: {other:?}" + ), + }), + } + } } impl InternalRow for ColumnarRow { @@ -433,6 +603,18 @@ impl InternalRow for ColumnarRow { write_arrow_values_to_fluss_array(&*values, &element_fluss_type, &mut writer)?; writer.complete() } + + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + let lock = self.nested_rows.get(pos).ok_or_else(|| IllegalArgument { + message: format!("column index {pos} out of bounds for get_row"), + })?; + let batch = Arc::clone(&self.record_batch); + let row_id = self.row_id; + Ok(lock.get_or_init(|| { + Self::extract_struct_at(&batch, pos, row_id) + .expect("failed to extract nested row from StructArray") + })) + } } /// Downcast to a primitive Arrow array type, then loop with null checks calling a writer method. @@ -794,9 +976,9 @@ mod tests { use arrow::array::{ ArrayRef, BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array, Int8Array, Int16Array, Int32Array, Int32Builder, Int64Array, ListBuilder, StringArray, - UInt32Builder, + StructArray, UInt32Builder, }; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::{DataType, Field, Fields, Schema}; fn single_column_row(array: ArrayRef) -> ColumnarRow { let batch = @@ -1011,4 +1193,112 @@ mod tests { "unexpected error: {err}" ); } + + fn make_struct_batch( + field_name: &str, + child_fields: Fields, + child_arrays: Vec>, + _num_rows: usize, + ) -> Arc { + let struct_array = StructArray::new(child_fields.clone(), child_arrays, None); + let schema = Arc::new(Schema::new(vec![Field::new( + field_name, + DataType::Struct(child_fields), + false, + )])); + Arc::new( + RecordBatch::try_new(schema, vec![Arc::new(struct_array)]) + .expect("record batch"), + ) + } + + #[test] + fn columnar_row_reads_nested_row() { + // Build a RecordBatch with a Struct column: {i32, string} + let child_fields = Fields::from(vec![ + Field::new("x", DataType::Int32, false), + Field::new("s", DataType::Utf8, false), + ]); + let child_arrays: Vec> = vec![ + Arc::new(Int32Array::from(vec![42, 99])), + Arc::new(StringArray::from(vec!["hello", "world"])), + ]; + let batch = make_struct_batch("nested", child_fields, child_arrays, 2); + + let mut row = ColumnarRow::new(batch); + + // row_id = 0 + let nested = row.get_row(0).unwrap(); + assert_eq!(nested.get_field_count(), 2); + assert_eq!(nested.get_int(0).unwrap(), 42); + assert_eq!(nested.get_string(1).unwrap(), "hello"); + + // row_id = 1 + row.set_row_id(1); + let nested = row.get_row(0).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 99); + assert_eq!(nested.get_string(1).unwrap(), "world"); + } + + #[test] + fn columnar_row_reads_deeply_nested_row() { + // Build: outer struct { i32, inner struct { string } } + let inner_fields = Fields::from(vec![Field::new("s", DataType::Utf8, false)]); + let inner_array = Arc::new(StructArray::new( + inner_fields.clone(), + vec![Arc::new(StringArray::from(vec!["deep", "deeper"])) as Arc], + None, + )); + + let outer_fields = Fields::from(vec![ + Field::new("n", DataType::Int32, false), + Field::new("inner", DataType::Struct(inner_fields), false), + ]); + let outer_array = Arc::new(StructArray::new( + outer_fields.clone(), + vec![ + Arc::new(Int32Array::from(vec![1, 2])) as Arc, + inner_array as Arc, + ], + None, + )); + + let schema = Arc::new(Schema::new(vec![Field::new( + "outer", + DataType::Struct(outer_fields), + false, + )])); + let batch = Arc::new( + RecordBatch::try_new(schema, vec![outer_array]).expect("record batch"), + ); + + let row = ColumnarRow::new(batch); + + // Access outer struct at column 0, row 0 + let outer = row.get_row(0).unwrap(); + assert_eq!(outer.get_int(0).unwrap(), 1); + + // Access inner struct (column 1 of outer) + let inner = outer.get_row(1).unwrap(); + assert_eq!(inner.get_string(0).unwrap(), "deep"); + } + + #[test] + fn columnar_row_get_row_cache_invalidated_on_set_row_id() { + let child_fields = Fields::from(vec![Field::new("x", DataType::Int32, false)]); + let child_arrays: Vec> = + vec![Arc::new(Int32Array::from(vec![10, 20]))]; + let batch = make_struct_batch("s", child_fields, child_arrays, 2); + + let mut row = ColumnarRow::new(batch); + + // row_id = 0: nested x = 10 + let nested_0 = row.get_row(0).unwrap(); + assert_eq!(nested_0.get_int(0).unwrap(), 10); + + // After set_row_id(1), cache is cleared → nested x = 20 + row.set_row_id(1); + let nested_1 = row.get_row(0).unwrap(); + assert_eq!(nested_1.get_int(0).unwrap(), 20); + } } diff --git a/crates/fluss/src/row/compacted/compacted_row.rs b/crates/fluss/src/row/compacted/compacted_row.rs index 267ae13c..24e4bd79 100644 --- a/crates/fluss/src/row/compacted/compacted_row.rs +++ b/crates/fluss/src/row/compacted/compacted_row.rs @@ -171,6 +171,10 @@ impl<'a> InternalRow for CompactedRow<'a> { self.decoded_row()?.get_array(pos) } + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + self.decoded_row().get_row(pos) + } + fn as_encoded_bytes(&self, write_format: WriteFormat) -> Option<&[u8]> { match write_format { WriteFormat::CompactedKv => Some(self.as_bytes()), diff --git a/crates/fluss/src/row/compacted/compacted_row_reader.rs b/crates/fluss/src/row/compacted/compacted_row_reader.rs index 4ae442fc..9d7fb6cb 100644 --- a/crates/fluss/src/row/compacted/compacted_row_reader.rs +++ b/crates/fluss/src/row/compacted/compacted_row_reader.rs @@ -187,6 +187,18 @@ impl<'a> CompactedRowDeserializer<'a> { let array = crate::row::binary_array::FlussArray::from_bytes(bytes)?; (Datum::Array(array), next) } + DataType::Row(row_type) => { + let (nested_bytes, next) = reader.read_bytes(cursor)?; + let nested_reader = CompactedRowReader::new( + row_type.fields().len(), + nested_bytes, + 0, + nested_bytes.len(), + ); + let nested_deser = CompactedRowDeserializer::new_from_owned(row_type.clone()); + let nested_row = nested_deser.deserialize(&nested_reader)?; + (Datum::Row(Box::new(nested_row)), next) + } _ => { return Err(IllegalArgument { message: format!( @@ -326,3 +338,142 @@ impl<'a> CompactedRowReader<'a> { Ok((s, next_pos)) } } + +#[cfg(test)] +mod row_type_tests { + use crate::metadata::{DataType, DataTypes, RowType}; + use crate::row::compacted::compacted_row_reader::{CompactedRowDeserializer, CompactedRowReader}; + use crate::row::compacted::compacted_row_writer::CompactedRowWriter; + use crate::row::binary::ValueWriter; + use crate::row::field_getter::FieldGetter; + use crate::row::{Datum, GenericRow, InternalRow}; + + fn round_trip(outer_row_type: &RowType, outer_row: &GenericRow, verify: F) + where + F: FnOnce(&GenericRow), + { + // Write + let field_getters = FieldGetter::create_field_getters(outer_row_type); + let value_writers: Vec = outer_row_type + .fields() + .iter() + .map(|f| ValueWriter::create_value_writer(f.data_type(), None).unwrap()) + .collect(); + let mut writer = CompactedRowWriter::new(outer_row_type.fields().len()); + for (i, (getter, vw)) in field_getters.iter().zip(value_writers.iter()).enumerate() { + let datum = getter.get_field(outer_row as &dyn InternalRow).unwrap(); + vw.write_value(&mut writer, i, &datum).unwrap(); + } + let bytes = writer.to_bytes(); + + // Read + let deser = CompactedRowDeserializer::new(outer_row_type); + let reader = CompactedRowReader::new( + outer_row_type.fields().len(), + bytes.as_ref(), + 0, + bytes.len(), + ); + let result = deser.deserialize(&reader); + verify(&result); + } + + #[test] + fn test_row_simple_nesting() { + // ROW nested inside an outer row + let inner_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int(), DataTypes::string()], + vec!["x", "label"], + ); + let outer_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int(), DataType::Row(inner_row_type.clone())], + vec!["id", "nested"], + ); + + let mut inner = GenericRow::new(2); + inner.set_field(0, 42_i32); + inner.set_field(1, "hello"); + + let mut outer = GenericRow::new(2); + outer.set_field(0, 1_i32); + outer.set_field(1, Datum::Row(Box::new(inner))); + + round_trip(&outer_row_type, &outer, |result| { + assert_eq!(result.get_int(0).unwrap(), 1); + let nested = result.get_row(1).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 42); + assert_eq!(nested.get_string(1).unwrap(), "hello"); + }); + } + + #[test] + fn test_row_deep_nesting() { + // ROW> — two levels of nesting + let inner_inner_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int()], + vec!["n"], + ); + let inner_row_type = RowType::with_data_types_and_field_names( + vec![DataType::Row(inner_inner_row_type.clone())], + vec!["inner"], + ); + let outer_row_type = RowType::with_data_types_and_field_names( + vec![DataType::Row(inner_row_type.clone())], + vec!["outer"], + ); + + let mut innermost = GenericRow::new(1); + innermost.set_field(0, 99_i32); + + let mut middle = GenericRow::new(1); + middle.set_field(0, Datum::Row(Box::new(innermost))); + + let mut outer = GenericRow::new(1); + outer.set_field(0, Datum::Row(Box::new(middle))); + + round_trip(&outer_row_type, &outer, |result| { + let mid = result.get_row(0).unwrap(); + let inner = mid.get_row(0).unwrap(); + assert_eq!(inner.get_int(0).unwrap(), 99); + }); + } + + #[test] + fn test_row_with_nullable_fields() { + // Outer nullable ROW column; nested row with a nullable STRING field set to null + let inner_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int(), DataTypes::string()], + vec!["id", "optional_name"], + ); + let outer_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int(), DataType::Row(inner_row_type.clone())], + vec!["k", "nested"], + ); + + // Case 1: non-null nested row with a null field inside + let mut inner = GenericRow::new(2); + inner.set_field(0, 7_i32); + inner.set_field(1, Datum::Null); + + let mut outer = GenericRow::new(2); + outer.set_field(0, 10_i32); + outer.set_field(1, Datum::Row(Box::new(inner))); + + round_trip(&outer_row_type, &outer, |result| { + assert_eq!(result.get_int(0).unwrap(), 10); + let nested = result.get_row(1).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 7); + assert!(nested.is_null_at(1).unwrap()); + }); + + // Case 2: outer ROW column is null + let mut outer_null = GenericRow::new(2); + outer_null.set_field(0, 20_i32); + outer_null.set_field(1, Datum::Null); + + round_trip(&outer_row_type, &outer_null, |result2| { + assert_eq!(result2.get_int(0).unwrap(), 20); + assert!(result2.is_null_at(1).unwrap()); + }); + } +} diff --git a/crates/fluss/src/row/datum.rs b/crates/fluss/src/row/datum.rs index 78dc5498..865d4b4f 100644 --- a/crates/fluss/src/row/datum.rs +++ b/crates/fluss/src/row/datum.rs @@ -18,6 +18,7 @@ use crate::error::Error::RowConvertError; use crate::error::Result; use crate::row::Decimal; +use crate::row::GenericRow; use crate::row::binary_array::FlussArray; use arrow::array::{ ArrayBuilder, BinaryBuilder, BooleanBuilder, Date32Builder, Decimal128Builder, @@ -72,6 +73,8 @@ pub enum Datum<'a> { TimestampLtz(TimestampLtz), #[display("{0}")] Array(FlussArray), + #[display("{0:?}")] + Row(Box>), } impl Datum<'_> { @@ -134,6 +137,13 @@ impl Datum<'_> { _ => panic!("not an array: {self:?}"), } } + + pub fn as_row(&self) -> &GenericRow<'_> { + match self { + Self::Row(r) => r.as_ref(), + _ => panic!("not a row: {self:?}"), + } + } } // ----------- implement from @@ -878,6 +888,11 @@ impl Datum<'_> { Datum::Array(arr) => { return append_fluss_array_to_list_builder(arr, builder, data_type); } + Datum::Row(_) => { + return Err(RowConvertError { + message: "append_to is not supported for Row type".to_string(), + }); + } } Err(RowConvertError { diff --git a/crates/fluss/src/row/encode/compacted_key_encoder.rs b/crates/fluss/src/row/encode/compacted_key_encoder.rs index 877b3ecd..a6ba580f 100644 --- a/crates/fluss/src/row/encode/compacted_key_encoder.rs +++ b/crates/fluss/src/row/encode/compacted_key_encoder.rs @@ -109,7 +109,7 @@ impl KeyEncoder for CompactedKeyEncoder { #[cfg(test)] mod tests { use super::*; - use crate::metadata::DataTypes; + use crate::metadata::{DataType, DataTypes}; use crate::row::binary_array::FlussArrayWriter; use crate::row::{Datum, GenericRow}; @@ -493,4 +493,50 @@ mod tests { encoded.iter().as_slice() ); } + + #[test] + fn test_row_as_primary_key() { + // ROW as a primary key column + let inner_row_type = RowType::with_data_types_and_field_names( + vec![DataTypes::int(), DataTypes::string()], + vec!["x", "label"], + ); + let row_type = RowType::with_data_types_and_field_names( + vec![ + DataTypes::int(), + DataType::Row(inner_row_type.clone()), + ], + vec!["id", "nested"], + ); + + let mut inner = GenericRow::new(2); + inner.set_field(0, 42_i32); + inner.set_field(1, "hello"); + + let mut row = GenericRow::new(2); + row.set_field(0, 1_i32); + row.set_field(1, Datum::Row(Box::new(inner))); + + let mut encoder = for_test_row_type(&row_type); + let encoded = encoder.encode_key(&row).unwrap(); + + // Verify it encodes without error and produces non-empty bytes + assert!(!encoded.is_empty()); + + // Encode the same row again to verify determinism + let encoded2 = encoder.encode_key(&row).unwrap(); + assert_eq!(encoded, encoded2); + + // Encode a different nested row and verify different output + let mut inner2 = GenericRow::new(2); + inner2.set_field(0, 99_i32); + inner2.set_field(1, "world"); + + let mut row2 = GenericRow::new(2); + row2.set_field(0, 1_i32); + row2.set_field(1, Datum::Row(Box::new(inner2))); + + let encoded3 = encoder.encode_key(&row2).unwrap(); + assert_ne!(encoded, encoded3); + } } diff --git a/crates/fluss/src/row/field_getter.rs b/crates/fluss/src/row/field_getter.rs index 69e08602..a1ea378f 100644 --- a/crates/fluss/src/row/field_getter.rs +++ b/crates/fluss/src/row/field_getter.rs @@ -82,8 +82,9 @@ impl FieldGetter { pos, precision: t.precision(), }, - // TODO: add Map and Row variants when get_map/get_row are available in InternalRow. + // TODO: add Map variant when get_map is available in InternalRow. DataType::Array(_) => InnerFieldGetter::Array { pos }, + DataType::Row(_) => InnerFieldGetter::Row { pos }, _ => unimplemented!("DataType {:?} is currently unimplemented", data_type), }; @@ -154,6 +155,9 @@ pub enum InnerFieldGetter { Array { pos: usize, }, + Row { + pos: usize, + }, } impl InnerFieldGetter { @@ -183,8 +187,9 @@ impl InnerFieldGetter { InnerFieldGetter::TimestampLtz { pos, precision } => { Datum::TimestampLtz(row.get_timestamp_ltz(*pos, *precision)?) } - // TODO: add Map and Row field getter support once their binary forms are implemented. + // TODO: add Map field getter support once its binary form is implemented. InnerFieldGetter::Array { pos } => Datum::Array(row.get_array(*pos)?), + InnerFieldGetter::Row { pos } => Datum::Row(Box::new(row.get_row(*pos)?.clone())), }) } @@ -206,7 +211,8 @@ impl InnerFieldGetter { | Self::Time { pos } | Self::Timestamp { pos, .. } | Self::TimestampLtz { pos, .. } - | Self::Array { pos } => *pos, + | Self::Array { pos } + | Self::Row { pos } => *pos, } } } diff --git a/crates/fluss/src/row/mod.rs b/crates/fluss/src/row/mod.rs index c6bdd7c5..fb599fc9 100644 --- a/crates/fluss/src/row/mod.rs +++ b/crates/fluss/src/row/mod.rs @@ -34,6 +34,7 @@ mod row_decoder; use crate::client::WriteFormat; pub use binary_array::FlussArray; use bytes::Bytes; +use serde::Serialize; pub use column::*; pub use compacted::CompactedRow; pub use datum::*; @@ -130,13 +131,20 @@ pub trait InternalRow: Send + Sync { /// Returns the array value at the given position fn get_array(&self, pos: usize) -> Result; + /// Returns the nested row value at the given position + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + Err(crate::error::Error::IllegalArgument { + message: format!("get_row not supported at position {pos}"), + }) + } + /// Returns encoded bytes if already encoded fn as_encoded_bytes(&self, _write_format: WriteFormat) -> Option<&[u8]> { None } } -#[derive(Debug)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize)] pub struct GenericRow<'a> { pub values: Vec>, } @@ -294,6 +302,15 @@ impl<'a> InternalRow for GenericRow<'a> { }), } } + + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + match self.get_value(pos)? { + Datum::Row(r) => Ok(r.as_ref()), + other => Err(IllegalArgument { + message: format!("type mismatch at position {pos}: expected Row, got {other:?}"), + }), + } + } } impl<'a> GenericRow<'a> { From fb693b6baab503eb10d035b25437880fc32d4693 Mon Sep 17 00:00:00 2001 From: Anton Borisov Date: Sun, 10 May 2026 19:14:32 +0100 Subject: [PATCH 2/2] fixes and improvements --- bindings/cpp/src/types.rs | 2 +- bindings/python/src/table.rs | 5 + crates/fluss/src/client/table/append.rs | 2 +- crates/fluss/src/client/table/scanner.rs | 58 +- crates/fluss/src/metadata/datatype.rs | 72 ++- crates/fluss/src/metadata/json_serde.rs | 342 ++++++++-- crates/fluss/src/metadata/table.rs | 148 ++++- crates/fluss/src/record/arrow.rs | 72 ++- crates/fluss/src/record/mod.rs | 2 +- crates/fluss/src/row/binary/binary_writer.rs | 74 ++- crates/fluss/src/row/binary_array.rs | 315 ++++++++- crates/fluss/src/row/column.rs | 544 +++++++++------- crates/fluss/src/row/column_writer.rs | 139 +++- .../src/row/compacted/compacted_key_writer.rs | 14 +- .../fluss/src/row/compacted/compacted_row.rs | 44 +- .../src/row/compacted/compacted_row_reader.rs | 192 ++++-- crates/fluss/src/row/compacted/mod.rs | 2 +- crates/fluss/src/row/datum.rs | 233 +++++-- .../src/row/encode/compacted_key_encoder.rs | 54 +- crates/fluss/src/row/lookup_row.rs | 5 +- crates/fluss/src/row/mod.rs | 15 +- crates/fluss/src/row/projected_row.rs | 6 +- crates/fluss/tests/integration/kv_table.rs | 606 ++++++++++++++++-- crates/fluss/tests/integration/log_table.rs | 489 +++++++++++++- 24 files changed, 2847 insertions(+), 588 deletions(-) diff --git a/bindings/cpp/src/types.rs b/bindings/cpp/src/types.rs index 19aa9a36..84aec25b 100644 --- a/bindings/cpp/src/types.rs +++ b/bindings/cpp/src/types.rs @@ -529,7 +529,7 @@ pub fn resolve_row_types( Datum::TimestampNtz(ts) => Datum::TimestampNtz(*ts), Datum::TimestampLtz(ts) => Datum::TimestampLtz(*ts), Datum::Array(a) => Datum::Array(a.clone()), - Datum::Row(r) => Datum::Row(Box::new(resolve_row_types(r, None)?)), + Datum::Row(_) => return Err(anyhow!("Row datum is not yet supported in C++ bindings")), }; out.set_field(idx, resolved); } diff --git a/bindings/python/src/table.rs b/bindings/python/src/table.rs index 98aee5e3..1800ea43 100644 --- a/bindings/python/src/table.rs +++ b/bindings/python/src/table.rs @@ -1322,6 +1322,11 @@ fn python_value_to_datum( } } Datum::Array(v) => writer.write_array(i, &v), + Datum::Row(_) => { + return Err(FlussError::new_err( + "Row datum is not supported as an array element", + )); + } } } } diff --git a/crates/fluss/src/client/table/append.rs b/crates/fluss/src/client/table/append.rs index a58433f3..53552008 100644 --- a/crates/fluss/src/client/table/append.rs +++ b/crates/fluss/src/client/table/append.rs @@ -126,7 +126,7 @@ impl AppendWriter { /// or dropped for fire-and-forget behavior (use `flush()` to ensure delivery). pub fn append_arrow_batch(&self, batch: RecordBatch) -> Result { let physical_table_path = if self.partition_getter.is_some() && batch.num_rows() > 0 { - let first_row = ColumnarRow::new(Arc::new(batch.clone())); + let first_row = ColumnarRow::new(Arc::new(batch.clone()), 0, None); Arc::new(get_physical_path( &self.table_path, self.partition_getter.as_ref(), diff --git a/crates/fluss/src/client/table/scanner.rs b/crates/fluss/src/client/table/scanner.rs index 00c5b238..c6228e59 100644 --- a/crates/fluss/src/client/table/scanner.rs +++ b/crates/fluss/src/client/table/scanner.rs @@ -23,10 +23,13 @@ use crate::client::table::log_fetch_buffer::{ LogFetchBuffer, RemotePendingFetch, }; use crate::client::table::remote_log::{RemoteLogDownloader, RemoteLogFetchInfo}; +use crate::config::Config; use crate::error::Error::UnsupportedOperation; use crate::error::{ApiError, Error, FlussError, Result}; -use crate::metadata::{LogFormat, PhysicalTablePath, TableBucket, TableInfo, TablePath}; -use crate::proto::{ErrorResponse, FetchLogRequest, PbFetchLogReqForBucket, PbFetchLogReqForTable}; +use crate::metadata::{LogFormat, PhysicalTablePath, RowType, TableBucket, TableInfo, TablePath}; +use crate::proto::{ + ErrorResponse, FetchLogRequest, FetchLogResponse, PbFetchLogReqForBucket, PbFetchLogReqForTable, +}; use crate::record::{ LogRecordsBatches, ReadContext, ScanBatch, ScanRecord, ScanRecords, to_arrow_schema, }; @@ -273,7 +276,7 @@ impl LogScannerInner { table_info: &TableInfo, metadata: Arc, connections: Arc, - config: &crate::config::Config, + config: &Config, projected_fields: Option>, ) -> Result { let log_scanner_status = Arc::new(LogScannerStatus::new()); @@ -651,14 +654,26 @@ impl LogFetcher { conns: Arc, metadata: Arc, log_scanner_status: Arc, - config: &crate::config::Config, + config: &Config, projected_fields: Option>, ) -> Result { - let full_arrow_schema = to_arrow_schema(table_info.get_row_type())?; + let full_row_type = table_info.get_row_type(); + let full_arrow_schema = to_arrow_schema(full_row_type)?; + let projected_row_type = match &projected_fields { + None => Arc::new(full_row_type.clone()), + Some(fields) => Arc::new(RowType::new( + fields + .iter() + .map(|&i| full_row_type.fields()[i].clone()) + .collect(), + )), + }; let read_context = - Self::create_read_context(full_arrow_schema.clone(), projected_fields.clone(), false)?; + Self::create_read_context(full_arrow_schema.clone(), projected_fields.clone(), false)? + .with_fluss_row_type(projected_row_type.clone()); let remote_read_context = - Self::create_read_context(full_arrow_schema, projected_fields.clone(), true)?; + Self::create_read_context(full_arrow_schema, projected_fields.clone(), true)? + .with_fluss_row_type(projected_row_type); let tmp_dir = TempDir::with_prefix("fluss-remote-logs")?; let log_fetch_buffer = Arc::new(LogFetchBuffer::new(read_context.clone())); @@ -928,7 +943,7 @@ impl LogFetcher { /// Handle fetch response and add completed fetches to buffer async fn handle_fetch_response( - fetch_response: crate::proto::FetchLogResponse, + fetch_response: FetchLogResponse, context: FetchResponseContext, ) { let FetchResponseContext { @@ -1704,6 +1719,7 @@ mod tests { DEFAULT_NON_ZSTD_COMPRESSION_LEVEL, }; use crate::metadata::{DataTypes, PhysicalTablePath, Schema, TableInfo, TablePath}; + use crate::proto::{PbFetchLogRespForBucket, PbFetchLogRespForTable}; use crate::record::MemoryLogRecordsArrowBuilder; use crate::row::{Datum, GenericRow}; use crate::rpc::FlussError; @@ -1743,7 +1759,7 @@ mod tests { Arc::new(RpcClient::new()), metadata, status.clone(), - &crate::config::Config::default(), + &Config::default(), None, )?; @@ -1775,7 +1791,7 @@ mod tests { Arc::new(RpcClient::new()), metadata, status, - &crate::config::Config::default(), + &Config::default(), None, )?; @@ -1811,7 +1827,7 @@ mod tests { Arc::new(RpcClient::new()), metadata, status, - &crate::config::Config::default(), + &Config::default(), None, )?; @@ -1835,14 +1851,14 @@ mod tests { Arc::new(RpcClient::new()), metadata.clone(), status.clone(), - &crate::config::Config::default(), + &Config::default(), None, )?; - let response = crate::proto::FetchLogResponse { - tables_resp: vec![crate::proto::PbFetchLogRespForTable { + let response = FetchLogResponse { + tables_resp: vec![PbFetchLogRespForTable { table_id: 1, - buckets_resp: vec![crate::proto::PbFetchLogRespForBucket { + buckets_resp: vec![PbFetchLogRespForBucket { partition_id: None, bucket_id: 0, error_code: Some(FlussError::AuthorizationException.code()), @@ -1885,17 +1901,17 @@ mod tests { Arc::new(RpcClient::new()), metadata.clone(), status.clone(), - &crate::config::Config::default(), + &Config::default(), None, )?; let bucket = TableBucket::new(1, 0); assert!(metadata.leader_for(&table_path, &bucket).await?.is_some()); - let response = crate::proto::FetchLogResponse { - tables_resp: vec![crate::proto::PbFetchLogRespForTable { + let response = FetchLogResponse { + tables_resp: vec![PbFetchLogRespForTable { table_id: 1, - buckets_resp: vec![crate::proto::PbFetchLogRespForBucket { + buckets_resp: vec![PbFetchLogRespForBucket { partition_id: None, bucket_id: 0, error_code: Some(FlussError::NotLeaderOrFollower.code()), @@ -2002,12 +2018,12 @@ mod tests { let status = Arc::new(LogScannerStatus::new()); status.assign_scan_bucket(TableBucket::new(1, 0), 0); - let config = crate::config::Config { + let config = Config { scanner_log_fetch_max_bytes: 1234, scanner_log_fetch_min_bytes: 7, scanner_log_fetch_wait_max_time_ms: 89, scanner_log_fetch_max_bytes_for_bucket: 512, - ..crate::config::Config::default() + ..Config::default() }; let fetcher = LogFetcher::new( diff --git a/crates/fluss/src/metadata/datatype.rs b/crates/fluss/src/metadata/datatype.rs index 4103aa3d..ffc48a8b 100644 --- a/crates/fluss/src/metadata/datatype.rs +++ b/crates/fluss/src/metadata/datatype.rs @@ -531,7 +531,6 @@ impl DecimalType { }); } // Validate scale - // Note: MIN_SCALE is 0, and scale is u32, so scale >= MIN_SCALE is always true if scale > precision { return Err(IllegalArgument { message: format!( @@ -1220,11 +1219,61 @@ impl DataTypes { } } -#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] +pub const UNASSIGNED_FIELD_ID: i32 = -1; + +pub fn reassign_field_ids(data_type: &DataType, counter: &mut i32) -> DataType { + match data_type { + DataType::Array(at) => DataType::Array(ArrayType::with_nullable( + at.nullable, + reassign_field_ids(at.get_element_type(), counter), + )), + DataType::Map(mt) => DataType::Map(MapType::with_nullable( + mt.nullable, + reassign_field_ids(mt.key_type(), counter), + reassign_field_ids(mt.value_type(), counter), + )), + DataType::Row(rt) => { + let new_fields: Vec = rt + .fields() + .iter() + .map(|f| { + *counter += 1; + let id = *counter; + let new_inner = reassign_field_ids(&f.data_type, counter); + DataField::with_field_id(f.name.clone(), new_inner, f.description.clone(), id) + }) + .collect(); + DataType::Row(RowType::with_nullable(rt.nullable, new_fields)) + } + _ => data_type.clone(), + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] pub struct DataField { pub name: String, pub data_type: DataType, pub description: Option, + pub field_id: i32, +} + +// field_id is excluded from PartialEq/Eq/Hash to match Java's DataField.equals/hashCode. +impl PartialEq for DataField { + fn eq(&self, other: &Self) -> bool { + self.name == other.name + && self.data_type == other.data_type + && self.description == other.description + } +} + +impl Eq for DataField {} + +impl std::hash::Hash for DataField { + fn hash(&self, state: &mut H) { + self.name.hash(state); + self.data_type.hash(state); + self.description.hash(state); + } } impl DataField { @@ -1237,6 +1286,21 @@ impl DataField { name: name.into(), data_type, description, + field_id: UNASSIGNED_FIELD_ID, + } + } + + pub fn with_field_id>( + name: N, + data_type: DataType, + description: Option, + field_id: i32, + ) -> DataField { + DataField { + name: name.into(), + data_type, + description, + field_id, } } @@ -1247,6 +1311,10 @@ impl DataField { pub fn data_type(&self) -> &DataType { &self.data_type } + + pub fn field_id(&self) -> i32 { + self.field_id + } } impl Display for DataField { diff --git a/crates/fluss/src/metadata/json_serde.rs b/crates/fluss/src/metadata/json_serde.rs index efbce84d..b08159ae 100644 --- a/crates/fluss/src/metadata/json_serde.rs +++ b/crates/fluss/src/metadata/json_serde.rs @@ -17,7 +17,10 @@ use crate::error::Error::JsonSerdeError; use crate::error::{Error, Result}; -use crate::metadata::datatype::{DataField, DataType, DataTypes}; +use crate::metadata::datatype::{ + DataField, DataType, DataTypes, DecimalType, TimeType, TimestampLTzType, TimestampType, + UNASSIGNED_FIELD_ID, +}; use crate::metadata::table::{Column, Schema, TableDescriptor}; use serde_json::{Value, json}; use std::collections::HashMap; @@ -202,12 +205,11 @@ impl JsonSerde for DataType { .get(Self::FIELD_NAME_SCALE) .and_then(|v| v.as_u64()) .unwrap_or(0) as u32; - DataType::Decimal( - crate::metadata::datatype::DecimalType::with_nullable(true, precision, scale) - .map_err(|e| Error::JsonSerdeError { + DataType::Decimal(DecimalType::with_nullable(true, precision, scale).map_err( + |e| Error::JsonSerdeError { message: format!("Invalid DECIMAL parameters: {e}"), - })?, - ) + }, + )?) } "DATE" => DataTypes::date(), "TIME_WITHOUT_TIME_ZONE" => { @@ -215,39 +217,33 @@ impl JsonSerde for DataType { .get(Self::FIELD_NAME_PRECISION) .and_then(|v| v.as_u64()) .unwrap_or(0) as u32; - DataType::Time( - crate::metadata::datatype::TimeType::with_nullable(true, precision).map_err( - |e| Error::JsonSerdeError { - message: format!("Invalid TIME_WITHOUT_TIME_ZONE precision: {e}"), - }, - )?, - ) + DataType::Time(TimeType::with_nullable(true, precision).map_err(|e| { + Error::JsonSerdeError { + message: format!("Invalid TIME_WITHOUT_TIME_ZONE precision: {e}"), + } + })?) } "TIMESTAMP_WITHOUT_TIME_ZONE" => { let precision = node .get(Self::FIELD_NAME_PRECISION) .and_then(|v| v.as_u64()) .unwrap_or(6) as u32; - DataType::Timestamp( - crate::metadata::datatype::TimestampType::with_nullable(true, precision) - .map_err(|e| Error::JsonSerdeError { - message: format!("Invalid TIMESTAMP_WITHOUT_TIME_ZONE precision: {e}"), - })?, - ) + DataType::Timestamp(TimestampType::with_nullable(true, precision).map_err(|e| { + Error::JsonSerdeError { + message: format!("Invalid TIMESTAMP_WITHOUT_TIME_ZONE precision: {e}"), + } + })?) } "TIMESTAMP_WITH_LOCAL_TIME_ZONE" => { let precision = node .get(Self::FIELD_NAME_PRECISION) .and_then(|v| v.as_u64()) .unwrap_or(6) as u32; - DataType::TimestampLTz( - crate::metadata::datatype::TimestampLTzType::with_nullable(true, precision) - .map_err(|e| Error::JsonSerdeError { - message: format!( - "Invalid TIMESTAMP_WITH_LOCAL_TIME_ZONE precision: {e}" - ), - })?, - ) + DataType::TimestampLTz(TimestampLTzType::with_nullable(true, precision).map_err( + |e| Error::JsonSerdeError { + message: format!("Invalid TIMESTAMP_WITH_LOCAL_TIME_ZONE precision: {e}"), + }, + )?) } "BYTES" => DataTypes::bytes(), "BINARY" => { @@ -328,6 +324,7 @@ impl DataField { const NAME: &'static str = "name"; const FIELD_TYPE: &'static str = "field_type"; const DESCRIPTION: &'static str = "description"; + const FIELD_ID: &'static str = "field_id"; } impl JsonSerde for DataField { @@ -344,6 +341,8 @@ impl JsonSerde for DataField { obj.insert(Self::DESCRIPTION.to_string(), json!(description)); } + obj.insert(Self::FIELD_ID.to_string(), json!(self.field_id())); + Ok(Value::Object(obj)) } @@ -369,7 +368,18 @@ impl JsonSerde for DataField { .and_then(|v| v.as_str()) .map(|s| s.to_string()); - Ok(DataField::new(name, data_type, description)) + let field_id = node + .get(Self::FIELD_ID) + .and_then(|v| v.as_i64()) + .map(|v| v as i32) + .unwrap_or(UNASSIGNED_FIELD_ID); + + Ok(DataField::with_field_id( + name, + data_type, + description, + field_id, + )) } } @@ -439,6 +449,7 @@ impl JsonSerde for Column { impl Schema { const COLUMNS_NAME: &'static str = "columns"; const PRIMARY_KEY_NAME: &'static str = "primary_key"; + const HIGHEST_FIELD_ID: &'static str = "highest_field_id"; const VERSION_KEY: &'static str = "version"; const VERSION: u32 = 1; } @@ -467,6 +478,12 @@ impl JsonSerde for Schema { .collect(); obj.insert(Self::PRIMARY_KEY_NAME.to_string(), json!(pk_values)); } + + obj.insert( + Self::HIGHEST_FIELD_ID.to_string(), + json!(self.highest_field_id()), + ); + Ok(Value::Object(obj)) } @@ -682,12 +699,13 @@ impl JsonSerde for TableDescriptor { #[cfg(test)] mod tests { use super::*; - use crate::metadata::DataTypes; + use crate::metadata::reassign_field_ids; + use crate::metadata::{ + Column, DataField, DataType, DataTypes as DT, DataTypes, MapType, Schema, + }; #[test] fn column_id_round_trip_through_json() { - use crate::metadata::Column; - let col = Column::new("a", DataTypes::int()) .with_id(7) .with_comment("desc"); @@ -699,8 +717,6 @@ mod tests { #[test] fn schema_assigns_ids_when_absent_and_preserves_when_present() { - use crate::metadata::{Column, Schema}; - let auto = Schema::builder() .column("a", DataTypes::int()) .column("b", DataTypes::string()) @@ -722,7 +738,6 @@ mod tests { #[test] fn schema_rejects_duplicate_ids() { - use crate::metadata::Column; let err = Schema::builder() .with_columns(vec![ Column::new("a", DataTypes::int()).with_id(7), @@ -730,12 +745,11 @@ mod tests { ]) .build() .unwrap_err(); - assert!(err.to_string().contains("Duplicate column id 7"), "{err}"); + assert!(err.to_string().contains("Duplicate field id 7"), "{err}"); } #[test] fn schema_rejects_negative_non_sentinel_ids() { - use crate::metadata::Column; let err = Schema::builder() .with_columns(vec![Column::new("a", DataTypes::int()).with_id(-7)]) .build() @@ -745,7 +759,6 @@ mod tests { #[test] fn column_json_id_overflow_errors() { - use crate::metadata::Column; let json = serde_json::json!({ "name": "a", "data_type": Column::new("a", DataTypes::int()).serialize_json().unwrap() @@ -758,8 +771,6 @@ mod tests { #[test] fn schema_rejects_partially_assigned_ids() { - use crate::metadata::Column; - let err = Schema::builder() .with_columns(vec![ Column::new("a", DataTypes::int()).with_id(0), @@ -773,6 +784,259 @@ mod tests { ); } + #[test] + fn schema_assigns_nested_field_ids_in_java_dfs_order() { + let inner_row = DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let nested_row = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let deep_row = DataTypes::row(vec![DataField::new("inner", inner_row, None)]); + + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("nested", nested_row) + .column("deep", deep_row) + .build() + .unwrap(); + + let top_ids: Vec = schema.columns().iter().map(|c| c.id()).collect(); + assert_eq!(top_ids, vec![0, 1, 4]); + + fn nested_field(dt: &DataType, idx: usize) -> &DataField { + match dt { + DataType::Row(rt) => &rt.fields()[idx], + _ => panic!("not a Row"), + } + } + let nested_dt = schema.columns()[1].data_type(); + assert_eq!(nested_field(nested_dt, 0).field_id(), 2); // x + assert_eq!(nested_field(nested_dt, 1).field_id(), 3); // label + + let deep_dt = schema.columns()[2].data_type(); + let inner_field = nested_field(deep_dt, 0); // inner + assert_eq!(inner_field.field_id(), 5); + let n_field = nested_field(inner_field.data_type(), 0); // n + assert_eq!(n_field.field_id(), 6); + + assert_eq!(schema.highest_field_id(), 6); + + for c in schema.columns() { + assert_ne!(c.id(), UNASSIGNED_FIELD_ID); + } + } + + #[test] + fn schema_array_of_row_assigns_nested_ids() { + let elem = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("events", DataTypes::array(elem)) + .build() + .unwrap(); + assert_eq!(schema.highest_field_id(), 3); + let array_dt = schema.columns()[1].data_type(); + let elem_dt = match array_dt { + DataType::Array(at) => at.get_element_type(), + _ => unreachable!(), + }; + let fields = match elem_dt { + DataType::Row(rt) => rt.fields(), + _ => unreachable!(), + }; + assert_eq!(fields[0].field_id(), 2); + assert_eq!(fields[1].field_id(), 3); + } + + #[test] + fn schema_nested_row_round_trips_through_json() { + let nested = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let original = Schema::builder() + .column("id", DataTypes::int()) + .column("nested", nested) + .build() + .unwrap(); + + let json = original.serialize_json().unwrap(); + + assert_eq!( + json.get("highest_field_id").and_then(|v| v.as_i64()), + Some(3) + ); + + let round_tripped = Schema::deserialize_json(&json).unwrap(); + assert_eq!(round_tripped.highest_field_id(), 3); + assert_eq!( + round_tripped + .columns() + .iter() + .map(|c| c.id()) + .collect::>(), + vec![0, 1], + ); + assert_eq!(round_tripped, original); + } + + #[test] + fn schema_rejects_duplicate_nested_field_ids() { + let nested = DataTypes::row(vec![ + DataField::with_field_id("x", DT::int(), None, 0), + DataField::with_field_id("y", DT::int(), None, 2), + ]); + let err = Schema::builder() + .with_columns(vec![ + Column::new("a", DT::int()).with_id(0), + Column::new("b", nested).with_id(1), + ]) + .build() + .unwrap_err(); + assert!(err.to_string().contains("Duplicate field id 0"), "{err}"); + } + + #[test] + fn schema_rejects_partially_assigned_nested_field_ids() { + let nested = DataTypes::row(vec![DataField::new("x", DT::int(), None)]); + let err = Schema::builder() + .with_columns(vec![ + Column::new("a", DT::int()).with_id(0), + Column::new("b", nested).with_id(1), + ]) + .build() + .unwrap_err(); + assert!( + err.to_string() + .contains("nested DataField ids are unassigned"), + "{err}" + ); + } + + #[test] + fn schema_preserves_nested_ids_with_gaps() { + // n2.m1=11), f2=2 (nested n0=9, n1=10). + let inner_for_n2 = DataTypes::row(vec![DataField::with_field_id( + "m1", + DataTypes::tinyint(), + None, + 11, + )]); + let f1_row = DataTypes::row(vec![ + DataField::with_field_id("n0", DataTypes::tinyint(), None, 6), + DataField::with_field_id("n1", DataTypes::string(), None, 7), + DataField::with_field_id("n2", inner_for_n2, None, 8), + ]); + let f2_row = DataTypes::row(vec![ + DataField::with_field_id("n0", DataTypes::tinyint(), None, 9), + DataField::with_field_id("n1", DataTypes::string(), None, 10), + ]); + + let schema = Schema::builder() + .with_columns(vec![ + Column::new("f0", DataTypes::string().as_non_nullable()).with_id(0), + Column::new("f1", f1_row).with_id(1), + Column::new("f2", f2_row).with_id(2), + ]) + .build() + .unwrap(); + + let top_ids: Vec = schema.columns().iter().map(|c| c.id()).collect(); + assert_eq!(top_ids, vec![0, 1, 2]); + + fn row_fields(dt: &DataType) -> &[DataField] { + match dt { + DataType::Row(rt) => rt.fields(), + _ => panic!("not a Row"), + } + } + let f1_fields = row_fields(schema.columns()[1].data_type()); + assert_eq!(f1_fields[0].field_id(), 6); // n0 + assert_eq!(f1_fields[1].field_id(), 7); // n1 + assert_eq!(f1_fields[2].field_id(), 8); // n2 + let n2_fields = row_fields(f1_fields[2].data_type()); + assert_eq!(n2_fields[0].field_id(), 11); // m1 — the "gap" + + let f2_fields = row_fields(schema.columns()[2].data_type()); + assert_eq!(f2_fields[0].field_id(), 9); + assert_eq!(f2_fields[1].field_id(), 10); + + assert_eq!(schema.highest_field_id(), 11); + } + + #[test] + fn schema_deserializes_legacy_json_without_column_ids() { + let legacy_json: Value = serde_json::from_str( + r#"{ + "version": 1, + "columns": [ + {"name": "a", "data_type": {"type": "INTEGER", "nullable": false}, "comment": "first"}, + {"name": "b", "data_type": {"type": "STRING"}, "comment": "second"}, + {"name": "c", "data_type": {"type": "CHAR", "nullable": false, "length": 10}, "comment": "third"} + ], + "primary_key": ["a", "c"] + }"#, + ) + .unwrap(); + + let schema = Schema::deserialize_json(&legacy_json).expect("legacy JSON must deserialize"); + let ids: Vec = schema.columns().iter().map(|c| c.id()).collect(); + assert_eq!(ids, vec![0, 1, 2], "missing IDs auto-assigned 0..N-1"); + assert_eq!(schema.highest_field_id(), 2); + assert!(schema.primary_key().is_some()); + } + + #[test] + fn empty_schema_has_minus_one_highest_field_id() { + let s = Schema::builder().build().unwrap(); + assert_eq!(s.highest_field_id(), -1); + let json = s.serialize_json().unwrap(); + assert_eq!( + json.get("highest_field_id").and_then(|v| v.as_i64()), + Some(-1) + ); + } + + #[test] + fn reassign_field_ids_walks_array_map_row() { + let dt = DataTypes::array(DataTypes::row(vec![ + DataField::new("a", DataTypes::int(), None), + DataField::new("b", DataTypes::string(), None), + ])); + let mut counter = -1_i32; + let assigned = reassign_field_ids(&dt, &mut counter); + match assigned { + DataType::Array(at) => match at.get_element_type() { + DataType::Row(rt) => { + assert_eq!(rt.fields()[0].field_id(), 0); + assert_eq!(rt.fields()[1].field_id(), 1); + } + _ => panic!("expected Row"), + }, + _ => panic!("expected Array"), + } + assert_eq!(counter, 1); + + let dt = DataType::Map(MapType::new( + DataTypes::int(), + DataTypes::row(vec![DataField::new("x", DataTypes::int(), None)]), + )); + let mut counter = -1_i32; + let assigned = reassign_field_ids(&dt, &mut counter); + let value_type = match &assigned { + DataType::Map(mt) => mt.value_type(), + _ => panic!("expected Map"), + }; + match value_type { + DataType::Row(rt) => assert_eq!(rt.fields()[0].field_id(), 0), + _ => panic!("expected Row"), + } + assert_eq!(counter, 0); + } + #[test] fn test_datatype_json_serde() { let data_types = vec![ diff --git a/crates/fluss/src/metadata/table.rs b/crates/fluss/src/metadata/table.rs index ebaf085e..390bdbfc 100644 --- a/crates/fluss/src/metadata/table.rs +++ b/crates/fluss/src/metadata/table.rs @@ -19,7 +19,9 @@ use crate::compression::ArrowCompressionInfo; use crate::error::Error::IllegalArgument; use crate::error::{Error, Result}; use crate::metadata::DataLakeFormat; -use crate::metadata::datatype::{DataField, DataType, RowType}; +use crate::metadata::datatype::{ + DataField, DataType, RowType, UNASSIGNED_FIELD_ID, reassign_field_ids, +}; use crate::{BucketId, PartitionId, TableId}; use core::fmt; use serde::{Deserialize, Serialize}; @@ -112,12 +114,55 @@ impl PrimaryKey { } } +fn collect_field_id_state(data_type: &DataType, max_id: &mut i32, has_unassigned: &mut bool) { + match data_type { + DataType::Row(rt) => { + for f in rt.fields() { + if f.field_id == UNASSIGNED_FIELD_ID { + *has_unassigned = true; + } else { + *max_id = (*max_id).max(f.field_id); + } + collect_field_id_state(&f.data_type, max_id, has_unassigned); + } + } + DataType::Array(at) => { + collect_field_id_state(at.get_element_type(), max_id, has_unassigned); + } + DataType::Map(mt) => { + collect_field_id_state(mt.key_type(), max_id, has_unassigned); + collect_field_id_state(mt.value_type(), max_id, has_unassigned); + } + _ => {} + } +} + +fn collect_nested_field_ids(data_type: &DataType, ids: &mut Vec) { + match data_type { + DataType::Row(rt) => { + for f in rt.fields() { + if f.field_id != UNASSIGNED_FIELD_ID { + ids.push(f.field_id); + } + collect_nested_field_ids(&f.data_type, ids); + } + } + DataType::Array(at) => collect_nested_field_ids(at.get_element_type(), ids), + DataType::Map(mt) => { + collect_nested_field_ids(mt.key_type(), ids); + collect_nested_field_ids(mt.value_type(), ids); + } + _ => {} + } +} + #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] pub struct Schema { columns: Vec, primary_key: Option, row_type: RowType, auto_increment_col_names: Vec, + highest_field_id: i32, } impl Schema { @@ -167,6 +212,10 @@ impl Schema { pub fn auto_increment_col_names(&self) -> &Vec { &self.auto_increment_col_names } + + pub fn highest_field_id(&self) -> i32 { + self.highest_field_id + } } /// A schema together with its server-assigned version id. @@ -278,9 +327,9 @@ impl SchemaBuilder { pub fn build(&self) -> Result { let columns = Self::normalize_columns(&self.columns, self.primary_key.as_ref())?; - let columns = Self::assign_column_ids(columns)?; + let (columns_with_ids, highest_field_id) = Self::assign_all_field_ids(columns)?; - let column_names: HashSet<_> = columns.iter().map(|c| &c.name).collect(); + let column_names: HashSet<_> = columns_with_ids.iter().map(|c| &c.name).collect(); for auto_inc_col in &self.auto_increment_col_names { if !column_names.contains(auto_inc_col) { return Err(IllegalArgument { @@ -291,26 +340,115 @@ impl SchemaBuilder { } } - let data_fields = columns + let data_fields = columns_with_ids .iter() .map(|c| DataField { name: c.name.clone(), data_type: c.data_type.clone(), description: c.comment.clone(), + field_id: c.id, }) .collect(); Ok(Schema { - columns, + columns: columns_with_ids, primary_key: self.primary_key.clone(), row_type: RowType::new(data_fields), auto_increment_col_names: self.auto_increment_col_names.clone(), + highest_field_id, }) } + fn assign_all_field_ids(columns: Vec) -> Result<(Vec, i32)> { + let with_top_id = columns.iter().filter(|c| c.id != UNKNOWN_COLUMN_ID).count(); + let none_set = with_top_id == 0; + let all_top_set = with_top_id == columns.len(); + + if !none_set && !all_top_set { + return Err(IllegalArgument { + message: "All columns must have an id assigned, or none of them must.".to_string(), + }); + } + + let mut max_nested_id = -1_i32; + let mut has_unassigned_nested = false; + for c in &columns { + collect_field_id_state(&c.data_type, &mut max_nested_id, &mut has_unassigned_nested); + } + + if all_top_set && !has_unassigned_nested { + let mut seen: HashSet = HashSet::new(); + let mut max_id = -1_i32; + for col in &columns { + if col.id < 0 { + return Err(IllegalArgument { + message: format!( + "Column '{}' has invalid id {}; ids must be non-negative", + col.name, col.id + ), + }); + } + if !seen.insert(col.id) { + return Err(IllegalArgument { + message: format!("Duplicate field id {} in schema", col.id), + }); + } + max_id = max_id.max(col.id); + + let mut nested_ids = Vec::new(); + collect_nested_field_ids(&col.data_type, &mut nested_ids); + for id in nested_ids { + if id < 0 { + return Err(IllegalArgument { + message: format!( + "Nested DataField in column '{}' has invalid id {}; ids must be non-negative", + col.name, id + ), + }); + } + if !seen.insert(id) { + return Err(IllegalArgument { + message: format!( + "Duplicate field id {} in schema (column '{}')", + id, col.name + ), + }); + } + } + } + max_id = max_id.max(max_nested_id); + return Ok((columns, max_id)); + } + + if all_top_set && has_unassigned_nested { + return Err(IllegalArgument { + message: "Top-level column ids are set but some nested DataField ids are unassigned; reassign all or none." + .to_string(), + }); + } + + let mut counter: i32 = -1; + let new_columns: Vec = columns + .into_iter() + .map(|c| { + counter += 1; + let id = counter; + let new_data_type = reassign_field_ids(&c.data_type, &mut counter); + Column { + name: c.name, + data_type: new_data_type, + comment: c.comment, + id, + } + }) + .collect(); + Ok((new_columns, counter)) + } + /// All-or-none: preserve ids if every column has one, auto-assign /// 0..N-1 if none do, error on mixed input. When preserving ids, /// also reject duplicates and negative-but-not-sentinel values. + #[allow(dead_code)] fn assign_column_ids(columns: Vec) -> Result> { let with_id = columns.iter().filter(|c| c.id != UNKNOWN_COLUMN_ID).count(); if with_id == 0 { diff --git a/crates/fluss/src/record/arrow.rs b/crates/fluss/src/record/arrow.rs index d03f1454..9f1a9784 100644 --- a/crates/fluss/src/record/arrow.rs +++ b/crates/fluss/src/record/arrow.rs @@ -20,10 +20,10 @@ use crate::compression::{ ArrowCompressionInfo, ArrowCompressionRatioEstimator, ArrowCompressionType, }; use crate::error::{Error, Result}; -use crate::metadata::{DataType, RowType}; +use crate::metadata::{DataField, DataType, RowType}; use crate::record::{ChangeType, ScanRecord}; use crate::row::column_writer::{ColumnWriter, round_up_to_8}; -use crate::row::{ColumnarRow, InternalRow}; +use crate::row::{ColumnarRow, InternalRow, arrow_row_column_indices, fluss_row_column_indices}; use arrow::array::{ArrayBuilder, ArrayRef}; use arrow::{ array::RecordBatch, @@ -992,7 +992,10 @@ impl LogRecordBatch { let data = &self.data[RECORDS_OFFSET..]; let record_batch = read_context.record_batch(data)?; - let arrow_reader = ArrowReader::new(Arc::new(record_batch)); + let arrow_reader = ArrowReader::new_with_fluss_row_type( + Arc::new(record_batch), + read_context.fluss_row_type().cloned(), + ); let log_record_iterator = LogRecordIterator::Arrow(ArrowLogRecordIterator { reader: arrow_reader, base_offset: self.base_log_offset(), @@ -1015,7 +1018,10 @@ impl LogRecordBatch { let log_record_iterator = match record_batch { None => LogRecordIterator::empty(), Some(record_batch) => { - let arrow_reader = ArrowReader::new(Arc::new(record_batch)); + let arrow_reader = ArrowReader::new_with_fluss_row_type( + Arc::new(record_batch), + read_context.fluss_row_type().cloned(), + ); LogRecordIterator::Arrow(ArrowLogRecordIterator { reader: arrow_reader, base_offset: self.base_log_offset(), @@ -1216,7 +1222,7 @@ pub fn to_arrow_type(fluss_type: &DataType) -> Result { DataType::Array(array_type) => ArrowDataType::List( Field::new_list_field( to_arrow_type(array_type.get_element_type())?, - fluss_type.is_nullable(), + array_type.get_element_type().is_nullable(), ) .into(), ), @@ -1310,6 +1316,19 @@ pub(crate) fn from_arrow_type(arrow_type: &ArrowDataType) -> Result { } } ArrowDataType::List(field) => DataTypes::array(from_arrow_type(field.data_type())?), + ArrowDataType::Struct(fields) => { + let row_fields: Result> = fields + .iter() + .map(|f| { + let mut dt = from_arrow_type(f.data_type())?; + if !f.is_nullable() { + dt = dt.as_non_nullable(); + } + Ok(DataField::new(f.name(), dt, None)) + }) + .collect(); + DataTypes::row(row_fields?) + } other => { return Err(Error::IllegalArgument { message: format!("Cannot convert Arrow type to Fluss type: {other:?}"), @@ -1324,6 +1343,7 @@ pub struct ReadContext { full_schema: SchemaRef, projection: Option, is_from_remote: bool, + fluss_row_type: Option>, } #[derive(Clone)] @@ -1343,9 +1363,19 @@ impl ReadContext { full_schema: arrow_schema, projection: None, is_from_remote, + fluss_row_type: None, } } + pub fn with_fluss_row_type(mut self, fluss_row_type: Arc) -> ReadContext { + self.fluss_row_type = Some(fluss_row_type); + self + } + + pub fn fluss_row_type(&self) -> Option<&Arc> { + self.fluss_row_type.as_ref() + } + pub fn with_projection_pushdown( arrow_schema: SchemaRef, projected_fields: Vec, @@ -1414,6 +1444,7 @@ impl ReadContext { full_schema: arrow_schema, projection: Some(project), is_from_remote, + fluss_row_type: None, }) } @@ -1604,11 +1635,33 @@ impl Iterator for ArrowLogRecordIterator { pub struct ArrowReader { record_batch: Arc, + fluss_row_type: Option>, + row_column_indices: Arc<[usize]>, } impl ArrowReader { pub fn new(record_batch: Arc) -> Self { - ArrowReader { record_batch } + let row_column_indices = arrow_row_column_indices(&record_batch); + ArrowReader { + record_batch, + fluss_row_type: None, + row_column_indices, + } + } + + pub fn new_with_fluss_row_type( + record_batch: Arc, + fluss_row_type: Option>, + ) -> Self { + let row_column_indices = match &fluss_row_type { + Some(rt) => fluss_row_column_indices(rt), + None => arrow_row_column_indices(&record_batch), + }; + ArrowReader { + record_batch, + fluss_row_type, + row_column_indices, + } } pub fn row_count(&self) -> usize { @@ -1616,7 +1669,12 @@ impl ArrowReader { } pub fn read(&self, row_id: usize) -> ColumnarRow { - ColumnarRow::new_with_row_id(self.record_batch.clone(), row_id) + ColumnarRow::with_indices( + self.record_batch.clone(), + row_id, + self.fluss_row_type.clone(), + self.row_column_indices.clone(), + ) } } pub struct MyVec(pub StreamReader); diff --git a/crates/fluss/src/record/mod.rs b/crates/fluss/src/record/mod.rs index 8438b16d..522fb03b 100644 --- a/crates/fluss/src/record/mod.rs +++ b/crates/fluss/src/record/mod.rs @@ -249,7 +249,7 @@ mod tests { let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int32, false)])); let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(values))]) .expect("record batch"); - ColumnarRow::new_with_row_id(Arc::new(batch), row_id) + ColumnarRow::new(Arc::new(batch), row_id, None) } #[test] diff --git a/crates/fluss/src/row/binary/binary_writer.rs b/crates/fluss/src/row/binary/binary_writer.rs index fd7da4c9..7192ee57 100644 --- a/crates/fluss/src/row/binary/binary_writer.rs +++ b/crates/fluss/src/row/binary/binary_writer.rs @@ -19,7 +19,9 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; use crate::metadata::{DataType, RowType}; use crate::row::Datum; +use crate::row::Decimal; use crate::row::binary::BinaryRowFormat; +use crate::row::datum::{TimestampLtz, TimestampNtz}; /// Writer to write a composite data format, like row, array, #[allow(dead_code)] @@ -52,7 +54,7 @@ pub trait BinaryWriter { fn write_binary(&mut self, bytes: &[u8], length: usize); - fn write_decimal(&mut self, value: &crate::row::Decimal, precision: u32); + fn write_decimal(&mut self, value: &Decimal, precision: u32); /// Writes a TIME value. /// @@ -63,9 +65,9 @@ pub trait BinaryWriter { /// currently vary by precision. fn write_time(&mut self, value: i32, precision: u32); - fn write_timestamp_ntz(&mut self, value: &crate::row::datum::TimestampNtz, precision: u32); + fn write_timestamp_ntz(&mut self, value: &TimestampNtz, precision: u32); - fn write_timestamp_ltz(&mut self, value: &crate::row::datum::TimestampLtz, precision: u32); + fn write_timestamp_ltz(&mut self, value: &TimestampLtz, precision: u32); fn write_array(&mut self, value: &[u8]); @@ -136,7 +138,36 @@ pub enum InnerValueWriter { TimestampNtz(u32), // precision TimestampLtz(u32), // precision Array, - Row(RowType), + Row(NestedRowWriter), +} + +#[derive(Debug)] +pub struct NestedRowWriter { + field_writers: Vec, + field_nullable: Vec, +} + +impl NestedRowWriter { + fn from_row_type(row_type: &RowType) -> Result { + let fields = row_type.fields(); + let mut field_writers = Vec::with_capacity(fields.len()); + let mut field_nullable = Vec::with_capacity(fields.len()); + for field in fields { + field_writers.push(InnerValueWriter::create_inner_value_writer( + field.data_type(), + None, + )?); + field_nullable.push(field.data_type().is_nullable()); + } + Ok(Self { + field_writers, + field_nullable, + }) + } + + fn field_count(&self) -> usize { + self.field_writers.len() + } } /// Accessor for writing the fields/elements of a binary writer during runtime, the @@ -176,7 +207,9 @@ impl InnerValueWriter { Ok(InnerValueWriter::TimestampLtz(t.precision())) } DataType::Array(_) => Ok(InnerValueWriter::Array), - DataType::Row(row_type) => Ok(InnerValueWriter::Row(row_type.clone())), + DataType::Row(row_type) => Ok(InnerValueWriter::Row(NestedRowWriter::from_row_type( + row_type, + )?)), _ => unimplemented!( "ValueWriter for DataType {:?} is currently not implemented", data_type @@ -242,22 +275,31 @@ impl InnerValueWriter { (InnerValueWriter::Array, Datum::Array(arr)) => { writer.write_array(arr.as_bytes()); } - (InnerValueWriter::Row(row_type), Datum::Row(inner_row)) => { + (InnerValueWriter::Row(nested_writer), Datum::Row(inner_row)) => { use crate::row::compacted::CompactedRowWriter; - let field_count = row_type.fields().len(); + let field_count = nested_writer.field_count(); + if inner_row.values.len() != field_count { + return Err(IllegalArgument { + message: format!( + "nested row arity mismatch: schema has {} fields, got {}", + field_count, + inner_row.values.len(), + ), + }); + } let mut nested = CompactedRowWriter::new(field_count); - for (i, field) in row_type.fields().iter().enumerate() { - let datum = &inner_row.values[i]; + for (i, datum) in inner_row.values.iter().enumerate() { if datum.is_null() { - if field.data_type.is_nullable() { - nested.set_null_at(i); + if !nested_writer.field_nullable[i] { + return Err(IllegalArgument { + message: format!( + "nested row field {i} is non-nullable but received null", + ), + }); } + nested.set_null_at(i); } else { - let vw = - InnerValueWriter::create_inner_value_writer(&field.data_type, None) - .expect("create_inner_value_writer failed for nested row field"); - vw.write_value(&mut nested, i, datum) - .expect("write_value failed for nested row field"); + nested_writer.field_writers[i].write_value(&mut nested, i, datum)?; } } writer.write_bytes(nested.buffer()); diff --git a/crates/fluss/src/row/binary_array.rs b/crates/fluss/src/row/binary_array.rs index 3383a366..d0e8c9a5 100644 --- a/crates/fluss/src/row/binary_array.rs +++ b/crates/fluss/src/row/binary_array.rs @@ -26,9 +26,13 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; -use crate::metadata::DataType; +use crate::metadata::{DataType, RowType}; use crate::row::Decimal; +use crate::row::InternalRow; +use crate::row::binary::{BinaryRowFormat, ValueWriter}; +use crate::row::compacted::{CompactedRow, CompactedRowWriter, calculate_bit_set_width_in_bytes}; use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; +use crate::row::field_getter::FieldGetter; use bytes::Bytes; use serde::Serialize; use std::fmt; @@ -418,6 +422,48 @@ impl FlussArray { let (start, len) = self.read_var_len_span(pos)?; FlussArray::from_owned_bytes(self.data.slice(start..start + len)) } + + pub fn get_row<'a>(&'a self, pos: usize, row_type: &'a RowType) -> Result> { + let bytes = self.read_var_len_bytes(pos)?; + let header_size = calculate_bit_set_width_in_bytes(row_type.fields().len()); + if bytes.len() < header_size { + return Err(IllegalArgument { + message: format!( + "FlussArray row bytes at position {} are too short for row type with {} fields: \ + need at least {} header bytes, got {}", + pos, + row_type.fields().len(), + header_size, + bytes.len() + ), + }); + } + Ok(CompactedRow::from_bytes(row_type, bytes)) + } +} + +struct RowFieldAccessor { + getter: FieldGetter, + writer: ValueWriter, + nullable: bool, +} + +fn build_row_accessors(row_type: &RowType) -> Result> { + row_type + .fields() + .iter() + .enumerate() + .map(|(i, f)| { + Ok(RowFieldAccessor { + getter: FieldGetter::create(f.data_type(), i), + writer: ValueWriter::create_value_writer( + f.data_type(), + Some(&BinaryRowFormat::Compacted), + )?, + nullable: f.data_type().is_nullable(), + }) + }) + .collect() } /// Writer for building a `FlussArray` element by element. @@ -429,17 +475,34 @@ pub struct FlussArrayWriter { element_size: usize, cursor: usize, num_elements: usize, + // Some(_) only when constructed with a DataType::Row(_) element type. + row_accessors: Option>, } impl FlussArrayWriter { /// Creates a new writer for an array with `num_elements` elements of the given element type. pub fn new(num_elements: usize, element_type: &DataType) -> Self { let element_size = calculate_fix_length_part_size(element_type); - Self::with_element_size(num_elements, element_size) + let row_accessors = match element_type { + DataType::Row(rt) => Some( + build_row_accessors(rt) + .expect("ROW element type contains a field with no ValueWriter"), + ), + _ => None, + }; + Self::with_state(num_elements, element_size, row_accessors) } - /// Creates a new writer with an explicit element size (in bytes). + /// Creates a new writer with an explicit element size (in bytes). Does not support `write_row`. pub fn with_element_size(num_elements: usize, element_size: usize) -> Self { + Self::with_state(num_elements, element_size, None) + } + + fn with_state( + num_elements: usize, + element_size: usize, + row_accessors: Option>, + ) -> Self { let header_in_bytes = calculate_header_in_bytes(num_elements); let fixed_size = round_to_nearest_word(header_in_bytes + element_size * num_elements); let mut data = vec![0u8; fixed_size]; @@ -454,6 +517,7 @@ impl FlussArrayWriter { element_size, cursor: fixed_size, num_elements, + row_accessors, } } @@ -608,6 +672,26 @@ impl FlussArrayWriter { self.write_bytes_to_var_len_part(pos, value.as_bytes()); } + /// Writes a nested row at `pos`. Requires the writer to have been + /// constructed via [`new`](Self::new) with a `DataType::Row(_)` element type. + pub fn write_row(&mut self, pos: usize, row: &dyn InternalRow) -> Result<()> { + let accessors = self.row_accessors.as_ref().ok_or_else(|| IllegalArgument { + message: "write_row requires a DataType::Row element type".to_string(), + })?; + let mut nested = CompactedRowWriter::new(accessors.len()); + for (i, accessor) in accessors.iter().enumerate() { + if !accessor.nullable && row.is_null_at(i)? { + return Err(IllegalArgument { + message: format!("nested row field {i} is non-nullable but received null"), + }); + } + let datum = accessor.getter.get_field(row)?; + accessor.writer.write_value(&mut nested, i, &datum)?; + } + self.write_bytes_to_var_len_part(pos, nested.buffer()); + Ok(()) + } + /// Finalizes the writer and returns the completed FlussArray. pub fn complete(self) -> Result { let mut data = self.data; @@ -621,7 +705,7 @@ impl FlussArrayWriter { } } -impl crate::row::InternalRow for FlussArray { +impl InternalRow for FlussArray { fn get_field_count(&self) -> usize { self.size() } @@ -693,7 +777,10 @@ impl crate::row::InternalRow for FlussArray { #[cfg(test)] mod tests { use super::*; - use crate::metadata::DataTypes; + use crate::metadata::{DataField, DataTypes}; + use crate::row::binary::BinaryWriter as BinaryWriterTrait; + use crate::row::compacted::CompactedRowWriter; + use crate::row::{Datum, GenericRow}; #[test] fn test_header_calculation() { @@ -843,18 +930,232 @@ mod tests { assert_eq!(array.get_double(1).unwrap(), -4.56); } + #[test] + fn test_round_trip_array_of_row() { + let row_type_owned = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let element_type = row_type_owned.clone(); + let row_type = match &row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + + // Build array> with two rows: (42, "hello"), (-1, null) + let mut writer = FlussArrayWriter::new(2, &element_type); + + let mut r0 = GenericRow::new(2); + r0.set_field(0, 42_i32); + r0.set_field(1, "hello"); + writer.write_row(0, &r0).expect("write row 0"); + + let mut r1 = GenericRow::new(2); + r1.set_field(0, -1_i32); + r1.set_field(1, Datum::Null); + writer.write_row(1, &r1).expect("write row 1"); + + let array = writer.complete().unwrap(); + assert_eq!(array.size(), 2); + + let row0 = array.get_row(0, row_type).expect("get row 0"); + assert_eq!(row0.get_int(0).unwrap(), 42); + assert_eq!(row0.get_string(1).unwrap(), "hello"); + + let row1 = array.get_row(1, row_type).expect("get row 1"); + assert_eq!(row1.get_int(0).unwrap(), -1); + assert!(row1.is_null_at(1).unwrap()); + } + + #[test] + fn test_get_row_rejects_oversized_row_type() { + let small_row_type_owned = + DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let small_row_type = match &small_row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + let mut writer = FlussArrayWriter::new(1, &small_row_type_owned); + let mut row = GenericRow::new(1); + row.set_field(0, 7_i32); + writer.write_row(0, &row).unwrap(); + let array = writer.complete().unwrap(); + + let oversized_owned = DataTypes::row( + (0..10) + .map(|i| DataField::new(format!("f{i}"), DataTypes::int(), None)) + .collect(), + ); + let oversized_row_type = match &oversized_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + let huge_owned = DataTypes::row( + (0..100) + .map(|i| DataField::new(format!("f{i}"), DataTypes::int(), None)) + .collect(), + ); + let huge_row_type = match &huge_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + match array.get_row(0, huge_row_type) { + Err(e) => assert!( + e.to_string().contains("too short for row type"), + "unexpected error: {e}" + ), + Ok(_) => panic!("expected oversized row_type to be rejected"), + } + + let recovered = array.get_row(0, small_row_type).unwrap(); + assert_eq!(recovered.get_int(0).unwrap(), 7); + + let _ = oversized_row_type; + } + + #[test] + fn test_round_trip_array_of_row_with_nullable_element() { + let row_type_owned = DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let element_type = row_type_owned.clone(); + let row_type = match &row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + + let mut writer = FlussArrayWriter::new(3, &element_type); + + let mut r0 = GenericRow::new(1); + r0.set_field(0, 7_i32); + writer.write_row(0, &r0).expect("write row 0"); + + writer.set_null_at(1); + + let mut r2 = GenericRow::new(1); + r2.set_field(0, 8_i32); + writer.write_row(2, &r2).expect("write row 2"); + + let array = writer.complete().unwrap(); + + let row0 = array.get_row(0, row_type).unwrap(); + assert_eq!(row0.get_int(0).unwrap(), 7); + assert!(array.is_null_at(1)); + let row2 = array.get_row(2, row_type).unwrap(); + assert_eq!(row2.get_int(0).unwrap(), 8); + + let strict_row_type_owned = DataTypes::row(vec![DataField::new( + "n", + DataTypes::int().as_non_nullable(), + None, + )]); + let mut bad_writer = FlussArrayWriter::new(1, &strict_row_type_owned); + let mut bad = GenericRow::new(1); + bad.set_field(0, Datum::Null); + let err = bad_writer.write_row(0, &bad).unwrap_err(); + assert!( + err.to_string().contains("non-nullable"), + "unexpected error: {err}" + ); + } + + #[test] + fn test_round_trip_array_of_row_of_array_of_string() { + let inner_array_type = DataTypes::array(DataTypes::string()); + let inner_row_type_owned = + DataTypes::row(vec![DataField::new("tags", inner_array_type.clone(), None)]); + let inner_row_type = match &inner_row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + + let mut tags1 = FlussArrayWriter::new(2, &DataTypes::string()); + tags1.write_string(0, "alpha"); + tags1.write_string(1, "beta"); + let tags1 = tags1.complete().unwrap(); + let mut row1 = GenericRow::new(1); + row1.set_field(0, tags1); + + let mut tags2 = FlussArrayWriter::new(3, &DataTypes::string()); + tags2.write_string(0, "x"); + tags2.set_null_at(1); + tags2.write_string(2, "z"); + let tags2 = tags2.complete().unwrap(); + let mut row2 = GenericRow::new(1); + row2.set_field(0, tags2); + + let mut outer_writer = FlussArrayWriter::new(2, &inner_row_type_owned); + outer_writer.write_row(0, &row1).unwrap(); + outer_writer.write_row(1, &row2).unwrap(); + let outer = outer_writer.complete().unwrap(); + + assert_eq!(outer.size(), 2); + + let r0 = outer.get_row(0, inner_row_type).unwrap(); + let r0_tags = r0.get_array(0).unwrap(); + assert_eq!(r0_tags.size(), 2); + assert_eq!(r0_tags.get_string(0).unwrap(), "alpha"); + assert_eq!(r0_tags.get_string(1).unwrap(), "beta"); + + let r1 = outer.get_row(1, inner_row_type).unwrap(); + let r1_tags = r1.get_array(0).unwrap(); + assert_eq!(r1_tags.size(), 3); + assert_eq!(r1_tags.get_string(0).unwrap(), "x"); + assert!(r1_tags.is_null_at(1)); + assert_eq!(r1_tags.get_string(2).unwrap(), "z"); + } + + #[test] + fn test_round_trip_row_of_array_of_row() { + let inner_row_type_owned = + DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let inner_array_type = DataTypes::array(inner_row_type_owned.clone()); + let outer_row_type_owned = + DataTypes::row(vec![DataField::new("arr", inner_array_type.clone(), None)]); + + let outer_row_type = match &outer_row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + let inner_row_type = match &inner_row_type_owned { + DataType::Row(rt) => rt, + _ => unreachable!(), + }; + + let mut arr_writer = FlussArrayWriter::new(2, &inner_row_type_owned); + let mut r0 = GenericRow::new(1); + r0.set_field(0, 1_i32); + arr_writer.write_row(0, &r0).unwrap(); + let mut r1 = GenericRow::new(1); + r1.set_field(0, 2_i32); + arr_writer.write_row(1, &r1).unwrap(); + let inner_arr = arr_writer.complete().unwrap(); + + let mut outer = GenericRow::new(1); + outer.set_field(0, inner_arr.clone()); + + let mut writer = CompactedRowWriter::new(1); + writer.write_array(inner_arr.as_bytes()); + let bytes = writer.to_bytes(); + + let outer_compacted = CompactedRow::from_bytes(outer_row_type, &bytes); + let recovered_arr = outer_compacted.get_array(0).unwrap(); + assert_eq!(recovered_arr.size(), 2); + + let recovered_r0 = recovered_arr.get_row(0, inner_row_type).unwrap(); + assert_eq!(recovered_r0.get_int(0).unwrap(), 1); + let recovered_r1 = recovered_arr.get_row(1, inner_row_type).unwrap(); + assert_eq!(recovered_r1.get_int(0).unwrap(), 2); + } + #[test] fn test_round_trip_nested_array() { let inner_type = DataTypes::int(); let outer_type = DataTypes::array(DataTypes::int()); - // Build inner array [1, 2] let mut inner_writer = FlussArrayWriter::new(2, &inner_type); inner_writer.write_int(0, 1); inner_writer.write_int(1, 2); let inner_array = inner_writer.complete().unwrap(); - // Build outer array containing the inner array let mut outer_writer = FlussArrayWriter::new(1, &outer_type); outer_writer.write_array(0, &inner_array); let outer_array = outer_writer.complete().unwrap(); diff --git a/crates/fluss/src/row/column.rs b/crates/fluss/src/row/column.rs index 95cd3395..5db1cd50 100644 --- a/crates/fluss/src/row/column.rs +++ b/crates/fluss/src/row/column.rs @@ -17,11 +17,17 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; -use crate::row::{GenericRow, InternalRow}; +use crate::metadata::{DataType, RowType}; +use crate::record::from_arrow_type; +use crate::row::binary_array::FlussArrayWriter; use crate::row::datum::{Date, Datum, Time, TimestampLtz, TimestampNtz}; +use crate::row::{Decimal, FlussArray, GenericRow, InternalRow}; use arrow::array::{ - Array, AsArray, BinaryArray, BooleanArray, FixedSizeBinaryArray, ListArray, RecordBatch, - StringArray, + Array, AsArray, BinaryArray, BooleanArray, Date32Array, Decimal128Array, FixedSizeBinaryArray, + Float32Array, Float64Array, Int8Array, Int16Array, Int32Array, Int64Array, ListArray, + RecordBatch, StringArray, StructArray, Time32MillisecondArray, Time32SecondArray, + Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, + TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, }; use arrow::datatypes::{ DataType as ArrowDataType, Date32Type, Decimal128Type, Float32Type, Float64Type, Int8Type, @@ -35,31 +41,69 @@ use std::sync::Arc; pub struct ColumnarRow { record_batch: Arc, row_id: usize, - nested_rows: Vec>>, + fluss_row_type: Option>, + row_column_indices: Arc<[usize]>, + row_caches: Box<[std::sync::OnceLock>]>, +} + +pub(crate) fn fluss_row_column_indices(row_type: &RowType) -> Arc<[usize]> { + row_type + .fields() + .iter() + .enumerate() + .filter_map(|(i, f)| matches!(f.data_type, DataType::Row(_)).then_some(i)) + .collect() +} + +pub(crate) fn arrow_row_column_indices(batch: &RecordBatch) -> Arc<[usize]> { + batch + .columns() + .iter() + .enumerate() + .filter_map(|(i, c)| matches!(c.data_type(), ArrowDataType::Struct(_)).then_some(i)) + .collect() +} + +fn make_row_caches(indices: &[usize]) -> Box<[std::sync::OnceLock>]> { + indices.iter().map(|_| std::sync::OnceLock::new()).collect() } impl ColumnarRow { - pub fn new(batch: Arc) -> Self { - let num_cols = batch.num_columns(); - ColumnarRow { - record_batch: batch, - row_id: 0, - nested_rows: (0..num_cols).map(|_| std::sync::OnceLock::new()).collect(), - } + pub fn new( + batch: Arc, + row_id: usize, + fluss_row_type: Option>, + ) -> Self { + let row_column_indices = match &fluss_row_type { + Some(rt) => fluss_row_column_indices(rt), + None => arrow_row_column_indices(&batch), + }; + Self::with_indices(batch, row_id, fluss_row_type, row_column_indices) } - pub fn new_with_row_id(bach: Arc, row_id: usize) -> Self { - let num_cols = bach.num_columns(); + pub(crate) fn with_indices( + batch: Arc, + row_id: usize, + fluss_row_type: Option>, + row_column_indices: Arc<[usize]>, + ) -> Self { + let row_caches = make_row_caches(&row_column_indices); ColumnarRow { - record_batch: bach, + record_batch: batch, row_id, - nested_rows: (0..num_cols).map(|_| std::sync::OnceLock::new()).collect(), + fluss_row_type, + row_column_indices, + row_caches, } } + pub fn fluss_row_type(&self) -> Option<&Arc> { + self.fluss_row_type.as_ref() + } + pub fn set_row_id(&mut self, row_id: usize) { self.row_id = row_id; - for lock in &mut self.nested_rows { + for lock in self.row_caches.iter_mut() { *lock = std::sync::OnceLock::new(); } } @@ -220,167 +264,161 @@ impl ColumnarRow { }), } } +} - /// Extract a `GenericRow<'static>` from a column in the RecordBatch at the given row_id. - fn extract_struct_at( - batch: &RecordBatch, - pos: usize, - row_id: usize, - ) -> Result> { - let col = batch.column(pos); - Self::extract_struct_from_array(col.as_ref(), row_id) +fn extract_struct_from_array( + array: &dyn Array, + row_id: usize, + row_type: Option<&RowType>, +) -> Result> { + let sa = array + .as_any() + .downcast_ref::() + .ok_or_else(|| IllegalArgument { + message: format!("expected StructArray, got {:?}", array.data_type()), + })?; + if let Some(rt) = row_type + && rt.fields().len() != sa.num_columns() + { + return Err(IllegalArgument { + message: format!( + "Fluss RowType has {} fields but Arrow StructArray has {}", + rt.fields().len(), + sa.num_columns(), + ), + }); } + let mut values = Vec::with_capacity(sa.num_columns()); + for i in 0..sa.num_columns() { + let child = sa.column(i); + let fluss_type = row_type.map(|rt| &rt.fields()[i].data_type); + values.push(arrow_value_to_datum(child.as_ref(), row_id, fluss_type)?); + } + Ok(GenericRow { values }) +} - /// Recursively extract a `GenericRow<'static>` from a `StructArray` at row_id. - fn extract_struct_from_array(array: &dyn Array, row_id: usize) -> Result> { - use arrow::array::StructArray; - let sa = array - .as_any() - .downcast_ref::() - .ok_or_else(|| IllegalArgument { - message: format!("expected StructArray, got {:?}", array.data_type()), - })?; - let mut values = Vec::with_capacity(sa.num_columns()); - for i in 0..sa.num_columns() { - let child = sa.column(i); - values.push(Self::arrow_value_to_datum(child.as_ref(), row_id)?); - } - Ok(GenericRow { values }) +fn arrow_value_to_datum( + array: &dyn Array, + row_id: usize, + fluss_type: Option<&DataType>, +) -> Result> { + if array.is_null(row_id) { + return Ok(Datum::Null); } - /// Convert a single element at `row_id` in an Arrow array to a `Datum<'static>`. - fn arrow_value_to_datum(array: &dyn Array, row_id: usize) -> Result> { - use arrow::array::{ - BooleanArray, Decimal128Array, Float32Array, Float64Array, Int8Array, Int16Array, - Int32Array, Int64Array, Time32MillisecondArray, Time32SecondArray, - Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, - TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, + macro_rules! downcast { + ($ty:ty) => { + array + .as_any() + .downcast_ref::<$ty>() + .ok_or_else(|| IllegalArgument { + message: format!( + "expected {} for arrow type {:?}", + stringify!($ty), + array.data_type() + ), + })? }; - use crate::row::Decimal; + } - if array.is_null(row_id) { - return Ok(Datum::Null); + match array.data_type() { + ArrowDataType::Boolean => Ok(Datum::Bool(downcast!(BooleanArray).value(row_id))), + ArrowDataType::Int8 => Ok(Datum::Int8(downcast!(Int8Array).value(row_id))), + ArrowDataType::Int16 => Ok(Datum::Int16(downcast!(Int16Array).value(row_id))), + ArrowDataType::Int32 => Ok(Datum::Int32(downcast!(Int32Array).value(row_id))), + ArrowDataType::Int64 => Ok(Datum::Int64(downcast!(Int64Array).value(row_id))), + ArrowDataType::Float32 => Ok(Datum::Float32(downcast!(Float32Array).value(row_id).into())), + ArrowDataType::Float64 => Ok(Datum::Float64(downcast!(Float64Array).value(row_id).into())), + ArrowDataType::Utf8 => Ok(Datum::String(std::borrow::Cow::Owned( + downcast!(StringArray).value(row_id).to_owned(), + ))), + ArrowDataType::Binary => Ok(Datum::Blob(std::borrow::Cow::Owned( + downcast!(BinaryArray).value(row_id).to_vec(), + ))), + ArrowDataType::FixedSizeBinary(_) => Ok(Datum::Blob(std::borrow::Cow::Owned( + downcast!(FixedSizeBinaryArray).value(row_id).to_vec(), + ))), + ArrowDataType::Decimal128(p, s) => { + let (p, s) = (*p, *s); + let i128_val = downcast!(Decimal128Array).value(row_id); + Ok(Datum::Decimal(Decimal::from_arrow_decimal128( + i128_val, s as i64, p as u32, s as u32, + )?)) } - - match array.data_type() { - ArrowDataType::Boolean => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Bool(a.value(row_id))) - } - ArrowDataType::Int8 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Int8(a.value(row_id))) - } - ArrowDataType::Int16 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Int16(a.value(row_id))) - } - ArrowDataType::Int32 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Int32(a.value(row_id))) - } - ArrowDataType::Int64 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Int64(a.value(row_id))) - } - ArrowDataType::Float32 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Float32(a.value(row_id).into())) - } - ArrowDataType::Float64 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Float64(a.value(row_id).into())) - } - ArrowDataType::Utf8 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::String(std::borrow::Cow::Owned(a.value(row_id).to_owned()))) - } - ArrowDataType::Binary => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Blob(std::borrow::Cow::Owned(a.value(row_id).to_vec()))) - } - ArrowDataType::Decimal128(p, s) => { - let (p, s) = (*p, *s); - let a = array.as_any().downcast_ref::().unwrap(); - let i128_val = a.value(row_id); - Ok(Datum::Decimal(Decimal::from_arrow_decimal128( - i128_val, - s as i64, - p as u32, - s as u32, - )?)) - } - ArrowDataType::Date32 => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Date(Date::new(a.value(row_id)))) - } - ArrowDataType::Time32(TimeUnit::Second) => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Time(Time::new(a.value(row_id) * 1000))) - } - ArrowDataType::Time32(TimeUnit::Millisecond) => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Time(Time::new(a.value(row_id)))) - } - ArrowDataType::Time64(TimeUnit::Microsecond) => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Time(Time::new((a.value(row_id) / 1000) as i32))) - } - ArrowDataType::Time64(TimeUnit::Nanosecond) => { - let a = array.as_any().downcast_ref::().unwrap(); - Ok(Datum::Time(Time::new((a.value(row_id) / 1_000_000) as i32))) - } - ArrowDataType::Timestamp(time_unit, tz) => { - let value: i64 = match time_unit { - TimeUnit::Second => { - array.as_any().downcast_ref::().unwrap().value(row_id) - } - TimeUnit::Millisecond => { - array.as_any().downcast_ref::().unwrap().value(row_id) - } - TimeUnit::Microsecond => { - array.as_any().downcast_ref::().unwrap().value(row_id) - } - TimeUnit::Nanosecond => { - array.as_any().downcast_ref::().unwrap().value(row_id) - } - }; - let (millis, nanos) = match time_unit { - TimeUnit::Second => (value * 1000, 0i32), - TimeUnit::Millisecond => (value, 0i32), - TimeUnit::Microsecond => { - let millis = value.div_euclid(1000); - let nanos = (value.rem_euclid(1000) * 1000) as i32; - (millis, nanos) - } - TimeUnit::Nanosecond => { - let millis = value.div_euclid(1_000_000); - let nanos = value.rem_euclid(1_000_000) as i32; - (millis, nanos) - } - }; - if tz.is_some() { - if nanos == 0 { - Ok(Datum::TimestampLtz(TimestampLtz::new(millis))) - } else { - Ok(Datum::TimestampLtz(TimestampLtz::from_millis_nanos(millis, nanos)?)) - } - } else if nanos == 0 { - Ok(Datum::TimestampNtz(TimestampNtz::new(millis))) + ArrowDataType::Date32 => Ok(Datum::Date(Date::new(downcast!(Date32Array).value(row_id)))), + ArrowDataType::Time32(TimeUnit::Second) => Ok(Datum::Time(Time::new( + downcast!(Time32SecondArray).value(row_id) * 1000, + ))), + ArrowDataType::Time32(TimeUnit::Millisecond) => Ok(Datum::Time(Time::new( + downcast!(Time32MillisecondArray).value(row_id), + ))), + ArrowDataType::Time64(TimeUnit::Microsecond) => Ok(Datum::Time(Time::new( + (downcast!(Time64MicrosecondArray).value(row_id) / 1000) as i32, + ))), + ArrowDataType::Time64(TimeUnit::Nanosecond) => Ok(Datum::Time(Time::new( + (downcast!(Time64NanosecondArray).value(row_id) / 1_000_000) as i32, + ))), + ArrowDataType::Timestamp(time_unit, _tz) => { + let value: i64 = match time_unit { + TimeUnit::Second => downcast!(TimestampSecondArray).value(row_id), + TimeUnit::Millisecond => downcast!(TimestampMillisecondArray).value(row_id), + TimeUnit::Microsecond => downcast!(TimestampMicrosecondArray).value(row_id), + TimeUnit::Nanosecond => downcast!(TimestampNanosecondArray).value(row_id), + }; + let (millis, nanos) = match time_unit { + TimeUnit::Second => (value * 1000, 0i32), + TimeUnit::Millisecond => (value, 0i32), + TimeUnit::Microsecond => { + let millis = value.div_euclid(1000); + let nanos = (value.rem_euclid(1000) * 1000) as i32; + (millis, nanos) + } + TimeUnit::Nanosecond => { + let millis = value.div_euclid(1_000_000); + let nanos = value.rem_euclid(1_000_000) as i32; + (millis, nanos) + } + }; + // TIMESTAMP and TIMESTAMP_LTZ both map to `Timestamp(unit, None)` in Arrow. + let is_ltz = matches!(fluss_type, Some(DataType::TimestampLTz(_))); + if is_ltz { + if nanos == 0 { + Ok(Datum::TimestampLtz(TimestampLtz::new(millis))) } else { - Ok(Datum::TimestampNtz(TimestampNtz::from_millis_nanos(millis, nanos)?)) + Ok(Datum::TimestampLtz(TimestampLtz::from_millis_nanos( + millis, nanos, + )?)) } + } else if nanos == 0 { + Ok(Datum::TimestampNtz(TimestampNtz::new(millis))) + } else { + Ok(Datum::TimestampNtz(TimestampNtz::from_millis_nanos( + millis, nanos, + )?)) } - ArrowDataType::Struct(_) => { - let nested = Self::extract_struct_from_array(array, row_id)?; - Ok(Datum::Row(Box::new(nested))) - } - other => Err(IllegalArgument { - message: format!( - "unsupported Arrow data type for nested row extraction: {other:?}" - ), - }), } + ArrowDataType::Struct(_) => { + let nested_row_type = fluss_type.and_then(|t| match t { + DataType::Row(rt) => Some(rt), + _ => None, + }); + let nested = extract_struct_from_array(array, row_id, nested_row_type)?; + Ok(Datum::Row(Box::new(nested))) + } + ArrowDataType::List(field) => { + let list_arr = downcast!(ListArray); + let values = list_arr.value(row_id); + let element_fluss_type = match fluss_type { + Some(DataType::Array(at)) => at.get_element_type().clone(), + _ => from_arrow_type(field.data_type())?, + }; + let mut writer = FlussArrayWriter::new(values.len(), &element_fluss_type); + write_arrow_values_to_fluss_array(&*values, &element_fluss_type, &mut writer)?; + Ok(Datum::Array(writer.complete()?)) + } + other => Err(IllegalArgument { + message: format!("unsupported Arrow data type for nested row extraction: {other:?}"), + }), } } @@ -485,14 +523,7 @@ impl InternalRow for ColumnarRow { .value(self.row_id)) } - fn get_decimal( - &self, - pos: usize, - precision: usize, - scale: usize, - ) -> Result { - use arrow::datatypes::DataType; - + fn get_decimal(&self, pos: usize, precision: usize, scale: usize) -> Result { let column = self.column(pos)?; let array = column .as_primitive_opt::() @@ -511,9 +542,8 @@ impl InternalRow for ColumnarRow { self.row_id ); - // Read scale from Arrow column data type let arrow_scale = match column.data_type() { - DataType::Decimal128(_p, s) => *s as i64, + ArrowDataType::Decimal128(_p, s) => *s as i64, dt => { return Err(IllegalArgument { message: format!( @@ -523,11 +553,8 @@ impl InternalRow for ColumnarRow { } }; - let i128_val = array.value(self.row_id); - - // Convert Arrow Decimal128 to Fluss Decimal (handles rescaling and validation) - crate::row::Decimal::from_arrow_decimal128( - i128_val, + Decimal::from_arrow_decimal128( + array.value(self.row_id), arrow_scale, precision as u32, scale as u32, @@ -581,10 +608,7 @@ impl InternalRow for ColumnarRow { .value(self.row_id)) } - fn get_array(&self, pos: usize) -> Result { - use crate::record::from_arrow_type; - use crate::row::binary_array::FlussArrayWriter; - + fn get_array(&self, pos: usize) -> Result { let column = self.column(pos)?; let values = if let Some(list_arr) = column.as_any().downcast_ref::() { list_arr.value(self.row_id) @@ -605,15 +629,37 @@ impl InternalRow for ColumnarRow { } fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { - let lock = self.nested_rows.get(pos).ok_or_else(|| IllegalArgument { - message: format!("column index {pos} out of bounds for get_row"), - })?; - let batch = Arc::clone(&self.record_batch); - let row_id = self.row_id; - Ok(lock.get_or_init(|| { - Self::extract_struct_at(&batch, pos, row_id) - .expect("failed to extract nested row from StructArray") - })) + let cache_idx = self + .row_column_indices + .iter() + .position(|&i| i == pos) + .ok_or_else(|| IllegalArgument { + message: format!("get_row called on non-ROW column at position {pos}"), + })?; + let column = self.record_batch.column(pos); + // Children of a null parent may carry stale bytes; caller must + // check is_null_at first rather than rely on what we'd read. + if column.is_null(self.row_id) { + return Err(IllegalArgument { + message: format!( + "get_row called on null ROW cell at position {pos}, row {}; \ + check is_null_at({pos}) first", + self.row_id + ), + }); + } + let lock = &self.row_caches[cache_idx]; + if let Some(row) = lock.get() { + return Ok(row); + } + let nested_row_type = self.fluss_row_type.as_ref().and_then(|rt| { + rt.fields().get(pos).and_then(|f| match &f.data_type { + DataType::Row(inner) => Some(inner), + _ => None, + }) + }); + let extracted = extract_struct_from_array(column.as_ref(), self.row_id, nested_row_type)?; + Ok(lock.get_or_init(|| extracted)) } } @@ -624,7 +670,7 @@ macro_rules! write_primitive_elements { .as_primitive_opt::<$arrow_type>() .ok_or_else(|| IllegalArgument { message: format!( - "Expected {} for {:?} element", + "expected {} for {:?} element", stringify!($arrow_type), $element_type ), @@ -647,7 +693,7 @@ macro_rules! write_downcast_elements { .downcast_ref::<$array_type>() .ok_or_else(|| IllegalArgument { message: format!( - "Expected {} for {:?} element", + "expected {} for {:?} element", stringify!($array_type), $element_type ), @@ -670,7 +716,7 @@ macro_rules! write_list_elements { .downcast_ref::<$list_array_type>() .ok_or_else(|| IllegalArgument { message: format!( - "Expected {} for {:?} element", + "expected {} for {:?} element", stringify!($list_array_type), $element_type ), @@ -699,13 +745,9 @@ macro_rules! write_list_elements { /// the Arrow array once per call rather than per element. fn write_arrow_values_to_fluss_array( values: &dyn Array, - element_type: &crate::metadata::DataType, - writer: &mut crate::row::binary_array::FlussArrayWriter, + element_type: &DataType, + writer: &mut FlussArrayWriter, ) -> Result<()> { - use crate::metadata::DataType; - use crate::record::from_arrow_type; - use crate::row::binary_array::FlussArrayWriter; - let len = values.len(); match element_type { @@ -756,14 +798,14 @@ fn write_arrow_values_to_fluss_array( values .as_primitive_opt::() .ok_or_else(|| IllegalArgument { - message: format!("Expected Decimal128Array for {element_type:?} element"), + message: format!("expected Decimal128Array for {element_type:?} element"), })?; let arrow_scale = match values.data_type() { ArrowDataType::Decimal128(_p, s) => *s as i64, other => { return Err(IllegalArgument { message: format!( - "Expected Decimal128 data type for {element_type:?} element, got {other:?}" + "expected Decimal128 data type for {element_type:?} element, got {other:?}" ), }); } @@ -774,7 +816,7 @@ fn write_arrow_values_to_fluss_array( if arr.is_null(i) { writer.set_null_at(i); } else { - let d = crate::row::Decimal::from_arrow_decimal128( + let d = Decimal::from_arrow_decimal128( arr.value(i), arrow_scale, precision, @@ -788,7 +830,7 @@ fn write_arrow_values_to_fluss_array( let arr = values .as_primitive_opt::() .ok_or_else(|| IllegalArgument { - message: format!("Expected Date32Array for {element_type:?} element"), + message: format!("expected Date32Array for {element_type:?} element"), })?; for i in 0..len { if arr.is_null(i) { @@ -829,16 +871,35 @@ fn write_arrow_values_to_fluss_array( } else { return Err(IllegalArgument { message: format!( - "Expected ListArray for {element_type:?} element, got {:?}", + "expected ListArray for {element_type:?} element, got {:?}", values.data_type() ), }); } } + DataType::Row(row_type) => { + let struct_arr = values + .as_any() + .downcast_ref::() + .ok_or_else(|| IllegalArgument { + message: format!( + "expected StructArray for {element_type:?} element, got {:?}", + values.data_type() + ), + })?; + for i in 0..len { + if struct_arr.is_null(i) { + writer.set_null_at(i); + } else { + let nested = extract_struct_from_array(struct_arr, i, Some(row_type))?; + writer.write_row(i, &nested)?; + } + } + } _ => { return Err(IllegalArgument { message: format!( - "Unsupported element type for Arrow → FlussArray conversion: {element_type:?}" + "unsupported element type for Arrow → FlussArray conversion: {element_type:?}" ), }); } @@ -848,8 +909,8 @@ fn write_arrow_values_to_fluss_array( fn write_time_elements( values: &dyn Array, - element_type: &crate::metadata::DataType, - writer: &mut crate::row::binary_array::FlussArrayWriter, + element_type: &DataType, + writer: &mut FlussArrayWriter, ) -> Result<()> { macro_rules! process_time { ($arrow_type:ty, $to_millis:expr) => {{ @@ -857,7 +918,7 @@ fn write_time_elements( .as_primitive_opt::<$arrow_type>() .ok_or_else(|| IllegalArgument { message: format!( - "Expected {} for {:?} element", + "expected {} for {:?} element", stringify!($arrow_type), element_type ), @@ -889,7 +950,7 @@ fn write_time_elements( other => { return Err(IllegalArgument { message: format!( - "Expected Time column for {element_type:?} element, got {other:?}" + "expected Time column for {element_type:?} element, got {other:?}" ), }); } @@ -916,19 +977,19 @@ fn convert_timestamp_raw(raw: i64, unit: &TimeUnit) -> (i64, i32) { fn write_timestamp_elements( values: &dyn Array, - element_type: &crate::metadata::DataType, - writer: &mut crate::row::binary_array::FlussArrayWriter, + element_type: &DataType, + writer: &mut FlussArrayWriter, precision: u32, construct_compact: impl Fn(i64) -> T, construct_with_nanos: impl Fn(i64, i32) -> Result, - write_fn: impl Fn(&mut crate::row::binary_array::FlussArrayWriter, usize, T, u32), + write_fn: impl Fn(&mut FlussArrayWriter, usize, T, u32), ) -> Result<()> { let unit = match values.data_type() { ArrowDataType::Timestamp(unit, _) => unit, other => { return Err(IllegalArgument { message: format!( - "Expected Timestamp column for {element_type:?} element, got {other:?}" + "expected Timestamp column for {element_type:?} element, got {other:?}" ), }); } @@ -940,7 +1001,7 @@ fn write_timestamp_elements( .as_primitive_opt::<$arrow_type>() .ok_or_else(|| IllegalArgument { message: format!( - "Expected {} for {:?} element", + "expected {} for {:?} element", stringify!($arrow_type), element_type ), @@ -983,7 +1044,7 @@ mod tests { fn single_column_row(array: ArrayRef) -> ColumnarRow { let batch = RecordBatch::try_from_iter(vec![("arr", array)]).expect("record batch with one column"); - ColumnarRow::new(Arc::new(batch)) + ColumnarRow::new(Arc::new(batch), 0, None) } #[test] @@ -1018,7 +1079,7 @@ mod tests { ) .expect("record batch"); - let mut row = ColumnarRow::new(Arc::new(batch)); + let mut row = ColumnarRow::new(Arc::new(batch), 0, None); assert_eq!(row.get_field_count(), 10); assert!(row.get_boolean(0).unwrap()); assert_eq!(row.get_byte(1).unwrap(), 1); @@ -1073,27 +1134,21 @@ mod tests { ) .expect("record batch"); - let row = ColumnarRow::new(Arc::new(batch)); + let row = ColumnarRow::new(Arc::new(batch), 0, None); assert_eq!(row.get_field_count(), 3); // Verify decimal values assert_eq!( row.get_decimal(0, 10, 2).unwrap(), - crate::row::Decimal::from_big_decimal(BigDecimal::new(BigInt::from(12345), 2), 10, 2) - .unwrap() + Decimal::from_big_decimal(BigDecimal::new(BigInt::from(12345), 2), 10, 2).unwrap() ); assert_eq!( row.get_decimal(1, 20, 5).unwrap(), - crate::row::Decimal::from_big_decimal( - BigDecimal::new(BigInt::from(1234567890), 5), - 20, - 5 - ) - .unwrap() + Decimal::from_big_decimal(BigDecimal::new(BigInt::from(1234567890), 5), 20, 5).unwrap() ); assert_eq!( row.get_decimal(2, 38, 10).unwrap(), - crate::row::Decimal::from_big_decimal( + Decimal::from_big_decimal( BigDecimal::new(BigInt::from(999999999999999999i128), 10), 38, 10 @@ -1206,15 +1261,11 @@ mod tests { DataType::Struct(child_fields), false, )])); - Arc::new( - RecordBatch::try_new(schema, vec![Arc::new(struct_array)]) - .expect("record batch"), - ) + Arc::new(RecordBatch::try_new(schema, vec![Arc::new(struct_array)]).expect("record batch")) } #[test] fn columnar_row_reads_nested_row() { - // Build a RecordBatch with a Struct column: {i32, string} let child_fields = Fields::from(vec![ Field::new("x", DataType::Int32, false), Field::new("s", DataType::Utf8, false), @@ -1225,7 +1276,7 @@ mod tests { ]; let batch = make_struct_batch("nested", child_fields, child_arrays, 2); - let mut row = ColumnarRow::new(batch); + let mut row = ColumnarRow::new(batch, 0, None); // row_id = 0 let nested = row.get_row(0).unwrap(); @@ -1268,29 +1319,32 @@ mod tests { DataType::Struct(outer_fields), false, )])); - let batch = Arc::new( - RecordBatch::try_new(schema, vec![outer_array]).expect("record batch"), - ); + let batch = + Arc::new(RecordBatch::try_new(schema, vec![outer_array]).expect("record batch")); - let row = ColumnarRow::new(batch); + let mut row = ColumnarRow::new(batch, 0, None); - // Access outer struct at column 0, row 0 + // row_id = 0 let outer = row.get_row(0).unwrap(); assert_eq!(outer.get_int(0).unwrap(), 1); - - // Access inner struct (column 1 of outer) let inner = outer.get_row(1).unwrap(); assert_eq!(inner.get_string(0).unwrap(), "deep"); + + // row_id = 1 + row.set_row_id(1); + let outer = row.get_row(0).unwrap(); + assert_eq!(outer.get_int(0).unwrap(), 2); + let inner = outer.get_row(1).unwrap(); + assert_eq!(inner.get_string(0).unwrap(), "deeper"); } #[test] fn columnar_row_get_row_cache_invalidated_on_set_row_id() { let child_fields = Fields::from(vec![Field::new("x", DataType::Int32, false)]); - let child_arrays: Vec> = - vec![Arc::new(Int32Array::from(vec![10, 20]))]; + let child_arrays: Vec> = vec![Arc::new(Int32Array::from(vec![10, 20]))]; let batch = make_struct_batch("s", child_fields, child_arrays, 2); - let mut row = ColumnarRow::new(batch); + let mut row = ColumnarRow::new(batch, 0, None); // row_id = 0: nested x = 10 let nested_0 = row.get_row(0).unwrap(); diff --git a/crates/fluss/src/row/column_writer.rs b/crates/fluss/src/row/column_writer.rs index be94cf06..d595df91 100644 --- a/crates/fluss/src/row/column_writer.rs +++ b/crates/fluss/src/row/column_writer.rs @@ -21,7 +21,7 @@ use crate::error::Error::RowConvertError; use crate::error::{Error, Result}; -use crate::metadata::DataType; +use crate::metadata::{DataType, RowType}; use crate::row::InternalRow; use crate::row::datum::{ MICROS_PER_MILLI, MILLIS_PER_SECOND, NANOS_PER_MILLI, append_decimal_to_builder, @@ -123,6 +123,12 @@ enum TypedWriter { offsets: Vec, validity: Vec, }, + Struct { + field_writers: Vec, + validity: Vec, + fields: arrow_schema::Fields, + row_type: RowType, + }, } /// Dispatch to the inner builder across all `TypedWriter` variants. @@ -156,6 +162,7 @@ macro_rules! with_builder { TypedWriter::TimestampLtzMicrosecond { builder: $b, .. } => $body, TypedWriter::TimestampLtzNanosecond { builder: $b, .. } => $body, TypedWriter::List { .. } => panic!("List variant not supported in with_builder!"), + TypedWriter::Struct { .. } => panic!("Struct variant not supported in with_builder!"), } }; } @@ -354,6 +361,39 @@ impl ColumnWriter { validity: Vec::with_capacity(capacity), } } + DataType::Row(row_type) => { + let arrow_fields = match arrow_type { + ArrowDataType::Struct(fields) => fields.clone(), + _ => { + return Err(Error::IllegalArgument { + message: format!( + "Expected Struct Arrow type for Row, got: {arrow_type:?}" + ), + }); + } + }; + if arrow_fields.len() != row_type.fields().len() { + return Err(Error::IllegalArgument { + message: format!( + "Row arity mismatch: Fluss type has {} fields, Arrow type has {}", + row_type.fields().len(), + arrow_fields.len(), + ), + }); + } + let field_writers: Result> = row_type + .fields() + .iter() + .zip(arrow_fields.iter()) + .map(|(f, af)| ColumnWriter::create(&f.data_type, af.data_type(), 0, capacity)) + .collect(); + TypedWriter::Struct { + field_writers: field_writers?, + validity: Vec::with_capacity(capacity), + fields: arrow_fields, + row_type: row_type.clone(), + } + } _ => { return Err(Error::IllegalArgument { message: format!("Unsupported Fluss DataType: {fluss_type:?}"), @@ -400,6 +440,17 @@ impl ColumnWriter { let taken_validity = std::mem::take(validity); finish_list_array(values, item_nullable, &taken_offsets, &taken_validity) } + TypedWriter::Struct { + field_writers, + validity, + fields, + .. + } => { + let taken_validity = std::mem::take(validity); + let child_arrays: Vec = + field_writers.iter_mut().map(|w| w.finish()).collect(); + finish_struct_array(fields.clone(), child_arrays, &taken_validity) + } _ => with_builder!(&mut self.inner, b => (b as &mut dyn ArrayBuilder).finish()), } } @@ -476,6 +527,15 @@ impl ColumnWriter { let offsets_bytes = round_up_to_8(offsets.len() * std::mem::size_of::()); validity_bytes + offsets_bytes + element_writer.buffer_size() } + TypedWriter::Struct { + field_writers, + validity, + .. + } => { + let validity_bytes = round_up_to_8(validity.len().div_ceil(8)); + let children_bytes: usize = field_writers.iter().map(|w| w.buffer_size()).sum(); + validity_bytes + children_bytes + } } } @@ -488,6 +548,17 @@ impl ColumnWriter { offsets.push(last); validity.push(false); } + TypedWriter::Struct { + field_writers, + validity, + .. + } => { + // Arrow StructArray children must match parent length. + for child in field_writers.iter_mut() { + child.append_null(); + } + validity.push(false); + } _ => with_builder!(&mut self.inner, b => b.append_null()), } } @@ -676,22 +747,75 @@ impl ColumnWriter { validity, } => { let array = row.get_array(pos)?; - for i in 0..array.size() { - element_writer.write_field_at(&array, i)?; + let size = array.size(); + if let TypedWriter::Struct { + field_writers, + validity: child_validity, + row_type, + .. + } = &mut element_writer.inner + { + for i in 0..size { + if array.is_null_at(i) { + for child in field_writers.iter_mut() { + child.append_null(); + } + child_validity.push(false); + } else { + let nested = array.get_row(i, row_type)?; + for (j, child) in field_writers.iter_mut().enumerate() { + child.write_field_at(&nested, j)?; + } + child_validity.push(true); + } + } + } else { + for i in 0..size { + element_writer.write_field_at(&array, i)?; + } } let last = *offsets.last().unwrap(); offsets.push( - last + i32::try_from(array.size()).map_err(|_| RowConvertError { - message: format!("Array size {} exceeds i32 range", array.size()), + last + i32::try_from(size).map_err(|_| RowConvertError { + message: format!("Array size {size} exceeds i32 range"), })?, ); validity.push(true); Ok(()) } + TypedWriter::Struct { + field_writers, + validity, + .. + } => { + let nested = row.get_row(pos)?; + for (i, child) in field_writers.iter_mut().enumerate() { + child.write_field_at(nested, i)?; + } + validity.push(true); + Ok(()) + } } } } +fn finish_struct_array( + fields: arrow_schema::Fields, + child_arrays: Vec, + validity: &[bool], +) -> ArrayRef { + use arrow::array::StructArray; + use arrow::buffer::NullBuffer; + use std::sync::Arc; + + let null_buffer = if validity.iter().any(|v| !v) { + Some(NullBuffer::from(validity.to_vec())) + } else { + None + }; + Arc::new(StructArray::new(fields, child_arrays, null_buffer)) +} + fn finish_list_array( values: ArrayRef, item_nullable: bool, @@ -726,7 +850,7 @@ mod tests { use crate::metadata::DataTypes; use crate::record::to_arrow_type; use crate::row::binary_array::FlussArrayWriter; - use crate::row::{Date, Datum, GenericRow, Time, TimestampLtz, TimestampNtz}; + use crate::row::{Date, Datum, Decimal, GenericRow, Time, TimestampLtz, TimestampNtz}; use arrow::array::*; use bigdecimal::BigDecimal; use std::str::FromStr; @@ -857,8 +981,7 @@ mod tests { // Decimal let decimal = - crate::row::Decimal::from_big_decimal(BigDecimal::from_str("123.45").unwrap(), 10, 2) - .unwrap(); + Decimal::from_big_decimal(BigDecimal::from_str("123.45").unwrap(), 10, 2).unwrap(); let arr = write_one(&DataTypes::decimal(10, 2), Datum::Decimal(decimal)); assert_eq!( arr.as_any() diff --git a/crates/fluss/src/row/compacted/compacted_key_writer.rs b/crates/fluss/src/row/compacted/compacted_key_writer.rs index c6940652..9e0ffa53 100644 --- a/crates/fluss/src/row/compacted/compacted_key_writer.rs +++ b/crates/fluss/src/row/compacted/compacted_key_writer.rs @@ -18,10 +18,12 @@ use crate::row::compacted::compacted_row_writer::CompactedRowWriter; use bytes::Bytes; +use crate::error::Error::IllegalArgument; use crate::error::Result; use crate::metadata::DataType; use crate::row::Decimal; use crate::row::binary::{BinaryRowFormat, BinaryWriter, ValueWriter}; +use crate::row::datum::{TimestampLtz, TimestampNtz}; use delegate::delegate; /// A wrapping of [`CompactedRowWriter`] used to encode key columns. @@ -47,12 +49,8 @@ impl CompactedKeyWriter { } pub fn create_value_writer(field_type: &DataType) -> Result { - // Java's CompactedKeyEncoder allows encoding Array types (Map/Row - // are not yet supported by ValueWriter). The server rejects - // unsupported key types at table-creation time, so encoding is - // allowed here to match Java parity. - if matches!(field_type, DataType::Map(_) | DataType::Row(_)) { - return Err(crate::error::Error::IllegalArgument { + if matches!(field_type, DataType::Map(_)) { + return Err(IllegalArgument { message: format!("Cannot use {field_type:?} as a key column type"), }); } @@ -107,9 +105,9 @@ impl BinaryWriter for CompactedKeyWriter { fn write_time(&mut self, value: i32, precision: u32); - fn write_timestamp_ntz(&mut self, value: &crate::row::datum::TimestampNtz, precision: u32); + fn write_timestamp_ntz(&mut self, value: &TimestampNtz, precision: u32); - fn write_timestamp_ltz(&mut self, value: &crate::row::datum::TimestampLtz, precision: u32); + fn write_timestamp_ltz(&mut self, value: &TimestampLtz, precision: u32); fn write_array(&mut self, value: &[u8]); } diff --git a/crates/fluss/src/row/compacted/compacted_row.rs b/crates/fluss/src/row/compacted/compacted_row.rs index 24e4bd79..fbf47c58 100644 --- a/crates/fluss/src/row/compacted/compacted_row.rs +++ b/crates/fluss/src/row/compacted/compacted_row.rs @@ -16,13 +16,18 @@ // under the License. use crate::client::WriteFormat; +use crate::error::Error::IllegalArgument; use crate::error::Result; use crate::metadata::RowType; use crate::row::compacted::compacted_row_reader::{CompactedRowDeserializer, CompactedRowReader}; use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; -use crate::row::{Decimal, GenericRow, InternalRow}; +use crate::row::{Decimal, FlussArray, GenericRow, InternalRow}; use std::sync::{Arc, OnceLock}; +pub fn calculate_bit_set_width_in_bytes(arity: usize) -> usize { + arity.div_ceil(8) +} + // Reference implementation: // https://github.com/apache/fluss/blob/main/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java #[allow(dead_code)] @@ -35,10 +40,6 @@ pub struct CompactedRow<'a> { data: &'a [u8], } -pub fn calculate_bit_set_width_in_bytes(arity: usize) -> usize { - arity.div_ceil(8) -} - #[allow(dead_code)] impl<'a> CompactedRow<'a> { pub fn from_bytes(row_type: &'a RowType, data: &'a [u8]) -> Self { @@ -93,7 +94,7 @@ impl<'a> InternalRow for CompactedRow<'a> { fn is_null_at(&self, pos: usize) -> Result { let fields = self.deserializer.get_row_type().fields(); if pos >= fields.len() { - return Err(crate::error::Error::IllegalArgument { + return Err(IllegalArgument { message: format!( "position {pos} out of bounds (row has {} fields)", fields.len() @@ -167,12 +168,12 @@ impl<'a> InternalRow for CompactedRow<'a> { self.decoded_row()?.get_bytes(pos) } - fn get_array(&self, pos: usize) -> Result { + fn get_array(&self, pos: usize) -> Result { self.decoded_row()?.get_array(pos) } fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { - self.decoded_row().get_row(pos) + self.decoded_row()?.get_row(pos) } fn as_encoded_bytes(&self, write_format: WriteFormat) -> Option<&[u8]> { @@ -187,13 +188,14 @@ impl<'a> InternalRow for CompactedRow<'a> { #[cfg(test)] mod tests { use super::*; - use crate::row::binary::BinaryWriter; - use crate::metadata::{ - BigIntType, BooleanType, BytesType, DataType, DoubleType, FloatType, IntType, SmallIntType, - StringType, TinyIntType, + BigIntType, BooleanType, BytesType, DataType, DataTypes, DecimalType, DoubleType, + FloatType, IntType, SmallIntType, StringType, TimestampLTzType, TimestampType, TinyIntType, }; + use crate::row::binary::BinaryWriter; + use crate::row::binary_array::FlussArrayWriter; use crate::row::compacted::compacted_row_writer::CompactedRowWriter; + use crate::row::datum::{TimestampLtz, TimestampNtz}; #[test] fn test_compacted_row() { @@ -263,9 +265,6 @@ mod tests { #[test] fn test_compacted_row_temporal_and_decimal_types() { // Comprehensive test covering DATE, TIME, TIMESTAMP (compact/non-compact), and DECIMAL (compact/non-compact) - use crate::metadata::{DataTypes, DecimalType, TimestampLTzType, TimestampType}; - use crate::row::Decimal; - use crate::row::datum::{TimestampLtz, TimestampNtz}; use bigdecimal::{BigDecimal, num_bigint::BigInt}; let row_type = RowType::with_data_types(vec![ @@ -345,9 +344,6 @@ mod tests { #[test] fn test_compacted_row_int_array() { - use crate::metadata::DataTypes; - use crate::row::binary_array::FlussArrayWriter; - let row_type = RowType::with_data_types(vec![DataTypes::int(), DataTypes::array(DataTypes::int())]); @@ -375,9 +371,6 @@ mod tests { #[test] fn test_compacted_row_string_array() { - use crate::metadata::DataTypes; - use crate::row::binary_array::FlussArrayWriter; - let row_type = RowType::with_data_types(vec![DataTypes::array(DataTypes::string())]); let mut writer = CompactedRowWriter::new(row_type.fields().len()); @@ -402,9 +395,6 @@ mod tests { #[test] fn test_compacted_row_array_with_nulls() { - use crate::metadata::DataTypes; - use crate::row::binary_array::FlussArrayWriter; - let row_type = RowType::with_data_types(vec![DataTypes::array(DataTypes::int())]); let mut writer = CompactedRowWriter::new(row_type.fields().len()); @@ -431,9 +421,6 @@ mod tests { #[test] fn test_compacted_row_empty_array() { - use crate::metadata::DataTypes; - use crate::row::binary_array::FlussArrayWriter; - let row_type = RowType::with_data_types(vec![DataTypes::array(DataTypes::int())]); let mut writer = CompactedRowWriter::new(row_type.fields().len()); @@ -452,9 +439,6 @@ mod tests { #[test] fn test_compacted_row_nested_array() { - use crate::metadata::DataTypes; - use crate::row::binary_array::FlussArrayWriter; - let row_type = RowType::with_data_types(vec![DataTypes::array(DataTypes::array(DataTypes::int()))]); diff --git a/crates/fluss/src/row/compacted/compacted_row_reader.rs b/crates/fluss/src/row/compacted/compacted_row_reader.rs index 9d7fb6cb..efcb39f2 100644 --- a/crates/fluss/src/row/compacted/compacted_row_reader.rs +++ b/crates/fluss/src/row/compacted/compacted_row_reader.rs @@ -15,34 +15,59 @@ // specific language governing permissions and limitations // under the License. -use crate::metadata::RowType; +use crate::error::Error::IllegalArgument; +use crate::error::Result; +use crate::metadata::{DataType, RowType}; use crate::row::compacted::compacted_row::calculate_bit_set_width_in_bytes; -use crate::{ - error::{Error::IllegalArgument, Result}, - metadata::DataType, - row::{Datum, Decimal, GenericRow, compacted::compacted_row_writer::CompactedRowWriter}, - util::varint::{read_unsigned_varint_at, read_unsigned_varint_u64_at}, -}; +use crate::row::compacted::compacted_row_writer::CompactedRowWriter; +use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; +use crate::row::{Datum, Decimal, FlussArray, GenericRow}; +use crate::util::varint::{read_unsigned_varint_at, read_unsigned_varint_u64_at}; use std::borrow::Cow; use std::str::from_utf8; +use std::sync::Arc; #[allow(dead_code)] #[derive(Clone)] pub struct CompactedRowDeserializer<'a> { row_type: Cow<'a, RowType>, + // Index-parallel to row_type.fields(); Some(_) only for ROW-typed fields. + nested: Vec>>>, +} + +fn build_nested_deserializers<'a>( + row_type: &RowType, +) -> Vec>>> { + row_type + .fields() + .iter() + .map(|f| { + if let DataType::Row(inner) = &f.data_type { + Some(Arc::new(CompactedRowDeserializer::new_from_owned( + inner.clone(), + ))) + } else { + None + } + }) + .collect() } #[allow(dead_code)] impl<'a> CompactedRowDeserializer<'a> { pub fn new(row_type: &'a RowType) -> Self { + let nested = build_nested_deserializers(row_type); Self { row_type: Cow::Borrowed(row_type), + nested, } } pub fn new_from_owned(row_type: RowType) -> Self { + let nested = build_nested_deserializers(&row_type); Self { row_type: Cow::Owned(row_type), + nested, } } @@ -127,65 +152,52 @@ impl<'a> CompactedRowDeserializer<'a> { } DataType::Date(_) => { let (val, next) = reader.read_int(cursor)?; - (Datum::Date(crate::row::datum::Date::new(val)), next) + (Datum::Date(Date::new(val)), next) } DataType::Time(_) => { let (val, next) = reader.read_int(cursor)?; - (Datum::Time(crate::row::datum::Time::new(val)), next) + (Datum::Time(Time::new(val)), next) } DataType::Timestamp(timestamp_type) => { let precision = timestamp_type.precision(); - if crate::row::datum::TimestampNtz::is_compact(precision) { - // Compact: only milliseconds + if TimestampNtz::is_compact(precision) { let (millis, next) = reader.read_long(cursor)?; - ( - Datum::TimestampNtz(crate::row::datum::TimestampNtz::new(millis)), - next, - ) + (Datum::TimestampNtz(TimestampNtz::new(millis)), next) } else { - // Non-compact: milliseconds + nanos let (millis, mid) = reader.read_long(cursor)?; let (nanos, next) = reader.read_int(mid)?; - let timestamp = crate::row::datum::TimestampNtz::from_millis_nanos( - millis, nanos, - ) - .map_err(|e| IllegalArgument { - message: format!( - "Invalid nano_of_millisecond value in compacted row timestamp: {e}" - ), - })?; + let timestamp = TimestampNtz::from_millis_nanos(millis, nanos).map_err( + |e| IllegalArgument { + message: format!( + "Invalid nano_of_millisecond value in compacted row timestamp: {e}" + ), + }, + )?; (Datum::TimestampNtz(timestamp), next) } } DataType::TimestampLTz(timestamp_ltz_type) => { let precision = timestamp_ltz_type.precision(); - if crate::row::datum::TimestampLtz::is_compact(precision) { - // Compact: only epoch milliseconds + if TimestampLtz::is_compact(precision) { let (epoch_millis, next) = reader.read_long(cursor)?; - ( - Datum::TimestampLtz(crate::row::datum::TimestampLtz::new(epoch_millis)), - next, - ) + (Datum::TimestampLtz(TimestampLtz::new(epoch_millis)), next) } else { - // Non-compact: epoch milliseconds + nanos let (epoch_millis, mid) = reader.read_long(cursor)?; let (nanos, next) = reader.read_int(mid)?; - let timestamp_ltz = crate::row::datum::TimestampLtz::from_millis_nanos( - epoch_millis, - nanos, - ) - .map_err(|e| IllegalArgument { - message: format!( - "Invalid nano_of_millisecond value in compacted row timestamp_ltz: {e}" - ), - })?; + let timestamp_ltz = + TimestampLtz::from_millis_nanos(epoch_millis, nanos).map_err(|e| { + IllegalArgument { + message: format!( + "Invalid nano_of_millisecond value in compacted row timestamp_ltz: {e}" + ), + } + })?; (Datum::TimestampLtz(timestamp_ltz), next) } } DataType::Array(_) => { let (bytes, next) = reader.read_bytes(cursor)?; - let array = crate::row::binary_array::FlussArray::from_bytes(bytes)?; - (Datum::Array(array), next) + (Datum::Array(FlussArray::from_bytes(bytes)?), next) } DataType::Row(row_type) => { let (nested_bytes, next) = reader.read_bytes(cursor)?; @@ -195,7 +207,9 @@ impl<'a> CompactedRowDeserializer<'a> { 0, nested_bytes.len(), ); - let nested_deser = CompactedRowDeserializer::new_from_owned(row_type.clone()); + let nested_deser = self.nested[col_pos] + .as_ref() + .expect("ROW field must have nested deserializer"); let nested_row = nested_deser.deserialize(&nested_reader)?; (Datum::Row(Box::new(nested_row)), next) } @@ -342,9 +356,12 @@ impl<'a> CompactedRowReader<'a> { #[cfg(test)] mod row_type_tests { use crate::metadata::{DataType, DataTypes, RowType}; - use crate::row::compacted::compacted_row_reader::{CompactedRowDeserializer, CompactedRowReader}; - use crate::row::compacted::compacted_row_writer::CompactedRowWriter; use crate::row::binary::ValueWriter; + use crate::row::compacted::compacted_row_reader::{ + CompactedRowDeserializer, CompactedRowReader, + }; + use crate::row::compacted::compacted_row_writer::CompactedRowWriter; + use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; use crate::row::field_getter::FieldGetter; use crate::row::{Datum, GenericRow, InternalRow}; @@ -374,13 +391,12 @@ mod row_type_tests { 0, bytes.len(), ); - let result = deser.deserialize(&reader); + let result = deser.deserialize(&reader).expect("deserialize"); verify(&result); } #[test] fn test_row_simple_nesting() { - // ROW nested inside an outer row let inner_row_type = RowType::with_data_types_and_field_names( vec![DataTypes::int(), DataTypes::string()], vec!["x", "label"], @@ -408,11 +424,8 @@ mod row_type_tests { #[test] fn test_row_deep_nesting() { - // ROW> — two levels of nesting - let inner_inner_row_type = RowType::with_data_types_and_field_names( - vec![DataTypes::int()], - vec!["n"], - ); + let inner_inner_row_type = + RowType::with_data_types_and_field_names(vec![DataTypes::int()], vec!["n"]); let inner_row_type = RowType::with_data_types_and_field_names( vec![DataType::Row(inner_inner_row_type.clone())], vec!["inner"], @@ -476,4 +489,79 @@ mod row_type_tests { assert!(result2.is_null_at(1).unwrap()); }); } + + #[test] + fn test_row_all_primitives_round_trip() { + let inner_row_type = RowType::with_data_types_and_field_names( + vec![ + DataTypes::boolean(), + DataTypes::tinyint(), + DataTypes::smallint(), + DataTypes::int(), + DataTypes::bigint(), + DataTypes::float(), + DataTypes::double(), + DataTypes::string(), + DataTypes::bytes(), + DataTypes::date(), + DataTypes::time(), + DataTypes::timestamp(), + DataTypes::timestamp_ltz(), + ], + vec![ + "b", "tin", "sm", "i", "lo", "fl", "db", "str", "by", "dt", "ti", "tsn", "tsl", + ], + ); + let outer_row_type = RowType::with_data_types_and_field_names( + vec![DataType::Row(inner_row_type.clone())], + vec!["nested"], + ); + + let mut inner = GenericRow::new(13); + inner.set_field(0, true); + inner.set_field(1, 7_i8); + inner.set_field(2, -42_i16); + inner.set_field(3, 100_000_i32); + inner.set_field(4, 9_876_543_210_i64); + inner.set_field(5, std::f32::consts::PI); + inner.set_field(6, std::f64::consts::E); + inner.set_field(7, "hello world"); + inner.set_field(8, b"binary".as_slice()); + inner.set_field(9, Datum::Date(Date::new(20476))); + inner.set_field(10, Datum::Time(Time::new(36_827_123))); + inner.set_field( + 11, + Datum::TimestampNtz(TimestampNtz::new(1_769_163_227_123)), + ); + inner.set_field( + 12, + Datum::TimestampLtz(TimestampLtz::new(1_769_163_227_123)), + ); + + let mut outer = GenericRow::new(1); + outer.set_field(0, Datum::Row(Box::new(inner))); + + round_trip(&outer_row_type, &outer, |result| { + let n = result.get_row(0).unwrap(); + assert!(n.get_boolean(0).unwrap()); + assert_eq!(n.get_byte(1).unwrap(), 7); + assert_eq!(n.get_short(2).unwrap(), -42); + assert_eq!(n.get_int(3).unwrap(), 100_000); + assert_eq!(n.get_long(4).unwrap(), 9_876_543_210); + assert!((n.get_float(5).unwrap() - std::f32::consts::PI).abs() < f32::EPSILON); + assert!((n.get_double(6).unwrap() - std::f64::consts::E).abs() < f64::EPSILON); + assert_eq!(n.get_string(7).unwrap(), "hello world"); + assert_eq!(n.get_bytes(8).unwrap(), b"binary"); + assert_eq!(n.get_date(9).unwrap().get_inner(), 20476); + assert_eq!(n.get_time(10).unwrap().get_inner(), 36_827_123); + assert_eq!( + n.get_timestamp_ntz(11, 6).unwrap().get_millisecond(), + 1_769_163_227_123, + ); + assert_eq!( + n.get_timestamp_ltz(12, 6).unwrap().get_epoch_millisecond(), + 1_769_163_227_123, + ); + }); + } } diff --git a/crates/fluss/src/row/compacted/mod.rs b/crates/fluss/src/row/compacted/mod.rs index 33610783..fa603d23 100644 --- a/crates/fluss/src/row/compacted/mod.rs +++ b/crates/fluss/src/row/compacted/mod.rs @@ -23,7 +23,7 @@ mod compacted_row_writer; pub use compacted_key_writer::CompactedKeyWriter; #[allow(unused_imports)] -pub use compacted_row::CompactedRow; +pub use compacted_row::{CompactedRow, calculate_bit_set_width_in_bytes}; #[allow(unused_imports)] pub use compacted_row_reader::{CompactedRowDeserializer, CompactedRowReader}; #[allow(unused_imports)] diff --git a/crates/fluss/src/row/datum.rs b/crates/fluss/src/row/datum.rs index 865d4b4f..d3f19a1e 100644 --- a/crates/fluss/src/row/datum.rs +++ b/crates/fluss/src/row/datum.rs @@ -15,18 +15,21 @@ // specific language governing permissions and limitations // under the License. -use crate::error::Error::RowConvertError; +use crate::error::Error::{IllegalArgument, RowConvertError}; use crate::error::Result; +use crate::metadata::{DataType, RowType}; use crate::row::Decimal; use crate::row::GenericRow; +use crate::row::InternalRow; use crate::row::binary_array::FlussArray; +use crate::row::field_getter::FieldGetter; use arrow::array::{ ArrayBuilder, BinaryBuilder, BooleanBuilder, Date32Builder, Decimal128Builder, FixedSizeBinaryBuilder, Float32Builder, Float64Builder, Int8Builder, Int16Builder, - Int32Builder, Int64Builder, ListBuilder, StringBuilder, Time32MillisecondBuilder, - Time32SecondBuilder, Time64MicrosecondBuilder, Time64NanosecondBuilder, - TimestampMicrosecondBuilder, TimestampMillisecondBuilder, TimestampNanosecondBuilder, - TimestampSecondBuilder, + Int32Builder, Int64Builder, ListBuilder, StringBuilder, StructBuilder, + Time32MillisecondBuilder, Time32SecondBuilder, Time64MicrosecondBuilder, + Time64NanosecondBuilder, TimestampMicrosecondBuilder, TimestampMillisecondBuilder, + TimestampNanosecondBuilder, TimestampSecondBuilder, }; use arrow::datatypes as arrow_schema; use arrow::error::ArrowError; @@ -146,6 +149,30 @@ impl Datum<'_> { } } +impl<'a> Datum<'a> { + pub fn into_owned(self) -> Datum<'static> { + match self { + Datum::Null => Datum::Null, + Datum::Bool(v) => Datum::Bool(v), + Datum::Int8(v) => Datum::Int8(v), + Datum::Int16(v) => Datum::Int16(v), + Datum::Int32(v) => Datum::Int32(v), + Datum::Int64(v) => Datum::Int64(v), + Datum::Float32(v) => Datum::Float32(v), + Datum::Float64(v) => Datum::Float64(v), + Datum::String(s) => Datum::String(Cow::Owned(s.into_owned())), + Datum::Blob(b) => Datum::Blob(Cow::Owned(b.into_owned())), + Datum::Decimal(d) => Datum::Decimal(d), + Datum::Date(d) => Datum::Date(d), + Datum::Time(t) => Datum::Time(t), + Datum::TimestampNtz(t) => Datum::TimestampNtz(t), + Datum::TimestampLtz(t) => Datum::TimestampLtz(t), + Datum::Array(a) => Datum::Array(a), + Datum::Row(boxed) => Datum::Row(Box::new(boxed.into_owned())), + } + } +} + // ----------- implement from impl<'a> From for Datum<'a> { #[inline] @@ -563,42 +590,29 @@ fn append_fluss_array_to_list_builder( fn read_datum_from_fluss_array<'a>( arr: &FlussArray, pos: usize, - element_type: &crate::metadata::DataType, + element_type: &DataType, ) -> Result> { - use crate::metadata::DataType; - - Ok(match element_type { - DataType::Boolean(_) => Datum::Bool(arr.get_boolean(pos)?), - DataType::TinyInt(_) => Datum::Int8(arr.get_byte(pos)?), - DataType::SmallInt(_) => Datum::Int16(arr.get_short(pos)?), - DataType::Int(_) => Datum::Int32(arr.get_int(pos)?), - DataType::BigInt(_) => Datum::Int64(arr.get_long(pos)?), - DataType::Float(_) => Datum::Float32(arr.get_float(pos)?.into()), - DataType::Double(_) => Datum::Float64(arr.get_double(pos)?.into()), - DataType::Char(_) | DataType::String(_) => { - Datum::String(Cow::Owned(arr.get_string(pos)?.to_string())) - } - DataType::Binary(_) | DataType::Bytes(_) => { - Datum::Blob(Cow::Owned(arr.get_binary(pos)?.to_vec())) - } - DataType::Decimal(dt) => { - Datum::Decimal(arr.get_decimal(pos, dt.precision(), dt.scale())?) - } - DataType::Date(_) => Datum::Date(arr.get_date(pos)?), - DataType::Time(_) => Datum::Time(arr.get_time(pos)?), - DataType::Timestamp(t) => Datum::TimestampNtz(arr.get_timestamp_ntz(pos, t.precision())?), - DataType::TimestampLTz(t) => { - Datum::TimestampLtz(arr.get_timestamp_ltz(pos, t.precision())?) - } - DataType::Array(_) => Datum::Array(arr.get_array(pos)?), - _ => { - return Err(RowConvertError { - message: format!( - "Unsupported element type for FlussArray → Arrow conversion: {element_type:?}" - ), - }); - } - }) + if let DataType::Row(row_type) = element_type { + let compacted = arr.get_row(pos, row_type)?; + return Ok(Datum::Row(Box::new(internal_row_to_owned_generic( + &compacted, row_type, + )?))); + } + + let getter = FieldGetter::create(element_type, pos); + Ok(getter.get_field(arr)?.into_owned()) +} + +fn internal_row_to_owned_generic( + row: &dyn InternalRow, + row_type: &RowType, +) -> Result> { + let mut owned = GenericRow::new(row_type.fields().len()); + for (i, field) in row_type.fields().iter().enumerate() { + let getter = FieldGetter::create(field.data_type(), i); + owned.set_field(i, getter.get_field(row)?.into_owned()); + } + Ok(owned) } fn append_null_for_type( @@ -661,12 +675,74 @@ fn append_null_for_type( arrow_schema::DataType::List(_) => { downcast_null!(ListBuilder>) } + arrow_schema::DataType::Struct(fields) => { + // StructBuilder::append_null only flips parent validity; children must each get a null too. + let struct_builder = builder + .as_any_mut() + .downcast_mut::() + .ok_or_else(|| RowConvertError { + message: format!( + "Builder type mismatch: expected StructBuilder for {data_type:?}", + ), + })?; + let cloned_fields = fields.clone(); + { + let field_builders = struct_builder.field_builders_mut(); + for (i, field) in cloned_fields.iter().enumerate() { + append_null_for_type(field_builders[i].as_mut(), field.data_type())?; + } + } + struct_builder.append(false); + Ok(()) + } _ => Err(RowConvertError { message: format!("Unsupported Arrow data type for null append: {data_type:?}"), }), } } +fn append_generic_row_to_struct_builder( + row: &GenericRow<'_>, + builder: &mut dyn ArrayBuilder, + data_type: &arrow_schema::DataType, +) -> Result<()> { + let struct_builder = builder + .as_any_mut() + .downcast_mut::() + .ok_or_else(|| RowConvertError { + message: "Builder type mismatch for Row: expected StructBuilder".to_string(), + })?; + + let fields = match data_type { + arrow_schema::DataType::Struct(fields) => fields.clone(), + _ => { + return Err(RowConvertError { + message: format!("Expected Struct Arrow type for Row datum, got: {data_type:?}"), + }); + } + }; + + if row.values.len() != fields.len() { + return Err(RowConvertError { + message: format!( + "Row arity mismatch: schema has {} fields, got {}", + fields.len(), + row.values.len(), + ), + }); + } + + { + let field_builders = struct_builder.field_builders_mut(); + for (i, datum) in row.values.iter().enumerate() { + let child = field_builders[i].as_mut(); + datum.append_to(child, fields[i].data_type())?; + } + } + struct_builder.append(true); + Ok(()) +} + impl Datum<'_> { pub fn append_to( &self, @@ -888,10 +964,8 @@ impl Datum<'_> { Datum::Array(arr) => { return append_fluss_array_to_list_builder(arr, builder, data_type); } - Datum::Row(_) => { - return Err(RowConvertError { - message: "append_to is not supported for Row type".to_string(), - }); + Datum::Row(row) => { + return append_generic_row_to_struct_builder(row, builder, data_type); } } @@ -981,7 +1055,7 @@ impl TimestampNtz { pub fn from_millis_nanos(millisecond: i64, nano_of_millisecond: i32) -> Result { if !(0..=MAX_NANO_OF_MILLISECOND).contains(&nano_of_millisecond) { - return Err(crate::error::Error::IllegalArgument { + return Err(IllegalArgument { message: format!( "nanoOfMillisecond must be in range [0, {MAX_NANO_OF_MILLISECOND}], got: {nano_of_millisecond}" ), @@ -1025,7 +1099,7 @@ impl TimestampLtz { pub fn from_millis_nanos(epoch_millisecond: i64, nano_of_millisecond: i32) -> Result { if !(0..=MAX_NANO_OF_MILLISECOND).contains(&nano_of_millisecond) { - return Err(crate::error::Error::IllegalArgument { + return Err(IllegalArgument { message: format!( "nanoOfMillisecond must be in range [0, {MAX_NANO_OF_MILLISECOND}], got: {nano_of_millisecond}" ), @@ -1149,7 +1223,7 @@ mod tests { let err = Datum::Int32(1) .append_to(&mut builder, &arrow_schema::DataType::Utf8) .unwrap_err(); - assert!(matches!(err, crate::error::Error::RowConvertError { .. })); + assert!(matches!(err, RowConvertError { .. })); } #[test] @@ -1177,6 +1251,13 @@ mod tests { #[cfg(test)] mod timestamp_tests { use super::*; + use crate::metadata::{DataField, DataTypes}; + use crate::record::to_arrow_type; + use crate::row::InternalRow; + use crate::row::column::ColumnarRow; + use arrow::array::{RecordBatch, StructArray, StructBuilder}; + use arrow::datatypes::{Field, Fields, Schema}; + use std::sync::Arc; #[test] fn test_timestamp_valid_nanos() { @@ -1221,4 +1302,62 @@ mod timestamp_tests { assert!(result_ltz.is_err()); assert!(result_ltz.unwrap_err().to_string().contains(&expected_msg)); } + + #[test] + fn test_row_arrow_struct_round_trip() { + let row_type_owned = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let arrow_struct_dt = to_arrow_type(&row_type_owned).unwrap(); + let struct_fields: Fields = match &arrow_struct_dt { + arrow_schema::DataType::Struct(f) => f.clone(), + _ => unreachable!(), + }; + + let mut struct_builder = StructBuilder::from_fields(struct_fields.clone(), 3); + + let mut r0 = GenericRow::new(2); + r0.set_field(0, 42_i32); + r0.set_field(1, "hello"); + Datum::Row(Box::new(r0)) + .append_to(&mut struct_builder, &arrow_struct_dt) + .expect("append row 0"); + + Datum::Null + .append_to(&mut struct_builder, &arrow_struct_dt) + .expect("append null row"); + + let mut r2 = GenericRow::new(2); + r2.set_field(0, -7_i32); + r2.set_field(1, Datum::Null); + Datum::Row(Box::new(r2)) + .append_to(&mut struct_builder, &arrow_struct_dt) + .expect("append row 2"); + + let struct_array: StructArray = struct_builder.finish(); + + let schema = Arc::new(Schema::new(vec![Field::new( + "nested", + arrow_struct_dt.clone(), + true, + )])); + let batch = Arc::new( + RecordBatch::try_new(schema, vec![Arc::new(struct_array)]).expect("record batch"), + ); + + let mut columnar = ColumnarRow::new(batch, 0, None); + + let nested = columnar.get_row(0).expect("get_row 0"); + assert_eq!(nested.get_int(0).unwrap(), 42); + assert_eq!(nested.get_string(1).unwrap(), "hello"); + + columnar.set_row_id(1); + assert!(columnar.is_null_at(0).unwrap(), "row 1 should be null"); + + columnar.set_row_id(2); + let nested = columnar.get_row(0).expect("get_row 2"); + assert_eq!(nested.get_int(0).unwrap(), -7); + assert!(nested.is_null_at(1).unwrap(), "label should be null"); + } } diff --git a/crates/fluss/src/row/encode/compacted_key_encoder.rs b/crates/fluss/src/row/encode/compacted_key_encoder.rs index a6ba580f..c7f16d6c 100644 --- a/crates/fluss/src/row/encode/compacted_key_encoder.rs +++ b/crates/fluss/src/row/encode/compacted_key_encoder.rs @@ -111,9 +111,10 @@ mod tests { use super::*; use crate::metadata::{DataType, DataTypes}; use crate::row::binary_array::FlussArrayWriter; - use crate::row::{Datum, GenericRow}; + use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; + use crate::row::{Datum, Decimal, FlussArray, GenericRow}; - fn build_int_array(values: &[i32]) -> crate::row::FlussArray { + fn build_int_array(values: &[i32]) -> FlussArray { let mut w = FlussArrayWriter::new(values.len(), &DataTypes::int()); for (i, v) in values.iter().enumerate() { w.write_int(i, *v); @@ -121,7 +122,7 @@ mod tests { w.complete().unwrap() } - fn build_nullable_int_array(values: &[Option]) -> crate::row::FlussArray { + fn build_nullable_int_array(values: &[Option]) -> FlussArray { let mut w = FlussArrayWriter::new(values.len(), &DataTypes::int()); for (i, v) in values.iter().enumerate() { match v { @@ -132,7 +133,7 @@ mod tests { w.complete().unwrap() } - fn build_float_array(values: &[f32]) -> crate::row::FlussArray { + fn build_float_array(values: &[f32]) -> FlussArray { let mut w = FlussArrayWriter::new(values.len(), &DataTypes::float().as_non_nullable()); for (i, v) in values.iter().enumerate() { w.write_float(i, *v); @@ -140,7 +141,7 @@ mod tests { w.complete().unwrap() } - fn build_nested_string_array() -> crate::row::FlussArray { + fn build_nested_string_array() -> FlussArray { let mut inner_1 = FlussArrayWriter::new(3, &DataTypes::string()); inner_1.write_string(0, "a"); inner_1.set_null_at(1); @@ -369,32 +370,32 @@ mod tests { // Exact values from Java's IndexedRowTest.genRecordForAllTypes() let row = GenericRow::from_data(vec![ - Datum::from(true), // BOOLEAN: true - Datum::from(2i8), // TINYINT: 2 - Datum::from(10i16), // SMALLINT: 10 - Datum::from(100i32), // INT: 100 - Datum::from(-6101065172474983726i64), // BIGINT - Datum::from(13.2f32), // FLOAT: 13.2 - Datum::from(15.21f64), // DOUBLE: 15.21 - Datum::Date(crate::row::datum::Date::new(19655)), // DATE: 2023-10-25 (19655 days since epoch) - Datum::Time(crate::row::datum::Time::new(34200000)), // TIME: 09:30:00.0 - Datum::from("1234567890".as_bytes()), // BINARY(20) - Datum::from("20".as_bytes()), // BYTES - Datum::from("1"), // CHAR(2): "1" - Datum::from("hello"), // STRING: "hello" - Datum::Decimal(crate::row::Decimal::from_unscaled_long(9, 5, 2).unwrap()), // DECIMAL(5,2) + Datum::from(true), // BOOLEAN: true + Datum::from(2i8), // TINYINT: 2 + Datum::from(10i16), // SMALLINT: 10 + Datum::from(100i32), // INT: 100 + Datum::from(-6101065172474983726i64), // BIGINT + Datum::from(13.2f32), // FLOAT: 13.2 + Datum::from(15.21f64), // DOUBLE: 15.21 + Datum::Date(Date::new(19655)), // DATE: 2023-10-25 (19655 days since epoch) + Datum::Time(Time::new(34200000)), // TIME: 09:30:00.0 + Datum::from("1234567890".as_bytes()), // BINARY(20) + Datum::from("20".as_bytes()), // BYTES + Datum::from("1"), // CHAR(2): "1" + Datum::from("hello"), // STRING: "hello" + Datum::Decimal(Decimal::from_unscaled_long(9, 5, 2).unwrap()), // DECIMAL(5,2) Datum::Decimal( - crate::row::Decimal::from_big_decimal( + Decimal::from_big_decimal( bigdecimal::BigDecimal::new(bigdecimal::num_bigint::BigInt::from(10), 0), 20, 0, ) .unwrap(), ), // DECIMAL(20,0) - Datum::TimestampNtz(crate::row::datum::TimestampNtz::new(1698235273182)), // TIMESTAMP(1) - Datum::TimestampNtz(crate::row::datum::TimestampNtz::new(1698235273182)), // TIMESTAMP(5) - Datum::TimestampLtz(crate::row::datum::TimestampLtz::new(1698235273182)), // TIMESTAMP_LTZ(1) - Datum::TimestampLtz(crate::row::datum::TimestampLtz::new(1698235273182)), // TIMESTAMP_LTZ(5) + Datum::TimestampNtz(TimestampNtz::new(1698235273182)), // TIMESTAMP(1) + Datum::TimestampNtz(TimestampNtz::new(1698235273182)), // TIMESTAMP(5) + Datum::TimestampLtz(TimestampLtz::new(1698235273182)), // TIMESTAMP_LTZ(1) + Datum::TimestampLtz(TimestampLtz::new(1698235273182)), // TIMESTAMP_LTZ(5) Datum::Array(build_nullable_int_array(&[ Some(1), Some(2), @@ -502,10 +503,7 @@ mod tests { vec!["x", "label"], ); let row_type = RowType::with_data_types_and_field_names( - vec![ - DataTypes::int(), - DataType::Row(inner_row_type.clone()), - ], + vec![DataTypes::int(), DataType::Row(inner_row_type.clone())], vec!["id", "nested"], ); diff --git a/crates/fluss/src/row/lookup_row.rs b/crates/fluss/src/row/lookup_row.rs index c002369d..76505abf 100644 --- a/crates/fluss/src/row/lookup_row.rs +++ b/crates/fluss/src/row/lookup_row.rs @@ -24,7 +24,7 @@ use crate::error::Result; use crate::row::compacted::CompactedRow; use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; use crate::row::projected_row::ProjectedRow; -use crate::row::{Decimal, FlussArray, InternalRow}; +use crate::row::{Decimal, FlussArray, GenericRow, InternalRow}; pub struct LookupRow<'a> { inner: Inner<'a>, @@ -116,6 +116,9 @@ impl<'a> InternalRow for LookupRow<'a> { fn get_array(&self, pos: usize) -> Result { delegate!(self, get_array, pos) } + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + delegate!(self, get_row, pos) + } fn as_encoded_bytes(&self, write_format: WriteFormat) -> Option<&[u8]> { delegate!(self, as_encoded_bytes, write_format) } diff --git a/crates/fluss/src/row/mod.rs b/crates/fluss/src/row/mod.rs index fb599fc9..36f9a1c2 100644 --- a/crates/fluss/src/row/mod.rs +++ b/crates/fluss/src/row/mod.rs @@ -34,7 +34,6 @@ mod row_decoder; use crate::client::WriteFormat; pub use binary_array::FlussArray; use bytes::Bytes; -use serde::Serialize; pub use column::*; pub use compacted::CompactedRow; pub use datum::*; @@ -44,6 +43,7 @@ pub(crate) use fixed_schema_decoder::FixedSchemaDecoder; pub use lookup_row::LookupRow; pub(crate) use projected_row::ProjectedRow; pub use row_decoder::{CompactedRowDecoder, RowDecoder, RowDecoderFactory}; +use serde::Serialize; pub struct BinaryRow<'a> { data: BinaryDataWrapper<'a>, @@ -133,7 +133,7 @@ pub trait InternalRow: Send + Sync { /// Returns the nested row value at the given position fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { - Err(crate::error::Error::IllegalArgument { + Err(IllegalArgument { message: format!("get_row not supported at position {pos}"), }) } @@ -313,6 +313,17 @@ impl<'a> InternalRow for GenericRow<'a> { } } +impl<'a> GenericRow<'a> { + /// Consumes this row and returns one whose `Datum` values are all + /// `'static` (borrowed `Cow`s are promoted to owned, nested rows recurse). + /// Lets a row outlive the bytes it was decoded from. + pub fn into_owned(self) -> GenericRow<'static> { + GenericRow { + values: self.values.into_iter().map(Datum::into_owned).collect(), + } + } +} + impl<'a> GenericRow<'a> { pub fn from_data(data: Vec>>) -> GenericRow<'a> { GenericRow { diff --git a/crates/fluss/src/row/projected_row.rs b/crates/fluss/src/row/projected_row.rs index cdd946a5..fc4a521e 100644 --- a/crates/fluss/src/row/projected_row.rs +++ b/crates/fluss/src/row/projected_row.rs @@ -23,7 +23,7 @@ use crate::error::Error::IllegalArgument; use crate::error::Result; use crate::metadata::UNEXIST_MAPPING; use crate::row::datum::{Date, Time, TimestampLtz, TimestampNtz}; -use crate::row::{Decimal, FlussArray, InternalRow}; +use crate::row::{Decimal, FlussArray, GenericRow, InternalRow}; use std::sync::Arc; pub(crate) struct ProjectedRow { @@ -142,6 +142,10 @@ impl InternalRow for ProjectedRow { project!(self, get_array, pos) } + fn get_row(&self, pos: usize) -> Result<&GenericRow<'_>> { + project!(self, get_row, pos) + } + fn as_encoded_bytes(&self, _write_format: WriteFormat) -> Option<&[u8]> { // Projection changes the field layout, so the inner row's // encoded form no longer matches. diff --git a/crates/fluss/tests/integration/kv_table.rs b/crates/fluss/tests/integration/kv_table.rs index 8787e881..ee496a5a 100644 --- a/crates/fluss/tests/integration/kv_table.rs +++ b/crates/fluss/tests/integration/kv_table.rs @@ -21,9 +21,11 @@ mod kv_table_test { use crate::integration::utils::{ create_partitions, create_table, get_shared_cluster, make_int_array, make_string_array, }; - use fluss::metadata::{DataTypes, Schema, TableDescriptor, TablePath}; + use fluss::metadata::{DataField, DataTypes, Schema, TableDescriptor, TablePath}; use fluss::row::binary_array::FlussArrayWriter; - use fluss::row::{FlussArray, GenericRow, InternalRow}; + use fluss::row::{ + Date, Datum, Decimal, FlussArray, GenericRow, InternalRow, Time, TimestampLtz, TimestampNtz, + }; fn make_key(id: i32) -> GenericRow<'static> { make_key_with_field_count(id, 3) @@ -284,8 +286,6 @@ mod kv_table_test { #[tokio::test] async fn partial_update() { - use fluss::row::Datum; - let cluster = get_shared_cluster(); let connection = cluster.get_fluss_connection().await; @@ -293,6 +293,11 @@ mod kv_table_test { let table_path = TablePath::new("fluss", "test_partial_update"); + let nested_type = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let table_descriptor = TableDescriptor::builder() .schema( Schema::builder() @@ -300,6 +305,7 @@ mod kv_table_test { .column("name", DataTypes::string()) .column("age", DataTypes::bigint()) .column("score", DataTypes::bigint()) + .column("nested", nested_type) .primary_key(vec!["id"]) .build() .expect("Failed to build schema"), @@ -314,24 +320,26 @@ mod kv_table_test { .await .expect("Failed to get table"); - // Insert initial record with all columns let table_upsert = table.new_upsert().expect("Failed to create upsert"); let upsert_writer = table_upsert .create_writer() .expect("Failed to create writer"); - let mut row = GenericRow::new(4); + let mut nested0 = GenericRow::new(2); + nested0.set_field(0, 10_i32); + nested0.set_field(1, "alpha"); + let mut row = GenericRow::new(5); row.set_field(0, 1); row.set_field(1, "Verso"); row.set_field(2, 32i64); row.set_field(3, 6942i64); + row.set_field(4, Datum::Row(Box::new(nested0))); upsert_writer .upsert(&row) .expect("Failed to upsert initial row") .await .expect("Failed to wait for upsert acknowledgment"); - // Verify initial record let mut lookuper = table .new_lookup() .expect("Failed to create lookup") @@ -351,8 +359,10 @@ mod kv_table_test { assert_eq!(found_row.get_string(1).unwrap(), "Verso"); assert_eq!(found_row.get_long(2).unwrap(), 32i64); assert_eq!(found_row.get_long(3).unwrap(), 6942i64); + let nested = found_row.get_row(4).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 10); + assert_eq!(nested.get_string(1).unwrap(), "alpha"); - // Create partial update writer to update only score column let partial_upsert = table_upsert .partial_update_with_column_names(&["id", "score"]) .expect("Failed to create TableUpsert with partial update"); @@ -360,19 +370,18 @@ mod kv_table_test { .create_writer() .expect("Failed to create UpsertWriter with partial write"); - // Update only the score column (await acknowledgment) - let mut partial_row = GenericRow::new(4); + let mut partial_row = GenericRow::new(5); partial_row.set_field(0, 1); - partial_row.set_field(1, Datum::Null); // not in partial update column - partial_row.set_field(2, Datum::Null); // not in partial update column + partial_row.set_field(1, Datum::Null); + partial_row.set_field(2, Datum::Null); partial_row.set_field(3, 420i64); + partial_row.set_field(4, Datum::Null); partial_writer .upsert(&partial_row) .expect("Failed to upsert") .await .expect("Failed to wait for upsert acknowledgment"); - // Verify partial update - name and age should remain unchanged let result = lookuper .lookup(&make_key(1)) .await @@ -398,6 +407,56 @@ mod kv_table_test { 420, "score should be updated to 420" ); + let nested = found_row.get_row(4).unwrap(); + assert_eq!( + nested.get_int(0).unwrap(), + 10, + "ROW preserved across non-ROW partial update" + ); + assert_eq!(nested.get_string(1).unwrap(), "alpha"); + + let partial_nested_upsert = table_upsert + .partial_update_with_column_names(&["id", "nested"]) + .expect("partial_update_with_column_names"); + let partial_nested_writer = partial_nested_upsert + .create_writer() + .expect("partial writer"); + let mut new_nested = GenericRow::new(2); + new_nested.set_field(0, 99_i32); + new_nested.set_field(1, "omega"); + let mut partial_nested = GenericRow::new(5); + partial_nested.set_field(0, 1); + partial_nested.set_field(1, Datum::Null); + partial_nested.set_field(2, Datum::Null); + partial_nested.set_field(3, Datum::Null); + partial_nested.set_field(4, Datum::Row(Box::new(new_nested))); + partial_nested_writer + .upsert(&partial_nested) + .expect("partial upsert") + .await + .expect("partial ack"); + + let result = lookuper + .lookup(&make_key(1)) + .await + .expect("Failed to lookup after nested partial"); + let found_row = result + .get_single_row() + .expect("Failed to get row") + .expect("Row should exist"); + assert_eq!( + found_row.get_string(1).unwrap(), + "Verso", + "name preserved when ROW updated" + ); + assert_eq!( + found_row.get_long(3).unwrap(), + 420, + "score preserved when ROW updated" + ); + let nested = found_row.get_row(4).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 99); + assert_eq!(nested.get_string(1).unwrap(), "omega"); admin .drop_table(&table_path, false) @@ -414,7 +473,11 @@ mod kv_table_test { let table_path = TablePath::new("fluss", "test_partitioned_kv_table"); - // Create a partitioned KV table with region as partition key + let nested_type = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let table_descriptor = TableDescriptor::builder() .schema( Schema::builder() @@ -422,6 +485,7 @@ mod kv_table_test { .column("user_id", DataTypes::int()) .column("name", DataTypes::string()) .column("score", DataTypes::bigint()) + .column("nested", nested_type) .primary_key(vec!["region", "user_id"]) .build() .expect("Failed to build schema"), @@ -432,7 +496,6 @@ mod kv_table_test { create_table(&admin, &table_path, &table_descriptor).await; - // Create partitions for each region before inserting data create_partitions(&admin, &table_path, "region", &["US", "EU", "APAC"]).await; let connection = cluster.get_fluss_connection().await; @@ -448,35 +511,38 @@ mod kv_table_test { .create_writer() .expect("Failed to create writer"); - // Insert records with different partitions let test_data = [ - ("US", 1, "Gustave", 100i64), - ("US", 2, "Lune", 200i64), - ("EU", 1, "Sciel", 150i64), - ("EU", 2, "Maelle", 250i64), - ("APAC", 1, "Noco", 300i64), + ("US", 1, "Gustave", 100i64, 11_i32, "a"), + ("US", 2, "Lune", 200i64, 22, "b"), + ("EU", 1, "Sciel", 150i64, 33, "c"), + ("EU", 2, "Maelle", 250i64, 44, "d"), + ("APAC", 1, "Noco", 300i64, 55, "e"), ]; - for (region, user_id, name, score) in &test_data { - let mut row = GenericRow::new(4); + for (region, user_id, name, score, seq, label) in &test_data { + let mut nested = GenericRow::new(2); + nested.set_field(0, *seq); + nested.set_field(1, *label); + let mut row = GenericRow::new(5); row.set_field(0, *region); row.set_field(1, *user_id); row.set_field(2, *name); row.set_field(3, *score); + row.set_field(4, Datum::Row(Box::new(nested))); upsert_writer.upsert(&row).expect("Failed to upsert"); } upsert_writer.flush().await.expect("Failed to flush"); - // Create lookuper let mut lookuper = table .new_lookup() .expect("Failed to create lookup") .create_lookuper() .expect("Failed to create lookuper"); - // Lookup records - the lookup key includes partition key columns - for (region, user_id, expected_name, expected_score) in &test_data { - let mut key = GenericRow::new(4); + for (region, user_id, expected_name, expected_score, expected_seq, expected_label) in + &test_data + { + let mut key = GenericRow::new(5); key.set_field(0, *region); key.set_field(1, *user_id); @@ -490,14 +556,28 @@ mod kv_table_test { assert_eq!(row.get_int(1).unwrap(), *user_id, "user_id mismatch"); assert_eq!(row.get_string(2).unwrap(), *expected_name, "name mismatch"); assert_eq!(row.get_long(3).unwrap(), *expected_score, "score mismatch"); + let nested = row.get_row(4).unwrap(); + assert_eq!( + nested.get_int(0).unwrap(), + *expected_seq, + "ROW seq mismatch" + ); + assert_eq!( + nested.get_string(1).unwrap(), + *expected_label, + "ROW label mismatch" + ); } - // Test update within a partition (await acknowledgment) - let mut updated_row = GenericRow::new(4); + let mut updated_nested = GenericRow::new(2); + updated_nested.set_field(0, 999_i32); + updated_nested.set_field(1, "updated"); + let mut updated_row = GenericRow::new(5); updated_row.set_field(0, "US"); updated_row.set_field(1, 1); updated_row.set_field(2, "Gustave Updated"); updated_row.set_field(3, 999i64); + updated_row.set_field(4, Datum::Row(Box::new(updated_nested))); upsert_writer .upsert(&updated_row) .expect("Failed to upsert updated row") @@ -505,7 +585,7 @@ mod kv_table_test { .expect("Failed to wait for upsert acknowledgment"); // Verify the update - let mut key = GenericRow::new(4); + let mut key = GenericRow::new(5); key.set_field(0, "US"); key.set_field(1, 1); let result = lookuper.lookup(&key).await.expect("Failed to lookup"); @@ -515,9 +595,12 @@ mod kv_table_test { .expect("Row should exist"); assert_eq!(row.get_string(2).unwrap(), "Gustave Updated"); assert_eq!(row.get_long(3).unwrap(), 999); + let nested = row.get_row(4).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 999); + assert_eq!(nested.get_string(1).unwrap(), "updated"); // Lookup in non-existent partition should return empty result - let mut non_existent_key = GenericRow::new(4); + let mut non_existent_key = GenericRow::new(5); non_existent_key.set_field(0, "UNKNOWN_REGION"); non_existent_key.set_field(1, 1); let result = lookuper @@ -533,7 +616,7 @@ mod kv_table_test { ); // Delete a record within a partition (await acknowledgment) - let mut delete_key = GenericRow::new(4); + let mut delete_key = GenericRow::new(5); delete_key.set_field(0, "EU"); delete_key.set_field(1, 1); upsert_writer @@ -543,7 +626,7 @@ mod kv_table_test { .expect("Failed to wait for delete acknowledgment"); // Verify deletion - let mut key = GenericRow::new(4); + let mut key = GenericRow::new(5); key.set_field(0, "EU"); key.set_field(1, 1); let result = lookuper.lookup(&key).await.expect("Failed to lookup"); @@ -556,7 +639,7 @@ mod kv_table_test { ); // Verify other records in the same partition still exist - let mut key = GenericRow::new(4); + let mut key = GenericRow::new(5); key.set_field(0, "EU"); key.set_field(1, 2); let result = lookuper.lookup(&key).await.expect("Failed to lookup"); @@ -572,11 +655,122 @@ mod kv_table_test { .expect("Failed to drop table"); } + #[tokio::test] + async fn upsert_and_lookup_with_row_rich_types() { + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_kv_row_rich_types"); + + let row_type_owned = DataTypes::row(vec![ + DataField::new("f_bool", DataTypes::boolean(), None), + DataField::new("f_long", DataTypes::bigint(), None), + DataField::new("f_float", DataTypes::float(), None), + DataField::new("f_double", DataTypes::double(), None), + DataField::new("f_str", DataTypes::string(), None), + DataField::new("f_bytes", DataTypes::bytes(), None), + DataField::new("f_decimal", DataTypes::decimal(10, 2), None), + DataField::new("f_date", DataTypes::date(), None), + DataField::new("f_time", DataTypes::time_with_precision(3), None), + DataField::new("f_ts_ntz", DataTypes::timestamp_with_precision(6), None), + DataField::new("f_ts_ltz", DataTypes::timestamp_ltz_with_precision(6), None), + ]); + + let table_descriptor = TableDescriptor::builder() + .schema( + Schema::builder() + .column("id", DataTypes::int()) + .column("nested", row_type_owned) + .primary_key(vec!["id"]) + .build() + .expect("Failed to build schema"), + ) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + let upsert = table.new_upsert().expect("Failed to create upsert"); + let upsert_writer = upsert.create_writer().expect("Failed to create writer"); + + let mut nested = GenericRow::new(11); + nested.set_field(0, true); + nested.set_field(1, 9_876_543_210_i64); + nested.set_field(2, f32::NEG_INFINITY); + nested.set_field(3, f64::NAN); + nested.set_field(4, "rich types here"); + nested.set_field(5, b"opaque".as_slice()); + nested.set_field(6, Decimal::from_unscaled_long(54321, 10, 2).unwrap()); + nested.set_field(7, Datum::Date(Date::new(20476))); + nested.set_field(8, Datum::Time(Time::new(36_827_123))); + nested.set_field(9, Datum::TimestampNtz(TimestampNtz::new(1_769_163_227_123))); + nested.set_field( + 10, + Datum::TimestampLtz(TimestampLtz::new(1_769_163_227_123)), + ); + + let mut row = GenericRow::new(2); + row.set_field(0, 1_i32); + row.set_field(1, Datum::Row(Box::new(nested))); + + upsert_writer + .upsert(&row) + .expect("upsert") + .await + .expect("ack"); + + let mut lookuper = table + .new_lookup() + .expect("Failed to create lookup") + .create_lookuper() + .expect("Failed to create lookuper"); + + let result = lookuper + .lookup(&make_key_with_field_count(1, 2)) + .await + .expect("lookup"); + let r = result + .get_single_row() + .expect("get row") + .expect("row should exist"); + + let n = r.get_row(1).unwrap(); + assert!(n.get_boolean(0).unwrap()); + assert_eq!(n.get_long(1).unwrap(), 9_876_543_210); + assert!(n.get_float(2).unwrap().is_infinite()); + assert!(n.get_float(2).unwrap().is_sign_negative()); + assert!(n.get_double(3).unwrap().is_nan()); + assert_eq!(n.get_string(4).unwrap(), "rich types here"); + assert_eq!(n.get_bytes(5).unwrap(), b"opaque"); + assert_eq!( + n.get_decimal(6, 10, 2).unwrap(), + Decimal::from_unscaled_long(54321, 10, 2).unwrap(), + ); + assert_eq!(n.get_date(7).unwrap().get_inner(), 20476); + assert_eq!(n.get_time(8).unwrap().get_inner(), 36_827_123); + assert_eq!( + n.get_timestamp_ntz(9, 6).unwrap().get_millisecond(), + 1_769_163_227_123, + ); + assert_eq!( + n.get_timestamp_ltz(10, 6).unwrap().get_epoch_millisecond(), + 1_769_163_227_123, + ); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + /// Integration test covering put and get operations for all supported datatypes. #[tokio::test] async fn all_supported_datatypes() { - use fluss::row::{Date, Datum, Decimal, Time, TimestampLtz, TimestampNtz}; - let cluster = get_shared_cluster(); let connection = cluster.get_fluss_connection().await; @@ -614,6 +808,13 @@ mod kv_table_test { .column("col_bytes", DataTypes::bytes()) .column("col_binary", DataTypes::binary(20)) .column("col_array", DataTypes::array(DataTypes::string())) + .column( + "col_row", + DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]), + ) .primary_key(vec!["pk_int"]) .build() .expect("Failed to build schema"), @@ -654,8 +855,12 @@ mod kv_table_test { let col_array = make_string_array(&[Some("fluss"), Some("rust")]); + let mut col_row_inner = GenericRow::new(2); + col_row_inner.set_field(0, 7_i32); + col_row_inner.set_field(1, "lumiere"); + // Upsert a row with all datatypes - let mut row = GenericRow::new(18); + let mut row = GenericRow::new(19); row.set_field(0, pk_int); row.set_field(1, col_boolean); row.set_field(2, col_tinyint); @@ -674,6 +879,7 @@ mod kv_table_test { row.set_field(15, col_bytes); row.set_field(16, col_binary); row.set_field(17, col_array); + row.set_field(18, Datum::Row(Box::new(col_row_inner))); upsert_writer .upsert(&row) @@ -688,7 +894,7 @@ mod kv_table_test { .create_lookuper() .expect("Failed to create lookuper"); - let mut key = GenericRow::new(18); + let mut key = GenericRow::new(19); key.set_field(0, pk_int); let result = lookuper.lookup(&key).await.expect("Failed to lookup"); @@ -787,10 +993,17 @@ mod kv_table_test { assert_eq!(arr.size(), 2, "col_array size mismatch"); assert_eq!(arr.get_string(0).unwrap(), "fluss", "col_array[0] mismatch"); assert_eq!(arr.get_string(1).unwrap(), "rust", "col_array[1] mismatch"); + let nested = found_row.get_row(18).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 7, "col_row.seq mismatch"); + assert_eq!( + nested.get_string(1).unwrap(), + "lumiere", + "col_row.label mismatch" + ); // Test with null values for nullable columns let pk_int_2 = 2i32; - let mut row_with_nulls = GenericRow::new(18); + let mut row_with_nulls = GenericRow::new(19); row_with_nulls.set_field(0, pk_int_2); row_with_nulls.set_field(1, Datum::Null); // col_boolean row_with_nulls.set_field(2, Datum::Null); // col_tinyint @@ -809,6 +1022,7 @@ mod kv_table_test { row_with_nulls.set_field(15, Datum::Null); // col_bytes row_with_nulls.set_field(16, Datum::Null); // col_binary row_with_nulls.set_field(17, Datum::Null); // col_array + row_with_nulls.set_field(18, Datum::Null); // col_row upsert_writer .upsert(&row_with_nulls) @@ -817,7 +1031,7 @@ mod kv_table_test { .expect("Failed to wait for upsert acknowledgment"); // Lookup row with nulls - let mut key2 = GenericRow::new(18); + let mut key2 = GenericRow::new(19); key2.set_field(0, pk_int_2); let result = lookuper.lookup(&key2).await.expect("Failed to lookup"); @@ -900,6 +1114,10 @@ mod kv_table_test { found_row_nulls.is_null_at(17).unwrap(), "col_array should be null" ); + assert!( + found_row_nulls.is_null_at(18).unwrap(), + "col_row should be null" + ); admin .drop_table(&table_path, false) @@ -908,9 +1126,307 @@ mod kv_table_test { } #[tokio::test] - async fn upsert_and_lookup_with_array() { - use fluss::row::Datum; + async fn upsert_and_lookup_with_row() { + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_kv_rows"); + let nested_row_type = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let deep_inner_row_type = DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let deep_row_type = + DataTypes::row(vec![DataField::new("inner", deep_inner_row_type, None)]); + + let table_descriptor = TableDescriptor::builder() + .schema( + Schema::builder() + .column("id", DataTypes::int()) + .column("nested", nested_row_type) + .column("deep", deep_row_type) + .primary_key(vec!["id"]) + .build() + .expect("Failed to build schema"), + ) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + + let upsert = table.new_upsert().expect("Failed to create upsert"); + let upsert_writer = upsert.create_writer().expect("Failed to create writer"); + + let mut nested1 = GenericRow::new(2); + nested1.set_field(0, 42_i32); + nested1.set_field(1, "hello"); + + let mut deep_inner1 = GenericRow::new(1); + deep_inner1.set_field(0, 99_i32); + let mut deep1 = GenericRow::new(1); + deep1.set_field(0, Datum::Row(Box::new(deep_inner1))); + + let mut row1 = GenericRow::new(3); + row1.set_field(0, 1_i32); + row1.set_field(1, Datum::Row(Box::new(nested1))); + row1.set_field(2, Datum::Row(Box::new(deep1))); + + upsert_writer + .upsert(&row1) + .expect("upsert row1") + .await + .expect("ack row1"); + + let mut nested2 = GenericRow::new(2); + nested2.set_field(0, 7_i32); + nested2.set_field(1, Datum::Null); + + let mut row2 = GenericRow::new(3); + row2.set_field(0, 2_i32); + row2.set_field(1, Datum::Row(Box::new(nested2))); + row2.set_field(2, Datum::Null); + + upsert_writer + .upsert(&row2) + .expect("upsert row2") + .await + .expect("ack row2"); + + let mut deep_inner3 = GenericRow::new(1); + deep_inner3.set_field(0, -1_i32); + let mut deep3 = GenericRow::new(1); + deep3.set_field(0, Datum::Row(Box::new(deep_inner3))); + + let mut row3 = GenericRow::new(3); + row3.set_field(0, 3_i32); + row3.set_field(1, Datum::Null); + row3.set_field(2, Datum::Row(Box::new(deep3))); + + upsert_writer + .upsert(&row3) + .expect("upsert row3") + .await + .expect("ack row3"); + + let mut lookuper = table + .new_lookup() + .expect("Failed to create lookup") + .create_lookuper() + .expect("Failed to create lookuper"); + + let result1 = lookuper + .lookup(&make_key_with_field_count(1, 3)) + .await + .expect("lookup row1"); + let r1 = result1 + .get_single_row() + .expect("get row1") + .expect("row1 should exist"); + assert_eq!(r1.get_int(0).unwrap(), 1); + let nested_r1 = r1.get_row(1).unwrap(); + assert_eq!(nested_r1.get_int(0).unwrap(), 42); + assert_eq!(nested_r1.get_string(1).unwrap(), "hello"); + let deep_r1 = r1.get_row(2).unwrap(); + let deep_inner_r1 = deep_r1.get_row(0).unwrap(); + assert_eq!(deep_inner_r1.get_int(0).unwrap(), 99); + + let result2 = lookuper + .lookup(&make_key_with_field_count(2, 3)) + .await + .expect("lookup row2"); + let r2 = result2 + .get_single_row() + .expect("get row2") + .expect("row2 should exist"); + assert_eq!(r2.get_int(0).unwrap(), 2); + let nested_r2 = r2.get_row(1).unwrap(); + assert_eq!(nested_r2.get_int(0).unwrap(), 7); + assert!(nested_r2.is_null_at(1).unwrap()); + assert!(r2.is_null_at(2).unwrap()); + let result3 = lookuper + .lookup(&make_key_with_field_count(3, 3)) + .await + .expect("lookup row3"); + let r3 = result3 + .get_single_row() + .expect("get row3") + .expect("row3 should exist"); + assert_eq!(r3.get_int(0).unwrap(), 3); + assert!(r3.is_null_at(1).unwrap()); + let deep_r3 = r3.get_row(2).unwrap(); + let deep_inner_r3 = deep_r3.get_row(0).unwrap(); + assert_eq!(deep_inner_r3.get_int(0).unwrap(), -1); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + #[tokio::test] + async fn upsert_and_lookup_with_array_of_row() { + use fluss::metadata::{DataField, DataType}; + + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_kv_array_of_row"); + + let event_row_type_owned = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let array_of_row_type = DataTypes::array(event_row_type_owned.clone()); + + let event_row_type = match &event_row_type_owned { + DataType::Row(rt) => rt.clone(), + _ => unreachable!(), + }; + + let table_descriptor = TableDescriptor::builder() + .schema( + Schema::builder() + .column("id", DataTypes::int()) + .column("events", array_of_row_type.clone()) + .primary_key(vec!["id"]) + .build() + .expect("Failed to build schema"), + ) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + + let upsert = table.new_upsert().expect("Failed to create upsert"); + let upsert_writer = upsert.create_writer().expect("Failed to create writer"); + + let mut events1 = FlussArrayWriter::new(2, &event_row_type_owned); + let mut e0 = GenericRow::new(2); + e0.set_field(0, 1_i32); + e0.set_field(1, "open"); + events1.write_row(0, &e0).expect("write e0"); + let mut e1 = GenericRow::new(2); + e1.set_field(0, 2_i32); + e1.set_field(1, "close"); + events1.write_row(1, &e1).expect("write e1"); + let events1 = events1.complete().expect("events1"); + + let mut row1 = GenericRow::new(2); + row1.set_field(0, 1_i32); + row1.set_field(1, events1); + + upsert_writer + .upsert(&row1) + .expect("upsert row1") + .await + .expect("ack row1"); + + let mut events2 = FlussArrayWriter::new(3, &event_row_type_owned); + let mut e2 = GenericRow::new(2); + e2.set_field(0, 7_i32); + e2.set_field(1, "x"); + events2.write_row(0, &e2).expect("write e2"); + events2.set_null_at(1); + let mut e3 = GenericRow::new(2); + e3.set_field(0, 8_i32); + e3.set_field(1, "y"); + events2.write_row(2, &e3).expect("write e3"); + let events2 = events2.complete().expect("events2"); + + let mut row2 = GenericRow::new(2); + row2.set_field(0, 2_i32); + row2.set_field(1, events2); + + upsert_writer + .upsert(&row2) + .expect("upsert row2") + .await + .expect("ack row2"); + + let mut row3 = GenericRow::new(2); + row3.set_field(0, 3_i32); + row3.set_field(1, Datum::Null); + + upsert_writer + .upsert(&row3) + .expect("upsert row3") + .await + .expect("ack row3"); + + let mut lookuper = table + .new_lookup() + .expect("Failed to create lookup") + .create_lookuper() + .expect("Failed to create lookuper"); + + let result1 = lookuper + .lookup(&make_key_with_field_count(1, 2)) + .await + .expect("lookup row1"); + let r1 = result1 + .get_single_row() + .expect("get row1") + .expect("row1 should exist"); + assert_eq!(r1.get_int(0).unwrap(), 1); + let events_r1 = r1.get_array(1).unwrap(); + assert_eq!(events_r1.size(), 2); + let e0_r1 = events_r1.get_row(0, &event_row_type).unwrap(); + assert_eq!(e0_r1.get_int(0).unwrap(), 1); + assert_eq!(e0_r1.get_string(1).unwrap(), "open"); + let e1_r1 = events_r1.get_row(1, &event_row_type).unwrap(); + assert_eq!(e1_r1.get_int(0).unwrap(), 2); + assert_eq!(e1_r1.get_string(1).unwrap(), "close"); + + let result2 = lookuper + .lookup(&make_key_with_field_count(2, 2)) + .await + .expect("lookup row2"); + let r2 = result2 + .get_single_row() + .expect("get row2") + .expect("row2 should exist"); + let events_r2 = r2.get_array(1).unwrap(); + assert_eq!(events_r2.size(), 3); + let e0_r2 = events_r2.get_row(0, &event_row_type).unwrap(); + assert_eq!(e0_r2.get_int(0).unwrap(), 7); + assert_eq!(e0_r2.get_string(1).unwrap(), "x"); + assert!(events_r2.is_null_at(1)); + let e2_r2 = events_r2.get_row(2, &event_row_type).unwrap(); + assert_eq!(e2_r2.get_int(0).unwrap(), 8); + assert_eq!(e2_r2.get_string(1).unwrap(), "y"); + + let result3 = lookuper + .lookup(&make_key_with_field_count(3, 2)) + .await + .expect("lookup row3"); + let r3 = result3 + .get_single_row() + .expect("get row3") + .expect("row3 should exist"); + assert_eq!(r3.get_int(0).unwrap(), 3); + assert!(r3.is_null_at(1).unwrap()); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + #[tokio::test] + async fn upsert_and_lookup_with_array() { let cluster = get_shared_cluster(); let connection = cluster.get_fluss_connection().await; let admin = connection.get_admin().expect("Failed to get admin"); @@ -942,7 +1458,6 @@ mod kv_table_test { let upsert = table.new_upsert().expect("Failed to create upsert"); let upsert_writer = upsert.create_writer().expect("Failed to create writer"); - // Row 1: id=1, tags=["hello", "world"], scores=[10, 20, 30], matrix=[[1,2],[3,4]] let mut row1 = GenericRow::new(4); row1.set_field(0, 1_i32); row1.set_field(1, make_string_array(&[Some("hello"), Some("world")])); @@ -961,7 +1476,6 @@ mod kv_table_test { .await .expect("ack row1"); - // Row 2: id=2, tags=[null element], scores=[] (empty), matrix=null let mut row2 = GenericRow::new(4); row2.set_field(0, 2_i32); row2.set_field(1, make_string_array(&[None])); @@ -974,7 +1488,6 @@ mod kv_table_test { .await .expect("ack row2"); - // Row 3: id=3, tags=null, scores=[42], matrix=[[5], null, []] let mut row3 = GenericRow::new(4); row3.set_field(0, 3_i32); row3.set_field(1, Datum::Null); @@ -1001,7 +1514,6 @@ mod kv_table_test { .create_lookuper() .expect("Failed to create lookuper"); - // Verify row 1: populated flat arrays + nested array let result1 = lookuper .lookup(&make_key_with_field_count(1, 4)) .await @@ -1031,7 +1543,6 @@ mod kv_table_test { assert_eq!(mr1_1.get_int(0).unwrap(), 3); assert_eq!(mr1_1.get_int(1).unwrap(), 4); - // Verify row 2: null element in array, empty array, null nested column let result2 = lookuper .lookup(&make_key_with_field_count(2, 4)) .await @@ -1048,7 +1559,6 @@ mod kv_table_test { assert_eq!(scores_r2.size(), 0); assert!(r2.is_null_at(3).unwrap()); - // Verify row 3: null flat column, nested array with mixed inner (value, null, empty) let result3 = lookuper .lookup(&make_key_with_field_count(3, 4)) .await diff --git a/crates/fluss/tests/integration/log_table.rs b/crates/fluss/tests/integration/log_table.rs index 1a6b514b..fadb4962 100644 --- a/crates/fluss/tests/integration/log_table.rs +++ b/crates/fluss/tests/integration/log_table.rs @@ -23,9 +23,12 @@ mod table_test { }; use arrow::array::record_batch; use fluss::client::{EARLIEST_OFFSET, FlussTable, TableScan}; - use fluss::metadata::{DataTypes, Schema, TableDescriptor, TablePath}; + use fluss::metadata::{DataField, DataTypes, Schema, TableDescriptor, TablePath}; use fluss::record::ScanRecord; - use fluss::row::InternalRow; + use fluss::row::binary_array::FlussArrayWriter; + use fluss::row::{ + Date, Datum, Decimal, FlussArray, GenericRow, InternalRow, Time, TimestampLtz, TimestampNtz, + }; use fluss::rpc::message::OffsetSpec; use std::collections::HashMap; use std::time::Duration; @@ -581,8 +584,6 @@ mod table_test { /// in log tables. #[tokio::test] async fn all_supported_datatypes() { - use fluss::row::{Date, Datum, Decimal, GenericRow, Time, TimestampLtz, TimestampNtz}; - let cluster = get_shared_cluster(); let connection = cluster.get_fluss_connection().await; @@ -661,6 +662,13 @@ mod table_test { DataTypes::timestamp_ltz_with_precision(9), ) .column("col_array", DataTypes::array(DataTypes::string())) + .column( + "col_row", + DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]), + ) .build() .expect("Failed to build schema"), ) @@ -724,7 +732,10 @@ mod table_test { let col_array = make_string_array(&[Some("fluss"), Some("rust")]); - // Append a row with all datatypes + let mut col_row_inner = GenericRow::new(2); + col_row_inner.set_field(0, 7_i32); + col_row_inner.set_field(1, "lumiere"); + let mut row = GenericRow::new(field_count); row.set_field(0, col_tinyint); row.set_field(1, col_smallint); @@ -756,6 +767,7 @@ mod table_test { row.set_field(27, col_timestamp_ltz_us_neg); row.set_field(28, col_timestamp_ltz_ns_neg); row.set_field(29, col_array); + row.set_field(30, Datum::Row(Box::new(col_row_inner))); append_writer .append(&row) @@ -1005,6 +1017,14 @@ mod table_test { assert_eq!(arr.get_string(0).unwrap(), "fluss", "col_array[0] mismatch"); assert_eq!(arr.get_string(1).unwrap(), "rust", "col_array[1] mismatch"); + let nested = found_row.get_row(30).unwrap(); + assert_eq!(nested.get_int(0).unwrap(), 7, "col_row.seq mismatch"); + assert_eq!( + nested.get_string(1).unwrap(), + "lumiere", + "col_row.label mismatch" + ); + // Verify row with all nulls (record index 1) let found_row_nulls = records[1].row(); for i in 0..field_count { @@ -1387,9 +1407,6 @@ mod table_test { #[tokio::test] async fn append_and_scan_with_array() { - use fluss::row::binary_array::FlussArrayWriter; - use fluss::row::{Datum, FlussArray, GenericRow}; - let cluster = get_shared_cluster(); let connection = cluster.get_fluss_connection().await; let admin = connection.get_admin().expect("Failed to get admin"); @@ -1423,7 +1440,6 @@ mod table_test { .create_writer() .expect("Failed to create writer"); - // Row 1: id=1, tags=["hello", "world"], scores=[10, 20, 30], matrix=[[1,2],[3,4]] let mut row1 = GenericRow::new(4); row1.set_field(0, 1_i32); row1.set_field(1, make_string_array(&[Some("hello"), Some("world")])); @@ -1436,7 +1452,6 @@ mod table_test { }; row1.set_field(3, m1); - // Row 2: id=2, tags=[null], scores=[], matrix=[[5], null, []] let mut row2 = GenericRow::new(4); row2.set_field(0, 2_i32); row2.set_field(1, make_string_array(&[None])); @@ -1450,7 +1465,6 @@ mod table_test { }; row2.set_field(3, m2); - // Row 3: id=3, tags=null, scores=[42], matrix=null let mut row3 = GenericRow::new(4); row3.set_field(0, 3_i32); row3.set_field(1, Datum::Null); @@ -1465,7 +1479,6 @@ mod table_test { let records = scan_table(&table, |scan| scan).await; assert_eq!(records.len(), 3, "expected three log records"); - // Verify row 1: populated flat arrays + nested array let r0 = records[0].row(); assert_eq!(r0.get_int(0).unwrap(), 1); let tags_r0 = r0.get_array(1).unwrap(); @@ -1488,7 +1501,6 @@ mod table_test { assert_eq!(mr0_1.get_int(0).unwrap(), 3); assert_eq!(mr0_1.get_int(1).unwrap(), 4); - // Verify row 2: null element in array, empty array, nested with mixed inner let r1 = records[1].row(); assert_eq!(r1.get_int(0).unwrap(), 2); let tags_r1 = r1.get_array(1).unwrap(); @@ -1505,7 +1517,6 @@ mod table_test { let mr1_2 = matrix_r1.get_array(2).unwrap(); assert_eq!(mr1_2.size(), 0); - // Verify row 3: null flat column, null nested column let r2 = records[2].row(); assert_eq!(r2.get_int(0).unwrap(), 3); assert!(r2.is_null_at(1).unwrap()); @@ -1521,10 +1532,454 @@ mod table_test { } #[tokio::test] - async fn append_and_scan_with_array_rich_types() { - use fluss::row::binary_array::FlussArrayWriter; - use fluss::row::{Date, Decimal, FlussArray, GenericRow, Time, TimestampNtz}; + async fn append_and_scan_with_array_of_row() { + use fluss::metadata::{DataField, DataType}; + + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_log_array_of_row"); + + let event_row_type_owned = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let array_of_row_type = DataTypes::array(event_row_type_owned.clone()); + + let event_row_type = match &event_row_type_owned { + DataType::Row(rt) => rt.clone(), + _ => unreachable!(), + }; + + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("events", array_of_row_type) + .build() + .expect("Failed to build schema"); + + let table_descriptor = TableDescriptor::builder() + .schema(schema) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + + let append_writer = table + .new_append() + .expect("Failed to create append") + .create_writer() + .expect("Failed to create writer"); + + let mut events1 = FlussArrayWriter::new(2, &event_row_type_owned); + let mut e0 = GenericRow::new(2); + e0.set_field(0, 1_i32); + e0.set_field(1, "open"); + events1.write_row(0, &e0).expect("write e0"); + let mut e1 = GenericRow::new(2); + e1.set_field(0, 2_i32); + e1.set_field(1, "close"); + events1.write_row(1, &e1).expect("write e1"); + let events1 = events1.complete().expect("events1"); + + let mut row1 = GenericRow::new(2); + row1.set_field(0, 1_i32); + row1.set_field(1, events1); + + let mut events2 = FlussArrayWriter::new(3, &event_row_type_owned); + let mut e2 = GenericRow::new(2); + e2.set_field(0, 7_i32); + e2.set_field(1, "x"); + events2.write_row(0, &e2).expect("write e2"); + events2.set_null_at(1); + let mut e3 = GenericRow::new(2); + e3.set_field(0, 8_i32); + e3.set_field(1, "y"); + events2.write_row(2, &e3).expect("write e3"); + let events2 = events2.complete().expect("events2"); + + let mut row2 = GenericRow::new(2); + row2.set_field(0, 2_i32); + row2.set_field(1, events2); + + let mut row3 = GenericRow::new(2); + row3.set_field(0, 3_i32); + row3.set_field(1, Datum::Null); + + append_writer.append(&row1).expect("append row1"); + append_writer.append(&row2).expect("append row2"); + append_writer.append(&row3).expect("append row3"); + append_writer.flush().await.expect("Failed to flush"); + + let records = scan_table(&table, |scan| scan).await; + assert_eq!(records.len(), 3, "expected three log records"); + + let r0 = records[0].row(); + assert_eq!(r0.get_int(0).unwrap(), 1); + let events_r0 = r0.get_array(1).unwrap(); + assert_eq!(events_r0.size(), 2); + let e0_r0 = events_r0.get_row(0, &event_row_type).unwrap(); + assert_eq!(e0_r0.get_int(0).unwrap(), 1); + assert_eq!(e0_r0.get_string(1).unwrap(), "open"); + let e1_r0 = events_r0.get_row(1, &event_row_type).unwrap(); + assert_eq!(e1_r0.get_int(0).unwrap(), 2); + assert_eq!(e1_r0.get_string(1).unwrap(), "close"); + + let r1 = records[1].row(); + let events_r1 = r1.get_array(1).unwrap(); + assert_eq!(events_r1.size(), 3); + let e0_r1 = events_r1.get_row(0, &event_row_type).unwrap(); + assert_eq!(e0_r1.get_int(0).unwrap(), 7); + assert_eq!(e0_r1.get_string(1).unwrap(), "x"); + assert!(events_r1.is_null_at(1)); + let e2_r1 = events_r1.get_row(2, &event_row_type).unwrap(); + assert_eq!(e2_r1.get_int(0).unwrap(), 8); + assert_eq!(e2_r1.get_string(1).unwrap(), "y"); + + let r2 = records[2].row(); + assert_eq!(r2.get_int(0).unwrap(), 3); + assert!(r2.is_null_at(1).unwrap()); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + #[tokio::test] + async fn append_and_scan_with_row() { + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_log_rows"); + let nested_row_type = DataTypes::row(vec![ + DataField::new("x", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + let deep_inner_row_type = DataTypes::row(vec![DataField::new("n", DataTypes::int(), None)]); + let deep_row_type = + DataTypes::row(vec![DataField::new("inner", deep_inner_row_type, None)]); + + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("nested", nested_row_type) + .column("deep", deep_row_type) + .build() + .expect("Failed to build schema"); + + let table_descriptor = TableDescriptor::builder() + .schema(schema) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + + let append_writer = table + .new_append() + .expect("Failed to create append") + .create_writer() + .expect("Failed to create writer"); + + let mut nested1 = GenericRow::new(2); + nested1.set_field(0, 42_i32); + nested1.set_field(1, "hello"); + let mut deep_inner1 = GenericRow::new(1); + deep_inner1.set_field(0, 99_i32); + let mut deep1 = GenericRow::new(1); + deep1.set_field(0, Datum::Row(Box::new(deep_inner1))); + + let mut row1 = GenericRow::new(3); + row1.set_field(0, 1_i32); + row1.set_field(1, Datum::Row(Box::new(nested1))); + row1.set_field(2, Datum::Row(Box::new(deep1))); + + let mut nested2 = GenericRow::new(2); + nested2.set_field(0, 7_i32); + nested2.set_field(1, Datum::Null); + + let mut row2 = GenericRow::new(3); + row2.set_field(0, 2_i32); + row2.set_field(1, Datum::Row(Box::new(nested2))); + row2.set_field(2, Datum::Null); + + let mut deep_inner3 = GenericRow::new(1); + deep_inner3.set_field(0, -1_i32); + let mut deep3 = GenericRow::new(1); + deep3.set_field(0, Datum::Row(Box::new(deep_inner3))); + + let mut row3 = GenericRow::new(3); + row3.set_field(0, 3_i32); + row3.set_field(1, Datum::Null); + row3.set_field(2, Datum::Row(Box::new(deep3))); + + append_writer.append(&row1).expect("append row1"); + append_writer.append(&row2).expect("append row2"); + append_writer.append(&row3).expect("append row3"); + append_writer.flush().await.expect("Failed to flush"); + + let records = scan_table(&table, |scan| scan).await; + assert_eq!(records.len(), 3, "expected three log records"); + + let r0 = records[0].row(); + assert_eq!(r0.get_int(0).unwrap(), 1); + let nested_r0 = r0.get_row(1).unwrap(); + assert_eq!(nested_r0.get_int(0).unwrap(), 42); + assert_eq!(nested_r0.get_string(1).unwrap(), "hello"); + let deep_r0 = r0.get_row(2).unwrap(); + let deep_inner_r0 = deep_r0.get_row(0).unwrap(); + assert_eq!(deep_inner_r0.get_int(0).unwrap(), 99); + + let r1 = records[1].row(); + assert_eq!(r1.get_int(0).unwrap(), 2); + let nested_r1 = r1.get_row(1).unwrap(); + assert_eq!(nested_r1.get_int(0).unwrap(), 7); + assert!(nested_r1.is_null_at(1).unwrap()); + assert!(r1.is_null_at(2).unwrap()); + + let r2 = records[2].row(); + assert_eq!(r2.get_int(0).unwrap(), 3); + assert!(r2.is_null_at(1).unwrap()); + let deep_r2 = r2.get_row(2).unwrap(); + let deep_inner_r2 = deep_r2.get_row(0).unwrap(); + assert_eq!(deep_inner_r2.get_int(0).unwrap(), -1); + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + /// Partitioned log table with a ROW column. Confirms partition routing + /// + ROW column encoding compose correctly across partitions. + /// ROW column with all rich element types (decimal, date, time, timestamps, + /// bytes, binary, float NaN/Inf, long strings) round-tripped through the + /// log path. Confirms the wire-level encoding of `ROW` matches + /// what the server expects — the unit-level `test_row_all_primitives_round_trip` + /// proves Rust↔Rust round-trip; this test proves Rust→server→Rust. + #[tokio::test] + async fn append_and_scan_with_row_rich_types() { + fn assert_f32_special(actual: f32, expected: f32) { + if expected.is_nan() { + assert!(actual.is_nan(), "expected NaN"); + } else if expected.is_infinite() { + assert!(actual.is_infinite()); + assert_eq!(actual.signum(), expected.signum()); + } else { + assert!((actual - expected).abs() < f32::EPSILON); + } + } + + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_log_row_rich_types"); + + let row_type_owned = DataTypes::row(vec![ + DataField::new("f_bool", DataTypes::boolean(), None), + DataField::new("f_int", DataTypes::int(), None), + DataField::new("f_long", DataTypes::bigint(), None), + DataField::new("f_float", DataTypes::float(), None), + DataField::new("f_double", DataTypes::double(), None), + DataField::new("f_str", DataTypes::string(), None), + DataField::new("f_bytes", DataTypes::bytes(), None), + DataField::new("f_decimal", DataTypes::decimal(10, 2), None), + DataField::new("f_date", DataTypes::date(), None), + DataField::new("f_time", DataTypes::time_with_precision(3), None), + DataField::new("f_ts_ntz", DataTypes::timestamp_with_precision(6), None), + DataField::new("f_ts_ltz", DataTypes::timestamp_ltz_with_precision(6), None), + DataField::new("f_binary_fixed", DataTypes::binary(4), None), + DataField::new("f_array_int", DataTypes::array(DataTypes::int()), None), + ]); + + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("nested", row_type_owned) + .build() + .expect("Failed to build schema"); + + let table_descriptor = TableDescriptor::builder() + .schema(schema) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + let append_writer = table + .new_append() + .expect("Failed to create append") + .create_writer() + .expect("Failed to create writer"); + + let mut nested1 = GenericRow::new(14); + nested1.set_field(0, true); + nested1.set_field(1, 100_000_i32); + nested1.set_field(2, 9_876_543_210_i64); + nested1.set_field(3, f32::INFINITY); + nested1.set_field(4, f64::NAN); + nested1.set_field(5, "hello world"); + nested1.set_field(6, b"binary".as_slice()); + nested1.set_field(7, Decimal::from_unscaled_long(12345, 10, 2).unwrap()); + nested1.set_field(8, Datum::Date(Date::new(20476))); + nested1.set_field(9, Datum::Time(Time::new(36_827_123))); + nested1.set_field( + 10, + Datum::TimestampNtz(TimestampNtz::new(1_769_163_227_123)), + ); + nested1.set_field( + 11, + Datum::TimestampLtz(TimestampLtz::new(1_769_163_227_456)), + ); + nested1.set_field(12, b"\x01\x02\x03\x04".as_slice()); + nested1.set_field(13, make_int_array(&[Some(7), None, Some(11)])); + + let mut row1 = GenericRow::new(2); + row1.set_field(0, 1_i32); + row1.set_field(1, Datum::Row(Box::new(nested1))); + + let mut row2 = GenericRow::new(2); + row2.set_field(0, 2_i32); + row2.set_field(1, Datum::Null); + + append_writer.append(&row1).expect("append row1"); + append_writer.append(&row2).expect("append row2"); + append_writer.flush().await.expect("Failed to flush"); + + let records = scan_table(&table, |scan| scan).await; + assert_eq!(records.len(), 2); + + let r0 = records[0].row(); + assert_eq!(r0.get_int(0).unwrap(), 1); + let nested = r0.get_row(1).unwrap(); + assert!(nested.get_boolean(0).unwrap()); + assert_eq!(nested.get_int(1).unwrap(), 100_000); + assert_eq!(nested.get_long(2).unwrap(), 9_876_543_210); + assert_f32_special(nested.get_float(3).unwrap(), f32::INFINITY); + assert!(nested.get_double(4).unwrap().is_nan()); + assert_eq!(nested.get_string(5).unwrap(), "hello world"); + assert_eq!(nested.get_bytes(6).unwrap(), b"binary"); + assert_eq!( + nested.get_decimal(7, 10, 2).unwrap(), + Decimal::from_unscaled_long(12345, 10, 2).unwrap(), + ); + assert_eq!(nested.get_date(8).unwrap().get_inner(), 20476); + assert_eq!(nested.get_time(9).unwrap().get_inner(), 36_827_123); + assert_eq!( + nested.get_timestamp_ntz(10, 6).unwrap().get_millisecond(), + 1_769_163_227_123, + ); + assert_eq!( + nested + .get_timestamp_ltz(11, 6) + .unwrap() + .get_epoch_millisecond(), + 1_769_163_227_456, + ); + assert_eq!(nested.get_binary(12, 4).unwrap(), b"\x01\x02\x03\x04"); + let arr = nested.get_array(13).unwrap(); + assert_eq!(arr.size(), 3); + assert_eq!(arr.get_int(0).unwrap(), 7); + assert!(arr.is_null_at(1)); + assert_eq!(arr.get_int(2).unwrap(), 11); + + let r1 = records[1].row(); + assert_eq!(r1.get_int(0).unwrap(), 2); + assert!(r1.is_null_at(1).unwrap()); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + /// Projection over a log table with ROW columns. Specifically tests that + /// `ProjectedRow::get_row` (added by this PR) works end-to-end against the + /// server — without this, the projection code path for ROW would have zero + /// integration coverage. + #[tokio::test] + async fn append_and_scan_with_row_projection() { + let cluster = get_shared_cluster(); + let connection = cluster.get_fluss_connection().await; + let admin = connection.get_admin().expect("Failed to get admin"); + + let table_path = TablePath::new("fluss", "test_log_row_projection"); + + let row_type = DataTypes::row(vec![ + DataField::new("seq", DataTypes::int(), None), + DataField::new("label", DataTypes::string(), None), + ]); + + let schema = Schema::builder() + .column("id", DataTypes::int()) + .column("nested", row_type) + .column("extra", DataTypes::string()) + .build() + .expect("Failed to build schema"); + + let table_descriptor = TableDescriptor::builder() + .schema(schema) + .build() + .expect("Failed to build table descriptor"); + + create_table(&admin, &table_path, &table_descriptor).await; + + let table = connection + .get_table(&table_path) + .await + .expect("Failed to get table"); + let append_writer = table + .new_append() + .expect("Failed to create append") + .create_writer() + .expect("Failed to create writer"); + + let mut nested = GenericRow::new(2); + nested.set_field(0, 42_i32); + nested.set_field(1, "hello"); + + let mut row = GenericRow::new(3); + row.set_field(0, 7_i32); + row.set_field(1, Datum::Row(Box::new(nested))); + row.set_field(2, "ignore-me"); + append_writer.append(&row).expect("append"); + append_writer.flush().await.expect("Failed to flush"); + + let records = scan_table(&table, |scan| { + scan.project_by_name(&["nested", "id"]) + .expect("project failed") + }) + .await; + assert_eq!(records.len(), 1); + + let r0 = records[0].row(); + let projected_nested = r0.get_row(0).expect("get_row over projection"); + assert_eq!(projected_nested.get_int(0).unwrap(), 42); + assert_eq!(projected_nested.get_string(1).unwrap(), "hello"); + assert_eq!(r0.get_int(1).unwrap(), 7); + + admin + .drop_table(&table_path, false) + .await + .expect("Failed to drop table"); + } + + #[tokio::test] + async fn append_and_scan_with_array_rich_types() { fn assert_f32_special(actual: f32, expected: f32) { if expected.is_nan() { assert!(actual.is_nan(), "expected NaN");