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        since = "28.0.0",
241        note = "This function will fail if the iterator produces only None values; prefer `try_from_sparse_iter_with_size`"
242    )]
243    pub fn try_from_sparse_iter<T, U>(mut iter: T) -> Result<Self, ArrowError>
244    where
245        T: Iterator<Item = Option<U>>,
246        U: AsRef<[u8]>,
247    {
248        let mut len = 0;
249        let mut size = None;
250        let mut byte = 0;
251
252        let iter_size_hint = iter.size_hint().0;
253        let mut null_buf = MutableBuffer::new(bit_util::ceil(iter_size_hint, 8));
254        let mut buffer = MutableBuffer::new(0);
255
256        let mut prepend = 0;
257        iter.try_for_each(|item| -> Result<(), ArrowError> {
258            // extend null bitmask by one byte per each 8 items
259            if byte == 0 {
260                null_buf.push(0u8);
261                byte = 8;
262            }
263            byte -= 1;
264
265            if let Some(slice) = item {
266                let slice = slice.as_ref();
267                if let Some(size) = size {
268                    if size != slice.len() {
269                        return Err(ArrowError::InvalidArgumentError(format!(
270                            "Nested array size mismatch: one is {}, and the other is {}",
271                            size,
272                            slice.len()
273                        )));
274                    }
275                } else {
276                    let len = slice.len();
277                    size = Some(len);
278                    // Now that we know how large each element is we can reserve
279                    // sufficient capacity in the underlying mutable buffer for
280                    // the data.
281                    buffer.reserve(iter_size_hint * len);
282                    buffer.extend_zeros(slice.len() * prepend);
283                }
284                bit_util::set_bit(null_buf.as_slice_mut(), len);
285                buffer.extend_from_slice(slice);
286            } else if let Some(size) = size {
287                buffer.extend_zeros(size);
288            } else {
289                prepend += 1;
290            }
291
292            len += 1;
293
294            Ok(())
295        })?;
296
297        if len == 0 {
298            return Err(ArrowError::InvalidArgumentError(
299                "Input iterable argument has no data".to_owned(),
300            ));
301        }
302
303        let null_buf = BooleanBuffer::new(null_buf.into(), 0, len);
304        let nulls = Some(NullBuffer::new(null_buf)).filter(|n| n.null_count() > 0);
305
306        let size = size.unwrap_or(0) as i32;
307        Ok(Self {
308            data_type: DataType::FixedSizeBinary(size),
309            value_data: buffer.into(),
310            nulls,
311            value_length: size,
312            len,
313        })
314    }
315
316    /// Create an array from an iterable argument of sparse byte slices.
317    /// Sparsity means that items returned by the iterator are optional, i.e input argument can
318    /// contain `None` items. In cases where the iterator returns only `None` values, this
319    /// also takes a size parameter to ensure that the a valid FixedSizeBinaryArray is still
320    /// created.
321    ///
322    /// # Examples
323    ///
324    /// ```
325    /// use arrow_array::FixedSizeBinaryArray;
326    /// let input_arg = vec![
327    ///     None,
328    ///     Some(vec![7, 8]),
329    ///     Some(vec![9, 10]),
330    ///     None,
331    ///     Some(vec![13, 14]),
332    ///     None,
333    /// ];
334    /// let array = FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 2).unwrap();
335    /// ```
336    ///
337    /// # Errors
338    ///
339    /// Returns error if argument has length zero, or sizes of nested slices don't match.
340    pub fn try_from_sparse_iter_with_size<T, U>(mut iter: T, size: i32) -> Result<Self, ArrowError>
341    where
342        T: Iterator<Item = Option<U>>,
343        U: AsRef<[u8]>,
344    {
345        let mut len = 0;
346        let mut byte = 0;
347
348        let iter_size_hint = iter.size_hint().0;
349        let mut null_buf = MutableBuffer::new(bit_util::ceil(iter_size_hint, 8));
350        let mut buffer = MutableBuffer::new(iter_size_hint * (size as usize));
351
352        iter.try_for_each(|item| -> Result<(), ArrowError> {
353            // extend null bitmask by one byte per each 8 items
354            if byte == 0 {
355                null_buf.push(0u8);
356                byte = 8;
357            }
358            byte -= 1;
359
360            if let Some(slice) = item {
361                let slice = slice.as_ref();
362                if size as usize != slice.len() {
363                    return Err(ArrowError::InvalidArgumentError(format!(
364                        "Nested array size mismatch: one is {}, and the other is {}",
365                        size,
366                        slice.len()
367                    )));
368                }
369
370                bit_util::set_bit(null_buf.as_slice_mut(), len);
371                buffer.extend_from_slice(slice);
372            } else {
373                buffer.extend_zeros(size as usize);
374            }
375
376            len += 1;
377
378            Ok(())
379        })?;
380
381        let null_buf = BooleanBuffer::new(null_buf.into(), 0, len);
382        let nulls = Some(NullBuffer::new(null_buf)).filter(|n| n.null_count() > 0);
383
384        Ok(Self {
385            data_type: DataType::FixedSizeBinary(size),
386            value_data: buffer.into(),
387            nulls,
388            len,
389            value_length: size,
390        })
391    }
392
393    /// Create an array from an iterable argument of byte slices.
394    ///
395    /// # Examples
396    ///
397    /// ```
398    /// use arrow_array::FixedSizeBinaryArray;
399    /// let input_arg = vec![
400    ///     vec![1, 2],
401    ///     vec![3, 4],
402    ///     vec![5, 6],
403    /// ];
404    /// let array = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap();
405    /// ```
406    ///
407    /// # Errors
408    ///
409    /// Returns error if argument has length zero, or sizes of nested slices don't match.
410    pub fn try_from_iter<T, U>(mut iter: T) -> Result<Self, ArrowError>
411    where
412        T: Iterator<Item = U>,
413        U: AsRef<[u8]>,
414    {
415        let mut len = 0;
416        let mut size = None;
417        let iter_size_hint = iter.size_hint().0;
418        let mut buffer = MutableBuffer::new(0);
419
420        iter.try_for_each(|item| -> Result<(), ArrowError> {
421            let slice = item.as_ref();
422            if let Some(size) = size {
423                if size != slice.len() {
424                    return Err(ArrowError::InvalidArgumentError(format!(
425                        "Nested array size mismatch: one is {}, and the other is {}",
426                        size,
427                        slice.len()
428                    )));
429                }
430            } else {
431                let len = slice.len();
432                size = Some(len);
433                buffer.reserve(iter_size_hint * len);
434            }
435
436            buffer.extend_from_slice(slice);
437
438            len += 1;
439
440            Ok(())
441        })?;
442
443        if len == 0 {
444            return Err(ArrowError::InvalidArgumentError(
445                "Input iterable argument has no data".to_owned(),
446            ));
447        }
448
449        let size = size.unwrap_or(0).try_into().unwrap();
450        Ok(Self {
451            data_type: DataType::FixedSizeBinary(size),
452            value_data: buffer.into(),
453            nulls: None,
454            value_length: size,
455            len,
456        })
457    }
458
459    #[inline]
460    fn value_offset_at(&self, i: usize) -> i32 {
461        self.value_length * i as i32
462    }
463
464    /// constructs a new iterator
465    pub fn iter(&self) -> FixedSizeBinaryIter<'_> {
466        FixedSizeBinaryIter::new(self)
467    }
468}
469
470impl From<ArrayData> for FixedSizeBinaryArray {
471    fn from(data: ArrayData) -> Self {
472        assert_eq!(
473            data.buffers().len(),
474            1,
475            "FixedSizeBinaryArray data should contain 1 buffer only (values)"
476        );
477        let value_length = match data.data_type() {
478            DataType::FixedSizeBinary(len) => *len,
479            _ => panic!("Expected data type to be FixedSizeBinary"),
480        };
481
482        let size = value_length as usize;
483        let value_data =
484            data.buffers()[0].slice_with_length(data.offset() * size, data.len() * size);
485
486        Self {
487            data_type: data.data_type().clone(),
488            nulls: data.nulls().cloned(),
489            len: data.len(),
490            value_data,
491            value_length,
492        }
493    }
494}
495
496impl From<FixedSizeBinaryArray> for ArrayData {
497    fn from(array: FixedSizeBinaryArray) -> Self {
498        let builder = ArrayDataBuilder::new(array.data_type)
499            .len(array.len)
500            .buffers(vec![array.value_data])
501            .nulls(array.nulls);
502
503        unsafe { builder.build_unchecked() }
504    }
505}
506
507/// Creates a `FixedSizeBinaryArray` from `FixedSizeList<u8>` array
508impl From<FixedSizeListArray> for FixedSizeBinaryArray {
509    fn from(v: FixedSizeListArray) -> Self {
510        let value_len = v.value_length();
511        let v = v.into_data();
512        assert_eq!(
513            v.child_data().len(),
514            1,
515            "FixedSizeBinaryArray can only be created from list array of u8 values \
516             (i.e. FixedSizeList<PrimitiveArray<u8>>)."
517        );
518        let child_data = &v.child_data()[0];
519
520        assert_eq!(
521            child_data.child_data().len(),
522            0,
523            "FixedSizeBinaryArray can only be created from list array of u8 values \
524             (i.e. FixedSizeList<PrimitiveArray<u8>>)."
525        );
526        assert_eq!(
527            child_data.data_type(),
528            &DataType::UInt8,
529            "FixedSizeBinaryArray can only be created from FixedSizeList<u8> arrays, mismatched data types."
530        );
531        assert_eq!(
532            child_data.null_count(),
533            0,
534            "The child array cannot contain null values."
535        );
536
537        let builder = ArrayData::builder(DataType::FixedSizeBinary(value_len))
538            .len(v.len())
539            .offset(v.offset())
540            .add_buffer(child_data.buffers()[0].slice(child_data.offset()))
541            .nulls(v.nulls().cloned());
542
543        let data = unsafe { builder.build_unchecked() };
544        Self::from(data)
545    }
546}
547
548impl From<Vec<Option<&[u8]>>> for FixedSizeBinaryArray {
549    fn from(v: Vec<Option<&[u8]>>) -> Self {
550        #[allow(deprecated)]
551        Self::try_from_sparse_iter(v.into_iter()).unwrap()
552    }
553}
554
555impl From<Vec<&[u8]>> for FixedSizeBinaryArray {
556    fn from(v: Vec<&[u8]>) -> Self {
557        Self::try_from_iter(v.into_iter()).unwrap()
558    }
559}
560
561impl<const N: usize> From<Vec<&[u8; N]>> for FixedSizeBinaryArray {
562    fn from(v: Vec<&[u8; N]>) -> Self {
563        Self::try_from_iter(v.into_iter()).unwrap()
564    }
565}
566
567impl std::fmt::Debug for FixedSizeBinaryArray {
568    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
569        write!(f, "FixedSizeBinaryArray<{}>\n[\n", self.value_length())?;
570        print_long_array(self, f, |array, index, f| {
571            std::fmt::Debug::fmt(&array.value(index), f)
572        })?;
573        write!(f, "]")
574    }
575}
576
577impl Array for FixedSizeBinaryArray {
578    fn as_any(&self) -> &dyn Any {
579        self
580    }
581
582    fn to_data(&self) -> ArrayData {
583        self.clone().into()
584    }
585
586    fn into_data(self) -> ArrayData {
587        self.into()
588    }
589
590    fn data_type(&self) -> &DataType {
591        &self.data_type
592    }
593
594    fn slice(&self, offset: usize, length: usize) -> ArrayRef {
595        Arc::new(self.slice(offset, length))
596    }
597
598    fn len(&self) -> usize {
599        self.len
600    }
601
602    fn is_empty(&self) -> bool {
603        self.len == 0
604    }
605
606    fn shrink_to_fit(&mut self) {
607        self.value_data.shrink_to_fit();
608        if let Some(nulls) = &mut self.nulls {
609            nulls.shrink_to_fit();
610        }
611    }
612
613    fn offset(&self) -> usize {
614        0
615    }
616
617    fn nulls(&self) -> Option<&NullBuffer> {
618        self.nulls.as_ref()
619    }
620
621    fn logical_null_count(&self) -> usize {
622        // More efficient that the default implementation
623        self.null_count()
624    }
625
626    fn get_buffer_memory_size(&self) -> usize {
627        let mut sum = self.value_data.capacity();
628        if let Some(n) = &self.nulls {
629            sum += n.buffer().capacity();
630        }
631        sum
632    }
633
634    fn get_array_memory_size(&self) -> usize {
635        std::mem::size_of::<Self>() + self.get_buffer_memory_size()
636    }
637}
638
639impl<'a> ArrayAccessor for &'a FixedSizeBinaryArray {
640    type Item = &'a [u8];
641
642    fn value(&self, index: usize) -> Self::Item {
643        FixedSizeBinaryArray::value(self, index)
644    }
645
646    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
647        FixedSizeBinaryArray::value_unchecked(self, index)
648    }
649}
650
651impl<'a> IntoIterator for &'a FixedSizeBinaryArray {
652    type Item = Option<&'a [u8]>;
653    type IntoIter = FixedSizeBinaryIter<'a>;
654
655    fn into_iter(self) -> Self::IntoIter {
656        FixedSizeBinaryIter::<'a>::new(self)
657    }
658}
659
660#[cfg(test)]
661mod tests {
662    use crate::RecordBatch;
663    use arrow_schema::{Field, Schema};
664
665    use super::*;
666
667    #[test]
668    fn test_fixed_size_binary_array() {
669        let values: [u8; 15] = *b"hellotherearrow";
670
671        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
672            .len(3)
673            .add_buffer(Buffer::from(&values))
674            .build()
675            .unwrap();
676        let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data);
677        assert_eq!(3, fixed_size_binary_array.len());
678        assert_eq!(0, fixed_size_binary_array.null_count());
679        assert_eq!(
680            [b'h', b'e', b'l', b'l', b'o'],
681            fixed_size_binary_array.value(0)
682        );
683        assert_eq!(
684            [b't', b'h', b'e', b'r', b'e'],
685            fixed_size_binary_array.value(1)
686        );
687        assert_eq!(
688            [b'a', b'r', b'r', b'o', b'w'],
689            fixed_size_binary_array.value(2)
690        );
691        assert_eq!(5, fixed_size_binary_array.value_length());
692        assert_eq!(10, fixed_size_binary_array.value_offset(2));
693        for i in 0..3 {
694            assert!(fixed_size_binary_array.is_valid(i));
695            assert!(!fixed_size_binary_array.is_null(i));
696        }
697
698        // Test binary array with offset
699        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
700            .len(2)
701            .offset(1)
702            .add_buffer(Buffer::from(&values))
703            .build()
704            .unwrap();
705        let fixed_size_binary_array = FixedSizeBinaryArray::from(array_data);
706        assert_eq!(
707            [b't', b'h', b'e', b'r', b'e'],
708            fixed_size_binary_array.value(0)
709        );
710        assert_eq!(
711            [b'a', b'r', b'r', b'o', b'w'],
712            fixed_size_binary_array.value(1)
713        );
714        assert_eq!(2, fixed_size_binary_array.len());
715        assert_eq!(0, fixed_size_binary_array.value_offset(0));
716        assert_eq!(5, fixed_size_binary_array.value_length());
717        assert_eq!(5, fixed_size_binary_array.value_offset(1));
718    }
719
720    #[test]
721    fn test_fixed_size_binary_array_from_fixed_size_list_array() {
722        let values = [0_u8, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13];
723        let values_data = ArrayData::builder(DataType::UInt8)
724            .len(12)
725            .offset(2)
726            .add_buffer(Buffer::from_slice_ref(values))
727            .build()
728            .unwrap();
729        // [null, [10, 11, 12, 13]]
730        let array_data = unsafe {
731            ArrayData::builder(DataType::FixedSizeList(
732                Arc::new(Field::new_list_field(DataType::UInt8, false)),
733                4,
734            ))
735            .len(2)
736            .offset(1)
737            .add_child_data(values_data)
738            .null_bit_buffer(Some(Buffer::from_slice_ref([0b101])))
739            .build_unchecked()
740        };
741        let list_array = FixedSizeListArray::from(array_data);
742        let binary_array = FixedSizeBinaryArray::from(list_array);
743
744        assert_eq!(2, binary_array.len());
745        assert_eq!(1, binary_array.null_count());
746        assert!(binary_array.is_null(0));
747        assert!(binary_array.is_valid(1));
748        assert_eq!(&[10, 11, 12, 13], binary_array.value(1));
749    }
750
751    #[test]
752    #[should_panic(
753        expected = "FixedSizeBinaryArray can only be created from FixedSizeList<u8> arrays"
754    )]
755    // Different error messages, so skip for now
756    // https://github.com/apache/arrow-rs/issues/1545
757    #[cfg(not(feature = "force_validate"))]
758    fn test_fixed_size_binary_array_from_incorrect_fixed_size_list_array() {
759        let values: [u32; 12] = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11];
760        let values_data = ArrayData::builder(DataType::UInt32)
761            .len(12)
762            .add_buffer(Buffer::from_slice_ref(values))
763            .build()
764            .unwrap();
765
766        let array_data = unsafe {
767            ArrayData::builder(DataType::FixedSizeList(
768                Arc::new(Field::new_list_field(DataType::Binary, false)),
769                4,
770            ))
771            .len(3)
772            .add_child_data(values_data)
773            .build_unchecked()
774        };
775        let list_array = FixedSizeListArray::from(array_data);
776        drop(FixedSizeBinaryArray::from(list_array));
777    }
778
779    #[test]
780    #[should_panic(expected = "The child array cannot contain null values.")]
781    fn test_fixed_size_binary_array_from_fixed_size_list_array_with_child_nulls_failed() {
782        let values = [0_u8, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11];
783        let values_data = ArrayData::builder(DataType::UInt8)
784            .len(12)
785            .add_buffer(Buffer::from_slice_ref(values))
786            .null_bit_buffer(Some(Buffer::from_slice_ref([0b101010101010])))
787            .build()
788            .unwrap();
789
790        let array_data = unsafe {
791            ArrayData::builder(DataType::FixedSizeList(
792                Arc::new(Field::new_list_field(DataType::UInt8, false)),
793                4,
794            ))
795            .len(3)
796            .add_child_data(values_data)
797            .build_unchecked()
798        };
799        let list_array = FixedSizeListArray::from(array_data);
800        drop(FixedSizeBinaryArray::from(list_array));
801    }
802
803    #[test]
804    fn test_fixed_size_binary_array_fmt_debug() {
805        let values: [u8; 15] = *b"hellotherearrow";
806
807        let array_data = ArrayData::builder(DataType::FixedSizeBinary(5))
808            .len(3)
809            .add_buffer(Buffer::from(&values))
810            .build()
811            .unwrap();
812        let arr = FixedSizeBinaryArray::from(array_data);
813        assert_eq!(
814            "FixedSizeBinaryArray<5>\n[\n  [104, 101, 108, 108, 111],\n  [116, 104, 101, 114, 101],\n  [97, 114, 114, 111, 119],\n]",
815            format!("{arr:?}")
816        );
817    }
818
819    #[test]
820    fn test_fixed_size_binary_array_from_iter() {
821        let input_arg = vec![vec![1, 2], vec![3, 4], vec![5, 6]];
822        let arr = FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap();
823
824        assert_eq!(2, arr.value_length());
825        assert_eq!(3, arr.len())
826    }
827
828    #[test]
829    fn test_all_none_fixed_size_binary_array_from_sparse_iter() {
830        let none_option: Option<[u8; 32]> = None;
831        let input_arg = vec![none_option, none_option, none_option];
832        #[allow(deprecated)]
833        let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.into_iter()).unwrap();
834        assert_eq!(0, arr.value_length());
835        assert_eq!(3, arr.len())
836    }
837
838    #[test]
839    fn test_fixed_size_binary_array_from_sparse_iter() {
840        let input_arg = vec![
841            None,
842            Some(vec![7, 8]),
843            Some(vec![9, 10]),
844            None,
845            Some(vec![13, 14]),
846        ];
847        #[allow(deprecated)]
848        let arr = FixedSizeBinaryArray::try_from_sparse_iter(input_arg.iter().cloned()).unwrap();
849        assert_eq!(2, arr.value_length());
850        assert_eq!(5, arr.len());
851
852        let arr =
853            FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 2).unwrap();
854        assert_eq!(2, arr.value_length());
855        assert_eq!(5, arr.len());
856    }
857
858    #[test]
859    fn test_fixed_size_binary_array_from_sparse_iter_with_size_all_none() {
860        let input_arg = vec![None, None, None, None, None] as Vec<Option<Vec<u8>>>;
861
862        let arr = FixedSizeBinaryArray::try_from_sparse_iter_with_size(input_arg.into_iter(), 16)
863            .unwrap();
864        assert_eq!(16, arr.value_length());
865        assert_eq!(5, arr.len())
866    }
867
868    #[test]
869    fn test_fixed_size_binary_array_from_vec() {
870        let values = vec!["one".as_bytes(), b"two", b"six", b"ten"];
871        let array = FixedSizeBinaryArray::from(values);
872        assert_eq!(array.len(), 4);
873        assert_eq!(array.null_count(), 0);
874        assert_eq!(array.logical_null_count(), 0);
875        assert_eq!(array.value(0), b"one");
876        assert_eq!(array.value(1), b"two");
877        assert_eq!(array.value(2), b"six");
878        assert_eq!(array.value(3), b"ten");
879        assert!(!array.is_null(0));
880        assert!(!array.is_null(1));
881        assert!(!array.is_null(2));
882        assert!(!array.is_null(3));
883    }
884
885    #[test]
886    #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")]
887    fn test_fixed_size_binary_array_from_vec_incorrect_length() {
888        let values = vec!["one".as_bytes(), b"two", b"three", b"four"];
889        let _ = FixedSizeBinaryArray::from(values);
890    }
891
892    #[test]
893    fn test_fixed_size_binary_array_from_opt_vec() {
894        let values = vec![
895            Some("one".as_bytes()),
896            Some(b"two"),
897            None,
898            Some(b"six"),
899            Some(b"ten"),
900        ];
901        let array = FixedSizeBinaryArray::from(values);
902        assert_eq!(array.len(), 5);
903        assert_eq!(array.value(0), b"one");
904        assert_eq!(array.value(1), b"two");
905        assert_eq!(array.value(3), b"six");
906        assert_eq!(array.value(4), b"ten");
907        assert!(!array.is_null(0));
908        assert!(!array.is_null(1));
909        assert!(array.is_null(2));
910        assert!(!array.is_null(3));
911        assert!(!array.is_null(4));
912    }
913
914    #[test]
915    #[should_panic(expected = "Nested array size mismatch: one is 3, and the other is 5")]
916    fn test_fixed_size_binary_array_from_opt_vec_incorrect_length() {
917        let values = vec![
918            Some("one".as_bytes()),
919            Some(b"two"),
920            None,
921            Some(b"three"),
922            Some(b"four"),
923        ];
924        let _ = FixedSizeBinaryArray::from(values);
925    }
926
927    #[test]
928    fn fixed_size_binary_array_all_null() {
929        let data = vec![None] as Vec<Option<String>>;
930        let array =
931            FixedSizeBinaryArray::try_from_sparse_iter_with_size(data.into_iter(), 0).unwrap();
932        array
933            .into_data()
934            .validate_full()
935            .expect("All null array has valid array data");
936    }
937
938    #[test]
939    // Test for https://github.com/apache/arrow-rs/issues/1390
940    fn fixed_size_binary_array_all_null_in_batch_with_schema() {
941        let schema = Schema::new(vec![Field::new("a", DataType::FixedSizeBinary(2), true)]);
942
943        let none_option: Option<[u8; 2]> = None;
944        let item = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
945            vec![none_option, none_option, none_option].into_iter(),
946            2,
947        )
948        .unwrap();
949
950        // Should not panic
951        RecordBatch::try_new(Arc::new(schema), vec![Arc::new(item)]).unwrap();
952    }
953
954    #[test]
955    #[should_panic(
956        expected = "Trying to access an element at index 4 from a FixedSizeBinaryArray of length 3"
957    )]
958    fn test_fixed_size_binary_array_get_value_index_out_of_bound() {
959        let values = vec![Some("one".as_bytes()), Some(b"two"), None];
960        let array = FixedSizeBinaryArray::from(values);
961
962        array.value(4);
963    }
964
965    #[test]
966    fn test_constructors() {
967        let buffer = Buffer::from_vec(vec![0_u8; 10]);
968        let a = FixedSizeBinaryArray::new(2, buffer.clone(), None);
969        assert_eq!(a.len(), 5);
970
971        let nulls = NullBuffer::new_null(5);
972        FixedSizeBinaryArray::new(2, buffer.clone(), Some(nulls));
973
974        let a = FixedSizeBinaryArray::new(3, buffer.clone(), None);
975        assert_eq!(a.len(), 3);
976
977        let nulls = NullBuffer::new_null(3);
978        FixedSizeBinaryArray::new(3, buffer.clone(), Some(nulls));
979
980        let err = FixedSizeBinaryArray::try_new(-1, buffer.clone(), None).unwrap_err();
981
982        assert_eq!(
983            err.to_string(),
984            "Invalid argument error: Size cannot be negative, got -1"
985        );
986
987        let nulls = NullBuffer::new_null(3);
988        let err = FixedSizeBinaryArray::try_new(2, buffer, Some(nulls)).unwrap_err();
989        assert_eq!(err.to_string(), "Invalid argument error: Incorrect length of null buffer for FixedSizeBinaryArray, expected 5 got 3");
990    }
991}