Path: blob/main/crates/polars-core/src/chunked_array/cast.rs
6940 views
//! Implementations of the ChunkCast Trait.12use std::borrow::Cow;34use polars_compute::cast::CastOptionsImpl;5#[cfg(feature = "serde-lazy")]6use serde::{Deserialize, Serialize};78use super::flags::StatisticsFlags;9#[cfg(feature = "dtype-datetime")]10use crate::prelude::DataType::Datetime;11use crate::prelude::*;12use crate::utils::handle_casting_failures;1314#[derive(Copy, Clone, Debug, Default, PartialEq, Hash, Eq)]15#[cfg_attr(feature = "serde-lazy", derive(Serialize, Deserialize))]16#[cfg_attr(feature = "dsl-schema", derive(schemars::JsonSchema))]17#[repr(u8)]18pub enum CastOptions {19/// Raises on overflow20#[default]21Strict,22/// Overflow is replaced with null23NonStrict,24/// Allows wrapping overflow25Overflowing,26}2728impl CastOptions {29pub fn is_strict(&self) -> bool {30matches!(self, CastOptions::Strict)31}32}3334impl From<CastOptions> for CastOptionsImpl {35fn from(value: CastOptions) -> Self {36let wrapped = match value {37CastOptions::Strict | CastOptions::NonStrict => false,38CastOptions::Overflowing => true,39};40CastOptionsImpl {41wrapped,42partial: false,43}44}45}4647pub(crate) fn cast_chunks(48chunks: &[ArrayRef],49dtype: &DataType,50options: CastOptions,51) -> PolarsResult<Vec<ArrayRef>> {52let check_nulls = matches!(options, CastOptions::Strict);53let options = options.into();5455let arrow_dtype = dtype.try_to_arrow(CompatLevel::newest())?;56chunks57.iter()58.map(|arr| {59let out = polars_compute::cast::cast(arr.as_ref(), &arrow_dtype, options);60if check_nulls {61out.and_then(|new| {62polars_ensure!(arr.null_count() == new.null_count(), ComputeError: "strict cast failed");63Ok(new)64})6566} else {67out68}69})70.collect::<PolarsResult<Vec<_>>>()71}7273fn cast_impl_inner(74name: PlSmallStr,75chunks: &[ArrayRef],76dtype: &DataType,77options: CastOptions,78) -> PolarsResult<Series> {79let chunks = match dtype {80#[cfg(feature = "dtype-decimal")]81DataType::Decimal(_, _) => {82let mut chunks = cast_chunks(chunks, dtype, options)?;83// @NOTE: We cannot cast here as that will lower the scale.84for chunk in chunks.iter_mut() {85*chunk = std::mem::take(86chunk87.as_any_mut()88.downcast_mut::<PrimitiveArray<i128>>()89.unwrap(),90)91.to(ArrowDataType::Int128)92.to_boxed();93}94chunks95},96_ => cast_chunks(chunks, &dtype.to_physical(), options)?,97};9899let out = Series::try_from((name, chunks))?;100use DataType::*;101let out = match dtype {102Date => out.into_date(),103Datetime(tu, tz) => match tz {104#[cfg(feature = "timezones")]105Some(tz) => {106TimeZone::validate_time_zone(tz)?;107out.into_datetime(*tu, Some(tz.clone()))108},109_ => out.into_datetime(*tu, None),110},111Duration(tu) => out.into_duration(*tu),112#[cfg(feature = "dtype-time")]113Time => out.into_time(),114#[cfg(feature = "dtype-decimal")]115Decimal(precision, scale) => out.into_decimal(*precision, scale.unwrap_or(0))?,116_ => out,117};118119Ok(out)120}121122fn cast_impl(123name: PlSmallStr,124chunks: &[ArrayRef],125dtype: &DataType,126options: CastOptions,127) -> PolarsResult<Series> {128cast_impl_inner(name, chunks, dtype, options)129}130131#[cfg(feature = "dtype-struct")]132fn cast_single_to_struct(133name: PlSmallStr,134chunks: &[ArrayRef],135fields: &[Field],136options: CastOptions,137) -> PolarsResult<Series> {138polars_ensure!(fields.len() == 1, InvalidOperation: "must specify one field in the struct");139let mut new_fields = Vec::with_capacity(fields.len());140// cast to first field dtype141let mut fields = fields.iter();142let fld = fields.next().unwrap();143let s = cast_impl_inner(fld.name.clone(), chunks, &fld.dtype, options)?;144let length = s.len();145new_fields.push(s);146147for fld in fields {148new_fields.push(Series::full_null(fld.name.clone(), length, &fld.dtype));149}150151StructChunked::from_series(name, length, new_fields.iter()).map(|ca| ca.into_series())152}153154impl<T> ChunkedArray<T>155where156T: PolarsNumericType,157{158fn cast_impl(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {159if self.dtype() == dtype {160// SAFETY: chunks are correct dtype161let mut out = unsafe {162Series::from_chunks_and_dtype_unchecked(163self.name().clone(),164self.chunks.clone(),165dtype,166)167};168out.set_sorted_flag(self.is_sorted_flag());169return Ok(out);170}171match dtype {172// LEGACY173// TODO @ cat-rework: remove after exposing to/from physical functions.174#[cfg(feature = "dtype-categorical")]175DataType::Categorical(cats, _mapping) => {176let s = self.cast_with_options(&cats.physical().dtype(), options)?;177with_match_categorical_physical_type!(cats.physical(), |$C| {178// SAFETY: we are guarded by the type system.179type PhysCa = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;180let ca: &PhysCa = s.as_ref().as_ref();181Ok(CategoricalChunked::<$C>::from_cats_and_dtype(ca.clone(), dtype.clone())182.into_series())183})184},185186// LEGACY187// TODO @ cat-rework: remove after exposing to/from physical functions.188#[cfg(feature = "dtype-categorical")]189DataType::Enum(fcats, _mapping) => {190let s = self.cast_with_options(&fcats.physical().dtype(), options)?;191with_match_categorical_physical_type!(fcats.physical(), |$C| {192// SAFETY: we are guarded by the type system.193type PhysCa = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;194let ca: &PhysCa = s.as_ref().as_ref();195Ok(CategoricalChunked::<$C>::from_cats_and_dtype(ca.clone(), dtype.clone()).into_series())196})197},198199#[cfg(feature = "dtype-struct")]200DataType::Struct(fields) => {201cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)202},203_ => cast_impl_inner(self.name().clone(), &self.chunks, dtype, options).map(|mut s| {204// maintain sorted if data types205// - remain signed206// - unsigned -> signed207// this may still fail with overflow?208let to_signed = dtype.is_signed_integer();209let unsigned2unsigned =210self.dtype().is_unsigned_integer() && dtype.is_unsigned_integer();211let allowed = to_signed || unsigned2unsigned;212213if (allowed)214&& (s.null_count() == self.null_count())215// physical to logicals216|| (self.dtype().to_physical() == dtype.to_physical())217{218let is_sorted = self.is_sorted_flag();219s.set_sorted_flag(is_sorted)220}221s222}),223}224}225}226227impl<T> ChunkCast for ChunkedArray<T>228where229T: PolarsNumericType,230{231fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {232self.cast_impl(dtype, options)233}234235unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {236match dtype {237// LEGACY238// TODO @ cat-rework: remove after exposing to/from physical functions.239#[cfg(feature = "dtype-categorical")]240DataType::Categorical(cats, _mapping) => {241polars_ensure!(self.dtype() == &cats.physical().dtype(), ComputeError: "cannot cast numeric types to 'Categorical'");242with_match_categorical_physical_type!(cats.physical(), |$C| {243// SAFETY: we are guarded by the type system.244type PhysCa = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;245let ca = unsafe { &*(self as *const ChunkedArray<T> as *const PhysCa) };246Ok(CategoricalChunked::<$C>::from_cats_and_dtype_unchecked(ca.clone(), dtype.clone())247.into_series())248})249},250251// LEGACY252// TODO @ cat-rework: remove after exposing to/from physical functions.253#[cfg(feature = "dtype-categorical")]254DataType::Enum(fcats, _mapping) => {255polars_ensure!(self.dtype() == &fcats.physical().dtype(), ComputeError: "cannot cast numeric types to 'Enum'");256with_match_categorical_physical_type!(fcats.physical(), |$C| {257// SAFETY: we are guarded by the type system.258type PhysCa = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;259let ca = unsafe { &*(self as *const ChunkedArray<T> as *const PhysCa) };260Ok(CategoricalChunked::<$C>::from_cats_and_dtype_unchecked(ca.clone(), dtype.clone()).into_series())261})262},263264_ => self.cast_impl(dtype, CastOptions::Overflowing),265}266}267}268269impl ChunkCast for StringChunked {270fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {271match dtype {272#[cfg(feature = "dtype-categorical")]273DataType::Categorical(cats, _mapping) => {274with_match_categorical_physical_type!(cats.physical(), |$C| {275Ok(CategoricalChunked::<$C>::from_str_iter(self.name().clone(), dtype.clone(), self.iter())?276.into_series())277})278},279#[cfg(feature = "dtype-categorical")]280DataType::Enum(fcats, _mapping) => {281let ret = with_match_categorical_physical_type!(fcats.physical(), |$C| {282CategoricalChunked::<$C>::from_str_iter(self.name().clone(), dtype.clone(), self.iter())?283.into_series()284});285286if options.is_strict() && self.null_count() != ret.null_count() {287handle_casting_failures(&self.clone().into_series(), &ret)?;288}289290Ok(ret)291},292#[cfg(feature = "dtype-struct")]293DataType::Struct(fields) => {294cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)295},296#[cfg(feature = "dtype-decimal")]297DataType::Decimal(precision, scale) => match (precision, scale) {298(precision, Some(scale)) => {299let chunks = self.downcast_iter().map(|arr| {300polars_compute::cast::binview_to_decimal(301&arr.to_binview(),302*precision,303*scale,304)305.to(ArrowDataType::Int128)306});307Ok(Int128Chunked::from_chunk_iter(self.name().clone(), chunks)308.into_decimal_unchecked(*precision, *scale)309.into_series())310},311(None, None) => self.to_decimal_infer(100),312_ => {313polars_bail!(ComputeError: "expected 'precision' or 'scale' when casting to Decimal")314},315},316#[cfg(feature = "dtype-date")]317DataType::Date => {318let result = cast_chunks(&self.chunks, dtype, options)?;319let out = Series::try_from((self.name().clone(), result))?;320Ok(out)321},322#[cfg(feature = "dtype-datetime")]323DataType::Datetime(time_unit, time_zone) => match time_zone {324#[cfg(feature = "timezones")]325Some(time_zone) => {326TimeZone::validate_time_zone(time_zone)?;327let result = cast_chunks(328&self.chunks,329&Datetime(time_unit.to_owned(), Some(time_zone.clone())),330options,331)?;332Series::try_from((self.name().clone(), result))333},334_ => {335let result =336cast_chunks(&self.chunks, &Datetime(time_unit.to_owned(), None), options)?;337Series::try_from((self.name().clone(), result))338},339},340_ => cast_impl(self.name().clone(), &self.chunks, dtype, options),341}342}343344unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {345self.cast_with_options(dtype, CastOptions::Overflowing)346}347}348349impl BinaryChunked {350/// # Safety351/// String is not validated352pub unsafe fn to_string_unchecked(&self) -> StringChunked {353let chunks = self354.downcast_iter()355.map(|arr| unsafe { arr.to_utf8view_unchecked() }.boxed())356.collect();357let field = Arc::new(Field::new(self.name().clone(), DataType::String));358359let mut ca = StringChunked::new_with_compute_len(field, chunks);360361use StatisticsFlags as F;362ca.retain_flags_from(self, F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST);363ca364}365}366367impl StringChunked {368pub fn as_binary(&self) -> BinaryChunked {369let chunks = self370.downcast_iter()371.map(|arr| arr.to_binview().boxed())372.collect();373let field = Arc::new(Field::new(self.name().clone(), DataType::Binary));374375let mut ca = BinaryChunked::new_with_compute_len(field, chunks);376377use StatisticsFlags as F;378ca.retain_flags_from(self, F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST);379ca380}381}382383impl ChunkCast for BinaryChunked {384fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {385match dtype {386#[cfg(feature = "dtype-struct")]387DataType::Struct(fields) => {388cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)389},390_ => cast_impl(self.name().clone(), &self.chunks, dtype, options),391}392}393394unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {395match dtype {396DataType::String => unsafe { Ok(self.to_string_unchecked().into_series()) },397_ => self.cast_with_options(dtype, CastOptions::Overflowing),398}399}400}401402impl ChunkCast for BinaryOffsetChunked {403fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {404match dtype {405#[cfg(feature = "dtype-struct")]406DataType::Struct(fields) => {407cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)408},409_ => cast_impl(self.name().clone(), &self.chunks, dtype, options),410}411}412413unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {414self.cast_with_options(dtype, CastOptions::Overflowing)415}416}417418impl ChunkCast for BooleanChunked {419fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {420match dtype {421#[cfg(feature = "dtype-struct")]422DataType::Struct(fields) => {423cast_single_to_struct(self.name().clone(), &self.chunks, fields, options)424},425#[cfg(feature = "dtype-categorical")]426DataType::Categorical(_, _) | DataType::Enum(_, _) => {427polars_bail!(InvalidOperation: "cannot cast Boolean to Categorical");428},429_ => cast_impl(self.name().clone(), &self.chunks, dtype, options),430}431}432433unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {434self.cast_with_options(dtype, CastOptions::Overflowing)435}436}437438/// We cannot cast anything to or from List/LargeList439/// So this implementation casts the inner type440impl ChunkCast for ListChunked {441fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {442let ca = self443.trim_lists_to_normalized_offsets()444.map_or(Cow::Borrowed(self), Cow::Owned);445let ca = ca.propagate_nulls().map_or(ca, Cow::Owned);446447use DataType::*;448match dtype {449List(child_type) => {450match (ca.inner_dtype(), &**child_type) {451(old, new) if old == new => Ok(ca.into_owned().into_series()),452// TODO @ cat-rework: can we implement this now?453#[cfg(feature = "dtype-categorical")]454(dt, Categorical(_, _) | Enum(_, _))455if !matches!(dt, Categorical(_, _) | Enum(_, _) | String | Null) =>456{457polars_bail!(InvalidOperation: "cannot cast List inner type: '{:?}' to Categorical", dt)458},459_ => {460// ensure the inner logical type bubbles up461let (arr, child_type) = cast_list(ca.as_ref(), child_type, options)?;462// SAFETY: we just cast so the dtype matches.463// we must take this path to correct for physical types.464unsafe {465Ok(Series::from_chunks_and_dtype_unchecked(466ca.name().clone(),467vec![arr],468&List(Box::new(child_type)),469))470}471},472}473},474#[cfg(feature = "dtype-array")]475Array(child_type, width) => {476let physical_type = dtype.to_physical();477478// TODO @ cat-rework: can we implement this now?479// TODO!: properly implement this recursively.480#[cfg(feature = "dtype-categorical")]481polars_ensure!(!matches!(&**child_type, Categorical(_, _)), InvalidOperation: "array of categorical is not yet supported");482483// cast to the physical type to avoid logical chunks.484let chunks = cast_chunks(ca.chunks(), &physical_type, options)?;485// SAFETY: we just cast so the dtype matches.486// we must take this path to correct for physical types.487unsafe {488Ok(Series::from_chunks_and_dtype_unchecked(489ca.name().clone(),490chunks,491&Array(child_type.clone(), *width),492))493}494},495#[cfg(feature = "dtype-u8")]496Binary => {497polars_ensure!(498matches!(self.inner_dtype(), UInt8),499InvalidOperation: "cannot cast List type (inner: '{:?}', to: '{:?}')",500self.inner_dtype(),501dtype,502);503let chunks = cast_chunks(self.chunks(), &DataType::Binary, options)?;504505// SAFETY: we just cast so the dtype matches.506unsafe {507Ok(Series::from_chunks_and_dtype_unchecked(508self.name().clone(),509chunks,510&DataType::Binary,511))512}513},514_ => {515polars_bail!(516InvalidOperation: "cannot cast List type (inner: '{:?}', to: '{:?}')",517ca.inner_dtype(),518dtype,519)520},521}522}523524unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {525use DataType::*;526match dtype {527List(child_type) => cast_list_unchecked(self, child_type),528_ => self.cast_with_options(dtype, CastOptions::Overflowing),529}530}531}532533/// We cannot cast anything to or from List/LargeList534/// So this implementation casts the inner type535#[cfg(feature = "dtype-array")]536impl ChunkCast for ArrayChunked {537fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Series> {538let ca = self539.trim_lists_to_normalized_offsets()540.map_or(Cow::Borrowed(self), Cow::Owned);541let ca = ca.propagate_nulls().map_or(ca, Cow::Owned);542543use DataType::*;544match dtype {545Array(child_type, width) => {546polars_ensure!(547*width == ca.width(),548InvalidOperation: "cannot cast Array to a different width"549);550551match (ca.inner_dtype(), &**child_type) {552(old, new) if old == new => Ok(ca.into_owned().into_series()),553// TODO @ cat-rework: can we implement this now?554#[cfg(feature = "dtype-categorical")]555(dt, Categorical(_, _) | Enum(_, _)) if !matches!(dt, String) => {556polars_bail!(InvalidOperation: "cannot cast Array inner type: '{:?}' to dtype: {:?}", dt, child_type)557},558_ => {559// ensure the inner logical type bubbles up560let (arr, child_type) =561cast_fixed_size_list(ca.as_ref(), child_type, options)?;562// SAFETY: we just cast so the dtype matches.563// we must take this path to correct for physical types.564unsafe {565Ok(Series::from_chunks_and_dtype_unchecked(566ca.name().clone(),567vec![arr],568&Array(Box::new(child_type), *width),569))570}571},572}573},574List(child_type) => {575let physical_type = dtype.to_physical();576// cast to the physical type to avoid logical chunks.577let chunks = cast_chunks(ca.chunks(), &physical_type, options)?;578// SAFETY: we just cast so the dtype matches.579// we must take this path to correct for physical types.580unsafe {581Ok(Series::from_chunks_and_dtype_unchecked(582ca.name().clone(),583chunks,584&List(child_type.clone()),585))586}587},588_ => {589polars_bail!(590InvalidOperation: "cannot cast Array type (inner: '{:?}', to: '{:?}')",591ca.inner_dtype(),592dtype,593)594},595}596}597598unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Series> {599self.cast_with_options(dtype, CastOptions::Overflowing)600}601}602603// Returns inner data type. This is needed because a cast can instantiate the dtype inner604// values for instance with categoricals605fn cast_list(606ca: &ListChunked,607child_type: &DataType,608options: CastOptions,609) -> PolarsResult<(ArrayRef, DataType)> {610// We still rechunk because we must bubble up a single data-type611// TODO!: consider a version that works on chunks and merges the data-types and arrays.612let ca = ca.rechunk();613let arr = ca.downcast_as_array();614// SAFETY: inner dtype is passed correctly615let s = unsafe {616Series::from_chunks_and_dtype_unchecked(617PlSmallStr::EMPTY,618vec![arr.values().clone()],619ca.inner_dtype(),620)621};622let new_inner = s.cast_with_options(child_type, options)?;623624let inner_dtype = new_inner.dtype().clone();625debug_assert_eq!(&inner_dtype, child_type);626627let new_values = new_inner.array_ref(0).clone();628629let dtype = ListArray::<i64>::default_datatype(new_values.dtype().clone());630let new_arr = ListArray::<i64>::new(631dtype,632arr.offsets().clone(),633new_values,634arr.validity().cloned(),635);636Ok((new_arr.boxed(), inner_dtype))637}638639unsafe fn cast_list_unchecked(ca: &ListChunked, child_type: &DataType) -> PolarsResult<Series> {640// TODO! add chunked, but this must correct for list offsets.641let ca = ca.rechunk();642let arr = ca.downcast_as_array();643// SAFETY: inner dtype is passed correctly644let s = unsafe {645Series::from_chunks_and_dtype_unchecked(646PlSmallStr::EMPTY,647vec![arr.values().clone()],648ca.inner_dtype(),649)650};651let new_inner = s.cast_unchecked(child_type)?;652let new_values = new_inner.array_ref(0).clone();653654let dtype = ListArray::<i64>::default_datatype(new_values.dtype().clone());655let new_arr = ListArray::<i64>::new(656dtype,657arr.offsets().clone(),658new_values,659arr.validity().cloned(),660);661Ok(ListChunked::from_chunks_and_dtype_unchecked(662ca.name().clone(),663vec![Box::new(new_arr)],664DataType::List(Box::new(child_type.clone())),665)666.into_series())667}668669// Returns inner data type. This is needed because a cast can instantiate the dtype inner670// values for instance with categoricals671#[cfg(feature = "dtype-array")]672fn cast_fixed_size_list(673ca: &ArrayChunked,674child_type: &DataType,675options: CastOptions,676) -> PolarsResult<(ArrayRef, DataType)> {677let ca = ca.rechunk();678let arr = ca.downcast_as_array();679// SAFETY: inner dtype is passed correctly680let s = unsafe {681Series::from_chunks_and_dtype_unchecked(682PlSmallStr::EMPTY,683vec![arr.values().clone()],684ca.inner_dtype(),685)686};687let new_inner = s.cast_with_options(child_type, options)?;688689let inner_dtype = new_inner.dtype().clone();690debug_assert_eq!(&inner_dtype, child_type);691692let new_values = new_inner.array_ref(0).clone();693694let dtype = FixedSizeListArray::default_datatype(new_values.dtype().clone(), ca.width());695let new_arr = FixedSizeListArray::new(dtype, ca.len(), new_values, arr.validity().cloned());696Ok((Box::new(new_arr), inner_dtype))697}698699#[cfg(test)]700mod test {701use crate::chunked_array::cast::CastOptions;702use crate::prelude::*;703704#[test]705fn test_cast_list() -> PolarsResult<()> {706let mut builder = ListPrimitiveChunkedBuilder::<Int32Type>::new(707PlSmallStr::from_static("a"),70810,70910,710DataType::Int32,711);712builder.append_opt_slice(Some(&[1i32, 2, 3]));713builder.append_opt_slice(Some(&[1i32, 2, 3]));714let ca = builder.finish();715716let new = ca.cast_with_options(717&DataType::List(DataType::Float64.into()),718CastOptions::Strict,719)?;720721assert_eq!(new.dtype(), &DataType::List(DataType::Float64.into()));722Ok(())723}724725#[test]726#[cfg(feature = "dtype-categorical")]727fn test_cast_noop() {728// check if we can cast categorical twice without panic729let ca = StringChunked::new(PlSmallStr::from_static("foo"), &["bar", "ham"]);730let cats = Categories::global();731let out = ca732.cast_with_options(733&DataType::from_categories(cats.clone()),734CastOptions::Strict,735)736.unwrap();737let out = out.cast(&DataType::from_categories(cats)).unwrap();738assert!(matches!(out.dtype(), &DataType::Categorical(_, _)))739}740}741742743