Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(arrow): improve the arrow conversion trait #16567

Merged
merged 9 commits into from
May 8, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 2 additions & 7 deletions src/batch/src/executor/iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,13 @@
// limitations under the License.

use std::hash::{DefaultHasher, Hash, Hasher};
use std::sync::Arc;

use anyhow::anyhow;
use arrow_array::RecordBatch;
use futures_async_stream::try_stream;
use futures_util::stream::StreamExt;
use icelake::io::{FileScan, TableScan};
use risingwave_common::array::arrow::{FromArrow, IcebergArrowConvert};
use risingwave_common::catalog::Schema;
use risingwave_connector::sink::iceberg::IcebergConfig;

Expand Down Expand Up @@ -150,12 +150,7 @@ impl IcebergScanExecutor {
}

fn record_batch_to_chunk(record_batch: RecordBatch) -> Result<DataChunk, BatchError> {
let mut columns = Vec::with_capacity(record_batch.num_columns());
for array in record_batch.columns() {
let column = Arc::new(array.try_into()?);
columns.push(column);
}
Ok(DataChunk::new(columns, record_batch.num_rows()))
Ok(IcebergArrowConvert.from_record_batch(&record_batch)?)
}
}

Expand Down
30 changes: 0 additions & 30 deletions src/common/src/array/arrow/arrow_default.rs

This file was deleted.

211 changes: 17 additions & 194 deletions src/common/src/array/arrow/arrow_deltalake.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,25 +21,29 @@ use std::ops::{Div, Mul};
use std::sync::Arc;

use arrow_array::ArrayRef;
use arrow_schema::DataType;
use itertools::Itertools;
use num_traits::abs;
use {
arrow_array_deltalake as arrow_array, arrow_buffer_deltalake as arrow_buffer,
arrow_cast_deltalake as arrow_cast, arrow_schema_deltalake as arrow_schema,
};

use self::arrow_impl::ToArrowArrayWithTypeConvert;
use crate::array::arrow::arrow_deltalake::arrow_impl::FromIntoArrow;
use crate::array::{Array, ArrayError, ArrayImpl, DataChunk, Decimal, DecimalArray, ListArray};
use crate::util::iter_util::ZipEqFast;
use self::arrow_impl::ToArrow;
use crate::array::{Array, ArrayError, DataChunk, Decimal, DecimalArray};
#[expect(clippy::duplicate_mod)]
#[path = "./arrow_impl.rs"]
mod arrow_impl;

struct DeltaLakeConvert;
pub struct DeltaLakeConvert;

