arrow_string/
substring.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
18//! Defines kernel to extract a substring of an Array
19//! Supported array types:
20//! [GenericStringArray], [GenericBinaryArray], [FixedSizeBinaryArray], [DictionaryArray]
21
22use arrow_array::builder::BufferBuilder;
23use arrow_array::types::*;
24use arrow_array::*;
25use arrow_buffer::{ArrowNativeType, Buffer, MutableBuffer};
26use arrow_data::ArrayData;
27use arrow_schema::{ArrowError, DataType};
28use num::Zero;
29use std::cmp::Ordering;
30use std::sync::Arc;
31
32/// Returns an [`ArrayRef`] with substrings of all the elements in `array`.
33///
34/// # Arguments
35///
36/// * `start` - The start index of all substrings.
37///   If `start >= 0`, then count from the start of the string,
38///   otherwise count from the end of the string.
39///
40/// * `length`(option) - The length of all substrings.
41///   If `length` is [None], then the substring is from `start` to the end of the string.
42///
43/// Attention: Both `start` and `length` are counted by byte, not by char.
44///
45/// # Basic usage
46/// ```
47/// # use arrow_array::StringArray;
48/// # use arrow_string::substring::substring;
49/// let array = StringArray::from(vec![Some("arrow"), None, Some("rust")]);
50/// let result = substring(&array, 1, Some(4)).unwrap();
51/// let result = result.as_any().downcast_ref::<StringArray>().unwrap();
52/// assert_eq!(result, &StringArray::from(vec![Some("rrow"), None, Some("ust")]));
53/// ```
54///
55/// # Error
56/// - The function errors when the passed array is not a [`GenericStringArray`],
57///   [`GenericBinaryArray`], [`FixedSizeBinaryArray`] or [`DictionaryArray`]
58///   with supported array type as its value type.
59/// - The function errors if the offset of a substring in the input array is
60///   at invalid char boundary (only for \[Large\]String array).
61///   It is recommended to use [`substring_by_char`] if the input array may
62///   contain non-ASCII chars.
63///
64/// ## Example of trying to get an invalid utf-8 format substring
65/// ```
66/// # use arrow_array::StringArray;
67/// # use arrow_string::substring::substring;
68/// let array = StringArray::from(vec![Some("E=mc²")]);
69/// let error = substring(&array, 0, Some(5)).unwrap_err().to_string();
70/// assert!(error.contains("invalid utf-8 boundary"));
71/// ```
72pub fn substring(
73    array: &dyn Array,
74    start: i64,
75    length: Option<u64>,
76) -> Result<ArrayRef, ArrowError> {
77    macro_rules! substring_dict {
78        ($kt: ident, $($t: ident: $gt: ident), *) => {
79            match $kt.as_ref() {
80                $(
81                    &DataType::$t => {
82                        let dict = array
83                            .as_any()
84                            .downcast_ref::<DictionaryArray<$gt>>()
85                            .unwrap_or_else(|| {
86                                panic!("Expect 'DictionaryArray<{}>' but got array of data type {:?}",
87                                       stringify!($gt), array.data_type())
88                            });
89                        let values = substring(dict.values(), start, length)?;
90                        let result = DictionaryArray::try_new(dict.keys().clone(), values)?;
91                        Ok(Arc::new(result))
92                    },
93                )*
94                    t => panic!("Unsupported dictionary key type: {}", t)
95            }
96        }
97    }
98
99    match array.data_type() {
100        DataType::Dictionary(kt, _) => {
101            substring_dict!(
102                kt,
103                Int8: Int8Type,
104                Int16: Int16Type,
105                Int32: Int32Type,
106                Int64: Int64Type,
107                UInt8: UInt8Type,
108                UInt16: UInt16Type,
109                UInt32: UInt32Type,
110                UInt64: UInt64Type
111            )
112        }
113        DataType::LargeBinary => byte_substring(
114            array
115                .as_any()
116                .downcast_ref::<LargeBinaryArray>()
117                .expect("A large binary is expected"),
118            start,
119            length.map(|e| e as i64),
120        ),
121        DataType::Binary => byte_substring(
122            array
123                .as_any()
124                .downcast_ref::<BinaryArray>()
125                .expect("A binary is expected"),
126            start as i32,
127            length.map(|e| e as i32),
128        ),
129        DataType::FixedSizeBinary(old_len) => fixed_size_binary_substring(
130            array
131                .as_any()
132                .downcast_ref::<FixedSizeBinaryArray>()
133                .expect("a fixed size binary is expected"),
134            *old_len,
135            start as i32,
136            length.map(|e| e as i32),
137        ),
138        DataType::LargeUtf8 => byte_substring(
139            array
140                .as_any()
141                .downcast_ref::<LargeStringArray>()
142                .expect("A large string is expected"),
143            start,
144            length.map(|e| e as i64),
145        ),
146        DataType::Utf8 => byte_substring(
147            array
148                .as_any()
149                .downcast_ref::<StringArray>()
150                .expect("A string is expected"),
151            start as i32,
152            length.map(|e| e as i32),
153        ),
154        _ => Err(ArrowError::ComputeError(format!(
155            "substring does not support type {:?}",
156            array.data_type()
157        ))),
158    }
159}
160
161/// Substrings based on character index
162///
163/// # Arguments
164/// * `array` - The input string array
165///
166/// * `start` - The start index of all substrings.
167///   If `start >= 0`, then count from the start of the string,
168///   otherwise count from the end of the string.
169///
170/// * `length`(option) - The length of all substrings.
171///   If `length` is `None`, then the substring is from `start` to the end of the string.
172///
173/// Attention: Both `start` and `length` are counted by char.
174///
175/// # Performance
176///
177/// This function is slower than [substring]. Theoretically, the time complexity
178/// is `O(n)` where `n` is the length of the value buffer. It is recommended to
179/// use [substring] if the input array only contains ASCII chars.
180///
181/// # Basic usage
182/// ```
183/// # use arrow_array::StringArray;
184/// # use arrow_string::substring::substring_by_char;
185/// let array = StringArray::from(vec![Some("arrow"), None, Some("Γ ⊢x:T")]);
186/// let result = substring_by_char(&array, 1, Some(4)).unwrap();
187/// assert_eq!(result, StringArray::from(vec![Some("rrow"), None, Some(" ⊢x:")]));
188/// ```
189pub fn substring_by_char<OffsetSize: OffsetSizeTrait>(
190    array: &GenericStringArray<OffsetSize>,
191    start: i64,
192    length: Option<u64>,
193) -> Result<GenericStringArray<OffsetSize>, ArrowError> {
194    let mut vals = BufferBuilder::<u8>::new({
195        let offsets = array.value_offsets();
196        (offsets[array.len()] - offsets[0]).to_usize().unwrap()
197    });
198    let mut new_offsets = BufferBuilder::<OffsetSize>::new(array.len() + 1);
199    new_offsets.append(OffsetSize::zero());
200    let length = length.map(|len| len.to_usize().unwrap());
201
202    array.iter().for_each(|val| {
203        if let Some(val) = val {
204            let char_count = val.chars().count();
205            let start = if start >= 0 {
206                start.to_usize().unwrap()
207            } else {
208                char_count - (-start).to_usize().unwrap().min(char_count)
209            };
210            let (start_offset, end_offset) = get_start_end_offset(val, start, length);
211            vals.append_slice(&val.as_bytes()[start_offset..end_offset]);
212        }
213        new_offsets.append(OffsetSize::from_usize(vals.len()).unwrap());
214    });
215    let data = unsafe {
216        ArrayData::new_unchecked(
217            GenericStringArray::<OffsetSize>::DATA_TYPE,
218            array.len(),
219            None,
220            array.nulls().map(|b| b.inner().sliced()),
221            0,
222            vec![new_offsets.finish(), vals.finish()],
223            vec![],
224        )
225    };
226    Ok(GenericStringArray::<OffsetSize>::from(data))
227}
228
229/// * `val` - string
230/// * `start` - the start char index of the substring
231/// * `length` - the char length of the substring
232///
233/// Return the `start` and `end` offset (by byte) of the substring
234fn get_start_end_offset(val: &str, start: usize, length: Option<usize>) -> (usize, usize) {
235    let len = val.len();
236    let mut offset_char_iter = val.char_indices();
237    let start_offset = offset_char_iter
238        .nth(start)
239        .map_or(len, |(offset, _)| offset);
240    let end_offset = length.map_or(len, |length| {
241        if length > 0 {
242            offset_char_iter
243                .nth(length - 1)
244                .map_or(len, |(offset, _)| offset)
245        } else {
246            start_offset
247        }
248    });
249    (start_offset, end_offset)
250}
251
252fn byte_substring<T: ByteArrayType>(
253    array: &GenericByteArray<T>,
254    start: T::Offset,
255    length: Option<T::Offset>,
256) -> Result<ArrayRef, ArrowError>
257where
258    <T as ByteArrayType>::Native: PartialEq,
259{
260    let offsets = array.value_offsets();
261    let data = array.value_data();
262    let zero = <T::Offset as Zero>::zero();
263
264    // When array is [Large]StringArray, we will check whether `offset` is at a valid char boundary.
265    let check_char_boundary = {
266        |offset: T::Offset| {
267            if !matches!(T::DATA_TYPE, DataType::Utf8 | DataType::LargeUtf8) {
268                return Ok(offset);
269            }
270            // Safety: a StringArray must contain valid UTF8 data
271            let data_str = unsafe { std::str::from_utf8_unchecked(data) };
272            let offset_usize = offset.as_usize();
273            if data_str.is_char_boundary(offset_usize) {
274                Ok(offset)
275            } else {
276                Err(ArrowError::ComputeError(format!(
277                    "The offset {offset_usize} is at an invalid utf-8 boundary."
278                )))
279            }
280        }
281    };
282
283    // start and end offsets of all substrings
284    let mut new_starts_ends: Vec<(T::Offset, T::Offset)> = Vec::with_capacity(array.len());
285    let mut new_offsets: Vec<T::Offset> = Vec::with_capacity(array.len() + 1);
286    let mut len_so_far = zero;
287    new_offsets.push(zero);
288
289    offsets
290        .windows(2)
291        .try_for_each(|pair| -> Result<(), ArrowError> {
292            let new_start = match start.cmp(&zero) {
293                Ordering::Greater => check_char_boundary((pair[0] + start).min(pair[1]))?,
294                Ordering::Equal => pair[0],
295                Ordering::Less => check_char_boundary((pair[1] + start).max(pair[0]))?,
296            };
297            let new_end = match length {
298                Some(length) => check_char_boundary((length + new_start).min(pair[1]))?,
299                None => pair[1],
300            };
301            len_so_far += new_end - new_start;
302            new_starts_ends.push((new_start, new_end));
303            new_offsets.push(len_so_far);
304            Ok(())
305        })?;
306
307    // concatenate substrings into a buffer
308    let mut new_values = MutableBuffer::new(new_offsets.last().unwrap().as_usize());
309
310    new_starts_ends
311        .iter()
312        .map(|(start, end)| {
313            let start = start.as_usize();
314            let end = end.as_usize();
315            &data[start..end]
316        })
317        .for_each(|slice| new_values.extend_from_slice(slice));
318
319    let data = unsafe {
320        ArrayData::new_unchecked(
321            GenericByteArray::<T>::DATA_TYPE,
322            array.len(),
323            None,
324            array.nulls().map(|b| b.inner().sliced()),
325            0,
326            vec![Buffer::from_vec(new_offsets), new_values.into()],
327            vec![],
328        )
329    };
330    Ok(make_array(data))
331}
332
333fn fixed_size_binary_substring(
334    array: &FixedSizeBinaryArray,
335    old_len: i32,
336    start: i32,
337    length: Option<i32>,
338) -> Result<ArrayRef, ArrowError> {
339    let new_start = if start >= 0 {
340        start.min(old_len)
341    } else {
342        (old_len + start).max(0)
343    };
344    let new_len = match length {
345        Some(len) => len.min(old_len - new_start),
346        None => old_len - new_start,
347    };
348
349    // build value buffer
350    let num_of_elements = array.len();
351    let data = array.value_data();
352    let mut new_values = MutableBuffer::new(num_of_elements * (new_len as usize));
353    (0..num_of_elements)
354        .map(|idx| {
355            let offset = array.value_offset(idx);
356            (
357                (offset + new_start) as usize,
358                (offset + new_start + new_len) as usize,
359            )
360        })
361        .for_each(|(start, end)| new_values.extend_from_slice(&data[start..end]));
362
363    let array_data = unsafe {
364        ArrayData::new_unchecked(
365            DataType::FixedSizeBinary(new_len),
366            num_of_elements,
367            None,
368            array.nulls().map(|b| b.inner().sliced()),
369            0,
370            vec![new_values.into()],
371            vec![],
372        )
373    };
374
375    Ok(make_array(array_data))
376}
377
378#[cfg(test)]
379mod tests {
380    use super::*;
381
382    /// A helper macro to generate test cases.
383    /// # Arguments
384    /// * `input` - A vector which array can be built from.
385    /// * `start` - The start index of the substring.
386    /// * `len` - The length of the substring.
387    /// * `result` - The expected result of substring, which is a vector that array can be built from.
388    /// # Return
389    /// A vector of `(input, start, len, result)`.
390    ///
391    /// Users can provide any number of `(start, len, result)` to generate test cases for one `input`.
392    macro_rules! gen_test_cases {
393        ($input:expr, $(($start:expr, $len:expr, $result:expr)), *) => {
394            [
395                $(
396                    ($input.clone(), $start, $len, $result),
397                )*
398            ]
399        };
400    }
401
402    /// A helper macro to test the substring functions.
403    /// # Arguments
404    /// * `cases` - The test cases which is a vector of `(input, start, len, result)`.
405    ///   Please look at [`gen_test_cases`] to find how to generate it.
406    /// * `array_ty` - The array type.
407    /// * `substring_fn` - Either [`substring`] or [`substring_by_char`].
408    macro_rules! do_test {
409        ($cases:expr, $array_ty:ty, $substring_fn:ident) => {
410            $cases
411                .into_iter()
412                .for_each(|(array, start, length, expected)| {
413                    let array = <$array_ty>::from(array);
414                    let result = $substring_fn(&array, start, length).unwrap();
415                    let result = result.as_any().downcast_ref::<$array_ty>().unwrap();
416                    let expected = <$array_ty>::from(expected);
417                    assert_eq!(&expected, result);
418                })
419        };
420    }
421
422    fn with_nulls_generic_binary<O: OffsetSizeTrait>() {
423        let input = vec![
424            Some("hello".as_bytes()),
425            None,
426            Some(&[0xf8, 0xf9, 0xff, 0xfa]),
427        ];
428        // all-nulls array is always identical
429        let base_case = gen_test_cases!(
430            vec![None, None, None],
431            (-1, Some(1), vec![None, None, None])
432        );
433        let cases = gen_test_cases!(
434            input,
435            // identity
436            (0, None, input.clone()),
437            // 0 length -> Nothing
438            (0, Some(0), vec![Some(&[]), None, Some(&[])]),
439            // high start -> Nothing
440            (1000, Some(0), vec![Some(&[]), None, Some(&[])]),
441            // high negative start -> identity
442            (-1000, None, input.clone()),
443            // high length -> identity
444            (0, Some(1000), input.clone())
445        );
446
447        do_test!(
448            [&base_case[..], &cases[..]].concat(),
449            GenericBinaryArray<O>,
450            substring
451        );
452    }
453
454    #[test]
455    fn with_nulls_binary() {
456        with_nulls_generic_binary::<i32>()
457    }
458
459    #[test]
460    fn with_nulls_large_binary() {
461        with_nulls_generic_binary::<i64>()
462    }
463
464    fn without_nulls_generic_binary<O: OffsetSizeTrait>() {
465        let input = vec!["hello".as_bytes(), b"", &[0xf8, 0xf9, 0xff, 0xfa]];
466        // empty array is always identical
467        let base_case = gen_test_cases!(
468            vec!["".as_bytes(), b"", b""],
469            (2, Some(1), vec!["".as_bytes(), b"", b""])
470        );
471        let cases = gen_test_cases!(
472            input,
473            // identity
474            (0, None, input.clone()),
475            // increase start
476            (1, None, vec![b"ello", b"", &[0xf9, 0xff, 0xfa]]),
477            (2, None, vec![b"llo", b"", &[0xff, 0xfa]]),
478            (3, None, vec![b"lo", b"", &[0xfa]]),
479            (10, None, vec![b"", b"", b""]),
480            // increase start negatively
481            (-1, None, vec![b"o", b"", &[0xfa]]),
482            (-2, None, vec![b"lo", b"", &[0xff, 0xfa]]),
483            (-3, None, vec![b"llo", b"", &[0xf9, 0xff, 0xfa]]),
484            (-10, None, input.clone()),
485            // increase length
486            (1, Some(1), vec![b"e", b"", &[0xf9]]),
487            (1, Some(2), vec![b"el", b"", &[0xf9, 0xff]]),
488            (1, Some(3), vec![b"ell", b"", &[0xf9, 0xff, 0xfa]]),
489            (1, Some(4), vec![b"ello", b"", &[0xf9, 0xff, 0xfa]]),
490            (-3, Some(1), vec![b"l", b"", &[0xf9]]),
491            (-3, Some(2), vec![b"ll", b"", &[0xf9, 0xff]]),
492            (-3, Some(3), vec![b"llo", b"", &[0xf9, 0xff, 0xfa]]),
493            (-3, Some(4), vec![b"llo", b"", &[0xf9, 0xff, 0xfa]])
494        );
495
496        do_test!(
497            [&base_case[..], &cases[..]].concat(),
498            GenericBinaryArray<O>,
499            substring
500        );
501    }
502
503    #[test]
504    fn without_nulls_binary() {
505        without_nulls_generic_binary::<i32>()
506    }
507
508    #[test]
509    fn without_nulls_large_binary() {
510        without_nulls_generic_binary::<i64>()
511    }
512
513    fn generic_binary_with_non_zero_offset<O: OffsetSizeTrait>() {
514        let values = 0_u8..15;
515        let offsets = &[
516            O::zero(),
517            O::from_usize(5).unwrap(),
518            O::from_usize(10).unwrap(),
519            O::from_usize(15).unwrap(),
520        ];
521        // set the first and third element to be valid
522        let bitmap = [0b101_u8];
523
524        let data = ArrayData::builder(GenericBinaryArray::<O>::DATA_TYPE)
525            .len(2)
526            .add_buffer(Buffer::from_slice_ref(offsets))
527            .add_buffer(Buffer::from_iter(values))
528            .null_bit_buffer(Some(Buffer::from(bitmap)))
529            .offset(1)
530            .build()
531            .unwrap();
532        // array is `[null, [10, 11, 12, 13, 14]]`
533        let array = GenericBinaryArray::<O>::from(data);
534        // result is `[null, [11, 12, 13, 14]]`
535        let result = substring(&array, 1, None).unwrap();
536        let result = result
537            .as_any()
538            .downcast_ref::<GenericBinaryArray<O>>()
539            .unwrap();
540        let expected =
541            GenericBinaryArray::<O>::from_opt_vec(vec![None, Some(&[11_u8, 12, 13, 14])]);
542        assert_eq!(result, &expected);
543    }
544
545    #[test]
546    fn binary_with_non_zero_offset() {
547        generic_binary_with_non_zero_offset::<i32>()
548    }
549
550    #[test]
551    fn large_binary_with_non_zero_offset() {
552        generic_binary_with_non_zero_offset::<i64>()
553    }
554
555    #[test]
556    fn with_nulls_fixed_size_binary() {
557        let input = vec![Some("cat".as_bytes()), None, Some(&[0xf8, 0xf9, 0xff])];
558        // all-nulls array is always identical
559        let base_case =
560            gen_test_cases!(vec![None, None, None], (3, Some(2), vec![None, None, None]));
561        let cases = gen_test_cases!(
562            input,
563            // identity
564            (0, None, input.clone()),
565            // increase start
566            (1, None, vec![Some(b"at"), None, Some(&[0xf9, 0xff])]),
567            (2, None, vec![Some(b"t"), None, Some(&[0xff])]),
568            (3, None, vec![Some(b""), None, Some(b"")]),
569            (10, None, vec![Some(b""), None, Some(b"")]),
570            // increase start negatively
571            (-1, None, vec![Some(b"t"), None, Some(&[0xff])]),
572            (-2, None, vec![Some(b"at"), None, Some(&[0xf9, 0xff])]),
573            (-3, None, input.clone()),
574            (-10, None, input.clone()),
575            // increase length
576            (1, Some(1), vec![Some(b"a"), None, Some(&[0xf9])]),
577            (1, Some(2), vec![Some(b"at"), None, Some(&[0xf9, 0xff])]),
578            (1, Some(3), vec![Some(b"at"), None, Some(&[0xf9, 0xff])]),
579            (-3, Some(1), vec![Some(b"c"), None, Some(&[0xf8])]),
580            (-3, Some(2), vec![Some(b"ca"), None, Some(&[0xf8, 0xf9])]),
581            (-3, Some(3), input.clone()),
582            (-3, Some(4), input.clone())
583        );
584
585        do_test!(
586            [&base_case[..], &cases[..]].concat(),
587            FixedSizeBinaryArray,
588            substring
589        );
590    }
591
592    #[test]
593    fn without_nulls_fixed_size_binary() {
594        let input = vec!["cat".as_bytes(), b"dog", &[0xf8, 0xf9, 0xff]];
595        // empty array is always identical
596        let base_case = gen_test_cases!(
597            vec!["".as_bytes(), &[], &[]],
598            (1, Some(2), vec!["".as_bytes(), &[], &[]])
599        );
600        let cases = gen_test_cases!(
601            input,
602            // identity
603            (0, None, input.clone()),
604            // increase start
605            (1, None, vec![b"at", b"og", &[0xf9, 0xff]]),
606            (2, None, vec![b"t", b"g", &[0xff]]),
607            (3, None, vec![&[], &[], &[]]),
608            (10, None, vec![&[], &[], &[]]),
609            // increase start negatively
610            (-1, None, vec![b"t", b"g", &[0xff]]),
611            (-2, None, vec![b"at", b"og", &[0xf9, 0xff]]),
612            (-3, None, input.clone()),
613            (-10, None, input.clone()),
614            // increase length
615            (1, Some(1), vec![b"a", b"o", &[0xf9]]),
616            (1, Some(2), vec![b"at", b"og", &[0xf9, 0xff]]),
617            (1, Some(3), vec![b"at", b"og", &[0xf9, 0xff]]),
618            (-3, Some(1), vec![b"c", b"d", &[0xf8]]),
619            (-3, Some(2), vec![b"ca", b"do", &[0xf8, 0xf9]]),
620            (-3, Some(3), input.clone()),
621            (-3, Some(4), input.clone())
622        );
623
624        do_test!(
625            [&base_case[..], &cases[..]].concat(),
626            FixedSizeBinaryArray,
627            substring
628        );
629    }
630
631    #[test]
632    fn fixed_size_binary_with_non_zero_offset() {
633        let values: [u8; 15] = *b"hellotherearrow";
634        // set the first and third element to be valid
635        let bits_v = [0b101_u8];
636
637        let data = ArrayData::builder(DataType::FixedSizeBinary(5))
638            .len(2)
639            .add_buffer(Buffer::from(&values))
640            .offset(1)
641            .null_bit_buffer(Some(Buffer::from(bits_v)))
642            .build()
643            .unwrap();
644        // array is `[null, "arrow"]`
645        let array = FixedSizeBinaryArray::from(data);
646        // result is `[null, "rrow"]`
647        let result = substring(&array, 1, None).unwrap();
648        let result = result
649            .as_any()
650            .downcast_ref::<FixedSizeBinaryArray>()
651            .unwrap();
652        let expected = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
653            vec![None, Some(b"rrow")].into_iter(),
654            4,
655        )
656        .unwrap();
657        assert_eq!(result, &expected);
658    }
659
660    fn with_nulls_generic_string<O: OffsetSizeTrait>() {
661        let input = vec![Some("hello"), None, Some("word")];
662        // all-nulls array is always identical
663        let base_case = gen_test_cases!(vec![None, None, None], (0, None, vec![None, None, None]));
664        let cases = gen_test_cases!(
665            input,
666            // identity
667            (0, None, input.clone()),
668            // 0 length -> Nothing
669            (0, Some(0), vec![Some(""), None, Some("")]),
670            // high start -> Nothing
671            (1000, Some(0), vec![Some(""), None, Some("")]),
672            // high negative start -> identity
673            (-1000, None, input.clone()),
674            // high length -> identity
675            (0, Some(1000), input.clone())
676        );
677
678        do_test!(
679            [&base_case[..], &cases[..]].concat(),
680            GenericStringArray<O>,
681            substring
682        );
683    }
684
685    #[test]
686    fn with_nulls_string() {
687        with_nulls_generic_string::<i32>()
688    }
689
690    #[test]
691    fn with_nulls_large_string() {
692        with_nulls_generic_string::<i64>()
693    }
694
695    fn without_nulls_generic_string<O: OffsetSizeTrait>() {
696        let input = vec!["hello", "", "word"];
697        // empty array is always identical
698        let base_case = gen_test_cases!(vec!["", "", ""], (0, None, vec!["", "", ""]));
699        let cases = gen_test_cases!(
700            input,
701            // identity
702            (0, None, input.clone()),
703            (1, None, vec!["ello", "", "ord"]),
704            (2, None, vec!["llo", "", "rd"]),
705            (3, None, vec!["lo", "", "d"]),
706            (10, None, vec!["", "", ""]),
707            // increase start negatively
708            (-1, None, vec!["o", "", "d"]),
709            (-2, None, vec!["lo", "", "rd"]),
710            (-3, None, vec!["llo", "", "ord"]),
711            (-10, None, input.clone()),
712            // increase length
713            (1, Some(1), vec!["e", "", "o"]),
714            (1, Some(2), vec!["el", "", "or"]),
715            (1, Some(3), vec!["ell", "", "ord"]),
716            (1, Some(4), vec!["ello", "", "ord"]),
717            (-3, Some(1), vec!["l", "", "o"]),
718            (-3, Some(2), vec!["ll", "", "or"]),
719            (-3, Some(3), vec!["llo", "", "ord"]),
720            (-3, Some(4), vec!["llo", "", "ord"])
721        );
722
723        do_test!(
724            [&base_case[..], &cases[..]].concat(),
725            GenericStringArray<O>,
726            substring
727        );
728    }
729
730    #[test]
731    fn without_nulls_string() {
732        without_nulls_generic_string::<i32>()
733    }
734
735    #[test]
736    fn without_nulls_large_string() {
737        without_nulls_generic_string::<i64>()
738    }
739
740    fn generic_string_with_non_zero_offset<O: OffsetSizeTrait>() {
741        let values = b"hellotherearrow";
742        let offsets = &[
743            O::zero(),
744            O::from_usize(5).unwrap(),
745            O::from_usize(10).unwrap(),
746            O::from_usize(15).unwrap(),
747        ];
748        // set the first and third element to be valid
749        let bitmap = [0b101_u8];
750
751        let data = ArrayData::builder(GenericStringArray::<O>::DATA_TYPE)
752            .len(2)
753            .add_buffer(Buffer::from_slice_ref(offsets))
754            .add_buffer(Buffer::from(values))
755            .null_bit_buffer(Some(Buffer::from(bitmap)))
756            .offset(1)
757            .build()
758            .unwrap();
759        // array is `[null, "arrow"]`
760        let array = GenericStringArray::<O>::from(data);
761        // result is `[null, "rrow"]`
762        let result = substring(&array, 1, None).unwrap();
763        let result = result
764            .as_any()
765            .downcast_ref::<GenericStringArray<O>>()
766            .unwrap();
767        let expected = GenericStringArray::<O>::from(vec![None, Some("rrow")]);
768        assert_eq!(result, &expected);
769    }
770
771    #[test]
772    fn string_with_non_zero_offset() {
773        generic_string_with_non_zero_offset::<i32>()
774    }
775
776    #[test]
777    fn large_string_with_non_zero_offset() {
778        generic_string_with_non_zero_offset::<i64>()
779    }
780
781    fn with_nulls_generic_string_by_char<O: OffsetSizeTrait>() {
782        let input = vec![Some("hello"), None, Some("Γ ⊢x:T")];
783        // all-nulls array is always identical
784        let base_case = gen_test_cases!(vec![None, None, None], (0, None, vec![None, None, None]));
785        let cases = gen_test_cases!(
786            input,
787            // identity
788            (0, None, input.clone()),
789            // 0 length -> Nothing
790            (0, Some(0), vec![Some(""), None, Some("")]),
791            // high start -> Nothing
792            (1000, Some(0), vec![Some(""), None, Some("")]),
793            // high negative start -> identity
794            (-1000, None, input.clone()),
795            // high length -> identity
796            (0, Some(1000), input.clone())
797        );
798
799        do_test!(
800            [&base_case[..], &cases[..]].concat(),
801            GenericStringArray<O>,
802            substring_by_char
803        );
804    }
805
806    #[test]
807    fn with_nulls_string_by_char() {
808        with_nulls_generic_string_by_char::<i32>()
809    }
810
811    #[test]
812    fn with_nulls_large_string_by_char() {
813        with_nulls_generic_string_by_char::<i64>()
814    }
815
816    fn without_nulls_generic_string_by_char<O: OffsetSizeTrait>() {
817        let input = vec!["hello", "", "Γ ⊢x:T"];
818        // empty array is always identical
819        let base_case = gen_test_cases!(vec!["", "", ""], (0, None, vec!["", "", ""]));
820        let cases = gen_test_cases!(
821            input,
822            //identity
823            (0, None, input.clone()),
824            // increase start
825            (1, None, vec!["ello", "", " ⊢x:T"]),
826            (2, None, vec!["llo", "", "⊢x:T"]),
827            (3, None, vec!["lo", "", "x:T"]),
828            (10, None, vec!["", "", ""]),
829            // increase start negatively
830            (-1, None, vec!["o", "", "T"]),
831            (-2, None, vec!["lo", "", ":T"]),
832            (-4, None, vec!["ello", "", "⊢x:T"]),
833            (-10, None, input.clone()),
834            // increase length
835            (1, Some(1), vec!["e", "", " "]),
836            (1, Some(2), vec!["el", "", " ⊢"]),
837            (1, Some(3), vec!["ell", "", " ⊢x"]),
838            (1, Some(6), vec!["ello", "", " ⊢x:T"]),
839            (-4, Some(1), vec!["e", "", "⊢"]),
840            (-4, Some(2), vec!["el", "", "⊢x"]),
841            (-4, Some(3), vec!["ell", "", "⊢x:"]),
842            (-4, Some(4), vec!["ello", "", "⊢x:T"])
843        );
844
845        do_test!(
846            [&base_case[..], &cases[..]].concat(),
847            GenericStringArray<O>,
848            substring_by_char
849        );
850    }
851
852    #[test]
853    fn without_nulls_string_by_char() {
854        without_nulls_generic_string_by_char::<i32>()
855    }
856
857    #[test]
858    fn without_nulls_large_string_by_char() {
859        without_nulls_generic_string_by_char::<i64>()
860    }
861
862    fn generic_string_by_char_with_non_zero_offset<O: OffsetSizeTrait>() {
863        let values = "S→T = Πx:S.T";
864        let offsets = &[
865            O::zero(),
866            O::from_usize(values.char_indices().nth(3).map(|(pos, _)| pos).unwrap()).unwrap(),
867            O::from_usize(values.char_indices().nth(6).map(|(pos, _)| pos).unwrap()).unwrap(),
868            O::from_usize(values.len()).unwrap(),
869        ];
870        // set the first and third element to be valid
871        let bitmap = [0b101_u8];
872
873        let data = ArrayData::builder(GenericStringArray::<O>::DATA_TYPE)
874            .len(2)
875            .add_buffer(Buffer::from_slice_ref(offsets))
876            .add_buffer(Buffer::from(values.as_bytes()))
877            .null_bit_buffer(Some(Buffer::from(bitmap)))
878            .offset(1)
879            .build()
880            .unwrap();
881        // array is `[null, "Πx:S.T"]`
882        let array = GenericStringArray::<O>::from(data);
883        // result is `[null, "x:S.T"]`
884        let result = substring_by_char(&array, 1, None).unwrap();
885        let expected = GenericStringArray::<O>::from(vec![None, Some("x:S.T")]);
886        assert_eq!(result, expected);
887    }
888
889    #[test]
890    fn string_with_non_zero_offset_by_char() {
891        generic_string_by_char_with_non_zero_offset::<i32>()
892    }
893
894    #[test]
895    fn large_string_with_non_zero_offset_by_char() {
896        generic_string_by_char_with_non_zero_offset::<i64>()
897    }
898
899    #[test]
900    fn dictionary() {
901        _dictionary::<Int8Type>();
902        _dictionary::<Int16Type>();
903        _dictionary::<Int32Type>();
904        _dictionary::<Int64Type>();
905        _dictionary::<UInt8Type>();
906        _dictionary::<UInt16Type>();
907        _dictionary::<UInt32Type>();
908        _dictionary::<UInt64Type>();
909    }
910
911    fn _dictionary<K: ArrowDictionaryKeyType>() {
912        const TOTAL: i32 = 100;
913
914        let v = ["aaa", "bbb", "ccc", "ddd", "eee"];
915        let data: Vec<Option<&str>> = (0..TOTAL)
916            .map(|n| {
917                let i = n % 5;
918                if i == 3 {
919                    None
920                } else {
921                    Some(v[i as usize])
922                }
923            })
924            .collect();
925
926        let dict_array: DictionaryArray<K> = data.clone().into_iter().collect();
927
928        let expected: Vec<Option<&str>> = data.iter().map(|opt| opt.map(|s| &s[1..3])).collect();
929
930        let res = substring(&dict_array, 1, Some(2)).unwrap();
931        let actual = res.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
932        let actual: Vec<Option<&str>> = actual
933            .values()
934            .as_any()
935            .downcast_ref::<GenericStringArray<i32>>()
936            .unwrap()
937            .take_iter(actual.keys_iter())
938            .collect();
939
940        for i in 0..TOTAL as usize {
941            assert_eq!(expected[i], actual[i],);
942        }
943    }
944
945    #[test]
946    fn check_invalid_array_type() {
947        let array = Int32Array::from(vec![Some(1), Some(2), Some(3)]);
948        let err = substring(&array, 0, None).unwrap_err().to_string();
949        assert!(err.contains("substring does not support type"));
950    }
951
952    // tests for the utf-8 validation checking
953    #[test]
954    fn check_start_index() {
955        let array = StringArray::from(vec![Some("E=mc²"), Some("ascii")]);
956        let err = substring(&array, -1, None).unwrap_err().to_string();
957        assert!(err.contains("invalid utf-8 boundary"));
958    }
959
960    #[test]
961    fn check_length() {
962        let array = StringArray::from(vec![Some("E=mc²"), Some("ascii")]);
963        let err = substring(&array, 0, Some(5)).unwrap_err().to_string();
964        assert!(err.contains("invalid utf-8 boundary"));
965    }
966
967    #[test]
968    fn non_utf8_bytes() {
969        // non-utf8 bytes
970        let bytes: &[u8] = &[0xE4, 0xBD, 0xA0, 0xE5, 0xA5, 0xBD, 0xE8, 0xAF, 0xAD];
971        let array = BinaryArray::from(vec![Some(bytes)]);
972        let arr = substring(&array, 0, Some(5)).unwrap();
973        let actual = arr.as_any().downcast_ref::<BinaryArray>().unwrap();
974
975        let expected_bytes: &[u8] = &[0xE4, 0xBD, 0xA0, 0xE5, 0xA5];
976        let expected = BinaryArray::from(vec![Some(expected_bytes)]);
977        assert_eq!(expected, *actual);
978    }
979}