diff --git a/rust/arrow/src/array/array.rs b/rust/arrow/src/array/array.rs index b7e13a03ad7..ad80881a88f 100644 --- a/rust/arrow/src/array/array.rs +++ b/rust/arrow/src/array/array.rs @@ -16,37 +16,12 @@ // under the License. use std::any::Any; -use std::borrow::Borrow; -use std::convert::{From, TryFrom}; use std::fmt; -use std::io::Write; -use std::iter::{FromIterator, IntoIterator}; -use std::mem; use std::sync::Arc; -use chrono::prelude::*; -use num::Num; - +use super::ArrayDataRef; use super::*; -use crate::array::builder::StringDictionaryBuilder; use crate::array::equal_json::JsonEqual; -use crate::buffer::{buffer_bin_or, Buffer, MutableBuffer}; -use crate::datatypes::DataType::Struct; -use crate::datatypes::*; -use crate::memory; -use crate::{ - error::{ArrowError, Result}, - util::bit_util, -}; - -/// Number of seconds in a day -const SECONDS_IN_DAY: i64 = 86_400; -/// Number of milliseconds in a second -const MILLISECONDS: i64 = 1_000; -/// Number of microseconds in a second -const MICROSECONDS: i64 = 1_000_000; -/// Number of nanoseconds in a second -const NANOSECONDS: i64 = 1_000_000_000; /// Trait for dealing with different types of array at runtime when the type of the /// array is not known in advance. @@ -336,624 +311,12 @@ pub fn make_array(data: ArrayDataRef) -> ArrayRef { } } -// creates a new MutableBuffer initializes all falsed -// this is useful to populate null bitmaps -fn make_null_buffer(len: usize) -> MutableBuffer { - let num_bytes = bit_util::ceil(len, 8); - MutableBuffer::new(num_bytes).with_bitset(num_bytes, false) -} - -/// ---------------------------------------------------------------------------- -/// Implementations of different array types - -struct RawPtrBox { - inner: *const T, -} - -impl RawPtrBox { - fn new(inner: *const T) -> Self { - Self { inner } - } - - fn get(&self) -> *const T { - self.inner - } -} - -unsafe impl Send for RawPtrBox {} -unsafe impl Sync for RawPtrBox {} - -fn as_aligned_pointer(p: *const u8) -> *const T { - assert!( - memory::is_aligned(p, mem::align_of::()), - "memory is not aligned" - ); - p as *const T -} - -/// Array whose elements are of primitive types. -pub struct PrimitiveArray { - data: ArrayDataRef, - /// Pointer to the value array. The lifetime of this must be <= to the value buffer - /// stored in `data`, so it's safe to store. - /// Also note that boolean arrays are bit-packed, so although the underlying pointer - /// is of type bool it should be cast back to u8 before being used. - /// i.e. `self.raw_values.get() as *const u8` - raw_values: RawPtrBox, -} - -impl PrimitiveArray { - /// Returns the length of this array. - pub fn len(&self) -> usize { - self.data.len() - } - - /// Returns whether this array is empty. - pub fn is_empty(&self) -> bool { - self.data.is_empty() - } - - /// Returns a raw pointer to the values of this array. - pub fn raw_values(&self) -> *const T::Native { - unsafe { self.raw_values.get().add(self.data.offset()) } - } - - /// Returns a slice for the given offset and length - /// - /// Note this doesn't do any bound checking, for performance reason. - pub fn value_slice(&self, offset: usize, len: usize) -> &[T::Native] { - let raw = - unsafe { std::slice::from_raw_parts(self.raw_values().add(offset), len) }; - &raw[..] - } - - // Returns a new primitive array builder - pub fn builder(capacity: usize) -> PrimitiveBuilder { - PrimitiveBuilder::::new(capacity) - } - - /// Returns a `Buffer` holding all the values of this array. - /// - /// Note this doesn't take the offset of this array into account. - pub fn values(&self) -> Buffer { - self.data.buffers()[0].clone() - } - - /// Returns the primitive value at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - pub fn value(&self, i: usize) -> T::Native { - let offset = i + self.offset(); - unsafe { T::index(self.raw_values.get(), offset) } - } -} - -impl Array for PrimitiveArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [PrimitiveArray]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [PrimitiveArray]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - -fn as_datetime(v: i64) -> Option { - match T::DATA_TYPE { - DataType::Date32(_) => { - // convert days into seconds - Some(NaiveDateTime::from_timestamp(v as i64 * SECONDS_IN_DAY, 0)) - } - DataType::Date64(_) => Some(NaiveDateTime::from_timestamp( - // extract seconds from milliseconds - v / MILLISECONDS, - // discard extracted seconds and convert milliseconds to nanoseconds - (v % MILLISECONDS * MICROSECONDS) as u32, - )), - DataType::Time32(_) | DataType::Time64(_) => None, - DataType::Timestamp(unit, _) => match unit { - TimeUnit::Second => Some(NaiveDateTime::from_timestamp(v, 0)), - TimeUnit::Millisecond => Some(NaiveDateTime::from_timestamp( - // extract seconds from milliseconds - v / MILLISECONDS, - // discard extracted seconds and convert milliseconds to nanoseconds - (v % MILLISECONDS * MICROSECONDS) as u32, - )), - TimeUnit::Microsecond => Some(NaiveDateTime::from_timestamp( - // extract seconds from microseconds - v / MICROSECONDS, - // discard extracted seconds and convert microseconds to nanoseconds - (v % MICROSECONDS * MILLISECONDS) as u32, - )), - TimeUnit::Nanosecond => Some(NaiveDateTime::from_timestamp( - // extract seconds from nanoseconds - v / NANOSECONDS, - // discard extracted seconds - (v % NANOSECONDS) as u32, - )), - }, - // interval is not yet fully documented [ARROW-3097] - DataType::Interval(_) => None, - _ => None, - } -} - -fn as_date(v: i64) -> Option { - as_datetime::(v).map(|datetime| datetime.date()) -} - -fn as_time(v: i64) -> Option { - match T::DATA_TYPE { - DataType::Time32(unit) => { - // safe to immediately cast to u32 as `self.value(i)` is positive i32 - let v = v as u32; - match unit { - TimeUnit::Second => Some(NaiveTime::from_num_seconds_from_midnight(v, 0)), - TimeUnit::Millisecond => { - Some(NaiveTime::from_num_seconds_from_midnight( - // extract seconds from milliseconds - v / MILLISECONDS as u32, - // discard extracted seconds and convert milliseconds to - // nanoseconds - v % MILLISECONDS as u32 * MICROSECONDS as u32, - )) - } - _ => None, - } - } - DataType::Time64(unit) => { - match unit { - TimeUnit::Microsecond => { - Some(NaiveTime::from_num_seconds_from_midnight( - // extract seconds from microseconds - (v / MICROSECONDS) as u32, - // discard extracted seconds and convert microseconds to - // nanoseconds - (v % MICROSECONDS * MILLISECONDS) as u32, - )) - } - TimeUnit::Nanosecond => { - Some(NaiveTime::from_num_seconds_from_midnight( - // extract seconds from nanoseconds - (v / NANOSECONDS) as u32, - // discard extracted seconds - (v % NANOSECONDS) as u32, - )) - } - _ => None, - } - } - DataType::Timestamp(_, _) => as_datetime::(v).map(|datetime| datetime.time()), - DataType::Date32(_) | DataType::Date64(_) => Some(NaiveTime::from_hms(0, 0, 0)), - DataType::Interval(_) => None, - _ => None, - } -} - -impl PrimitiveArray -where - i64: std::convert::From, -{ - /// Returns value as a chrono `NaiveDateTime`, handling time resolution - /// - /// If a data type cannot be converted to `NaiveDateTime`, a `None` is returned. - /// A valid value is expected, thus the user should first check for validity. - pub fn value_as_datetime(&self, i: usize) -> Option { - as_datetime::(i64::from(self.value(i))) - } - - /// Returns value as a chrono `NaiveDate` by using `Self::datetime()` - /// - /// If a data type cannot be converted to `NaiveDate`, a `None` is returned - pub fn value_as_date(&self, i: usize) -> Option { - self.value_as_datetime(i).map(|datetime| datetime.date()) - } - - /// Returns a value as a chrono `NaiveTime` - /// - /// `Date32` and `Date64` return UTC midnight as they do not have time resolution - pub fn value_as_time(&self, i: usize) -> Option { - as_time::(i64::from(self.value(i))) - } -} - -impl fmt::Debug for PrimitiveArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "PrimitiveArray<{:?}>\n[\n", T::DATA_TYPE)?; - print_long_array(self, f, |array, index, f| match T::DATA_TYPE { - DataType::Date32(_) | DataType::Date64(_) => { - let v = self.value(index).to_usize().unwrap() as i64; - match as_date::(v) { - Some(date) => write!(f, "{:?}", date), - None => write!(f, "null"), - } - } - DataType::Time32(_) | DataType::Time64(_) => { - let v = self.value(index).to_usize().unwrap() as i64; - match as_time::(v) { - Some(time) => write!(f, "{:?}", time), - None => write!(f, "null"), - } - } - DataType::Timestamp(_, _) => { - let v = self.value(index).to_usize().unwrap() as i64; - match as_datetime::(v) { - Some(datetime) => write!(f, "{:?}", datetime), - None => write!(f, "null"), - } - } - _ => fmt::Debug::fmt(&array.value(index), f), - })?; - write!(f, "]") - } -} - -impl<'a, T: ArrowPrimitiveType> IntoIterator for &'a PrimitiveArray { - type Item = Option<::Native>; - type IntoIter = PrimitiveIter<'a, T>; - - fn into_iter(self) -> Self::IntoIter { - PrimitiveIter::<'a, T>::new(self) - } -} - -impl<'a, T: ArrowPrimitiveType> PrimitiveArray { - /// constructs a new iterator - pub fn iter(&'a self) -> PrimitiveIter<'a, T> { - PrimitiveIter::<'a, T>::new(&self) - } -} - -impl::Native>>> - FromIterator for PrimitiveArray -{ - fn from_iter>(iter: I) -> Self { - let iter = iter.into_iter(); - let (_, data_len) = iter.size_hint(); - let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. - - let num_bytes = bit_util::ceil(data_len, 8); - let mut null_buf = MutableBuffer::new(num_bytes).with_bitset(num_bytes, false); - let mut val_buf = MutableBuffer::new( - data_len * mem::size_of::<::Native>(), - ); - - let null = vec![0; mem::size_of::<::Native>()]; - - let null_slice = null_buf.data_mut(); - iter.enumerate().for_each(|(i, item)| { - if let Some(a) = item.borrow() { - bit_util::set_bit(null_slice, i); - val_buf.write_all(a.to_byte_slice()).unwrap(); - } else { - val_buf.write_all(&null).unwrap(); - } - }); - - let data = ArrayData::new( - T::DATA_TYPE, - data_len, - None, - Some(null_buf.freeze()), - 0, - vec![val_buf.freeze()], - vec![], - ); - PrimitiveArray::from(Arc::new(data)) - } -} - -// TODO: the macro is needed here because we'd get "conflicting implementations" error -// otherwise with both `From>` and `From>>`. -// We should revisit this in future. -macro_rules! def_numeric_from_vec { - ( $ty:ident ) => { - impl From::Native>> for PrimitiveArray<$ty> { - fn from(data: Vec<<$ty as ArrowPrimitiveType>::Native>) -> Self { - let array_data = ArrayData::builder($ty::DATA_TYPE) - .len(data.len()) - .add_buffer(Buffer::from(data.to_byte_slice())) - .build(); - PrimitiveArray::from(array_data) - } - } - - // Constructs a primitive array from a vector. Should only be used for testing. - impl From::Native>>> - for PrimitiveArray<$ty> - { - fn from(data: Vec::Native>>) -> Self { - PrimitiveArray::from_iter(data.iter()) - } - } - }; -} - -def_numeric_from_vec!(Int8Type); -def_numeric_from_vec!(Int16Type); -def_numeric_from_vec!(Int32Type); -def_numeric_from_vec!(Int64Type); -def_numeric_from_vec!(UInt8Type); -def_numeric_from_vec!(UInt16Type); -def_numeric_from_vec!(UInt32Type); -def_numeric_from_vec!(UInt64Type); -def_numeric_from_vec!(Float32Type); -def_numeric_from_vec!(Float64Type); - -def_numeric_from_vec!(Date32Type); -def_numeric_from_vec!(Date64Type); -def_numeric_from_vec!(Time32SecondType); -def_numeric_from_vec!(Time32MillisecondType); -def_numeric_from_vec!(Time64MicrosecondType); -def_numeric_from_vec!(Time64NanosecondType); -def_numeric_from_vec!(IntervalYearMonthType); -def_numeric_from_vec!(IntervalDayTimeType); -def_numeric_from_vec!(DurationSecondType); -def_numeric_from_vec!(DurationMillisecondType); -def_numeric_from_vec!(DurationMicrosecondType); -def_numeric_from_vec!(DurationNanosecondType); -def_numeric_from_vec!(TimestampMillisecondType); -def_numeric_from_vec!(TimestampMicrosecondType); - -impl PrimitiveArray { - /// Construct a timestamp array from a vec of i64 values and an optional timezone - pub fn from_vec(data: Vec, timezone: Option>) -> Self { - let array_data = - ArrayData::builder(DataType::Timestamp(T::get_time_unit(), timezone)) - .len(data.len()) - .add_buffer(Buffer::from(data.to_byte_slice())) - .build(); - PrimitiveArray::from(array_data) - } -} - -impl PrimitiveArray { - /// Construct a timestamp array from a vec of Option values and an optional timezone - pub fn from_opt_vec(data: Vec>, timezone: Option>) -> Self { - // TODO: duplicated from def_numeric_from_vec! macro, it looks possible to convert to generic - let data_len = data.len(); - let mut null_buf = make_null_buffer(data_len); - let mut val_buf = MutableBuffer::new(data_len * mem::size_of::()); - - { - let null = vec![0; mem::size_of::()]; - let null_slice = null_buf.data_mut(); - for (i, v) in data.iter().enumerate() { - if let Some(n) = v { - bit_util::set_bit(null_slice, i); - // unwrap() in the following should be safe here since we've - // made sure enough space is allocated for the values. - val_buf.write_all(&n.to_byte_slice()).unwrap(); - } else { - val_buf.write_all(&null).unwrap(); - } - } - } - - let array_data = - ArrayData::builder(DataType::Timestamp(T::get_time_unit(), timezone)) - .len(data_len) - .add_buffer(val_buf.freeze()) - .null_bit_buffer(null_buf.freeze()) - .build(); - PrimitiveArray::from(array_data) - } -} - -/// Constructs a boolean array from a vector. Should only be used for testing. -impl From> for BooleanArray { - fn from(data: Vec) -> Self { - let mut mut_buf = make_null_buffer(data.len()); - { - let mut_slice = mut_buf.data_mut(); - for (i, b) in data.iter().enumerate() { - if *b { - bit_util::set_bit(mut_slice, i); - } - } - } - let array_data = ArrayData::builder(DataType::Boolean) - .len(data.len()) - .add_buffer(mut_buf.freeze()) - .build(); - BooleanArray::from(array_data) - } -} - -impl From>> for BooleanArray { - fn from(data: Vec>) -> Self { - let data_len = data.len(); - let num_byte = bit_util::ceil(data_len, 8); - let mut null_buf = make_null_buffer(data.len()); - let mut val_buf = MutableBuffer::new(num_byte).with_bitset(num_byte, false); - - { - let null_slice = null_buf.data_mut(); - let val_slice = val_buf.data_mut(); - - for (i, v) in data.iter().enumerate() { - if let Some(b) = v { - bit_util::set_bit(null_slice, i); - if *b { - bit_util::set_bit(val_slice, i); - } - } - } - } - - let array_data = ArrayData::builder(DataType::Boolean) - .len(data_len) - .add_buffer(val_buf.freeze()) - .null_bit_buffer(null_buf.freeze()) - .build(); - BooleanArray::from(array_data) - } -} - -/// Constructs a `PrimitiveArray` from an array data reference. -impl From for PrimitiveArray { - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.buffers().len(), - 1, - "PrimitiveArray data should contain a single buffer only (values buffer)" - ); - let raw_values = data.buffers()[0].raw_data(); - assert!( - memory::is_aligned::(raw_values, mem::align_of::()), - "memory is not aligned" - ); - Self { - data, - raw_values: RawPtrBox::new(raw_values as *const T::Native), - } - } -} - -/// trait declaring an offset size, relevant for i32 vs i64 array types. -pub trait OffsetSizeTrait: ArrowNativeType + Num + Ord { - fn prefix() -> &'static str; - - fn to_isize(&self) -> isize; -} - -impl OffsetSizeTrait for i32 { - fn prefix() -> &'static str { - "" - } - - fn to_isize(&self) -> isize { - num::ToPrimitive::to_isize(self).unwrap() - } -} - -impl OffsetSizeTrait for i64 { - fn prefix() -> &'static str { - "Large" - } - - fn to_isize(&self) -> isize { - num::ToPrimitive::to_isize(self).unwrap() - } -} - -pub struct GenericListArray { - data: ArrayDataRef, - values: ArrayRef, - value_offsets: RawPtrBox, -} - -impl GenericListArray { - /// Returns a reference to the values of this list. - pub fn values(&self) -> ArrayRef { - self.values.clone() - } - - /// Returns a clone of the value type of this list. - pub fn value_type(&self) -> DataType { - self.values.data_ref().data_type().clone() - } - - /// Returns ith value of this list array. - pub fn value(&self, i: usize) -> ArrayRef { - self.values.slice( - self.value_offset(i).to_usize().unwrap(), - self.value_length(i).to_usize().unwrap(), - ) - } - - /// Returns the offset for value at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_offset(&self, i: usize) -> OffsetSize { - self.value_offset_at(self.data.offset() + i) - } - - /// Returns the length for value at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_length(&self, mut i: usize) -> OffsetSize { - i += self.data.offset(); - self.value_offset_at(i + 1) - self.value_offset_at(i) - } - - #[inline] - fn value_offset_at(&self, i: usize) -> OffsetSize { - unsafe { *self.value_offsets.get().add(i) } - } -} - -impl From for GenericListArray { - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.buffers().len(), - 1, - "ListArray data should contain a single buffer only (value offsets)" - ); - assert_eq!( - data.child_data().len(), - 1, - "ListArray should contain a single child array (values array)" - ); - let values = make_array(data.child_data()[0].clone()); - let raw_value_offsets = data.buffers()[0].raw_data(); - let value_offsets: *const OffsetSize = as_aligned_pointer(raw_value_offsets); - unsafe { - assert!( - (*value_offsets.offset(0)).is_zero(), - "offsets do not start at zero" - ); - } - Self { - data, - values, - value_offsets: RawPtrBox::new(value_offsets), - } - } -} - -impl Array for GenericListArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [ListArray]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [ListArray]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - // Helper function for printing potentially long arrays. -fn print_long_array(array: &A, f: &mut fmt::Formatter, print_item: F) -> fmt::Result +pub(super) fn print_long_array( + array: &A, + f: &mut fmt::Formatter, + print_item: F, +) -> fmt::Result where A: Array, F: Fn(&A, usize, &mut fmt::Formatter) -> fmt::Result, @@ -988,3182 +351,3 @@ where } Ok(()) } - -impl fmt::Debug for GenericListArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}ListArray\n[\n", OffsetSize::prefix())?; - print_long_array(self, f, |array, index, f| { - fmt::Debug::fmt(&array.value(index), f) - })?; - write!(f, "]") - } -} - -/// A list array where each element is a variable-sized sequence of values with the same -/// type whose memory offsets between elements are represented by a i32. -pub type ListArray = GenericListArray; - -/// A list array where each element is a variable-sized sequence of values with the same -/// type whose memory offsets between elements are represented by a i64. -pub type LargeListArray = GenericListArray; - -/// A list array where each element is a fixed-size sequence of values with the same -/// type whose maximum length is represented by a i32. -pub struct FixedSizeListArray { - data: ArrayDataRef, - values: ArrayRef, - length: i32, -} - -impl FixedSizeListArray { - /// Returns a reference to the values of this list. - pub fn values(&self) -> ArrayRef { - self.values.clone() - } - - /// Returns a clone of the value type of this list. - pub fn value_type(&self) -> DataType { - self.values.data_ref().data_type().clone() - } - - /// Returns ith value of this list array. - pub fn value(&self, i: usize) -> ArrayRef { - self.values - .slice(self.value_offset(i) as usize, self.value_length() as usize) - } - - /// Returns the offset for value at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_offset(&self, i: usize) -> i32 { - self.value_offset_at(self.data.offset() + i) - } - - /// Returns the length for value at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub const fn value_length(&self) -> i32 { - self.length - } - - #[inline] - const fn value_offset_at(&self, i: usize) -> i32 { - i as i32 * self.length - } -} - -impl From for FixedSizeListArray { - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.buffers().len(), - 0, - "FixedSizeListArray data should not contain a buffer for value offsets" - ); - assert_eq!( - data.child_data().len(), - 1, - "FixedSizeListArray should contain a single child array (values array)" - ); - let values = make_array(data.child_data()[0].clone()); - let length = match data.data_type() { - DataType::FixedSizeList(_, len) => { - // check that child data is multiple of length - assert_eq!( - values.len() % *len as usize, - 0, - "FixedSizeListArray child array length should be a multiple of {}", - len - ); - *len - } - _ => { - panic!("FixedSizeListArray data should contain a FixedSizeList data type") - } - }; - Self { - data, - values, - length, - } - } -} - -impl Array for FixedSizeListArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [FixedSizeListArray]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() + self.values().get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [FixedSizeListArray]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() - + self.values().get_array_memory_size() - + mem::size_of_val(self) - } -} - -impl fmt::Debug for FixedSizeListArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "FixedSizeListArray<{}>\n[\n", self.value_length())?; - print_long_array(self, f, |array, index, f| { - fmt::Debug::fmt(&array.value(index), f) - })?; - write!(f, "]") - } -} - -/// Like OffsetSizeTrait, but specialized for Binary -// This allow us to expose a constant datatype for the GenericBinaryArray -pub trait BinaryOffsetSizeTrait: OffsetSizeTrait { - const DATA_TYPE: DataType; -} - -impl BinaryOffsetSizeTrait for i32 { - const DATA_TYPE: DataType = DataType::Binary; -} - -impl BinaryOffsetSizeTrait for i64 { - const DATA_TYPE: DataType = DataType::LargeBinary; -} - -pub struct GenericBinaryArray { - data: ArrayDataRef, - value_offsets: RawPtrBox, - value_data: RawPtrBox, -} - -impl GenericBinaryArray { - /// Returns the offset for the element at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_offset(&self, i: usize) -> OffsetSize { - self.value_offset_at(self.data.offset() + i) - } - - /// Returns the length for the element at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_length(&self, mut i: usize) -> OffsetSize { - i += self.data.offset(); - self.value_offset_at(i + 1) - self.value_offset_at(i) - } - - /// Returns a clone of the value offset buffer - pub fn value_offsets(&self) -> Buffer { - self.data.buffers()[0].clone() - } - - /// Returns a clone of the value data buffer - pub fn value_data(&self) -> Buffer { - self.data.buffers()[1].clone() - } - - #[inline] - fn value_offset_at(&self, i: usize) -> OffsetSize { - unsafe { *self.value_offsets.get().add(i) } - } - - /// Returns the element at index `i` as a byte slice. - pub fn value(&self, i: usize) -> &[u8] { - assert!(i < self.data.len(), "BinaryArray out of bounds access"); - let offset = i.checked_add(self.data.offset()).unwrap(); - unsafe { - let pos = self.value_offset_at(offset); - std::slice::from_raw_parts( - self.value_data.get().offset(pos.to_isize()), - (self.value_offset_at(offset + 1) - pos).to_usize().unwrap(), - ) - } - } - - /// Creates a [GenericBinaryArray] from a vector of byte slices - pub fn from_vec(v: Vec<&[u8]>) -> Self { - let mut offsets = Vec::with_capacity(v.len() + 1); - let mut values = Vec::new(); - let mut length_so_far: OffsetSize = OffsetSize::zero(); - offsets.push(length_so_far); - for s in &v { - length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); - offsets.push(length_so_far); - values.extend_from_slice(s); - } - let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(v.len()) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - GenericBinaryArray::::from(array_data) - } - - /// Creates a [GenericBinaryArray] from a vector of Optional (null) byte slices - pub fn from_opt_vec(v: Vec>) -> Self { - v.into_iter().collect() - } - - fn from_list(v: GenericListArray) -> Self { - assert_eq!( - v.data_ref().child_data()[0].child_data().len(), - 0, - "BinaryArray can only be created from list array of u8 values \ - (i.e. List>)." - ); - assert_eq!( - v.data_ref().child_data()[0].data_type(), - &DataType::UInt8, - "BinaryArray can only be created from List arrays, mismatched data types." - ); - - let mut builder = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(v.len()) - .add_buffer(v.data_ref().buffers()[0].clone()) - .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); - if let Some(bitmap) = v.data_ref().null_bitmap() { - builder = builder - .null_count(v.data_ref().null_count()) - .null_bit_buffer(bitmap.bits.clone()) - } - - let data = builder.build(); - Self::from(data) - } -} - -impl<'a, T: BinaryOffsetSizeTrait> GenericBinaryArray { - /// constructs a new iterator - pub fn iter(&'a self) -> GenericBinaryIter<'a, T> { - GenericBinaryIter::<'a, T>::new(&self) - } -} - -impl fmt::Debug for GenericBinaryArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}BinaryArray\n[\n", OffsetSize::prefix())?; - print_long_array(self, f, |array, index, f| { - fmt::Debug::fmt(&array.value(index), f) - })?; - write!(f, "]") - } -} - -impl Array for GenericBinaryArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [$name]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [$name]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - -impl From - for GenericBinaryArray -{ - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.data_type(), - &::DATA_TYPE, - "[Large]BinaryArray expects Datatype::[Large]Binary" - ); - assert_eq!( - data.buffers().len(), - 2, - "BinaryArray data should contain 2 buffers only (offsets and values)" - ); - let raw_value_offsets = data.buffers()[0].raw_data(); - let value_data = data.buffers()[1].raw_data(); - Self { - data, - value_offsets: RawPtrBox::new(as_aligned_pointer::( - raw_value_offsets, - )), - value_data: RawPtrBox::new(value_data), - } - } -} - -impl FromIterator> - for GenericBinaryArray -where - Ptr: AsRef<[u8]>, -{ - fn from_iter>>(iter: I) -> Self { - let iter = iter.into_iter(); - let (_, data_len) = iter.size_hint(); - let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. - - let mut offsets = Vec::with_capacity(data_len + 1); - let mut values = Vec::new(); - let mut null_buf = make_null_buffer(data_len); - let mut length_so_far: OffsetSize = OffsetSize::zero(); - offsets.push(length_so_far); - - { - let null_slice = null_buf.data_mut(); - - for (i, s) in iter.enumerate() { - if let Some(s) = s { - let s = s.as_ref(); - bit_util::set_bit(null_slice, i); - length_so_far = - length_so_far + OffsetSize::from_usize(s.len()).unwrap(); - values.extend_from_slice(s); - } - // always add an element in offsets - offsets.push(length_so_far); - } - } - - let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(data_len) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .null_bit_buffer(null_buf.freeze()) - .build(); - Self::from(array_data) - } -} - -/// An array where each element is a byte whose maximum length is represented by a i32. -pub type BinaryArray = GenericBinaryArray; - -/// An array where each element is a byte whose maximum length is represented by a i64. -pub type LargeBinaryArray = GenericBinaryArray; - -impl<'a, T: BinaryOffsetSizeTrait> IntoIterator for &'a GenericBinaryArray { - type Item = Option<&'a [u8]>; - type IntoIter = GenericBinaryIter<'a, T>; - - fn into_iter(self) -> Self::IntoIter { - GenericBinaryIter::<'a, T>::new(self) - } -} - -impl From> for BinaryArray { - fn from(v: Vec<&[u8]>) -> Self { - BinaryArray::from_vec(v) - } -} - -impl From>> for BinaryArray { - fn from(v: Vec>) -> Self { - BinaryArray::from_opt_vec(v) - } -} - -impl From> for LargeBinaryArray { - fn from(v: Vec<&[u8]>) -> Self { - LargeBinaryArray::from_vec(v) - } -} - -impl From>> for LargeBinaryArray { - fn from(v: Vec>) -> Self { - LargeBinaryArray::from_opt_vec(v) - } -} - -impl From for BinaryArray { - fn from(v: ListArray) -> Self { - BinaryArray::from_list(v) - } -} - -impl From for LargeBinaryArray { - fn from(v: LargeListArray) -> Self { - LargeBinaryArray::from_list(v) - } -} - -/// Like OffsetSizeTrait, but specialized for Strings -// This allow us to expose a constant datatype for the GenericStringArray -pub trait StringOffsetSizeTrait: OffsetSizeTrait { - const DATA_TYPE: DataType; -} - -impl StringOffsetSizeTrait for i32 { - const DATA_TYPE: DataType = DataType::Utf8; -} - -impl StringOffsetSizeTrait for i64 { - const DATA_TYPE: DataType = DataType::LargeUtf8; -} - -/// Generic struct for \[Large\]StringArray -pub struct GenericStringArray { - data: ArrayDataRef, - value_offsets: RawPtrBox, - value_data: RawPtrBox, -} - -impl GenericStringArray { - /// Returns the offset for the element at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_offset(&self, i: usize) -> OffsetSize { - self.value_offset_at(self.data.offset() + i) - } - - /// Returns the length for the element at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_length(&self, mut i: usize) -> OffsetSize { - i += self.data.offset(); - self.value_offset_at(i + 1) - self.value_offset_at(i) - } - - /// Returns a clone of the value offset buffer - pub fn value_offsets(&self) -> Buffer { - self.data.buffers()[0].clone() - } - - /// Returns a clone of the value data buffer - pub fn value_data(&self) -> Buffer { - self.data.buffers()[1].clone() - } - - #[inline] - fn value_offset_at(&self, i: usize) -> OffsetSize { - unsafe { *self.value_offsets.get().add(i) } - } - - /// Returns the element at index `i` as &str - pub fn value(&self, i: usize) -> &str { - assert!(i < self.data.len(), "StringArray out of bounds access"); - let offset = i.checked_add(self.data.offset()).unwrap(); - unsafe { - let pos = self.value_offset_at(offset); - let slice = std::slice::from_raw_parts( - self.value_data.get().offset(pos.to_isize()), - (self.value_offset_at(offset + 1) - pos).to_usize().unwrap(), - ); - - std::str::from_utf8_unchecked(slice) - } - } - - fn from_list(v: GenericListArray) -> Self { - assert_eq!( - v.data().child_data()[0].child_data().len(), - 0, - "StringArray can only be created from list array of u8 values \ - (i.e. List>)." - ); - assert_eq!( - v.data_ref().child_data()[0].data_type(), - &DataType::UInt8, - "StringArray can only be created from List arrays, mismatched data types." - ); - - let mut builder = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(v.len()) - .add_buffer(v.data_ref().buffers()[0].clone()) - .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); - if let Some(bitmap) = v.data().null_bitmap() { - builder = builder - .null_count(v.data_ref().null_count()) - .null_bit_buffer(bitmap.bits.clone()) - } - - let data = builder.build(); - Self::from(data) - } - - pub(crate) fn from_vec(v: Vec<&str>) -> Self { - let mut offsets = Vec::with_capacity(v.len() + 1); - let mut values = Vec::new(); - let mut length_so_far = OffsetSize::zero(); - offsets.push(length_so_far); - for s in &v { - length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); - offsets.push(length_so_far); - values.extend_from_slice(s.as_bytes()); - } - let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(v.len()) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - Self::from(array_data) - } - - pub(crate) fn from_opt_vec(v: Vec>) -> Self { - GenericStringArray::from_iter(v.into_iter()) - } -} - -impl<'a, Ptr, OffsetSize: StringOffsetSizeTrait> FromIterator> - for GenericStringArray -where - Ptr: AsRef, -{ - fn from_iter>>(iter: I) -> Self { - let iter = iter.into_iter(); - let (_, data_len) = iter.size_hint(); - let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. - - let mut offsets = Vec::with_capacity(data_len + 1); - let mut values = Vec::new(); - let mut null_buf = make_null_buffer(data_len); - let mut length_so_far = OffsetSize::zero(); - offsets.push(length_so_far); - - for (i, s) in iter.enumerate() { - if let Some(s) = s { - let s = s.as_ref(); - // set null bit - let null_slice = null_buf.data_mut(); - bit_util::set_bit(null_slice, i); - - length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); - offsets.push(length_so_far); - values.extend_from_slice(s.as_bytes()); - } else { - offsets.push(length_so_far); - values.extend_from_slice(b""); - } - } - - let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) - .len(data_len) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .null_bit_buffer(null_buf.freeze()) - .build(); - Self::from(array_data) - } -} - -impl<'a, T: StringOffsetSizeTrait> IntoIterator for &'a GenericStringArray { - type Item = Option<&'a str>; - type IntoIter = GenericStringIter<'a, T>; - - fn into_iter(self) -> Self::IntoIter { - GenericStringIter::<'a, T>::new(self) - } -} - -impl<'a, T: StringOffsetSizeTrait> GenericStringArray { - /// constructs a new iterator - pub fn iter(&'a self) -> GenericStringIter<'a, T> { - GenericStringIter::<'a, T>::new(&self) - } -} - -impl fmt::Debug for GenericStringArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}StringArray\n[\n", OffsetSize::prefix())?; - print_long_array(self, f, |array, index, f| { - fmt::Debug::fmt(&array.value(index), f) - })?; - write!(f, "]") - } -} - -impl Array for GenericStringArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [$name]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [$name]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - -impl From - for GenericStringArray -{ - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.data_type(), - &::DATA_TYPE, - "[Large]StringArray expects Datatype::[Large]Utf8" - ); - assert_eq!( - data.buffers().len(), - 2, - "StringArray data should contain 2 buffers only (offsets and values)" - ); - let raw_value_offsets = data.buffers()[0].raw_data(); - let value_data = data.buffers()[1].raw_data(); - Self { - data, - value_offsets: RawPtrBox::new(as_aligned_pointer::( - raw_value_offsets, - )), - value_data: RawPtrBox::new(value_data), - } - } -} - -/// An array where each element is a variable-sized sequence of bytes representing a string -/// whose maximum length (in bytes) is represented by a i32. -pub type StringArray = GenericStringArray; - -/// An array where each element is a variable-sized sequence of bytes representing a string -/// whose maximum length (in bytes) is represented by a i64. -pub type LargeStringArray = GenericStringArray; - -impl From for StringArray { - fn from(v: ListArray) -> Self { - StringArray::from_list(v) - } -} - -impl From for LargeStringArray { - fn from(v: LargeListArray) -> Self { - LargeStringArray::from_list(v) - } -} - -impl From> for StringArray { - fn from(v: Vec<&str>) -> Self { - StringArray::from_vec(v) - } -} - -impl From> for LargeStringArray { - fn from(v: Vec<&str>) -> Self { - LargeStringArray::from_vec(v) - } -} - -impl From>> for StringArray { - fn from(v: Vec>) -> Self { - StringArray::from_opt_vec(v) - } -} - -impl From>> for LargeStringArray { - fn from(v: Vec>) -> Self { - LargeStringArray::from_opt_vec(v) - } -} - -/// A type of `FixedSizeListArray` whose elements are binaries. -pub struct FixedSizeBinaryArray { - data: ArrayDataRef, - value_data: RawPtrBox, - length: i32, -} - -impl FixedSizeBinaryArray { - /// Returns the element at index `i` as a byte slice. - pub fn value(&self, i: usize) -> &[u8] { - assert!( - i < self.data.len(), - "FixedSizeBinaryArray out of bounds access" - ); - let offset = i.checked_add(self.data.offset()).unwrap(); - unsafe { - let pos = self.value_offset_at(offset); - std::slice::from_raw_parts( - self.value_data.get().offset(pos as isize), - (self.value_offset_at(offset + 1) - pos) as usize, - ) - } - } - - /// Returns the offset for the element at index `i`. - /// - /// Note this doesn't do any bound checking, for performance reason. - #[inline] - pub fn value_offset(&self, i: usize) -> i32 { - self.value_offset_at(self.data.offset() + i) - } - - /// Returns the length for an element. - /// - /// All elements have the same length as the array is a fixed size. - #[inline] - pub fn value_length(&self) -> i32 { - self.length - } - - /// Returns a clone of the value data buffer - pub fn value_data(&self) -> Buffer { - self.data.buffers()[0].clone() - } - - #[inline] - fn value_offset_at(&self, i: usize) -> i32 { - self.length * i as i32 - } -} - -impl From for FixedSizeBinaryArray { - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.buffers().len(), - 1, - "FixedSizeBinaryArray data should contain 1 buffer only (values)" - ); - let value_data = data.buffers()[0].raw_data(); - let length = match data.data_type() { - DataType::FixedSizeBinary(len) => *len, - _ => panic!("Expected data type to be FixedSizeBinary"), - }; - Self { - data, - value_data: RawPtrBox::new(value_data), - length, - } - } -} - -/// Creates a `FixedSizeBinaryArray` from `FixedSizeList` array -impl From for FixedSizeBinaryArray { - fn from(v: FixedSizeListArray) -> Self { - assert_eq!( - v.data_ref().child_data()[0].child_data().len(), - 0, - "FixedSizeBinaryArray can only be created from list array of u8 values \ - (i.e. FixedSizeList>)." - ); - assert_eq!( - v.data_ref().child_data()[0].data_type(), - &DataType::UInt8, - "FixedSizeBinaryArray can only be created from FixedSizeList arrays, mismatched data types." - ); - - let mut builder = ArrayData::builder(DataType::FixedSizeBinary(v.value_length())) - .len(v.len()) - .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); - if let Some(bitmap) = v.data_ref().null_bitmap() { - builder = builder - .null_count(v.data_ref().null_count()) - .null_bit_buffer(bitmap.bits.clone()) - } - - let data = builder.build(); - Self::from(data) - } -} - -impl fmt::Debug for FixedSizeBinaryArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "FixedSizeBinaryArray<{}>\n[\n", self.value_length())?; - print_long_array(self, f, |array, index, f| { - fmt::Debug::fmt(&array.value(index), f) - })?; - write!(f, "]") - } -} - -impl Array for FixedSizeBinaryArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [FixedSizeBinaryArray]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [FixedSizeBinaryArray]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - -/// A nested array type where each child (called *field*) is represented by a separate -/// array. -pub struct StructArray { - data: ArrayDataRef, - pub(crate) boxed_fields: Vec, -} - -impl StructArray { - /// Returns the field at `pos`. - pub fn column(&self, pos: usize) -> &ArrayRef { - &self.boxed_fields[pos] - } - - /// Return the number of fields in this struct array - pub fn num_columns(&self) -> usize { - self.boxed_fields.len() - } - - /// Returns the fields of the struct array - pub fn columns(&self) -> Vec<&ArrayRef> { - self.boxed_fields.iter().collect() - } - - /// Returns child array refs of the struct array - pub fn columns_ref(&self) -> Vec { - self.boxed_fields.clone() - } - - /// Return field names in this struct array - pub fn column_names(&self) -> Vec<&str> { - match self.data.data_type() { - Struct(fields) => fields - .iter() - .map(|f| f.name().as_str()) - .collect::>(), - _ => unreachable!("Struct array's data type is not struct!"), - } - } - - /// Return child array whose field name equals to column_name - pub fn column_by_name(&self, column_name: &str) -> Option<&ArrayRef> { - self.column_names() - .iter() - .position(|c| c == &column_name) - .map(|pos| self.column(pos)) - } -} - -impl From for StructArray { - fn from(data: ArrayDataRef) -> Self { - let mut boxed_fields = vec![]; - for cd in data.child_data() { - let child_data = if data.offset() != 0 || data.len() != cd.len() { - Arc::new(cd.as_ref().slice(data.offset(), data.len())) - } else { - cd.clone() - }; - boxed_fields.push(make_array(child_data)); - } - Self { data, boxed_fields } - } -} - -impl TryFrom> for StructArray { - type Error = ArrowError; - - /// builds a StructArray from a vector of names and arrays. - /// This errors if the values have a different length. - /// An entry is set to Null when all values are null. - fn try_from(values: Vec<(&str, ArrayRef)>) -> Result { - let values_len = values.len(); - - // these will be populated - let mut fields = Vec::with_capacity(values_len); - let mut child_data = Vec::with_capacity(values_len); - - // len: the size of the arrays. - let mut len: Option = None; - // null: the null mask of the arrays. - let mut null: Option = None; - for (field_name, array) in values { - let child_datum = array.data(); - let child_datum_len = child_datum.len(); - if let Some(len) = len { - if len != child_datum_len { - return Err(ArrowError::InvalidArgumentError( - format!("Array of field \"{}\" has length {}, but previous elements have length {}. - All arrays in every entry in a struct array must have the same length.", field_name, child_datum_len, len) - )); - } - } else { - len = Some(child_datum_len) - } - child_data.push(child_datum.clone()); - fields.push(Field::new( - field_name, - array.data_type().clone(), - child_datum.null_buffer().is_some(), - )); - - if let Some(child_null_buffer) = child_datum.null_buffer() { - null = Some(if let Some(null_buffer) = &null { - buffer_bin_or(null_buffer, 0, child_null_buffer, 0, child_datum_len) - } else { - child_null_buffer.clone() - }); - } else if null.is_some() { - // when one of the fields has no nulls, them there is no null in the array - null = None; - } - } - let len = len.unwrap(); - - let mut builder = ArrayData::builder(DataType::Struct(fields)) - .len(len) - .child_data(child_data); - if let Some(null_buffer) = null { - let null_count = len - bit_util::count_set_bits(null_buffer.data()); - builder = builder.null_count(null_count).null_bit_buffer(null_buffer); - } - - Ok(StructArray::from(builder.build())) - } -} - -impl Array for StructArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - /// Returns the length (i.e., number of elements) of this array - fn len(&self) -> usize { - self.data_ref().len() - } - - /// Returns the total number of bytes of memory occupied by the buffers owned by this [StructArray]. - fn get_buffer_memory_size(&self) -> usize { - self.data.get_buffer_memory_size() - } - - /// Returns the total number of bytes of memory occupied physically by this [StructArray]. - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() + mem::size_of_val(self) - } -} - -impl From> for StructArray { - fn from(v: Vec<(Field, ArrayRef)>) -> Self { - let (field_types, field_values): (Vec<_>, Vec<_>) = v.into_iter().unzip(); - - // Check the length of the child arrays - let length = field_values[0].len(); - for i in 1..field_values.len() { - assert_eq!( - length, - field_values[i].len(), - "all child arrays of a StructArray must have the same length" - ); - assert_eq!( - field_types[i].data_type(), - field_values[i].data().data_type(), - "the field data types must match the array data in a StructArray" - ) - } - - let data = ArrayData::builder(DataType::Struct(field_types)) - .child_data(field_values.into_iter().map(|a| a.data()).collect()) - .len(length) - .build(); - Self::from(data) - } -} - -impl fmt::Debug for StructArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "StructArray\n[\n")?; - for (child_index, name) in self.column_names().iter().enumerate() { - let column = self.column(child_index); - writeln!( - f, - "-- child {}: \"{}\" ({:?})", - child_index, - name, - column.data_type() - )?; - fmt::Debug::fmt(column, f)?; - writeln!(f)?; - } - write!(f, "]") - } -} - -impl From<(Vec<(Field, ArrayRef)>, Buffer, usize)> for StructArray { - fn from(triple: (Vec<(Field, ArrayRef)>, Buffer, usize)) -> Self { - let (field_types, field_values): (Vec<_>, Vec<_>) = triple.0.into_iter().unzip(); - - // Check the length of the child arrays - let length = field_values[0].len(); - for i in 1..field_values.len() { - assert_eq!( - length, - field_values[i].len(), - "all child arrays of a StructArray must have the same length" - ); - assert_eq!( - field_types[i].data_type(), - field_values[i].data().data_type(), - "the field data types must match the array data in a StructArray" - ) - } - - let data = ArrayData::builder(DataType::Struct(field_types)) - .null_bit_buffer(triple.1) - .child_data(field_values.into_iter().map(|a| a.data()).collect()) - .len(length) - .null_count(triple.2) - .build(); - Self::from(data) - } -} - -/// A dictionary array where each element is a single value indexed by an integer key. -/// This is mostly used to represent strings or a limited set of primitive types as integers, -/// for example when doing NLP analysis or representing chromosomes by name. -/// -/// Example **with nullable** data: -/// -/// ``` -/// use arrow::array::{DictionaryArray, Int8Array}; -/// use arrow::datatypes::Int8Type; -/// let test = vec!["a", "a", "b", "c"]; -/// let array : DictionaryArray = test.iter().map(|&x| if x == "b" {None} else {Some(x)}).collect(); -/// assert_eq!(array.keys(), &Int8Array::from(vec![Some(0), Some(0), None, Some(1)])); -/// ``` -/// -/// Example **without nullable** data: -/// -/// ``` -/// use arrow::array::{DictionaryArray, Int8Array}; -/// use arrow::datatypes::Int8Type; -/// let test = vec!["a", "a", "b", "c"]; -/// let array : DictionaryArray = test.into_iter().collect(); -/// assert_eq!(array.keys(), &Int8Array::from(vec![0, 0, 1, 2])); -/// ``` -pub struct DictionaryArray { - /// Data of this dictionary. Note that this is _not_ compatible with the C Data interface, - /// as, in the current implementation, `values` below are the first child of this struct. - data: ArrayDataRef, - - /// The keys of this dictionary. These are constructed from the buffer and null bitmap - /// of `data`. - /// Also, note that these do not correspond to the true values of this array. Rather, they map - /// to the real values. - keys: PrimitiveArray, - - /// Array of dictionary values (can by any DataType). - values: ArrayRef, - - /// Values are ordered. - is_ordered: bool, -} - -impl<'a, K: ArrowPrimitiveType> DictionaryArray { - /// Return an iterator to the keys of this dictionary. - pub fn keys(&self) -> &PrimitiveArray { - &self.keys - } - - /// Returns an array view of the keys of this dictionary - pub fn keys_array(&self) -> PrimitiveArray { - let data = self.data_ref(); - let keys_data = ArrayData::new( - K::DATA_TYPE, - data.len(), - Some(data.null_count()), - data.null_buffer().cloned(), - data.offset(), - data.buffers().to_vec(), - vec![], - ); - PrimitiveArray::::from(Arc::new(keys_data)) - } - - /// Returns the lookup key by doing reverse dictionary lookup - pub fn lookup_key(&self, value: &str) -> Option { - let rd_buf: &StringArray = - self.values.as_any().downcast_ref::().unwrap(); - - (0..rd_buf.len()) - .position(|i| rd_buf.value(i) == value) - .map(K::Native::from_usize) - .flatten() - } - - /// Returns an `ArrayRef` to the dictionary values. - pub fn values(&self) -> ArrayRef { - self.values.clone() - } - - /// Returns a clone of the value type of this list. - pub fn value_type(&self) -> DataType { - self.values.data_ref().data_type().clone() - } - - /// The length of the dictionary is the length of the keys array. - pub fn len(&self) -> usize { - self.keys.len() - } - - /// Whether this dictionary is empty - pub fn is_empty(&self) -> bool { - self.keys.is_empty() - } - - // Currently exists for compatibility purposes with Arrow IPC. - pub fn is_ordered(&self) -> bool { - self.is_ordered - } -} - -/// Constructs a `DictionaryArray` from an array data reference. -impl From for DictionaryArray { - fn from(data: ArrayDataRef) -> Self { - assert_eq!( - data.buffers().len(), - 1, - "DictionaryArray data should contain a single buffer only (keys)." - ); - assert_eq!( - data.child_data().len(), - 1, - "DictionaryArray should contain a single child array (values)." - ); - - if let DataType::Dictionary(key_data_type, _) = data.data_type() { - if key_data_type.as_ref() != &T::DATA_TYPE { - panic!("DictionaryArray's data type must match.") - }; - // create a zero-copy of the keys' data - let keys = PrimitiveArray::::from(Arc::new(ArrayData::new( - T::DATA_TYPE, - data.len(), - Some(data.null_count()), - data.null_buffer().cloned(), - data.offset(), - data.buffers().to_vec(), - vec![], - ))); - let values = make_array(data.child_data()[0].clone()); - Self { - data, - keys, - values, - is_ordered: false, - } - } else { - panic!("DictionaryArray must have Dictionary data type.") - } - } -} - -/// Constructs a `DictionaryArray` from an iterator of optional strings. -impl<'a, T: ArrowPrimitiveType + ArrowDictionaryKeyType> FromIterator> - for DictionaryArray -{ - fn from_iter>>(iter: I) -> Self { - let it = iter.into_iter(); - let (lower, _) = it.size_hint(); - let key_builder = PrimitiveBuilder::::new(lower); - let value_builder = StringBuilder::new(256); - let mut builder = StringDictionaryBuilder::new(key_builder, value_builder); - it.for_each(|i| { - if let Some(i) = i { - // Note: impl ... for Result> fails with - // error[E0117]: only traits defined in the current crate can be implemented for arbitrary types - builder - .append(i) - .expect("Unable to append a value to a dictionary array."); - } else { - builder - .append_null() - .expect("Unable to append a null value to a dictionary array."); - } - }); - - builder.finish() - } -} - -/// Constructs a `DictionaryArray` from an iterator of strings. -impl<'a, T: ArrowPrimitiveType + ArrowDictionaryKeyType> FromIterator<&'a str> - for DictionaryArray -{ - fn from_iter>(iter: I) -> Self { - let it = iter.into_iter(); - let (lower, _) = it.size_hint(); - let key_builder = PrimitiveBuilder::::new(lower); - let value_builder = StringBuilder::new(256); - let mut builder = StringDictionaryBuilder::new(key_builder, value_builder); - it.for_each(|i| { - builder - .append(i) - .expect("Unable to append a value to a dictionary array."); - }); - - builder.finish() - } -} - -impl Array for DictionaryArray { - fn as_any(&self) -> &Any { - self - } - - fn data(&self) -> ArrayDataRef { - self.data.clone() - } - - fn data_ref(&self) -> &ArrayDataRef { - &self.data - } - - fn get_buffer_memory_size(&self) -> usize { - // Since both `keys` and `values` derive (are references from) `data`, we only need to account for `data`. - self.data.get_buffer_memory_size() - } - - fn get_array_memory_size(&self) -> usize { - self.data.get_array_memory_size() - + self.keys.get_array_memory_size() - + self.values.get_array_memory_size() - + mem::size_of_val(self) - } -} - -impl fmt::Debug for DictionaryArray { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - writeln!( - f, - "DictionaryArray {{keys: {:?} values: {:?}}}", - self.keys, self.values - ) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - use std::sync::Arc; - use std::thread; - - use crate::buffer::Buffer; - use crate::datatypes::{DataType, Field}; - use crate::{bitmap::Bitmap, memory}; - - #[test] - fn test_primitive_array_from_vec() { - let buf = Buffer::from(&[0, 1, 2, 3, 4].to_byte_slice()); - let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); - let slice = unsafe { std::slice::from_raw_parts(arr.raw_values(), 5) }; - assert_eq!(buf, arr.values()); - assert_eq!(&[0, 1, 2, 3, 4], slice); - assert_eq!(5, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - for i in 0..5 { - assert!(!arr.is_null(i)); - assert!(arr.is_valid(i)); - assert_eq!(i as i32, arr.value(i)); - } - - assert_eq!(64, arr.get_buffer_memory_size()); - let internals_of_primitive_array = 8 + 72; // RawPtrBox & Arc combined. - assert_eq!( - arr.get_buffer_memory_size() + internals_of_primitive_array, - arr.get_array_memory_size() - ); - } - - #[test] - fn test_primitive_array_from_vec_option() { - // Test building a primitive array with null values - let arr = Int32Array::from(vec![Some(0), None, Some(2), None, Some(4)]); - assert_eq!(5, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(2, arr.null_count()); - for i in 0..5 { - if i % 2 == 0 { - assert!(!arr.is_null(i)); - assert!(arr.is_valid(i)); - assert_eq!(i as i32, arr.value(i)); - } else { - assert!(arr.is_null(i)); - assert!(!arr.is_valid(i)); - } - } - - assert_eq!(128, arr.get_buffer_memory_size()); - let internals_of_primitive_array = 8 + 72 + 16; // RawPtrBox & Arc and it's null_bitmap combined. - assert_eq!( - arr.get_buffer_memory_size() + internals_of_primitive_array, - arr.get_array_memory_size() - ); - } - - #[test] - fn test_date64_array_from_vec_option() { - // Test building a primitive array with null values - // we use Int32 and Int64 as a backing array, so all Int32 and Int64 conventions - // work - let arr: PrimitiveArray = - vec![Some(1550902545147), None, Some(1550902545147)].into(); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - for i in 0..3 { - if i % 2 == 0 { - assert!(!arr.is_null(i)); - assert!(arr.is_valid(i)); - assert_eq!(1550902545147, arr.value(i)); - // roundtrip to and from datetime - assert_eq!( - 1550902545147, - arr.value_as_datetime(i).unwrap().timestamp_millis() - ); - } else { - assert!(arr.is_null(i)); - assert!(!arr.is_valid(i)); - } - } - } - - #[test] - fn test_time32_millisecond_array_from_vec() { - // 1: 00:00:00.001 - // 37800005: 10:30:00.005 - // 86399210: 23:59:59.210 - let arr: PrimitiveArray = - vec![1, 37_800_005, 86_399_210].into(); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - let formatted = vec!["00:00:00.001", "10:30:00.005", "23:59:59.210"]; - for i in 0..3 { - // check that we can't create dates or datetimes from time instances - assert_eq!(None, arr.value_as_datetime(i)); - assert_eq!(None, arr.value_as_date(i)); - let time = arr.value_as_time(i).unwrap(); - assert_eq!(formatted[i], time.format("%H:%M:%S%.3f").to_string()); - } - } - - #[test] - fn test_time64_nanosecond_array_from_vec() { - // Test building a primitive array with null values - // we use Int32 and Int64 as a backing array, so all Int32 and Int64 conventions - // work - - // 1e6: 00:00:00.001 - // 37800005e6: 10:30:00.005 - // 86399210e6: 23:59:59.210 - let arr: PrimitiveArray = - vec![1_000_000, 37_800_005_000_000, 86_399_210_000_000].into(); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - let formatted = vec!["00:00:00.001", "10:30:00.005", "23:59:59.210"]; - for i in 0..3 { - // check that we can't create dates or datetimes from time instances - assert_eq!(None, arr.value_as_datetime(i)); - assert_eq!(None, arr.value_as_date(i)); - let time = arr.value_as_time(i).unwrap(); - assert_eq!(formatted[i], time.format("%H:%M:%S%.3f").to_string()); - } - } - - #[test] - fn test_interval_array_from_vec() { - // intervals are currently not treated specially, but are Int32 and Int64 arrays - let arr = IntervalYearMonthArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - - // a day_time interval contains days and milliseconds, but we do not yet have accessors for the values - let arr = IntervalDayTimeArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - } - - #[test] - fn test_duration_array_from_vec() { - let arr = DurationSecondArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - - let arr = DurationMillisecondArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - - let arr = DurationMicrosecondArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - - let arr = DurationNanosecondArray::from(vec![Some(1), None, Some(-5)]); - assert_eq!(3, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert!(arr.is_null(1)); - assert_eq!(-5, arr.value(2)); - } - - #[test] - fn test_timestamp_array_from_vec() { - let arr = TimestampSecondArray::from_vec(vec![1, -5], None); - assert_eq!(2, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert_eq!(-5, arr.value(1)); - - let arr = TimestampMillisecondArray::from_vec(vec![1, -5], None); - assert_eq!(2, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert_eq!(-5, arr.value(1)); - - let arr = TimestampMicrosecondArray::from_vec(vec![1, -5], None); - assert_eq!(2, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert_eq!(-5, arr.value(1)); - - let arr = TimestampNanosecondArray::from_vec(vec![1, -5], None); - assert_eq!(2, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - assert_eq!(1, arr.value(0)); - assert_eq!(-5, arr.value(1)); - } - - #[test] - fn test_primitive_array_slice() { - let arr = Int32Array::from(vec![ - Some(0), - None, - Some(2), - None, - Some(4), - Some(5), - Some(6), - None, - None, - ]); - assert_eq!(9, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(4, arr.null_count()); - - let arr2 = arr.slice(2, 5); - assert_eq!(5, arr2.len()); - assert_eq!(2, arr2.offset()); - assert_eq!(1, arr2.null_count()); - - for i in 0..arr2.len() { - assert_eq!(i == 1, arr2.is_null(i)); - assert_eq!(i != 1, arr2.is_valid(i)); - } - - let arr3 = arr2.slice(2, 3); - assert_eq!(3, arr3.len()); - assert_eq!(4, arr3.offset()); - assert_eq!(0, arr3.null_count()); - - let int_arr = arr3.as_any().downcast_ref::().unwrap(); - assert_eq!(4, int_arr.value(0)); - assert_eq!(5, int_arr.value(1)); - assert_eq!(6, int_arr.value(2)); - } - - #[test] - fn test_boolean_array_slice() { - let arr = BooleanArray::from(vec![ - Some(true), - None, - Some(false), - None, - Some(true), - Some(false), - Some(true), - Some(false), - None, - Some(true), - ]); - - assert_eq!(10, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(3, arr.null_count()); - - let arr2 = arr.slice(3, 5); - assert_eq!(5, arr2.len()); - assert_eq!(3, arr2.offset()); - assert_eq!(1, arr2.null_count()); - - let bool_arr = arr2.as_any().downcast_ref::().unwrap(); - - assert_eq!(false, bool_arr.is_valid(0)); - - assert_eq!(true, bool_arr.is_valid(1)); - assert_eq!(true, bool_arr.value(1)); - - assert_eq!(true, bool_arr.is_valid(2)); - assert_eq!(false, bool_arr.value(2)); - - assert_eq!(true, bool_arr.is_valid(3)); - assert_eq!(true, bool_arr.value(3)); - - assert_eq!(true, bool_arr.is_valid(4)); - assert_eq!(false, bool_arr.value(4)); - } - - #[test] - fn test_value_slice_no_bounds_check() { - let arr = Int32Array::from(vec![2, 3, 4]); - let _slice = arr.value_slice(0, 4); - } - - #[test] - fn test_int32_fmt_debug() { - let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); - assert_eq!( - "PrimitiveArray\n[\n 0,\n 1,\n 2,\n 3,\n 4,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_fmt_debug_up_to_20_elements() { - (1..=20).for_each(|i| { - let values = (0..i).collect::>(); - let array_expected = format!( - "PrimitiveArray\n[\n{}\n]", - values - .iter() - .map(|v| { format!(" {},", v) }) - .collect::>() - .join("\n") - ); - let array = Int16Array::from(values); - - assert_eq!(array_expected, format!("{:?}", array)); - }) - } - - #[test] - fn test_int32_with_null_fmt_debug() { - let mut builder = Int32Array::builder(3); - builder.append_slice(&[0, 1]).unwrap(); - builder.append_null().unwrap(); - builder.append_slice(&[3, 4]).unwrap(); - let arr = builder.finish(); - assert_eq!( - "PrimitiveArray\n[\n 0,\n 1,\n null,\n 3,\n 4,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_boolean_fmt_debug() { - let arr = BooleanArray::from(vec![true, false, false]); - assert_eq!( - "PrimitiveArray\n[\n true,\n false,\n false,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_boolean_with_null_fmt_debug() { - let mut builder = BooleanArray::builder(3); - builder.append_value(true).unwrap(); - builder.append_null().unwrap(); - builder.append_value(false).unwrap(); - let arr = builder.finish(); - assert_eq!( - "PrimitiveArray\n[\n true,\n null,\n false,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_timestamp_fmt_debug() { - let arr: PrimitiveArray = - TimestampMillisecondArray::from_vec(vec![1546214400000, 1546214400000], None); - assert_eq!( - "PrimitiveArray\n[\n 2018-12-31T00:00:00,\n 2018-12-31T00:00:00,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_date32_fmt_debug() { - let arr: PrimitiveArray = vec![12356, 13548].into(); - assert_eq!( - "PrimitiveArray\n[\n 2003-10-31,\n 2007-02-04,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_time32second_fmt_debug() { - let arr: PrimitiveArray = vec![7201, 60054].into(); - assert_eq!( - "PrimitiveArray\n[\n 02:00:01,\n 16:40:54,\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_primitive_array_builder() { - // Test building a primitive array with ArrayData builder and offset - let buf = Buffer::from(&[0, 1, 2, 3, 4].to_byte_slice()); - let buf2 = buf.clone(); - let data = ArrayData::builder(DataType::Int32) - .len(5) - .offset(2) - .add_buffer(buf) - .build(); - let arr = Int32Array::from(data); - assert_eq!(buf2, arr.values()); - assert_eq!(5, arr.len()); - assert_eq!(0, arr.null_count()); - for i in 0..3 { - assert_eq!((i + 2) as i32, arr.value(i)); - } - } - - #[test] - #[should_panic(expected = "PrimitiveArray data should contain a single buffer only \ - (values buffer)")] - fn test_primitive_array_invalid_buffer_len() { - let data = ArrayData::builder(DataType::Int32).len(5).build(); - Int32Array::from(data); - } - - #[test] - fn test_boolean_array_from_vec() { - let buf = Buffer::from([10_u8]); - let arr = BooleanArray::from(vec![false, true, false, true]); - assert_eq!(buf, arr.values()); - assert_eq!(4, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(0, arr.null_count()); - for i in 0..4 { - assert!(!arr.is_null(i)); - assert!(arr.is_valid(i)); - assert_eq!(i == 1 || i == 3, arr.value(i), "failed at {}", i) - } - } - - #[test] - fn test_boolean_array_from_vec_option() { - let buf = Buffer::from([10_u8]); - let arr = BooleanArray::from(vec![Some(false), Some(true), None, Some(true)]); - assert_eq!(buf, arr.values()); - assert_eq!(4, arr.len()); - assert_eq!(0, arr.offset()); - assert_eq!(1, arr.null_count()); - for i in 0..4 { - if i == 2 { - assert!(arr.is_null(i)); - assert!(!arr.is_valid(i)); - } else { - assert!(!arr.is_null(i)); - assert!(arr.is_valid(i)); - assert_eq!(i == 1 || i == 3, arr.value(i), "failed at {}", i) - } - } - } - - #[test] - fn test_boolean_array_builder() { - // Test building a boolean array with ArrayData builder and offset - // 000011011 - let buf = Buffer::from([27_u8]); - let buf2 = buf.clone(); - let data = ArrayData::builder(DataType::Boolean) - .len(5) - .offset(2) - .add_buffer(buf) - .build(); - let arr = BooleanArray::from(data); - assert_eq!(buf2, arr.values()); - assert_eq!(5, arr.len()); - assert_eq!(2, arr.offset()); - assert_eq!(0, arr.null_count()); - for i in 0..3 { - assert_eq!(i != 0, arr.value(i), "failed at {}", i); - } - } - - #[test] - #[should_panic(expected = "PrimitiveArray data should contain a single buffer only \ - (values buffer)")] - fn test_boolean_array_invalid_buffer_len() { - let data = ArrayData::builder(DataType::Boolean).len(5).build(); - BooleanArray::from(data); - } - - #[test] - fn test_list_array() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(8) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) - .build(); - - // Construct a buffer for value offsets, for the nested array: - // [[0, 1, 2], [3, 4, 5], [6, 7]] - let value_offsets = Buffer::from(&[0, 3, 6, 8].to_byte_slice()); - - // Construct a list array from the above two - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type.clone()) - .len(3) - .add_buffer(value_offsets.clone()) - .add_child_data(value_data.clone()) - .build(); - let list_array = ListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!(6, list_array.value_offset(2)); - assert_eq!(2, list_array.value_length(2)); - assert_eq!( - 0, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - for i in 0..3 { - assert!(list_array.is_valid(i)); - assert!(!list_array.is_null(i)); - } - - // Now test with a non-zero offset - let list_data = ArrayData::builder(list_data_type) - .len(3) - .offset(1) - .add_buffer(value_offsets) - .add_child_data(value_data.clone()) - .build(); - let list_array = ListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!(6, list_array.value_offset(1)); - assert_eq!(2, list_array.value_length(1)); - assert_eq!( - 3, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - } - - #[test] - fn test_large_list_array() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(8) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) - .build(); - - // Construct a buffer for value offsets, for the nested array: - // [[0, 1, 2], [3, 4, 5], [6, 7]] - let value_offsets = Buffer::from(&[0i64, 3, 6, 8].to_byte_slice()); - - // Construct a list array from the above two - let list_data_type = - DataType::LargeList(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type.clone()) - .len(3) - .add_buffer(value_offsets.clone()) - .add_child_data(value_data.clone()) - .build(); - let list_array = LargeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!(6, list_array.value_offset(2)); - assert_eq!(2, list_array.value_length(2)); - assert_eq!( - 0, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - for i in 0..3 { - assert!(list_array.is_valid(i)); - assert!(!list_array.is_null(i)); - } - - // Now test with a non-zero offset - let list_data = ArrayData::builder(list_data_type) - .len(3) - .offset(1) - .add_buffer(value_offsets) - .add_child_data(value_data.clone()) - .build(); - let list_array = LargeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!(6, list_array.value_offset(1)); - assert_eq!(2, list_array.value_length(1)); - assert_eq!( - 3, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - } - - #[test] - fn test_dictionary_array() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int8) - .len(8) - .add_buffer(Buffer::from( - &[10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(), - )) - .build(); - - // Construct a buffer for value offsets, for the nested array: - let keys = Buffer::from(&[2_i16, 3, 4].to_byte_slice()); - - // Construct a dictionary array from the above two - let key_type = DataType::Int16; - let value_type = DataType::Int8; - let dict_data_type = - DataType::Dictionary(Box::new(key_type), Box::new(value_type)); - let dict_data = ArrayData::builder(dict_data_type.clone()) - .len(3) - .add_buffer(keys.clone()) - .add_child_data(value_data.clone()) - .build(); - let dict_array = Int16DictionaryArray::from(dict_data); - - let values = dict_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int8, dict_array.value_type()); - assert_eq!(3, dict_array.len()); - - // Null count only makes sense in terms of the component arrays. - assert_eq!(0, dict_array.null_count()); - assert_eq!(0, dict_array.values().null_count()); - assert_eq!(dict_array.keys(), &Int16Array::from(vec![2_i16, 3, 4])); - - // Now test with a non-zero offset - let dict_data = ArrayData::builder(dict_data_type) - .len(2) - .offset(1) - .add_buffer(keys) - .add_child_data(value_data.clone()) - .build(); - let dict_array = Int16DictionaryArray::from(dict_data); - - let values = dict_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int8, dict_array.value_type()); - assert_eq!(2, dict_array.len()); - assert_eq!(dict_array.keys(), &Int16Array::from(vec![3_i16, 4])); - } - - #[test] - fn test_fixed_size_list_array() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(9) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7, 8].to_byte_slice())) - .build(); - - // Construct a list array from the above two - let list_data_type = DataType::FixedSizeList( - Box::new(Field::new("item", DataType::Int32, false)), - 3, - ); - let list_data = ArrayData::builder(list_data_type.clone()) - .len(3) - .add_child_data(value_data.clone()) - .build(); - let list_array = FixedSizeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!(6, list_array.value_offset(2)); - assert_eq!(3, list_array.value_length()); - assert_eq!( - 0, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - for i in 0..3 { - assert!(list_array.is_valid(i)); - assert!(!list_array.is_null(i)); - } - - // Now test with a non-zero offset - let list_data = ArrayData::builder(list_data_type) - .len(3) - .offset(1) - .add_child_data(value_data.clone()) - .build(); - let list_array = FixedSizeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(3, list_array.len()); - assert_eq!(0, list_array.null_count()); - assert_eq!( - 3, - list_array - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - ); - assert_eq!(6, list_array.value_offset(1)); - assert_eq!(3, list_array.value_length()); - } - - #[test] - #[should_panic( - expected = "FixedSizeListArray child array length should be a multiple of 3" - )] - fn test_fixed_size_list_array_unequal_children() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(8) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) - .build(); - - // Construct a list array from the above two - let list_data_type = DataType::FixedSizeList( - Box::new(Field::new("item", DataType::Int32, false)), - 3, - ); - let list_data = ArrayData::builder(list_data_type) - .len(3) - .add_child_data(value_data) - .build(); - FixedSizeListArray::from(list_data); - } - - #[test] - fn test_list_array_slice() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(10) - .add_buffer(Buffer::from( - &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), - )) - .build(); - - // Construct a buffer for value offsets, for the nested array: - // [[0, 1], null, null, [2, 3], [4, 5], null, [6, 7, 8], null, [9]] - let value_offsets = - Buffer::from(&[0, 2, 2, 2, 4, 6, 6, 9, 9, 10].to_byte_slice()); - // 01011001 00000001 - let mut null_bits: [u8; 2] = [0; 2]; - bit_util::set_bit(&mut null_bits, 0); - bit_util::set_bit(&mut null_bits, 3); - bit_util::set_bit(&mut null_bits, 4); - bit_util::set_bit(&mut null_bits, 6); - bit_util::set_bit(&mut null_bits, 8); - - // Construct a list array from the above two - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .len(9) - .add_buffer(value_offsets) - .add_child_data(value_data.clone()) - .null_bit_buffer(Buffer::from(null_bits)) - .build(); - let list_array = ListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(9, list_array.len()); - assert_eq!(4, list_array.null_count()); - assert_eq!(2, list_array.value_offset(3)); - assert_eq!(2, list_array.value_length(3)); - - let sliced_array = list_array.slice(1, 6); - assert_eq!(6, sliced_array.len()); - assert_eq!(1, sliced_array.offset()); - assert_eq!(3, sliced_array.null_count()); - - for i in 0..sliced_array.len() { - if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { - assert!(sliced_array.is_valid(i)); - } else { - assert!(sliced_array.is_null(i)); - } - } - - // Check offset and length for each non-null value. - let sliced_list_array = - sliced_array.as_any().downcast_ref::().unwrap(); - assert_eq!(2, sliced_list_array.value_offset(2)); - assert_eq!(2, sliced_list_array.value_length(2)); - assert_eq!(4, sliced_list_array.value_offset(3)); - assert_eq!(2, sliced_list_array.value_length(3)); - assert_eq!(6, sliced_list_array.value_offset(5)); - assert_eq!(3, sliced_list_array.value_length(5)); - } - - #[test] - fn test_large_list_array_slice() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(10) - .add_buffer(Buffer::from( - &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), - )) - .build(); - - // Construct a buffer for value offsets, for the nested array: - // [[0, 1], null, null, [2, 3], [4, 5], null, [6, 7, 8], null, [9]] - let value_offsets = - Buffer::from(&[0i64, 2, 2, 2, 4, 6, 6, 9, 9, 10].to_byte_slice()); - // 01011001 00000001 - let mut null_bits: [u8; 2] = [0; 2]; - bit_util::set_bit(&mut null_bits, 0); - bit_util::set_bit(&mut null_bits, 3); - bit_util::set_bit(&mut null_bits, 4); - bit_util::set_bit(&mut null_bits, 6); - bit_util::set_bit(&mut null_bits, 8); - - // Construct a list array from the above two - let list_data_type = - DataType::LargeList(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .len(9) - .add_buffer(value_offsets) - .add_child_data(value_data.clone()) - .null_bit_buffer(Buffer::from(null_bits)) - .build(); - let list_array = LargeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(9, list_array.len()); - assert_eq!(4, list_array.null_count()); - assert_eq!(2, list_array.value_offset(3)); - assert_eq!(2, list_array.value_length(3)); - - let sliced_array = list_array.slice(1, 6); - assert_eq!(6, sliced_array.len()); - assert_eq!(1, sliced_array.offset()); - assert_eq!(3, sliced_array.null_count()); - - for i in 0..sliced_array.len() { - if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { - assert!(sliced_array.is_valid(i)); - } else { - assert!(sliced_array.is_null(i)); - } - } - - // Check offset and length for each non-null value. - let sliced_list_array = sliced_array - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(2, sliced_list_array.value_offset(2)); - assert_eq!(2, sliced_list_array.value_length(2)); - assert_eq!(4, sliced_list_array.value_offset(3)); - assert_eq!(2, sliced_list_array.value_length(3)); - assert_eq!(6, sliced_list_array.value_offset(5)); - assert_eq!(3, sliced_list_array.value_length(5)); - } - - #[test] - fn test_fixed_size_list_array_slice() { - // Construct a value array - let value_data = ArrayData::builder(DataType::Int32) - .len(10) - .add_buffer(Buffer::from( - &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), - )) - .build(); - - // Set null buts for the nested array: - // [[0, 1], null, null, [6, 7], [8, 9]] - // 01011001 00000001 - let mut null_bits: [u8; 1] = [0; 1]; - bit_util::set_bit(&mut null_bits, 0); - bit_util::set_bit(&mut null_bits, 3); - bit_util::set_bit(&mut null_bits, 4); - - // Construct a fixed size list array from the above two - let list_data_type = DataType::FixedSizeList( - Box::new(Field::new("item", DataType::Int32, false)), - 2, - ); - let list_data = ArrayData::builder(list_data_type) - .len(5) - .add_child_data(value_data.clone()) - .null_bit_buffer(Buffer::from(null_bits)) - .build(); - let list_array = FixedSizeListArray::from(list_data); - - let values = list_array.values(); - assert_eq!(value_data, values.data()); - assert_eq!(DataType::Int32, list_array.value_type()); - assert_eq!(5, list_array.len()); - assert_eq!(2, list_array.null_count()); - assert_eq!(6, list_array.value_offset(3)); - assert_eq!(2, list_array.value_length()); - - let sliced_array = list_array.slice(1, 4); - assert_eq!(4, sliced_array.len()); - assert_eq!(1, sliced_array.offset()); - assert_eq!(2, sliced_array.null_count()); - - for i in 0..sliced_array.len() { - if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { - assert!(sliced_array.is_valid(i)); - } else { - assert!(sliced_array.is_null(i)); - } - } - - // Check offset and length for each non-null value. - let sliced_list_array = sliced_array - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(2, sliced_list_array.value_length()); - assert_eq!(6, sliced_list_array.value_offset(2)); - assert_eq!(8, sliced_list_array.value_offset(3)); - } - - #[test] - #[should_panic( - expected = "ListArray data should contain a single buffer only (value offsets)" - )] - fn test_list_array_invalid_buffer_len() { - let value_data = ArrayData::builder(DataType::Int32) - .len(8) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) - .build(); - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .len(3) - .add_child_data(value_data) - .build(); - ListArray::from(list_data); - } - - #[test] - #[should_panic( - expected = "ListArray should contain a single child array (values array)" - )] - fn test_list_array_invalid_child_array_len() { - let value_offsets = Buffer::from(&[0, 2, 5, 7].to_byte_slice()); - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .len(3) - .add_buffer(value_offsets) - .build(); - ListArray::from(list_data); - } - - #[test] - #[should_panic(expected = "offsets do not start at zero")] - fn test_list_array_invalid_value_offset_start() { - let value_data = ArrayData::builder(DataType::Int32) - .len(8) - .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) - .build(); - - let value_offsets = Buffer::from(&[2, 2, 5, 7].to_byte_slice()); - - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .len(3) - .add_buffer(value_offsets) - .add_child_data(value_data) - .build(); - ListArray::from(list_data); - } - - #[test] - fn test_binary_array() { - let values: [u8; 12] = [ - b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', - ]; - let offsets: [i32; 4] = [0, 5, 5, 12]; - - // Array data: ["hello", "", "parquet"] - let array_data = ArrayData::builder(DataType::Binary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array = BinaryArray::from(array_data); - assert_eq!(3, binary_array.len()); - assert_eq!(0, binary_array.null_count()); - assert_eq!([b'h', b'e', b'l', b'l', b'o'], binary_array.value(0)); - assert_eq!([] as [u8; 0], binary_array.value(1)); - assert_eq!( - [b'p', b'a', b'r', b'q', b'u', b'e', b't'], - binary_array.value(2) - ); - assert_eq!(5, binary_array.value_offset(2)); - assert_eq!(7, binary_array.value_length(2)); - for i in 0..3 { - assert!(binary_array.is_valid(i)); - assert!(!binary_array.is_null(i)); - } - - // Test binary array with offset - let array_data = ArrayData::builder(DataType::Binary) - .len(4) - .offset(1) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array = BinaryArray::from(array_data); - assert_eq!( - [b'p', b'a', b'r', b'q', b'u', b'e', b't'], - binary_array.value(1) - ); - assert_eq!(5, binary_array.value_offset(0)); - assert_eq!(0, binary_array.value_length(0)); - assert_eq!(5, binary_array.value_offset(1)); - assert_eq!(7, binary_array.value_length(1)); - } - - #[test] - fn test_large_binary_array() { - let values: [u8; 12] = [ - b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', - ]; - let offsets: [i64; 4] = [0, 5, 5, 12]; - - // Array data: ["hello", "", "parquet"] - let array_data = ArrayData::builder(DataType::LargeBinary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array = LargeBinaryArray::from(array_data); - assert_eq!(3, binary_array.len()); - assert_eq!(0, binary_array.null_count()); - assert_eq!([b'h', b'e', b'l', b'l', b'o'], binary_array.value(0)); - assert_eq!([] as [u8; 0], binary_array.value(1)); - assert_eq!( - [b'p', b'a', b'r', b'q', b'u', b'e', b't'], - binary_array.value(2) - ); - assert_eq!(5, binary_array.value_offset(2)); - assert_eq!(7, binary_array.value_length(2)); - for i in 0..3 { - assert!(binary_array.is_valid(i)); - assert!(!binary_array.is_null(i)); - } - - // Test binary array with offset - let array_data = ArrayData::builder(DataType::LargeBinary) - .len(4) - .offset(1) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array = LargeBinaryArray::from(array_data); - assert_eq!( - [b'p', b'a', b'r', b'q', b'u', b'e', b't'], - binary_array.value(1) - ); - assert_eq!(5, binary_array.value_offset(0)); - assert_eq!(0, binary_array.value_length(0)); - assert_eq!(5, binary_array.value_offset(1)); - assert_eq!(7, binary_array.value_length(1)); - } - - #[test] - fn test_binary_array_from_list_array() { - let values: [u8; 12] = [ - b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', - ]; - let values_data = ArrayData::builder(DataType::UInt8) - .len(12) - .add_buffer(Buffer::from(&values[..])) - .build(); - let offsets: [i32; 4] = [0, 5, 5, 12]; - - // Array data: ["hello", "", "parquet"] - let array_data1 = ArrayData::builder(DataType::Binary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array1 = BinaryArray::from(array_data1); - - let array_data2 = ArrayData::builder(DataType::Binary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_child_data(values_data) - .build(); - let list_array = ListArray::from(array_data2); - let binary_array2 = BinaryArray::from(list_array); - - assert_eq!(2, binary_array2.data().buffers().len()); - assert_eq!(0, binary_array2.data().child_data().len()); - - assert_eq!(binary_array1.len(), binary_array2.len()); - assert_eq!(binary_array1.null_count(), binary_array2.null_count()); - for i in 0..binary_array1.len() { - assert_eq!(binary_array1.value(i), binary_array2.value(i)); - assert_eq!(binary_array1.value_offset(i), binary_array2.value_offset(i)); - assert_eq!(binary_array1.value_length(i), binary_array2.value_length(i)); - } - } - - #[test] - fn test_large_binary_array_from_list_array() { - let values: [u8; 12] = [ - b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', - ]; - let values_data = ArrayData::builder(DataType::UInt8) - .len(12) - .add_buffer(Buffer::from(&values[..])) - .build(); - let offsets: [i64; 4] = [0, 5, 5, 12]; - - // Array data: ["hello", "", "parquet"] - let array_data1 = ArrayData::builder(DataType::LargeBinary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array1 = LargeBinaryArray::from(array_data1); - - let array_data2 = ArrayData::builder(DataType::Binary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_child_data(values_data) - .build(); - let list_array = LargeListArray::from(array_data2); - let binary_array2 = LargeBinaryArray::from(list_array); - - assert_eq!(2, binary_array2.data().buffers().len()); - assert_eq!(0, binary_array2.data().child_data().len()); - - assert_eq!(binary_array1.len(), binary_array2.len()); - assert_eq!(binary_array1.null_count(), binary_array2.null_count()); - for i in 0..binary_array1.len() { - assert_eq!(binary_array1.value(i), binary_array2.value(i)); - assert_eq!(binary_array1.value_offset(i), binary_array2.value_offset(i)); - assert_eq!(binary_array1.value_length(i), binary_array2.value_length(i)); - } - } - - fn test_generic_binary_array_from_opt_vec() { - let values: Vec> = - vec![Some(b"one"), Some(b"two"), None, Some(b""), Some(b"three")]; - let array = GenericBinaryArray::::from_opt_vec(values); - assert_eq!(array.len(), 5); - assert_eq!(array.value(0), b"one"); - assert_eq!(array.value(1), b"two"); - assert_eq!(array.value(3), b""); - assert_eq!(array.value(4), b"three"); - assert_eq!(array.is_null(0), false); - assert_eq!(array.is_null(1), false); - assert_eq!(array.is_null(2), true); - assert_eq!(array.is_null(3), false); - assert_eq!(array.is_null(4), false); - } - - #[test] - fn test_large_binary_array_from_opt_vec() { - test_generic_binary_array_from_opt_vec::() - } - - #[test] - fn test_binary_array_from_opt_vec() { - test_generic_binary_array_from_opt_vec::() - } - - #[test] - fn test_string_array_from_u8_slice() { - let values: Vec<&str> = vec!["hello", "", "parquet"]; - - // Array data: ["hello", "", "parquet"] - let string_array = StringArray::from(values); - - assert_eq!(3, string_array.len()); - assert_eq!(0, string_array.null_count()); - assert_eq!("hello", string_array.value(0)); - assert_eq!("", string_array.value(1)); - assert_eq!("parquet", string_array.value(2)); - assert_eq!(5, string_array.value_offset(2)); - assert_eq!(7, string_array.value_length(2)); - for i in 0..3 { - assert!(string_array.is_valid(i)); - assert!(!string_array.is_null(i)); - } - } - - #[test] - #[should_panic(expected = "[Large]StringArray expects Datatype::[Large]Utf8")] - fn test_string_array_from_int() { - let array = LargeStringArray::from(vec!["a", "b"]); - StringArray::from(array.data()); - } - - #[test] - fn test_large_string_array_from_u8_slice() { - let values: Vec<&str> = vec!["hello", "", "parquet"]; - - // Array data: ["hello", "", "parquet"] - let string_array = LargeStringArray::from(values); - - assert_eq!(3, string_array.len()); - assert_eq!(0, string_array.null_count()); - assert_eq!("hello", string_array.value(0)); - assert_eq!("", string_array.value(1)); - assert_eq!("parquet", string_array.value(2)); - assert_eq!(5, string_array.value_offset(2)); - assert_eq!(7, string_array.value_length(2)); - for i in 0..3 { - assert!(string_array.is_valid(i)); - assert!(!string_array.is_null(i)); - } - } - - #[test] - fn test_nested_string_array() { - let string_builder = StringBuilder::new(3); - let mut list_of_string_builder = ListBuilder::new(string_builder); - - list_of_string_builder.values().append_value("foo").unwrap(); - list_of_string_builder.values().append_value("bar").unwrap(); - list_of_string_builder.append(true).unwrap(); - - list_of_string_builder - .values() - .append_value("foobar") - .unwrap(); - list_of_string_builder.append(true).unwrap(); - let list_of_strings = list_of_string_builder.finish(); - - assert_eq!(list_of_strings.len(), 2); - - let first_slot = list_of_strings.value(0); - let first_list = first_slot.as_any().downcast_ref::().unwrap(); - assert_eq!(first_list.len(), 2); - assert_eq!(first_list.value(0), "foo"); - assert_eq!(first_list.value(1), "bar"); - - let second_slot = list_of_strings.value(1); - let second_list = second_slot.as_any().downcast_ref::().unwrap(); - assert_eq!(second_list.len(), 1); - assert_eq!(second_list.value(0), "foobar"); - } - - #[test] - #[should_panic( - expected = "BinaryArray can only be created from List arrays, mismatched \ - data types." - )] - fn test_binary_array_from_incorrect_list_array_type() { - let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; - let values_data = ArrayData::builder(DataType::UInt32) - .len(12) - .add_buffer(Buffer::from(values[..].to_byte_slice())) - .build(); - let offsets: [i32; 4] = [0, 5, 5, 12]; - - let array_data = ArrayData::builder(DataType::Utf8) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_child_data(values_data) - .build(); - let list_array = ListArray::from(array_data); - BinaryArray::from(list_array); - } - - #[test] - #[should_panic( - expected = "BinaryArray can only be created from list array of u8 values \ - (i.e. List>)." - )] - fn test_binary_array_from_incorrect_list_array() { - let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; - let values_data = ArrayData::builder(DataType::UInt32) - .len(12) - .add_buffer(Buffer::from(values[..].to_byte_slice())) - .add_child_data(ArrayData::builder(DataType::Boolean).build()) - .build(); - let offsets: [i32; 4] = [0, 5, 5, 12]; - - let array_data = ArrayData::builder(DataType::Utf8) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_child_data(values_data) - .build(); - let list_array = ListArray::from(array_data); - BinaryArray::from(list_array); - } - - #[test] - fn test_fixed_size_binary_array() { - let values: [u8; 15] = *b"hellotherearrow"; - - let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) - .len(3) - .add_buffer(Buffer::from(&values[..])) - .build(); - let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data); - assert_eq!(3, fixed_size_binary_array.len()); - assert_eq!(0, fixed_size_binary_array.null_count()); - assert_eq!( - [b'h', b'e', b'l', b'l', b'o'], - fixed_size_binary_array.value(0) - ); - assert_eq!( - [b't', b'h', b'e', b'r', b'e'], - fixed_size_binary_array.value(1) - ); - assert_eq!( - [b'a', b'r', b'r', b'o', b'w'], - fixed_size_binary_array.value(2) - ); - assert_eq!(5, fixed_size_binary_array.value_length()); - assert_eq!(10, fixed_size_binary_array.value_offset(2)); - for i in 0..3 { - assert!(fixed_size_binary_array.is_valid(i)); - assert!(!fixed_size_binary_array.is_null(i)); - } - - // Test binary array with offset - let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) - .len(2) - .offset(1) - .add_buffer(Buffer::from(&values[..])) - .build(); - let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data); - assert_eq!( - [b't', b'h', b'e', b'r', b'e'], - fixed_size_binary_array.value(0) - ); - assert_eq!( - [b'a', b'r', b'r', b'o', b'w'], - fixed_size_binary_array.value(1) - ); - assert_eq!(2, fixed_size_binary_array.len()); - assert_eq!(5, fixed_size_binary_array.value_offset(0)); - assert_eq!(5, fixed_size_binary_array.value_length()); - assert_eq!(10, fixed_size_binary_array.value_offset(1)); - } - - #[test] - #[should_panic( - expected = "FixedSizeBinaryArray can only be created from list array of u8 values \ - (i.e. FixedSizeList>)." - )] - fn test_fixed_size_binary_array_from_incorrect_list_array() { - let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; - let values_data = ArrayData::builder(DataType::UInt32) - .len(12) - .add_buffer(Buffer::from(values[..].to_byte_slice())) - .add_child_data(ArrayData::builder(DataType::Boolean).build()) - .build(); - - let array_data = ArrayData::builder(DataType::FixedSizeList( - Box::new(Field::new("item", DataType::Binary, false)), - 4, - )) - .len(3) - .add_child_data(values_data) - .build(); - let list_array = FixedSizeListArray::from(array_data); - FixedSizeBinaryArray::from(list_array); - } - - #[test] - #[should_panic(expected = "BinaryArray out of bounds access")] - fn test_binary_array_get_value_index_out_of_bound() { - let values: [u8; 12] = - [104, 101, 108, 108, 111, 112, 97, 114, 113, 117, 101, 116]; - let offsets: [i32; 4] = [0, 5, 5, 12]; - let array_data = ArrayData::builder(DataType::Binary) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let binary_array = BinaryArray::from(array_data); - binary_array.value(4); - } - - #[test] - #[should_panic(expected = "StringArray out of bounds access")] - fn test_string_array_get_value_index_out_of_bound() { - let values: [u8; 12] = [ - b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', - ]; - let offsets: [i32; 4] = [0, 5, 5, 12]; - let array_data = ArrayData::builder(DataType::Utf8) - .len(3) - .add_buffer(Buffer::from(offsets.to_byte_slice())) - .add_buffer(Buffer::from(&values[..])) - .build(); - let string_array = StringArray::from(array_data); - string_array.value(4); - } - - #[test] - fn test_binary_array_fmt_debug() { - let values: [u8; 15] = *b"hellotherearrow"; - - let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) - .len(3) - .add_buffer(Buffer::from(&values[..])) - .build(); - let arr = FixedSizeBinaryArray::from(array_data); - assert_eq!( - "FixedSizeBinaryArray<5>\n[\n [104, 101, 108, 108, 111],\n [116, 104, 101, 114, 101],\n [97, 114, 114, 111, 119],\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_string_array_fmt_debug() { - let arr: StringArray = vec!["hello", "arrow"].into(); - assert_eq!( - "StringArray\n[\n \"hello\",\n \"arrow\",\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_large_string_array_fmt_debug() { - let arr: LargeStringArray = vec!["hello", "arrow"].into(); - assert_eq!( - "LargeStringArray\n[\n \"hello\",\n \"arrow\",\n]", - format!("{:?}", arr) - ); - } - - #[test] - fn test_string_array_from_iter() { - let data = vec![Some("hello"), None, Some("arrow")]; - // from Vec> - let array1 = StringArray::from(data.clone()); - // from Iterator> - let array2: StringArray = data.clone().into_iter().collect(); - // from Iterator> - let array3: StringArray = data - .into_iter() - .map(|x| x.map(|s| format!("{}", s))) - .collect(); - - assert_eq!(array1, array2); - assert_eq!(array2, array3); - } - - #[test] - fn test_struct_array_builder() { - let boolean_data = ArrayData::builder(DataType::Boolean) - .len(4) - .add_buffer(Buffer::from([false, false, true, true].to_byte_slice())) - .build(); - let int_data = ArrayData::builder(DataType::Int64) - .len(4) - .add_buffer(Buffer::from([42, 28, 19, 31].to_byte_slice())) - .build(); - let mut field_types = vec![]; - field_types.push(Field::new("a", DataType::Boolean, false)); - field_types.push(Field::new("b", DataType::Int64, false)); - let struct_array_data = ArrayData::builder(DataType::Struct(field_types)) - .len(4) - .add_child_data(boolean_data.clone()) - .add_child_data(int_data.clone()) - .build(); - let struct_array = StructArray::from(struct_array_data); - - assert_eq!(boolean_data, struct_array.column(0).data()); - assert_eq!(int_data, struct_array.column(1).data()); - } - - #[test] - fn test_struct_array_from() { - let boolean_data = ArrayData::builder(DataType::Boolean) - .len(4) - .add_buffer(Buffer::from([12_u8])) - .build(); - let int_data = ArrayData::builder(DataType::Int32) - .len(4) - .add_buffer(Buffer::from([42, 28, 19, 31].to_byte_slice())) - .build(); - let struct_array = StructArray::from(vec![ - ( - Field::new("b", DataType::Boolean, false), - Arc::new(BooleanArray::from(vec![false, false, true, true])) - as Arc, - ), - ( - Field::new("c", DataType::Int32, false), - Arc::new(Int32Array::from(vec![42, 28, 19, 31])), - ), - ]); - assert_eq!(boolean_data, struct_array.column(0).data()); - assert_eq!(int_data, struct_array.column(1).data()); - assert_eq!(4, struct_array.len()); - assert_eq!(0, struct_array.null_count()); - assert_eq!(0, struct_array.offset()); - } - - /// validates that the in-memory representation follows [the spec](https://arrow.apache.org/docs/format/Columnar.html#struct-layout) - #[test] - fn test_struct_array_from_vec() { - let strings: ArrayRef = Arc::new(StringArray::from(vec![ - Some("joe"), - None, - None, - Some("mark"), - ])); - let ints: ArrayRef = - Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])); - - let arr = - StructArray::try_from(vec![("f1", strings.clone()), ("f2", ints.clone())]) - .unwrap(); - - let struct_data = arr.data(); - assert_eq!(4, struct_data.len()); - assert_eq!(1, struct_data.null_count()); - assert_eq!( - // 00001011 - &Some(Bitmap::from(Buffer::from(&[11_u8]))), - struct_data.null_bitmap() - ); - - let expected_string_data = ArrayData::builder(DataType::Utf8) - .len(4) - .null_count(2) - .null_bit_buffer(Buffer::from(&[9_u8])) - .add_buffer(Buffer::from(&[0, 3, 3, 3, 7].to_byte_slice())) - .add_buffer(Buffer::from("joemark".as_bytes())) - .build(); - - let expected_int_data = ArrayData::builder(DataType::Int32) - .len(4) - .null_count(1) - .null_bit_buffer(Buffer::from(&[11_u8])) - .add_buffer(Buffer::from(&[1, 2, 0, 4].to_byte_slice())) - .build(); - - assert_eq!(expected_string_data, arr.column(0).data()); - - // TODO: implement equality for ArrayData - assert_eq!(expected_int_data.len(), arr.column(1).data().len()); - assert_eq!( - expected_int_data.null_count(), - arr.column(1).data().null_count() - ); - assert_eq!( - expected_int_data.null_bitmap(), - arr.column(1).data().null_bitmap() - ); - let expected_value_buf = expected_int_data.buffers()[0].clone(); - let actual_value_buf = arr.column(1).data().buffers()[0].clone(); - for i in 0..expected_int_data.len() { - if !expected_int_data.is_null(i) { - assert_eq!( - expected_value_buf.data()[i * 4..(i + 1) * 4], - actual_value_buf.data()[i * 4..(i + 1) * 4] - ); - } - } - } - - #[test] - fn test_struct_array_from_vec_error() { - let strings: ArrayRef = Arc::new(StringArray::from(vec![ - Some("joe"), - None, - None, - // 3 elements, not 4 - ])); - let ints: ArrayRef = - Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])); - - let arr = - StructArray::try_from(vec![("f1", strings.clone()), ("f2", ints.clone())]); - - match arr { - Err(ArrowError::InvalidArgumentError(e)) => { - assert!(e.starts_with("Array of field \"f2\" has length 4, but previous elements have length 3.")); - } - _ => assert!(false, "This test got an unexpected error type"), - }; - } - - #[test] - #[should_panic( - expected = "the field data types must match the array data in a StructArray" - )] - fn test_struct_array_from_mismatched_types() { - StructArray::from(vec![ - ( - Field::new("b", DataType::Int16, false), - Arc::new(BooleanArray::from(vec![false, false, true, true])) - as Arc, - ), - ( - Field::new("c", DataType::Utf8, false), - Arc::new(Int32Array::from(vec![42, 28, 19, 31])), - ), - ]); - } - - #[test] - fn test_struct_array_slice() { - let boolean_data = ArrayData::builder(DataType::Boolean) - .len(5) - .add_buffer(Buffer::from([0b00010000])) - .null_bit_buffer(Buffer::from([0b00010001])) - .build(); - let int_data = ArrayData::builder(DataType::Int32) - .len(5) - .add_buffer(Buffer::from([0, 28, 42, 0, 0].to_byte_slice())) - .null_bit_buffer(Buffer::from([0b00000110])) - .build(); - - let mut field_types = vec![]; - field_types.push(Field::new("a", DataType::Boolean, false)); - field_types.push(Field::new("b", DataType::Int32, false)); - let struct_array_data = ArrayData::builder(DataType::Struct(field_types)) - .len(5) - .add_child_data(boolean_data.clone()) - .add_child_data(int_data.clone()) - .null_bit_buffer(Buffer::from([0b00010111])) - .build(); - let struct_array = StructArray::from(struct_array_data); - - assert_eq!(5, struct_array.len()); - assert_eq!(1, struct_array.null_count()); - assert!(struct_array.is_valid(0)); - assert!(struct_array.is_valid(1)); - assert!(struct_array.is_valid(2)); - assert!(struct_array.is_null(3)); - assert!(struct_array.is_valid(4)); - assert_eq!(boolean_data, struct_array.column(0).data()); - assert_eq!(int_data, struct_array.column(1).data()); - - let c0 = struct_array.column(0); - let c0 = c0.as_any().downcast_ref::().unwrap(); - assert_eq!(5, c0.len()); - assert_eq!(3, c0.null_count()); - assert!(c0.is_valid(0)); - assert_eq!(false, c0.value(0)); - assert!(c0.is_null(1)); - assert!(c0.is_null(2)); - assert!(c0.is_null(3)); - assert!(c0.is_valid(4)); - assert_eq!(true, c0.value(4)); - - let c1 = struct_array.column(1); - let c1 = c1.as_any().downcast_ref::().unwrap(); - assert_eq!(5, c1.len()); - assert_eq!(3, c1.null_count()); - assert!(c1.is_null(0)); - assert!(c1.is_valid(1)); - assert_eq!(28, c1.value(1)); - assert!(c1.is_valid(2)); - assert_eq!(42, c1.value(2)); - assert!(c1.is_null(3)); - assert!(c1.is_null(4)); - - let sliced_array = struct_array.slice(2, 3); - let sliced_array = sliced_array.as_any().downcast_ref::().unwrap(); - assert_eq!(3, sliced_array.len()); - assert_eq!(2, sliced_array.offset()); - assert_eq!(1, sliced_array.null_count()); - assert!(sliced_array.is_valid(0)); - assert!(sliced_array.is_null(1)); - assert!(sliced_array.is_valid(2)); - - let sliced_c0 = sliced_array.column(0); - let sliced_c0 = sliced_c0.as_any().downcast_ref::().unwrap(); - assert_eq!(3, sliced_c0.len()); - assert_eq!(2, sliced_c0.offset()); - assert!(sliced_c0.is_null(0)); - assert!(sliced_c0.is_null(1)); - assert!(sliced_c0.is_valid(2)); - assert_eq!(true, sliced_c0.value(2)); - - let sliced_c1 = sliced_array.column(1); - let sliced_c1 = sliced_c1.as_any().downcast_ref::().unwrap(); - assert_eq!(3, sliced_c1.len()); - assert_eq!(2, sliced_c1.offset()); - assert!(sliced_c1.is_valid(0)); - assert_eq!(42, sliced_c1.value(0)); - assert!(sliced_c1.is_null(1)); - assert!(sliced_c1.is_null(2)); - } - - #[test] - #[should_panic( - expected = "all child arrays of a StructArray must have the same length" - )] - fn test_invalid_struct_child_array_lengths() { - StructArray::from(vec![ - ( - Field::new("b", DataType::Float32, false), - Arc::new(Float32Array::from(vec![1.1])) as Arc, - ), - ( - Field::new("c", DataType::Float64, false), - Arc::new(Float64Array::from(vec![2.2, 3.3])), - ), - ]); - } - - #[test] - #[should_panic(expected = "memory is not aligned")] - fn test_primitive_array_alignment() { - let ptr = memory::allocate_aligned(8); - let buf = unsafe { Buffer::from_raw_parts(ptr, 8, 8) }; - let buf2 = buf.slice(1); - let array_data = ArrayData::builder(DataType::Int32).add_buffer(buf2).build(); - Int32Array::from(array_data); - } - - #[test] - #[should_panic(expected = "memory is not aligned")] - fn test_list_array_alignment() { - let ptr = memory::allocate_aligned(8); - let buf = unsafe { Buffer::from_raw_parts(ptr, 8, 8) }; - let buf2 = buf.slice(1); - - let values: [i32; 8] = [0; 8]; - let value_data = ArrayData::builder(DataType::Int32) - .add_buffer(Buffer::from(values.to_byte_slice())) - .build(); - - let list_data_type = - DataType::List(Box::new(Field::new("item", DataType::Int32, false))); - let list_data = ArrayData::builder(list_data_type) - .add_buffer(buf2) - .add_child_data(value_data) - .build(); - ListArray::from(list_data); - } - - #[test] - #[should_panic(expected = "memory is not aligned")] - fn test_binary_array_alignment() { - let ptr = memory::allocate_aligned(8); - let buf = unsafe { Buffer::from_raw_parts(ptr, 8, 8) }; - let buf2 = buf.slice(1); - - let values: [u8; 12] = [0; 12]; - - let array_data = ArrayData::builder(DataType::Binary) - .add_buffer(buf2) - .add_buffer(Buffer::from(&values[..])) - .build(); - BinaryArray::from(array_data); - } - - #[test] - fn test_access_array_concurrently() { - let a = Int32Array::from(vec![5, 6, 7, 8, 9]); - let ret = thread::spawn(move || a.value(3)).join(); - - assert!(ret.is_ok()); - assert_eq!(8, ret.ok().unwrap()); - } - - #[test] - fn test_dictionary_array_fmt_debug() { - let key_builder = PrimitiveBuilder::::new(3); - let value_builder = PrimitiveBuilder::::new(2); - let mut builder = PrimitiveDictionaryBuilder::new(key_builder, value_builder); - builder.append(12345678).unwrap(); - builder.append_null().unwrap(); - builder.append(22345678).unwrap(); - let array = builder.finish(); - assert_eq!( - "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n null,\n 1,\n] values: PrimitiveArray\n[\n 12345678,\n 22345678,\n]}\n", - format!("{:?}", array) - ); - - let key_builder = PrimitiveBuilder::::new(20); - let value_builder = PrimitiveBuilder::::new(2); - let mut builder = PrimitiveDictionaryBuilder::new(key_builder, value_builder); - for _ in 0..20 { - builder.append(1).unwrap(); - } - let array = builder.finish(); - assert_eq!( - "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n] values: PrimitiveArray\n[\n 1,\n]}\n", - format!("{:?}", array) - ); - } - - #[test] - fn test_dictionary_array_from_iter() { - let test = vec!["a", "a", "b", "c"]; - let array: DictionaryArray = test - .iter() - .map(|&x| if x == "b" { None } else { Some(x) }) - .collect(); - assert_eq!( - "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n null,\n 1,\n] values: StringArray\n[\n \"a\",\n \"c\",\n]}\n", - format!("{:?}", array) - ); - - let array: DictionaryArray = test.into_iter().collect(); - assert_eq!( - "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n 1,\n 2,\n] values: StringArray\n[\n \"a\",\n \"b\",\n \"c\",\n]}\n", - format!("{:?}", array) - ); - } - - #[test] - fn test_dictionary_array_reverse_lookup_key() { - let test = vec!["a", "a", "b", "c"]; - let array: DictionaryArray = test.into_iter().collect(); - - assert_eq!(array.lookup_key("c"), Some(2)); - - // Direction of building a dictionary is the iterator direction - let test = vec!["t3", "t3", "t2", "t2", "t1", "t3", "t4", "t1", "t0"]; - let array: DictionaryArray = test.into_iter().collect(); - - assert_eq!(array.lookup_key("t1"), Some(2)); - assert_eq!(array.lookup_key("non-existent"), None); - } - - #[test] - fn test_dictionary_keys_as_primitive_array() { - let test = vec!["a", "b", "c", "a"]; - let array: DictionaryArray = test.into_iter().collect(); - - let keys = array.keys_array(); - assert_eq!(&DataType::Int8, keys.data_type()); - assert_eq!(0, keys.null_count()); - assert_eq!(&[0, 1, 2, 0], keys.value_slice(0, keys.len())); - } - - #[test] - fn test_dictionary_keys_as_primitive_array_with_null() { - let test = vec![Some("a"), None, Some("b"), None, None, Some("a")]; - let array: DictionaryArray = test.into_iter().collect(); - - let keys = array.keys_array(); - assert_eq!(&DataType::Int32, keys.data_type()); - assert_eq!(3, keys.null_count()); - - assert_eq!(true, keys.is_valid(0)); - assert_eq!(false, keys.is_valid(1)); - assert_eq!(true, keys.is_valid(2)); - assert_eq!(false, keys.is_valid(3)); - assert_eq!(false, keys.is_valid(4)); - assert_eq!(true, keys.is_valid(5)); - - assert_eq!(0, keys.value(0)); - assert_eq!(1, keys.value(2)); - assert_eq!(0, keys.value(5)); - } -} diff --git a/rust/arrow/src/array/array_binary.rs b/rust/arrow/src/array/array_binary.rs new file mode 100644 index 00000000000..d7a3eb7217a --- /dev/null +++ b/rust/arrow/src/array/array_binary.rs @@ -0,0 +1,797 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::convert::From; +use std::fmt; +use std::mem; +use std::{any::Any, iter::FromIterator}; + +use super::{ + array::print_long_array, raw_pointer::as_aligned_pointer, raw_pointer::RawPtrBox, + Array, ArrayData, ArrayDataRef, FixedSizeListArray, GenericBinaryIter, + GenericListArray, LargeListArray, ListArray, OffsetSizeTrait, +}; +use crate::util::bit_util; +use crate::{buffer::Buffer, datatypes::ToByteSlice}; +use crate::{buffer::MutableBuffer, datatypes::DataType}; + +/// Like OffsetSizeTrait, but specialized for Binary +// This allow us to expose a constant datatype for the GenericBinaryArray +pub trait BinaryOffsetSizeTrait: OffsetSizeTrait { + const DATA_TYPE: DataType; +} + +impl BinaryOffsetSizeTrait for i32 { + const DATA_TYPE: DataType = DataType::Binary; +} + +impl BinaryOffsetSizeTrait for i64 { + const DATA_TYPE: DataType = DataType::LargeBinary; +} + +pub struct GenericBinaryArray { + data: ArrayDataRef, + value_offsets: RawPtrBox, + value_data: RawPtrBox, +} + +impl GenericBinaryArray { + /// Returns the offset for the element at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_offset(&self, i: usize) -> OffsetSize { + self.value_offset_at(self.data.offset() + i) + } + + /// Returns the length for the element at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_length(&self, mut i: usize) -> OffsetSize { + i += self.data.offset(); + self.value_offset_at(i + 1) - self.value_offset_at(i) + } + + /// Returns a clone of the value offset buffer + pub fn value_offsets(&self) -> Buffer { + self.data.buffers()[0].clone() + } + + /// Returns a clone of the value data buffer + pub fn value_data(&self) -> Buffer { + self.data.buffers()[1].clone() + } + + #[inline] + fn value_offset_at(&self, i: usize) -> OffsetSize { + unsafe { *self.value_offsets.get().add(i) } + } + + /// Returns the element at index `i` as a byte slice. + pub fn value(&self, i: usize) -> &[u8] { + assert!(i < self.data.len(), "BinaryArray out of bounds access"); + let offset = i.checked_add(self.data.offset()).unwrap(); + unsafe { + let pos = self.value_offset_at(offset); + std::slice::from_raw_parts( + self.value_data.get().offset(pos.to_isize()), + (self.value_offset_at(offset + 1) - pos).to_usize().unwrap(), + ) + } + } + + /// Creates a [GenericBinaryArray] from a vector of byte slices + pub fn from_vec(v: Vec<&[u8]>) -> Self { + let mut offsets = Vec::with_capacity(v.len() + 1); + let mut values = Vec::new(); + let mut length_so_far: OffsetSize = OffsetSize::zero(); + offsets.push(length_so_far); + for s in &v { + length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); + offsets.push(length_so_far); + values.extend_from_slice(s); + } + let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(v.len()) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + GenericBinaryArray::::from(array_data) + } + + /// Creates a [GenericBinaryArray] from a vector of Optional (null) byte slices + pub fn from_opt_vec(v: Vec>) -> Self { + v.into_iter().collect() + } + + fn from_list(v: GenericListArray) -> Self { + assert_eq!( + v.data_ref().child_data()[0].child_data().len(), + 0, + "BinaryArray can only be created from list array of u8 values \ + (i.e. List>)." + ); + assert_eq!( + v.data_ref().child_data()[0].data_type(), + &DataType::UInt8, + "BinaryArray can only be created from List arrays, mismatched data types." + ); + + let mut builder = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(v.len()) + .add_buffer(v.data_ref().buffers()[0].clone()) + .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); + if let Some(bitmap) = v.data_ref().null_bitmap() { + builder = builder + .null_count(v.data_ref().null_count()) + .null_bit_buffer(bitmap.bits.clone()) + } + + let data = builder.build(); + Self::from(data) + } +} + +impl<'a, T: BinaryOffsetSizeTrait> GenericBinaryArray { + /// constructs a new iterator + pub fn iter(&'a self) -> GenericBinaryIter<'a, T> { + GenericBinaryIter::<'a, T>::new(&self) + } +} + +impl fmt::Debug for GenericBinaryArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}BinaryArray\n[\n", OffsetSize::prefix())?; + print_long_array(self, f, |array, index, f| { + fmt::Debug::fmt(&array.value(index), f) + })?; + write!(f, "]") + } +} + +impl Array for GenericBinaryArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [$name]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [$name]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +impl From + for GenericBinaryArray +{ + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.data_type(), + &::DATA_TYPE, + "[Large]BinaryArray expects Datatype::[Large]Binary" + ); + assert_eq!( + data.buffers().len(), + 2, + "BinaryArray data should contain 2 buffers only (offsets and values)" + ); + let raw_value_offsets = data.buffers()[0].raw_data(); + let value_data = data.buffers()[1].raw_data(); + Self { + data, + value_offsets: RawPtrBox::new(as_aligned_pointer::( + raw_value_offsets, + )), + value_data: RawPtrBox::new(value_data), + } + } +} + +impl FromIterator> + for GenericBinaryArray +where + Ptr: AsRef<[u8]>, +{ + fn from_iter>>(iter: I) -> Self { + let iter = iter.into_iter(); + let (_, data_len) = iter.size_hint(); + let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. + + let mut offsets = Vec::with_capacity(data_len + 1); + let mut values = Vec::new(); + let mut null_buf = MutableBuffer::new_null(data_len); + let mut length_so_far: OffsetSize = OffsetSize::zero(); + offsets.push(length_so_far); + + { + let null_slice = null_buf.data_mut(); + + for (i, s) in iter.enumerate() { + if let Some(s) = s { + let s = s.as_ref(); + bit_util::set_bit(null_slice, i); + length_so_far = + length_so_far + OffsetSize::from_usize(s.len()).unwrap(); + values.extend_from_slice(s); + } + // always add an element in offsets + offsets.push(length_so_far); + } + } + + let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(data_len) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .null_bit_buffer(null_buf.freeze()) + .build(); + Self::from(array_data) + } +} + +/// An array where each element is a byte whose maximum length is represented by a i32. +pub type BinaryArray = GenericBinaryArray; + +/// An array where each element is a byte whose maximum length is represented by a i64. +pub type LargeBinaryArray = GenericBinaryArray; + +impl<'a, T: BinaryOffsetSizeTrait> IntoIterator for &'a GenericBinaryArray { + type Item = Option<&'a [u8]>; + type IntoIter = GenericBinaryIter<'a, T>; + + fn into_iter(self) -> Self::IntoIter { + GenericBinaryIter::<'a, T>::new(self) + } +} + +impl From> for BinaryArray { + fn from(v: Vec<&[u8]>) -> Self { + BinaryArray::from_vec(v) + } +} + +impl From>> for BinaryArray { + fn from(v: Vec>) -> Self { + BinaryArray::from_opt_vec(v) + } +} + +impl From> for LargeBinaryArray { + fn from(v: Vec<&[u8]>) -> Self { + LargeBinaryArray::from_vec(v) + } +} + +impl From>> for LargeBinaryArray { + fn from(v: Vec>) -> Self { + LargeBinaryArray::from_opt_vec(v) + } +} + +impl From for BinaryArray { + fn from(v: ListArray) -> Self { + BinaryArray::from_list(v) + } +} + +impl From for LargeBinaryArray { + fn from(v: LargeListArray) -> Self { + LargeBinaryArray::from_list(v) + } +} + +/// A type of `FixedSizeListArray` whose elements are binaries. +pub struct FixedSizeBinaryArray { + data: ArrayDataRef, + value_data: RawPtrBox, + length: i32, +} + +impl FixedSizeBinaryArray { + /// Returns the element at index `i` as a byte slice. + pub fn value(&self, i: usize) -> &[u8] { + assert!( + i < self.data.len(), + "FixedSizeBinaryArray out of bounds access" + ); + let offset = i.checked_add(self.data.offset()).unwrap(); + unsafe { + let pos = self.value_offset_at(offset); + std::slice::from_raw_parts( + self.value_data.get().offset(pos as isize), + (self.value_offset_at(offset + 1) - pos) as usize, + ) + } + } + + /// Returns the offset for the element at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_offset(&self, i: usize) -> i32 { + self.value_offset_at(self.data.offset() + i) + } + + /// Returns the length for an element. + /// + /// All elements have the same length as the array is a fixed size. + #[inline] + pub fn value_length(&self) -> i32 { + self.length + } + + /// Returns a clone of the value data buffer + pub fn value_data(&self) -> Buffer { + self.data.buffers()[0].clone() + } + + #[inline] + fn value_offset_at(&self, i: usize) -> i32 { + self.length * i as i32 + } +} + +impl From for FixedSizeBinaryArray { + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.buffers().len(), + 1, + "FixedSizeBinaryArray data should contain 1 buffer only (values)" + ); + let value_data = data.buffers()[0].raw_data(); + let length = match data.data_type() { + DataType::FixedSizeBinary(len) => *len, + _ => panic!("Expected data type to be FixedSizeBinary"), + }; + Self { + data, + value_data: RawPtrBox::new(value_data), + length, + } + } +} + +/// Creates a `FixedSizeBinaryArray` from `FixedSizeList` array +impl From for FixedSizeBinaryArray { + fn from(v: FixedSizeListArray) -> Self { + assert_eq!( + v.data_ref().child_data()[0].child_data().len(), + 0, + "FixedSizeBinaryArray can only be created from list array of u8 values \ + (i.e. FixedSizeList>)." + ); + assert_eq!( + v.data_ref().child_data()[0].data_type(), + &DataType::UInt8, + "FixedSizeBinaryArray can only be created from FixedSizeList arrays, mismatched data types." + ); + + let mut builder = ArrayData::builder(DataType::FixedSizeBinary(v.value_length())) + .len(v.len()) + .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); + if let Some(bitmap) = v.data_ref().null_bitmap() { + builder = builder + .null_count(v.data_ref().null_count()) + .null_bit_buffer(bitmap.bits.clone()) + } + + let data = builder.build(); + Self::from(data) + } +} + +impl fmt::Debug for FixedSizeBinaryArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "FixedSizeBinaryArray<{}>\n[\n", self.value_length())?; + print_long_array(self, f, |array, index, f| { + fmt::Debug::fmt(&array.value(index), f) + })?; + write!(f, "]") + } +} + +impl Array for FixedSizeBinaryArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [FixedSizeBinaryArray]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [FixedSizeBinaryArray]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +#[cfg(test)] +mod tests { + use crate::datatypes::Field; + + use super::*; + + #[test] + fn test_binary_array() { + let values: [u8; 12] = [ + b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', + ]; + let offsets: [i32; 4] = [0, 5, 5, 12]; + + // Array data: ["hello", "", "parquet"] + let array_data = ArrayData::builder(DataType::Binary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array = BinaryArray::from(array_data); + assert_eq!(3, binary_array.len()); + assert_eq!(0, binary_array.null_count()); + assert_eq!([b'h', b'e', b'l', b'l', b'o'], binary_array.value(0)); + assert_eq!([] as [u8; 0], binary_array.value(1)); + assert_eq!( + [b'p', b'a', b'r', b'q', b'u', b'e', b't'], + binary_array.value(2) + ); + assert_eq!(5, binary_array.value_offset(2)); + assert_eq!(7, binary_array.value_length(2)); + for i in 0..3 { + assert!(binary_array.is_valid(i)); + assert!(!binary_array.is_null(i)); + } + + // Test binary array with offset + let array_data = ArrayData::builder(DataType::Binary) + .len(4) + .offset(1) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array = BinaryArray::from(array_data); + assert_eq!( + [b'p', b'a', b'r', b'q', b'u', b'e', b't'], + binary_array.value(1) + ); + assert_eq!(5, binary_array.value_offset(0)); + assert_eq!(0, binary_array.value_length(0)); + assert_eq!(5, binary_array.value_offset(1)); + assert_eq!(7, binary_array.value_length(1)); + } + + #[test] + fn test_large_binary_array() { + let values: [u8; 12] = [ + b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', + ]; + let offsets: [i64; 4] = [0, 5, 5, 12]; + + // Array data: ["hello", "", "parquet"] + let array_data = ArrayData::builder(DataType::LargeBinary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array = LargeBinaryArray::from(array_data); + assert_eq!(3, binary_array.len()); + assert_eq!(0, binary_array.null_count()); + assert_eq!([b'h', b'e', b'l', b'l', b'o'], binary_array.value(0)); + assert_eq!([] as [u8; 0], binary_array.value(1)); + assert_eq!( + [b'p', b'a', b'r', b'q', b'u', b'e', b't'], + binary_array.value(2) + ); + assert_eq!(5, binary_array.value_offset(2)); + assert_eq!(7, binary_array.value_length(2)); + for i in 0..3 { + assert!(binary_array.is_valid(i)); + assert!(!binary_array.is_null(i)); + } + + // Test binary array with offset + let array_data = ArrayData::builder(DataType::LargeBinary) + .len(4) + .offset(1) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array = LargeBinaryArray::from(array_data); + assert_eq!( + [b'p', b'a', b'r', b'q', b'u', b'e', b't'], + binary_array.value(1) + ); + assert_eq!(5, binary_array.value_offset(0)); + assert_eq!(0, binary_array.value_length(0)); + assert_eq!(5, binary_array.value_offset(1)); + assert_eq!(7, binary_array.value_length(1)); + } + + #[test] + fn test_binary_array_from_list_array() { + let values: [u8; 12] = [ + b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', + ]; + let values_data = ArrayData::builder(DataType::UInt8) + .len(12) + .add_buffer(Buffer::from(&values[..])) + .build(); + let offsets: [i32; 4] = [0, 5, 5, 12]; + + // Array data: ["hello", "", "parquet"] + let array_data1 = ArrayData::builder(DataType::Binary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array1 = BinaryArray::from(array_data1); + + let array_data2 = ArrayData::builder(DataType::Binary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_child_data(values_data) + .build(); + let list_array = ListArray::from(array_data2); + let binary_array2 = BinaryArray::from(list_array); + + assert_eq!(2, binary_array2.data().buffers().len()); + assert_eq!(0, binary_array2.data().child_data().len()); + + assert_eq!(binary_array1.len(), binary_array2.len()); + assert_eq!(binary_array1.null_count(), binary_array2.null_count()); + for i in 0..binary_array1.len() { + assert_eq!(binary_array1.value(i), binary_array2.value(i)); + assert_eq!(binary_array1.value_offset(i), binary_array2.value_offset(i)); + assert_eq!(binary_array1.value_length(i), binary_array2.value_length(i)); + } + } + + #[test] + fn test_large_binary_array_from_list_array() { + let values: [u8; 12] = [ + b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', + ]; + let values_data = ArrayData::builder(DataType::UInt8) + .len(12) + .add_buffer(Buffer::from(&values[..])) + .build(); + let offsets: [i64; 4] = [0, 5, 5, 12]; + + // Array data: ["hello", "", "parquet"] + let array_data1 = ArrayData::builder(DataType::LargeBinary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array1 = LargeBinaryArray::from(array_data1); + + let array_data2 = ArrayData::builder(DataType::Binary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_child_data(values_data) + .build(); + let list_array = LargeListArray::from(array_data2); + let binary_array2 = LargeBinaryArray::from(list_array); + + assert_eq!(2, binary_array2.data().buffers().len()); + assert_eq!(0, binary_array2.data().child_data().len()); + + assert_eq!(binary_array1.len(), binary_array2.len()); + assert_eq!(binary_array1.null_count(), binary_array2.null_count()); + for i in 0..binary_array1.len() { + assert_eq!(binary_array1.value(i), binary_array2.value(i)); + assert_eq!(binary_array1.value_offset(i), binary_array2.value_offset(i)); + assert_eq!(binary_array1.value_length(i), binary_array2.value_length(i)); + } + } + + fn test_generic_binary_array_from_opt_vec() { + let values: Vec> = + vec![Some(b"one"), Some(b"two"), None, Some(b""), Some(b"three")]; + let array = GenericBinaryArray::::from_opt_vec(values); + assert_eq!(array.len(), 5); + assert_eq!(array.value(0), b"one"); + assert_eq!(array.value(1), b"two"); + assert_eq!(array.value(3), b""); + assert_eq!(array.value(4), b"three"); + assert_eq!(array.is_null(0), false); + assert_eq!(array.is_null(1), false); + assert_eq!(array.is_null(2), true); + assert_eq!(array.is_null(3), false); + assert_eq!(array.is_null(4), false); + } + + #[test] + fn test_large_binary_array_from_opt_vec() { + test_generic_binary_array_from_opt_vec::() + } + + #[test] + fn test_binary_array_from_opt_vec() { + test_generic_binary_array_from_opt_vec::() + } + + #[test] + #[should_panic( + expected = "BinaryArray can only be created from List arrays, mismatched \ + data types." + )] + fn test_binary_array_from_incorrect_list_array_type() { + let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; + let values_data = ArrayData::builder(DataType::UInt32) + .len(12) + .add_buffer(Buffer::from(values[..].to_byte_slice())) + .build(); + let offsets: [i32; 4] = [0, 5, 5, 12]; + + let array_data = ArrayData::builder(DataType::Utf8) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_child_data(values_data) + .build(); + let list_array = ListArray::from(array_data); + BinaryArray::from(list_array); + } + + #[test] + #[should_panic( + expected = "BinaryArray can only be created from list array of u8 values \ + (i.e. List>)." + )] + fn test_binary_array_from_incorrect_list_array() { + let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; + let values_data = ArrayData::builder(DataType::UInt32) + .len(12) + .add_buffer(Buffer::from(values[..].to_byte_slice())) + .add_child_data(ArrayData::builder(DataType::Boolean).build()) + .build(); + let offsets: [i32; 4] = [0, 5, 5, 12]; + + let array_data = ArrayData::builder(DataType::Utf8) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_child_data(values_data) + .build(); + let list_array = ListArray::from(array_data); + BinaryArray::from(list_array); + } + + #[test] + fn test_fixed_size_binary_array() { + let values: [u8; 15] = *b"hellotherearrow"; + + let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) + .len(3) + .add_buffer(Buffer::from(&values[..])) + .build(); + let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data); + assert_eq!(3, fixed_size_binary_array.len()); + assert_eq!(0, fixed_size_binary_array.null_count()); + assert_eq!( + [b'h', b'e', b'l', b'l', b'o'], + fixed_size_binary_array.value(0) + ); + assert_eq!( + [b't', b'h', b'e', b'r', b'e'], + fixed_size_binary_array.value(1) + ); + assert_eq!( + [b'a', b'r', b'r', b'o', b'w'], + fixed_size_binary_array.value(2) + ); + assert_eq!(5, fixed_size_binary_array.value_length()); + assert_eq!(10, fixed_size_binary_array.value_offset(2)); + for i in 0..3 { + assert!(fixed_size_binary_array.is_valid(i)); + assert!(!fixed_size_binary_array.is_null(i)); + } + + // Test binary array with offset + let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) + .len(2) + .offset(1) + .add_buffer(Buffer::from(&values[..])) + .build(); + let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data); + assert_eq!( + [b't', b'h', b'e', b'r', b'e'], + fixed_size_binary_array.value(0) + ); + assert_eq!( + [b'a', b'r', b'r', b'o', b'w'], + fixed_size_binary_array.value(1) + ); + assert_eq!(2, fixed_size_binary_array.len()); + assert_eq!(5, fixed_size_binary_array.value_offset(0)); + assert_eq!(5, fixed_size_binary_array.value_length()); + assert_eq!(10, fixed_size_binary_array.value_offset(1)); + } + + #[test] + #[should_panic( + expected = "FixedSizeBinaryArray can only be created from list array of u8 values \ + (i.e. FixedSizeList>)." + )] + fn test_fixed_size_binary_array_from_incorrect_list_array() { + let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]; + let values_data = ArrayData::builder(DataType::UInt32) + .len(12) + .add_buffer(Buffer::from(values[..].to_byte_slice())) + .add_child_data(ArrayData::builder(DataType::Boolean).build()) + .build(); + + let array_data = ArrayData::builder(DataType::FixedSizeList( + Box::new(Field::new("item", DataType::Binary, false)), + 4, + )) + .len(3) + .add_child_data(values_data) + .build(); + let list_array = FixedSizeListArray::from(array_data); + FixedSizeBinaryArray::from(list_array); + } + + #[test] + #[should_panic(expected = "BinaryArray out of bounds access")] + fn test_binary_array_get_value_index_out_of_bound() { + let values: [u8; 12] = + [104, 101, 108, 108, 111, 112, 97, 114, 113, 117, 101, 116]; + let offsets: [i32; 4] = [0, 5, 5, 12]; + let array_data = ArrayData::builder(DataType::Binary) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let binary_array = BinaryArray::from(array_data); + binary_array.value(4); + } + + #[test] + fn test_binary_array_fmt_debug() { + let values: [u8; 15] = *b"hellotherearrow"; + + let array_data = ArrayData::builder(DataType::FixedSizeBinary(5)) + .len(3) + .add_buffer(Buffer::from(&values[..])) + .build(); + let arr = FixedSizeBinaryArray::from(array_data); + assert_eq!( + "FixedSizeBinaryArray<5>\n[\n [104, 101, 108, 108, 111],\n [116, 104, 101, 114, 101],\n [97, 114, 114, 111, 119],\n]", + format!("{:?}", arr) + ); + } +} diff --git a/rust/arrow/src/array/array_dictionary.rs b/rust/arrow/src/array/array_dictionary.rs new file mode 100644 index 00000000000..3cca55440f4 --- /dev/null +++ b/rust/arrow/src/array/array_dictionary.rs @@ -0,0 +1,412 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt; +use std::iter::IntoIterator; +use std::mem; +use std::{any::Any, sync::Arc}; +use std::{convert::From, iter::FromIterator}; + +use super::{ + make_array, Array, ArrayData, ArrayDataRef, ArrayRef, PrimitiveArray, + PrimitiveBuilder, StringArray, StringBuilder, StringDictionaryBuilder, +}; +use crate::datatypes::ArrowNativeType; +use crate::datatypes::{ArrowDictionaryKeyType, ArrowPrimitiveType, DataType}; + +/// A dictionary array where each element is a single value indexed by an integer key. +/// This is mostly used to represent strings or a limited set of primitive types as integers, +/// for example when doing NLP analysis or representing chromosomes by name. +/// +/// Example **with nullable** data: +/// +/// ``` +/// use arrow::array::{DictionaryArray, Int8Array}; +/// use arrow::datatypes::Int8Type; +/// let test = vec!["a", "a", "b", "c"]; +/// let array : DictionaryArray = test.iter().map(|&x| if x == "b" {None} else {Some(x)}).collect(); +/// assert_eq!(array.keys(), &Int8Array::from(vec![Some(0), Some(0), None, Some(1)])); +/// ``` +/// +/// Example **without nullable** data: +/// +/// ``` +/// use arrow::array::{DictionaryArray, Int8Array}; +/// use arrow::datatypes::Int8Type; +/// let test = vec!["a", "a", "b", "c"]; +/// let array : DictionaryArray = test.into_iter().collect(); +/// assert_eq!(array.keys(), &Int8Array::from(vec![0, 0, 1, 2])); +/// ``` +pub struct DictionaryArray { + /// Data of this dictionary. Note that this is _not_ compatible with the C Data interface, + /// as, in the current implementation, `values` below are the first child of this struct. + data: ArrayDataRef, + + /// The keys of this dictionary. These are constructed from the buffer and null bitmap + /// of `data`. + /// Also, note that these do not correspond to the true values of this array. Rather, they map + /// to the real values. + keys: PrimitiveArray, + + /// Array of dictionary values (can by any DataType). + values: ArrayRef, + + /// Values are ordered. + is_ordered: bool, +} + +impl<'a, K: ArrowPrimitiveType> DictionaryArray { + /// Return an iterator to the keys of this dictionary. + pub fn keys(&self) -> &PrimitiveArray { + &self.keys + } + + /// Returns an array view of the keys of this dictionary + pub fn keys_array(&self) -> PrimitiveArray { + let data = self.data_ref(); + let keys_data = ArrayData::new( + K::DATA_TYPE, + data.len(), + Some(data.null_count()), + data.null_buffer().cloned(), + data.offset(), + data.buffers().to_vec(), + vec![], + ); + PrimitiveArray::::from(Arc::new(keys_data)) + } + + /// Returns the lookup key by doing reverse dictionary lookup + pub fn lookup_key(&self, value: &str) -> Option { + let rd_buf: &StringArray = + self.values.as_any().downcast_ref::().unwrap(); + + (0..rd_buf.len()) + .position(|i| rd_buf.value(i) == value) + .map(K::Native::from_usize) + .flatten() + } + + /// Returns an `ArrayRef` to the dictionary values. + pub fn values(&self) -> ArrayRef { + self.values.clone() + } + + /// Returns a clone of the value type of this list. + pub fn value_type(&self) -> DataType { + self.values.data_ref().data_type().clone() + } + + /// The length of the dictionary is the length of the keys array. + pub fn len(&self) -> usize { + self.keys.len() + } + + /// Whether this dictionary is empty + pub fn is_empty(&self) -> bool { + self.keys.is_empty() + } + + // Currently exists for compatibility purposes with Arrow IPC. + pub fn is_ordered(&self) -> bool { + self.is_ordered + } +} + +/// Constructs a `DictionaryArray` from an array data reference. +impl From for DictionaryArray { + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.buffers().len(), + 1, + "DictionaryArray data should contain a single buffer only (keys)." + ); + assert_eq!( + data.child_data().len(), + 1, + "DictionaryArray should contain a single child array (values)." + ); + + if let DataType::Dictionary(key_data_type, _) = data.data_type() { + if key_data_type.as_ref() != &T::DATA_TYPE { + panic!("DictionaryArray's data type must match.") + }; + // create a zero-copy of the keys' data + let keys = PrimitiveArray::::from(Arc::new(ArrayData::new( + T::DATA_TYPE, + data.len(), + Some(data.null_count()), + data.null_buffer().cloned(), + data.offset(), + data.buffers().to_vec(), + vec![], + ))); + let values = make_array(data.child_data()[0].clone()); + Self { + data, + keys, + values, + is_ordered: false, + } + } else { + panic!("DictionaryArray must have Dictionary data type.") + } + } +} + +/// Constructs a `DictionaryArray` from an iterator of optional strings. +impl<'a, T: ArrowPrimitiveType + ArrowDictionaryKeyType> FromIterator> + for DictionaryArray +{ + fn from_iter>>(iter: I) -> Self { + let it = iter.into_iter(); + let (lower, _) = it.size_hint(); + let key_builder = PrimitiveBuilder::::new(lower); + let value_builder = StringBuilder::new(256); + let mut builder = StringDictionaryBuilder::new(key_builder, value_builder); + it.for_each(|i| { + if let Some(i) = i { + // Note: impl ... for Result> fails with + // error[E0117]: only traits defined in the current crate can be implemented for arbitrary types + builder + .append(i) + .expect("Unable to append a value to a dictionary array."); + } else { + builder + .append_null() + .expect("Unable to append a null value to a dictionary array."); + } + }); + + builder.finish() + } +} + +/// Constructs a `DictionaryArray` from an iterator of strings. +impl<'a, T: ArrowPrimitiveType + ArrowDictionaryKeyType> FromIterator<&'a str> + for DictionaryArray +{ + fn from_iter>(iter: I) -> Self { + let it = iter.into_iter(); + let (lower, _) = it.size_hint(); + let key_builder = PrimitiveBuilder::::new(lower); + let value_builder = StringBuilder::new(256); + let mut builder = StringDictionaryBuilder::new(key_builder, value_builder); + it.for_each(|i| { + builder + .append(i) + .expect("Unable to append a value to a dictionary array."); + }); + + builder.finish() + } +} + +impl Array for DictionaryArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + fn get_buffer_memory_size(&self) -> usize { + // Since both `keys` and `values` derive (are references from) `data`, we only need to account for `data`. + self.data.get_buffer_memory_size() + } + + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + + self.keys.get_array_memory_size() + + self.values.get_array_memory_size() + + mem::size_of_val(self) + } +} + +impl fmt::Debug for DictionaryArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + writeln!( + f, + "DictionaryArray {{keys: {:?} values: {:?}}}", + self.keys, self.values + ) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use crate::{ + array::Int16Array, + datatypes::{Int32Type, Int8Type, UInt32Type, UInt8Type}, + }; + use crate::{ + array::Int16DictionaryArray, array::PrimitiveDictionaryBuilder, + datatypes::DataType, + }; + use crate::{buffer::Buffer, datatypes::ToByteSlice}; + + #[test] + fn test_dictionary_array() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int8) + .len(8) + .add_buffer(Buffer::from( + &[10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(), + )) + .build(); + + // Construct a buffer for value offsets, for the nested array: + let keys = Buffer::from(&[2_i16, 3, 4].to_byte_slice()); + + // Construct a dictionary array from the above two + let key_type = DataType::Int16; + let value_type = DataType::Int8; + let dict_data_type = + DataType::Dictionary(Box::new(key_type), Box::new(value_type)); + let dict_data = ArrayData::builder(dict_data_type.clone()) + .len(3) + .add_buffer(keys.clone()) + .add_child_data(value_data.clone()) + .build(); + let dict_array = Int16DictionaryArray::from(dict_data); + + let values = dict_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int8, dict_array.value_type()); + assert_eq!(3, dict_array.len()); + + // Null count only makes sense in terms of the component arrays. + assert_eq!(0, dict_array.null_count()); + assert_eq!(0, dict_array.values().null_count()); + assert_eq!(dict_array.keys(), &Int16Array::from(vec![2_i16, 3, 4])); + + // Now test with a non-zero offset + let dict_data = ArrayData::builder(dict_data_type) + .len(2) + .offset(1) + .add_buffer(keys) + .add_child_data(value_data.clone()) + .build(); + let dict_array = Int16DictionaryArray::from(dict_data); + + let values = dict_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int8, dict_array.value_type()); + assert_eq!(2, dict_array.len()); + assert_eq!(dict_array.keys(), &Int16Array::from(vec![3_i16, 4])); + } + + #[test] + fn test_dictionary_array_fmt_debug() { + let key_builder = PrimitiveBuilder::::new(3); + let value_builder = PrimitiveBuilder::::new(2); + let mut builder = PrimitiveDictionaryBuilder::new(key_builder, value_builder); + builder.append(12345678).unwrap(); + builder.append_null().unwrap(); + builder.append(22345678).unwrap(); + let array = builder.finish(); + assert_eq!( + "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n null,\n 1,\n] values: PrimitiveArray\n[\n 12345678,\n 22345678,\n]}\n", + format!("{:?}", array) + ); + + let key_builder = PrimitiveBuilder::::new(20); + let value_builder = PrimitiveBuilder::::new(2); + let mut builder = PrimitiveDictionaryBuilder::new(key_builder, value_builder); + for _ in 0..20 { + builder.append(1).unwrap(); + } + let array = builder.finish(); + assert_eq!( + "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n 0,\n] values: PrimitiveArray\n[\n 1,\n]}\n", + format!("{:?}", array) + ); + } + + #[test] + fn test_dictionary_array_from_iter() { + let test = vec!["a", "a", "b", "c"]; + let array: DictionaryArray = test + .iter() + .map(|&x| if x == "b" { None } else { Some(x) }) + .collect(); + assert_eq!( + "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n null,\n 1,\n] values: StringArray\n[\n \"a\",\n \"c\",\n]}\n", + format!("{:?}", array) + ); + + let array: DictionaryArray = test.into_iter().collect(); + assert_eq!( + "DictionaryArray {keys: PrimitiveArray\n[\n 0,\n 0,\n 1,\n 2,\n] values: StringArray\n[\n \"a\",\n \"b\",\n \"c\",\n]}\n", + format!("{:?}", array) + ); + } + + #[test] + fn test_dictionary_array_reverse_lookup_key() { + let test = vec!["a", "a", "b", "c"]; + let array: DictionaryArray = test.into_iter().collect(); + + assert_eq!(array.lookup_key("c"), Some(2)); + + // Direction of building a dictionary is the iterator direction + let test = vec!["t3", "t3", "t2", "t2", "t1", "t3", "t4", "t1", "t0"]; + let array: DictionaryArray = test.into_iter().collect(); + + assert_eq!(array.lookup_key("t1"), Some(2)); + assert_eq!(array.lookup_key("non-existent"), None); + } + + #[test] + fn test_dictionary_keys_as_primitive_array() { + let test = vec!["a", "b", "c", "a"]; + let array: DictionaryArray = test.into_iter().collect(); + + let keys = array.keys_array(); + assert_eq!(&DataType::Int8, keys.data_type()); + assert_eq!(0, keys.null_count()); + assert_eq!(&[0, 1, 2, 0], keys.value_slice(0, keys.len())); + } + + #[test] + fn test_dictionary_keys_as_primitive_array_with_null() { + let test = vec![Some("a"), None, Some("b"), None, None, Some("a")]; + let array: DictionaryArray = test.into_iter().collect(); + + let keys = array.keys_array(); + assert_eq!(&DataType::Int32, keys.data_type()); + assert_eq!(3, keys.null_count()); + + assert_eq!(true, keys.is_valid(0)); + assert_eq!(false, keys.is_valid(1)); + assert_eq!(true, keys.is_valid(2)); + assert_eq!(false, keys.is_valid(3)); + assert_eq!(false, keys.is_valid(4)); + assert_eq!(true, keys.is_valid(5)); + + assert_eq!(0, keys.value(0)); + assert_eq!(1, keys.value(2)); + assert_eq!(0, keys.value(5)); + } +} diff --git a/rust/arrow/src/array/array_list.rs b/rust/arrow/src/array/array_list.rs new file mode 100644 index 00000000000..4eb8dc56640 --- /dev/null +++ b/rust/arrow/src/array/array_list.rs @@ -0,0 +1,815 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::any::Any; +use std::convert::From; +use std::fmt; +use std::mem; + +use num::Num; + +use super::{ + array::print_long_array, make_array, raw_pointer::as_aligned_pointer, + raw_pointer::RawPtrBox, Array, ArrayDataRef, ArrayRef, +}; +use crate::datatypes::ArrowNativeType; +use crate::datatypes::DataType; + +/// trait declaring an offset size, relevant for i32 vs i64 array types. +pub trait OffsetSizeTrait: ArrowNativeType + Num + Ord { + fn prefix() -> &'static str; + + fn to_isize(&self) -> isize; +} + +impl OffsetSizeTrait for i32 { + fn prefix() -> &'static str { + "" + } + + fn to_isize(&self) -> isize { + num::ToPrimitive::to_isize(self).unwrap() + } +} + +impl OffsetSizeTrait for i64 { + fn prefix() -> &'static str { + "Large" + } + + fn to_isize(&self) -> isize { + num::ToPrimitive::to_isize(self).unwrap() + } +} + +pub struct GenericListArray { + data: ArrayDataRef, + values: ArrayRef, + value_offsets: RawPtrBox, +} + +impl GenericListArray { + /// Returns a reference to the values of this list. + pub fn values(&self) -> ArrayRef { + self.values.clone() + } + + /// Returns a clone of the value type of this list. + pub fn value_type(&self) -> DataType { + self.values.data_ref().data_type().clone() + } + + /// Returns ith value of this list array. + pub fn value(&self, i: usize) -> ArrayRef { + self.values.slice( + self.value_offset(i).to_usize().unwrap(), + self.value_length(i).to_usize().unwrap(), + ) + } + + /// Returns the offset for value at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_offset(&self, i: usize) -> OffsetSize { + self.value_offset_at(self.data.offset() + i) + } + + /// Returns the length for value at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_length(&self, mut i: usize) -> OffsetSize { + i += self.data.offset(); + self.value_offset_at(i + 1) - self.value_offset_at(i) + } + + #[inline] + fn value_offset_at(&self, i: usize) -> OffsetSize { + unsafe { *self.value_offsets.get().add(i) } + } +} + +impl From for GenericListArray { + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.buffers().len(), + 1, + "ListArray data should contain a single buffer only (value offsets)" + ); + assert_eq!( + data.child_data().len(), + 1, + "ListArray should contain a single child array (values array)" + ); + let values = make_array(data.child_data()[0].clone()); + let raw_value_offsets = data.buffers()[0].raw_data(); + let value_offsets: *const OffsetSize = as_aligned_pointer(raw_value_offsets); + unsafe { + assert!( + (*value_offsets.offset(0)).is_zero(), + "offsets do not start at zero" + ); + } + Self { + data, + values, + value_offsets: RawPtrBox::new(value_offsets), + } + } +} + +impl Array for GenericListArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [ListArray]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [ListArray]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +impl fmt::Debug for GenericListArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}ListArray\n[\n", OffsetSize::prefix())?; + print_long_array(self, f, |array, index, f| { + fmt::Debug::fmt(&array.value(index), f) + })?; + write!(f, "]") + } +} + +/// A list array where each element is a variable-sized sequence of values with the same +/// type whose memory offsets between elements are represented by a i32. +pub type ListArray = GenericListArray; + +/// A list array where each element is a variable-sized sequence of values with the same +/// type whose memory offsets between elements are represented by a i64. +pub type LargeListArray = GenericListArray; + +/// A list array where each element is a fixed-size sequence of values with the same +/// type whose maximum length is represented by a i32. +pub struct FixedSizeListArray { + data: ArrayDataRef, + values: ArrayRef, + length: i32, +} + +impl FixedSizeListArray { + /// Returns a reference to the values of this list. + pub fn values(&self) -> ArrayRef { + self.values.clone() + } + + /// Returns a clone of the value type of this list. + pub fn value_type(&self) -> DataType { + self.values.data_ref().data_type().clone() + } + + /// Returns ith value of this list array. + pub fn value(&self, i: usize) -> ArrayRef { + self.values + .slice(self.value_offset(i) as usize, self.value_length() as usize) + } + + /// Returns the offset for value at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_offset(&self, i: usize) -> i32 { + self.value_offset_at(self.data.offset() + i) + } + + /// Returns the length for value at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub const fn value_length(&self) -> i32 { + self.length + } + + #[inline] + const fn value_offset_at(&self, i: usize) -> i32 { + i as i32 * self.length + } +} + +impl From for FixedSizeListArray { + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.buffers().len(), + 0, + "FixedSizeListArray data should not contain a buffer for value offsets" + ); + assert_eq!( + data.child_data().len(), + 1, + "FixedSizeListArray should contain a single child array (values array)" + ); + let values = make_array(data.child_data()[0].clone()); + let length = match data.data_type() { + DataType::FixedSizeList(_, len) => { + // check that child data is multiple of length + assert_eq!( + values.len() % *len as usize, + 0, + "FixedSizeListArray child array length should be a multiple of {}", + len + ); + *len + } + _ => { + panic!("FixedSizeListArray data should contain a FixedSizeList data type") + } + }; + Self { + data, + values, + length, + } + } +} + +impl Array for FixedSizeListArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [FixedSizeListArray]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + self.values().get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [FixedSizeListArray]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + + self.values().get_array_memory_size() + + mem::size_of_val(self) + } +} + +impl fmt::Debug for FixedSizeListArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "FixedSizeListArray<{}>\n[\n", self.value_length())?; + print_long_array(self, f, |array, index, f| { + fmt::Debug::fmt(&array.value(index), f) + })?; + write!(f, "]") + } +} + +#[cfg(test)] +mod tests { + use crate::{ + array::ArrayData, + array::Int32Array, + buffer::Buffer, + datatypes::{Field, ToByteSlice}, + memory, + util::bit_util, + }; + + use super::*; + + #[test] + fn test_list_array() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) + .build(); + + // Construct a buffer for value offsets, for the nested array: + // [[0, 1, 2], [3, 4, 5], [6, 7]] + let value_offsets = Buffer::from(&[0, 3, 6, 8].to_byte_slice()); + + // Construct a list array from the above two + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type.clone()) + .len(3) + .add_buffer(value_offsets.clone()) + .add_child_data(value_data.clone()) + .build(); + let list_array = ListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!(6, list_array.value_offset(2)); + assert_eq!(2, list_array.value_length(2)); + assert_eq!( + 0, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + for i in 0..3 { + assert!(list_array.is_valid(i)); + assert!(!list_array.is_null(i)); + } + + // Now test with a non-zero offset + let list_data = ArrayData::builder(list_data_type) + .len(3) + .offset(1) + .add_buffer(value_offsets) + .add_child_data(value_data.clone()) + .build(); + let list_array = ListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!(6, list_array.value_offset(1)); + assert_eq!(2, list_array.value_length(1)); + assert_eq!( + 3, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + } + + #[test] + fn test_large_list_array() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) + .build(); + + // Construct a buffer for value offsets, for the nested array: + // [[0, 1, 2], [3, 4, 5], [6, 7]] + let value_offsets = Buffer::from(&[0i64, 3, 6, 8].to_byte_slice()); + + // Construct a list array from the above two + let list_data_type = + DataType::LargeList(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type.clone()) + .len(3) + .add_buffer(value_offsets.clone()) + .add_child_data(value_data.clone()) + .build(); + let list_array = LargeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!(6, list_array.value_offset(2)); + assert_eq!(2, list_array.value_length(2)); + assert_eq!( + 0, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + for i in 0..3 { + assert!(list_array.is_valid(i)); + assert!(!list_array.is_null(i)); + } + + // Now test with a non-zero offset + let list_data = ArrayData::builder(list_data_type) + .len(3) + .offset(1) + .add_buffer(value_offsets) + .add_child_data(value_data.clone()) + .build(); + let list_array = LargeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!(6, list_array.value_offset(1)); + assert_eq!(2, list_array.value_length(1)); + assert_eq!( + 3, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + } + + #[test] + fn test_fixed_size_list_array() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(9) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7, 8].to_byte_slice())) + .build(); + + // Construct a list array from the above two + let list_data_type = DataType::FixedSizeList( + Box::new(Field::new("item", DataType::Int32, false)), + 3, + ); + let list_data = ArrayData::builder(list_data_type.clone()) + .len(3) + .add_child_data(value_data.clone()) + .build(); + let list_array = FixedSizeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!(6, list_array.value_offset(2)); + assert_eq!(3, list_array.value_length()); + assert_eq!( + 0, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + for i in 0..3 { + assert!(list_array.is_valid(i)); + assert!(!list_array.is_null(i)); + } + + // Now test with a non-zero offset + let list_data = ArrayData::builder(list_data_type) + .len(3) + .offset(1) + .add_child_data(value_data.clone()) + .build(); + let list_array = FixedSizeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(3, list_array.len()); + assert_eq!(0, list_array.null_count()); + assert_eq!( + 3, + list_array + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0) + ); + assert_eq!(6, list_array.value_offset(1)); + assert_eq!(3, list_array.value_length()); + } + + #[test] + #[should_panic( + expected = "FixedSizeListArray child array length should be a multiple of 3" + )] + fn test_fixed_size_list_array_unequal_children() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) + .build(); + + // Construct a list array from the above two + let list_data_type = DataType::FixedSizeList( + Box::new(Field::new("item", DataType::Int32, false)), + 3, + ); + let list_data = ArrayData::builder(list_data_type) + .len(3) + .add_child_data(value_data) + .build(); + FixedSizeListArray::from(list_data); + } + + #[test] + fn test_list_array_slice() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(10) + .add_buffer(Buffer::from( + &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), + )) + .build(); + + // Construct a buffer for value offsets, for the nested array: + // [[0, 1], null, null, [2, 3], [4, 5], null, [6, 7, 8], null, [9]] + let value_offsets = + Buffer::from(&[0, 2, 2, 2, 4, 6, 6, 9, 9, 10].to_byte_slice()); + // 01011001 00000001 + let mut null_bits: [u8; 2] = [0; 2]; + bit_util::set_bit(&mut null_bits, 0); + bit_util::set_bit(&mut null_bits, 3); + bit_util::set_bit(&mut null_bits, 4); + bit_util::set_bit(&mut null_bits, 6); + bit_util::set_bit(&mut null_bits, 8); + + // Construct a list array from the above two + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .len(9) + .add_buffer(value_offsets) + .add_child_data(value_data.clone()) + .null_bit_buffer(Buffer::from(null_bits)) + .build(); + let list_array = ListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(9, list_array.len()); + assert_eq!(4, list_array.null_count()); + assert_eq!(2, list_array.value_offset(3)); + assert_eq!(2, list_array.value_length(3)); + + let sliced_array = list_array.slice(1, 6); + assert_eq!(6, sliced_array.len()); + assert_eq!(1, sliced_array.offset()); + assert_eq!(3, sliced_array.null_count()); + + for i in 0..sliced_array.len() { + if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { + assert!(sliced_array.is_valid(i)); + } else { + assert!(sliced_array.is_null(i)); + } + } + + // Check offset and length for each non-null value. + let sliced_list_array = + sliced_array.as_any().downcast_ref::().unwrap(); + assert_eq!(2, sliced_list_array.value_offset(2)); + assert_eq!(2, sliced_list_array.value_length(2)); + assert_eq!(4, sliced_list_array.value_offset(3)); + assert_eq!(2, sliced_list_array.value_length(3)); + assert_eq!(6, sliced_list_array.value_offset(5)); + assert_eq!(3, sliced_list_array.value_length(5)); + } + + #[test] + fn test_large_list_array_slice() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(10) + .add_buffer(Buffer::from( + &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), + )) + .build(); + + // Construct a buffer for value offsets, for the nested array: + // [[0, 1], null, null, [2, 3], [4, 5], null, [6, 7, 8], null, [9]] + let value_offsets = + Buffer::from(&[0i64, 2, 2, 2, 4, 6, 6, 9, 9, 10].to_byte_slice()); + // 01011001 00000001 + let mut null_bits: [u8; 2] = [0; 2]; + bit_util::set_bit(&mut null_bits, 0); + bit_util::set_bit(&mut null_bits, 3); + bit_util::set_bit(&mut null_bits, 4); + bit_util::set_bit(&mut null_bits, 6); + bit_util::set_bit(&mut null_bits, 8); + + // Construct a list array from the above two + let list_data_type = + DataType::LargeList(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .len(9) + .add_buffer(value_offsets) + .add_child_data(value_data.clone()) + .null_bit_buffer(Buffer::from(null_bits)) + .build(); + let list_array = LargeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(9, list_array.len()); + assert_eq!(4, list_array.null_count()); + assert_eq!(2, list_array.value_offset(3)); + assert_eq!(2, list_array.value_length(3)); + + let sliced_array = list_array.slice(1, 6); + assert_eq!(6, sliced_array.len()); + assert_eq!(1, sliced_array.offset()); + assert_eq!(3, sliced_array.null_count()); + + for i in 0..sliced_array.len() { + if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { + assert!(sliced_array.is_valid(i)); + } else { + assert!(sliced_array.is_null(i)); + } + } + + // Check offset and length for each non-null value. + let sliced_list_array = sliced_array + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(2, sliced_list_array.value_offset(2)); + assert_eq!(2, sliced_list_array.value_length(2)); + assert_eq!(4, sliced_list_array.value_offset(3)); + assert_eq!(2, sliced_list_array.value_length(3)); + assert_eq!(6, sliced_list_array.value_offset(5)); + assert_eq!(3, sliced_list_array.value_length(5)); + } + + #[test] + fn test_fixed_size_list_array_slice() { + // Construct a value array + let value_data = ArrayData::builder(DataType::Int32) + .len(10) + .add_buffer(Buffer::from( + &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9].to_byte_slice(), + )) + .build(); + + // Set null buts for the nested array: + // [[0, 1], null, null, [6, 7], [8, 9]] + // 01011001 00000001 + let mut null_bits: [u8; 1] = [0; 1]; + bit_util::set_bit(&mut null_bits, 0); + bit_util::set_bit(&mut null_bits, 3); + bit_util::set_bit(&mut null_bits, 4); + + // Construct a fixed size list array from the above two + let list_data_type = DataType::FixedSizeList( + Box::new(Field::new("item", DataType::Int32, false)), + 2, + ); + let list_data = ArrayData::builder(list_data_type) + .len(5) + .add_child_data(value_data.clone()) + .null_bit_buffer(Buffer::from(null_bits)) + .build(); + let list_array = FixedSizeListArray::from(list_data); + + let values = list_array.values(); + assert_eq!(value_data, values.data()); + assert_eq!(DataType::Int32, list_array.value_type()); + assert_eq!(5, list_array.len()); + assert_eq!(2, list_array.null_count()); + assert_eq!(6, list_array.value_offset(3)); + assert_eq!(2, list_array.value_length()); + + let sliced_array = list_array.slice(1, 4); + assert_eq!(4, sliced_array.len()); + assert_eq!(1, sliced_array.offset()); + assert_eq!(2, sliced_array.null_count()); + + for i in 0..sliced_array.len() { + if bit_util::get_bit(&null_bits, sliced_array.offset() + i) { + assert!(sliced_array.is_valid(i)); + } else { + assert!(sliced_array.is_null(i)); + } + } + + // Check offset and length for each non-null value. + let sliced_list_array = sliced_array + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(2, sliced_list_array.value_length()); + assert_eq!(6, sliced_list_array.value_offset(2)); + assert_eq!(8, sliced_list_array.value_offset(3)); + } + + #[test] + #[should_panic( + expected = "ListArray data should contain a single buffer only (value offsets)" + )] + fn test_list_array_invalid_buffer_len() { + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) + .build(); + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .len(3) + .add_child_data(value_data) + .build(); + ListArray::from(list_data); + } + + #[test] + #[should_panic( + expected = "ListArray should contain a single child array (values array)" + )] + fn test_list_array_invalid_child_array_len() { + let value_offsets = Buffer::from(&[0, 2, 5, 7].to_byte_slice()); + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .len(3) + .add_buffer(value_offsets) + .build(); + ListArray::from(list_data); + } + + #[test] + #[should_panic(expected = "offsets do not start at zero")] + fn test_list_array_invalid_value_offset_start() { + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from(&[0, 1, 2, 3, 4, 5, 6, 7].to_byte_slice())) + .build(); + + let value_offsets = Buffer::from(&[2, 2, 5, 7].to_byte_slice()); + + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .len(3) + .add_buffer(value_offsets) + .add_child_data(value_data) + .build(); + ListArray::from(list_data); + } + + #[test] + #[should_panic(expected = "memory is not aligned")] + fn test_primitive_array_alignment() { + let ptr = memory::allocate_aligned(8); + let buf = unsafe { Buffer::from_raw_parts(ptr, 8, 8) }; + let buf2 = buf.slice(1); + let array_data = ArrayData::builder(DataType::Int32).add_buffer(buf2).build(); + Int32Array::from(array_data); + } + + #[test] + #[should_panic(expected = "memory is not aligned")] + fn test_list_array_alignment() { + let ptr = memory::allocate_aligned(8); + let buf = unsafe { Buffer::from_raw_parts(ptr, 8, 8) }; + let buf2 = buf.slice(1); + + let values: [i32; 8] = [0; 8]; + let value_data = ArrayData::builder(DataType::Int32) + .add_buffer(Buffer::from(values.to_byte_slice())) + .build(); + + let list_data_type = + DataType::List(Box::new(Field::new("item", DataType::Int32, false))); + let list_data = ArrayData::builder(list_data_type) + .add_buffer(buf2) + .add_child_data(value_data) + .build(); + ListArray::from(list_data); + } +} diff --git a/rust/arrow/src/array/array_primitive.rs b/rust/arrow/src/array/array_primitive.rs new file mode 100644 index 00000000000..ade18dbe329 --- /dev/null +++ b/rust/arrow/src/array/array_primitive.rs @@ -0,0 +1,989 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::any::Any; +use std::borrow::Borrow; +use std::convert::From; +use std::fmt; +use std::io::Write; +use std::iter::{FromIterator, IntoIterator}; +use std::mem; +use std::sync::Arc; + +use chrono::prelude::*; + +use super::array::print_long_array; +use super::raw_pointer::RawPtrBox; +use super::*; +use crate::buffer::{Buffer, MutableBuffer}; +use crate::memory; +use crate::util::bit_util; + +/// Number of seconds in a day +const SECONDS_IN_DAY: i64 = 86_400; +/// Number of milliseconds in a second +const MILLISECONDS: i64 = 1_000; +/// Number of microseconds in a second +const MICROSECONDS: i64 = 1_000_000; +/// Number of nanoseconds in a second +const NANOSECONDS: i64 = 1_000_000_000; + +/// Array whose elements are of primitive types. +pub struct PrimitiveArray { + data: ArrayDataRef, + /// Pointer to the value array. The lifetime of this must be <= to the value buffer + /// stored in `data`, so it's safe to store. + /// Also note that boolean arrays are bit-packed, so although the underlying pointer + /// is of type bool it should be cast back to u8 before being used. + /// i.e. `self.raw_values.get() as *const u8` + raw_values: RawPtrBox, +} + +impl PrimitiveArray { + /// Returns the length of this array. + pub fn len(&self) -> usize { + self.data.len() + } + + /// Returns whether this array is empty. + pub fn is_empty(&self) -> bool { + self.data.is_empty() + } + + /// Returns a raw pointer to the values of this array. + pub fn raw_values(&self) -> *const T::Native { + unsafe { self.raw_values.get().add(self.data.offset()) } + } + + /// Returns a slice for the given offset and length + /// + /// Note this doesn't do any bound checking, for performance reason. + pub fn value_slice(&self, offset: usize, len: usize) -> &[T::Native] { + let raw = + unsafe { std::slice::from_raw_parts(self.raw_values().add(offset), len) }; + &raw[..] + } + + // Returns a new primitive array builder + pub fn builder(capacity: usize) -> PrimitiveBuilder { + PrimitiveBuilder::::new(capacity) + } + + /// Returns a `Buffer` holding all the values of this array. + /// + /// Note this doesn't take the offset of this array into account. + pub fn values(&self) -> Buffer { + self.data.buffers()[0].clone() + } + + /// Returns the primitive value at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + pub fn value(&self, i: usize) -> T::Native { + let offset = i + self.offset(); + unsafe { T::index(self.raw_values.get(), offset) } + } +} + +impl Array for PrimitiveArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [PrimitiveArray]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [PrimitiveArray]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +fn as_datetime(v: i64) -> Option { + match T::DATA_TYPE { + DataType::Date32(_) => { + // convert days into seconds + Some(NaiveDateTime::from_timestamp(v as i64 * SECONDS_IN_DAY, 0)) + } + DataType::Date64(_) => Some(NaiveDateTime::from_timestamp( + // extract seconds from milliseconds + v / MILLISECONDS, + // discard extracted seconds and convert milliseconds to nanoseconds + (v % MILLISECONDS * MICROSECONDS) as u32, + )), + DataType::Time32(_) | DataType::Time64(_) => None, + DataType::Timestamp(unit, _) => match unit { + TimeUnit::Second => Some(NaiveDateTime::from_timestamp(v, 0)), + TimeUnit::Millisecond => Some(NaiveDateTime::from_timestamp( + // extract seconds from milliseconds + v / MILLISECONDS, + // discard extracted seconds and convert milliseconds to nanoseconds + (v % MILLISECONDS * MICROSECONDS) as u32, + )), + TimeUnit::Microsecond => Some(NaiveDateTime::from_timestamp( + // extract seconds from microseconds + v / MICROSECONDS, + // discard extracted seconds and convert microseconds to nanoseconds + (v % MICROSECONDS * MILLISECONDS) as u32, + )), + TimeUnit::Nanosecond => Some(NaiveDateTime::from_timestamp( + // extract seconds from nanoseconds + v / NANOSECONDS, + // discard extracted seconds + (v % NANOSECONDS) as u32, + )), + }, + // interval is not yet fully documented [ARROW-3097] + DataType::Interval(_) => None, + _ => None, + } +} + +fn as_date(v: i64) -> Option { + as_datetime::(v).map(|datetime| datetime.date()) +} + +fn as_time(v: i64) -> Option { + match T::DATA_TYPE { + DataType::Time32(unit) => { + // safe to immediately cast to u32 as `self.value(i)` is positive i32 + let v = v as u32; + match unit { + TimeUnit::Second => Some(NaiveTime::from_num_seconds_from_midnight(v, 0)), + TimeUnit::Millisecond => { + Some(NaiveTime::from_num_seconds_from_midnight( + // extract seconds from milliseconds + v / MILLISECONDS as u32, + // discard extracted seconds and convert milliseconds to + // nanoseconds + v % MILLISECONDS as u32 * MICROSECONDS as u32, + )) + } + _ => None, + } + } + DataType::Time64(unit) => { + match unit { + TimeUnit::Microsecond => { + Some(NaiveTime::from_num_seconds_from_midnight( + // extract seconds from microseconds + (v / MICROSECONDS) as u32, + // discard extracted seconds and convert microseconds to + // nanoseconds + (v % MICROSECONDS * MILLISECONDS) as u32, + )) + } + TimeUnit::Nanosecond => { + Some(NaiveTime::from_num_seconds_from_midnight( + // extract seconds from nanoseconds + (v / NANOSECONDS) as u32, + // discard extracted seconds + (v % NANOSECONDS) as u32, + )) + } + _ => None, + } + } + DataType::Timestamp(_, _) => as_datetime::(v).map(|datetime| datetime.time()), + DataType::Date32(_) | DataType::Date64(_) => Some(NaiveTime::from_hms(0, 0, 0)), + DataType::Interval(_) => None, + _ => None, + } +} + +impl PrimitiveArray +where + i64: std::convert::From, +{ + /// Returns value as a chrono `NaiveDateTime`, handling time resolution + /// + /// If a data type cannot be converted to `NaiveDateTime`, a `None` is returned. + /// A valid value is expected, thus the user should first check for validity. + pub fn value_as_datetime(&self, i: usize) -> Option { + as_datetime::(i64::from(self.value(i))) + } + + /// Returns value as a chrono `NaiveDate` by using `Self::datetime()` + /// + /// If a data type cannot be converted to `NaiveDate`, a `None` is returned + pub fn value_as_date(&self, i: usize) -> Option { + self.value_as_datetime(i).map(|datetime| datetime.date()) + } + + /// Returns a value as a chrono `NaiveTime` + /// + /// `Date32` and `Date64` return UTC midnight as they do not have time resolution + pub fn value_as_time(&self, i: usize) -> Option { + as_time::(i64::from(self.value(i))) + } +} + +impl fmt::Debug for PrimitiveArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "PrimitiveArray<{:?}>\n[\n", T::DATA_TYPE)?; + print_long_array(self, f, |array, index, f| match T::DATA_TYPE { + DataType::Date32(_) | DataType::Date64(_) => { + let v = self.value(index).to_usize().unwrap() as i64; + match as_date::(v) { + Some(date) => write!(f, "{:?}", date), + None => write!(f, "null"), + } + } + DataType::Time32(_) | DataType::Time64(_) => { + let v = self.value(index).to_usize().unwrap() as i64; + match as_time::(v) { + Some(time) => write!(f, "{:?}", time), + None => write!(f, "null"), + } + } + DataType::Timestamp(_, _) => { + let v = self.value(index).to_usize().unwrap() as i64; + match as_datetime::(v) { + Some(datetime) => write!(f, "{:?}", datetime), + None => write!(f, "null"), + } + } + _ => fmt::Debug::fmt(&array.value(index), f), + })?; + write!(f, "]") + } +} + +impl<'a, T: ArrowPrimitiveType> IntoIterator for &'a PrimitiveArray { + type Item = Option<::Native>; + type IntoIter = PrimitiveIter<'a, T>; + + fn into_iter(self) -> Self::IntoIter { + PrimitiveIter::<'a, T>::new(self) + } +} + +impl<'a, T: ArrowPrimitiveType> PrimitiveArray { + /// constructs a new iterator + pub fn iter(&'a self) -> PrimitiveIter<'a, T> { + PrimitiveIter::<'a, T>::new(&self) + } +} + +impl::Native>>> + FromIterator for PrimitiveArray +{ + fn from_iter>(iter: I) -> Self { + let iter = iter.into_iter(); + let (_, data_len) = iter.size_hint(); + let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. + + let num_bytes = bit_util::ceil(data_len, 8); + let mut null_buf = MutableBuffer::new(num_bytes).with_bitset(num_bytes, false); + let mut val_buf = MutableBuffer::new( + data_len * mem::size_of::<::Native>(), + ); + + let null = vec![0; mem::size_of::<::Native>()]; + + let null_slice = null_buf.data_mut(); + iter.enumerate().for_each(|(i, item)| { + if let Some(a) = item.borrow() { + bit_util::set_bit(null_slice, i); + val_buf.write_all(a.to_byte_slice()).unwrap(); + } else { + val_buf.write_all(&null).unwrap(); + } + }); + + let data = ArrayData::new( + T::DATA_TYPE, + data_len, + None, + Some(null_buf.freeze()), + 0, + vec![val_buf.freeze()], + vec![], + ); + PrimitiveArray::from(Arc::new(data)) + } +} + +// TODO: the macro is needed here because we'd get "conflicting implementations" error +// otherwise with both `From>` and `From>>`. +// We should revisit this in future. +macro_rules! def_numeric_from_vec { + ( $ty:ident ) => { + impl From::Native>> for PrimitiveArray<$ty> { + fn from(data: Vec<<$ty as ArrowPrimitiveType>::Native>) -> Self { + let array_data = ArrayData::builder($ty::DATA_TYPE) + .len(data.len()) + .add_buffer(Buffer::from(data.to_byte_slice())) + .build(); + PrimitiveArray::from(array_data) + } + } + + // Constructs a primitive array from a vector. Should only be used for testing. + impl From::Native>>> + for PrimitiveArray<$ty> + { + fn from(data: Vec::Native>>) -> Self { + PrimitiveArray::from_iter(data.iter()) + } + } + }; +} + +def_numeric_from_vec!(Int8Type); +def_numeric_from_vec!(Int16Type); +def_numeric_from_vec!(Int32Type); +def_numeric_from_vec!(Int64Type); +def_numeric_from_vec!(UInt8Type); +def_numeric_from_vec!(UInt16Type); +def_numeric_from_vec!(UInt32Type); +def_numeric_from_vec!(UInt64Type); +def_numeric_from_vec!(Float32Type); +def_numeric_from_vec!(Float64Type); + +def_numeric_from_vec!(Date32Type); +def_numeric_from_vec!(Date64Type); +def_numeric_from_vec!(Time32SecondType); +def_numeric_from_vec!(Time32MillisecondType); +def_numeric_from_vec!(Time64MicrosecondType); +def_numeric_from_vec!(Time64NanosecondType); +def_numeric_from_vec!(IntervalYearMonthType); +def_numeric_from_vec!(IntervalDayTimeType); +def_numeric_from_vec!(DurationSecondType); +def_numeric_from_vec!(DurationMillisecondType); +def_numeric_from_vec!(DurationMicrosecondType); +def_numeric_from_vec!(DurationNanosecondType); +def_numeric_from_vec!(TimestampMillisecondType); +def_numeric_from_vec!(TimestampMicrosecondType); + +impl PrimitiveArray { + /// Construct a timestamp array from a vec of i64 values and an optional timezone + pub fn from_vec(data: Vec, timezone: Option>) -> Self { + let array_data = + ArrayData::builder(DataType::Timestamp(T::get_time_unit(), timezone)) + .len(data.len()) + .add_buffer(Buffer::from(data.to_byte_slice())) + .build(); + PrimitiveArray::from(array_data) + } +} + +impl PrimitiveArray { + /// Construct a timestamp array from a vec of Option values and an optional timezone + pub fn from_opt_vec(data: Vec>, timezone: Option>) -> Self { + // TODO: duplicated from def_numeric_from_vec! macro, it looks possible to convert to generic + let data_len = data.len(); + let mut null_buf = MutableBuffer::new_null(data_len); + let mut val_buf = MutableBuffer::new(data_len * mem::size_of::()); + + { + let null = vec![0; mem::size_of::()]; + let null_slice = null_buf.data_mut(); + for (i, v) in data.iter().enumerate() { + if let Some(n) = v { + bit_util::set_bit(null_slice, i); + // unwrap() in the following should be safe here since we've + // made sure enough space is allocated for the values. + val_buf.write_all(&n.to_byte_slice()).unwrap(); + } else { + val_buf.write_all(&null).unwrap(); + } + } + } + + let array_data = + ArrayData::builder(DataType::Timestamp(T::get_time_unit(), timezone)) + .len(data_len) + .add_buffer(val_buf.freeze()) + .null_bit_buffer(null_buf.freeze()) + .build(); + PrimitiveArray::from(array_data) + } +} + +/// Constructs a boolean array from a vector. Should only be used for testing. +impl From> for BooleanArray { + fn from(data: Vec) -> Self { + let mut mut_buf = MutableBuffer::new_null(data.len()); + { + let mut_slice = mut_buf.data_mut(); + for (i, b) in data.iter().enumerate() { + if *b { + bit_util::set_bit(mut_slice, i); + } + } + } + let array_data = ArrayData::builder(DataType::Boolean) + .len(data.len()) + .add_buffer(mut_buf.freeze()) + .build(); + BooleanArray::from(array_data) + } +} + +impl From>> for BooleanArray { + fn from(data: Vec>) -> Self { + let data_len = data.len(); + let num_byte = bit_util::ceil(data_len, 8); + let mut null_buf = MutableBuffer::new_null(data.len()); + let mut val_buf = MutableBuffer::new(num_byte).with_bitset(num_byte, false); + + { + let null_slice = null_buf.data_mut(); + let val_slice = val_buf.data_mut(); + + for (i, v) in data.iter().enumerate() { + if let Some(b) = v { + bit_util::set_bit(null_slice, i); + if *b { + bit_util::set_bit(val_slice, i); + } + } + } + } + + let array_data = ArrayData::builder(DataType::Boolean) + .len(data_len) + .add_buffer(val_buf.freeze()) + .null_bit_buffer(null_buf.freeze()) + .build(); + BooleanArray::from(array_data) + } +} + +/// Constructs a `PrimitiveArray` from an array data reference. +impl From for PrimitiveArray { + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.buffers().len(), + 1, + "PrimitiveArray data should contain a single buffer only (values buffer)" + ); + let raw_values = data.buffers()[0].raw_data(); + assert!( + memory::is_aligned::(raw_values, mem::align_of::()), + "memory is not aligned" + ); + Self { + data, + raw_values: RawPtrBox::new(raw_values as *const T::Native), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use std::thread; + + use crate::buffer::Buffer; + use crate::datatypes::DataType; + + #[test] + fn test_primitive_array_from_vec() { + let buf = Buffer::from(&[0, 1, 2, 3, 4].to_byte_slice()); + let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); + let slice = unsafe { std::slice::from_raw_parts(arr.raw_values(), 5) }; + assert_eq!(buf, arr.values()); + assert_eq!(&[0, 1, 2, 3, 4], slice); + assert_eq!(5, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + for i in 0..5 { + assert!(!arr.is_null(i)); + assert!(arr.is_valid(i)); + assert_eq!(i as i32, arr.value(i)); + } + + assert_eq!(64, arr.get_buffer_memory_size()); + let internals_of_primitive_array = 8 + 72; // RawPtrBox & Arc combined. + assert_eq!( + arr.get_buffer_memory_size() + internals_of_primitive_array, + arr.get_array_memory_size() + ); + } + + #[test] + fn test_primitive_array_from_vec_option() { + // Test building a primitive array with null values + let arr = Int32Array::from(vec![Some(0), None, Some(2), None, Some(4)]); + assert_eq!(5, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(2, arr.null_count()); + for i in 0..5 { + if i % 2 == 0 { + assert!(!arr.is_null(i)); + assert!(arr.is_valid(i)); + assert_eq!(i as i32, arr.value(i)); + } else { + assert!(arr.is_null(i)); + assert!(!arr.is_valid(i)); + } + } + + assert_eq!(128, arr.get_buffer_memory_size()); + let internals_of_primitive_array = 8 + 72 + 16; // RawPtrBox & Arc and it's null_bitmap combined. + assert_eq!( + arr.get_buffer_memory_size() + internals_of_primitive_array, + arr.get_array_memory_size() + ); + } + + #[test] + fn test_date64_array_from_vec_option() { + // Test building a primitive array with null values + // we use Int32 and Int64 as a backing array, so all Int32 and Int64 conventions + // work + let arr: PrimitiveArray = + vec![Some(1550902545147), None, Some(1550902545147)].into(); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + for i in 0..3 { + if i % 2 == 0 { + assert!(!arr.is_null(i)); + assert!(arr.is_valid(i)); + assert_eq!(1550902545147, arr.value(i)); + // roundtrip to and from datetime + assert_eq!( + 1550902545147, + arr.value_as_datetime(i).unwrap().timestamp_millis() + ); + } else { + assert!(arr.is_null(i)); + assert!(!arr.is_valid(i)); + } + } + } + + #[test] + fn test_time32_millisecond_array_from_vec() { + // 1: 00:00:00.001 + // 37800005: 10:30:00.005 + // 86399210: 23:59:59.210 + let arr: PrimitiveArray = + vec![1, 37_800_005, 86_399_210].into(); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + let formatted = vec!["00:00:00.001", "10:30:00.005", "23:59:59.210"]; + for i in 0..3 { + // check that we can't create dates or datetimes from time instances + assert_eq!(None, arr.value_as_datetime(i)); + assert_eq!(None, arr.value_as_date(i)); + let time = arr.value_as_time(i).unwrap(); + assert_eq!(formatted[i], time.format("%H:%M:%S%.3f").to_string()); + } + } + + #[test] + fn test_time64_nanosecond_array_from_vec() { + // Test building a primitive array with null values + // we use Int32 and Int64 as a backing array, so all Int32 and Int64 conventions + // work + + // 1e6: 00:00:00.001 + // 37800005e6: 10:30:00.005 + // 86399210e6: 23:59:59.210 + let arr: PrimitiveArray = + vec![1_000_000, 37_800_005_000_000, 86_399_210_000_000].into(); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + let formatted = vec!["00:00:00.001", "10:30:00.005", "23:59:59.210"]; + for i in 0..3 { + // check that we can't create dates or datetimes from time instances + assert_eq!(None, arr.value_as_datetime(i)); + assert_eq!(None, arr.value_as_date(i)); + let time = arr.value_as_time(i).unwrap(); + assert_eq!(formatted[i], time.format("%H:%M:%S%.3f").to_string()); + } + } + + #[test] + fn test_interval_array_from_vec() { + // intervals are currently not treated specially, but are Int32 and Int64 arrays + let arr = IntervalYearMonthArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + + // a day_time interval contains days and milliseconds, but we do not yet have accessors for the values + let arr = IntervalDayTimeArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + } + + #[test] + fn test_duration_array_from_vec() { + let arr = DurationSecondArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + + let arr = DurationMillisecondArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + + let arr = DurationMicrosecondArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + + let arr = DurationNanosecondArray::from(vec![Some(1), None, Some(-5)]); + assert_eq!(3, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert!(arr.is_null(1)); + assert_eq!(-5, arr.value(2)); + } + + #[test] + fn test_timestamp_array_from_vec() { + let arr = TimestampSecondArray::from_vec(vec![1, -5], None); + assert_eq!(2, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert_eq!(-5, arr.value(1)); + + let arr = TimestampMillisecondArray::from_vec(vec![1, -5], None); + assert_eq!(2, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert_eq!(-5, arr.value(1)); + + let arr = TimestampMicrosecondArray::from_vec(vec![1, -5], None); + assert_eq!(2, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert_eq!(-5, arr.value(1)); + + let arr = TimestampNanosecondArray::from_vec(vec![1, -5], None); + assert_eq!(2, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + assert_eq!(1, arr.value(0)); + assert_eq!(-5, arr.value(1)); + } + + #[test] + fn test_primitive_array_slice() { + let arr = Int32Array::from(vec![ + Some(0), + None, + Some(2), + None, + Some(4), + Some(5), + Some(6), + None, + None, + ]); + assert_eq!(9, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(4, arr.null_count()); + + let arr2 = arr.slice(2, 5); + assert_eq!(5, arr2.len()); + assert_eq!(2, arr2.offset()); + assert_eq!(1, arr2.null_count()); + + for i in 0..arr2.len() { + assert_eq!(i == 1, arr2.is_null(i)); + assert_eq!(i != 1, arr2.is_valid(i)); + } + + let arr3 = arr2.slice(2, 3); + assert_eq!(3, arr3.len()); + assert_eq!(4, arr3.offset()); + assert_eq!(0, arr3.null_count()); + + let int_arr = arr3.as_any().downcast_ref::().unwrap(); + assert_eq!(4, int_arr.value(0)); + assert_eq!(5, int_arr.value(1)); + assert_eq!(6, int_arr.value(2)); + } + + #[test] + fn test_boolean_array_slice() { + let arr = BooleanArray::from(vec![ + Some(true), + None, + Some(false), + None, + Some(true), + Some(false), + Some(true), + Some(false), + None, + Some(true), + ]); + + assert_eq!(10, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(3, arr.null_count()); + + let arr2 = arr.slice(3, 5); + assert_eq!(5, arr2.len()); + assert_eq!(3, arr2.offset()); + assert_eq!(1, arr2.null_count()); + + let bool_arr = arr2.as_any().downcast_ref::().unwrap(); + + assert_eq!(false, bool_arr.is_valid(0)); + + assert_eq!(true, bool_arr.is_valid(1)); + assert_eq!(true, bool_arr.value(1)); + + assert_eq!(true, bool_arr.is_valid(2)); + assert_eq!(false, bool_arr.value(2)); + + assert_eq!(true, bool_arr.is_valid(3)); + assert_eq!(true, bool_arr.value(3)); + + assert_eq!(true, bool_arr.is_valid(4)); + assert_eq!(false, bool_arr.value(4)); + } + + #[test] + fn test_value_slice_no_bounds_check() { + let arr = Int32Array::from(vec![2, 3, 4]); + let _slice = arr.value_slice(0, 4); + } + + #[test] + fn test_int32_fmt_debug() { + let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); + assert_eq!( + "PrimitiveArray\n[\n 0,\n 1,\n 2,\n 3,\n 4,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_fmt_debug_up_to_20_elements() { + (1..=20).for_each(|i| { + let values = (0..i).collect::>(); + let array_expected = format!( + "PrimitiveArray\n[\n{}\n]", + values + .iter() + .map(|v| { format!(" {},", v) }) + .collect::>() + .join("\n") + ); + let array = Int16Array::from(values); + + assert_eq!(array_expected, format!("{:?}", array)); + }) + } + + #[test] + fn test_int32_with_null_fmt_debug() { + let mut builder = Int32Array::builder(3); + builder.append_slice(&[0, 1]).unwrap(); + builder.append_null().unwrap(); + builder.append_slice(&[3, 4]).unwrap(); + let arr = builder.finish(); + assert_eq!( + "PrimitiveArray\n[\n 0,\n 1,\n null,\n 3,\n 4,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_boolean_fmt_debug() { + let arr = BooleanArray::from(vec![true, false, false]); + assert_eq!( + "PrimitiveArray\n[\n true,\n false,\n false,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_boolean_with_null_fmt_debug() { + let mut builder = BooleanArray::builder(3); + builder.append_value(true).unwrap(); + builder.append_null().unwrap(); + builder.append_value(false).unwrap(); + let arr = builder.finish(); + assert_eq!( + "PrimitiveArray\n[\n true,\n null,\n false,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_timestamp_fmt_debug() { + let arr: PrimitiveArray = + TimestampMillisecondArray::from_vec(vec![1546214400000, 1546214400000], None); + assert_eq!( + "PrimitiveArray\n[\n 2018-12-31T00:00:00,\n 2018-12-31T00:00:00,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_date32_fmt_debug() { + let arr: PrimitiveArray = vec![12356, 13548].into(); + assert_eq!( + "PrimitiveArray\n[\n 2003-10-31,\n 2007-02-04,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_time32second_fmt_debug() { + let arr: PrimitiveArray = vec![7201, 60054].into(); + assert_eq!( + "PrimitiveArray\n[\n 02:00:01,\n 16:40:54,\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_primitive_array_builder() { + // Test building a primitive array with ArrayData builder and offset + let buf = Buffer::from(&[0, 1, 2, 3, 4].to_byte_slice()); + let buf2 = buf.clone(); + let data = ArrayData::builder(DataType::Int32) + .len(5) + .offset(2) + .add_buffer(buf) + .build(); + let arr = Int32Array::from(data); + assert_eq!(buf2, arr.values()); + assert_eq!(5, arr.len()); + assert_eq!(0, arr.null_count()); + for i in 0..3 { + assert_eq!((i + 2) as i32, arr.value(i)); + } + } + + #[test] + #[should_panic(expected = "PrimitiveArray data should contain a single buffer only \ + (values buffer)")] + fn test_primitive_array_invalid_buffer_len() { + let data = ArrayData::builder(DataType::Int32).len(5).build(); + Int32Array::from(data); + } + + #[test] + fn test_boolean_array_from_vec() { + let buf = Buffer::from([10_u8]); + let arr = BooleanArray::from(vec![false, true, false, true]); + assert_eq!(buf, arr.values()); + assert_eq!(4, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(0, arr.null_count()); + for i in 0..4 { + assert!(!arr.is_null(i)); + assert!(arr.is_valid(i)); + assert_eq!(i == 1 || i == 3, arr.value(i), "failed at {}", i) + } + } + + #[test] + fn test_boolean_array_from_vec_option() { + let buf = Buffer::from([10_u8]); + let arr = BooleanArray::from(vec![Some(false), Some(true), None, Some(true)]); + assert_eq!(buf, arr.values()); + assert_eq!(4, arr.len()); + assert_eq!(0, arr.offset()); + assert_eq!(1, arr.null_count()); + for i in 0..4 { + if i == 2 { + assert!(arr.is_null(i)); + assert!(!arr.is_valid(i)); + } else { + assert!(!arr.is_null(i)); + assert!(arr.is_valid(i)); + assert_eq!(i == 1 || i == 3, arr.value(i), "failed at {}", i) + } + } + } + + #[test] + fn test_boolean_array_builder() { + // Test building a boolean array with ArrayData builder and offset + // 000011011 + let buf = Buffer::from([27_u8]); + let buf2 = buf.clone(); + let data = ArrayData::builder(DataType::Boolean) + .len(5) + .offset(2) + .add_buffer(buf) + .build(); + let arr = BooleanArray::from(data); + assert_eq!(buf2, arr.values()); + assert_eq!(5, arr.len()); + assert_eq!(2, arr.offset()); + assert_eq!(0, arr.null_count()); + for i in 0..3 { + assert_eq!(i != 0, arr.value(i), "failed at {}", i); + } + } + + #[test] + #[should_panic(expected = "PrimitiveArray data should contain a single buffer only \ + (values buffer)")] + fn test_boolean_array_invalid_buffer_len() { + let data = ArrayData::builder(DataType::Boolean).len(5).build(); + BooleanArray::from(data); + } + + #[test] + fn test_access_array_concurrently() { + let a = Int32Array::from(vec![5, 6, 7, 8, 9]); + let ret = thread::spawn(move || a.value(3)).join(); + + assert!(ret.is_ok()); + assert_eq!(8, ret.ok().unwrap()); + } +} diff --git a/rust/arrow/src/array/array_string.rs b/rust/arrow/src/array/array_string.rs new file mode 100644 index 00000000000..ec79ac9010d --- /dev/null +++ b/rust/arrow/src/array/array_string.rs @@ -0,0 +1,445 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::convert::From; +use std::fmt; +use std::mem; +use std::{any::Any, iter::FromIterator}; + +use super::{ + array::print_long_array, raw_pointer::as_aligned_pointer, raw_pointer::RawPtrBox, + Array, ArrayData, ArrayDataRef, GenericListArray, GenericStringIter, LargeListArray, + ListArray, OffsetSizeTrait, +}; +use crate::util::bit_util; +use crate::{buffer::Buffer, datatypes::ToByteSlice}; +use crate::{buffer::MutableBuffer, datatypes::DataType}; + +/// Like OffsetSizeTrait, but specialized for Strings +// This allow us to expose a constant datatype for the GenericStringArray +pub trait StringOffsetSizeTrait: OffsetSizeTrait { + const DATA_TYPE: DataType; +} + +impl StringOffsetSizeTrait for i32 { + const DATA_TYPE: DataType = DataType::Utf8; +} + +impl StringOffsetSizeTrait for i64 { + const DATA_TYPE: DataType = DataType::LargeUtf8; +} + +/// Generic struct for \[Large\]StringArray +pub struct GenericStringArray { + data: ArrayDataRef, + value_offsets: RawPtrBox, + value_data: RawPtrBox, +} + +impl GenericStringArray { + /// Returns the offset for the element at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_offset(&self, i: usize) -> OffsetSize { + self.value_offset_at(self.data.offset() + i) + } + + /// Returns the length for the element at index `i`. + /// + /// Note this doesn't do any bound checking, for performance reason. + #[inline] + pub fn value_length(&self, mut i: usize) -> OffsetSize { + i += self.data.offset(); + self.value_offset_at(i + 1) - self.value_offset_at(i) + } + + /// Returns a clone of the value offset buffer + pub fn value_offsets(&self) -> Buffer { + self.data.buffers()[0].clone() + } + + /// Returns a clone of the value data buffer + pub fn value_data(&self) -> Buffer { + self.data.buffers()[1].clone() + } + + #[inline] + fn value_offset_at(&self, i: usize) -> OffsetSize { + unsafe { *self.value_offsets.get().add(i) } + } + + /// Returns the element at index `i` as &str + pub fn value(&self, i: usize) -> &str { + assert!(i < self.data.len(), "StringArray out of bounds access"); + let offset = i.checked_add(self.data.offset()).unwrap(); + unsafe { + let pos = self.value_offset_at(offset); + let slice = std::slice::from_raw_parts( + self.value_data.get().offset(pos.to_isize()), + (self.value_offset_at(offset + 1) - pos).to_usize().unwrap(), + ); + + std::str::from_utf8_unchecked(slice) + } + } + + fn from_list(v: GenericListArray) -> Self { + assert_eq!( + v.data().child_data()[0].child_data().len(), + 0, + "StringArray can only be created from list array of u8 values \ + (i.e. List>)." + ); + assert_eq!( + v.data_ref().child_data()[0].data_type(), + &DataType::UInt8, + "StringArray can only be created from List arrays, mismatched data types." + ); + + let mut builder = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(v.len()) + .add_buffer(v.data_ref().buffers()[0].clone()) + .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()); + if let Some(bitmap) = v.data().null_bitmap() { + builder = builder + .null_count(v.data_ref().null_count()) + .null_bit_buffer(bitmap.bits.clone()) + } + + let data = builder.build(); + Self::from(data) + } + + pub(crate) fn from_vec(v: Vec<&str>) -> Self { + let mut offsets = Vec::with_capacity(v.len() + 1); + let mut values = Vec::new(); + let mut length_so_far = OffsetSize::zero(); + offsets.push(length_so_far); + for s in &v { + length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); + offsets.push(length_so_far); + values.extend_from_slice(s.as_bytes()); + } + let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(v.len()) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + Self::from(array_data) + } + + pub(crate) fn from_opt_vec(v: Vec>) -> Self { + GenericStringArray::from_iter(v.into_iter()) + } +} + +impl<'a, Ptr, OffsetSize: StringOffsetSizeTrait> FromIterator> + for GenericStringArray +where + Ptr: AsRef, +{ + fn from_iter>>(iter: I) -> Self { + let iter = iter.into_iter(); + let (_, data_len) = iter.size_hint(); + let data_len = data_len.expect("Iterator must be sized"); // panic if no upper bound. + + let mut offsets = Vec::with_capacity(data_len + 1); + let mut values = Vec::new(); + let mut null_buf = MutableBuffer::new_null(data_len); + let mut length_so_far = OffsetSize::zero(); + offsets.push(length_so_far); + + for (i, s) in iter.enumerate() { + if let Some(s) = s { + let s = s.as_ref(); + // set null bit + let null_slice = null_buf.data_mut(); + bit_util::set_bit(null_slice, i); + + length_so_far = length_so_far + OffsetSize::from_usize(s.len()).unwrap(); + offsets.push(length_so_far); + values.extend_from_slice(s.as_bytes()); + } else { + offsets.push(length_so_far); + values.extend_from_slice(b""); + } + } + + let array_data = ArrayData::builder(OffsetSize::DATA_TYPE) + .len(data_len) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .null_bit_buffer(null_buf.freeze()) + .build(); + Self::from(array_data) + } +} + +impl<'a, T: StringOffsetSizeTrait> IntoIterator for &'a GenericStringArray { + type Item = Option<&'a str>; + type IntoIter = GenericStringIter<'a, T>; + + fn into_iter(self) -> Self::IntoIter { + GenericStringIter::<'a, T>::new(self) + } +} + +impl<'a, T: StringOffsetSizeTrait> GenericStringArray { + /// constructs a new iterator + pub fn iter(&'a self) -> GenericStringIter<'a, T> { + GenericStringIter::<'a, T>::new(&self) + } +} + +impl fmt::Debug for GenericStringArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}StringArray\n[\n", OffsetSize::prefix())?; + print_long_array(self, f, |array, index, f| { + fmt::Debug::fmt(&array.value(index), f) + })?; + write!(f, "]") + } +} + +impl Array for GenericStringArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [$name]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [$name]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +impl From + for GenericStringArray +{ + fn from(data: ArrayDataRef) -> Self { + assert_eq!( + data.data_type(), + &::DATA_TYPE, + "[Large]StringArray expects Datatype::[Large]Utf8" + ); + assert_eq!( + data.buffers().len(), + 2, + "StringArray data should contain 2 buffers only (offsets and values)" + ); + let raw_value_offsets = data.buffers()[0].raw_data(); + let value_data = data.buffers()[1].raw_data(); + Self { + data, + value_offsets: RawPtrBox::new(as_aligned_pointer::( + raw_value_offsets, + )), + value_data: RawPtrBox::new(value_data), + } + } +} + +/// An array where each element is a variable-sized sequence of bytes representing a string +/// whose maximum length (in bytes) is represented by a i32. +pub type StringArray = GenericStringArray; + +/// An array where each element is a variable-sized sequence of bytes representing a string +/// whose maximum length (in bytes) is represented by a i64. +pub type LargeStringArray = GenericStringArray; + +impl From for StringArray { + fn from(v: ListArray) -> Self { + StringArray::from_list(v) + } +} + +impl From for LargeStringArray { + fn from(v: LargeListArray) -> Self { + LargeStringArray::from_list(v) + } +} + +impl From> for StringArray { + fn from(v: Vec<&str>) -> Self { + StringArray::from_vec(v) + } +} + +impl From> for LargeStringArray { + fn from(v: Vec<&str>) -> Self { + LargeStringArray::from_vec(v) + } +} + +impl From>> for StringArray { + fn from(v: Vec>) -> Self { + StringArray::from_opt_vec(v) + } +} + +impl From>> for LargeStringArray { + fn from(v: Vec>) -> Self { + LargeStringArray::from_opt_vec(v) + } +} + +#[cfg(test)] +mod tests { + use crate::array::{ListBuilder, StringBuilder}; + + use super::*; + + #[test] + fn test_string_array_from_u8_slice() { + let values: Vec<&str> = vec!["hello", "", "parquet"]; + + // Array data: ["hello", "", "parquet"] + let string_array = StringArray::from(values); + + assert_eq!(3, string_array.len()); + assert_eq!(0, string_array.null_count()); + assert_eq!("hello", string_array.value(0)); + assert_eq!("", string_array.value(1)); + assert_eq!("parquet", string_array.value(2)); + assert_eq!(5, string_array.value_offset(2)); + assert_eq!(7, string_array.value_length(2)); + for i in 0..3 { + assert!(string_array.is_valid(i)); + assert!(!string_array.is_null(i)); + } + } + + #[test] + #[should_panic(expected = "[Large]StringArray expects Datatype::[Large]Utf8")] + fn test_string_array_from_int() { + let array = LargeStringArray::from(vec!["a", "b"]); + StringArray::from(array.data()); + } + + #[test] + fn test_large_string_array_from_u8_slice() { + let values: Vec<&str> = vec!["hello", "", "parquet"]; + + // Array data: ["hello", "", "parquet"] + let string_array = LargeStringArray::from(values); + + assert_eq!(3, string_array.len()); + assert_eq!(0, string_array.null_count()); + assert_eq!("hello", string_array.value(0)); + assert_eq!("", string_array.value(1)); + assert_eq!("parquet", string_array.value(2)); + assert_eq!(5, string_array.value_offset(2)); + assert_eq!(7, string_array.value_length(2)); + for i in 0..3 { + assert!(string_array.is_valid(i)); + assert!(!string_array.is_null(i)); + } + } + + #[test] + fn test_nested_string_array() { + let string_builder = StringBuilder::new(3); + let mut list_of_string_builder = ListBuilder::new(string_builder); + + list_of_string_builder.values().append_value("foo").unwrap(); + list_of_string_builder.values().append_value("bar").unwrap(); + list_of_string_builder.append(true).unwrap(); + + list_of_string_builder + .values() + .append_value("foobar") + .unwrap(); + list_of_string_builder.append(true).unwrap(); + let list_of_strings = list_of_string_builder.finish(); + + assert_eq!(list_of_strings.len(), 2); + + let first_slot = list_of_strings.value(0); + let first_list = first_slot.as_any().downcast_ref::().unwrap(); + assert_eq!(first_list.len(), 2); + assert_eq!(first_list.value(0), "foo"); + assert_eq!(first_list.value(1), "bar"); + + let second_slot = list_of_strings.value(1); + let second_list = second_slot.as_any().downcast_ref::().unwrap(); + assert_eq!(second_list.len(), 1); + assert_eq!(second_list.value(0), "foobar"); + } + + #[test] + #[should_panic(expected = "StringArray out of bounds access")] + fn test_string_array_get_value_index_out_of_bound() { + let values: [u8; 12] = [ + b'h', b'e', b'l', b'l', b'o', b'p', b'a', b'r', b'q', b'u', b'e', b't', + ]; + let offsets: [i32; 4] = [0, 5, 5, 12]; + let array_data = ArrayData::builder(DataType::Utf8) + .len(3) + .add_buffer(Buffer::from(offsets.to_byte_slice())) + .add_buffer(Buffer::from(&values[..])) + .build(); + let string_array = StringArray::from(array_data); + string_array.value(4); + } + + #[test] + fn test_string_array_fmt_debug() { + let arr: StringArray = vec!["hello", "arrow"].into(); + assert_eq!( + "StringArray\n[\n \"hello\",\n \"arrow\",\n]", + format!("{:?}", arr) + ); + } + + #[test] + fn test_large_string_array_fmt_debug() { + let arr: LargeStringArray = vec!["hello", "arrow"].into(); + assert_eq!( + "LargeStringArray\n[\n \"hello\",\n \"arrow\",\n]", + format!("{:?}", arr) + ); + } + + fn test_string_array_from_iter() { + let data = vec![Some("hello"), None, Some("arrow")]; + // from Vec> + let array1 = StringArray::from(data.clone()); + // from Iterator> + let array2: StringArray = data.clone().into_iter().collect(); + // from Iterator> + let array3: StringArray = data + .into_iter() + .map(|x| x.map(|s| format!("{}", s))) + .collect(); + + assert_eq!(array1, array2); + assert_eq!(array2, array3); + } +} diff --git a/rust/arrow/src/array/array_struct.rs b/rust/arrow/src/array/array_struct.rs new file mode 100644 index 00000000000..7f190b83008 --- /dev/null +++ b/rust/arrow/src/array/array_struct.rs @@ -0,0 +1,535 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::convert::{From, TryFrom}; +use std::fmt; +use std::iter::IntoIterator; +use std::mem; +use std::{any::Any, sync::Arc}; + +use super::{make_array, Array, ArrayData, ArrayDataRef, ArrayRef}; +use crate::datatypes::DataType; +use crate::{ + buffer::{buffer_bin_or, Buffer}, + datatypes::Field, +}; +use crate::{ + error::{ArrowError, Result}, + util::bit_util, +}; + +/// A nested array type where each child (called *field*) is represented by a separate +/// array. +pub struct StructArray { + data: ArrayDataRef, + pub(crate) boxed_fields: Vec, +} + +impl StructArray { + /// Returns the field at `pos`. + pub fn column(&self, pos: usize) -> &ArrayRef { + &self.boxed_fields[pos] + } + + /// Return the number of fields in this struct array + pub fn num_columns(&self) -> usize { + self.boxed_fields.len() + } + + /// Returns the fields of the struct array + pub fn columns(&self) -> Vec<&ArrayRef> { + self.boxed_fields.iter().collect() + } + + /// Returns child array refs of the struct array + pub fn columns_ref(&self) -> Vec { + self.boxed_fields.clone() + } + + /// Return field names in this struct array + pub fn column_names(&self) -> Vec<&str> { + match self.data.data_type() { + DataType::Struct(fields) => fields + .iter() + .map(|f| f.name().as_str()) + .collect::>(), + _ => unreachable!("Struct array's data type is not struct!"), + } + } + + /// Return child array whose field name equals to column_name + pub fn column_by_name(&self, column_name: &str) -> Option<&ArrayRef> { + self.column_names() + .iter() + .position(|c| c == &column_name) + .map(|pos| self.column(pos)) + } +} + +impl From for StructArray { + fn from(data: ArrayDataRef) -> Self { + let mut boxed_fields = vec![]; + for cd in data.child_data() { + let child_data = if data.offset() != 0 || data.len() != cd.len() { + Arc::new(cd.slice(data.offset(), data.len())) + } else { + cd.clone() + }; + boxed_fields.push(make_array(child_data)); + } + Self { data, boxed_fields } + } +} + +impl TryFrom> for StructArray { + type Error = ArrowError; + + /// builds a StructArray from a vector of names and arrays. + /// This errors if the values have a different length. + /// An entry is set to Null when all values are null. + fn try_from(values: Vec<(&str, ArrayRef)>) -> Result { + let values_len = values.len(); + + // these will be populated + let mut fields = Vec::with_capacity(values_len); + let mut child_data = Vec::with_capacity(values_len); + + // len: the size of the arrays. + let mut len: Option = None; + // null: the null mask of the arrays. + let mut null: Option = None; + for (field_name, array) in values { + let child_datum = array.data(); + let child_datum_len = child_datum.len(); + if let Some(len) = len { + if len != child_datum_len { + return Err(ArrowError::InvalidArgumentError( + format!("Array of field \"{}\" has length {}, but previous elements have length {}. + All arrays in every entry in a struct array must have the same length.", field_name, child_datum_len, len) + )); + } + } else { + len = Some(child_datum_len) + } + child_data.push(child_datum.clone()); + fields.push(Field::new( + field_name, + array.data_type().clone(), + child_datum.null_buffer().is_some(), + )); + + if let Some(child_null_buffer) = child_datum.null_buffer() { + null = Some(if let Some(null_buffer) = &null { + buffer_bin_or(null_buffer, 0, child_null_buffer, 0, child_datum_len) + } else { + child_null_buffer.clone() + }); + } else if null.is_some() { + // when one of the fields has no nulls, them there is no null in the array + null = None; + } + } + let len = len.unwrap(); + + let mut builder = ArrayData::builder(DataType::Struct(fields)) + .len(len) + .child_data(child_data); + if let Some(null_buffer) = null { + let null_count = len - bit_util::count_set_bits(null_buffer.data()); + builder = builder.null_count(null_count).null_bit_buffer(null_buffer); + } + + Ok(StructArray::from(builder.build())) + } +} + +impl Array for StructArray { + fn as_any(&self) -> &Any { + self + } + + fn data(&self) -> ArrayDataRef { + self.data.clone() + } + + fn data_ref(&self) -> &ArrayDataRef { + &self.data + } + + /// Returns the length (i.e., number of elements) of this array + fn len(&self) -> usize { + self.data_ref().len() + } + + /// Returns the total number of bytes of memory occupied by the buffers owned by this [StructArray]. + fn get_buffer_memory_size(&self) -> usize { + self.data.get_buffer_memory_size() + } + + /// Returns the total number of bytes of memory occupied physically by this [StructArray]. + fn get_array_memory_size(&self) -> usize { + self.data.get_array_memory_size() + mem::size_of_val(self) + } +} + +impl From> for StructArray { + fn from(v: Vec<(Field, ArrayRef)>) -> Self { + let (field_types, field_values): (Vec<_>, Vec<_>) = v.into_iter().unzip(); + + // Check the length of the child arrays + let length = field_values[0].len(); + for i in 1..field_values.len() { + assert_eq!( + length, + field_values[i].len(), + "all child arrays of a StructArray must have the same length" + ); + assert_eq!( + field_types[i].data_type(), + field_values[i].data().data_type(), + "the field data types must match the array data in a StructArray" + ) + } + + let data = ArrayData::builder(DataType::Struct(field_types)) + .child_data(field_values.into_iter().map(|a| a.data()).collect()) + .len(length) + .build(); + Self::from(data) + } +} + +impl fmt::Debug for StructArray { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "StructArray\n[\n")?; + for (child_index, name) in self.column_names().iter().enumerate() { + let column = self.column(child_index); + writeln!( + f, + "-- child {}: \"{}\" ({:?})", + child_index, + name, + column.data_type() + )?; + fmt::Debug::fmt(column, f)?; + writeln!(f)?; + } + write!(f, "]") + } +} + +impl From<(Vec<(Field, ArrayRef)>, Buffer, usize)> for StructArray { + fn from(triple: (Vec<(Field, ArrayRef)>, Buffer, usize)) -> Self { + let (field_types, field_values): (Vec<_>, Vec<_>) = triple.0.into_iter().unzip(); + + // Check the length of the child arrays + let length = field_values[0].len(); + for i in 1..field_values.len() { + assert_eq!( + length, + field_values[i].len(), + "all child arrays of a StructArray must have the same length" + ); + assert_eq!( + field_types[i].data_type(), + field_values[i].data().data_type(), + "the field data types must match the array data in a StructArray" + ) + } + + let data = ArrayData::builder(DataType::Struct(field_types)) + .null_bit_buffer(triple.1) + .child_data(field_values.into_iter().map(|a| a.data()).collect()) + .len(length) + .null_count(triple.2) + .build(); + Self::from(data) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use std::sync::Arc; + + use crate::datatypes::{DataType, Field}; + use crate::{ + array::BooleanArray, array::Float32Array, array::Float64Array, array::Int32Array, + array::StringArray, bitmap::Bitmap, + }; + use crate::{buffer::Buffer, datatypes::ToByteSlice}; + + #[test] + fn test_struct_array_builder() { + let boolean_data = ArrayData::builder(DataType::Boolean) + .len(4) + .add_buffer(Buffer::from([false, false, true, true].to_byte_slice())) + .build(); + let int_data = ArrayData::builder(DataType::Int64) + .len(4) + .add_buffer(Buffer::from([42, 28, 19, 31].to_byte_slice())) + .build(); + let mut field_types = vec![]; + field_types.push(Field::new("a", DataType::Boolean, false)); + field_types.push(Field::new("b", DataType::Int64, false)); + let struct_array_data = ArrayData::builder(DataType::Struct(field_types)) + .len(4) + .add_child_data(boolean_data.clone()) + .add_child_data(int_data.clone()) + .build(); + let struct_array = StructArray::from(struct_array_data); + + assert_eq!(boolean_data, struct_array.column(0).data()); + assert_eq!(int_data, struct_array.column(1).data()); + } + + #[test] + fn test_struct_array_from() { + let boolean_data = ArrayData::builder(DataType::Boolean) + .len(4) + .add_buffer(Buffer::from([12_u8])) + .build(); + let int_data = ArrayData::builder(DataType::Int32) + .len(4) + .add_buffer(Buffer::from([42, 28, 19, 31].to_byte_slice())) + .build(); + let struct_array = StructArray::from(vec![ + ( + Field::new("b", DataType::Boolean, false), + Arc::new(BooleanArray::from(vec![false, false, true, true])) + as Arc, + ), + ( + Field::new("c", DataType::Int32, false), + Arc::new(Int32Array::from(vec![42, 28, 19, 31])), + ), + ]); + assert_eq!(boolean_data, struct_array.column(0).data()); + assert_eq!(int_data, struct_array.column(1).data()); + assert_eq!(4, struct_array.len()); + assert_eq!(0, struct_array.null_count()); + assert_eq!(0, struct_array.offset()); + } + + /// validates that the in-memory representation follows [the spec](https://arrow.apache.org/docs/format/Columnar.html#struct-layout) + #[test] + fn test_struct_array_from_vec() { + let strings: ArrayRef = Arc::new(StringArray::from(vec![ + Some("joe"), + None, + None, + Some("mark"), + ])); + let ints: ArrayRef = + Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])); + + let arr = + StructArray::try_from(vec![("f1", strings.clone()), ("f2", ints.clone())]) + .unwrap(); + + let struct_data = arr.data(); + assert_eq!(4, struct_data.len()); + assert_eq!(1, struct_data.null_count()); + assert_eq!( + // 00001011 + &Some(Bitmap::from(Buffer::from(&[11_u8]))), + struct_data.null_bitmap() + ); + + let expected_string_data = ArrayData::builder(DataType::Utf8) + .len(4) + .null_count(2) + .null_bit_buffer(Buffer::from(&[9_u8])) + .add_buffer(Buffer::from(&[0, 3, 3, 3, 7].to_byte_slice())) + .add_buffer(Buffer::from("joemark".as_bytes())) + .build(); + + let expected_int_data = ArrayData::builder(DataType::Int32) + .len(4) + .null_count(1) + .null_bit_buffer(Buffer::from(&[11_u8])) + .add_buffer(Buffer::from(&[1, 2, 0, 4].to_byte_slice())) + .build(); + + assert_eq!(expected_string_data, arr.column(0).data()); + + // TODO: implement equality for ArrayData + assert_eq!(expected_int_data.len(), arr.column(1).data().len()); + assert_eq!( + expected_int_data.null_count(), + arr.column(1).data().null_count() + ); + assert_eq!( + expected_int_data.null_bitmap(), + arr.column(1).data().null_bitmap() + ); + let expected_value_buf = expected_int_data.buffers()[0].clone(); + let actual_value_buf = arr.column(1).data().buffers()[0].clone(); + for i in 0..expected_int_data.len() { + if !expected_int_data.is_null(i) { + assert_eq!( + expected_value_buf.data()[i * 4..(i + 1) * 4], + actual_value_buf.data()[i * 4..(i + 1) * 4] + ); + } + } + } + + #[test] + fn test_struct_array_from_vec_error() { + let strings: ArrayRef = Arc::new(StringArray::from(vec![ + Some("joe"), + None, + None, + // 3 elements, not 4 + ])); + let ints: ArrayRef = + Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])); + + let arr = + StructArray::try_from(vec![("f1", strings.clone()), ("f2", ints.clone())]); + + match arr { + Err(ArrowError::InvalidArgumentError(e)) => { + assert!(e.starts_with("Array of field \"f2\" has length 4, but previous elements have length 3.")); + } + _ => assert!(false, "This test got an unexpected error type"), + }; + } + + #[test] + #[should_panic( + expected = "the field data types must match the array data in a StructArray" + )] + fn test_struct_array_from_mismatched_types() { + StructArray::from(vec![ + ( + Field::new("b", DataType::Int16, false), + Arc::new(BooleanArray::from(vec![false, false, true, true])) + as Arc, + ), + ( + Field::new("c", DataType::Utf8, false), + Arc::new(Int32Array::from(vec![42, 28, 19, 31])), + ), + ]); + } + + #[test] + fn test_struct_array_slice() { + let boolean_data = ArrayData::builder(DataType::Boolean) + .len(5) + .add_buffer(Buffer::from([0b00010000])) + .null_bit_buffer(Buffer::from([0b00010001])) + .build(); + let int_data = ArrayData::builder(DataType::Int32) + .len(5) + .add_buffer(Buffer::from([0, 28, 42, 0, 0].to_byte_slice())) + .null_bit_buffer(Buffer::from([0b00000110])) + .build(); + + let mut field_types = vec![]; + field_types.push(Field::new("a", DataType::Boolean, false)); + field_types.push(Field::new("b", DataType::Int32, false)); + let struct_array_data = ArrayData::builder(DataType::Struct(field_types)) + .len(5) + .add_child_data(boolean_data.clone()) + .add_child_data(int_data.clone()) + .null_bit_buffer(Buffer::from([0b00010111])) + .build(); + let struct_array = StructArray::from(struct_array_data); + + assert_eq!(5, struct_array.len()); + assert_eq!(1, struct_array.null_count()); + assert!(struct_array.is_valid(0)); + assert!(struct_array.is_valid(1)); + assert!(struct_array.is_valid(2)); + assert!(struct_array.is_null(3)); + assert!(struct_array.is_valid(4)); + assert_eq!(boolean_data, struct_array.column(0).data()); + assert_eq!(int_data, struct_array.column(1).data()); + + let c0 = struct_array.column(0); + let c0 = c0.as_any().downcast_ref::().unwrap(); + assert_eq!(5, c0.len()); + assert_eq!(3, c0.null_count()); + assert!(c0.is_valid(0)); + assert_eq!(false, c0.value(0)); + assert!(c0.is_null(1)); + assert!(c0.is_null(2)); + assert!(c0.is_null(3)); + assert!(c0.is_valid(4)); + assert_eq!(true, c0.value(4)); + + let c1 = struct_array.column(1); + let c1 = c1.as_any().downcast_ref::().unwrap(); + assert_eq!(5, c1.len()); + assert_eq!(3, c1.null_count()); + assert!(c1.is_null(0)); + assert!(c1.is_valid(1)); + assert_eq!(28, c1.value(1)); + assert!(c1.is_valid(2)); + assert_eq!(42, c1.value(2)); + assert!(c1.is_null(3)); + assert!(c1.is_null(4)); + + let sliced_array = struct_array.slice(2, 3); + let sliced_array = sliced_array.as_any().downcast_ref::().unwrap(); + assert_eq!(3, sliced_array.len()); + assert_eq!(2, sliced_array.offset()); + assert_eq!(1, sliced_array.null_count()); + assert!(sliced_array.is_valid(0)); + assert!(sliced_array.is_null(1)); + assert!(sliced_array.is_valid(2)); + + let sliced_c0 = sliced_array.column(0); + let sliced_c0 = sliced_c0.as_any().downcast_ref::().unwrap(); + assert_eq!(3, sliced_c0.len()); + assert_eq!(2, sliced_c0.offset()); + assert!(sliced_c0.is_null(0)); + assert!(sliced_c0.is_null(1)); + assert!(sliced_c0.is_valid(2)); + assert_eq!(true, sliced_c0.value(2)); + + let sliced_c1 = sliced_array.column(1); + let sliced_c1 = sliced_c1.as_any().downcast_ref::().unwrap(); + assert_eq!(3, sliced_c1.len()); + assert_eq!(2, sliced_c1.offset()); + assert!(sliced_c1.is_valid(0)); + assert_eq!(42, sliced_c1.value(0)); + assert!(sliced_c1.is_null(1)); + assert!(sliced_c1.is_null(2)); + } + + #[test] + #[should_panic( + expected = "all child arrays of a StructArray must have the same length" + )] + fn test_invalid_struct_child_array_lengths() { + StructArray::from(vec![ + ( + Field::new("b", DataType::Float32, false), + Arc::new(Float32Array::from(vec![1.1])) as Arc, + ), + ( + Field::new("c", DataType::Float64, false), + Arc::new(Float64Array::from(vec![2.2, 3.3])), + ), + ]); + } +} diff --git a/rust/arrow/src/array/equal/mod.rs b/rust/arrow/src/array/equal/mod.rs index 400a6e24cf7..ab063754051 100644 --- a/rust/arrow/src/array/equal/mod.rs +++ b/rust/arrow/src/array/equal/mod.rs @@ -20,9 +20,9 @@ //! depend on dynamic casting of `Array`. use super::{ - array::BinaryOffsetSizeTrait, Array, ArrayData, FixedSizeBinaryArray, - GenericBinaryArray, GenericListArray, GenericStringArray, OffsetSizeTrait, - PrimitiveArray, StringOffsetSizeTrait, StructArray, + Array, ArrayData, BinaryOffsetSizeTrait, FixedSizeBinaryArray, GenericBinaryArray, + GenericListArray, GenericStringArray, OffsetSizeTrait, PrimitiveArray, + StringOffsetSizeTrait, StructArray, }; use crate::datatypes::{ArrowPrimitiveType, DataType, IntervalUnit}; @@ -216,7 +216,7 @@ mod tests { use std::sync::Arc; use crate::array::{ - array::Array, array::BinaryOffsetSizeTrait, ArrayDataRef, ArrayRef, BooleanArray, + array::Array, ArrayDataRef, ArrayRef, BinaryOffsetSizeTrait, BooleanArray, FixedSizeBinaryBuilder, FixedSizeListBuilder, GenericBinaryArray, Int32Builder, ListBuilder, NullArray, PrimitiveBuilder, StringArray, StringDictionaryBuilder, StringOffsetSizeTrait, StructArray, diff --git a/rust/arrow/src/array/equal_json.rs b/rust/arrow/src/array/equal_json.rs index d29b84529b6..42c0964c68b 100644 --- a/rust/arrow/src/array/equal_json.rs +++ b/rust/arrow/src/array/equal_json.rs @@ -17,10 +17,7 @@ use super::*; use crate::datatypes::*; -use array::{ - Array, BinaryOffsetSizeTrait, GenericBinaryArray, GenericListArray, - GenericStringArray, OffsetSizeTrait, StringOffsetSizeTrait, -}; +use array::Array; use hex::FromHex; use serde_json::value::Value::{Null as JNull, Object, String as JString}; use serde_json::Value; diff --git a/rust/arrow/src/array/iterator.rs b/rust/arrow/src/array/iterator.rs index 8b3aba84e9c..d93f86d9614 100644 --- a/rust/arrow/src/array/iterator.rs +++ b/rust/arrow/src/array/iterator.rs @@ -18,8 +18,8 @@ use crate::datatypes::ArrowPrimitiveType; use super::{ - array::BinaryOffsetSizeTrait, Array, GenericBinaryArray, GenericStringArray, - PrimitiveArray, StringOffsetSizeTrait, + Array, BinaryOffsetSizeTrait, GenericBinaryArray, GenericStringArray, PrimitiveArray, + StringOffsetSizeTrait, }; /// an iterator that returns Some(T) or None, that can be used on any non-boolean PrimitiveArray diff --git a/rust/arrow/src/array/mod.rs b/rust/arrow/src/array/mod.rs index 0c96b7948ed..bbe454c9036 100644 --- a/rust/arrow/src/array/mod.rs +++ b/rust/arrow/src/array/mod.rs @@ -83,6 +83,12 @@ #[allow(clippy::module_inception)] mod array; +mod array_binary; +mod array_dictionary; +mod array_list; +mod array_primitive; +mod array_string; +mod array_struct; mod builder; mod cast; mod data; @@ -91,6 +97,7 @@ mod equal_json; mod iterator; mod null; mod ord; +mod raw_pointer; mod union; use crate::datatypes::*; @@ -103,17 +110,17 @@ pub use self::data::ArrayData; pub use self::data::ArrayDataBuilder; pub use self::data::ArrayDataRef; -pub use self::array::BinaryArray; -pub use self::array::DictionaryArray; -pub use self::array::FixedSizeBinaryArray; -pub use self::array::FixedSizeListArray; -pub use self::array::LargeBinaryArray; -pub use self::array::LargeListArray; -pub use self::array::LargeStringArray; -pub use self::array::ListArray; -pub use self::array::PrimitiveArray; -pub use self::array::StringArray; -pub use self::array::StructArray; +pub use self::array_binary::BinaryArray; +pub use self::array_binary::FixedSizeBinaryArray; +pub use self::array_binary::LargeBinaryArray; +pub use self::array_dictionary::DictionaryArray; +pub use self::array_list::FixedSizeListArray; +pub use self::array_list::LargeListArray; +pub use self::array_list::ListArray; +pub use self::array_primitive::PrimitiveArray; +pub use self::array_string::LargeStringArray; +pub use self::array_string::StringArray; +pub use self::array_struct::StructArray; pub use self::null::NullArray; pub use self::union::UnionArray; @@ -157,11 +164,12 @@ pub type DurationMillisecondArray = PrimitiveArray; pub type DurationMicrosecondArray = PrimitiveArray; pub type DurationNanosecondArray = PrimitiveArray; -pub use self::array::GenericBinaryArray; -pub use self::array::GenericListArray; -pub use self::array::GenericStringArray; -pub use self::array::OffsetSizeTrait; -pub use self::array::StringOffsetSizeTrait; +pub use self::array_binary::BinaryOffsetSizeTrait; +pub use self::array_binary::GenericBinaryArray; +pub use self::array_list::GenericListArray; +pub use self::array_list::OffsetSizeTrait; +pub use self::array_string::GenericStringArray; +pub use self::array_string::StringOffsetSizeTrait; // --------------------- Array Builder --------------------- diff --git a/rust/arrow/src/array/raw_pointer.rs b/rust/arrow/src/array/raw_pointer.rs new file mode 100644 index 00000000000..8eeadfe9390 --- /dev/null +++ b/rust/arrow/src/array/raw_pointer.rs @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::memory; + +pub(super) struct RawPtrBox { + inner: *const T, +} + +impl RawPtrBox { + pub(super) fn new(inner: *const T) -> Self { + Self { inner } + } + + pub(super) fn get(&self) -> *const T { + self.inner + } +} + +unsafe impl Send for RawPtrBox {} +unsafe impl Sync for RawPtrBox {} + +pub(super) fn as_aligned_pointer(p: *const u8) -> *const T { + assert!( + memory::is_aligned(p, std::mem::align_of::()), + "memory is not aligned" + ); + p as *const T +} diff --git a/rust/arrow/src/buffer.rs b/rust/arrow/src/buffer.rs index 3512ae34261..5b51a2f5203 100644 --- a/rust/arrow/src/buffer.rs +++ b/rust/arrow/src/buffer.rs @@ -617,6 +617,12 @@ impl MutableBuffer { } } + /// creates a new [MutableBuffer] where every bit is initialized to `0` + pub fn new_null(len: usize) -> Self { + let num_bytes = bit_util::ceil(len, 8); + MutableBuffer::new(num_bytes).with_bitset(num_bytes, false) + } + /// Set the bits in the range of `[0, end)` to 0 (if `val` is false), or 1 (if `val` /// is true). Also extend the length of this buffer to be `end`. ///