From f41fb1c833c4d89dd1d11fb08200bbe36722b2ca Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Sat, 30 Jul 2022 22:06:58 +0800 Subject: [PATCH] Move `FixedSizeBinaryArray` to `array_fixed_size_binary.rs` (#2218) * split Signed-off-by: remzi <13716567376yh@gmail.com> * rename Signed-off-by: remzi <13716567376yh@gmail.com> --- arrow/src/array/array_binary.rs | 610 ++------------------- arrow/src/array/array_fixed_size_binary.rs | 579 +++++++++++++++++++ arrow/src/array/mod.rs | 3 +- 3 files changed, 614 insertions(+), 578 deletions(-) create mode 100644 arrow/src/array/array_fixed_size_binary.rs diff --git a/arrow/src/array/array_binary.rs b/arrow/src/array/array_binary.rs index 4848a25a058d..5e0651078bde 100644 --- a/arrow/src/array/array_binary.rs +++ b/arrow/src/array/array_binary.rs @@ -20,12 +20,11 @@ use std::fmt; use std::{any::Any, iter::FromIterator}; use super::{ - array::print_long_array, raw_pointer::RawPtrBox, Array, ArrayData, - FixedSizeListArray, GenericBinaryIter, GenericListArray, OffsetSizeTrait, + array::print_long_array, raw_pointer::RawPtrBox, Array, ArrayData, GenericBinaryIter, + GenericListArray, OffsetSizeTrait, }; use crate::array::array::ArrayAccessor; use crate::buffer::Buffer; -use crate::error::{ArrowError, Result}; use crate::util::bit_util; use crate::{buffer::MutableBuffer, datatypes::DataType}; @@ -287,6 +286,26 @@ impl From> for Array } } +impl From>> + for GenericBinaryArray +{ + fn from(v: Vec>) -> Self { + Self::from_opt_vec(v) + } +} + +impl From> for GenericBinaryArray { + fn from(v: Vec<&[u8]>) -> Self { + Self::from_iter_values(v) + } +} + +impl From> for GenericBinaryArray { + fn from(v: GenericListArray) -> Self { + Self::from_list(v) + } +} + impl FromIterator> for GenericBinaryArray where @@ -330,6 +349,15 @@ where } } +impl<'a, T: OffsetSizeTrait> 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) + } +} + /// An array where each element contains 0 or more bytes. /// The byte length of each element is represented by an i32. /// @@ -410,368 +438,14 @@ pub type BinaryArray = GenericBinaryArray; /// pub type LargeBinaryArray = GenericBinaryArray; -impl<'a, T: OffsetSizeTrait> 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 GenericBinaryArray -{ - fn from(v: Vec>) -> Self { - Self::from_opt_vec(v) - } -} - -impl From> for GenericBinaryArray { - fn from(v: Vec<&[u8]>) -> Self { - Self::from_iter_values(v) - } -} - -impl From> for GenericBinaryArray { - fn from(v: GenericListArray) -> Self { - Self::from_list(v) - } -} - -/// An array where each element is a fixed-size sequence of bytes. -/// -/// # Examples -/// -/// Create an array from an iterable argument of byte slices. -/// -/// ``` -/// use arrow::array::{Array, FixedSizeBinaryArray}; -/// let input_arg = vec![ vec![1, 2], vec![3, 4], vec![5, 6] ]; -/// let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); -/// -/// assert_eq!(3, arr.len()); -/// -/// ``` -/// Create an array from an iterable argument of sparse byte slices. -/// Sparsity means that the input argument can contain `None` items. -/// ``` -/// use arrow::array::{Array, FixedSizeBinaryArray}; -/// let input_arg = vec![ None, Some(vec![7, 8]), Some(vec![9, 10]), None, Some(vec![13, 14]) ]; -/// let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); -/// assert_eq!(5, arr.len()) -/// -/// ``` -/// -pub struct FixedSizeBinaryArray { - data: ArrayData, - 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 + self.data.offset(); - unsafe { - let pos = self.value_offset_at(offset); - std::slice::from_raw_parts( - self.value_data.as_ptr().offset(pos as isize), - (self.value_offset_at(offset + 1) - pos) as usize, - ) - } - } - - /// Returns the element at index `i` as a byte slice. - /// # Safety - /// Caller is responsible for ensuring that the index is within the bounds of the array - pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { - let offset = i + self.data.offset(); - let pos = self.value_offset_at(offset); - std::slice::from_raw_parts( - self.value_data.as_ptr().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() - } - - /// Create an array from an iterable argument of sparse byte slices. - /// Sparsity means that items returned by the iterator are optional, i.e input argument can - /// contain `None` items. - /// - /// # Examples - /// - /// ``` - /// use arrow::array::FixedSizeBinaryArray; - /// let input_arg = vec![ - /// None, - /// Some(vec![7, 8]), - /// Some(vec![9, 10]), - /// None, - /// Some(vec![13, 14]), - /// None, - /// ]; - /// let array = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); - /// ``` - /// - /// # Errors - /// - /// Returns error if argument has length zero, or sizes of nested slices don't match. - pub fn try_from_sparse_iter(mut iter: T) -> Result - where - T: Iterator>, - U: AsRef<[u8]>, - { - let mut len = 0; - let mut size = None; - let mut byte = 0; - let mut null_buf = MutableBuffer::from_len_zeroed(0); - let mut buffer = MutableBuffer::from_len_zeroed(0); - let mut prepend = 0; - iter.try_for_each(|item| -> Result<()> { - // extend null bitmask by one byte per each 8 items - if byte == 0 { - null_buf.push(0u8); - byte = 8; - } - byte -= 1; - - if let Some(slice) = item { - let slice = slice.as_ref(); - if let Some(size) = size { - if size != slice.len() { - return Err(ArrowError::InvalidArgumentError(format!( - "Nested array size mismatch: one is {}, and the other is {}", - size, - slice.len() - ))); - } - } else { - size = Some(slice.len()); - buffer.extend_zeros(slice.len() * prepend); - } - bit_util::set_bit(null_buf.as_slice_mut(), len); - buffer.extend_from_slice(slice); - } else if let Some(size) = size { - buffer.extend_zeros(size); - } else { - prepend += 1; - } - - len += 1; - - Ok(()) - })?; - - if len == 0 { - return Err(ArrowError::InvalidArgumentError( - "Input iterable argument has no data".to_owned(), - )); - } - - let size = size.unwrap_or(0); - let array_data = unsafe { - ArrayData::new_unchecked( - DataType::FixedSizeBinary(size as i32), - len, - None, - Some(null_buf.into()), - 0, - vec![buffer.into()], - vec![], - ) - }; - Ok(FixedSizeBinaryArray::from(array_data)) - } - - /// Create an array from an iterable argument of byte slices. - /// - /// # Examples - /// - /// ``` - /// use arrow::array::FixedSizeBinaryArray; - /// let input_arg = vec![ - /// vec![1, 2], - /// vec![3, 4], - /// vec![5, 6], - /// ]; - /// let array = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); - /// ``` - /// - /// # Errors - /// - /// Returns error if argument has length zero, or sizes of nested slices don't match. - pub fn try_from_iter(mut iter: T) -> Result - where - T: Iterator, - U: AsRef<[u8]>, - { - let mut len = 0; - let mut size = None; - let mut buffer = MutableBuffer::from_len_zeroed(0); - iter.try_for_each(|item| -> Result<()> { - let slice = item.as_ref(); - if let Some(size) = size { - if size != slice.len() { - return Err(ArrowError::InvalidArgumentError(format!( - "Nested array size mismatch: one is {}, and the other is {}", - size, - slice.len() - ))); - } - } else { - size = Some(slice.len()); - } - buffer.extend_from_slice(slice); - - len += 1; - - Ok(()) - })?; - - if len == 0 { - return Err(ArrowError::InvalidArgumentError( - "Input iterable argument has no data".to_owned(), - )); - } - - let size = size.unwrap_or(0); - let array_data = ArrayData::builder(DataType::FixedSizeBinary(size as i32)) - .len(len) - .add_buffer(buffer.into()); - let array_data = unsafe { array_data.build_unchecked() }; - Ok(FixedSizeBinaryArray::from(array_data)) - } - - #[inline] - fn value_offset_at(&self, i: usize) -> i32 { - self.length * i as i32 - } -} - -impl From for FixedSizeBinaryArray { - fn from(data: ArrayData) -> Self { - assert_eq!( - data.buffers().len(), - 1, - "FixedSizeBinaryArray data should contain 1 buffer only (values)" - ); - let value_data = data.buffers()[0].as_ptr(); - let length = match data.data_type() { - DataType::FixedSizeBinary(len) => *len, - _ => panic!("Expected data type to be FixedSizeBinary"), - }; - Self { - data, - value_data: unsafe { RawPtrBox::new(value_data) }, - length, - } - } -} - -impl From for ArrayData { - fn from(array: FixedSizeBinaryArray) -> Self { - array.data - } -} - -/// 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 builder = ArrayData::builder(DataType::FixedSizeBinary(v.value_length())) - .len(v.len()) - .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()) - .null_bit_buffer(v.data_ref().null_buffer().cloned()); - - let data = unsafe { builder.build_unchecked() }; - Self::from(data) - } -} - -impl From>> for FixedSizeBinaryArray { - fn from(v: Vec>) -> Self { - Self::try_from_sparse_iter(v.into_iter()).unwrap() - } -} - -impl From> for FixedSizeBinaryArray { - fn from(v: Vec<&[u8]>) -> Self { - Self::try_from_iter(v.into_iter()).unwrap() - } -} - -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) -> &dyn Any { - self - } - - fn data(&self) -> &ArrayData { - &self.data - } - - fn into_data(self) -> ArrayData { - self.into() - } -} - #[cfg(test)] mod tests { - use std::sync::Arc; - + use super::*; use crate::{ array::{LargeListArray, ListArray}, - datatypes::{Field, Schema}, - record_batch::RecordBatch, + datatypes::Field, }; - use super::*; - #[test] fn test_binary_array() { let values: [u8; 12] = [ @@ -1074,87 +748,6 @@ mod tests { drop(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() - .unwrap(); - 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() - .unwrap(); - 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 FixedSizeList arrays" - )] - // Different error messages, so skip for now - // https://github.com/apache/arrow-rs/issues/1545 - #[cfg(not(feature = "force_validate"))] - 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_slice_ref(&values)) - .build() - .unwrap(); - - let array_data = unsafe { - ArrayData::builder(DataType::FixedSizeList( - Box::new(Field::new("item", DataType::Binary, false)), - 4, - )) - .len(3) - .add_child_data(values_data) - .build_unchecked() - }; - let list_array = FixedSizeListArray::from(array_data); - drop(FixedSizeBinaryArray::from(list_array)); - } - #[test] #[should_panic(expected = "BinaryArray out of bounds access")] fn test_binary_array_get_value_index_out_of_bound() { @@ -1171,114 +764,6 @@ mod tests { 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() - .unwrap(); - 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_fixed_size_binary_array_from_iter() { - let input_arg = vec![vec![1, 2], vec![3, 4], vec![5, 6]]; - let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); - - assert_eq!(2, arr.value_length()); - assert_eq!(3, arr.len()) - } - - #[test] - fn test_all_none_fixed_size_binary_array_from_sparse_iter() { - let none_option: Option<[u8; 32]> = None; - let input_arg = vec![none_option, none_option, none_option]; - let arr = - FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); - assert_eq!(0, arr.value_length()); - assert_eq!(3, arr.len()) - } - - #[test] - fn test_fixed_size_binary_array_from_sparse_iter() { - let input_arg = vec![ - None, - Some(vec![7, 8]), - Some(vec![9, 10]), - None, - Some(vec![13, 14]), - ]; - let arr = - FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); - assert_eq!(2, arr.value_length()); - assert_eq!(5, arr.len()) - } - - #[test] - fn test_fixed_size_binary_array_from_vec() { - let values = vec!["one".as_bytes(), b"two", b"six", b"ten"]; - let array = FixedSizeBinaryArray::from(values); - assert_eq!(array.len(), 4); - assert_eq!(array.null_count(), 0); - assert_eq!(array.value(0), b"one"); - assert_eq!(array.value(1), b"two"); - assert_eq!(array.value(2), b"six"); - assert_eq!(array.value(3), b"ten"); - assert!(!array.is_null(0)); - assert!(!array.is_null(1)); - assert!(!array.is_null(2)); - assert!(!array.is_null(3)); - } - - #[test] - #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")] - fn test_fixed_size_binary_array_from_vec_incorrect_length() { - let values = vec!["one".as_bytes(), b"two", b"three", b"four"]; - let _ = FixedSizeBinaryArray::from(values); - } - - #[test] - fn test_fixed_size_binary_array_from_opt_vec() { - let values = vec![ - Some("one".as_bytes()), - Some(b"two"), - None, - Some(b"six"), - Some(b"ten"), - ]; - let array = FixedSizeBinaryArray::from(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"six"); - assert_eq!(array.value(4), b"ten"); - assert!(!array.is_null(0)); - assert!(!array.is_null(1)); - assert!(array.is_null(2)); - assert!(!array.is_null(3)); - assert!(!array.is_null(4)); - } - - #[test] - #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")] - fn test_fixed_size_binary_array_from_opt_vec_incorrect_length() { - let values = vec![ - Some("one".as_bytes()), - Some(b"two"), - None, - Some(b"three"), - Some(b"four"), - ]; - let _ = FixedSizeBinaryArray::from(values); - } - #[test] fn test_binary_array_all_null() { let data = vec![None]; @@ -1298,33 +783,4 @@ mod tests { .validate_full() .expect("All null array has valid array data"); } - - #[test] - fn fixed_size_binary_array_all_null() { - let data = vec![None] as Vec>; - let array = FixedSizeBinaryArray::try_from_sparse_iter(data.into_iter()).unwrap(); - array - .data() - .validate_full() - .expect("All null array has valid array data"); - } - - #[test] - // Test for https://github.com/apache/arrow-rs/issues/1390 - #[should_panic( - expected = "column types must match schema types, expected FixedSizeBinary(2) but found FixedSizeBinary(0) at column index 0" - )] - fn fixed_size_binary_array_all_null_in_batch_with_schema() { - let schema = - Schema::new(vec![Field::new("a", DataType::FixedSizeBinary(2), true)]); - - let none_option: Option<[u8; 2]> = None; - let item = FixedSizeBinaryArray::try_from_sparse_iter( - vec![none_option, none_option, none_option].into_iter(), - ) - .unwrap(); - - // Should not panic - RecordBatch::try_new(Arc::new(schema), vec![Arc::new(item)]).unwrap(); - } } diff --git a/arrow/src/array/array_fixed_size_binary.rs b/arrow/src/array/array_fixed_size_binary.rs new file mode 100644 index 000000000000..e851fd3921b5 --- /dev/null +++ b/arrow/src/array/array_fixed_size_binary.rs @@ -0,0 +1,579 @@ +// 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 super::{ + array::print_long_array, raw_pointer::RawPtrBox, Array, ArrayData, FixedSizeListArray, +}; +use crate::buffer::Buffer; +use crate::error::{ArrowError, Result}; +use crate::util::bit_util; +use crate::{buffer::MutableBuffer, datatypes::DataType}; + +/// An array where each element is a fixed-size sequence of bytes. +/// +/// # Examples +/// +/// Create an array from an iterable argument of byte slices. +/// +/// ``` +/// use arrow::array::{Array, FixedSizeBinaryArray}; +/// let input_arg = vec![ vec![1, 2], vec![3, 4], vec![5, 6] ]; +/// let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); +/// +/// assert_eq!(3, arr.len()); +/// +/// ``` +/// Create an array from an iterable argument of sparse byte slices. +/// Sparsity means that the input argument can contain `None` items. +/// ``` +/// use arrow::array::{Array, FixedSizeBinaryArray}; +/// let input_arg = vec![ None, Some(vec![7, 8]), Some(vec![9, 10]), None, Some(vec![13, 14]) ]; +/// let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); +/// assert_eq!(5, arr.len()) +/// +/// ``` +/// +pub struct FixedSizeBinaryArray { + data: ArrayData, + 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 + self.data.offset(); + unsafe { + let pos = self.value_offset_at(offset); + std::slice::from_raw_parts( + self.value_data.as_ptr().offset(pos as isize), + (self.value_offset_at(offset + 1) - pos) as usize, + ) + } + } + + /// Returns the element at index `i` as a byte slice. + /// # Safety + /// Caller is responsible for ensuring that the index is within the bounds of the array + pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { + let offset = i + self.data.offset(); + let pos = self.value_offset_at(offset); + std::slice::from_raw_parts( + self.value_data.as_ptr().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() + } + + /// Create an array from an iterable argument of sparse byte slices. + /// Sparsity means that items returned by the iterator are optional, i.e input argument can + /// contain `None` items. + /// + /// # Examples + /// + /// ``` + /// use arrow::array::FixedSizeBinaryArray; + /// let input_arg = vec![ + /// None, + /// Some(vec![7, 8]), + /// Some(vec![9, 10]), + /// None, + /// Some(vec![13, 14]), + /// None, + /// ]; + /// let array = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); + /// ``` + /// + /// # Errors + /// + /// Returns error if argument has length zero, or sizes of nested slices don't match. + pub fn try_from_sparse_iter(mut iter: T) -> Result + where + T: Iterator>, + U: AsRef<[u8]>, + { + let mut len = 0; + let mut size = None; + let mut byte = 0; + let mut null_buf = MutableBuffer::from_len_zeroed(0); + let mut buffer = MutableBuffer::from_len_zeroed(0); + let mut prepend = 0; + iter.try_for_each(|item| -> Result<()> { + // extend null bitmask by one byte per each 8 items + if byte == 0 { + null_buf.push(0u8); + byte = 8; + } + byte -= 1; + + if let Some(slice) = item { + let slice = slice.as_ref(); + if let Some(size) = size { + if size != slice.len() { + return Err(ArrowError::InvalidArgumentError(format!( + "Nested array size mismatch: one is {}, and the other is {}", + size, + slice.len() + ))); + } + } else { + size = Some(slice.len()); + buffer.extend_zeros(slice.len() * prepend); + } + bit_util::set_bit(null_buf.as_slice_mut(), len); + buffer.extend_from_slice(slice); + } else if let Some(size) = size { + buffer.extend_zeros(size); + } else { + prepend += 1; + } + + len += 1; + + Ok(()) + })?; + + if len == 0 { + return Err(ArrowError::InvalidArgumentError( + "Input iterable argument has no data".to_owned(), + )); + } + + let size = size.unwrap_or(0); + let array_data = unsafe { + ArrayData::new_unchecked( + DataType::FixedSizeBinary(size as i32), + len, + None, + Some(null_buf.into()), + 0, + vec![buffer.into()], + vec![], + ) + }; + Ok(FixedSizeBinaryArray::from(array_data)) + } + + /// Create an array from an iterable argument of byte slices. + /// + /// # Examples + /// + /// ``` + /// use arrow::array::FixedSizeBinaryArray; + /// let input_arg = vec![ + /// vec![1, 2], + /// vec![3, 4], + /// vec![5, 6], + /// ]; + /// let array = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); + /// ``` + /// + /// # Errors + /// + /// Returns error if argument has length zero, or sizes of nested slices don't match. + pub fn try_from_iter(mut iter: T) -> Result + where + T: Iterator, + U: AsRef<[u8]>, + { + let mut len = 0; + let mut size = None; + let mut buffer = MutableBuffer::from_len_zeroed(0); + iter.try_for_each(|item| -> Result<()> { + let slice = item.as_ref(); + if let Some(size) = size { + if size != slice.len() { + return Err(ArrowError::InvalidArgumentError(format!( + "Nested array size mismatch: one is {}, and the other is {}", + size, + slice.len() + ))); + } + } else { + size = Some(slice.len()); + } + buffer.extend_from_slice(slice); + + len += 1; + + Ok(()) + })?; + + if len == 0 { + return Err(ArrowError::InvalidArgumentError( + "Input iterable argument has no data".to_owned(), + )); + } + + let size = size.unwrap_or(0); + let array_data = ArrayData::builder(DataType::FixedSizeBinary(size as i32)) + .len(len) + .add_buffer(buffer.into()); + let array_data = unsafe { array_data.build_unchecked() }; + Ok(FixedSizeBinaryArray::from(array_data)) + } + + #[inline] + fn value_offset_at(&self, i: usize) -> i32 { + self.length * i as i32 + } +} + +impl From for FixedSizeBinaryArray { + fn from(data: ArrayData) -> Self { + assert_eq!( + data.buffers().len(), + 1, + "FixedSizeBinaryArray data should contain 1 buffer only (values)" + ); + let value_data = data.buffers()[0].as_ptr(); + let length = match data.data_type() { + DataType::FixedSizeBinary(len) => *len, + _ => panic!("Expected data type to be FixedSizeBinary"), + }; + Self { + data, + value_data: unsafe { RawPtrBox::new(value_data) }, + length, + } + } +} + +impl From for ArrayData { + fn from(array: FixedSizeBinaryArray) -> Self { + array.data + } +} + +/// 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 builder = ArrayData::builder(DataType::FixedSizeBinary(v.value_length())) + .len(v.len()) + .add_buffer(v.data_ref().child_data()[0].buffers()[0].clone()) + .null_bit_buffer(v.data_ref().null_buffer().cloned()); + + let data = unsafe { builder.build_unchecked() }; + Self::from(data) + } +} + +impl From>> for FixedSizeBinaryArray { + fn from(v: Vec>) -> Self { + Self::try_from_sparse_iter(v.into_iter()).unwrap() + } +} + +impl From> for FixedSizeBinaryArray { + fn from(v: Vec<&[u8]>) -> Self { + Self::try_from_iter(v.into_iter()).unwrap() + } +} + +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) -> &dyn Any { + self + } + + fn data(&self) -> &ArrayData { + &self.data + } + + fn into_data(self) -> ArrayData { + self.into() + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use crate::{ + datatypes::{Field, Schema}, + record_batch::RecordBatch, + }; + + use super::*; + + #[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() + .unwrap(); + 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() + .unwrap(); + 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 FixedSizeList arrays" + )] + // Different error messages, so skip for now + // https://github.com/apache/arrow-rs/issues/1545 + #[cfg(not(feature = "force_validate"))] + 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_slice_ref(&values)) + .build() + .unwrap(); + + let array_data = unsafe { + ArrayData::builder(DataType::FixedSizeList( + Box::new(Field::new("item", DataType::Binary, false)), + 4, + )) + .len(3) + .add_child_data(values_data) + .build_unchecked() + }; + let list_array = FixedSizeListArray::from(array_data); + drop(FixedSizeBinaryArray::from(list_array)); + } + + #[test] + fn test_fixed_size_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() + .unwrap(); + 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_fixed_size_binary_array_from_iter() { + let input_arg = vec![vec![1, 2], vec![3, 4], vec![5, 6]]; + let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap(); + + assert_eq!(2, arr.value_length()); + assert_eq!(3, arr.len()) + } + + #[test] + fn test_all_none_fixed_size_binary_array_from_sparse_iter() { + let none_option: Option<[u8; 32]> = None; + let input_arg = vec![none_option, none_option, none_option]; + let arr = + FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); + assert_eq!(0, arr.value_length()); + assert_eq!(3, arr.len()) + } + + #[test] + fn test_fixed_size_binary_array_from_sparse_iter() { + let input_arg = vec![ + None, + Some(vec![7, 8]), + Some(vec![9, 10]), + None, + Some(vec![13, 14]), + ]; + let arr = + FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap(); + assert_eq!(2, arr.value_length()); + assert_eq!(5, arr.len()) + } + + #[test] + fn test_fixed_size_binary_array_from_vec() { + let values = vec!["one".as_bytes(), b"two", b"six", b"ten"]; + let array = FixedSizeBinaryArray::from(values); + assert_eq!(array.len(), 4); + assert_eq!(array.null_count(), 0); + assert_eq!(array.value(0), b"one"); + assert_eq!(array.value(1), b"two"); + assert_eq!(array.value(2), b"six"); + assert_eq!(array.value(3), b"ten"); + assert!(!array.is_null(0)); + assert!(!array.is_null(1)); + assert!(!array.is_null(2)); + assert!(!array.is_null(3)); + } + + #[test] + #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")] + fn test_fixed_size_binary_array_from_vec_incorrect_length() { + let values = vec!["one".as_bytes(), b"two", b"three", b"four"]; + let _ = FixedSizeBinaryArray::from(values); + } + + #[test] + fn test_fixed_size_binary_array_from_opt_vec() { + let values = vec![ + Some("one".as_bytes()), + Some(b"two"), + None, + Some(b"six"), + Some(b"ten"), + ]; + let array = FixedSizeBinaryArray::from(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"six"); + assert_eq!(array.value(4), b"ten"); + assert!(!array.is_null(0)); + assert!(!array.is_null(1)); + assert!(array.is_null(2)); + assert!(!array.is_null(3)); + assert!(!array.is_null(4)); + } + + #[test] + #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")] + fn test_fixed_size_binary_array_from_opt_vec_incorrect_length() { + let values = vec![ + Some("one".as_bytes()), + Some(b"two"), + None, + Some(b"three"), + Some(b"four"), + ]; + let _ = FixedSizeBinaryArray::from(values); + } + + #[test] + fn fixed_size_binary_array_all_null() { + let data = vec![None] as Vec>; + let array = FixedSizeBinaryArray::try_from_sparse_iter(data.into_iter()).unwrap(); + array + .data() + .validate_full() + .expect("All null array has valid array data"); + } + + #[test] + // Test for https://github.com/apache/arrow-rs/issues/1390 + #[should_panic( + expected = "column types must match schema types, expected FixedSizeBinary(2) but found FixedSizeBinary(0) at column index 0" + )] + fn fixed_size_binary_array_all_null_in_batch_with_schema() { + let schema = + Schema::new(vec![Field::new("a", DataType::FixedSizeBinary(2), true)]); + + let none_option: Option<[u8; 2]> = None; + let item = FixedSizeBinaryArray::try_from_sparse_iter( + vec![none_option, none_option, none_option].into_iter(), + ) + .unwrap(); + + // Should not panic + RecordBatch::try_new(Arc::new(schema), vec![Arc::new(item)]).unwrap(); + } +} diff --git a/arrow/src/array/mod.rs b/arrow/src/array/mod.rs index 8acc33c7b879..61eb4c70fc8e 100644 --- a/arrow/src/array/mod.rs +++ b/arrow/src/array/mod.rs @@ -163,6 +163,7 @@ mod array_binary; mod array_boolean; mod array_decimal; mod array_dictionary; +mod array_fixed_size_binary; mod array_list; mod array_map; mod array_primitive; @@ -195,12 +196,12 @@ pub use self::data::ArrayDataRef; pub(crate) use self::data::BufferSpec; pub use self::array_binary::BinaryArray; -pub use self::array_binary::FixedSizeBinaryArray; pub use self::array_binary::LargeBinaryArray; pub use self::array_boolean::BooleanArray; pub use self::array_decimal::BasicDecimalArray; pub use self::array_decimal::Decimal128Array; pub use self::array_decimal::Decimal256Array; +pub use self::array_fixed_size_binary::FixedSizeBinaryArray; #[deprecated(note = "Please use `Decimal128Array` instead")] pub type DecimalArray = Decimal128Array;