polars_core/series/
mod.rs

1#![allow(unsafe_op_in_unsafe_fn)]
2//! Type agnostic columnar data structure.
3use crate::chunked_array::flags::StatisticsFlags;
4pub use crate::prelude::ChunkCompareEq;
5use crate::prelude::*;
6use crate::{HEAD_DEFAULT_LENGTH, TAIL_DEFAULT_LENGTH};
7
8macro_rules! invalid_operation_panic {
9    ($op:ident, $s:expr) => {
10        panic!(
11            "`{}` operation not supported for dtype `{}`",
12            stringify!($op),
13            $s._dtype()
14        )
15    };
16}
17
18pub mod amortized_iter;
19mod any_value;
20pub mod arithmetic;
21pub mod builder;
22mod comparison;
23mod from;
24pub mod implementations;
25mod into;
26pub(crate) mod iterator;
27pub mod ops;
28mod series_trait;
29
30use std::borrow::Cow;
31use std::hash::{Hash, Hasher};
32use std::ops::Deref;
33
34use arrow::compute::aggregate::estimated_bytes_size;
35use arrow::offset::Offsets;
36pub use from::*;
37pub use iterator::{SeriesIter, SeriesPhysIter};
38use num_traits::NumCast;
39use polars_error::feature_gated;
40pub use series_trait::{IsSorted, *};
41
42use crate::POOL;
43use crate::chunked_array::cast::CastOptions;
44#[cfg(feature = "zip_with")]
45use crate::series::arithmetic::coerce_lhs_rhs;
46use crate::utils::{Wrap, handle_casting_failures, materialize_dyn_int};
47
48/// # Series
49/// The columnar data type for a DataFrame.
50///
51/// Most of the available functions are defined in the [SeriesTrait trait](crate::series::SeriesTrait).
52///
53/// The `Series` struct consists
54/// of typed [ChunkedArray]'s. To quickly cast
55/// a `Series` to a `ChunkedArray` you can call the method with the name of the type:
56///
57/// ```
58/// # use polars_core::prelude::*;
59/// let s: Series = [1, 2, 3].iter().collect();
60/// // Quickly obtain the ChunkedArray wrapped by the Series.
61/// let chunked_array = s.i32().unwrap();
62/// ```
63///
64/// ## Arithmetic
65///
66/// You can do standard arithmetic on series.
67/// ```
68/// # use polars_core::prelude::*;
69/// let s = Series::new("a".into(), [1 , 2, 3]);
70/// let out_add = &s + &s;
71/// let out_sub = &s - &s;
72/// let out_div = &s / &s;
73/// let out_mul = &s * &s;
74/// ```
75///
76/// Or with series and numbers.
77///
78/// ```
79/// # use polars_core::prelude::*;
80/// let s: Series = (1..3).collect();
81/// let out_add_one = &s + 1;
82/// let out_multiply = &s * 10;
83///
84/// // Could not overload left hand side operator.
85/// let out_divide = 1.div(&s);
86/// let out_add = 1.add(&s);
87/// let out_subtract = 1.sub(&s);
88/// let out_multiply = 1.mul(&s);
89/// ```
90///
91/// ## Comparison
92/// You can obtain boolean mask by comparing series.
93///
94/// ```
95/// # use polars_core::prelude::*;
96/// let s = Series::new("dollars".into(), &[1, 2, 3]);
97/// let mask = s.equal(1).unwrap();
98/// let valid = [true, false, false].iter();
99/// assert!(mask
100///     .into_iter()
101///     .map(|opt_bool| opt_bool.unwrap()) // option, because series can be null
102///     .zip(valid)
103///     .all(|(a, b)| a == *b))
104/// ```
105///
106/// See all the comparison operators in the [ChunkCompareEq trait](crate::chunked_array::ops::ChunkCompareEq) and
107/// [ChunkCompareIneq trait](crate::chunked_array::ops::ChunkCompareIneq).
108///
109/// ## Iterators
110/// The Series variants contain differently typed [ChunkedArray]s.
111/// These structs can be turned into iterators, making it possible to use any function/ closure you want
112/// on a Series.
113///
114/// These iterators return an `Option<T>` because the values of a series may be null.
115///
116/// ```
117/// use polars_core::prelude::*;
118/// let pi = 3.14;
119/// let s = Series::new("angle".into(), [2f32 * pi, pi, 1.5 * pi].as_ref());
120/// let s_cos: Series = s.f32()
121///                     .expect("series was not an f32 dtype")
122///                     .into_iter()
123///                     .map(|opt_angle| opt_angle.map(|angle| angle.cos()))
124///                     .collect();
125/// ```
126///
127/// ## Creation
128/// Series can be create from different data structures. Below we'll show a few ways we can create
129/// a Series object.
130///
131/// ```
132/// # use polars_core::prelude::*;
133/// // Series can be created from Vec's, slices and arrays
134/// Series::new("boolean series".into(), &[true, false, true]);
135/// Series::new("int series".into(), &[1, 2, 3]);
136/// // And can be nullable
137/// Series::new("got nulls".into(), &[Some(1), None, Some(2)]);
138///
139/// // Series can also be collected from iterators
140/// let from_iter: Series = (0..10)
141///     .into_iter()
142///     .collect();
143///
144/// ```
145#[derive(Clone)]
146#[must_use]
147pub struct Series(pub Arc<dyn SeriesTrait>);
148
149impl PartialEq for Wrap<Series> {
150    fn eq(&self, other: &Self) -> bool {
151        self.0.equals_missing(other)
152    }
153}
154
155impl Eq for Wrap<Series> {}
156
157impl Hash for Wrap<Series> {
158    fn hash<H: Hasher>(&self, state: &mut H) {
159        let rs = PlSeedableRandomStateQuality::fixed();
160        let mut h = vec![];
161        if self.0.vec_hash(rs, &mut h).is_ok() {
162            let h = h.into_iter().fold(0, |a: u64, b| a.wrapping_add(b));
163            h.hash(state)
164        } else {
165            self.len().hash(state);
166            self.null_count().hash(state);
167            self.dtype().hash(state);
168        }
169    }
170}
171
172impl Series {
173    /// Create a new empty Series.
174    pub fn new_empty(name: PlSmallStr, dtype: &DataType) -> Series {
175        Series::full_null(name, 0, dtype)
176    }
177
178    pub fn clear(&self) -> Series {
179        if self.is_empty() {
180            self.clone()
181        } else {
182            match self.dtype() {
183                #[cfg(feature = "object")]
184                DataType::Object(_) => self
185                    .take(&ChunkedArray::<IdxType>::new_vec(PlSmallStr::EMPTY, vec![]))
186                    .unwrap(),
187                dt => Series::new_empty(self.name().clone(), dt),
188            }
189        }
190    }
191
192    #[doc(hidden)]
193    pub fn _get_inner_mut(&mut self) -> &mut dyn SeriesTrait {
194        if Arc::weak_count(&self.0) + Arc::strong_count(&self.0) != 1 {
195            self.0 = self.0.clone_inner();
196        }
197        Arc::get_mut(&mut self.0).expect("implementation error")
198    }
199
200    /// Take or clone a owned copy of the inner [`ChunkedArray`].
201    pub fn take_inner<T: PolarsPhysicalType>(self) -> ChunkedArray<T> {
202        let arc_any = self.0.as_arc_any();
203        let downcast = arc_any
204            .downcast::<implementations::SeriesWrap<ChunkedArray<T>>>()
205            .unwrap();
206
207        match Arc::try_unwrap(downcast) {
208            Ok(ca) => ca.0,
209            Err(ca) => ca.as_ref().as_ref().clone(),
210        }
211    }
212
213    /// # Safety
214    /// The caller must ensure the length and the data types of `ArrayRef` does not change.
215    /// And that the null_count is updated (e.g. with a `compute_len()`)
216    pub unsafe fn chunks_mut(&mut self) -> &mut Vec<ArrayRef> {
217        #[allow(unused_mut)]
218        let mut ca = self._get_inner_mut();
219        ca.chunks_mut()
220    }
221
222    pub fn into_chunks(mut self) -> Vec<ArrayRef> {
223        let ca = self._get_inner_mut();
224        let chunks = std::mem::take(unsafe { ca.chunks_mut() });
225        ca.compute_len();
226        chunks
227    }
228
229    // TODO! this probably can now be removed, now we don't have special case for structs.
230    pub fn select_chunk(&self, i: usize) -> Self {
231        let mut new = self.clear();
232        let mut flags = self.get_flags();
233
234        use StatisticsFlags as F;
235        flags &= F::IS_SORTED_ANY | F::CAN_FAST_EXPLODE_LIST;
236
237        // Assign mut so we go through arc only once.
238        let mut_new = new._get_inner_mut();
239        let chunks = unsafe { mut_new.chunks_mut() };
240        let chunk = self.chunks()[i].clone();
241        chunks.clear();
242        chunks.push(chunk);
243        mut_new.compute_len();
244        mut_new._set_flags(flags);
245        new
246    }
247
248    pub fn is_sorted_flag(&self) -> IsSorted {
249        if self.len() <= 1 {
250            return IsSorted::Ascending;
251        }
252        self.get_flags().is_sorted()
253    }
254
255    pub fn set_sorted_flag(&mut self, sorted: IsSorted) {
256        let mut flags = self.get_flags();
257        flags.set_sorted(sorted);
258        self.set_flags(flags);
259    }
260
261    pub(crate) fn clear_flags(&mut self) {
262        self.set_flags(StatisticsFlags::empty());
263    }
264    pub fn get_flags(&self) -> StatisticsFlags {
265        self.0._get_flags()
266    }
267
268    pub(crate) fn set_flags(&mut self, flags: StatisticsFlags) {
269        self._get_inner_mut()._set_flags(flags)
270    }
271
272    pub fn into_frame(self) -> DataFrame {
273        // SAFETY: A single-column dataframe cannot have length mismatches or duplicate names
274        unsafe { DataFrame::new_no_checks(self.len(), vec![self.into()]) }
275    }
276
277    /// Rename series.
278    pub fn rename(&mut self, name: PlSmallStr) -> &mut Series {
279        self._get_inner_mut().rename(name);
280        self
281    }
282
283    /// Return this Series with a new name.
284    pub fn with_name(mut self, name: PlSmallStr) -> Series {
285        self.rename(name);
286        self
287    }
288
289    pub fn from_arrow_chunks(name: PlSmallStr, arrays: Vec<ArrayRef>) -> PolarsResult<Series> {
290        Self::try_from((name, arrays))
291    }
292
293    pub fn from_arrow(name: PlSmallStr, array: ArrayRef) -> PolarsResult<Series> {
294        Self::try_from((name, array))
295    }
296
297    /// Shrink the capacity of this array to fit its length.
298    pub fn shrink_to_fit(&mut self) {
299        self._get_inner_mut().shrink_to_fit()
300    }
301
302    /// Append in place. This is done by adding the chunks of `other` to this [`Series`].
303    ///
304    /// See [`ChunkedArray::append`] and [`ChunkedArray::extend`].
305    pub fn append(&mut self, other: &Series) -> PolarsResult<&mut Self> {
306        let must_cast = other.dtype().matches_schema_type(self.dtype())?;
307        if must_cast {
308            let other = other.cast(self.dtype())?;
309            self.append_owned(other)?;
310        } else {
311            self._get_inner_mut().append(other)?;
312        }
313        Ok(self)
314    }
315
316    /// Append in place. This is done by adding the chunks of `other` to this [`Series`].
317    ///
318    /// See [`ChunkedArray::append_owned`] and [`ChunkedArray::extend`].
319    pub fn append_owned(&mut self, other: Series) -> PolarsResult<&mut Self> {
320        let must_cast = other.dtype().matches_schema_type(self.dtype())?;
321        if must_cast {
322            let other = other.cast(self.dtype())?;
323            self._get_inner_mut().append_owned(other)?;
324        } else {
325            self._get_inner_mut().append_owned(other)?;
326        }
327        Ok(self)
328    }
329
330    /// Redo a length and null_count compute
331    pub fn compute_len(&mut self) {
332        self._get_inner_mut().compute_len()
333    }
334
335    /// Extend the memory backed by this array with the values from `other`.
336    ///
337    /// See [`ChunkedArray::extend`] and [`ChunkedArray::append`].
338    pub fn extend(&mut self, other: &Series) -> PolarsResult<&mut Self> {
339        let must_cast = other.dtype().matches_schema_type(self.dtype())?;
340        if must_cast {
341            let other = other.cast(self.dtype())?;
342            self._get_inner_mut().extend(&other)?;
343        } else {
344            self._get_inner_mut().extend(other)?;
345        }
346        Ok(self)
347    }
348
349    /// Sort the series with specific options.
350    ///
351    /// # Example
352    ///
353    /// ```rust
354    /// # use polars_core::prelude::*;
355    /// # fn main() -> PolarsResult<()> {
356    /// let s = Series::new("foo".into(), [2, 1, 3]);
357    /// let sorted = s.sort(SortOptions::default())?;
358    /// assert_eq!(sorted, Series::new("foo".into(), [1, 2, 3]));
359    /// # Ok(())
360    /// }
361    /// ```
362    ///
363    /// See [`SortOptions`] for more options.
364    pub fn sort(&self, sort_options: SortOptions) -> PolarsResult<Self> {
365        self.sort_with(sort_options)
366    }
367
368    /// Only implemented for numeric types
369    pub fn as_single_ptr(&mut self) -> PolarsResult<usize> {
370        self._get_inner_mut().as_single_ptr()
371    }
372
373    pub fn cast(&self, dtype: &DataType) -> PolarsResult<Self> {
374        self.cast_with_options(dtype, CastOptions::NonStrict)
375    }
376
377    /// Cast [`Series`] to another [`DataType`].
378    pub fn cast_with_options(&self, dtype: &DataType, options: CastOptions) -> PolarsResult<Self> {
379        let slf = self
380            .trim_lists_to_normalized_offsets()
381            .map_or(Cow::Borrowed(self), Cow::Owned);
382        let slf = slf.propagate_nulls().map_or(slf, Cow::Owned);
383
384        use DataType as D;
385        let do_clone = match dtype {
386            D::Unknown(UnknownKind::Any | UnknownKind::Ufunc) => true,
387            D::Unknown(UnknownKind::Int(_)) if slf.dtype().is_integer() => true,
388            D::Unknown(UnknownKind::Float) if slf.dtype().is_float() => true,
389            D::Unknown(UnknownKind::Str)
390                if slf.dtype().is_string() | slf.dtype().is_categorical() =>
391            {
392                true
393            },
394            dt if dt.is_primitive() && dt == slf.dtype() => true,
395            _ => false,
396        };
397
398        if do_clone {
399            return Ok(slf.into_owned());
400        }
401
402        pub fn cast_dtype(dtype: &DataType) -> Option<DataType> {
403            match dtype {
404                D::Unknown(UnknownKind::Int(v)) => Some(materialize_dyn_int(*v).dtype()),
405                D::Unknown(UnknownKind::Float) => Some(DataType::Float64),
406                D::Unknown(UnknownKind::Str) => Some(DataType::String),
407                // Best leave as is.
408                D::List(inner) => cast_dtype(inner.as_ref()).map(Box::new).map(D::List),
409                #[cfg(feature = "dtype-struct")]
410                D::Struct(fields) => {
411                    // @NOTE: We only allocate if we really need to.
412
413                    let mut field_iter = fields.iter().enumerate();
414                    let mut new_fields = loop {
415                        let (i, field) = field_iter.next()?;
416
417                        if let Some(dtype) = cast_dtype(&field.dtype) {
418                            let mut new_fields = Vec::with_capacity(fields.len());
419                            new_fields.extend(fields.iter().take(i).cloned());
420                            new_fields.push(Field {
421                                name: field.name.clone(),
422                                dtype,
423                            });
424                            break new_fields;
425                        }
426                    };
427
428                    new_fields.extend(fields.iter().skip(new_fields.len()).cloned().map(|field| {
429                        let dtype = cast_dtype(&field.dtype).unwrap_or(field.dtype);
430                        Field {
431                            name: field.name,
432                            dtype,
433                        }
434                    }));
435
436                    Some(D::Struct(new_fields))
437                },
438                _ => None,
439            }
440        }
441
442        let mut casted = cast_dtype(dtype);
443        if dtype.is_list() && dtype.inner_dtype().is_some_and(|dt| dt.is_null()) {
444            if let Some(from_inner_dtype) = slf.dtype().inner_dtype() {
445                casted = Some(DataType::List(Box::new(from_inner_dtype.clone())));
446            }
447        }
448        let dtype = match casted {
449            None => dtype,
450            Some(ref dtype) => dtype,
451        };
452
453        // Always allow casting all nulls to other all nulls.
454        let len = slf.len();
455        if slf.null_count() == len {
456            return Ok(Series::full_null(slf.name().clone(), len, dtype));
457        }
458
459        let new_options = match options {
460            // Strictness is handled on this level to improve error messages.
461            CastOptions::Strict => CastOptions::NonStrict,
462            opt => opt,
463        };
464
465        let out = slf.0.cast(dtype, new_options)?;
466        if options.is_strict() {
467            handle_casting_failures(slf.as_ref(), &out)?;
468        }
469        Ok(out)
470    }
471
472    /// Cast from physical to logical types without any checks on the validity of the cast.
473    ///
474    /// # Safety
475    ///
476    /// This can lead to invalid memory access in downstream code.
477    pub unsafe fn cast_unchecked(&self, dtype: &DataType) -> PolarsResult<Self> {
478        match self.dtype() {
479            #[cfg(feature = "dtype-struct")]
480            DataType::Struct(_) => self.struct_().unwrap().cast_unchecked(dtype),
481            DataType::List(_) => self.list().unwrap().cast_unchecked(dtype),
482            dt if dt.is_primitive_numeric() => {
483                with_match_physical_numeric_polars_type!(dt, |$T| {
484                    let ca: &ChunkedArray<$T> = self.as_ref().as_ref().as_ref();
485                        ca.cast_unchecked(dtype)
486                })
487            },
488            DataType::Binary => self.binary().unwrap().cast_unchecked(dtype),
489            _ => self.cast_with_options(dtype, CastOptions::Overflowing),
490        }
491    }
492
493    /// Convert a non-logical series back into a logical series without casting.
494    ///
495    /// # Safety
496    ///
497    /// This can lead to invalid memory access in downstream code.
498    pub unsafe fn from_physical_unchecked(&self, dtype: &DataType) -> PolarsResult<Self> {
499        debug_assert!(!self.dtype().is_logical(), "{:?}", self.dtype());
500
501        if self.dtype() == dtype {
502            return Ok(self.clone());
503        }
504
505        use DataType as D;
506        match (self.dtype(), dtype) {
507            #[cfg(feature = "dtype-decimal")]
508            (D::Int128, D::Decimal(precision, scale)) => {
509                self.clone().into_decimal(*precision, scale.unwrap())
510            },
511
512            #[cfg(feature = "dtype-categorical")]
513            (phys, D::Categorical(cats, _)) if &cats.physical().dtype() == phys => {
514                with_match_categorical_physical_type!(cats.physical(), |$C| {
515                    type CA = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;
516                    let ca = self.as_ref().as_any().downcast_ref::<CA>().unwrap();
517                    Ok(CategoricalChunked::<$C>::from_cats_and_dtype_unchecked(
518                        ca.clone(),
519                        dtype.clone(),
520                    )
521                    .into_series())
522                })
523            },
524            #[cfg(feature = "dtype-categorical")]
525            (phys, D::Enum(fcats, _)) if &fcats.physical().dtype() == phys => {
526                with_match_categorical_physical_type!(fcats.physical(), |$C| {
527                    type CA = ChunkedArray<<$C as PolarsCategoricalType>::PolarsPhysical>;
528                    let ca = self.as_ref().as_any().downcast_ref::<CA>().unwrap();
529                    Ok(CategoricalChunked::<$C>::from_cats_and_dtype_unchecked(
530                        ca.clone(),
531                        dtype.clone(),
532                    )
533                    .into_series())
534                })
535            },
536
537            (D::Int32, D::Date) => feature_gated!("dtype-time", Ok(self.clone().into_date())),
538            (D::Int64, D::Datetime(tu, tz)) => feature_gated!(
539                "dtype-datetime",
540                Ok(self.clone().into_datetime(*tu, tz.clone()))
541            ),
542            (D::Int64, D::Duration(tu)) => {
543                feature_gated!("dtype-duration", Ok(self.clone().into_duration(*tu)))
544            },
545            (D::Int64, D::Time) => feature_gated!("dtype-time", Ok(self.clone().into_time())),
546
547            (D::List(_), D::List(to)) => unsafe {
548                self.list()
549                    .unwrap()
550                    .from_physical_unchecked(to.as_ref().clone())
551                    .map(|ca| ca.into_series())
552            },
553            #[cfg(feature = "dtype-array")]
554            (D::Array(_, lw), D::Array(to, rw)) if lw == rw => unsafe {
555                self.array()
556                    .unwrap()
557                    .from_physical_unchecked(to.as_ref().clone())
558                    .map(|ca| ca.into_series())
559            },
560            #[cfg(feature = "dtype-struct")]
561            (D::Struct(_), D::Struct(to)) => unsafe {
562                self.struct_()
563                    .unwrap()
564                    .from_physical_unchecked(to.as_slice())
565                    .map(|ca| ca.into_series())
566            },
567
568            _ => panic!("invalid from_physical({dtype:?}) for {:?}", self.dtype()),
569        }
570    }
571
572    /// Cast numerical types to f64, and keep floats as is.
573    pub fn to_float(&self) -> PolarsResult<Series> {
574        match self.dtype() {
575            DataType::Float32 | DataType::Float64 => Ok(self.clone()),
576            _ => self.cast_with_options(&DataType::Float64, CastOptions::Overflowing),
577        }
578    }
579
580    /// Compute the sum of all values in this Series.
581    /// Returns `Some(0)` if the array is empty, and `None` if the array only
582    /// contains null values.
583    ///
584    /// If the [`DataType`] is one of `{Int8, UInt8, Int16, UInt16}` the `Series` is
585    /// first cast to `Int64` to prevent overflow issues.
586    pub fn sum<T>(&self) -> PolarsResult<T>
587    where
588        T: NumCast,
589    {
590        let sum = self.sum_reduce()?;
591        let sum = sum.value().extract().unwrap();
592        Ok(sum)
593    }
594
595    /// Returns the minimum value in the array, according to the natural order.
596    /// Returns an option because the array is nullable.
597    pub fn min<T>(&self) -> PolarsResult<Option<T>>
598    where
599        T: NumCast,
600    {
601        let min = self.min_reduce()?;
602        let min = min.value().extract::<T>();
603        Ok(min)
604    }
605
606    /// Returns the maximum value in the array, according to the natural order.
607    /// Returns an option because the array is nullable.
608    pub fn max<T>(&self) -> PolarsResult<Option<T>>
609    where
610        T: NumCast,
611    {
612        let max = self.max_reduce()?;
613        let max = max.value().extract::<T>();
614        Ok(max)
615    }
616
617    /// Explode a list Series. This expands every item to a new row..
618    pub fn explode(&self, skip_empty: bool) -> PolarsResult<Series> {
619        match self.dtype() {
620            DataType::List(_) => self.list().unwrap().explode(skip_empty),
621            #[cfg(feature = "dtype-array")]
622            DataType::Array(_, _) => self.array().unwrap().explode(skip_empty),
623            _ => Ok(self.clone()),
624        }
625    }
626
627    /// Check if numeric value is NaN (note this is different than missing/ null)
628    pub fn is_nan(&self) -> PolarsResult<BooleanChunked> {
629        match self.dtype() {
630            DataType::Float32 => Ok(self.f32().unwrap().is_nan()),
631            DataType::Float64 => Ok(self.f64().unwrap().is_nan()),
632            DataType::Null => Ok(BooleanChunked::full_null(self.name().clone(), self.len())),
633            dt if dt.is_primitive_numeric() => {
634                let arr = BooleanArray::full(self.len(), false, ArrowDataType::Boolean)
635                    .with_validity(self.rechunk_validity());
636                Ok(BooleanChunked::with_chunk(self.name().clone(), arr))
637            },
638            _ => polars_bail!(opq = is_nan, self.dtype()),
639        }
640    }
641
642    /// Check if numeric value is NaN (note this is different than missing/null)
643    pub fn is_not_nan(&self) -> PolarsResult<BooleanChunked> {
644        match self.dtype() {
645            DataType::Float32 => Ok(self.f32().unwrap().is_not_nan()),
646            DataType::Float64 => Ok(self.f64().unwrap().is_not_nan()),
647            dt if dt.is_primitive_numeric() => {
648                let arr = BooleanArray::full(self.len(), true, ArrowDataType::Boolean)
649                    .with_validity(self.rechunk_validity());
650                Ok(BooleanChunked::with_chunk(self.name().clone(), arr))
651            },
652            _ => polars_bail!(opq = is_not_nan, self.dtype()),
653        }
654    }
655
656    /// Check if numeric value is finite
657    pub fn is_finite(&self) -> PolarsResult<BooleanChunked> {
658        match self.dtype() {
659            DataType::Float32 => Ok(self.f32().unwrap().is_finite()),
660            DataType::Float64 => Ok(self.f64().unwrap().is_finite()),
661            DataType::Null => Ok(BooleanChunked::full_null(self.name().clone(), self.len())),
662            dt if dt.is_primitive_numeric() => {
663                let arr = BooleanArray::full(self.len(), true, ArrowDataType::Boolean)
664                    .with_validity(self.rechunk_validity());
665                Ok(BooleanChunked::with_chunk(self.name().clone(), arr))
666            },
667            _ => polars_bail!(opq = is_finite, self.dtype()),
668        }
669    }
670
671    /// Check if numeric value is infinite
672    pub fn is_infinite(&self) -> PolarsResult<BooleanChunked> {
673        match self.dtype() {
674            DataType::Float32 => Ok(self.f32().unwrap().is_infinite()),
675            DataType::Float64 => Ok(self.f64().unwrap().is_infinite()),
676            DataType::Null => Ok(BooleanChunked::full_null(self.name().clone(), self.len())),
677            dt if dt.is_primitive_numeric() => {
678                let arr = BooleanArray::full(self.len(), false, ArrowDataType::Boolean)
679                    .with_validity(self.rechunk_validity());
680                Ok(BooleanChunked::with_chunk(self.name().clone(), arr))
681            },
682            _ => polars_bail!(opq = is_infinite, self.dtype()),
683        }
684    }
685
686    /// Create a new ChunkedArray with values from self where the mask evaluates `true` and values
687    /// from `other` where the mask evaluates `false`. This function automatically broadcasts unit
688    /// length inputs.
689    #[cfg(feature = "zip_with")]
690    pub fn zip_with(&self, mask: &BooleanChunked, other: &Series) -> PolarsResult<Series> {
691        let (lhs, rhs) = coerce_lhs_rhs(self, other)?;
692        lhs.zip_with_same_type(mask, rhs.as_ref())
693    }
694
695    /// Converts a Series to their physical representation, if they have one,
696    /// otherwise the series is left unchanged.
697    ///
698    /// * Date -> Int32
699    /// * Datetime -> Int64
700    /// * Duration -> Int64
701    /// * Decimal -> Int128
702    /// * Time -> Int64
703    /// * Categorical -> U8/U16/U32
704    /// * List(inner) -> List(physical of inner)
705    /// * Array(inner) -> Array(physical of inner)
706    /// * Struct -> Struct with physical repr of each struct column
707    pub fn to_physical_repr(&self) -> Cow<'_, Series> {
708        use DataType::*;
709        match self.dtype() {
710            // NOTE: Don't use cast here, as it might rechunk (if all nulls)
711            // which is not allowed in a phys repr.
712            #[cfg(feature = "dtype-date")]
713            Date => Cow::Owned(self.date().unwrap().phys.clone().into_series()),
714            #[cfg(feature = "dtype-datetime")]
715            Datetime(_, _) => Cow::Owned(self.datetime().unwrap().phys.clone().into_series()),
716            #[cfg(feature = "dtype-duration")]
717            Duration(_) => Cow::Owned(self.duration().unwrap().phys.clone().into_series()),
718            #[cfg(feature = "dtype-time")]
719            Time => Cow::Owned(self.time().unwrap().phys.clone().into_series()),
720            #[cfg(feature = "dtype-categorical")]
721            dt @ (Categorical(_, _) | Enum(_, _)) => {
722                with_match_categorical_physical_type!(dt.cat_physical().unwrap(), |$C| {
723                    let ca = self.cat::<$C>().unwrap();
724                    Cow::Owned(ca.physical().clone().into_series())
725                })
726            },
727            #[cfg(feature = "dtype-decimal")]
728            Decimal(_, _) => Cow::Owned(self.decimal().unwrap().phys.clone().into_series()),
729            List(_) => match self.list().unwrap().to_physical_repr() {
730                Cow::Borrowed(_) => Cow::Borrowed(self),
731                Cow::Owned(ca) => Cow::Owned(ca.into_series()),
732            },
733            #[cfg(feature = "dtype-array")]
734            Array(_, _) => match self.array().unwrap().to_physical_repr() {
735                Cow::Borrowed(_) => Cow::Borrowed(self),
736                Cow::Owned(ca) => Cow::Owned(ca.into_series()),
737            },
738            #[cfg(feature = "dtype-struct")]
739            Struct(_) => match self.struct_().unwrap().to_physical_repr() {
740                Cow::Borrowed(_) => Cow::Borrowed(self),
741                Cow::Owned(ca) => Cow::Owned(ca.into_series()),
742            },
743            _ => Cow::Borrowed(self),
744        }
745    }
746
747    /// Traverse and collect every nth element in a new array.
748    pub fn gather_every(&self, n: usize, offset: usize) -> PolarsResult<Series> {
749        polars_ensure!(n > 0, ComputeError: "cannot perform gather every for `n=0`");
750        let idx = ((offset as IdxSize)..self.len() as IdxSize)
751            .step_by(n)
752            .collect_ca(PlSmallStr::EMPTY);
753        // SAFETY: we stay in-bounds.
754        Ok(unsafe { self.take_unchecked(&idx) })
755    }
756
757    #[cfg(feature = "dot_product")]
758    pub fn dot(&self, other: &Series) -> PolarsResult<f64> {
759        std::ops::Mul::mul(self, other)?.sum::<f64>()
760    }
761
762    /// Get the sum of the Series as a new Series of length 1.
763    /// Returns a Series with a single zeroed entry if self is an empty numeric series.
764    ///
765    /// If the [`DataType`] is one of `{Int8, UInt8, Int16, UInt16}` the `Series` is
766    /// first cast to `Int64` to prevent overflow issues.
767    pub fn sum_reduce(&self) -> PolarsResult<Scalar> {
768        use DataType::*;
769        match self.dtype() {
770            Int8 | UInt8 | Int16 | UInt16 => self.cast(&Int64).unwrap().sum_reduce(),
771            _ => self.0.sum_reduce(),
772        }
773    }
774
775    /// Get the product of an array.
776    ///
777    /// If the [`DataType`] is one of `{Int8, UInt8, Int16, UInt16}` the `Series` is
778    /// first cast to `Int64` to prevent overflow issues.
779    pub fn product(&self) -> PolarsResult<Scalar> {
780        #[cfg(feature = "product")]
781        {
782            use DataType::*;
783            match self.dtype() {
784                Boolean => self.cast(&DataType::Int64).unwrap().product(),
785                Int8 | UInt8 | Int16 | UInt16 | Int32 | UInt32 => {
786                    let s = self.cast(&Int64).unwrap();
787                    s.product()
788                },
789                Int64 => Ok(self.i64().unwrap().prod_reduce()),
790                UInt64 => Ok(self.u64().unwrap().prod_reduce()),
791                #[cfg(feature = "dtype-i128")]
792                Int128 => Ok(self.i128().unwrap().prod_reduce()),
793                Float32 => Ok(self.f32().unwrap().prod_reduce()),
794                Float64 => Ok(self.f64().unwrap().prod_reduce()),
795                dt => {
796                    polars_bail!(InvalidOperation: "`product` operation not supported for dtype `{dt}`")
797                },
798            }
799        }
800        #[cfg(not(feature = "product"))]
801        {
802            panic!("activate 'product' feature")
803        }
804    }
805
806    /// Cast throws an error if conversion had overflows
807    pub fn strict_cast(&self, dtype: &DataType) -> PolarsResult<Series> {
808        self.cast_with_options(dtype, CastOptions::Strict)
809    }
810
811    #[cfg(feature = "dtype-decimal")]
812    pub(crate) fn into_decimal(
813        self,
814        precision: Option<usize>,
815        scale: usize,
816    ) -> PolarsResult<Series> {
817        match self.dtype() {
818            DataType::Int128 => Ok(self
819                .i128()
820                .unwrap()
821                .clone()
822                .into_decimal(precision, scale)?
823                .into_series()),
824            DataType::Decimal(cur_prec, cur_scale)
825                if (cur_prec.is_none() || precision.is_none() || *cur_prec == precision)
826                    && *cur_scale == Some(scale) =>
827            {
828                Ok(self)
829            },
830            dt => panic!("into_decimal({precision:?}, {scale}) not implemented for {dt:?}"),
831        }
832    }
833
834    #[cfg(feature = "dtype-time")]
835    pub(crate) fn into_time(self) -> Series {
836        match self.dtype() {
837            DataType::Int64 => self.i64().unwrap().clone().into_time().into_series(),
838            DataType::Time => self
839                .time()
840                .unwrap()
841                .physical()
842                .clone()
843                .into_time()
844                .into_series(),
845            dt => panic!("date not implemented for {dt:?}"),
846        }
847    }
848
849    pub(crate) fn into_date(self) -> Series {
850        #[cfg(not(feature = "dtype-date"))]
851        {
852            panic!("activate feature dtype-date")
853        }
854        #[cfg(feature = "dtype-date")]
855        match self.dtype() {
856            DataType::Int32 => self.i32().unwrap().clone().into_date().into_series(),
857            DataType::Date => self
858                .date()
859                .unwrap()
860                .physical()
861                .clone()
862                .into_date()
863                .into_series(),
864            dt => panic!("date not implemented for {dt:?}"),
865        }
866    }
867
868    #[allow(unused_variables)]
869    pub(crate) fn into_datetime(self, timeunit: TimeUnit, tz: Option<TimeZone>) -> Series {
870        #[cfg(not(feature = "dtype-datetime"))]
871        {
872            panic!("activate feature dtype-datetime")
873        }
874
875        #[cfg(feature = "dtype-datetime")]
876        match self.dtype() {
877            DataType::Int64 => self
878                .i64()
879                .unwrap()
880                .clone()
881                .into_datetime(timeunit, tz)
882                .into_series(),
883            DataType::Datetime(_, _) => self
884                .datetime()
885                .unwrap()
886                .physical()
887                .clone()
888                .into_datetime(timeunit, tz)
889                .into_series(),
890            dt => panic!("into_datetime not implemented for {dt:?}"),
891        }
892    }
893
894    #[allow(unused_variables)]
895    pub(crate) fn into_duration(self, timeunit: TimeUnit) -> Series {
896        #[cfg(not(feature = "dtype-duration"))]
897        {
898            panic!("activate feature dtype-duration")
899        }
900        #[cfg(feature = "dtype-duration")]
901        match self.dtype() {
902            DataType::Int64 => self
903                .i64()
904                .unwrap()
905                .clone()
906                .into_duration(timeunit)
907                .into_series(),
908            DataType::Duration(_) => self
909                .duration()
910                .unwrap()
911                .physical()
912                .clone()
913                .into_duration(timeunit)
914                .into_series(),
915            dt => panic!("into_duration not implemented for {dt:?}"),
916        }
917    }
918
919    // used for formatting
920    pub fn str_value(&self, index: usize) -> PolarsResult<Cow<'_, str>> {
921        Ok(self.0.get(index)?.str_value())
922    }
923    /// Get the head of the Series.
924    pub fn head(&self, length: Option<usize>) -> Series {
925        let len = length.unwrap_or(HEAD_DEFAULT_LENGTH);
926        self.slice(0, std::cmp::min(len, self.len()))
927    }
928
929    /// Get the tail of the Series.
930    pub fn tail(&self, length: Option<usize>) -> Series {
931        let len = length.unwrap_or(TAIL_DEFAULT_LENGTH);
932        let len = std::cmp::min(len, self.len());
933        self.slice(-(len as i64), len)
934    }
935
936    pub fn mean_reduce(&self) -> Scalar {
937        crate::scalar::reduce::mean_reduce(self.mean(), self.dtype().clone())
938    }
939
940    /// Compute the unique elements, but maintain order. This requires more work
941    /// than a naive [`Series::unique`](SeriesTrait::unique).
942    pub fn unique_stable(&self) -> PolarsResult<Series> {
943        let idx = self.arg_unique()?;
944        // SAFETY: Indices are in bounds.
945        unsafe { Ok(self.take_unchecked(&idx)) }
946    }
947
948    pub fn try_idx(&self) -> Option<&IdxCa> {
949        #[cfg(feature = "bigidx")]
950        {
951            self.try_u64()
952        }
953        #[cfg(not(feature = "bigidx"))]
954        {
955            self.try_u32()
956        }
957    }
958
959    pub fn idx(&self) -> PolarsResult<&IdxCa> {
960        #[cfg(feature = "bigidx")]
961        {
962            self.u64()
963        }
964        #[cfg(not(feature = "bigidx"))]
965        {
966            self.u32()
967        }
968    }
969
970    /// Returns an estimation of the total (heap) allocated size of the `Series` in bytes.
971    ///
972    /// # Implementation
973    /// This estimation is the sum of the size of its buffers, validity, including nested arrays.
974    /// Multiple arrays may share buffers and bitmaps. Therefore, the size of 2 arrays is not the
975    /// sum of the sizes computed from this function. In particular, [`StructArray`]'s size is an upper bound.
976    ///
977    /// When an array is sliced, its allocated size remains constant because the buffer unchanged.
978    /// However, this function will yield a smaller number. This is because this function returns
979    /// the visible size of the buffer, not its total capacity.
980    ///
981    /// FFI buffers are included in this estimation.
982    pub fn estimated_size(&self) -> usize {
983        let mut size = 0;
984        match self.dtype() {
985            // TODO @ cat-rework: include mapping size here?
986            #[cfg(feature = "object")]
987            DataType::Object(_) => {
988                let ArrowDataType::FixedSizeBinary(size) = self.chunks()[0].dtype() else {
989                    unreachable!()
990                };
991                // This is only the pointer size in python. So will be a huge underestimation.
992                return self.len() * *size;
993            },
994            _ => {},
995        }
996
997        size += self
998            .chunks()
999            .iter()
1000            .map(|arr| estimated_bytes_size(&**arr))
1001            .sum::<usize>();
1002
1003        size
1004    }
1005
1006    /// Packs every element into a list.
1007    pub fn as_list(&self) -> ListChunked {
1008        let s = self.rechunk();
1009        // don't  use `to_arrow` as we need the physical types
1010        let values = s.chunks()[0].clone();
1011        let offsets = (0i64..(s.len() as i64 + 1)).collect::<Vec<_>>();
1012        let offsets = unsafe { Offsets::new_unchecked(offsets) };
1013
1014        let dtype = LargeListArray::default_datatype(
1015            s.dtype().to_physical().to_arrow(CompatLevel::newest()),
1016        );
1017        let new_arr = LargeListArray::new(dtype, offsets.into(), values, None);
1018        let mut out = ListChunked::with_chunk(s.name().clone(), new_arr);
1019        out.set_inner_dtype(s.dtype().clone());
1020        out
1021    }
1022
1023    pub fn row_encode_unordered(&self) -> PolarsResult<BinaryOffsetChunked> {
1024        row_encode::_get_rows_encoded_ca_unordered(
1025            self.name().clone(),
1026            &[self.clone().into_column()],
1027        )
1028    }
1029
1030    pub fn row_encode_ordered(
1031        &self,
1032        descending: bool,
1033        nulls_last: bool,
1034    ) -> PolarsResult<BinaryOffsetChunked> {
1035        row_encode::_get_rows_encoded_ca(
1036            self.name().clone(),
1037            &[self.clone().into_column()],
1038            &[descending],
1039            &[nulls_last],
1040        )
1041    }
1042}
1043
1044impl Deref for Series {
1045    type Target = dyn SeriesTrait;
1046
1047    fn deref(&self) -> &Self::Target {
1048        self.0.as_ref()
1049    }
1050}
1051
1052impl<'a> AsRef<dyn SeriesTrait + 'a> for Series {
1053    fn as_ref(&self) -> &(dyn SeriesTrait + 'a) {
1054        self.0.as_ref()
1055    }
1056}
1057
1058impl Default for Series {
1059    fn default() -> Self {
1060        Int64Chunked::default().into_series()
1061    }
1062}
1063
1064impl<T: PolarsPhysicalType> AsRef<ChunkedArray<T>> for dyn SeriesTrait + '_ {
1065    fn as_ref(&self) -> &ChunkedArray<T> {
1066        // @NOTE: SeriesTrait `as_any` returns a std::any::Any for the underlying ChunkedArray /
1067        // Logical (so not the SeriesWrap).
1068        let Some(ca) = self.as_any().downcast_ref::<ChunkedArray<T>>() else {
1069            panic!(
1070                "implementation error, cannot get ref {:?} from {:?}",
1071                T::get_static_dtype(),
1072                self.dtype()
1073            );
1074        };
1075
1076        ca
1077    }
1078}
1079
1080impl<T: PolarsPhysicalType> AsMut<ChunkedArray<T>> for dyn SeriesTrait + '_ {
1081    fn as_mut(&mut self) -> &mut ChunkedArray<T> {
1082        if !self.as_any_mut().is::<ChunkedArray<T>>() {
1083            panic!(
1084                "implementation error, cannot get ref {:?} from {:?}",
1085                T::get_static_dtype(),
1086                self.dtype()
1087            );
1088        }
1089
1090        // @NOTE: SeriesTrait `as_any` returns a std::any::Any for the underlying ChunkedArray /
1091        // Logical (so not the SeriesWrap).
1092        self.as_any_mut().downcast_mut::<ChunkedArray<T>>().unwrap()
1093    }
1094}
1095
1096#[cfg(test)]
1097mod test {
1098    use crate::prelude::*;
1099    use crate::series::*;
1100
1101    #[test]
1102    fn cast() {
1103        let ar = UInt32Chunked::new("a".into(), &[1, 2]);
1104        let s = ar.into_series();
1105        let s2 = s.cast(&DataType::Int64).unwrap();
1106
1107        assert!(s2.i64().is_ok());
1108        let s2 = s.cast(&DataType::Float32).unwrap();
1109        assert!(s2.f32().is_ok());
1110    }
1111
1112    #[test]
1113    fn new_series() {
1114        let _ = Series::new("boolean series".into(), &vec![true, false, true]);
1115        let _ = Series::new("int series".into(), &[1, 2, 3]);
1116        let ca = Int32Chunked::new("a".into(), &[1, 2, 3]);
1117        let _ = ca.into_series();
1118    }
1119
1120    #[test]
1121    #[cfg(feature = "dtype-date")]
1122    fn roundtrip_list_logical_20311() {
1123        let list = ListChunked::from_chunk_iter(
1124            PlSmallStr::from_static("a"),
1125            [ListArray::new(
1126                ArrowDataType::LargeList(Box::new(ArrowField::new(
1127                    LIST_VALUES_NAME,
1128                    ArrowDataType::Int32,
1129                    true,
1130                ))),
1131                unsafe { Offsets::new_unchecked(vec![0, 1]) }.into(),
1132                PrimitiveArray::new(ArrowDataType::Int32, vec![1i32].into(), None).to_boxed(),
1133                None,
1134            )],
1135        );
1136        let list = unsafe { list.from_physical_unchecked(DataType::Date) }.unwrap();
1137        assert_eq!(list.dtype(), &DataType::List(Box::new(DataType::Date)));
1138    }
1139
1140    #[test]
1141    #[cfg(feature = "dtype-struct")]
1142    fn new_series_from_empty_structs() {
1143        let dtype = DataType::Struct(vec![]);
1144        let empties = vec![AnyValue::StructOwned(Box::new((vec![], vec![]))); 3];
1145        let s = Series::from_any_values_and_dtype("".into(), &empties, &dtype, false).unwrap();
1146        assert_eq!(s.len(), 3);
1147    }
1148    #[test]
1149    fn new_series_from_arrow_primitive_array() {
1150        let array = UInt32Array::from_slice([1, 2, 3, 4, 5]);
1151        let array_ref: ArrayRef = Box::new(array);
1152
1153        let _ = Series::try_new("foo".into(), array_ref).unwrap();
1154    }
1155
1156    #[test]
1157    fn series_append() {
1158        let mut s1 = Series::new("a".into(), &[1, 2]);
1159        let s2 = Series::new("b".into(), &[3]);
1160        s1.append(&s2).unwrap();
1161        assert_eq!(s1.len(), 3);
1162
1163        // add wrong type
1164        let s2 = Series::new("b".into(), &[3.0]);
1165        assert!(s1.append(&s2).is_err())
1166    }
1167
1168    #[test]
1169    #[cfg(feature = "dtype-decimal")]
1170    fn series_append_decimal() {
1171        let s1 = Series::new("a".into(), &[1.1, 2.3])
1172            .cast(&DataType::Decimal(None, Some(2)))
1173            .unwrap();
1174        let s2 = Series::new("b".into(), &[3])
1175            .cast(&DataType::Decimal(None, Some(0)))
1176            .unwrap();
1177
1178        {
1179            let mut s1 = s1.clone();
1180            s1.append(&s2).unwrap();
1181            assert_eq!(s1.len(), 3);
1182            assert_eq!(s1.get(2).unwrap(), AnyValue::Decimal(300, 2));
1183        }
1184
1185        {
1186            let mut s2 = s2;
1187            s2.extend(&s1).unwrap();
1188            assert_eq!(s2.get(2).unwrap(), AnyValue::Decimal(2, 0));
1189        }
1190    }
1191
1192    #[test]
1193    fn series_slice_works() {
1194        let series = Series::new("a".into(), &[1i64, 2, 3, 4, 5]);
1195
1196        let slice_1 = series.slice(-3, 3);
1197        let slice_2 = series.slice(-5, 5);
1198        let slice_3 = series.slice(0, 5);
1199
1200        assert_eq!(slice_1.get(0).unwrap(), AnyValue::Int64(3));
1201        assert_eq!(slice_2.get(0).unwrap(), AnyValue::Int64(1));
1202        assert_eq!(slice_3.get(0).unwrap(), AnyValue::Int64(1));
1203    }
1204
1205    #[test]
1206    fn out_of_range_slice_does_not_panic() {
1207        let series = Series::new("a".into(), &[1i64, 2, 3, 4, 5]);
1208
1209        let _ = series.slice(-3, 4);
1210        let _ = series.slice(-6, 2);
1211        let _ = series.slice(4, 2);
1212    }
1213}