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