polars_core/chunked_array/
cast.rs

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