impl DeltaLakeConvert {
pub fn to_record_batch(
&self,
schema: arrow_schema::SchemaRef,
chunk: &DataChunk,
) -> Result<arrow_array::RecordBatch, ArrayError> {
ToArrow::to_record_batch(self, schema, chunk)
}

fn decimal_to_i128(decimal: Decimal, precision: u8, max_scale: i8) -> Option<i128> {
match decimal {
crate::array::Decimal::Normalized(e) => {
Expand Down Expand Up @@ -67,7 +71,7 @@ impl DeltaLakeConvert {
}
}

impl arrow_impl::ToArrowArrayWithTypeConvert for DeltaLakeConvert {
impl ToArrow for DeltaLakeConvert {
fn decimal_to_arrow(
&self,
data_type: &arrow_schema::DataType,
Expand All @@ -89,189 +93,6 @@ impl arrow_impl::ToArrowArrayWithTypeConvert for DeltaLakeConvert {
.map_err(ArrayError::from_arrow)?;
Ok(Arc::new(array) as ArrayRef)
}

#[inline]
fn list_to_arrow(
&self,
data_type: &arrow_schema::DataType,
array: &ListArray,
) -> Result<arrow_array::ArrayRef, ArrayError> {
use arrow_array::builder::*;
fn build<A, B, F>(
array: &ListArray,
a: &A,
builder: B,
mut append: F,
) -> arrow_array::ListArray
where
A: Array,
B: arrow_array::builder::ArrayBuilder,
F: FnMut(&mut B, Option<A::RefItem<'_>>),
{
let mut builder = ListBuilder::with_capacity(builder, a.len());
for i in 0..array.len() {
for j in array.offsets[i]..array.offsets[i + 1] {
append(builder.values(), a.value_at(j as usize));
}
builder.append(!array.is_null(i));
}
builder.finish()
}
let inner_type = match data_type {
arrow_schema::DataType::List(inner) => inner.data_type(),
_ => return Err(ArrayError::to_arrow("Invalid list type")),
};
let arr: arrow_array::ListArray = match &*array.value {
ArrayImpl::Int16(a) => build(array, a, Int16Builder::with_capacity(a.len()), |b, v| {
b.append_option(v)
}),
ArrayImpl::Int32(a) => build(array, a, Int32Builder::with_capacity(a.len()), |b, v| {
b.append_option(v)
}),
ArrayImpl::Int64(a) => build(array, a, Int64Builder::with_capacity(a.len()), |b, v| {
b.append_option(v)
}),

ArrayImpl::Float32(a) => {
build(array, a, Float32Builder::with_capacity(a.len()), |b, v| {
b.append_option(v.map(|f| f.0))
})
}
ArrayImpl::Float64(a) => {
build(array, a, Float64Builder::with_capacity(a.len()), |b, v| {
b.append_option(v.map(|f| f.0))
})
}
ArrayImpl::Utf8(a) => build(
array,
a,
StringBuilder::with_capacity(a.len(), a.data().len()),
|b, v| b.append_option(v),
),
ArrayImpl::Int256(a) => build(
array,
a,
Decimal256Builder::with_capacity(a.len()).with_data_type(
arrow_schema::DataType::Decimal256(arrow_schema::DECIMAL256_MAX_PRECISION, 0),
),
|b, v| b.append_option(v.map(Into::into)),
),
ArrayImpl::Bool(a) => {
build(array, a, BooleanBuilder::with_capacity(a.len()), |b, v| {
b.append_option(v)
})
}
ArrayImpl::Decimal(a) => {
let (precision, max_scale) = match inner_type {
arrow_schema::DataType::Decimal128(precision, scale) => (*precision, *scale),
_ => return Err(ArrayError::to_arrow("Invalid decimal type")),
};
build(
array,
a,
Decimal128Builder::with_capacity(a.len())
.with_data_type(DataType::Decimal128(precision, max_scale)),
|b, v| {
let v = v.and_then(|v| {
DeltaLakeConvert::decimal_to_i128(v, precision, max_scale)
});
b.append_option(v);
},
)
}
ArrayImpl::Interval(a) => build(
array,
a,
IntervalMonthDayNanoBuilder::with_capacity(a.len()),
|b, v| b.append_option(v.map(|d| d.into_arrow())),
),
ArrayImpl::Date(a) => build(array, a, Date32Builder::with_capacity(a.len()), |b, v| {
b.append_option(v.map(|d| d.into_arrow()))
}),
ArrayImpl::Timestamp(a) => build(
array,
a,
TimestampMicrosecondBuilder::with_capacity(a.len()),
|b, v| b.append_option(v.map(|d| d.into_arrow())),
),
ArrayImpl::Timestamptz(a) => build(
array,
a,
TimestampMicrosecondBuilder::with_capacity(a.len()),
|b, v| b.append_option(v.map(|d| d.into_arrow())),
),
ArrayImpl::Time(a) => build(
array,
a,
Time64MicrosecondBuilder::with_capacity(a.len()),
|b, v| b.append_option(v.map(|d| d.into_arrow())),
),
ArrayImpl::Jsonb(a) => build(
array,
a,
LargeStringBuilder::with_capacity(a.len(), a.len() * 16),
|b, v| b.append_option(v.map(|j| j.to_string())),
),
ArrayImpl::Serial(_) => todo!("list of serial"),
ArrayImpl::Struct(a) => {
let values = Arc::new(arrow_array::StructArray::try_from(a)?);
arrow_array::ListArray::new(
Arc::new(arrow_schema::Field::new(
"item",
a.data_type().try_into()?,
true,
)),
arrow_buffer::OffsetBuffer::new(arrow_buffer::ScalarBuffer::from(
array
.offsets()
.iter()
.map(|o| *o as i32)
.collect::<Vec<i32>>(),
)),
values,
Some(array.null_bitmap().into()),
)
}
ArrayImpl::List(_) => todo!("list of list"),
ArrayImpl::Bytea(a) => build(
array,
a,
BinaryBuilder::with_capacity(a.len(), a.data().len()),
|b, v| b.append_option(v),
),
};
Ok(Arc::new(arr))
}
}

/// Converts RisingWave array to Arrow array with the schema.
/// This function will try to convert the array if the type is not same with the schema.
pub fn to_deltalake_record_batch_with_schema(
schema: arrow_schema::SchemaRef,
chunk: &DataChunk,
) -> Result<arrow_array::RecordBatch, ArrayError> {
if !chunk.is_compacted() {
let c = chunk.clone();
return to_deltalake_record_batch_with_schema(schema, &c.compact());
}
let columns: Vec<_> = chunk
.columns()
.iter()
.zip_eq_fast(schema.fields().iter())
.map(|(column, field)| {
let column: arrow_array::ArrayRef =
DeltaLakeConvert.to_arrow_with_type(field.data_type(), column)?;
if column.data_type() == field.data_type() {
Ok(column)
} else {
arrow_cast::cast(&column, field.data_type()).map_err(ArrayError::from_arrow)
}
})
.try_collect::<_, _, ArrayError>()?;

let opts = arrow_array::RecordBatchOptions::default().with_row_count(Some(chunk.capacity()));
arrow_array::RecordBatch::try_new_with_options(schema, columns, &opts)
.map_err(ArrayError::to_arrow)
}

#[cfg(test)]
Expand All @@ -283,6 +104,7 @@ mod test {
use arrow_schema::Field;
use {arrow_array_deltalake as arrow_array, arrow_schema_deltalake as arrow_schema};

use crate::array::arrow::arrow_deltalake::DeltaLakeConvert;
use crate::array::{ArrayImpl, Decimal, DecimalArray, ListArray, ListValue};
use crate::buffer::Bitmap;

Expand All @@ -304,13 +126,14 @@ mod test {
arrow_schema::DataType::List(Arc::new(Field::new(
"test",
arrow_schema::DataType::Decimal128(10, 0),
false,
true,
))),
false,
)]);

let record_batch =
super::to_deltalake_record_batch_with_schema(Arc::new(schema), &chunk).unwrap();
let record_batch = DeltaLakeConvert
.to_record_batch(Arc::new(schema), &chunk)
.unwrap();
let expect_array = Arc::new(
arrow_array::Decimal128Array::from(vec![
None,
Expand Down
Loading
Loading