arrow_rayon/parallel_array/
parallel_string_array.rs1use arrow_array::types::GenericStringType;
2use arrow_array::{LargeStringArray, StringArray};
3use rayon::iter::ParallelIterator;
4
5use crate::parallel_byte_array::{ParallelGenericByteArray, ParallelGenericByteArrayRef};
6
7pub type ParallelGenericStringArray<OffsetSize> =
8 ParallelGenericByteArray<GenericStringType<OffsetSize>>;
9pub type ParallelGenericStringArrayRef<'data, OffsetSize> =
10 ParallelGenericByteArrayRef<'data, GenericStringType<OffsetSize>>;
11
12pub type ParallelStringArray = ParallelGenericStringArray<i32>;
13pub type ParallelStringArrayRef<'data> = ParallelGenericStringArrayRef<'data, i32>;
14pub type ParallelLargeStringArray = ParallelGenericStringArray<i64>;
15pub type ParallelLargeStringArrayRef<'data> = ParallelGenericStringArrayRef<'data, i64>;
16
17pub trait StringArrayRefParallelIterator<'data> {
18 type Iter: ParallelIterator<Item = Option<&'data str>>;
19
20 fn par_iter(&'data self) -> Self::Iter;
21}
22
23impl<'data> StringArrayRefParallelIterator<'data> for StringArray {
24 type Iter = ParallelStringArrayRef<'data>;
25
26 fn par_iter(&'data self) -> Self::Iter {
27 ParallelStringArrayRef::new(self)
28 }
29}
30
31pub trait LargeStringArrayRefParallelIterator<'data> {
32 type Iter: ParallelIterator<Item = Option<&'data str>>;
33
34 fn par_iter(&'data self) -> Self::Iter;
35}
36
37impl<'data> LargeStringArrayRefParallelIterator<'data> for LargeStringArray {
38 type Iter = ParallelLargeStringArrayRef<'data>;
39
40 fn par_iter(&'data self) -> Self::Iter {
41 ParallelLargeStringArrayRef::new(self)
42 }
43}
44
45#[cfg(test)]
46mod tests {
47 use arrow_array::Array;
48
49 use super::*;
50
51 #[test]
52 fn test_par_iter() {
53 let array = StringArray::from(vec![Some("one"), None, Some("two"), Some("three")]);
54 let items: Vec<String> = array
55 .par_iter()
56 .map(|item| item.map_or_else(String::new, |item| item.to_uppercase()))
57 .collect();
58 assert_eq!(
59 items,
60 vec![
61 "ONE".to_owned(),
62 "".to_owned(),
63 "TWO".to_owned(),
64 "THREE".to_owned()
65 ]
66 );
67 }
68
69 #[test]
70 fn test_collect_array() {
71 let array = StringArray::from(vec![Some("one"), None, Some("two"), Some("three")]);
72 let collected_array: ParallelStringArray = array
73 .par_iter()
74 .map(|item| item.map(|item| item.to_uppercase()))
75 .collect();
76 let string_array = collected_array.into_inner();
77 assert_eq!(string_array.value(0), "ONE");
78 assert!(string_array.is_null(1));
79 assert_eq!(string_array.value(2), "TWO");
80 assert_eq!(string_array.value(3), "THREE");
81 }
82}