polars_core/series/
mod.rs

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