arrow_array/array/
fixed_size_binary_array.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use crate::array::print_long_array;
19use crate::iterator::FixedSizeBinaryIter;
20use crate::{Array, ArrayAccessor, ArrayRef, FixedSizeListArray, Scalar};
21use arrow_buffer::buffer::NullBuffer;
22use arrow_buffer::{bit_util, ArrowNativeType, BooleanBuffer, Buffer, MutableBuffer};
23use arrow_data::{ArrayData, ArrayDataBuilder};
24use arrow_schema::{ArrowError, DataType};
25use std::any::Any;
26use std::sync::Arc;
27
28/// An array of [fixed size binary arrays](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
29///
30/// # Examples
31///
32/// Create an array from an iterable argument of byte slices.
33///
34/// ```
35///    use arrow_array::{Array, FixedSizeBinaryArray};
36///    let input_arg = vec![ vec![1, 2], vec![3, 4], vec![5, 6] ];
37///    let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap();
38///
39///    assert_eq!(3, arr.len());
40///
41/// ```
42/// Create an array from an iterable argument of sparse byte slices.
43/// Sparsity means that the input argument can contain `None` items.
44/// ```
45///    use arrow_array::{Array, FixedSizeBinaryArray};
46///    let input_arg = vec![ None, Some(vec![7, 8]), Some(vec![9, 10]), None, Some(vec![13, 14]) ];
47///    let arr = FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 2).unwrap();
48///    assert_eq!(5, arr.len())
49///
50/// ```
51///
52#[derive(Clone)]
53pub struct FixedSizeBinaryArray {
54    data_type: DataType, // Must be DataType::FixedSizeBinary(value_length)
55    value_data: Buffer,
56    nulls: Option<NullBuffer>,
57    len: usize,
58    value_length: i32,
59}
60
61impl FixedSizeBinaryArray {
62    /// Create a new [`FixedSizeBinaryArray`] with `size` element size, panicking on failure
63    ///
64    /// # Panics
65    ///
66    /// Panics if [`Self::try_new`] returns an error
67    pub fn new(size: i32, values: Buffer, nulls: Option<NullBuffer>) -> Self {
68        Self::try_new(size, values, nulls).unwrap()
69    }
70
71    /// Create a new [`Scalar`] from `value`
72    pub fn new_scalar(value: impl AsRef<[u8]>) -> Scalar<Self> {
73        let v = value.as_ref();
74        Scalar::new(Self::new(v.len() as _, Buffer::from(v), None))
75    }
76
77    /// Create a new [`FixedSizeBinaryArray`] from the provided parts, returning an error on failure
78    ///
79    /// # Errors
80    ///
81    /// * `size < 0`
82    /// * `values.len() / size != nulls.len()`
83    pub fn try_new(
84        size: i32,
85        values: Buffer,
86        nulls: Option<NullBuffer>,
87    ) -> Result<Self, ArrowError> {
88        let data_type = DataType::FixedSizeBinary(size);
89        let s = size.to_usize().ok_or_else(|| {
90            ArrowError::InvalidArgumentError(format!("Size cannot be negative, got {}", size))
91        })?;
92
93        let len = values.len() / s;
94        if let Some(n) = nulls.as_ref() {
95            if n.len() != len {
96                return Err(ArrowError::InvalidArgumentError(format!(
97                    "Incorrect length of null buffer for FixedSizeBinaryArray, expected {} got {}",
98                    len,
99                    n.len(),
100                )));
101            }
102        }
103
104        Ok(Self {
105            data_type,
106            value_data: values,
107            value_length: size,
108            nulls,
109            len,
110        })
111    }
112
113    /// Create a new [`FixedSizeBinaryArray`] of length `len` where all values are null
114    ///
115    /// # Panics
116    ///
117    /// Panics if
118    ///
119    /// * `size < 0`
120    /// * `size * len` would overflow `usize`
121    pub fn new_null(size: i32, len: usize) -> Self {
122        let capacity = size.to_usize().unwrap().checked_mul(len).unwrap();
123        Self {
124            data_type: DataType::FixedSizeBinary(size),
125            value_data: MutableBuffer::new(capacity).into(),
126            nulls: Some(NullBuffer::new_null(len)),
127            value_length: size,
128            len,
129        }
130    }
131
132    /// Deconstruct this array into its constituent parts
133    pub fn into_parts(self) -> (i32, Buffer, Option<NullBuffer>) {
134        (self.value_length, self.value_data, self.nulls)
135    }
136
137    /// Returns the element at index `i` as a byte slice.
138    /// # Panics
139    /// Panics if index `i` is out of bounds.
140    pub fn value(&self, i: usize) -> &[u8] {
141        assert!(
142            i < self.len(),
143            "Trying to access an element at index {} from a FixedSizeBinaryArray of length {}",
144            i,
145            self.len()
146        );
147        let offset = i + self.offset();
148        unsafe {
149            let pos = self.value_offset_at(offset);
150            std::slice::from_raw_parts(
151                self.value_data.as_ptr().offset(pos as isize),
152                (self.value_offset_at(offset + 1) - pos) as usize,
153            )
154        }
155    }
156
157    /// Returns the element at index `i` as a byte slice.
158    /// # Safety
159    /// Caller is responsible for ensuring that the index is within the bounds of the array
160    pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] {
161        let offset = i + self.offset();
162        let pos = self.value_offset_at(offset);
163        std::slice::from_raw_parts(
164            self.value_data.as_ptr().offset(pos as isize),
165            (self.value_offset_at(offset + 1) - pos) as usize,
166        )
167    }
168
169    /// Returns the offset for the element at index `i`.
170    ///
171    /// Note this doesn't do any bound checking, for performance reason.
172    #[inline]
173    pub fn value_offset(&self, i: usize) -> i32 {
174        self.value_offset_at(self.offset() + i)
175    }
176
177    /// Returns the length for an element.
178    ///
179    /// All elements have the same length as the array is a fixed size.
180    #[inline]
181    pub fn value_length(&self) -> i32 {
182        self.value_length
183    }
184
185    /// Returns the values of this array.
186    ///
187    /// Unlike [`Self::value_data`] this returns the [`Buffer`]
188    /// allowing for zero-copy cloning.
189    #[inline]
190    pub fn values(&self) -> &Buffer {
191        &self.value_data
192    }
193
194    /// Returns the raw value data.
195    pub fn value_data(&self) -> &[u8] {
196        self.value_data.as_slice()
197    }
198
199    /// Returns a zero-copy slice of this array with the indicated offset and length.
200    pub fn slice(&self, offset: usize, len: usize) -> Self {
201        assert!(
202            offset.saturating_add(len) <= self.len,
203            "the length + offset of the sliced FixedSizeBinaryArray cannot exceed the existing length"
204        );
205
206        let size = self.value_length as usize;
207
208        Self {
209            data_type: self.data_type.clone(),
210            nulls: self.nulls.as_ref().map(|n| n.slice(offset, len)),
211            value_length: self.value_length,
212            value_data: self.value_data.slice_with_length(offset * size, len * size),
213            len,
214        }
215    }
216
217    /// Create an array from an iterable argument of sparse byte slices.
218    /// Sparsity means that items returned by the iterator are optional, i.e input argument can
219    /// contain `None` items.
220    ///
221    /// # Examples
222    ///
223    /// ```
224    /// use arrow_array::FixedSizeBinaryArray;
225    /// let input_arg = vec![
226    ///     None,
227    ///     Some(vec![7, 8]),
228    ///     Some(vec![9, 10]),
229    ///     None,
230    ///     Some(vec![13, 14]),
231    ///     None,
232    /// ];
233    /// let array = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap();
234    /// ```
235    ///
236    /// # Errors
237    ///
238    /// Returns error if argument has length zero, or sizes of nested slices don't match.
239    #[deprecated(
240        note = "This function will fail if the iterator produces only None values; prefer `try_from_sparse_iter_with_size`"
241    )]
242    pub fn try_from_sparse_iter<T, U>(mut iter: T) -> Result<Self, ArrowError>
243    where
244        T: Iterator<Item = Option<U>>,
245        U: AsRef<[u8]>,
246    {
247        let mut len = 0;
248        let mut size = None;
249        let mut byte = 0;
250
251        let iter_size_hint = iter.size_hint().0;
252        let mut null_buf = MutableBuffer::new(bit_util::ceil(iter_size_hint, 8));
253        let mut buffer = MutableBuffer::new(0);
254
255        let mut prepend = 0;
256        iter.try_for_each(|item| -> Result<(), ArrowError> {
257            // extend null bitmask by one byte per each 8 items
258            if byte == 0 {
259                null_buf.push(0u8);
260                byte = 8;
261            }
262            byte -= 1;
263
264            if let Some(slice) = item {
265                let slice = slice.as_ref();
266                if let Some(size) = size {
267                    if size != slice.len() {
268                        return Err(ArrowError::InvalidArgumentError(format!(
269                            "Nested array size mismatch: one is {}, and the other is {}",
270                            size,
271                            slice.len()
272                        )));
273                    }
274                } else {
275                    let len = slice.len();
276                    size = Some(len);
277                    // Now that we know how large each element is we can reserve
278                    // sufficient capacity in the underlying mutable buffer for
279                    // the data.
280                    buffer.reserve(iter_size_hint * len);
281                    buffer.extend_zeros(slice.len() * prepend);
282                }
283                bit_util::set_bit(null_buf.as_slice_mut(), len);
284                buffer.extend_from_slice(slice);
285            } else if let Some(size) = size {
286                buffer.extend_zeros(size);
287            } else {
288                prepend += 1;
289            }
290
291            len += 1;
292
293            Ok(())
294        })?;
295
296        if len == 0 {
297            return Err(ArrowError::InvalidArgumentError(
298                "Input iterable argument has no data".to_owned(),
299            ));
300        }
301
302        let null_buf = BooleanBuffer::new(null_buf.into(), 0, len);
303        let nulls = Some(NullBuffer::new(null_buf)).filter(|n| n.null_count() > 0);
304
305        let size = size.unwrap_or(0) as i32;
306        Ok(Self {
307            data_type: DataType::FixedSizeBinary(size),
308            value_data: buffer.into(),
309            nulls,
310            value_length: size,
311            len,
312        })
313    }
314
315    /// Create an array from an iterable argument of sparse byte slices.
316    /// Sparsity means that items returned by the iterator are optional, i.e input argument can
317    /// contain `None` items. In cases where the iterator returns only `None` values, this
318    /// also takes a size parameter to ensure that the a valid FixedSizeBinaryArray is still
319    /// created.
320    ///
321    /// # Examples
322    ///
323    /// ```
324    /// use arrow_array::FixedSizeBinaryArray;
325    /// let input_arg = vec![
326    ///     None,
327    ///     Some(vec![7, 8]),
328    ///     Some(vec![9, 10]),
329    ///     None,
330    ///     Some(vec![13, 14]),
331    ///     None,
332    /// ];
333    /// let array = FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 2).unwrap();
334    /// ```
335    ///
336    /// # Errors
337    ///
338    /// Returns error if argument has length zero, or sizes of nested slices don't match.
339    pub fn try_from_sparse_iter_with_size<T, U>(mut iter: T, size: i32) -> Result<Self, ArrowError>
340    where
341        T: Iterator<Item = Option<U>>,
342        U: AsRef<[u8]>,
343    {
344        let mut len = 0;
345        let mut byte = 0;
346
347        let iter_size_hint = iter.size_hint().0;
348        let mut null_buf = MutableBuffer::new(bit_util::ceil(iter_size_hint, 8));
349        let mut buffer = MutableBuffer::new(iter_size_hint * (size as usize));
350
351        iter.try_for_each(|item| -> Result<(), ArrowError> {
352            // extend null bitmask by one byte per each 8 items
353            if byte == 0 {
354                null_buf.push(0u8);
355                byte = 8;
356            }
357            byte -= 1;
358
359            if let Some(slice) = item {
360                let slice = slice.as_ref();
361                if size as usize != slice.len() {
362                    return Err(ArrowError::InvalidArgumentError(format!(
363                        "Nested array size mismatch: one is {}, and the other is {}",
364                        size,
365                        slice.len()
366                    )));
367                }
368
369                bit_util::set_bit(null_buf.as_slice_mut(), len);
370                buffer.extend_from_slice(slice);
371            } else {
372                buffer.extend_zeros(size as usize);
373            }
374
375            len += 1;
376
377            Ok(())
378        })?;
379
380        let null_buf = BooleanBuffer::new(null_buf.into(), 0, len);
381        let nulls = Some(NullBuffer::new(null_buf)).filter(|n| n.null_count() > 0);
382
383        Ok(Self {
384            data_type: DataType::FixedSizeBinary(size),
385            value_data: buffer.into(),
386            nulls,
387            len,
388            value_length: size,
389        })
390    }
391
392    /// Create an array from an iterable argument of byte slices.
393    ///
394    /// # Examples
395    ///
396    /// ```
397    /// use arrow_array::FixedSizeBinaryArray;
398    /// let input_arg = vec![
399    ///     vec![1, 2],
400    ///     vec![3, 4],
401    ///     vec![5, 6],
402    /// ];
403    /// let array = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap();
404    /// ```
405    ///
406    /// # Errors
407    ///
408    /// Returns error if argument has length zero, or sizes of nested slices don't match.
409    pub fn try_from_iter<T, U>(mut iter: T) -> Result<Self, ArrowError>
410    where
411        T: Iterator<Item = U>,
412        U: AsRef<[u8]>,
413    {
414        let mut len = 0;
415        let mut size = None;
416        let iter_size_hint = iter.size_hint().0;
417        let mut buffer = MutableBuffer::new(0);
418
419        iter.try_for_each(|item| -> Result<(), ArrowError> {
420            let slice = item.as_ref();
421            if let Some(size) = size {
422                if size != slice.len() {
423                    return Err(ArrowError::InvalidArgumentError(format!(
424                        "Nested array size mismatch: one is {}, and the other is {}",
425                        size,
426                        slice.len()
427                    )));
428                }
429            } else {
430                let len = slice.len();
431                size = Some(len);
432                buffer.reserve(iter_size_hint * len);
433            }
434
435            buffer.extend_from_slice(slice);
436
437            len += 1;
438
439            Ok(())
440        })?;
441
442        if len == 0 {
443            return Err(ArrowError::InvalidArgumentError(
444                "Input iterable argument has no data".to_owned(),
445            ));
446        }
447
448        let size = size.unwrap_or(0).try_into().unwrap();
449        Ok(Self {
450            data_type: DataType::FixedSizeBinary(size),
451            value_data: buffer.into(),
452            nulls: None,
453            value_length: size,
454            len,
455        })
456    }
457
458    #[inline]
459    fn value_offset_at(&self, i: usize) -> i32 {
460        self.value_length * i as i32
461    }
462
463    /// constructs a new iterator
464    pub fn iter(&self) -> FixedSizeBinaryIter<'_> {
465        FixedSizeBinaryIter::new(self)
466    }
467}
468
469impl From<ArrayData> for FixedSizeBinaryArray {
470    fn from(data: ArrayData) -> Self {
471        assert_eq!(
472            data.buffers().len(),
473            1,
474            "FixedSizeBinaryArray data should contain 1 buffer only (values)"
475        );
476        let value_length = match data.data_type() {
477            DataType::FixedSizeBinary(len) => *len,
478            _ => panic!("Expected data type to be FixedSizeBinary"),
479        };
480
481        let size = value_length as usize;
482        let value_data =
483            data.buffers()[0].slice_with_length(data.offset() * size, data.len() * size);
484
485        Self {
486            data_type: data.data_type().clone(),
487            nulls: data.nulls().cloned(),
488            len: data.len(),
489            value_data,
490            value_length,
491        }
492    }
493}
494
495impl From<FixedSizeBinaryArray> for ArrayData {
496    fn from(array: FixedSizeBinaryArray) -> Self {
497        let builder = ArrayDataBuilder::new(array.data_type)
498            .len(array.len)
499            .buffers(vec![array.value_data])
500            .nulls(array.nulls);
501
502        unsafe { builder.build_unchecked() }
503    }
504}
505
506/// Creates a `FixedSizeBinaryArray` from `FixedSizeList<u8>` array
507impl From<FixedSizeListArray> for FixedSizeBinaryArray {
508    fn from(v: FixedSizeListArray) -> Self {
509        let value_len = v.value_length();
510        let v = v.into_data();
511        assert_eq!(
512            v.child_data().len(),
513            1,
514            "FixedSizeBinaryArray can only be created from list array of u8 values \
515             (i.e. FixedSizeList<PrimitiveArray<u8>>)."
516        );
517        let child_data = &v.child_data()[0];
518
519        assert_eq!(
520            child_data.child_data().len(),
521            0,
522            "FixedSizeBinaryArray can only be created from list array of u8 values \
523             (i.e. FixedSizeList<PrimitiveArray<u8>>)."
524        );
525        assert_eq!(
526            child_data.data_type(),
527            &DataType::UInt8,
528            "FixedSizeBinaryArray can only be created from FixedSizeList<u8> arrays, mismatched data types."
529        );
530        assert_eq!(
531            child_data.null_count(),
532            0,
533            "The child array cannot contain null values."
534        );
535
536        let builder = ArrayData::builder(DataType::FixedSizeBinary(value_len))
537            .len(v.len())
538            .offset(v.offset())
539            .add_buffer(child_data.buffers()[0].slice(child_data.offset()))
540            .nulls(v.nulls().cloned());
541
542        let data = unsafe { builder.build_unchecked() };
543        Self::from(data)
544    }
545}
546
547impl From<Vec<Option<&[u8]>>> for FixedSizeBinaryArray {
548    fn from(v: Vec<Option<&[u8]>>) -> Self {
549        #[allow(deprecated)]
550        Self::try_from_sparse_iter(v.into_iter()).unwrap()
551    }
552}
553
554impl From<Vec<&[u8]>> for FixedSizeBinaryArray {
555    fn from(v: Vec<&[u8]>) -> Self {
556        Self::try_from_iter(v.into_iter()).unwrap()
557    }
558}
559
560impl<const N: usize> From<Vec<&[u8; N]>> for FixedSizeBinaryArray {
561    fn from(v: Vec<&[u8; N]>) -> Self {
562        Self::try_from_iter(v.into_iter()).unwrap()
563    }
564}
565
566impl std::fmt::Debug for FixedSizeBinaryArray {
567    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
568        write!(f, "FixedSizeBinaryArray<{}>\n[\n", self.value_length())?;
569        print_long_array(self, f, |array, index, f| {
570            std::fmt::Debug::fmt(&array.value(index), f)
571        })?;
572        write!(f, "]")
573    }
574}
575
576impl Array for FixedSizeBinaryArray {
577    fn as_any(&self) -> &dyn Any {
578        self
579    }
580
581    fn to_data(&self) -> ArrayData {
582        self.clone().into()
583    }
584
585    fn into_data(self) -> ArrayData {
586        self.into()
587    }
588
589    fn data_type(&self) -> &DataType {
590        &self.data_type
591    }
592
593    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
594        Arc::new(self.slice(offset, length))
595    }
596
597    fn len(&self) -> usize {
598        self.len
599    }
600
601    fn is_empty(&self) -> bool {
602        self.len == 0
603    }
604
605    fn offset(&self) -> usize {
606        0
607    }
608
609    fn nulls(&self) -> Option<&NullBuffer> {
610        self.nulls.as_ref()
611    }
612
613    fn logical_null_count(&self) -> usize {
614        // More efficient that the default implementation
615        self.null_count()
616    }
617
618    fn get_buffer_memory_size(&self) -> usize {
619        let mut sum = self.value_data.capacity();
620        if let Some(n) = &self.nulls {
621            sum += n.buffer().capacity();
622        }
623        sum
624    }
625
626    fn get_array_memory_size(&self) -> usize {
627        std::mem::size_of::<Self>() + self.get_buffer_memory_size()
628    }
629}
630
631impl<'a> ArrayAccessor for &'a FixedSizeBinaryArray {
632    type Item = &'a [u8];
633
634    fn value(&self, index: usize) -> Self::Item {
635        FixedSizeBinaryArray::value(self, index)
636    }
637
638    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
639        FixedSizeBinaryArray::value_unchecked(self, index)
640    }
641}
642
643impl<'a> IntoIterator for &'a FixedSizeBinaryArray {
644    type Item = Option<&'a [u8]>;
645    type IntoIter = FixedSizeBinaryIter<'a>;
646
647    fn into_iter(self) -> Self::IntoIter {
648        FixedSizeBinaryIter::<'a>::new(self)
649    }
650}
651
652#[cfg(test)]
653mod tests {
654    use crate::RecordBatch;
655    use arrow_schema::{Field, Schema};
656
657    use super::*;
658
659    #[test]
660    fn test_fixed_size_binary_array() {
661        let values: [u8; 15] = *b"hellotherearrow";
662
663        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
664            .len(3)
665            .add_buffer(Buffer::from(&values[..]))
666            .build()
667            .unwrap();
668        let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data);
669        assert_eq!(3, fixed_size_binary_array.len());
670        assert_eq!(0, fixed_size_binary_array.null_count());
671        assert_eq!(
672            [b'h', b'e', b'l', b'l', b'o'],
673            fixed_size_binary_array.value(0)
674        );
675        assert_eq!(
676            [b't', b'h', b'e', b'r', b'e'],
677            fixed_size_binary_array.value(1)
678        );
679        assert_eq!(
680            [b'a', b'r', b'r', b'o', b'w'],
681            fixed_size_binary_array.value(2)
682        );
683        assert_eq!(5, fixed_size_binary_array.value_length());
684        assert_eq!(10, fixed_size_binary_array.value_offset(2));
685        for i in 0..3 {
686            assert!(fixed_size_binary_array.is_valid(i));
687            assert!(!fixed_size_binary_array.is_null(i));
688        }
689
690        // Test binary array with offset
691        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
692            .len(2)
693            .offset(1)
694            .add_buffer(Buffer::from(&values[..]))
695            .build()
696            .unwrap();
697        let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data);
698        assert_eq!(
699            [b't', b'h', b'e', b'r', b'e'],
700            fixed_size_binary_array.value(0)
701        );
702        assert_eq!(
703            [b'a', b'r', b'r', b'o', b'w'],
704            fixed_size_binary_array.value(1)
705        );
706        assert_eq!(2, fixed_size_binary_array.len());
707        assert_eq!(0, fixed_size_binary_array.value_offset(0));
708        assert_eq!(5, fixed_size_binary_array.value_length());
709        assert_eq!(5, fixed_size_binary_array.value_offset(1));
710    }
711
712    #[test]
713    fn test_fixed_size_binary_array_from_fixed_size_list_array() {
714        let values = [0_u8, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13];
715        let values_data = ArrayData::builder(DataType::UInt8)
716            .len(12)
717            .offset(2)
718            .add_buffer(Buffer::from_slice_ref(values))
719            .build()
720            .unwrap();
721        // [null, [10, 11, 12, 13]]
722        let array_data = unsafe {
723            ArrayData::builder(DataType::FixedSizeList(
724                Arc::new(Field::new("item", DataType::UInt8, false)),
725                4,
726            ))
727            .len(2)
728            .offset(1)
729            .add_child_data(values_data)
730            .null_bit_buffer(Some(Buffer::from_slice_ref([0b101])))
731            .build_unchecked()
732        };
733        let list_array = FixedSizeListArray::from(array_data);
734        let binary_array = FixedSizeBinaryArray::from(list_array);
735
736        assert_eq!(2, binary_array.len());
737        assert_eq!(1, binary_array.null_count());
738        assert!(binary_array.is_null(0));
739        assert!(binary_array.is_valid(1));
740        assert_eq!(&[10, 11, 12, 13], binary_array.value(1));
741    }
742
743    #[test]
744    #[should_panic(
745        expected = "FixedSizeBinaryArray can only be created from FixedSizeList<u8> arrays"
746    )]
747    // Different error messages, so skip for now
748    // https://github.com/apache/arrow-rs/issues/1545
749    #[cfg(not(feature = "force_validate"))]
750    fn test_fixed_size_binary_array_from_incorrect_fixed_size_list_array() {
751        let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11];
752        let values_data = ArrayData::builder(DataType::UInt32)
753            .len(12)
754            .add_buffer(Buffer::from_slice_ref(values))
755            .build()
756            .unwrap();
757
758        let array_data = unsafe {
759            ArrayData::builder(DataType::FixedSizeList(
760                Arc::new(Field::new("item", DataType::Binary, false)),
761                4,
762            ))
763            .len(3)
764            .add_child_data(values_data)
765            .build_unchecked()
766        };
767        let list_array = FixedSizeListArray::from(array_data);
768        drop(FixedSizeBinaryArray::from(list_array));
769    }
770
771    #[test]
772    #[should_panic(expected = "The child array cannot contain null values.")]
773    fn test_fixed_size_binary_array_from_fixed_size_list_array_with_child_nulls_failed() {
774        let values = [0_u8, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11];
775        let values_data = ArrayData::builder(DataType::UInt8)
776            .len(12)
777            .add_buffer(Buffer::from_slice_ref(values))
778            .null_bit_buffer(Some(Buffer::from_slice_ref([0b101010101010])))
779            .build()
780            .unwrap();
781
782        let array_data = unsafe {
783            ArrayData::builder(DataType::FixedSizeList(
784                Arc::new(Field::new("item", DataType::UInt8, false)),
785                4,
786            ))
787            .len(3)
788            .add_child_data(values_data)
789            .build_unchecked()
790        };
791        let list_array = FixedSizeListArray::from(array_data);
792        drop(FixedSizeBinaryArray::from(list_array));
793    }
794
795    #[test]
796    fn test_fixed_size_binary_array_fmt_debug() {
797        let values: [u8; 15] = *b"hellotherearrow";
798
799        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
800            .len(3)
801            .add_buffer(Buffer::from(&values[..]))
802            .build()
803            .unwrap();
804        let arr = FixedSizeBinaryArray::from(array_data);
805        assert_eq!(
806            "FixedSizeBinaryArray<5>\n[\n  [104, 101, 108, 108, 111],\n  [116, 104, 101, 114, 101],\n  [97, 114, 114, 111, 119],\n]",
807            format!("{arr:?}")
808        );
809    }
810
811    #[test]
812    fn test_fixed_size_binary_array_from_iter() {
813        let input_arg = vec![vec![1, 2], vec![3, 4], vec![5, 6]];
814        let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap();
815
816        assert_eq!(2, arr.value_length());
817        assert_eq!(3, arr.len())
818    }
819
820    #[test]
821    fn test_all_none_fixed_size_binary_array_from_sparse_iter() {
822        let none_option: Option<[u8; 32]> = None;
823        let input_arg = vec![none_option, none_option, none_option];
824        #[allow(deprecated)]
825        let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap();
826        assert_eq!(0, arr.value_length());
827        assert_eq!(3, arr.len())
828    }
829
830    #[test]
831    fn test_fixed_size_binary_array_from_sparse_iter() {
832        let input_arg = vec![
833            None,
834            Some(vec![7, 8]),
835            Some(vec![9, 10]),
836            None,
837            Some(vec![13, 14]),
838        ];
839        #[allow(deprecated)]
840        let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.iter().cloned()).unwrap();
841        assert_eq!(2, arr.value_length());
842        assert_eq!(5, arr.len());
843
844        let arr =
845            FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 2).unwrap();
846        assert_eq!(2, arr.value_length());
847        assert_eq!(5, arr.len());
848    }
849
850    #[test]
851    fn test_fixed_size_binary_array_from_sparse_iter_with_size_all_none() {
852        let input_arg = vec![None, None, None, None, None] as Vec<Option<Vec<u8>>>;
853
854        let arr = FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 16)
855            .unwrap();
856        assert_eq!(16, arr.value_length());
857        assert_eq!(5, arr.len())
858    }
859
860    #[test]
861    fn test_fixed_size_binary_array_from_vec() {
862        let values = vec!["one".as_bytes(), b"two", b"six", b"ten"];
863        let array = FixedSizeBinaryArray::from(values);
864        assert_eq!(array.len(), 4);
865        assert_eq!(array.null_count(), 0);
866        assert_eq!(array.logical_null_count(), 0);
867        assert_eq!(array.value(0), b"one");
868        assert_eq!(array.value(1), b"two");
869        assert_eq!(array.value(2), b"six");
870        assert_eq!(array.value(3), b"ten");
871        assert!(!array.is_null(0));
872        assert!(!array.is_null(1));
873        assert!(!array.is_null(2));
874        assert!(!array.is_null(3));
875    }
876
877    #[test]
878    #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")]
879    fn test_fixed_size_binary_array_from_vec_incorrect_length() {
880        let values = vec!["one".as_bytes(), b"two", b"three", b"four"];
881        let _ = FixedSizeBinaryArray::from(values);
882    }
883
884    #[test]
885    fn test_fixed_size_binary_array_from_opt_vec() {
886        let values = vec![
887            Some("one".as_bytes()),
888            Some(b"two"),
889            None,
890            Some(b"six"),
891            Some(b"ten"),
892        ];
893        let array = FixedSizeBinaryArray::from(values);
894        assert_eq!(array.len(), 5);
895        assert_eq!(array.value(0), b"one");
896        assert_eq!(array.value(1), b"two");
897        assert_eq!(array.value(3), b"six");
898        assert_eq!(array.value(4), b"ten");
899        assert!(!array.is_null(0));
900        assert!(!array.is_null(1));
901        assert!(array.is_null(2));
902        assert!(!array.is_null(3));
903        assert!(!array.is_null(4));
904    }
905
906    #[test]
907    #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")]
908    fn test_fixed_size_binary_array_from_opt_vec_incorrect_length() {
909        let values = vec![
910            Some("one".as_bytes()),
911            Some(b"two"),
912            None,
913            Some(b"three"),
914            Some(b"four"),
915        ];
916        let _ = FixedSizeBinaryArray::from(values);
917    }
918
919    #[test]
920    fn fixed_size_binary_array_all_null() {
921        let data = vec![None] as Vec<Option<String>>;
922        let array =
923            FixedSizeBinaryArray::try_from_sparse_iter_with_size(data.into_iter(), 0).unwrap();
924        array
925            .into_data()
926            .validate_full()
927            .expect("All null array has valid array data");
928    }
929
930    #[test]
931    // Test for https://github.com/apache/arrow-rs/issues/1390
932    fn fixed_size_binary_array_all_null_in_batch_with_schema() {
933        let schema = Schema::new(vec![Field::new("a", DataType::FixedSizeBinary(2), true)]);
934
935        let none_option: Option<[u8; 2]> = None;
936        let item = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
937            vec![none_option, none_option, none_option].into_iter(),
938            2,
939        )
940        .unwrap();
941
942        // Should not panic
943        RecordBatch::try_new(Arc::new(schema), vec![Arc::new(item)]).unwrap();
944    }
945
946    #[test]
947    #[should_panic(
948        expected = "Trying to access an element at index 4 from a FixedSizeBinaryArray of length 3"
949    )]
950    fn test_fixed_size_binary_array_get_value_index_out_of_bound() {
951        let values = vec![Some("one".as_bytes()), Some(b"two"), None];
952        let array = FixedSizeBinaryArray::from(values);
953
954        array.value(4);
955    }
956
957    #[test]
958    fn test_constructors() {
959        let buffer = Buffer::from_vec(vec![0_u8; 10]);
960        let a = FixedSizeBinaryArray::new(2, buffer.clone(), None);
961        assert_eq!(a.len(), 5);
962
963        let nulls = NullBuffer::new_null(5);
964        FixedSizeBinaryArray::new(2, buffer.clone(), Some(nulls));
965
966        let a = FixedSizeBinaryArray::new(3, buffer.clone(), None);
967        assert_eq!(a.len(), 3);
968
969        let nulls = NullBuffer::new_null(3);
970        FixedSizeBinaryArray::new(3, buffer.clone(), Some(nulls));
971
972        let err = FixedSizeBinaryArray::try_new(-1, buffer.clone(), None).unwrap_err();
973
974        assert_eq!(
975            err.to_string(),
976            "Invalid argument error: Size cannot be negative, got -1"
977        );
978
979        let nulls = NullBuffer::new_null(3);
980        let err = FixedSizeBinaryArray::try_new(2, buffer, Some(nulls)).unwrap_err();
981        assert_eq!(err.to_string(), "Invalid argument error: Incorrect length of null buffer for FixedSizeBinaryArray, expected 5 got 3");
982    }
983}