Path: blob/main/crates/polars-core/src/series/series_trait.rs
6940 views
use std::any::Any;1use std::borrow::Cow;23use arrow::bitmap::{Bitmap, BitmapBuilder};4use polars_compute::rolling::QuantileMethod;5#[cfg(feature = "serde")]6use serde::{Deserialize, Serialize};78use crate::chunked_array::cast::CastOptions;9#[cfg(feature = "object")]10use crate::chunked_array::object::PolarsObjectSafe;11use crate::prelude::*;1213#[derive(Debug, Copy, Clone, Eq, PartialEq, Hash)]14#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]15#[cfg_attr(feature = "dsl-schema", derive(schemars::JsonSchema))]16pub enum IsSorted {17Ascending,18Descending,19Not,20}2122impl IsSorted {23pub fn reverse(self) -> Self {24use IsSorted::*;25match self {26Ascending => Descending,27Descending => Ascending,28Not => Not,29}30}31}3233pub enum BitRepr {34U8(UInt8Chunked),35U16(UInt16Chunked),36U32(UInt32Chunked),37U64(UInt64Chunked),38#[cfg(feature = "dtype-i128")]39I128(Int128Chunked),40}4142pub(crate) mod private {43use polars_utils::aliases::PlSeedableRandomStateQuality;4445use super::*;46use crate::chunked_array::flags::StatisticsFlags;47use crate::chunked_array::ops::compare_inner::{TotalEqInner, TotalOrdInner};4849pub trait PrivateSeriesNumeric {50/// Return a bit representation51///52/// If there is no available bit representation this returns `None`.53fn bit_repr(&self) -> Option<BitRepr>;54}5556pub trait PrivateSeries {57#[cfg(feature = "object")]58fn get_list_builder(59&self,60_name: PlSmallStr,61_values_capacity: usize,62_list_capacity: usize,63) -> Box<dyn ListBuilderTrait> {64invalid_operation_panic!(get_list_builder, self)65}6667/// Get field (used in schema)68fn _field(&self) -> Cow<'_, Field>;6970fn _dtype(&self) -> &DataType;7172fn compute_len(&mut self);7374fn _get_flags(&self) -> StatisticsFlags;7576fn _set_flags(&mut self, flags: StatisticsFlags);7778unsafe fn equal_element(79&self,80_idx_self: usize,81_idx_other: usize,82_other: &Series,83) -> bool {84invalid_operation_panic!(equal_element, self)85}86#[expect(clippy::wrong_self_convention)]87fn into_total_eq_inner<'a>(&'a self) -> Box<dyn TotalEqInner + 'a>;88#[expect(clippy::wrong_self_convention)]89fn into_total_ord_inner<'a>(&'a self) -> Box<dyn TotalOrdInner + 'a>;9091fn vec_hash(92&self,93_build_hasher: PlSeedableRandomStateQuality,94_buf: &mut Vec<u64>,95) -> PolarsResult<()>;96fn vec_hash_combine(97&self,98_build_hasher: PlSeedableRandomStateQuality,99_hashes: &mut [u64],100) -> PolarsResult<()>;101102/// # Safety103///104/// Does no bounds checks, groups must be correct.105#[cfg(feature = "algorithm_group_by")]106unsafe fn agg_min(&self, groups: &GroupsType) -> Series {107Series::full_null(self._field().name().clone(), groups.len(), self._dtype())108}109/// # Safety110///111/// Does no bounds checks, groups must be correct.112#[cfg(feature = "algorithm_group_by")]113unsafe fn agg_max(&self, groups: &GroupsType) -> Series {114Series::full_null(self._field().name().clone(), groups.len(), self._dtype())115}116/// If the [`DataType`] is one of `{Int8, UInt8, Int16, UInt16}` the `Series` is117/// first cast to `Int64` to prevent overflow issues.118#[cfg(feature = "algorithm_group_by")]119unsafe fn agg_sum(&self, groups: &GroupsType) -> Series {120Series::full_null(self._field().name().clone(), groups.len(), self._dtype())121}122/// # Safety123///124/// Does no bounds checks, groups must be correct.125#[cfg(feature = "algorithm_group_by")]126unsafe fn agg_std(&self, groups: &GroupsType, _ddof: u8) -> Series {127Series::full_null(self._field().name().clone(), groups.len(), self._dtype())128}129/// # Safety130///131/// Does no bounds checks, groups must be correct.132#[cfg(feature = "algorithm_group_by")]133unsafe fn agg_var(&self, groups: &GroupsType, _ddof: u8) -> Series {134Series::full_null(self._field().name().clone(), groups.len(), self._dtype())135}136/// # Safety137///138/// Does no bounds checks, groups must be correct.139#[cfg(feature = "algorithm_group_by")]140unsafe fn agg_list(&self, groups: &GroupsType) -> Series {141Series::full_null(self._field().name().clone(), groups.len(), self._dtype())142}143144/// # Safety145///146/// Does no bounds checks, groups must be correct.147#[cfg(feature = "bitwise")]148unsafe fn agg_and(&self, groups: &GroupsType) -> Series {149Series::full_null(self._field().name().clone(), groups.len(), self._dtype())150}151152/// # Safety153///154/// Does no bounds checks, groups must be correct.155#[cfg(feature = "bitwise")]156unsafe fn agg_or(&self, groups: &GroupsType) -> Series {157Series::full_null(self._field().name().clone(), groups.len(), self._dtype())158}159160/// # Safety161///162/// Does no bounds checks, groups must be correct.163#[cfg(feature = "bitwise")]164unsafe fn agg_xor(&self, groups: &GroupsType) -> Series {165Series::full_null(self._field().name().clone(), groups.len(), self._dtype())166}167168fn subtract(&self, _rhs: &Series) -> PolarsResult<Series> {169polars_bail!(opq = subtract, self._dtype());170}171fn add_to(&self, _rhs: &Series) -> PolarsResult<Series> {172polars_bail!(opq = add, self._dtype());173}174fn multiply(&self, _rhs: &Series) -> PolarsResult<Series> {175polars_bail!(opq = multiply, self._dtype());176}177fn divide(&self, _rhs: &Series) -> PolarsResult<Series> {178polars_bail!(opq = divide, self._dtype());179}180fn remainder(&self, _rhs: &Series) -> PolarsResult<Series> {181polars_bail!(opq = remainder, self._dtype());182}183#[cfg(feature = "algorithm_group_by")]184fn group_tuples(&self, _multithreaded: bool, _sorted: bool) -> PolarsResult<GroupsType> {185polars_bail!(opq = group_tuples, self._dtype());186}187#[cfg(feature = "zip_with")]188fn zip_with_same_type(189&self,190_mask: &BooleanChunked,191_other: &Series,192) -> PolarsResult<Series> {193polars_bail!(opq = zip_with_same_type, self._dtype());194}195196#[allow(unused_variables)]197fn arg_sort_multiple(198&self,199by: &[Column],200_options: &SortMultipleOptions,201) -> PolarsResult<IdxCa> {202polars_bail!(opq = arg_sort_multiple, self._dtype());203}204}205}206207pub trait SeriesTrait:208Send + Sync + private::PrivateSeries + private::PrivateSeriesNumeric209{210/// Rename the Series.211fn rename(&mut self, name: PlSmallStr);212213/// Get the lengths of the underlying chunks214fn chunk_lengths(&self) -> ChunkLenIter<'_>;215216/// Name of series.217fn name(&self) -> &PlSmallStr;218219/// Get field (used in schema)220fn field(&self) -> Cow<'_, Field> {221self._field()222}223224/// Get datatype of series.225fn dtype(&self) -> &DataType {226self._dtype()227}228229/// Underlying chunks.230fn chunks(&self) -> &Vec<ArrayRef>;231232/// Underlying chunks.233///234/// # Safety235/// The caller must ensure the length and the data types of `ArrayRef` does not change.236unsafe fn chunks_mut(&mut self) -> &mut Vec<ArrayRef>;237238/// Number of chunks in this Series239fn n_chunks(&self) -> usize {240self.chunks().len()241}242243/// Shrink the capacity of this array to fit its length.244fn shrink_to_fit(&mut self) {245// no-op246}247248/// Take `num_elements` from the top as a zero copy view.249fn limit(&self, num_elements: usize) -> Series {250self.slice(0, num_elements)251}252253/// Get a zero copy view of the data.254///255/// When offset is negative the offset is counted from the256/// end of the array257fn slice(&self, _offset: i64, _length: usize) -> Series;258259/// Get a zero copy view of the data.260///261/// When offset is negative the offset is counted from the262/// end of the array263fn split_at(&self, _offset: i64) -> (Series, Series);264265fn append(&mut self, other: &Series) -> PolarsResult<()>;266fn append_owned(&mut self, other: Series) -> PolarsResult<()>;267268#[doc(hidden)]269fn extend(&mut self, _other: &Series) -> PolarsResult<()>;270271/// Filter by boolean mask. This operation clones data.272fn filter(&self, _filter: &BooleanChunked) -> PolarsResult<Series>;273274/// Take from `self` at the indexes given by `idx`.275///276/// Null values in `idx` because null values in the output array.277///278/// This operation is clone.279fn take(&self, _indices: &IdxCa) -> PolarsResult<Series>;280281/// Take from `self` at the indexes given by `idx`.282///283/// Null values in `idx` because null values in the output array.284///285/// # Safety286/// This doesn't check any bounds.287unsafe fn take_unchecked(&self, _idx: &IdxCa) -> Series;288289/// Take from `self` at the indexes given by `idx`.290///291/// This operation is clone.292fn take_slice(&self, _indices: &[IdxSize]) -> PolarsResult<Series>;293294/// Take from `self` at the indexes given by `idx`.295///296/// # Safety297/// This doesn't check any bounds.298unsafe fn take_slice_unchecked(&self, _idx: &[IdxSize]) -> Series;299300/// Get length of series.301fn len(&self) -> usize;302303/// Check if Series is empty.304fn is_empty(&self) -> bool {305self.len() == 0306}307308/// Aggregate all chunks to a contiguous array of memory.309fn rechunk(&self) -> Series;310311fn rechunk_validity(&self) -> Option<Bitmap> {312if self.chunks().len() == 1 {313return self.chunks()[0].validity().cloned();314}315316if !self.has_nulls() || self.is_empty() {317return None;318}319320let mut bm = BitmapBuilder::with_capacity(self.len());321for arr in self.chunks() {322if let Some(v) = arr.validity() {323bm.extend_from_bitmap(v);324} else {325bm.extend_constant(arr.len(), true);326}327}328bm.into_opt_validity()329}330331/// Drop all null values and return a new Series.332fn drop_nulls(&self) -> Series {333if self.null_count() == 0 {334Series(self.clone_inner())335} else {336self.filter(&self.is_not_null()).unwrap()337}338}339340/// Returns the sum of the array as an f64.341fn _sum_as_f64(&self) -> f64 {342invalid_operation_panic!(_sum_as_f64, self)343}344345/// Returns the mean value in the array346/// Returns an option because the array is nullable.347fn mean(&self) -> Option<f64> {348None349}350351/// Returns the std value in the array352/// Returns an option because the array is nullable.353fn std(&self, _ddof: u8) -> Option<f64> {354None355}356357/// Returns the var value in the array358/// Returns an option because the array is nullable.359fn var(&self, _ddof: u8) -> Option<f64> {360None361}362363/// Returns the median value in the array364/// Returns an option because the array is nullable.365fn median(&self) -> Option<f64> {366None367}368369/// Create a new Series filled with values from the given index.370///371/// # Example372///373/// ```rust374/// use polars_core::prelude::*;375/// let s = Series::new("a".into(), [0i32, 1, 8]);376/// let s2 = s.new_from_index(2, 4);377/// assert_eq!(Vec::from(s2.i32().unwrap()), &[Some(8), Some(8), Some(8), Some(8)])378/// ```379fn new_from_index(&self, _index: usize, _length: usize) -> Series;380381/// Trim all lists of unused start and end elements recursively.382///383/// - `None` if nothing needed to be done.384/// - `Some(series)` if something changed.385fn trim_lists_to_normalized_offsets(&self) -> Option<Series> {386None387}388389/// Propagate down nulls in nested types.390///391/// - `None` if nothing needed to be done.392/// - `Some(series)` if something changed.393fn propagate_nulls(&self) -> Option<Series> {394None395}396397/// Find the indices of elements where the null masks are different recursively.398fn find_validity_mismatch(&self, other: &Series, idxs: &mut Vec<IdxSize>);399400fn cast(&self, _dtype: &DataType, options: CastOptions) -> PolarsResult<Series>;401402/// Get a single value by index. Don't use this operation for loops as a runtime cast is403/// needed for every iteration.404fn get(&self, index: usize) -> PolarsResult<AnyValue<'_>> {405polars_ensure!(index < self.len(), oob = index, self.len());406// SAFETY: Just did bounds check407let value = unsafe { self.get_unchecked(index) };408Ok(value)409}410411/// Get a single value by index. Don't use this operation for loops as a runtime cast is412/// needed for every iteration.413///414/// This may refer to physical types415///416/// # Safety417/// Does not do any bounds checking418unsafe fn get_unchecked(&self, _index: usize) -> AnyValue<'_>;419420fn sort_with(&self, _options: SortOptions) -> PolarsResult<Series> {421polars_bail!(opq = sort_with, self._dtype());422}423424/// Retrieve the indexes needed for a sort.425#[allow(unused)]426fn arg_sort(&self, options: SortOptions) -> IdxCa {427invalid_operation_panic!(arg_sort, self)428}429430/// Count the null values.431fn null_count(&self) -> usize;432433/// Return if any the chunks in this [`ChunkedArray`] have nulls.434fn has_nulls(&self) -> bool;435436/// Get unique values in the Series.437fn unique(&self) -> PolarsResult<Series> {438polars_bail!(opq = unique, self._dtype());439}440441/// Get unique values in the Series.442///443/// A `null` value also counts as a unique value.444fn n_unique(&self) -> PolarsResult<usize> {445polars_bail!(opq = n_unique, self._dtype());446}447448/// Get first indexes of unique values.449fn arg_unique(&self) -> PolarsResult<IdxCa> {450polars_bail!(opq = arg_unique, self._dtype());451}452453/// Get a mask of the null values.454fn is_null(&self) -> BooleanChunked;455456/// Get a mask of the non-null values.457fn is_not_null(&self) -> BooleanChunked;458459/// return a Series in reversed order460fn reverse(&self) -> Series;461462/// Rechunk and return a pointer to the start of the Series.463/// Only implemented for numeric types464fn as_single_ptr(&mut self) -> PolarsResult<usize> {465polars_bail!(opq = as_single_ptr, self._dtype());466}467468/// Shift the values by a given period and fill the parts that will be empty due to this operation469/// with `Nones`.470///471/// *NOTE: If you want to fill the Nones with a value use the472/// [`shift` operation on `ChunkedArray<T>`](../chunked_array/ops/trait.ChunkShift.html).*473///474/// # Example475///476/// ```rust477/// # use polars_core::prelude::*;478/// fn example() -> PolarsResult<()> {479/// let s = Series::new("series".into(), &[1, 2, 3]);480///481/// let shifted = s.shift(1);482/// assert_eq!(Vec::from(shifted.i32()?), &[None, Some(1), Some(2)]);483///484/// let shifted = s.shift(-1);485/// assert_eq!(Vec::from(shifted.i32()?), &[Some(2), Some(3), None]);486///487/// let shifted = s.shift(2);488/// assert_eq!(Vec::from(shifted.i32()?), &[None, None, Some(1)]);489///490/// Ok(())491/// }492/// example();493/// ```494fn shift(&self, _periods: i64) -> Series;495496/// Get the sum of the Series as a new Scalar.497///498/// If the [`DataType`] is one of `{Int8, UInt8, Int16, UInt16}` the `Series` is499/// first cast to `Int64` to prevent overflow issues.500fn sum_reduce(&self) -> PolarsResult<Scalar> {501polars_bail!(opq = sum, self._dtype());502}503/// Get the max of the Series as a new Series of length 1.504fn max_reduce(&self) -> PolarsResult<Scalar> {505polars_bail!(opq = max, self._dtype());506}507/// Get the min of the Series as a new Series of length 1.508fn min_reduce(&self) -> PolarsResult<Scalar> {509polars_bail!(opq = min, self._dtype());510}511/// Get the median of the Series as a new Series of length 1.512fn median_reduce(&self) -> PolarsResult<Scalar> {513polars_bail!(opq = median, self._dtype());514}515/// Get the variance of the Series as a new Series of length 1.516fn var_reduce(&self, _ddof: u8) -> PolarsResult<Scalar> {517polars_bail!(opq = var, self._dtype());518}519/// Get the standard deviation of the Series as a new Series of length 1.520fn std_reduce(&self, _ddof: u8) -> PolarsResult<Scalar> {521polars_bail!(opq = std, self._dtype());522}523/// Get the quantile of the ChunkedArray as a new Series of length 1.524fn quantile_reduce(&self, _quantile: f64, _method: QuantileMethod) -> PolarsResult<Scalar> {525polars_bail!(opq = quantile, self._dtype());526}527/// Get the bitwise AND of the Series as a new Series of length 1,528fn and_reduce(&self) -> PolarsResult<Scalar> {529polars_bail!(opq = and_reduce, self._dtype());530}531/// Get the bitwise OR of the Series as a new Series of length 1,532fn or_reduce(&self) -> PolarsResult<Scalar> {533polars_bail!(opq = or_reduce, self._dtype());534}535/// Get the bitwise XOR of the Series as a new Series of length 1,536fn xor_reduce(&self) -> PolarsResult<Scalar> {537polars_bail!(opq = xor_reduce, self._dtype());538}539540/// Get the first element of the [`Series`] as a [`Scalar`]541///542/// If the [`Series`] is empty, a [`Scalar`] with a [`AnyValue::Null`] is returned.543fn first(&self) -> Scalar {544let dt = self.dtype();545let av = self.get(0).map_or(AnyValue::Null, AnyValue::into_static);546547Scalar::new(dt.clone(), av)548}549550/// Get the last element of the [`Series`] as a [`Scalar`]551///552/// If the [`Series`] is empty, a [`Scalar`] with a [`AnyValue::Null`] is returned.553fn last(&self) -> Scalar {554let dt = self.dtype();555let av = if self.len() == 0 {556AnyValue::Null557} else {558// SAFETY: len-1 < len if len != 0559unsafe { self.get_unchecked(self.len() - 1) }.into_static()560};561562Scalar::new(dt.clone(), av)563}564565#[cfg(feature = "approx_unique")]566fn approx_n_unique(&self) -> PolarsResult<IdxSize> {567polars_bail!(opq = approx_n_unique, self._dtype());568}569570/// Clone inner ChunkedArray and wrap in a new Arc571fn clone_inner(&self) -> Arc<dyn SeriesTrait>;572573#[cfg(feature = "object")]574/// Get the value at this index as a downcastable Any trait ref.575fn get_object(&self, _index: usize) -> Option<&dyn PolarsObjectSafe> {576invalid_operation_panic!(get_object, self)577}578579#[cfg(feature = "object")]580/// Get the value at this index as a downcastable Any trait ref.581///582/// # Safety583/// This function doesn't do any bound checks.584unsafe fn get_object_chunked_unchecked(585&self,586_chunk: usize,587_index: usize,588) -> Option<&dyn PolarsObjectSafe> {589invalid_operation_panic!(get_object_chunked_unchecked, self)590}591592/// Get a hold of the [`ChunkedArray`], [`Logical`] or `NullChunked` as an `Any` trait593/// reference.594fn as_any(&self) -> &dyn Any;595596/// Get a hold of the [`ChunkedArray`], [`Logical`] or `NullChunked` as an `Any` trait mutable597/// reference.598fn as_any_mut(&mut self) -> &mut dyn Any;599600/// Get a hold of the [`ChunkedArray`] or `NullChunked` as an `Any` trait reference. This601/// pierces through `Logical` types to get the underlying physical array.602fn as_phys_any(&self) -> &dyn Any;603604fn as_arc_any(self: Arc<Self>) -> Arc<dyn Any + Send + Sync>;605606#[cfg(feature = "checked_arithmetic")]607fn checked_div(&self, _rhs: &Series) -> PolarsResult<Series> {608polars_bail!(opq = checked_div, self._dtype());609}610611#[cfg(feature = "rolling_window")]612/// Apply a custom function over a rolling/ moving window of the array.613/// This has quite some dynamic dispatch, so prefer rolling_min, max, mean, sum over this.614fn rolling_map(615&self,616_f: &dyn Fn(&Series) -> PolarsResult<Series>,617_options: RollingOptionsFixedWindow,618) -> PolarsResult<Series> {619polars_bail!(opq = rolling_map, self._dtype());620}621}622623impl dyn SeriesTrait + '_ {624pub fn unpack<T: PolarsPhysicalType>(&self) -> PolarsResult<&ChunkedArray<T>> {625polars_ensure!(&T::get_static_dtype() == self.dtype(), unpack);626Ok(self.as_ref())627}628}629630631