1#![allow(unsafe_op_in_unsafe_fn)]
3use std::iter::Map;
4use std::sync::Arc;
5
6use arrow::array::*;
7use arrow::bitmap::Bitmap;
8use arrow::compute::concatenate::concatenate_unchecked;
9use polars_compute::filter::filter_with_bitmap;
10
11use crate::prelude::*;
12
13pub mod ops;
14#[macro_use]
15pub mod arithmetic;
16pub mod builder;
17pub mod cast;
18pub mod collect;
19pub mod comparison;
20pub mod flags;
21pub mod float;
22pub mod iterator;
23#[cfg(feature = "ndarray")]
24pub(crate) mod ndarray;
25
26#[cfg(feature = "dtype-array")]
27pub(crate) mod array;
28mod binary;
29mod bitwise;
30#[cfg(feature = "object")]
31mod drop;
32mod from;
33mod from_iterator;
34pub mod from_iterator_par;
35pub(crate) mod list;
36pub(crate) mod logical;
37#[cfg(feature = "object")]
38pub mod object;
39#[cfg(feature = "random")]
40mod random;
41#[cfg(feature = "dtype-struct")]
42mod struct_;
43#[cfg(any(
44    feature = "temporal",
45    feature = "dtype-datetime",
46    feature = "dtype-date"
47))]
48pub mod temporal;
49mod to_vec;
50mod trusted_len;
51
52use std::slice::Iter;
53
54use arrow::legacy::prelude::*;
55#[cfg(feature = "dtype-struct")]
56pub use struct_::StructChunked;
57
58use self::flags::{StatisticsFlags, StatisticsFlagsIM};
59use crate::series::IsSorted;
60use crate::utils::{first_non_null, last_non_null};
61
62#[cfg(not(feature = "dtype-categorical"))]
63pub struct RevMapping {}
64
65pub type ChunkLenIter<'a> = std::iter::Map<std::slice::Iter<'a, ArrayRef>, fn(&ArrayRef) -> usize>;
66
67pub struct ChunkedArray<T: PolarsDataType> {
142    pub(crate) field: Arc<Field>,
143    pub(crate) chunks: Vec<ArrayRef>,
144
145    pub(crate) flags: StatisticsFlagsIM,
146
147    length: usize,
148    null_count: usize,
149    _pd: std::marker::PhantomData<T>,
150}
151
152impl<T: PolarsDataType> ChunkedArray<T> {
153    fn should_rechunk(&self) -> bool {
154        self.chunks.len() > 1 && self.chunks.len() > self.len() / 3
155    }
156
157    fn optional_rechunk(mut self) -> Self {
158        if self.should_rechunk() {
160            self.rechunk_mut()
161        }
162        self
163    }
164
165    pub(crate) fn as_any(&self) -> &dyn std::any::Any {
166        self
167    }
168
169    pub fn unpack_series_matching_type<'a>(
171        &self,
172        series: &'a Series,
173    ) -> PolarsResult<&'a ChunkedArray<T>> {
174        match self.dtype() {
175            #[cfg(feature = "dtype-decimal")]
176            DataType::Decimal(_, _) => {
177                let logical = series.decimal()?;
178
179                let ca = logical.physical();
180                Ok(ca.as_any().downcast_ref::<ChunkedArray<T>>().unwrap())
181            },
182            dt => {
183                polars_ensure!(
184                    dt == series.dtype(),
185                    SchemaMismatch: "cannot unpack series of type `{}` into `{}`",
186                    series.dtype(),
187                    dt,
188                );
189
190                Ok(unsafe { self.unpack_series_matching_physical_type(series) })
193            },
194        }
195    }
196
197    fn new_with_compute_len(field: Arc<Field>, chunks: Vec<ArrayRef>) -> Self {
202        unsafe {
203            let mut chunked_arr = Self::new_with_dims(field, chunks, 0, 0);
204            chunked_arr.compute_len();
205            chunked_arr
206        }
207    }
208
209    pub unsafe fn new_with_dims(
213        field: Arc<Field>,
214        chunks: Vec<ArrayRef>,
215        length: usize,
216        null_count: usize,
217    ) -> Self {
218        Self {
219            field,
220            chunks,
221            flags: StatisticsFlagsIM::empty(),
222
223            _pd: Default::default(),
224            length,
225            null_count,
226        }
227    }
228
229    pub(crate) fn is_sorted_ascending_flag(&self) -> bool {
230        self.get_flags().is_sorted_ascending()
231    }
232
233    pub(crate) fn is_sorted_descending_flag(&self) -> bool {
234        self.get_flags().is_sorted_descending()
235    }
236
237    pub(crate) fn is_sorted_any(&self) -> bool {
239        self.get_flags().is_sorted_any()
240    }
241
242    pub fn unset_fast_explode_list(&mut self) {
243        self.set_fast_explode_list(false)
244    }
245
246    pub fn set_fast_explode_list(&mut self, value: bool) {
247        let mut flags = self.flags.get_mut();
248        flags.set(StatisticsFlags::CAN_FAST_EXPLODE_LIST, value);
249        self.flags.set_mut(flags);
250    }
251
252    pub fn get_fast_explode_list(&self) -> bool {
253        self.get_flags().can_fast_explode_list()
254    }
255
256    pub fn get_flags(&self) -> StatisticsFlags {
257        self.flags.get()
258    }
259
260    pub(crate) fn set_flags(&mut self, flags: StatisticsFlags) {
262        self.flags = StatisticsFlagsIM::new(flags);
263    }
264
265    pub fn is_sorted_flag(&self) -> IsSorted {
266        self.get_flags().is_sorted()
267    }
268
269    pub fn retain_flags_from<U: PolarsDataType>(
270        &mut self,
271        from: &ChunkedArray<U>,
272        retain_flags: StatisticsFlags,
273    ) {
274        let flags = from.flags.get();
275        if !flags.is_empty() {
277            self.set_flags(flags & retain_flags)
278        }
279    }
280
281    pub fn set_sorted_flag(&mut self, sorted: IsSorted) {
283        let mut flags = self.flags.get_mut();
284        flags.set_sorted(sorted);
285        self.flags.set_mut(flags);
286    }
287
288    pub fn with_sorted_flag(&self, sorted: IsSorted) -> Self {
290        let mut out = self.clone();
291        out.set_sorted_flag(sorted);
292        out
293    }
294
295    pub fn first_non_null(&self) -> Option<usize> {
297        if self.null_count() == self.len() {
298            None
299        }
300        else if self.null_count() == 0 {
302            Some(0)
303        } else if self.is_sorted_any() {
304            let out = if unsafe { self.downcast_get_unchecked(0).is_null_unchecked(0) } {
305                self.null_count()
307            } else {
308                0
310            };
311
312            debug_assert!(
313                unsafe { self.get_unchecked(out) }.is_some(),
315                "incorrect sorted flag"
316            );
317
318            Some(out)
319        } else {
320            first_non_null(self.iter_validities())
321        }
322    }
323
324    pub fn last_non_null(&self) -> Option<usize> {
326        if self.null_count() == self.len() {
327            None
328        }
329        else if self.null_count() == 0 {
331            Some(self.len() - 1)
332        } else if self.is_sorted_any() {
333            let out = if unsafe { self.downcast_get_unchecked(0).is_null_unchecked(0) } {
334                self.len() - 1
336            } else {
337                self.len() - self.null_count() - 1
339            };
340
341            debug_assert!(
342                unsafe { self.get_unchecked(out) }.is_some(),
344                "incorrect sorted flag"
345            );
346
347            Some(out)
348        } else {
349            last_non_null(self.iter_validities(), self.len())
350        }
351    }
352
353    pub fn drop_nulls(&self) -> Self {
354        if self.null_count() == 0 {
355            self.clone()
356        } else {
357            let chunks = self
358                .downcast_iter()
359                .map(|arr| {
360                    if arr.null_count() == 0 {
361                        arr.to_boxed()
362                    } else {
363                        filter_with_bitmap(arr, arr.validity().unwrap())
364                    }
365                })
366                .collect();
367            unsafe {
368                Self::new_with_dims(
369                    self.field.clone(),
370                    chunks,
371                    self.len() - self.null_count(),
372                    0,
373                )
374            }
375        }
376    }
377
378    #[inline]
380    #[allow(clippy::type_complexity)]
381    pub fn iter_validities(&self) -> Map<Iter<'_, ArrayRef>, fn(&ArrayRef) -> Option<&Bitmap>> {
382        fn to_validity(arr: &ArrayRef) -> Option<&Bitmap> {
383            arr.validity()
384        }
385        self.chunks.iter().map(to_validity)
386    }
387
388    #[inline]
389    pub fn has_nulls(&self) -> bool {
391        self.null_count > 0
392    }
393
394    pub fn shrink_to_fit(&mut self) {
396        self.chunks = vec![concatenate_unchecked(self.chunks.as_slice()).unwrap()];
397    }
398
399    pub fn clear(&self) -> Self {
400        let mut ca = unsafe {
402            self.copy_with_chunks(vec![new_empty_array(
403                self.chunks.first().unwrap().dtype().clone(),
404            )])
405        };
406
407        use StatisticsFlags as F;
408        ca.retain_flags_from(self, F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST);
409        ca
410    }
411
412    pub(crate) unsafe fn unpack_series_matching_physical_type<'a>(
419        &self,
420        series: &'a Series,
421    ) -> &'a ChunkedArray<T> {
422        let series_trait = &**series;
423        if self.dtype() == series.dtype() {
424            &*(series_trait as *const dyn SeriesTrait as *const ChunkedArray<T>)
425        } else {
426            use DataType::*;
427            match (self.dtype(), series.dtype()) {
428                (Int64, Datetime(_, _)) | (Int64, Duration(_)) | (Int32, Date) => {
429                    &*(series_trait as *const dyn SeriesTrait as *const ChunkedArray<T>)
430                },
431                _ => panic!(
432                    "cannot unpack series {:?} into matching type {:?}",
433                    series,
434                    self.dtype()
435                ),
436            }
437        }
438    }
439
440    pub fn chunk_lengths(&self) -> ChunkLenIter {
442        self.chunks.iter().map(|chunk| chunk.len())
443    }
444
445    #[inline]
447    pub fn chunks(&self) -> &Vec<ArrayRef> {
448        &self.chunks
449    }
450
451    #[inline]
457    pub unsafe fn chunks_mut(&mut self) -> &mut Vec<ArrayRef> {
458        &mut self.chunks
459    }
460
461    pub fn is_optimal_aligned(&self) -> bool {
463        self.chunks.len() == 1 && self.null_count() == 0
464    }
465
466    unsafe fn copy_with_chunks(&self, chunks: Vec<ArrayRef>) -> Self {
471        Self::new_with_compute_len(self.field.clone(), chunks)
472    }
473
474    pub fn dtype(&self) -> &DataType {
476        self.field.dtype()
477    }
478
479    pub(crate) unsafe fn set_dtype(&mut self, dtype: DataType) {
480        self.field = Arc::new(Field::new(self.name().clone(), dtype))
481    }
482
483    pub fn name(&self) -> &PlSmallStr {
485        self.field.name()
486    }
487
488    pub fn ref_field(&self) -> &Field {
490        &self.field
491    }
492
493    pub fn rename(&mut self, name: PlSmallStr) {
495        self.field = Arc::new(Field::new(name, self.field.dtype().clone()));
496    }
497
498    pub fn with_name(mut self, name: PlSmallStr) -> Self {
500        self.rename(name);
501        self
502    }
503}
504
505impl<T> ChunkedArray<T>
506where
507    T: PolarsDataType,
508{
509    #[inline]
515    pub fn get(&self, idx: usize) -> Option<T::Physical<'_>> {
516        let (chunk_idx, arr_idx) = self.index_to_chunked_index(idx);
517        assert!(
518            chunk_idx < self.chunks().len(),
519            "index: {} out of bounds for len: {}",
520            idx,
521            self.len()
522        );
523        unsafe {
524            let arr = self.downcast_get_unchecked(chunk_idx);
525            assert!(
526                arr_idx < arr.len(),
527                "index: {} out of bounds for len: {}",
528                idx,
529                self.len()
530            );
531            arr.get_unchecked(arr_idx)
532        }
533    }
534
535    #[inline]
541    pub unsafe fn get_unchecked(&self, idx: usize) -> Option<T::Physical<'_>> {
542        let (chunk_idx, arr_idx) = self.index_to_chunked_index(idx);
543
544        unsafe {
545            self.downcast_get_unchecked(chunk_idx)
547                .get_unchecked(arr_idx)
548        }
549    }
550
551    #[inline]
557    pub unsafe fn value_unchecked(&self, idx: usize) -> T::Physical<'_> {
558        let (chunk_idx, arr_idx) = self.index_to_chunked_index(idx);
559
560        unsafe {
561            self.downcast_get_unchecked(chunk_idx)
563                .value_unchecked(arr_idx)
564        }
565    }
566
567    #[inline]
568    pub fn first(&self) -> Option<T::Physical<'_>> {
569        unsafe {
570            let arr = self.downcast_get_unchecked(0);
571            arr.get_unchecked(0)
572        }
573    }
574
575    #[inline]
576    pub fn last(&self) -> Option<T::Physical<'_>> {
577        unsafe {
578            let arr = self.downcast_get_unchecked(self.chunks.len().checked_sub(1)?);
579            arr.get_unchecked(arr.len().checked_sub(1)?)
580        }
581    }
582}
583
584impl ListChunked {
585    #[inline]
586    pub fn get_as_series(&self, idx: usize) -> Option<Series> {
587        unsafe {
588            Some(Series::from_chunks_and_dtype_unchecked(
589                self.name().clone(),
590                vec![self.get(idx)?],
591                &self.inner_dtype().to_physical(),
592            ))
593        }
594    }
595}
596
597#[cfg(feature = "dtype-array")]
598impl ArrayChunked {
599    #[inline]
600    pub fn get_as_series(&self, idx: usize) -> Option<Series> {
601        unsafe {
602            Some(Series::from_chunks_and_dtype_unchecked(
603                self.name().clone(),
604                vec![self.get(idx)?],
605                &self.inner_dtype().to_physical(),
606            ))
607        }
608    }
609}
610
611impl<T> ChunkedArray<T>
612where
613    T: PolarsDataType,
614{
615    pub(crate) fn match_chunks<I>(&self, chunk_id: I) -> Self
619    where
620        I: Iterator<Item = usize>,
621    {
622        debug_assert!(self.chunks.len() == 1);
623        let slice = |ca: &Self| {
625            let array = &ca.chunks[0];
626
627            let mut offset = 0;
628            let chunks = chunk_id
629                .map(|len| {
630                    debug_assert!((offset + len) <= array.len());
632                    let out = unsafe { array.sliced_unchecked(offset, len) };
633                    offset += len;
634                    out
635                })
636                .collect();
637
638            debug_assert_eq!(offset, array.len());
639
640            unsafe {
642                Self::from_chunks_and_dtype(self.name().clone(), chunks, self.dtype().clone())
643            }
644        };
645
646        if self.chunks.len() != 1 {
647            let out = self.rechunk();
648            slice(&out)
649        } else {
650            slice(self)
651        }
652    }
653}
654
655impl<T: PolarsDataType> AsRefDataType for ChunkedArray<T> {
656    fn as_ref_dtype(&self) -> &DataType {
657        self.dtype()
658    }
659}
660
661pub(crate) trait AsSinglePtr: AsRefDataType {
662    fn as_single_ptr(&mut self) -> PolarsResult<usize> {
664        polars_bail!(opq = as_single_ptr, self.as_ref_dtype());
665    }
666}
667
668impl<T> AsSinglePtr for ChunkedArray<T>
669where
670    T: PolarsNumericType,
671{
672    fn as_single_ptr(&mut self) -> PolarsResult<usize> {
673        self.rechunk_mut();
674        let a = self.data_views().next().unwrap();
675        let ptr = a.as_ptr();
676        Ok(ptr as usize)
677    }
678}
679
680impl AsSinglePtr for BooleanChunked {}
681impl AsSinglePtr for ListChunked {}
682#[cfg(feature = "dtype-array")]
683impl AsSinglePtr for ArrayChunked {}
684impl AsSinglePtr for StringChunked {}
685impl AsSinglePtr for BinaryChunked {}
686#[cfg(feature = "object")]
687impl<T: PolarsObject> AsSinglePtr for ObjectChunked<T> {}
688
689pub enum ChunkedArrayLayout<'a, T: PolarsDataType> {
690    SingleNoNull(&'a T::Array),
691    Single(&'a T::Array),
692    MultiNoNull(&'a ChunkedArray<T>),
693    Multi(&'a ChunkedArray<T>),
694}
695
696impl<T> ChunkedArray<T>
697where
698    T: PolarsDataType,
699{
700    pub fn layout(&self) -> ChunkedArrayLayout<'_, T> {
701        if self.chunks.len() == 1 {
702            let arr = self.downcast_iter().next().unwrap();
703            return if arr.null_count() == 0 {
704                ChunkedArrayLayout::SingleNoNull(arr)
705            } else {
706                ChunkedArrayLayout::Single(arr)
707            };
708        }
709
710        if self.downcast_iter().all(|a| a.null_count() == 0) {
711            ChunkedArrayLayout::MultiNoNull(self)
712        } else {
713            ChunkedArrayLayout::Multi(self)
714        }
715    }
716}
717
718impl<T> ChunkedArray<T>
719where
720    T: PolarsNumericType,
721{
722    pub fn cont_slice(&self) -> PolarsResult<&[T::Native]> {
724        polars_ensure!(
725            self.chunks.len() == 1 && self.chunks[0].null_count() == 0,
726            ComputeError: "chunked array is not contiguous"
727        );
728        Ok(self.downcast_iter().next().map(|arr| arr.values()).unwrap())
729    }
730
731    pub(crate) fn cont_slice_mut(&mut self) -> Option<&mut [T::Native]> {
733        if self.chunks.len() == 1 && self.chunks[0].null_count() == 0 {
734            let arr = unsafe { self.downcast_iter_mut().next().unwrap() };
736            arr.get_mut_values()
737        } else {
738            None
739        }
740    }
741
742    pub fn data_views(&self) -> impl DoubleEndedIterator<Item = &[T::Native]> {
746        self.downcast_iter().map(|arr| arr.values().as_slice())
747    }
748
749    #[allow(clippy::wrong_self_convention)]
750    pub fn into_no_null_iter(
751        &self,
752    ) -> impl '_ + Send + Sync + ExactSizeIterator<Item = T::Native> + DoubleEndedIterator + TrustedLen
753    {
754        #[allow(clippy::map_clone)]
756        unsafe {
758            self.data_views()
759                .flatten()
760                .map(|v| *v)
761                .trust_my_length(self.len())
762        }
763    }
764}
765
766impl<T: PolarsDataType> Clone for ChunkedArray<T> {
767    fn clone(&self) -> Self {
768        ChunkedArray {
769            field: self.field.clone(),
770            chunks: self.chunks.clone(),
771            flags: self.flags.clone(),
772
773            _pd: Default::default(),
774            length: self.length,
775            null_count: self.null_count,
776        }
777    }
778}
779
780impl<T: PolarsDataType> AsRef<ChunkedArray<T>> for ChunkedArray<T> {
781    fn as_ref(&self) -> &ChunkedArray<T> {
782        self
783    }
784}
785
786impl ValueSize for ListChunked {
787    fn get_values_size(&self) -> usize {
788        self.chunks
789            .iter()
790            .fold(0usize, |acc, arr| acc + arr.get_values_size())
791    }
792}
793
794#[cfg(feature = "dtype-array")]
795impl ValueSize for ArrayChunked {
796    fn get_values_size(&self) -> usize {
797        self.chunks
798            .iter()
799            .fold(0usize, |acc, arr| acc + arr.get_values_size())
800    }
801}
802impl ValueSize for StringChunked {
803    fn get_values_size(&self) -> usize {
804        self.chunks
805            .iter()
806            .fold(0usize, |acc, arr| acc + arr.get_values_size())
807    }
808}
809
810impl ValueSize for BinaryOffsetChunked {
811    fn get_values_size(&self) -> usize {
812        self.chunks
813            .iter()
814            .fold(0usize, |acc, arr| acc + arr.get_values_size())
815    }
816}
817
818pub(crate) fn to_primitive<T: PolarsNumericType>(
819    values: Vec<T::Native>,
820    validity: Option<Bitmap>,
821) -> PrimitiveArray<T::Native> {
822    PrimitiveArray::new(
823        T::get_dtype().to_arrow(CompatLevel::newest()),
824        values.into(),
825        validity,
826    )
827}
828
829pub(crate) fn to_array<T: PolarsNumericType>(
830    values: Vec<T::Native>,
831    validity: Option<Bitmap>,
832) -> ArrayRef {
833    Box::new(to_primitive::<T>(values, validity))
834}
835
836impl<T: PolarsDataType> Default for ChunkedArray<T> {
837    fn default() -> Self {
838        let dtype = T::get_dtype();
839        let arrow_dtype = dtype.to_physical().to_arrow(CompatLevel::newest());
840        ChunkedArray {
841            field: Arc::new(Field::new(PlSmallStr::EMPTY, dtype)),
842            chunks: vec![new_empty_array(arrow_dtype)],
844            flags: StatisticsFlagsIM::empty(),
845
846            _pd: Default::default(),
847            length: 0,
848            null_count: 0,
849        }
850    }
851}
852
853#[cfg(test)]
854pub(crate) mod test {
855    use crate::prelude::*;
856
857    pub(crate) fn get_chunked_array() -> Int32Chunked {
858        ChunkedArray::new(PlSmallStr::from_static("a"), &[1, 2, 3])
859    }
860
861    #[test]
862    fn test_sort() {
863        let a = Int32Chunked::new(PlSmallStr::from_static("a"), &[1, 9, 3, 2]);
864        let b = a
865            .sort(false)
866            .into_iter()
867            .map(|opt| opt.unwrap())
868            .collect::<Vec<_>>();
869        assert_eq!(b, [1, 2, 3, 9]);
870        let a = StringChunked::new(PlSmallStr::from_static("a"), &["b", "a", "c"]);
871        let a = a.sort(false);
872        let b = a.into_iter().collect::<Vec<_>>();
873        assert_eq!(b, [Some("a"), Some("b"), Some("c")]);
874        assert!(a.is_sorted_ascending_flag());
875    }
876
877    #[test]
878    fn arithmetic() {
879        let a = &Int32Chunked::new(PlSmallStr::from_static("a"), &[1, 100, 6, 40]);
880        let b = &Int32Chunked::new(PlSmallStr::from_static("b"), &[-1, 2, 3, 4]);
881
882        println!("{:?}", a + b);
885        println!("{:?}", a - b);
886        println!("{:?}", a * b);
887        println!("{:?}", a / b);
888    }
889
890    #[test]
891    fn iter() {
892        let s1 = get_chunked_array();
893        assert_eq!(s1.into_iter().fold(0, |acc, val| { acc + val.unwrap() }), 6)
895    }
896
897    #[test]
898    fn limit() {
899        let a = get_chunked_array();
900        let b = a.limit(2);
901        println!("{:?}", b);
902        assert_eq!(b.len(), 2)
903    }
904
905    #[test]
906    fn filter() {
907        let a = get_chunked_array();
908        let b = a
909            .filter(&BooleanChunked::new(
910                PlSmallStr::from_static("filter"),
911                &[true, false, false],
912            ))
913            .unwrap();
914        assert_eq!(b.len(), 1);
915        assert_eq!(b.into_iter().next(), Some(Some(1)));
916    }
917
918    #[test]
919    fn aggregates() {
920        let a = &Int32Chunked::new(PlSmallStr::from_static("a"), &[1, 100, 10, 9]);
921        assert_eq!(a.max(), Some(100));
922        assert_eq!(a.min(), Some(1));
923        assert_eq!(a.sum(), Some(120))
924    }
925
926    #[test]
927    fn take() {
928        let a = get_chunked_array();
929        let new = a.take(&[0 as IdxSize, 1]).unwrap();
930        assert_eq!(new.len(), 2)
931    }
932
933    #[test]
934    fn cast() {
935        let a = get_chunked_array();
936        let b = a.cast(&DataType::Int64).unwrap();
937        assert_eq!(b.dtype(), &DataType::Int64)
938    }
939
940    fn assert_slice_equal<T>(ca: &ChunkedArray<T>, eq: &[T::Native])
941    where
942        T: PolarsNumericType,
943    {
944        assert_eq!(ca.iter().map(|opt| opt.unwrap()).collect::<Vec<_>>(), eq)
945    }
946
947    #[test]
948    fn slice() {
949        let mut first = UInt32Chunked::new(PlSmallStr::from_static("first"), &[0, 1, 2]);
950        let second = UInt32Chunked::new(PlSmallStr::from_static("second"), &[3, 4, 5]);
951        first.append(&second).unwrap();
952        assert_slice_equal(&first.slice(0, 3), &[0, 1, 2]);
953        assert_slice_equal(&first.slice(0, 4), &[0, 1, 2, 3]);
954        assert_slice_equal(&first.slice(1, 4), &[1, 2, 3, 4]);
955        assert_slice_equal(&first.slice(3, 2), &[3, 4]);
956        assert_slice_equal(&first.slice(3, 3), &[3, 4, 5]);
957        assert_slice_equal(&first.slice(-3, 3), &[3, 4, 5]);
958        assert_slice_equal(&first.slice(-6, 6), &[0, 1, 2, 3, 4, 5]);
959
960        assert_eq!(first.slice(-7, 2).len(), 1);
961        assert_eq!(first.slice(-3, 4).len(), 3);
962        assert_eq!(first.slice(3, 4).len(), 3);
963        assert_eq!(first.slice(10, 4).len(), 0);
964    }
965
966    #[test]
967    fn sorting() {
968        let s = UInt32Chunked::new(PlSmallStr::EMPTY, &[9, 2, 4]);
969        let sorted = s.sort(false);
970        assert_slice_equal(&sorted, &[2, 4, 9]);
971        let sorted = s.sort(true);
972        assert_slice_equal(&sorted, &[9, 4, 2]);
973
974        let s: StringChunked = ["b", "a", "z"].iter().collect();
975        let sorted = s.sort(false);
976        assert_eq!(
977            sorted.into_iter().collect::<Vec<_>>(),
978            &[Some("a"), Some("b"), Some("z")]
979        );
980        let sorted = s.sort(true);
981        assert_eq!(
982            sorted.into_iter().collect::<Vec<_>>(),
983            &[Some("z"), Some("b"), Some("a")]
984        );
985        let s: StringChunked = [Some("b"), None, Some("z")].iter().copied().collect();
986        let sorted = s.sort(false);
987        assert_eq!(
988            sorted.into_iter().collect::<Vec<_>>(),
989            &[None, Some("b"), Some("z")]
990        );
991    }
992
993    #[test]
994    fn reverse() {
995        let s = UInt32Chunked::new(PlSmallStr::EMPTY, &[1, 2, 3]);
996        assert_slice_equal(&s.reverse(), &[3, 2, 1]);
998        let s = UInt32Chunked::new(PlSmallStr::EMPTY, &[Some(1), None, Some(3)]);
1000        assert_eq!(Vec::from(&s.reverse()), &[Some(3), None, Some(1)]);
1001        let s = BooleanChunked::new(PlSmallStr::EMPTY, &[true, false]);
1002        assert_eq!(Vec::from(&s.reverse()), &[Some(false), Some(true)]);
1003
1004        let s = StringChunked::new(PlSmallStr::EMPTY, &["a", "b", "c"]);
1005        assert_eq!(Vec::from(&s.reverse()), &[Some("c"), Some("b"), Some("a")]);
1006
1007        let s = StringChunked::new(PlSmallStr::EMPTY, &[Some("a"), None, Some("c")]);
1008        assert_eq!(Vec::from(&s.reverse()), &[Some("c"), None, Some("a")]);
1009    }
1010
1011    #[test]
1012    #[cfg(feature = "dtype-categorical")]
1013    fn test_iter_categorical() {
1014        use crate::{SINGLE_LOCK, disable_string_cache};
1015        let _lock = SINGLE_LOCK.lock();
1016        disable_string_cache();
1017        let ca = StringChunked::new(
1018            PlSmallStr::EMPTY,
1019            &[Some("foo"), None, Some("bar"), Some("ham")],
1020        );
1021        let ca = ca
1022            .cast(&DataType::Categorical(None, Default::default()))
1023            .unwrap();
1024        let ca = ca.categorical().unwrap();
1025        let v: Vec<_> = ca.physical().into_iter().collect();
1026        assert_eq!(v, &[Some(0), None, Some(1), Some(2)]);
1027    }
1028
1029    #[test]
1030    #[ignore]
1031    fn test_shrink_to_fit() {
1032        let mut builder = StringChunkedBuilder::new(PlSmallStr::from_static("foo"), 2048);
1033        builder.append_value("foo");
1034        let mut arr = builder.finish();
1035        let before = arr
1036            .chunks()
1037            .iter()
1038            .map(|arr| arrow::compute::aggregate::estimated_bytes_size(arr.as_ref()))
1039            .sum::<usize>();
1040        arr.shrink_to_fit();
1041        let after = arr
1042            .chunks()
1043            .iter()
1044            .map(|arr| arrow::compute::aggregate::estimated_bytes_size(arr.as_ref()))
1045            .sum::<usize>();
1046        assert!(before > after);
1047    }
1048}