polars_core/chunked_array/
cast.rs

1//! Implementations of the ChunkCast Trait.
2
3use std::borrow::Cow;
4
5use polars_compute::cast::CastOptionsImpl;
6#[cfg(feature = "serde-lazy")]
7use serde::{Deserialize, Serialize};
8
9use super::flags::StatisticsFlags;
10#[cfg(feature = "timezones")]
11use crate::chunked_array::temporal::validate_time_zone;
12#[cfg(feature = "dtype-datetime")]
13use crate::prelude::DataType::Datetime;
14use crate::prelude::*;
15
16#[derive(Copy, Clone, Debug, Default, PartialEq, Hash, Eq)]
17#[cfg_attr(feature = "serde-lazy", derive(Serialize, Deserialize))]
18#[repr(u8)]
19pub enum CastOptions {
20    /// Raises on overflow
21    #[default]
22    Strict,
23    /// Overflow is replaced with null
24    NonStrict,
25    /// Allows wrapping overflow
26    Overflowing,
27}
28
29impl CastOptions {
30    pub fn is_strict(&self) -> bool {
31        matches!(self, CastOptions::Strict)
32    }
33}
34
35impl From<CastOptions> for CastOptionsImpl {
36    fn from(value: CastOptions) -> Self {
37        let wrapped = match value {
38            CastOptions::Strict | CastOptions::NonStrict => false,
39            CastOptions::Overflowing => true,
40        };
41        CastOptionsImpl {
42            wrapped,
43            partial: false,
44        }
45    }
46}
47
48pub(crate) fn cast_chunks(
49    chunks: &[ArrayRef],
50    dtype: &DataType,
51    options: CastOptions,
52) -> PolarsResult<Vec<ArrayRef>> {
53    let check_nulls = matches!(options, CastOptions::Strict);
54    let options = options.into();
55
56    let arrow_dtype = dtype.try_to_arrow(CompatLevel::newest())?;
57    chunks
58        .iter()
59        .map(|arr| {
60            let out = polars_compute::cast::cast(arr.as_ref(), &arrow_dtype, options);
61            if check_nulls {
62                out.and_then(|new| {
63                    polars_ensure!(arr.null_count() == new.null_count(), ComputeError: "strict cast failed");
64                    Ok(new)
65                })
66
67            } else {
68                out
69            }
70        })
71        .collect::<PolarsResult<Vec<_>>>()
72}
73
74fn cast_impl_inner(
75    name: PlSmallStr,
76    chunks: &[ArrayRef],
77    dtype: &DataType,
78    options: CastOptions,
79) -> PolarsResult<Series> {
80    let chunks = match dtype {
81        #[cfg(feature = "dtype-decimal")]
82        DataType::Decimal(_, _) => {
83            let mut chunks = cast_chunks(chunks, dtype, options)?;
84            // @NOTE: We cannot cast here as that will lower the scale.
85            for chunk in chunks.iter_mut() {
86                *chunk = std::mem::take(
87                    chunk
88                        .as_any_mut()
89                        .downcast_mut::<PrimitiveArray<i128>>()
90                        .unwrap(),
91                )
92                .to(ArrowDataType::Int128)
93                .to_boxed();
94            }
95            chunks
96        },
97        _ => cast_chunks(chunks, &dtype.to_physical(), options)?,
98    };
99
100    let out = Series::try_from((name, chunks))?;
101    use DataType::*;
102    let out = match dtype {
103        Date => out.into_date(),
104        Datetime(tu, tz) => match tz {
105            #[cfg(feature = "timezones")]
106            Some(tz) => {
107                validate_time_zone(tz)?;
108                out.into_datetime(*tu, Some(tz.clone()))
109            },
110            _ => out.into_datetime(*tu, None),
111        },
112        Duration(tu) => out.into_duration(*tu),
113        #[cfg(feature = "dtype-time")]
114        Time => out.into_time(),
115        #[cfg(feature = "dtype-decimal")]
116        Decimal(precision, scale) => out.into_decimal(*precision, scale.unwrap_or(0))?,
117        _ => out,
118    };
119
120    Ok(out)
121}
122
123fn cast_impl(
124    name: PlSmallStr,
125    chunks: &[ArrayRef],
126    dtype: &DataType,
127    options: CastOptions,
128) -> PolarsResult<Series> {
129    cast_impl_inner(name, chunks, dtype, options)
130}
131
132#[cfg(feature = "dtype-struct")]
133fn cast_single_to_struct(
134    name: PlSmallStr,
135    chunks: &[ArrayRef],
136    fields: &[Field],
137    options: CastOptions,
138) -> PolarsResult<Series> {
139    polars_ensure!(fields.len() == 1, InvalidOperation: "must specify one field in the struct");
140    let mut new_fields = Vec::with_capacity(fields.len());
141    // cast to first field dtype
142    let mut fields = fields.iter();
143    let fld = fields.next().unwrap();
144    let s = cast_impl_inner(fld.name.clone(), chunks, &fld.dtype, options)?;
145    let length = s.len();
146    new_fields.push(s);
147
148    for fld in fields {
149        new_fields.push(Series::full_null(fld.name.clone(), length, &fld.dtype));
150    }
151
152    StructChunked::from_series(name, length, new_fields.iter()).map(|ca| ca.into_series())
153}
154
155impl<T> ChunkedArray<T>
156where
157    T: PolarsNumericType,
158{
159    fn cast_impl(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
160        if self.dtype() == dtype {
161            // SAFETY: chunks are correct dtype
162            let mut out = unsafe {
163                Series::from_chunks_and_dtype_unchecked(
164                    self.name().clone(),
165                    self.chunks.clone(),
166                    dtype,
167                )
168            };
169            out.set_sorted_flag(self.is_sorted_flag());
170            return Ok(out);
171        }
172        match dtype {
173            #[cfg(feature = "dtype-categorical")]
174            DataType::Categorical(_, ordering) => {
175                polars_ensure!(
176                    self.dtype() == &DataType::UInt32,
177                    ComputeError: "cannot cast numeric types to 'Categorical'"
178                );
179                // SAFETY:
180                // we are guarded by the type system
181                let ca = unsafe { &*(self as *const ChunkedArray<T> as *const UInt32Chunked) };
182
183                CategoricalChunked::from_global_indices(ca.clone(), *ordering)
184                    .map(|ca| ca.into_series())
185            },
186            #[cfg(feature = "dtype-categorical")]
187            DataType::Enum(rev_map, ordering) => {
188                let ca = match self.dtype() {
189                    DataType::UInt32 => {
190                        // SAFETY: we are guarded by the type system
191                        unsafe { &*(self as *const ChunkedArray<T> as *const UInt32Chunked) }
192                            .clone()
193                    },
194                    dt if dt.is_integer() => self
195                        .cast_with_options(self.dtype(), options)?
196                        .strict_cast(&DataType::UInt32)?
197                        .u32()?
198                        .clone(),
199                    _ => {
200                        polars_bail!(ComputeError: "cannot cast non integer types to 'Enum'")
201                    },
202                };
203                let Some(rev_map) = rev_map else {
204                    polars_bail!(ComputeError: "cannot cast to Enum without categories");
205                };
206                let categories = rev_map.get_categories();
207                // Check if indices are in bounds
208                if let Some(m) = ChunkAgg::max(&ca) {
209                    if m >= categories.len() as u32 {
210                        polars_bail!(OutOfBounds: "index {} is bigger than the number of categories {}",m,categories.len());
211                    }
212                }
213                // SAFETY: indices are in bound
214                unsafe {
215                    Ok(CategoricalChunked::from_cats_and_rev_map_unchecked(
216                        ca.clone(),
217                        rev_map.clone(),
218                        true,
219                        *ordering,
220                    )
221                    .into_series())
222                }
223            },
224            #[cfg(feature = "dtype-struct")]
225            DataType::Struct(fields) => {
226                cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)
227            },
228            _ => cast_impl_inner(self.name().clone(), &self.chunks, dtype, options).map(|mut s| {
229                // maintain sorted if data types
230                // - remain signed
231                // - unsigned -> signed
232                // this may still fail with overflow?
233                let to_signed = dtype.is_signed_integer();
234                let unsigned2unsigned =
235                    self.dtype().is_unsigned_integer() && dtype.is_unsigned_integer();
236                let allowed = to_signed || unsigned2unsigned;
237
238                if (allowed)
239                    && (s.null_count() == self.null_count())
240                    // physical to logicals
241                    || (self.dtype().to_physical() == dtype.to_physical())
242                {
243                    let is_sorted = self.is_sorted_flag();
244                    s.set_sorted_flag(is_sorted)
245                }
246                s
247            }),
248        }
249    }
250}
251
252impl<T> ChunkCast for ChunkedArray<T>
253where
254    T: PolarsNumericType,
255{
256    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
257        self.cast_impl(dtype, options)
258    }
259
260    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
261        match dtype {
262            #[cfg(feature = "dtype-categorical")]
263            DataType::Categorical(Some(rev_map), ordering)
264            | DataType::Enum(Some(rev_map), ordering) => {
265                if self.dtype() == &DataType::UInt32 {
266                    // SAFETY:
267                    // we are guarded by the type system.
268                    let ca = unsafe { &*(self as *const ChunkedArray<T> as *const UInt32Chunked) };
269                    Ok(unsafe {
270                        CategoricalChunked::from_cats_and_rev_map_unchecked(
271                            ca.clone(),
272                            rev_map.clone(),
273                            matches!(dtype, DataType::Enum(_, _)),
274                            *ordering,
275                        )
276                    }
277                    .into_series())
278                } else {
279                    polars_bail!(ComputeError: "cannot cast numeric types to 'Categorical'");
280                }
281            },
282            _ => self.cast_impl(dtype, CastOptions::Overflowing),
283        }
284    }
285}
286
287impl ChunkCast for StringChunked {
288    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
289        match dtype {
290            #[cfg(feature = "dtype-categorical")]
291            DataType::Categorical(rev_map, ordering) => match rev_map {
292                None => {
293                    // SAFETY: length is correct
294                    let iter =
295                        unsafe { self.downcast_iter().flatten().trust_my_length(self.len()) };
296                    let builder =
297                        CategoricalChunkedBuilder::new(self.name().clone(), self.len(), *ordering);
298                    let ca = builder.drain_iter_and_finish(iter);
299                    Ok(ca.into_series())
300                },
301                Some(_) => {
302                    polars_bail!(InvalidOperation: "casting to a categorical with rev map is not allowed");
303                },
304            },
305            #[cfg(feature = "dtype-categorical")]
306            DataType::Enum(rev_map, ordering) => {
307                let Some(rev_map) = rev_map else {
308                    polars_bail!(InvalidOperation: "cannot cast / initialize Enum without categories present")
309                };
310                CategoricalChunked::from_string_to_enum(self, rev_map.get_categories(), *ordering)
311                    .map(|ca| {
312                        let mut s = ca.into_series();
313                        s.rename(self.name().clone());
314                        s
315                    })
316            },
317            #[cfg(feature = "dtype-struct")]
318            DataType::Struct(fields) => {
319                cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)
320            },
321            #[cfg(feature = "dtype-decimal")]
322            DataType::Decimal(precision, scale) => match (precision, scale) {
323                (precision, Some(scale)) => {
324                    let chunks = self.downcast_iter().map(|arr| {
325                        polars_compute::cast::binview_to_decimal(
326                            &arr.to_binview(),
327                            *precision,
328                            *scale,
329                        )
330                        .to(ArrowDataType::Int128)
331                    });
332                    Ok(Int128Chunked::from_chunk_iter(self.name().clone(), chunks)
333                        .into_decimal_unchecked(*precision, *scale)
334                        .into_series())
335                },
336                (None, None) => self.to_decimal(100),
337                _ => {
338                    polars_bail!(ComputeError: "expected 'precision' or 'scale' when casting to Decimal")
339                },
340            },
341            #[cfg(feature = "dtype-date")]
342            DataType::Date => {
343                let result = cast_chunks(&self.chunks, dtype, options)?;
344                let out = Series::try_from((self.name().clone(), result))?;
345                Ok(out)
346            },
347            #[cfg(feature = "dtype-datetime")]
348            DataType::Datetime(time_unit, time_zone) => match time_zone {
349                #[cfg(feature = "timezones")]
350                Some(time_zone) => {
351                    validate_time_zone(time_zone)?;
352                    let result = cast_chunks(
353                        &self.chunks,
354                        &Datetime(time_unit.to_owned(), Some(time_zone.clone())),
355                        options,
356                    )?;
357                    Series::try_from((self.name().clone(), result))
358                },
359                _ => {
360                    let result =
361                        cast_chunks(&self.chunks, &Datetime(time_unit.to_owned(), None), options)?;
362                    Series::try_from((self.name().clone(), result))
363                },
364            },
365            _ => cast_impl(self.name().clone(), &self.chunks, dtype, options),
366        }
367    }
368
369    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
370        self.cast_with_options(dtype, CastOptions::Overflowing)
371    }
372}
373
374impl BinaryChunked {
375    /// # Safety
376    /// String is not validated
377    pub unsafe fn to_string_unchecked(&self) -> StringChunked {
378        let chunks = self
379            .downcast_iter()
380            .map(|arr| unsafe { arr.to_utf8view_unchecked() }.boxed())
381            .collect();
382        let field = Arc::new(Field::new(self.name().clone(), DataType::String));
383
384        let mut ca = StringChunked::new_with_compute_len(field, chunks);
385
386        use StatisticsFlags as F;
387        ca.retain_flags_from(self, F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST);
388        ca
389    }
390}
391
392impl StringChunked {
393    pub fn as_binary(&self) -> BinaryChunked {
394        let chunks = self
395            .downcast_iter()
396            .map(|arr| arr.to_binview().boxed())
397            .collect();
398        let field = Arc::new(Field::new(self.name().clone(), DataType::Binary));
399
400        let mut ca = BinaryChunked::new_with_compute_len(field, chunks);
401
402        use StatisticsFlags as F;
403        ca.retain_flags_from(self, F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST);
404        ca
405    }
406}
407
408impl ChunkCast for BinaryChunked {
409    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
410        match dtype {
411            #[cfg(feature = "dtype-struct")]
412            DataType::Struct(fields) => {
413                cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)
414            },
415            _ => cast_impl(self.name().clone(), &self.chunks, dtype, options),
416        }
417    }
418
419    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
420        match dtype {
421            DataType::String => unsafe { Ok(self.to_string_unchecked().into_series()) },
422            _ => self.cast_with_options(dtype, CastOptions::Overflowing),
423        }
424    }
425}
426
427impl ChunkCast for BinaryOffsetChunked {
428    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
429        match dtype {
430            #[cfg(feature = "dtype-struct")]
431            DataType::Struct(fields) => {
432                cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)
433            },
434            _ => cast_impl(self.name().clone(), &self.chunks, dtype, options),
435        }
436    }
437
438    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
439        self.cast_with_options(dtype, CastOptions::Overflowing)
440    }
441}
442
443impl ChunkCast for BooleanChunked {
444    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
445        match dtype {
446            #[cfg(feature = "dtype-struct")]
447            DataType::Struct(fields) => {
448                cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)
449            },
450            #[cfg(feature = "dtype-categorical")]
451            DataType::Categorical(_, _) | DataType::Enum(_, _) => {
452                polars_bail!(InvalidOperation: "cannot cast Boolean to Categorical");
453            },
454            _ => cast_impl(self.name().clone(), &self.chunks, dtype, options),
455        }
456    }
457
458    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
459        self.cast_with_options(dtype, CastOptions::Overflowing)
460    }
461}
462
463/// We cannot cast anything to or from List/LargeList
464/// So this implementation casts the inner type
465impl ChunkCast for ListChunked {
466    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
467        let ca = self
468            .trim_lists_to_normalized_offsets()
469            .map_or(Cow::Borrowed(self), Cow::Owned);
470        let ca = ca.propagate_nulls().map_or(ca, Cow::Owned);
471
472        use DataType::*;
473        match dtype {
474            List(child_type) => {
475                match (ca.inner_dtype(), &**child_type) {
476                    (old, new) if old == new => Ok(ca.into_owned().into_series()),
477                    #[cfg(feature = "dtype-categorical")]
478                    (dt, Categorical(None, _) | Enum(_, _))
479                        if !matches!(dt, Categorical(_, _) | Enum(_, _) | String | Null) =>
480                    {
481                        polars_bail!(InvalidOperation: "cannot cast List inner type: '{:?}' to Categorical", dt)
482                    },
483                    _ => {
484                        // ensure the inner logical type bubbles up
485                        let (arr, child_type) = cast_list(ca.as_ref(), child_type, options)?;
486                        // SAFETY: we just cast so the dtype matches.
487                        // we must take this path to correct for physical types.
488                        unsafe {
489                            Ok(Series::from_chunks_and_dtype_unchecked(
490                                ca.name().clone(),
491                                vec![arr],
492                                &List(Box::new(child_type)),
493                            ))
494                        }
495                    },
496                }
497            },
498            #[cfg(feature = "dtype-array")]
499            Array(child_type, width) => {
500                let physical_type = dtype.to_physical();
501
502                // TODO!: properly implement this recursively.
503                #[cfg(feature = "dtype-categorical")]
504                polars_ensure!(!matches!(&**child_type, Categorical(_, _)), InvalidOperation: "array of categorical is not yet supported");
505
506                // cast to the physical type to avoid logical chunks.
507                let chunks = cast_chunks(ca.chunks(), &physical_type, options)?;
508                // SAFETY: we just cast so the dtype matches.
509                // we must take this path to correct for physical types.
510                unsafe {
511                    Ok(Series::from_chunks_and_dtype_unchecked(
512                        ca.name().clone(),
513                        chunks,
514                        &Array(child_type.clone(), *width),
515                    ))
516                }
517            },
518            _ => {
519                polars_bail!(
520                    InvalidOperation: "cannot cast List type (inner: '{:?}', to: '{:?}')",
521                    ca.inner_dtype(),
522                    dtype,
523                )
524            },
525        }
526    }
527
528    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
529        use DataType::*;
530        match dtype {
531            List(child_type) => cast_list_unchecked(self, child_type),
532            _ => self.cast_with_options(dtype, CastOptions::Overflowing),
533        }
534    }
535}
536
537/// We cannot cast anything to or from List/LargeList
538/// So this implementation casts the inner type
539#[cfg(feature = "dtype-array")]
540impl ChunkCast for ArrayChunked {
541    fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {
542        let ca = self
543            .trim_lists_to_normalized_offsets()
544            .map_or(Cow::Borrowed(self), Cow::Owned);
545        let ca = ca.propagate_nulls().map_or(ca, Cow::Owned);
546
547        use DataType::*;
548        match dtype {
549            Array(child_type, width) => {
550                polars_ensure!(
551                    *width == ca.width(),
552                    InvalidOperation: "cannot cast Array to a different width"
553                );
554
555                match (ca.inner_dtype(), &**child_type) {
556                    (old, new) if old == new => Ok(ca.into_owned().into_series()),
557                    #[cfg(feature = "dtype-categorical")]
558                    (dt, Categorical(None, _) | Enum(_, _)) if !matches!(dt, String) => {
559                        polars_bail!(InvalidOperation: "cannot cast Array inner type: '{:?}' to dtype: {:?}", dt, child_type)
560                    },
561                    _ => {
562                        // ensure the inner logical type bubbles up
563                        let (arr, child_type) =
564                            cast_fixed_size_list(ca.as_ref(), child_type, options)?;
565                        // SAFETY: we just cast so the dtype matches.
566                        // we must take this path to correct for physical types.
567                        unsafe {
568                            Ok(Series::from_chunks_and_dtype_unchecked(
569                                ca.name().clone(),
570                                vec![arr],
571                                &Array(Box::new(child_type), *width),
572                            ))
573                        }
574                    },
575                }
576            },
577            List(child_type) => {
578                let physical_type = dtype.to_physical();
579                // cast to the physical type to avoid logical chunks.
580                let chunks = cast_chunks(ca.chunks(), &physical_type, options)?;
581                // SAFETY: we just cast so the dtype matches.
582                // we must take this path to correct for physical types.
583                unsafe {
584                    Ok(Series::from_chunks_and_dtype_unchecked(
585                        ca.name().clone(),
586                        chunks,
587                        &List(child_type.clone()),
588                    ))
589                }
590            },
591            _ => {
592                polars_bail!(
593                    InvalidOperation: "cannot cast Array type (inner: '{:?}', to: '{:?}')",
594                    ca.inner_dtype(),
595                    dtype,
596                )
597            },
598        }
599    }
600
601    unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {
602        self.cast_with_options(dtype, CastOptions::Overflowing)
603    }
604}
605
606// Returns inner data type. This is needed because a cast can instantiate the dtype inner
607// values for instance with categoricals
608fn cast_list(
609    ca: &ListChunked,
610    child_type: &DataType,
611    options: CastOptions,
612) -> PolarsResult<(ArrayRef, DataType)> {
613    // We still rechunk because we must bubble up a single data-type
614    // TODO!: consider a version that works on chunks and merges the data-types and arrays.
615    let ca = ca.rechunk();
616    let arr = ca.downcast_as_array();
617    // SAFETY: inner dtype is passed correctly
618    let s = unsafe {
619        Series::from_chunks_and_dtype_unchecked(
620            PlSmallStr::EMPTY,
621            vec![arr.values().clone()],
622            ca.inner_dtype(),
623        )
624    };
625    let new_inner = s.cast_with_options(child_type, options)?;
626
627    let inner_dtype = new_inner.dtype().clone();
628    debug_assert_eq!(&inner_dtype, child_type);
629
630    let new_values = new_inner.array_ref(0).clone();
631
632    let dtype = ListArray::<i64>::default_datatype(new_values.dtype().clone());
633    let new_arr = ListArray::<i64>::new(
634        dtype,
635        arr.offsets().clone(),
636        new_values,
637        arr.validity().cloned(),
638    );
639    Ok((new_arr.boxed(), inner_dtype))
640}
641
642unsafe fn cast_list_unchecked(ca: &ListChunked, child_type: &DataType) -> PolarsResult<Series> {
643    // TODO! add chunked, but this must correct for list offsets.
644    let ca = ca.rechunk();
645    let arr = ca.downcast_as_array();
646    // SAFETY: inner dtype is passed correctly
647    let s = unsafe {
648        Series::from_chunks_and_dtype_unchecked(
649            PlSmallStr::EMPTY,
650            vec![arr.values().clone()],
651            ca.inner_dtype(),
652        )
653    };
654    let new_inner = s.cast_unchecked(child_type)?;
655    let new_values = new_inner.array_ref(0).clone();
656
657    let dtype = ListArray::<i64>::default_datatype(new_values.dtype().clone());
658    let new_arr = ListArray::<i64>::new(
659        dtype,
660        arr.offsets().clone(),
661        new_values,
662        arr.validity().cloned(),
663    );
664    Ok(ListChunked::from_chunks_and_dtype_unchecked(
665        ca.name().clone(),
666        vec![Box::new(new_arr)],
667        DataType::List(Box::new(child_type.clone())),
668    )
669    .into_series())
670}
671
672// Returns inner data type. This is needed because a cast can instantiate the dtype inner
673// values for instance with categoricals
674#[cfg(feature = "dtype-array")]
675fn cast_fixed_size_list(
676    ca: &ArrayChunked,
677    child_type: &DataType,
678    options: CastOptions,
679) -> PolarsResult<(ArrayRef, DataType)> {
680    let ca = ca.rechunk();
681    let arr = ca.downcast_as_array();
682    // SAFETY: inner dtype is passed correctly
683    let s = unsafe {
684        Series::from_chunks_and_dtype_unchecked(
685            PlSmallStr::EMPTY,
686            vec![arr.values().clone()],
687            ca.inner_dtype(),
688        )
689    };
690    let new_inner = s.cast_with_options(child_type, options)?;
691
692    let inner_dtype = new_inner.dtype().clone();
693    debug_assert_eq!(&inner_dtype, child_type);
694
695    let new_values = new_inner.array_ref(0).clone();
696
697    let dtype = FixedSizeListArray::default_datatype(new_values.dtype().clone(), ca.width());
698    let new_arr = FixedSizeListArray::new(dtype, ca.len(), new_values, arr.validity().cloned());
699    Ok((Box::new(new_arr), inner_dtype))
700}
701
702#[cfg(test)]
703mod test {
704    use crate::chunked_array::cast::CastOptions;
705    use crate::prelude::*;
706
707    #[test]
708    fn test_cast_list() -> PolarsResult<()> {
709        let mut builder = ListPrimitiveChunkedBuilder::<Int32Type>::new(
710            PlSmallStr::from_static("a"),
711            10,
712            10,
713            DataType::Int32,
714        );
715        builder.append_opt_slice(Some(&[1i32, 2, 3]));
716        builder.append_opt_slice(Some(&[1i32, 2, 3]));
717        let ca = builder.finish();
718
719        let new = ca.cast_with_options(
720            &DataType::List(DataType::Float64.into()),
721            CastOptions::Strict,
722        )?;
723
724        assert_eq!(new.dtype(), &DataType::List(DataType::Float64.into()));
725        Ok(())
726    }
727
728    #[test]
729    #[cfg(feature = "dtype-categorical")]
730    fn test_cast_noop() {
731        // check if we can cast categorical twice without panic
732        let ca = StringChunked::new(PlSmallStr::from_static("foo"), &["bar", "ham"]);
733        let out = ca
734            .cast_with_options(
735                &DataType::Categorical(None, Default::default()),
736                CastOptions::Strict,
737            )
738            .unwrap();
739        let out = out
740            .cast(&DataType::Categorical(None, Default::default()))
741            .unwrap();
742        assert!(matches!(out.dtype(), &DataType::Categorical(_, _)))
743    }
744}