Skip to content

Commit

Permalink
Fix FFI and add support for Struct type (#287)
Browse files Browse the repository at this point in the history
* fix: support nested types in FFI

Ported from https://github.com/jorgecarleitao/arrow2

Fix #20
Fix #251

Signed-off-by: roee88 <[email protected]>

* Removed Clone from FFI_ArrowArray

Signed-off-by: roee88 <[email protected]>

* Add nesting to FFI struct test

Signed-off-by: roee88 <[email protected]>
roee88 authored May 17, 2021

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature. The key has expired.
1 parent de44c8c commit c863a2c
Showing 2 changed files with 387 additions and 341 deletions.
122 changes: 42 additions & 80 deletions arrow/src/array/ffi.rs
Original file line number Diff line number Diff line change
@@ -22,111 +22,40 @@ use std::convert::TryFrom;
use crate::{
error::{ArrowError, Result},
ffi,
ffi::ArrowArrayRef,
};

use super::ArrayData;
use crate::datatypes::DataType;
use crate::ffi::ArrowArray;

impl TryFrom<ffi::ArrowArray> for ArrayData {
type Error = ArrowError;

fn try_from(value: ffi::ArrowArray) -> Result<Self> {
let child_data = value.children()?;

let child_type = if !child_data.is_empty() {
Some(child_data[0].data_type().clone())
} else {
None
};

let data_type = value.data_type(child_type)?;

let len = value.len();
let offset = value.offset();
let null_count = value.null_count();
let buffers = value.buffers()?;
let null_bit_buffer = value.null_bit_buffer();

Ok(ArrayData::new(
data_type,
len,
Some(null_count),
null_bit_buffer,
offset,
buffers,
child_data,
))
value.to_data()
}
}

impl TryFrom<ArrayData> for ffi::ArrowArray {
type Error = ArrowError;

fn try_from(value: ArrayData) -> Result<Self> {
// If parent is nullable, then children also must be nullable
// so we pass this nullable to the creation of hte child data
let nullable = match value.data_type() {
DataType::List(field) => field.is_nullable(),
DataType::LargeList(field) => field.is_nullable(),
_ => false,
};

let len = value.len();
let offset = value.offset() as usize;
let null_count = value.null_count();
let buffers = value.buffers().to_vec();
let null_buffer = value.null_buffer().cloned();
let child_data = value
.child_data()
.iter()
.map(|arr| {
let len = arr.len();
let offset = arr.offset() as usize;
let null_count = arr.null_count();
let buffers = arr.buffers().to_vec();
let null_buffer = arr.null_buffer().cloned();

// Note: the nullable comes from the parent data.
unsafe {
ArrowArray::try_new(
arr.data_type(),
len,
null_count,
null_buffer,
offset,
buffers,
vec![],
nullable,
)
.expect("infallible")
}
})
.collect::<Vec<_>>();

unsafe {
ffi::ArrowArray::try_new(
value.data_type(),
len,
null_count,
null_buffer,
offset,
buffers,
child_data,
nullable,
)
}
unsafe { ffi::ArrowArray::try_new(value) }
}
}

#[cfg(test)]
mod tests {
use crate::error::Result;
use crate::{
array::{Array, ArrayData, Int64Array, UInt32Array, UInt64Array},
array::{
Array, ArrayData, BooleanArray, Int64Array, StructArray, UInt32Array,
UInt64Array,
},
datatypes::{DataType, Field},
ffi::ArrowArray,
};
use std::convert::TryFrom;
use std::sync::Arc;

fn test_round_trip(expected: &ArrayData) -> Result<()> {
// create a `ArrowArray` from the data.
@@ -165,4 +94,37 @@ mod tests {
let data = array.data();
test_round_trip(data)
}

#[test]
fn test_struct() -> Result<()> {
let inner = StructArray::from(vec![
(
Field::new("a1", DataType::Boolean, false),
Arc::new(BooleanArray::from(vec![true, true, false, false]))
as Arc<dyn Array>,
),
(
Field::new("a2", DataType::UInt32, false),
Arc::new(UInt32Array::from(vec![1, 2, 3, 4])),
),
]);

let array = StructArray::from(vec![
(
Field::new("a", inner.data_type().clone(), false),
Arc::new(inner) as Arc<dyn Array>,
),
(
Field::new("b", DataType::Boolean, false),
Arc::new(BooleanArray::from(vec![false, false, true, true]))
as Arc<dyn Array>,
),
(
Field::new("c", DataType::UInt32, false),
Arc::new(UInt32Array::from(vec![42, 28, 19, 31])),
),
]);
let data = array.data();
test_round_trip(data)
}
}
606 changes: 345 additions & 261 deletions arrow/src/ffi.rs
Original file line number Diff line number Diff line change
@@ -77,12 +77,10 @@ To export an array, create an `ArrowArray` using [ArrowArray::try_new].
*/

use std::{
convert::TryFrom,
ffi::CStr,
ffi::CString,
iter,
mem::{size_of, ManuallyDrop},
os::raw::c_char,
mem::size_of,
ptr::{self, NonNull},
sync::Arc,
};
@@ -93,6 +91,12 @@ use crate::datatypes::{DataType, Field, TimeUnit};
use crate::error::{ArrowError, Result};
use crate::util::bit_util;

#[allow(dead_code)]
struct SchemaPrivateData {
field: Field,
children_ptr: Box<[*mut FFI_ArrowSchema]>,
}

/// ABI-compatible struct for `ArrowSchema` from C Data Interface
/// See <https://arrow.apache.org/docs/format/CDataInterface.html#structure-definitions>
/// This was created by bindgen
@@ -112,45 +116,65 @@ pub struct FFI_ArrowSchema {

// callback used to drop [FFI_ArrowSchema] when it is exported.
unsafe extern "C" fn release_schema(schema: *mut FFI_ArrowSchema) {
if schema.is_null() {
return;
}
let schema = &mut *schema;

// take ownership back to release it.
CString::from_raw(schema.format as *mut std::os::raw::c_char);
CString::from_raw(schema.name as *mut std::os::raw::c_char);
let private = Box::from_raw(schema.private_data as *mut SchemaPrivateData);
for child in private.children_ptr.iter() {
let _ = Box::from_raw(*child);
}

schema.release = None;
}

struct SchemaPrivateData {
children: Box<[*mut FFI_ArrowSchema]>,
}

impl FFI_ArrowSchema {
/// create a new [FFI_ArrowSchema] from a format.
fn new(
format: &str,
children: Vec<*mut FFI_ArrowSchema>,
nullable: bool,
) -> FFI_ArrowSchema {
let children = children.into_boxed_slice();
let n_children = children.len() as i64;
let children_ptr = children.as_ptr() as *mut *mut FFI_ArrowSchema;

let flags = if nullable { 2 } else { 0 };
/// create a new [`Ffi_ArrowSchema`]. This fails if the fields' [`DataType`] is not supported.
fn try_new(field: Field) -> Result<FFI_ArrowSchema> {
let format = to_format(field.data_type())?;
let name = field.name().clone();

// allocate (and hold) the children
let children_vec = match field.data_type() {
DataType::List(field) => {
vec![Box::new(FFI_ArrowSchema::try_new(field.as_ref().clone())?)]
}
DataType::LargeList(field) => {
vec![Box::new(FFI_ArrowSchema::try_new(field.as_ref().clone())?)]
}
DataType::Struct(fields) => fields
.iter()
.map(|field| Ok(Box::new(FFI_ArrowSchema::try_new(field.clone())?)))
.collect::<Result<Vec<_>>>()?,
_ => vec![],
};
// note: this cannot be done along with the above because the above is fallible and this op leaks.
let mut children_ptr = children_vec
.into_iter()
.map(Box::into_raw)
.collect::<Box<_>>();
let n_children = children_ptr.len() as i64;
let children = children_ptr.as_mut_ptr();

let private_data = Box::new(SchemaPrivateData { children });
// <https://arrow.apache.org/docs/format/CDataInterface.html#c.ArrowSchema>
FFI_ArrowSchema {
Ok(FFI_ArrowSchema {
format: CString::new(format).unwrap().into_raw(),
// For child data a non null string is expected and is called item
name: CString::new("item").unwrap().into_raw(),
name: CString::new(name).unwrap().into_raw(),
metadata: std::ptr::null_mut(),
flags,
flags: field.is_nullable() as i64 * 2,
n_children,
children: children_ptr,
children,
dictionary: std::ptr::null_mut(),
release: Some(release_schema),
private_data: Box::into_raw(private_data) as *mut ::std::os::raw::c_void,
}
private_data: Box::into_raw(Box::new(SchemaPrivateData {
field,
children_ptr,
})) as *mut ::std::os::raw::c_void,
})
}

/// create an empty [FFI_ArrowSchema]
@@ -170,10 +194,29 @@ impl FFI_ArrowSchema {

/// returns the format of this schema.
pub fn format(&self) -> &str {
assert!(!self.format.is_null());
// safe because the lifetime of `self.format` equals `self`
unsafe { CStr::from_ptr(self.format) }
.to_str()
.expect("The external API has a non-utf8 as format")
}

/// returns the name of this schema.
pub fn name(&self) -> &str {
assert!(!self.name.is_null());
// safe because the lifetime of `self.name` equals `self`
unsafe { CStr::from_ptr(self.name) }.to_str().unwrap()
}

pub fn child(&self, index: usize) -> &Self {
assert!(index < self.n_children as usize);
assert!(!self.name.is_null());
unsafe { self.children.add(index).as_ref().unwrap().as_ref().unwrap() }
}

pub fn nullable(&self) -> bool {
(self.flags / 2) & 1 == 1
}
}

impl Drop for FFI_ArrowSchema {
@@ -185,14 +228,9 @@ impl Drop for FFI_ArrowSchema {
}
}

/// maps a DataType `format` to a [DataType](arrow::datatypes::DataType).
/// See https://arrow.apache.org/docs/format/CDataInterface.html#data-type-description-format-strings
fn to_datatype(
format: &str,
child_type: Option<DataType>,
schema: &FFI_ArrowSchema,
) -> Result<DataType> {
Ok(match format {
fn to_field(schema: &FFI_ArrowSchema) -> Result<Field> {
let data_type = match schema.format() {
"n" => DataType::Null,
"b" => DataType::Boolean,
"c" => DataType::Int8,
@@ -216,54 +254,33 @@ fn to_datatype(
"ttm" => DataType::Time32(TimeUnit::Millisecond),
"ttu" => DataType::Time64(TimeUnit::Microsecond),
"ttn" => DataType::Time64(TimeUnit::Nanosecond),

// Note: The datatype null will only be created when called from ArrowArray::buffer_len
// at that point the child data is not yet known, but it is also not required to determine
// the buffer length of the list arrays.
"+l" => {
let nullable = schema.flags == 2;
// Safety
// Should be set as this is expected from the C FFI definition
debug_assert!(!schema.name.is_null());
let name = unsafe { CString::from_raw(schema.name as *mut c_char) }
.into_string()
.unwrap();
// prevent a double free
let name = ManuallyDrop::new(name);
DataType::List(Box::new(Field::new(
&name,
child_type.unwrap_or(DataType::Null),
nullable,
)))
let child = schema.child(0);
DataType::List(Box::new(to_field(child)?))
}
"+L" => {
let nullable = schema.flags == 2;
// Safety
// Should be set as this is expected from the C FFI definition
debug_assert!(!schema.name.is_null());
let name = unsafe { CString::from_raw(schema.name as *mut c_char) }
.into_string()
.unwrap();
// prevent a double free
let name = ManuallyDrop::new(name);
DataType::LargeList(Box::new(Field::new(
&name,
child_type.unwrap_or(DataType::Null),
nullable,
)))
let child = schema.child(0);
DataType::LargeList(Box::new(to_field(child)?))
}
dt => {
"+s" => {
let children = (0..schema.n_children as usize)
.map(|x| to_field(schema.child(x)))
.collect::<Result<Vec<_>>>()?;
DataType::Struct(children)
}
other => {
return Err(ArrowError::CDataInterface(format!(
"The datatype \"{}\" is not supported in the Rust implementation",
dt
"The datatype \"{:?}\" is still not supported in Rust implementation",
other
)))
}
})
};
Ok(Field::new(schema.name(), data_type, schema.nullable()))
}

/// the inverse of [to_datatype]
fn from_datatype(datatype: &DataType) -> Result<String> {
Ok(match datatype {
/// See https://arrow.apache.org/docs/format/CDataInterface.html#data-type-description-format-strings
fn to_format(data_type: &DataType) -> Result<String> {
Ok(match data_type {
DataType::Null => "n",
DataType::Boolean => "b",
DataType::Int8 => "c",
@@ -289,6 +306,7 @@ fn from_datatype(datatype: &DataType) -> Result<String> {
DataType::Time64(TimeUnit::Nanosecond) => "ttn",
DataType::List(_) => "+l",
DataType::LargeList(_) => "+L",
DataType::Struct(_) => "+s",
z => {
return Err(ArrowError::CDataInterface(format!(
"The datatype \"{:?}\" is still not supported in Rust implementation",
@@ -386,14 +404,27 @@ pub struct FFI_ArrowArray {
private_data: *mut ::std::os::raw::c_void,
}

impl Drop for FFI_ArrowArray {
fn drop(&mut self) {
match self.release {
None => (),
Some(release) => unsafe { release(self) },
};
}
}

// callback used to drop [FFI_ArrowArray] when it is exported
unsafe extern "C" fn release_array(array: *mut FFI_ArrowArray) {
if array.is_null() {
return;
}
let array = &mut *array;
// take ownership of `private_data`, therefore dropping it
Box::from_raw(array.private_data as *mut PrivateData);

// take ownership of `private_data`, therefore dropping it`
let private = Box::from_raw(array.private_data as *mut PrivateData);
for child in private.children.iter() {
let _ = Box::from_raw(*child);
}

array.release = None;
}
@@ -409,26 +440,30 @@ impl FFI_ArrowArray {
/// # Safety
/// This method releases `buffers`. Consumers of this struct *must* call `release` before
/// releasing this struct, or contents in `buffers` leak.
unsafe fn new(
length: i64,
null_count: i64,
offset: i64,
n_buffers: i64,
buffers: Vec<Option<Buffer>>,
children: Vec<*mut FFI_ArrowArray>,
) -> Self {
let buffers_ptr = buffers
fn new(data: &ArrayData) -> Self {
// * insert the null buffer at the start
// * make all others `Option<Buffer>`.
let buffers = iter::once(data.null_buffer().cloned())
.chain(data.buffers().iter().map(|b| Some(b.clone())))
.collect::<Vec<_>>();
let n_buffers = buffers.len() as i64;

let mut buffers_ptr = buffers
.iter()
.map(|maybe_buffer| match maybe_buffer {
// note that `raw_data` takes into account the buffer's offset
Some(b) => b.as_ptr() as *const std::os::raw::c_void,
None => std::ptr::null(),
})
.collect::<Box<[_]>>();
let pointer = buffers_ptr.as_ptr() as *mut *const std::ffi::c_void;
let pointer = buffers_ptr.as_mut_ptr();

let children = children.into_boxed_slice();
let children_ptr = children.as_ptr() as *mut *mut FFI_ArrowArray;
let mut children = data
.child_data()
.iter()
.map(|child| Box::into_raw(Box::new(FFI_ArrowArray::new(child))))
.collect::<Box<_>>();
let children_ptr = children.as_mut_ptr();
let n_children = children.len() as i64;

// create the private data owning everything.
@@ -440,9 +475,9 @@ impl FFI_ArrowArray {
});

Self {
length,
null_count,
offset,
length: data.len() as i64,
null_count: data.null_count() as i64,
offset: data.offset() as i64,
n_buffers,
n_children,
buffers: pointer,
@@ -468,6 +503,26 @@ impl FFI_ArrowArray {
private_data: std::ptr::null_mut(),
}
}

/// the length of the array
pub fn len(&self) -> usize {
self.length as usize
}

/// whether the array is empty
pub fn is_empty(&self) -> bool {
self.length == 0
}

/// the offset of the array
pub fn offset(&self) -> usize {
self.offset as usize
}

/// the null count of the array
pub fn null_count(&self) -> usize {
self.null_count as usize
}
}

/// returns a new buffer corresponding to the index `i` of the FFI array. It may not exist (null pointer).
@@ -478,7 +533,8 @@ impl FFI_ArrowArray {
/// # Safety
/// This function assumes that `ceil(self.length * bits, 8)` is the size of the buffer
unsafe fn create_buffer(
array: Arc<FFI_ArrowArray>,
owner: Arc<FFI_ArrowArray>,
array: &FFI_ArrowArray,
index: usize,
len: usize,
) -> Option<Buffer> {
@@ -490,33 +546,149 @@ unsafe fn create_buffer(
assert!(index < array.n_buffers as usize);
let ptr = *buffers.add(index);

NonNull::new(ptr as *mut u8).map(|ptr| Buffer::from_unowned(ptr, len, array))
NonNull::new(ptr as *mut u8).map(|ptr| Buffer::from_unowned(ptr, len, owner))
}

unsafe fn create_child_arrays(
array: Arc<FFI_ArrowArray>,
schema: Arc<FFI_ArrowSchema>,
) -> Result<Vec<ArrayData>> {
(0..array.n_children as usize)
.map(|i| {
let arr_ptr = *array.children.add(i);
let schema_ptr = *schema.children.add(i);
let arrow_arr = ArrowArray::try_from_raw(
arr_ptr as *const FFI_ArrowArray,
schema_ptr as *const FFI_ArrowSchema,
)?;
ArrayData::try_from(arrow_arr)
})
.collect()
fn create_child(
owner: Arc<FFI_ArrowArray>,
array: &FFI_ArrowArray,
schema: &FFI_ArrowSchema,
index: usize,
) -> ArrowArrayChild<'static> {
assert!(index < array.n_children as usize);
assert!(!array.children.is_null());
assert!(!array.children.is_null());
unsafe {
let arr_ptr = *array.children.add(index);
let schema_ptr = *schema.children.add(index);
assert!(!arr_ptr.is_null());
assert!(!schema_ptr.is_null());
let arr_ptr = &*arr_ptr;
let schema_ptr = &*schema_ptr;
ArrowArrayChild::from_raw(arr_ptr, schema_ptr, owner)
}
}

impl Drop for FFI_ArrowArray {
fn drop(&mut self) {
match self.release {
None => (),
Some(release) => unsafe { release(self) },
};
pub trait ArrowArrayRef {
fn to_data(&self) -> Result<ArrayData> {
let data_type = self.data_type()?;
let len = self.array().len();
let offset = self.array().offset();
let null_count = self.array().null_count();
let buffers = self.buffers()?;
let null_bit_buffer = self.null_bit_buffer();

let child_data = (0..self.array().n_children as usize)
.map(|i| {
let child = self.child(i);
child.to_data()
})
.map(|d| d.unwrap())
.collect();

Ok(ArrayData::new(
data_type,
len,
Some(null_count),
null_bit_buffer,
offset,
buffers,
child_data,
))
}

/// returns all buffers, as organized by Rust (i.e. null buffer is skipped)
fn buffers(&self) -> Result<Vec<Buffer>> {
(0..self.array().n_buffers - 1)
.map(|index| {
// + 1: skip null buffer
let index = (index + 1) as usize;

let len = self.buffer_len(index)?;

unsafe { create_buffer(self.owner().clone(), self.array(), index, len) }
.ok_or_else(|| {
ArrowError::CDataInterface(format!(
"The external buffer at position {} is null.",
index - 1
))
})
})
.collect()
}

/// Returns the length, in bytes, of the buffer `i` (indexed according to the C data interface)
// Rust implementation uses fixed-sized buffers, which require knowledge of their `len`.
// for variable-sized buffers, such as the second buffer of a stringArray, we need
// to fetch offset buffer's len to build the second buffer.
fn buffer_len(&self, i: usize) -> Result<usize> {
// Inner type is not important for buffer length.
let data_type = &self.data_type()?;

Ok(match (data_type, i) {
(DataType::Utf8, 1)
| (DataType::LargeUtf8, 1)
| (DataType::Binary, 1)
| (DataType::LargeBinary, 1)
| (DataType::List(_), 1)
| (DataType::LargeList(_), 1) => {
// the len of the offset buffer (buffer 1) equals length + 1
let bits = bit_width(data_type, i)?;
debug_assert_eq!(bits % 8, 0);
(self.array().length as usize + 1) * (bits / 8)
}
(DataType::Utf8, 2) | (DataType::Binary, 2) | (DataType::List(_), 2) => {
// the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1)
let len = self.buffer_len(1)?;
// first buffer is the null buffer => add(1)
// we assume that pointer is aligned for `i32`, as Utf8 uses `i32` offsets.
#[allow(clippy::cast_ptr_alignment)]
let offset_buffer = unsafe {
*(self.array().buffers as *mut *const u8).add(1) as *const i32
};
// get last offset
(unsafe { *offset_buffer.add(len / size_of::<i32>() - 1) }) as usize
}
(DataType::LargeUtf8, 2)
| (DataType::LargeBinary, 2)
| (DataType::LargeList(_), 2) => {
// the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1)
let len = self.buffer_len(1)?;
// first buffer is the null buffer => add(1)
// we assume that pointer is aligned for `i64`, as Large uses `i64` offsets.
#[allow(clippy::cast_ptr_alignment)]
let offset_buffer = unsafe {
*(self.array().buffers as *mut *const u8).add(1) as *const i64
};
// get last offset
(unsafe { *offset_buffer.add(len / size_of::<i64>() - 1) }) as usize
}
// buffer len of primitive types
_ => {
let bits = bit_width(data_type, i)?;
bit_util::ceil(self.array().length as usize * bits, 8)
}
})
}

/// returns the null bit buffer.
/// Rust implementation uses a buffer that is not part of the array of buffers.
/// The C Data interface's null buffer is part of the array of buffers.
fn null_bit_buffer(&self) -> Option<Buffer> {
// similar to `self.buffer_len(0)`, but without `Result`.
let buffer_len = bit_util::ceil(self.array().length as usize, 8);

unsafe { create_buffer(self.owner().clone(), self.array(), 0, buffer_len) }
}

fn child(&self, index: usize) -> ArrowArrayChild {
create_child(self.owner().clone(), self.array(), self.schema(), index)
}

fn owner(&self) -> &Arc<FFI_ArrowArray>;
fn array(&self) -> &FFI_ArrowArray;
fn schema(&self) -> &FFI_ArrowSchema;
fn data_type(&self) -> Result<DataType>;
}

/// Struct used to move an Array from and to the C Data Interface.
@@ -540,51 +712,64 @@ impl Drop for FFI_ArrowArray {
/// Furthermore, this struct assumes that the incoming data agrees with the C data interface.
#[derive(Debug)]
pub struct ArrowArray {
// these are ref-counted because they can be shared by multiple buffers.
array: Arc<FFI_ArrowArray>,
schema: Arc<FFI_ArrowSchema>,
}

#[derive(Debug)]
pub struct ArrowArrayChild<'a> {
array: &'a FFI_ArrowArray,
schema: &'a FFI_ArrowSchema,
owner: Arc<FFI_ArrowArray>,
}

impl ArrowArrayRef for ArrowArray {
/// the data_type as declared in the schema
fn data_type(&self) -> Result<DataType> {
to_field(&self.schema).map(|x| x.data_type().clone())
}

fn array(&self) -> &FFI_ArrowArray {
self.array.as_ref()
}

fn schema(&self) -> &FFI_ArrowSchema {
self.schema.as_ref()
}

fn owner(&self) -> &Arc<FFI_ArrowArray> {
&self.array
}
}

impl<'a> ArrowArrayRef for ArrowArrayChild<'a> {
/// the data_type as declared in the schema
fn data_type(&self) -> Result<DataType> {
to_field(self.schema).map(|x| x.data_type().clone())
}

fn array(&self) -> &FFI_ArrowArray {
self.array
}

fn schema(&self) -> &FFI_ArrowSchema {
self.schema
}

fn owner(&self) -> &Arc<FFI_ArrowArray> {
&self.owner
}
}

impl ArrowArray {
/// creates a new `ArrowArray`. This is used to export to the C Data Interface.
/// # Safety
/// See safety of [ArrowArray]
#[allow(clippy::too_many_arguments)]
pub unsafe fn try_new(
data_type: &DataType,
len: usize,
null_count: usize,
null_buffer: Option<Buffer>,
offset: usize,
buffers: Vec<Buffer>,
child_data: Vec<ArrowArray>,
nullable: bool,
) -> Result<Self> {
let format = from_datatype(data_type)?;
// * insert the null buffer at the start
// * make all others `Option<Buffer>`.
let new_buffers = iter::once(null_buffer)
.chain(buffers.iter().map(|b| Some(b.clone())))
.collect::<Vec<_>>();

let mut ffi_arrow_arrays = Vec::with_capacity(child_data.len());
let mut ffi_arrow_schemas = Vec::with_capacity(child_data.len());

child_data.into_iter().for_each(|arrow_arr| {
let (arr, schema) = ArrowArray::into_raw(arrow_arr);
ffi_arrow_arrays.push(arr as *mut FFI_ArrowArray);
ffi_arrow_schemas.push(schema as *mut FFI_ArrowSchema);
});

let schema = Arc::new(FFI_ArrowSchema::new(&format, ffi_arrow_schemas, nullable));
let array = Arc::new(FFI_ArrowArray::new(
len as i64,
null_count as i64,
offset as i64,
new_buffers.len() as i64,
new_buffers,
ffi_arrow_arrays,
));
pub unsafe fn try_new(data: ArrayData) -> Result<Self> {
let field = Field::new("", data.data_type().clone(), data.null_count() != 0);
let array = Arc::new(FFI_ArrowArray::new(&data));
let schema = Arc::new(FFI_ArrowSchema::try_new(field)?);

Ok(ArrowArray { array, schema })
}
@@ -623,120 +808,19 @@ impl ArrowArray {
pub fn into_raw(this: ArrowArray) -> (*const FFI_ArrowArray, *const FFI_ArrowSchema) {
(Arc::into_raw(this.array), Arc::into_raw(this.schema))
}
}

/// returns the null bit buffer.
/// Rust implementation uses a buffer that is not part of the array of buffers.
/// The C Data interface's null buffer is part of the array of buffers.
pub fn null_bit_buffer(&self) -> Option<Buffer> {
// similar to `self.buffer_len(0)`, but without `Result`.
let buffer_len = bit_util::ceil(self.array.length as usize, 8);

unsafe { create_buffer(self.array.clone(), 0, buffer_len) }
}

/// Returns the length, in bytes, of the buffer `i` (indexed according to the C data interface)
// Rust implementation uses fixed-sized buffers, which require knowledge of their `len`.
// for variable-sized buffers, such as the second buffer of a stringArray, we need
// to fetch offset buffer's len to build the second buffer.
fn buffer_len(&self, i: usize) -> Result<usize> {
// Inner type is not important for buffer length.
let data_type = &self.data_type(None)?;

Ok(match (data_type, i) {
(DataType::Utf8, 1)
| (DataType::LargeUtf8, 1)
| (DataType::Binary, 1)
| (DataType::LargeBinary, 1)
| (DataType::List(_), 1)
| (DataType::LargeList(_), 1) => {
// the len of the offset buffer (buffer 1) equals length + 1
let bits = bit_width(data_type, i)?;
debug_assert_eq!(bits % 8, 0);
(self.array.length as usize + 1) * (bits / 8)
}
(DataType::Utf8, 2) | (DataType::Binary, 2) | (DataType::List(_), 2) => {
// the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1)
let len = self.buffer_len(1)?;
// first buffer is the null buffer => add(1)
// we assume that pointer is aligned for `i32`, as Utf8 uses `i32` offsets.
#[allow(clippy::cast_ptr_alignment)]
let offset_buffer = unsafe {
*(self.array.buffers as *mut *const u8).add(1) as *const i32
};
// get last offset
(unsafe { *offset_buffer.add(len / size_of::<i32>() - 1) }) as usize
}
(DataType::LargeUtf8, 2)
| (DataType::LargeBinary, 2)
| (DataType::LargeList(_), 2) => {
// the len of the data buffer (buffer 2) equals the last value of the offset buffer (buffer 1)
let len = self.buffer_len(1)?;
// first buffer is the null buffer => add(1)
// we assume that pointer is aligned for `i64`, as Large uses `i64` offsets.
#[allow(clippy::cast_ptr_alignment)]
let offset_buffer = unsafe {
*(self.array.buffers as *mut *const u8).add(1) as *const i64
};
// get last offset
(unsafe { *offset_buffer.add(len / size_of::<i64>() - 1) }) as usize
}
// buffer len of primitive types
_ => {
let bits = bit_width(data_type, i)?;
bit_util::ceil(self.array.length as usize * bits, 8)
}
})
}

/// returns all buffers, as organized by Rust (i.e. null buffer is skipped)
pub fn buffers(&self) -> Result<Vec<Buffer>> {
(0..self.array.n_buffers - 1)
.map(|index| {
// + 1: skip null buffer
let index = (index + 1) as usize;

let len = self.buffer_len(index)?;

unsafe { create_buffer(self.array.clone(), index, len) }.ok_or_else(
|| {
ArrowError::CDataInterface(format!(
"The external buffer at position {} is null.",
index - 1
))
},
)
})
.collect()
}

/// returns the child data of this array
pub fn children(&self) -> Result<Vec<ArrayData>> {
unsafe { create_child_arrays(self.array.clone(), self.schema.clone()) }
}

/// the length of the array
pub fn len(&self) -> usize {
self.array.length as usize
}

/// whether the array is empty
pub fn is_empty(&self) -> bool {
self.array.length == 0
}

/// the offset of the array
pub fn offset(&self) -> usize {
self.array.offset as usize
}

/// the null count of the array
pub fn null_count(&self) -> usize {
self.array.null_count as usize
}

/// the data_type as declared in the schema
pub fn data_type(&self, child_type: Option<DataType>) -> Result<DataType> {
to_datatype(self.schema.format(), child_type, self.schema.as_ref())
impl<'a> ArrowArrayChild<'a> {
fn from_raw(
array: &'a FFI_ArrowArray,
schema: &'a FFI_ArrowSchema,
owner: Arc<FFI_ArrowArray>,
) -> Self {
Self {
array,
schema,
owner,
}
}
}

0 comments on commit c863a2c

Please sign in to comment.