Path: blob/main/crates/polars-core/src/series/implementations/duration.rs
6940 views
use polars_compute::rolling::QuantileMethod;12use super::*;3use crate::chunked_array::comparison::*;4#[cfg(feature = "algorithm_group_by")]5use crate::frame::group_by::*;6use crate::prelude::*;78unsafe impl IntoSeries for DurationChunked {9fn into_series(self) -> Series {10Series(Arc::new(SeriesWrap(self)))11}12}1314impl private::PrivateSeriesNumeric for SeriesWrap<DurationChunked> {15fn bit_repr(&self) -> Option<BitRepr> {16Some(self.0.physical().to_bit_repr())17}18}1920impl private::PrivateSeries for SeriesWrap<DurationChunked> {21fn compute_len(&mut self) {22self.0.physical_mut().compute_len()23}24fn _field(&self) -> Cow<'_, Field> {25Cow::Owned(self.0.field())26}27fn _dtype(&self) -> &DataType {28self.0.dtype()29}3031fn _set_flags(&mut self, flags: StatisticsFlags) {32self.0.physical_mut().set_flags(flags)33}3435fn _get_flags(&self) -> StatisticsFlags {36self.0.physical().get_flags()37}3839unsafe fn equal_element(&self, idx_self: usize, idx_other: usize, other: &Series) -> bool {40self.0.physical().equal_element(idx_self, idx_other, other)41}4243#[cfg(feature = "zip_with")]44fn zip_with_same_type(&self, mask: &BooleanChunked, other: &Series) -> PolarsResult<Series> {45let other = other.to_physical_repr().into_owned();46self.047.physical()48.zip_with(mask, other.as_ref().as_ref())49.map(|ca| ca.into_duration(self.0.time_unit()).into_series())50}5152fn into_total_eq_inner<'a>(&'a self) -> Box<dyn TotalEqInner + 'a> {53self.0.physical().into_total_eq_inner()54}55fn into_total_ord_inner<'a>(&'a self) -> Box<dyn TotalOrdInner + 'a> {56self.0.physical().into_total_ord_inner()57}5859fn vec_hash(60&self,61random_state: PlSeedableRandomStateQuality,62buf: &mut Vec<u64>,63) -> PolarsResult<()> {64self.0.physical().vec_hash(random_state, buf)?;65Ok(())66}6768fn vec_hash_combine(69&self,70build_hasher: PlSeedableRandomStateQuality,71hashes: &mut [u64],72) -> PolarsResult<()> {73self.0.physical().vec_hash_combine(build_hasher, hashes)?;74Ok(())75}7677#[cfg(feature = "algorithm_group_by")]78unsafe fn agg_min(&self, groups: &GroupsType) -> Series {79self.080.physical()81.agg_min(groups)82.into_duration(self.0.time_unit())83.into_series()84}8586#[cfg(feature = "algorithm_group_by")]87unsafe fn agg_max(&self, groups: &GroupsType) -> Series {88self.089.physical()90.agg_max(groups)91.into_duration(self.0.time_unit())92.into_series()93}9495#[cfg(feature = "algorithm_group_by")]96unsafe fn agg_sum(&self, groups: &GroupsType) -> Series {97self.098.physical()99.agg_sum(groups)100.into_duration(self.0.time_unit())101.into_series()102}103104#[cfg(feature = "algorithm_group_by")]105unsafe fn agg_std(&self, groups: &GroupsType, ddof: u8) -> Series {106self.0107.physical()108.agg_std(groups, ddof)109// cast f64 back to physical type110.cast(&DataType::Int64)111.unwrap()112.into_duration(self.0.time_unit())113.into_series()114}115116#[cfg(feature = "algorithm_group_by")]117unsafe fn agg_var(&self, groups: &GroupsType, ddof: u8) -> Series {118self.0119.physical()120.agg_var(groups, ddof)121// cast f64 back to physical type122.cast(&DataType::Int64)123.unwrap()124.into_duration(self.0.time_unit())125.into_series()126}127128#[cfg(feature = "algorithm_group_by")]129unsafe fn agg_list(&self, groups: &GroupsType) -> Series {130// we cannot cast and dispatch as the inner type of the list would be incorrect131self.0132.physical()133.agg_list(groups)134.cast(&DataType::List(Box::new(self.dtype().clone())))135.unwrap()136}137138fn subtract(&self, rhs: &Series) -> PolarsResult<Series> {139match (self.dtype(), rhs.dtype()) {140(DataType::Duration(tu), DataType::Duration(tur)) => {141polars_ensure!(tu == tur, InvalidOperation: "units are different");142let lhs = self.cast(&DataType::Int64, CastOptions::NonStrict).unwrap();143let rhs = rhs.cast(&DataType::Int64).unwrap();144Ok(lhs.subtract(&rhs)?.into_duration(*tu).into_series())145},146(dtl, dtr) => polars_bail!(opq = sub, dtl, dtr),147}148}149fn add_to(&self, rhs: &Series) -> PolarsResult<Series> {150match (self.dtype(), rhs.dtype()) {151(DataType::Duration(tu), DataType::Duration(tur)) => {152polars_ensure!(tu == tur, InvalidOperation: "units are different");153let lhs = self.cast(&DataType::Int64, CastOptions::NonStrict).unwrap();154let rhs = rhs.cast(&DataType::Int64).unwrap();155Ok(lhs.add_to(&rhs)?.into_duration(*tu).into_series())156},157(DataType::Duration(tu), DataType::Date) => {158let one_day_in_tu: i64 = match tu {159TimeUnit::Milliseconds => 86_400_000,160TimeUnit::Microseconds => 86_400_000_000,161TimeUnit::Nanoseconds => 86_400_000_000_000,162};163let lhs =164self.cast(&DataType::Int64, CastOptions::NonStrict).unwrap() / one_day_in_tu;165let rhs = rhs166.cast(&DataType::Int32)167.unwrap()168.cast(&DataType::Int64)169.unwrap();170Ok(lhs171.add_to(&rhs)?172.cast(&DataType::Int32)?173.into_date()174.into_series())175},176(DataType::Duration(tu), DataType::Datetime(tur, tz)) => {177polars_ensure!(tu == tur, InvalidOperation: "units are different");178let lhs = self.cast(&DataType::Int64, CastOptions::NonStrict).unwrap();179let rhs = rhs.cast(&DataType::Int64).unwrap();180Ok(lhs181.add_to(&rhs)?182.into_datetime(*tu, tz.clone())183.into_series())184},185(dtl, dtr) => polars_bail!(opq = add, dtl, dtr),186}187}188fn multiply(&self, rhs: &Series) -> PolarsResult<Series> {189let tul = self.0.time_unit();190match rhs.dtype() {191DataType::Int64 => Ok((&self.0.phys * rhs.i64().unwrap())192.into_duration(tul)193.into_series()),194dt if dt.is_integer() => {195let rhs = rhs.cast(&DataType::Int64)?;196self.multiply(&rhs)197},198dt if dt.is_float() => {199let phys = &self.0.phys;200let phys_float = phys.cast(dt).unwrap();201let out = std::ops::Mul::mul(&phys_float, rhs)?202.cast(&DataType::Int64)203.unwrap();204let phys = out.i64().unwrap().clone();205Ok(phys.into_duration(tul).into_series())206},207_ => {208polars_bail!(opq = mul, self.dtype(), rhs.dtype());209},210}211}212fn divide(&self, rhs: &Series) -> PolarsResult<Series> {213let tul = self.0.time_unit();214match rhs.dtype() {215DataType::Duration(tur) => {216if tul == *tur {217// Returns a constant as f64.218Ok(std::ops::Div::div(219&self.0.phys.cast(&DataType::Float64).unwrap(),220&rhs.duration()221.unwrap()222.phys223.cast(&DataType::Float64)224.unwrap(),225)?226.into_series())227} else {228let rhs = rhs.cast(self.dtype())?;229self.divide(&rhs)230}231},232DataType::Int64 => Ok((&self.0.phys / rhs.i64().unwrap())233.into_duration(tul)234.into_series()),235dt if dt.is_integer() => {236let rhs = rhs.cast(&DataType::Int64)?;237self.divide(&rhs)238},239dt if dt.is_float() => {240let phys = &self.0.phys;241let phys_float = phys.cast(dt).unwrap();242let out = std::ops::Div::div(&phys_float, rhs)?243.cast(&DataType::Int64)244.unwrap();245let phys = out.i64().unwrap().clone();246Ok(phys.into_duration(tul).into_series())247},248_ => {249polars_bail!(opq = div, self.dtype(), rhs.dtype());250},251}252}253fn remainder(&self, rhs: &Series) -> PolarsResult<Series> {254polars_ensure!(self.dtype() == rhs.dtype(), InvalidOperation: "dtypes and units must be equal in duration arithmetic");255let lhs = self.cast(&DataType::Int64, CastOptions::NonStrict).unwrap();256let rhs = rhs.cast(&DataType::Int64).unwrap();257Ok(lhs258.remainder(&rhs)?259.into_duration(self.0.time_unit())260.into_series())261}262#[cfg(feature = "algorithm_group_by")]263fn group_tuples(&self, multithreaded: bool, sorted: bool) -> PolarsResult<GroupsType> {264self.0.physical().group_tuples(multithreaded, sorted)265}266267fn arg_sort_multiple(268&self,269by: &[Column],270options: &SortMultipleOptions,271) -> PolarsResult<IdxCa> {272self.0.physical().arg_sort_multiple(by, options)273}274}275276impl SeriesTrait for SeriesWrap<DurationChunked> {277fn rename(&mut self, name: PlSmallStr) {278self.0.rename(name);279}280281fn chunk_lengths(&self) -> ChunkLenIter<'_> {282self.0.physical().chunk_lengths()283}284fn name(&self) -> &PlSmallStr {285self.0.name()286}287288fn chunks(&self) -> &Vec<ArrayRef> {289self.0.physical().chunks()290}291unsafe fn chunks_mut(&mut self) -> &mut Vec<ArrayRef> {292self.0.physical_mut().chunks_mut()293}294295fn shrink_to_fit(&mut self) {296self.0.physical_mut().shrink_to_fit()297}298299fn slice(&self, offset: i64, length: usize) -> Series {300self.0.slice(offset, length).into_series()301}302303fn split_at(&self, offset: i64) -> (Series, Series) {304let (a, b) = self.0.split_at(offset);305(a.into_series(), b.into_series())306}307308fn _sum_as_f64(&self) -> f64 {309self.0.physical()._sum_as_f64()310}311312fn mean(&self) -> Option<f64> {313self.0.physical().mean()314}315316fn median(&self) -> Option<f64> {317self.0.physical().median()318}319320fn std(&self, ddof: u8) -> Option<f64> {321self.0.physical().std(ddof)322}323324fn var(&self, ddof: u8) -> Option<f64> {325self.0.physical().var(ddof)326}327328fn append(&mut self, other: &Series) -> PolarsResult<()> {329polars_ensure!(self.0.dtype() == other.dtype(), append);330let mut other = other.to_physical_repr().into_owned();331self.0332.physical_mut()333.append_owned(std::mem::take(other._get_inner_mut().as_mut()))334}335fn append_owned(&mut self, mut other: Series) -> PolarsResult<()> {336polars_ensure!(self.0.dtype() == other.dtype(), append);337self.0.physical_mut().append_owned(std::mem::take(338&mut other339._get_inner_mut()340.as_any_mut()341.downcast_mut::<DurationChunked>()342.unwrap()343.phys,344))345}346347fn extend(&mut self, other: &Series) -> PolarsResult<()> {348polars_ensure!(self.0.dtype() == other.dtype(), extend);349let other = other.to_physical_repr();350self.0351.physical_mut()352.extend(other.as_ref().as_ref().as_ref())?;353Ok(())354}355356fn filter(&self, filter: &BooleanChunked) -> PolarsResult<Series> {357self.0358.physical()359.filter(filter)360.map(|ca| ca.into_duration(self.0.time_unit()).into_series())361}362363fn take(&self, indices: &IdxCa) -> PolarsResult<Series> {364Ok(self365.0366.physical()367.take(indices)?368.into_duration(self.0.time_unit())369.into_series())370}371372unsafe fn take_unchecked(&self, indices: &IdxCa) -> Series {373self.0374.physical()375.take_unchecked(indices)376.into_duration(self.0.time_unit())377.into_series()378}379380fn take_slice(&self, indices: &[IdxSize]) -> PolarsResult<Series> {381Ok(self382.0383.physical()384.take(indices)?385.into_duration(self.0.time_unit())386.into_series())387}388389unsafe fn take_slice_unchecked(&self, indices: &[IdxSize]) -> Series {390self.0391.physical()392.take_unchecked(indices)393.into_duration(self.0.time_unit())394.into_series()395}396397fn len(&self) -> usize {398self.0.len()399}400401fn rechunk(&self) -> Series {402self.0403.physical()404.rechunk()405.into_owned()406.into_duration(self.0.time_unit())407.into_series()408}409410fn new_from_index(&self, index: usize, length: usize) -> Series {411self.0412.physical()413.new_from_index(index, length)414.into_duration(self.0.time_unit())415.into_series()416}417418fn cast(&self, dtype: &DataType, cast_options: CastOptions) -> PolarsResult<Series> {419self.0.cast_with_options(dtype, cast_options)420}421422#[inline]423unsafe fn get_unchecked(&self, index: usize) -> AnyValue<'_> {424self.0.get_any_value_unchecked(index)425}426427fn sort_with(&self, options: SortOptions) -> PolarsResult<Series> {428Ok(self429.0430.physical()431.sort_with(options)432.into_duration(self.0.time_unit())433.into_series())434}435436fn arg_sort(&self, options: SortOptions) -> IdxCa {437self.0.physical().arg_sort(options)438}439440fn null_count(&self) -> usize {441self.0.null_count()442}443444fn has_nulls(&self) -> bool {445self.0.has_nulls()446}447448#[cfg(feature = "algorithm_group_by")]449fn unique(&self) -> PolarsResult<Series> {450self.0451.physical()452.unique()453.map(|ca| ca.into_duration(self.0.time_unit()).into_series())454}455456#[cfg(feature = "algorithm_group_by")]457fn n_unique(&self) -> PolarsResult<usize> {458self.0.physical().n_unique()459}460461#[cfg(feature = "algorithm_group_by")]462fn arg_unique(&self) -> PolarsResult<IdxCa> {463self.0.physical().arg_unique()464}465466fn is_null(&self) -> BooleanChunked {467self.0.is_null()468}469470fn is_not_null(&self) -> BooleanChunked {471self.0.is_not_null()472}473474fn reverse(&self) -> Series {475self.0476.physical()477.reverse()478.into_duration(self.0.time_unit())479.into_series()480}481482fn as_single_ptr(&mut self) -> PolarsResult<usize> {483self.0.physical_mut().as_single_ptr()484}485486fn shift(&self, periods: i64) -> Series {487self.0488.physical()489.shift(periods)490.into_duration(self.0.time_unit())491.into_series()492}493494fn sum_reduce(&self) -> PolarsResult<Scalar> {495let sc = self.0.physical().sum_reduce();496let v = sc.value().as_duration(self.0.time_unit());497Ok(Scalar::new(self.dtype().clone(), v))498}499500fn max_reduce(&self) -> PolarsResult<Scalar> {501let sc = self.0.physical().max_reduce();502let v = sc.value().as_duration(self.0.time_unit());503Ok(Scalar::new(self.dtype().clone(), v))504}505fn min_reduce(&self) -> PolarsResult<Scalar> {506let sc = self.0.physical().min_reduce();507let v = sc.value().as_duration(self.0.time_unit());508Ok(Scalar::new(self.dtype().clone(), v))509}510fn std_reduce(&self, ddof: u8) -> PolarsResult<Scalar> {511let sc = self.0.physical().std_reduce(ddof);512let to = self.dtype().to_physical();513let v = sc.value().cast(&to);514Ok(Scalar::new(515self.dtype().clone(),516v.as_duration(self.0.time_unit()),517))518}519520fn median_reduce(&self) -> PolarsResult<Scalar> {521let v: AnyValue = self.median().map(|v| v as i64).into();522let to = self.dtype().to_physical();523let v = v.cast(&to);524Ok(Scalar::new(525self.dtype().clone(),526v.as_duration(self.0.time_unit()),527))528}529fn quantile_reduce(&self, quantile: f64, method: QuantileMethod) -> PolarsResult<Scalar> {530let v = self.0.physical().quantile_reduce(quantile, method)?;531let to = self.dtype().to_physical();532let v = v.value().cast(&to);533Ok(Scalar::new(534self.dtype().clone(),535v.as_duration(self.0.time_unit()),536))537}538539fn clone_inner(&self) -> Arc<dyn SeriesTrait> {540Arc::new(SeriesWrap(Clone::clone(&self.0)))541}542543fn find_validity_mismatch(&self, other: &Series, idxs: &mut Vec<IdxSize>) {544self.0.physical().find_validity_mismatch(other, idxs)545}546547fn as_any(&self) -> &dyn Any {548&self.0549}550551fn as_any_mut(&mut self) -> &mut dyn Any {552&mut self.0553}554555fn as_phys_any(&self) -> &dyn Any {556self.0.physical()557}558559fn as_arc_any(self: Arc<Self>) -> Arc<dyn Any + Send + Sync> {560self as _561}562}563564565