pub use crate::prelude::ChunkCompare;
use crate::prelude::*;
use arrow::{array::ArrayRef, buffer::Buffer};
pub(crate) mod arithmetic;
mod comparison;
pub mod implementations;
pub(crate) mod iterator;
use crate::chunked_array::{builder::get_list_builder, float::IsNan, ChunkIdIter};
use crate::series::arithmetic::coerce_lhs_rhs;
use crate::utils::get_supertype;
use arrow::array::ArrayData;
use arrow::compute::cast;
use itertools::Itertools;
use num::NumCast;
use std::any::Any;
use std::borrow::Cow;
use std::convert::TryFrom;
use std::ops::Deref;
use std::sync::Arc;
pub trait IntoSeries {
fn into_series(self) -> Series
where
Self: Sized;
}
pub(crate) mod private {
use super::*;
#[cfg(feature = "pivot")]
use crate::frame::groupby::pivot::PivotAgg;
use crate::frame::groupby::GroupTuples;
use ahash::RandomState;
pub trait PrivateSeries {
unsafe fn equal_element(
&self,
_idx_self: usize,
_idx_other: usize,
_other: &Series,
) -> bool {
unimplemented!()
}
fn vec_hash(&self, _build_hasher: RandomState) -> UInt64Chunked {
unimplemented!()
}
fn agg_mean(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_min(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_max(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_sum(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_std(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_var(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_first(&self, _groups: &[(u32, Vec<u32>)]) -> Series {
unimplemented!()
}
fn agg_last(&self, _groups: &[(u32, Vec<u32>)]) -> Series {
unimplemented!()
}
fn agg_n_unique(&self, _groups: &[(u32, Vec<u32>)]) -> Option<UInt32Chunked> {
unimplemented!()
}
fn agg_list(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
fn agg_quantile(&self, _groups: &[(u32, Vec<u32>)], _quantile: f64) -> Option<Series> {
unimplemented!()
}
fn agg_median(&self, _groups: &[(u32, Vec<u32>)]) -> Option<Series> {
unimplemented!()
}
#[cfg(feature = "pivot")]
fn pivot<'a>(
&self,
_pivot_series: &'a (dyn SeriesTrait + 'a),
_keys: Vec<Series>,
_groups: &[(u32, Vec<u32>)],
_agg_type: PivotAgg,
) -> Result<DataFrame> {
unimplemented!()
}
#[cfg(feature = "pivot")]
fn pivot_count<'a>(
&self,
_pivot_series: &'a (dyn SeriesTrait + 'a),
_keys: Vec<Series>,
_groups: &[(u32, Vec<u32>)],
) -> Result<DataFrame> {
unimplemented!()
}
fn hash_join_inner(&self, _other: &Series) -> Vec<(u32, u32)> {
unimplemented!()
}
fn hash_join_left(&self, _other: &Series) -> Vec<(u32, Option<u32>)> {
unimplemented!()
}
fn hash_join_outer(&self, _other: &Series) -> Vec<(Option<u32>, Option<u32>)> {
unimplemented!()
}
fn zip_outer_join_column(
&self,
_right_column: &Series,
_opt_join_tuples: &[(Option<u32>, Option<u32>)],
) -> Series {
unimplemented!()
}
fn subtract(&self, _rhs: &Series) -> Result<Series> {
unimplemented!()
}
fn add_to(&self, _rhs: &Series) -> Result<Series> {
unimplemented!()
}
fn multiply(&self, _rhs: &Series) -> Result<Series> {
unimplemented!()
}
fn divide(&self, _rhs: &Series) -> Result<Series> {
unimplemented!()
}
fn remainder(&self, _rhs: &Series) -> Result<Series> {
unimplemented!()
}
fn group_tuples(&self, _multithreaded: bool) -> GroupTuples {
unimplemented!()
}
fn zip_with_same_type(&self, _mask: &BooleanChunked, _other: &Series) -> Result<Series> {
unimplemented!()
}
fn is_in_same_type(&self, _list_array: &ListChunked) -> Result<BooleanChunked> {
unimplemented!()
}
}
}
pub trait SeriesTrait: Send + Sync + private::PrivateSeries {
fn cum_max(&self, _reverse: bool) -> Series {
panic!("operation cum_max not supported for this dtype")
}
fn cum_min(&self, _reverse: bool) -> Series {
panic!("operation cum_min not supported for this dtype")
}
fn cum_sum(&self, _reverse: bool) -> Series {
panic!("operation cum_sum not supported for this dtype")
}
fn rename(&mut self, name: &str);
fn array_data(&self) -> Vec<&ArrayData> {
unimplemented!()
}
fn chunk_lengths(&self) -> ChunkIdIter {
unimplemented!()
}
fn name(&self) -> &str {
unimplemented!()
}
fn field(&self) -> &Field {
unimplemented!()
}
fn dtype(&self) -> &DataType {
self.field().data_type()
}
fn chunks(&self) -> &Vec<ArrayRef> {
unimplemented!()
}
fn n_chunks(&self) -> usize {
self.chunks().len()
}
fn i8(&self) -> Result<&Int8Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != i8", self.dtype()).into(),
))
}
fn i16(&self) -> Result<&Int16Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != i16", self.dtype()).into(),
))
}
fn i32(&self) -> Result<&Int32Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != i32", self.dtype()).into(),
))
}
fn i64(&self) -> Result<&Int64Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != i64", self.dtype()).into(),
))
}
fn f32(&self) -> Result<&Float32Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != f32", self.dtype()).into(),
))
}
fn f64(&self) -> Result<&Float64Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != f64", self.dtype()).into(),
))
}
fn u8(&self) -> Result<&UInt8Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != u8", self.dtype()).into(),
))
}
fn u16(&self) -> Result<&UInt16Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != u16", self.dtype()).into(),
))
}
fn u32(&self) -> Result<&UInt32Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != u32", self.dtype()).into(),
))
}
fn u64(&self) -> Result<&UInt64Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != u32", self.dtype()).into(),
))
}
fn bool(&self) -> Result<&BooleanChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != bool", self.dtype()).into(),
))
}
fn utf8(&self) -> Result<&Utf8Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != utf8", self.dtype()).into(),
))
}
fn date32(&self) -> Result<&Date32Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != date32", self.dtype()).into(),
))
}
fn date64(&self) -> Result<&Date64Chunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != date64", self.dtype()).into(),
))
}
fn time64_nanosecond(&self) -> Result<&Time64NanosecondChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != time64", self.dtype()).into(),
))
}
fn duration_nanosecond(&self) -> Result<&DurationNanosecondChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != duration_nanosecond", self.dtype()).into(),
))
}
fn duration_millisecond(&self) -> Result<&DurationMillisecondChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} !== duration_millisecond", self.dtype()).into(),
))
}
fn list(&self) -> Result<&ListChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != list", self.dtype()).into(),
))
}
fn categorical(&self) -> Result<&CategoricalChunked> {
Err(PolarsError::DataTypeMisMatch(
format!("{:?} != categorical", self.dtype()).into(),
))
}
fn append_array(&mut self, _other: ArrayRef) -> Result<()> {
unimplemented!()
}
fn limit(&self, num_elements: usize) -> Series {
self.slice(0, num_elements)
}
fn slice(&self, _offset: i64, _length: usize) -> Series {
unimplemented!()
}
fn append(&mut self, _other: &Series) -> Result<()> {
unimplemented!()
}
fn filter(&self, _filter: &BooleanChunked) -> Result<Series> {
unimplemented!()
}
fn take_iter(&self, _iter: &mut dyn Iterator<Item = usize>) -> Series {
unimplemented!()
}
unsafe fn take_iter_unchecked(&self, _iter: &mut dyn Iterator<Item = usize>) -> Series {
unimplemented!()
}
unsafe fn take_unchecked(&self, _idx: &UInt32Chunked) -> Result<Series> {
unimplemented!()
}
unsafe fn take_opt_iter_unchecked(
&self,
_iter: &mut dyn Iterator<Item = Option<usize>>,
) -> Series {
unimplemented!()
}
fn take_opt_iter(&self, _iter: &mut dyn Iterator<Item = Option<usize>>) -> Series {
unimplemented!()
}
fn take(&self, _indices: &UInt32Chunked) -> Series {
unimplemented!()
}
fn len(&self) -> usize {
unimplemented!()
}
fn is_empty(&self) -> bool {
self.len() == 0
}
fn rechunk(&self) -> Series {
unimplemented!()
}
fn head(&self, _length: Option<usize>) -> Series {
unimplemented!()
}
fn tail(&self, _length: Option<usize>) -> Series {
unimplemented!()
}
fn take_every(&self, n: usize) -> Series;
fn drop_nulls(&self) -> Series {
if self.null_count() == 0 {
Series(self.clone_inner())
} else {
self.filter(&self.is_not_null()).unwrap()
}
}
fn mean(&self) -> Option<f64> {
unimplemented!()
}
fn median(&self) -> Option<f64> {
unimplemented!()
}
fn expand_at_index(&self, _index: usize, _length: usize) -> Series {
unimplemented!()
}
fn cast_with_datatype(&self, _data_type: &DataType) -> Result<Series> {
unimplemented!()
}
fn to_dummies(&self) -> Result<DataFrame> {
unimplemented!()
}
fn value_counts(&self) -> Result<DataFrame> {
unimplemented!()
}
fn get(&self, _index: usize) -> AnyValue {
unimplemented!()
}
unsafe fn get_unchecked(&self, _index: usize) -> AnyValue {
unimplemented!()
}
fn sort_in_place(&mut self, _reverse: bool) {
unimplemented!()
}
fn sort(&self, _reverse: bool) -> Series {
unimplemented!()
}
fn argsort(&self, _reverse: bool) -> UInt32Chunked {
unimplemented!()
}
fn null_count(&self) -> usize {
unimplemented!()
}
fn unique(&self) -> Result<Series> {
unimplemented!()
}
fn n_unique(&self) -> Result<usize> {
unimplemented!()
}
fn arg_unique(&self) -> Result<UInt32Chunked> {
unimplemented!()
}
fn arg_min(&self) -> Option<usize> {
unimplemented!()
}
fn arg_max(&self) -> Option<usize> {
unimplemented!()
}
fn arg_true(&self) -> Result<UInt32Chunked> {
Err(PolarsError::InvalidOperation(
"arg_true can only be called for boolean dtype".into(),
))
}
fn is_null(&self) -> BooleanChunked {
unimplemented!()
}
fn is_not_null(&self) -> BooleanChunked {
unimplemented!()
}
fn is_unique(&self) -> Result<BooleanChunked> {
unimplemented!()
}
fn is_duplicated(&self) -> Result<BooleanChunked> {
unimplemented!()
}
fn null_bits(&self) -> Vec<(usize, Option<Buffer>)> {
unimplemented!()
}
fn reverse(&self) -> Series {
unimplemented!()
}
fn as_single_ptr(&mut self) -> Result<usize> {
Err(PolarsError::InvalidOperation(
"operation 'as_single_ptr' not supported".into(),
))
}
fn shift(&self, _periods: i64) -> Series {
unimplemented!()
}
fn fill_none(&self, _strategy: FillNoneStrategy) -> Result<Series> {
unimplemented!()
}
fn sum_as_series(&self) -> Series {
unimplemented!()
}
fn max_as_series(&self) -> Series {
unimplemented!()
}
fn min_as_series(&self) -> Series {
unimplemented!()
}
fn mean_as_series(&self) -> Series {
unimplemented!()
}
fn median_as_series(&self) -> Series {
unimplemented!()
}
fn var_as_series(&self) -> Series {
unimplemented!()
}
fn std_as_series(&self) -> Series {
unimplemented!()
}
fn quantile_as_series(&self, _quantile: f64) -> Result<Series> {
unimplemented!()
}
fn rolling_mean(
&self,
_window_size: u32,
_weight: Option<&[f64]>,
_ignore_null: bool,
_min_periods: u32,
) -> Result<Series> {
unimplemented!()
}
fn rolling_sum(
&self,
_window_size: u32,
_weight: Option<&[f64]>,
_ignore_null: bool,
_min_periods: u32,
) -> Result<Series> {
unimplemented!()
}
fn rolling_min(
&self,
_window_size: u32,
_weight: Option<&[f64]>,
_ignore_null: bool,
_min_periods: u32,
) -> Result<Series> {
unimplemented!()
}
fn rolling_max(
&self,
_window_size: u32,
_weight: Option<&[f64]>,
_ignore_null: bool,
_min_periods: u32,
) -> Result<Series> {
unimplemented!()
}
fn fmt_list(&self) -> String {
"fmt implemented".into()
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn hour(&self) -> Result<UInt32Chunked> {
self.date64().map(|ca| ca.hour())
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn minute(&self) -> Result<UInt32Chunked> {
self.date64().map(|ca| ca.minute())
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn second(&self) -> Result<UInt32Chunked> {
self.date64().map(|ca| ca.second())
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn nanosecond(&self) -> Result<UInt32Chunked> {
self.date64().map(|ca| ca.nanosecond())
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn day(&self) -> Result<UInt32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.day()),
DataType::Date64 => self.date64().map(|ca| ca.day()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn weekday(&self) -> Result<UInt32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.weekday()),
DataType::Date64 => self.date64().map(|ca| ca.weekday()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn week(&self) -> Result<UInt32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.week()),
DataType::Date64 => self.date64().map(|ca| ca.week()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn ordinal_day(&self) -> Result<UInt32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.ordinal()),
DataType::Date64 => self.date64().map(|ca| ca.ordinal()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn month(&self) -> Result<UInt32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.month()),
DataType::Date64 => self.date64().map(|ca| ca.month()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn year(&self) -> Result<Int32Chunked> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.year()),
DataType::Date64 => self.date64().map(|ca| ca.year()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
fn datetime_str_fmt(&self, fmt: &str) -> Result<Series> {
match self.dtype() {
DataType::Date32 => self.date32().map(|ca| ca.str_fmt(fmt).into_series()),
DataType::Date64 => self.date64().map(|ca| ca.str_fmt(fmt).into_series()),
_ => Err(PolarsError::InvalidOperation(
format!("operation not supported on dtype {:?}", self.dtype()).into(),
)),
}
}
fn clone_inner(&self) -> Arc<dyn SeriesTrait> {
unimplemented!()
}
#[cfg(feature = "random")]
#[cfg_attr(docsrs, doc(cfg(feature = "random")))]
fn sample_n(&self, n: usize, with_replacement: bool) -> Result<Series>;
#[cfg(feature = "random")]
#[cfg_attr(docsrs, doc(cfg(feature = "random")))]
fn sample_frac(&self, frac: f64, with_replacement: bool) -> Result<Series>;
fn get_as_any(&self, _index: usize) -> &dyn Any {
unimplemented!()
}
fn pow(&self, _exponent: f64) -> Result<Series> {
Err(PolarsError::InvalidOperation(
format!("power operation not supported on dtype {:?}", self.dtype()).into(),
))
}
fn peak_max(&self) -> BooleanChunked {
unimplemented!()
}
fn peak_min(&self) -> BooleanChunked {
unimplemented!()
}
}
impl<'a> (dyn SeriesTrait + 'a) {
pub fn unpack<N: 'static>(&self) -> Result<&ChunkedArray<N>>
where
N: PolarsDataType,
{
if &N::get_dtype() == self.dtype() {
Ok(self.as_ref())
} else {
Err(PolarsError::DataTypeMisMatch(
"cannot unpack Series; data types don't match".into(),
))
}
}
}
#[derive(Clone)]
pub struct Series(pub Arc<dyn SeriesTrait>);
impl Series {
fn get_inner_mut(&mut self) -> &mut dyn SeriesTrait {
if Arc::weak_count(&self.0) + Arc::strong_count(&self.0) != 1 {
self.0 = self.0.clone_inner();
}
Arc::get_mut(&mut self.0).expect("implementation error")
}
pub fn rename(&mut self, name: &str) -> &mut Series {
self.get_inner_mut().rename(name);
self
}
pub fn append_array(&mut self, other: ArrayRef) -> Result<&mut Self> {
self.get_inner_mut().append_array(other)?;
Ok(self)
}
pub fn append(&mut self, other: &Series) -> Result<&mut Self> {
self.get_inner_mut().append(other)?;
Ok(self)
}
pub fn sort_in_place(&mut self, reverse: bool) -> &mut Self {
self.get_inner_mut().sort_in_place(reverse);
self
}
pub fn as_single_ptr(&mut self) -> Result<usize> {
self.get_inner_mut().as_single_ptr()
}
pub fn cast<N>(&self) -> Result<Self>
where
N: PolarsDataType,
{
self.0.cast_with_datatype(&N::get_dtype())
}
pub fn sum<T>(&self) -> Option<T>
where
T: NumCast,
{
self.sum_as_series()
.cast::<Float64Type>()
.ok()
.and_then(|s| s.f64().unwrap().get(0).and_then(T::from))
}
pub fn min<T>(&self) -> Option<T>
where
T: NumCast,
{
self.min_as_series()
.cast::<Float64Type>()
.ok()
.and_then(|s| s.f64().unwrap().get(0).and_then(T::from))
}
pub fn max<T>(&self) -> Option<T>
where
T: NumCast,
{
self.max_as_series()
.cast::<Float64Type>()
.ok()
.and_then(|s| s.f64().unwrap().get(0).and_then(T::from))
}
pub fn explode(&self) -> Result<Series> {
match self.dtype() {
DataType::List(_) => self.list().unwrap().explode(),
DataType::Utf8 => self.utf8().unwrap().explode(),
_ => Err(PolarsError::InvalidOperation(
format!(
"explode not supported for Series with dtype {:?}",
self.dtype()
)
.into(),
)),
}
}
pub fn is_nan(&self) -> Result<BooleanChunked> {
match self.dtype() {
DataType::Float32 => Ok(self.f32().unwrap().is_nan()),
DataType::Float64 => Ok(self.f64().unwrap().is_nan()),
_ => Err(PolarsError::InvalidOperation(
format!(
"is_nan not supported for series with dtype {:?}",
self.dtype()
)
.into(),
)),
}
}
pub fn is_not_nan(&self) -> Result<BooleanChunked> {
match self.dtype() {
DataType::Float32 => Ok(self.f32().unwrap().is_not_nan()),
DataType::Float64 => Ok(self.f64().unwrap().is_not_nan()),
_ => Err(PolarsError::InvalidOperation(
format!(
"is_nan not supported for series with dtype {:?}",
self.dtype()
)
.into(),
)),
}
}
pub fn is_finite(&self) -> Result<BooleanChunked> {
match self.dtype() {
DataType::Float32 => Ok(self.f32().unwrap().is_finite()),
DataType::Float64 => Ok(self.f64().unwrap().is_finite()),
_ => Err(PolarsError::InvalidOperation(
format!(
"is_nan not supported for series with dtype {:?}",
self.dtype()
)
.into(),
)),
}
}
pub fn is_infinite(&self) -> Result<BooleanChunked> {
match self.dtype() {
DataType::Float32 => Ok(self.f32().unwrap().is_infinite()),
DataType::Float64 => Ok(self.f64().unwrap().is_infinite()),
_ => Err(PolarsError::InvalidOperation(
format!(
"is_nan not supported for series with dtype {:?}",
self.dtype()
)
.into(),
)),
}
}
pub fn zip_with(&self, mask: &BooleanChunked, other: &Series) -> Result<Series> {
let (lhs, rhs) = coerce_lhs_rhs(self, other)?;
lhs.zip_with_same_type(mask, rhs.as_ref())
}
pub fn is_in(&self, list_array: &ListChunked) -> Result<BooleanChunked> {
let inner_dt = list_array.inner_dtype();
let my_dt = self.dtype();
let st = get_supertype(my_dt, &inner_dt)?;
let left = if &st != my_dt {
Cow::Owned(self.cast_with_datatype(&st)?)
} else {
Cow::Borrowed(self)
};
left.is_in_same_type(list_array)
}
pub fn to_physical_repr(&self) -> Series {
use DataType::*;
let out = match self.dtype() {
Date32 => self.cast_with_datatype(&DataType::Int32),
Date64 => self.cast_with_datatype(&DataType::Int64),
Time64(_) => self.cast_with_datatype(&DataType::Int64),
Duration(_) => self.cast_with_datatype(&DataType::Int64),
_ => return self.clone(),
};
out.unwrap()
}
}
impl Deref for Series {
type Target = dyn SeriesTrait;
fn deref(&self) -> &Self::Target {
&*self.0
}
}
impl<'a> AsRef<(dyn SeriesTrait + 'a)> for Series {
fn as_ref(&self) -> &(dyn SeriesTrait + 'a) {
&*self.0
}
}
pub trait NamedFrom<T, Phantom: ?Sized> {
fn new(name: &str, _: T) -> Self;
}
macro_rules! impl_named_from {
($type:ty, $series_var:ident, $method:ident) => {
impl<T: AsRef<$type>> NamedFrom<T, $type> for Series {
fn new(name: &str, v: T) -> Self {
ChunkedArray::<$series_var>::$method(name, v.as_ref()).into_series()
}
}
};
}
impl<'a, T: AsRef<[&'a str]>> NamedFrom<T, [&'a str]> for Series {
fn new(name: &str, v: T) -> Self {
Utf8Chunked::new_from_slice(name, v.as_ref()).into_series()
}
}
impl<'a, T: AsRef<[Option<&'a str>]>> NamedFrom<T, [Option<&'a str>]> for Series {
fn new(name: &str, v: T) -> Self {
Utf8Chunked::new_from_opt_slice(name, v.as_ref()).into_series()
}
}
impl_named_from!([String], Utf8Type, new_from_slice);
impl_named_from!([bool], BooleanType, new_from_slice);
#[cfg(feature = "dtype-u8")]
impl_named_from!([u8], UInt8Type, new_from_slice);
#[cfg(feature = "dtype-u16")]
impl_named_from!([u16], UInt16Type, new_from_slice);
impl_named_from!([u32], UInt32Type, new_from_slice);
#[cfg(feature = "dtype-u64")]
impl_named_from!([u64], UInt64Type, new_from_slice);
#[cfg(feature = "dtype-i8")]
impl_named_from!([i8], Int8Type, new_from_slice);
#[cfg(feature = "dtype-i16")]
impl_named_from!([i16], Int16Type, new_from_slice);
impl_named_from!([i32], Int32Type, new_from_slice);
impl_named_from!([i64], Int64Type, new_from_slice);
impl_named_from!([f32], Float32Type, new_from_slice);
impl_named_from!([f64], Float64Type, new_from_slice);
impl_named_from!([Option<String>], Utf8Type, new_from_opt_slice);
impl_named_from!([Option<bool>], BooleanType, new_from_opt_slice);
#[cfg(feature = "dtype-u8")]
impl_named_from!([Option<u8>], UInt8Type, new_from_opt_slice);
#[cfg(feature = "dtype-u16")]
impl_named_from!([Option<u16>], UInt16Type, new_from_opt_slice);
impl_named_from!([Option<u32>], UInt32Type, new_from_opt_slice);
#[cfg(feature = "dtype-u64")]
impl_named_from!([Option<u64>], UInt64Type, new_from_opt_slice);
#[cfg(feature = "dtype-i8")]
impl_named_from!([Option<i8>], Int8Type, new_from_opt_slice);
#[cfg(feature = "dtype-i16")]
impl_named_from!([Option<i16>], Int16Type, new_from_opt_slice);
impl_named_from!([Option<i32>], Int32Type, new_from_opt_slice);
impl_named_from!([Option<i64>], Int64Type, new_from_opt_slice);
impl_named_from!([Option<f32>], Float32Type, new_from_opt_slice);
impl_named_from!([Option<f64>], Float64Type, new_from_opt_slice);
impl<T: AsRef<[Series]>> NamedFrom<T, ListType> for Series {
fn new(name: &str, s: T) -> Self {
let series_slice = s.as_ref();
let values_cap = series_slice.iter().fold(0, |acc, s| acc + s.len());
let dt = series_slice[0].dtype();
let mut builder = get_list_builder(dt, values_cap, series_slice.len(), name);
for series in series_slice {
builder.append_series(series)
}
builder.finish().into_series()
}
}
impl std::convert::TryFrom<(&str, Vec<ArrayRef>)> for Series {
type Error = PolarsError;
fn try_from(name_arr: (&str, Vec<ArrayRef>)) -> Result<Self> {
let (name, chunks) = name_arr;
let mut chunks_iter = chunks.iter();
let data_type: &ArrowDataType = chunks_iter
.next()
.ok_or_else(|| PolarsError::NoData("Expected at least on ArrayRef".into()))?
.data_type();
for chunk in chunks_iter {
if chunk.data_type() != data_type {
return Err(PolarsError::InvalidOperation(
"Cannot create series from multiple arrays with different types".into(),
));
}
}
match data_type {
ArrowDataType::LargeUtf8 => {
Ok(Utf8Chunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::Utf8 => {
let chunks = chunks
.iter()
.map(|arr| cast(arr, &ArrowDataType::LargeUtf8).unwrap())
.collect_vec();
Ok(Utf8Chunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::List(_) => {
let chunks = chunks
.iter()
.map(|arr| {
cast(
arr,
&ArrowDataType::LargeList(Box::new(arrow::datatypes::Field::new(
"",
ArrowDataType::Null,
true,
))),
)
.unwrap()
})
.collect_vec();
Ok(ListChunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::Boolean => {
Ok(BooleanChunked::new_from_chunks(name, chunks).into_series())
}
#[cfg(feature = "dtype-u8")]
ArrowDataType::UInt8 => Ok(UInt8Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-u16")]
ArrowDataType::UInt16 => Ok(UInt16Chunked::new_from_chunks(name, chunks).into_series()),
ArrowDataType::UInt32 => Ok(UInt32Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-u64")]
ArrowDataType::UInt64 => Ok(UInt64Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-i8")]
ArrowDataType::Int8 => Ok(Int8Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-i16")]
ArrowDataType::Int16 => Ok(Int16Chunked::new_from_chunks(name, chunks).into_series()),
ArrowDataType::Int32 => Ok(Int32Chunked::new_from_chunks(name, chunks).into_series()),
ArrowDataType::Int64 => Ok(Int64Chunked::new_from_chunks(name, chunks).into_series()),
ArrowDataType::Float32 => {
Ok(Float32Chunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::Float64 => {
Ok(Float64Chunked::new_from_chunks(name, chunks).into_series())
}
#[cfg(feature = "dtype-date32")]
ArrowDataType::Date32 => Ok(Date32Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-date64")]
ArrowDataType::Date64 => Ok(Date64Chunked::new_from_chunks(name, chunks).into_series()),
#[cfg(feature = "dtype-time64-ns")]
ArrowDataType::Time64(TimeUnit::Nanosecond) => {
Ok(Time64NanosecondChunked::new_from_chunks(name, chunks).into_series())
}
#[cfg(feature = "dtype-duration-ns")]
ArrowDataType::Duration(TimeUnit::Nanosecond) => {
Ok(DurationNanosecondChunked::new_from_chunks(name, chunks).into_series())
}
#[cfg(feature = "dtype-duration-ms")]
ArrowDataType::Duration(TimeUnit::Millisecond) => {
Ok(DurationMillisecondChunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::LargeList(_) => {
Ok(ListChunked::new_from_chunks(name, chunks).into_series())
}
ArrowDataType::Null => {
let len = chunks.iter().fold(0, |acc, array| acc + array.len());
#[cfg(feature = "dtype-i8")]
return Ok(Int8Chunked::full_null(name, len).into_series());
#[cfg(not(feature = "dtype-i8"))]
Ok(UInt32Chunked::full_null(name, len).into_series())
}
dt => Err(PolarsError::InvalidOperation(
format!("Cannot create polars series from {:?} type", dt).into(),
)),
}
}
}
impl TryFrom<(&str, ArrayRef)> for Series {
type Error = PolarsError;
fn try_from(name_arr: (&str, ArrayRef)) -> Result<Self> {
let (name, arr) = name_arr;
Series::try_from((name, vec![arr]))
}
}
impl Default for Series {
fn default() -> Self {
Int64Chunked::default().into_series()
}
}
impl<T> From<ChunkedArray<T>> for Series
where
T: PolarsDataType,
ChunkedArray<T>: IntoSeries,
{
fn from(ca: ChunkedArray<T>) -> Self {
ca.into_series()
}
}
impl IntoSeries for Arc<dyn SeriesTrait> {
fn into_series(self) -> Series {
Series(self)
}
}
impl IntoSeries for Series {
fn into_series(self) -> Series {
self
}
}
impl<'a, T> AsRef<ChunkedArray<T>> for dyn SeriesTrait + 'a
where
T: 'static + PolarsDataType,
{
fn as_ref(&self) -> &ChunkedArray<T> {
if &T::get_dtype() == self.dtype() ||
(matches!(T::get_dtype(), DataType::List(_)) && matches!(self.dtype(), DataType::List(_)) )
{
unsafe { &*(self as *const dyn SeriesTrait as *const ChunkedArray<T>) }
} else {
panic!(
"implementation error, cannot get ref {:?} from {:?}",
T::get_dtype(),
self.dtype()
)
}
}
}
#[cfg(test)]
mod test {
use crate::prelude::*;
use crate::series::*;
use arrow::array::*;
#[test]
fn cast() {
let ar = UInt32Chunked::new_from_slice("a", &[1, 2]);
let s = ar.into_series();
let s2 = s.cast::<Int64Type>().unwrap();
assert!(s2.i64().is_ok());
let s2 = s.cast::<Float32Type>().unwrap();
assert!(s2.f32().is_ok());
}
#[test]
fn new_series() {
Series::new("boolean series", &vec![true, false, true]);
Series::new("int series", &[1, 2, 3]);
let ca = Int32Chunked::new_from_slice("a", &[1, 2, 3]);
ca.into_series();
}
#[test]
fn new_series_from_arrow_primitive_array() {
let array = UInt32Array::from(vec![1, 2, 3, 4, 5]);
let array_ref: ArrayRef = Arc::new(array);
Series::try_from(("foo", array_ref)).unwrap();
}
#[test]
fn series_append() {
let mut s1 = Series::new("a", &[1, 2]);
let s2 = Series::new("b", &[3]);
s1.append(&s2).unwrap();
assert_eq!(s1.len(), 3);
let s2 = Series::new("b", &[3.0]);
assert!(s1.append(&s2).is_err())
}
#[test]
fn series_slice_works() {
let series = Series::new("a", &[1i64, 2, 3, 4, 5]);
let slice_1 = series.slice(-3, 3);
let slice_2 = series.slice(-5, 5);
let slice_3 = series.slice(0, 5);
assert_eq!(slice_1.get(0), AnyValue::Int64(3));
assert_eq!(slice_2.get(0), AnyValue::Int64(1));
assert_eq!(slice_3.get(0), AnyValue::Int64(1));
}
#[test]
fn out_of_range_slice_does_not_panic() {
let series = Series::new("a", &[1i64, 2, 3, 4, 5]);
series.slice(-3, 4);
series.slice(-6, 2);
series.slice(4, 2);
}
}