pub struct ChunkedArray<T: PolarsDataType> { /* private fields */ }
Expand description

ChunkedArray

Every Series contains a ChunkedArray<T>. Unlike Series, ChunkedArray’s are typed. This allows us to apply closures to the data and collect the results to a ChunkedArray of the same type T. Below we use an apply to use the cosine function to the values of a ChunkedArray.

fn apply_cosine(ca: &Float32Chunked) -> Float32Chunked {
    ca.apply(|v| v.cos())
}

If we would like to cast the result we could use a Rust Iterator instead of an apply method. Note that Iterators are slightly slower as the null values aren’t ignored implicitly.

fn apply_cosine_and_cast(ca: &Float32Chunked) -> Float64Chunked {
    ca.into_iter()
        .map(|opt_v| {
        opt_v.map(|v| v.cos() as f64)
    }).collect()
}

Another option is to first cast and then use an apply.

fn apply_cosine_and_cast(ca: &Float32Chunked) -> Float64Chunked {
    ca.apply_cast_numeric(|v| v.cos() as f64)
}

Conversion between Series and ChunkedArray’s

Conversion from a Series to a ChunkedArray is effortless.

fn to_chunked_array(series: &Series) -> PolarsResult<&Int32Chunked>{
    series.i32()
}

fn to_series(ca: Int32Chunked) -> Series {
    ca.into_series()
}

Iterators

ChunkedArrays fully support Rust native Iterator and DoubleEndedIterator traits, thereby giving access to all the excellent methods available for Iterators.


fn iter_forward(ca: &Float32Chunked) {
    ca.into_iter()
        .for_each(|opt_v| println!("{:?}", opt_v))
}

fn iter_backward(ca: &Float32Chunked) {
    ca.into_iter()
        .rev()
        .for_each(|opt_v| println!("{:?}", opt_v))
}

Memory layout

ChunkedArray’s use Apache Arrow as backend for the memory layout. Arrows memory is immutable which makes it possible to make multiple zero copy (sub)-views from a single array.

To be able to append data, Polars uses chunks to append new memory locations, hence the ChunkedArray<T> data structure. Appends are cheap, because it will not lead to a full reallocation of the whole array (as could be the case with a Rust Vec).

However, multiple chunks in a ChunkArray will slow down many operations that need random access because we have an extra indirection and indexes need to be mapped to the proper chunk. Arithmetic may also be slowed down by this. When multiplying two ChunkArray's with different chunk sizes they cannot utilize SIMD for instance.

If you want to have predictable performance (no unexpected re-allocation of memory), it is advised to call the ChunkedArray::rechunk after multiple append operations.

See also ChunkedArray::extend for appends within a chunk.

Implementations§

source§

impl<T: PolarsNumericType> ChunkedArray<T>where T::Native: Signed,

source

pub fn abs(&self) -> Self

Available on crate feature abs only.

Convert all values to their absolute/positive value.

source§

impl ChunkedArray<BooleanType>

Booleans are casted to 1 or 0.

source

pub fn sum(&self) -> Option<IdxSize>

Returns None if the array is empty or only contains null values.

source

pub fn min(&self) -> Option<bool>

source

pub fn max(&self) -> Option<bool>

source

pub fn mean(&self) -> Option<f64>

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source

pub fn append(&mut self, other: &Self)

Append in place. This is done by adding the chunks of other to this ChunkedArray.

See also extend for appends to the underlying memory

source§

impl<T: PolarsNumericType> ChunkedArray<T>

source

pub fn cast_and_apply_in_place<F, S>(&self, f: F) -> ChunkedArray<S>where F: Fn(S::Native) -> S::Native + Copy, S: PolarsNumericType,

Cast a numeric array to another numeric data type and apply a function in place. This saves an allocation.

source

pub fn apply_in_place<F>(self, f: F) -> Selfwhere F: Fn(T::Native) -> T::Native + Copy,

Cast a numeric array to another numeric data type and apply a function in place. This saves an allocation.

source§

impl<T: PolarsNumericType> ChunkedArray<T>

source

pub fn apply_mut<F>(&mut self, f: F)where F: Fn(T::Native) -> T::Native + Copy,

source§

impl ChunkedArray<Utf8Type>

source

pub fn apply_mut<'a, F>(&'a self, f: F) -> Selfwhere F: FnMut(&'a str) -> &'a str,

source§

impl ChunkedArray<BinaryType>

source

pub fn apply_mut<'a, F>(&'a self, f: F) -> Selfwhere F: FnMut(&'a [u8]) -> &'a [u8],

source§

impl<T: PolarsDataType> ChunkedArray<T>

source

pub fn len(&self) -> usize

Get the length of the ChunkedArray

source

pub fn is_empty(&self) -> bool

Check if ChunkedArray is empty.

source

pub fn rechunk(&self) -> Self

source

pub fn slice(&self, offset: i64, length: usize) -> Self

Slice the array. The chunks are reallocated the underlying data slices are zero copy.

When offset is negative it will be counted from the end of the array. This method will never error, and will slice the best match when offset, or length is out of bounds

source

pub fn limit(&self, num_elements: usize) -> Selfwhere Self: Sized,

Take a view of top n elements

source

pub fn head(&self, length: Option<usize>) -> Selfwhere Self: Sized,

Get the head of the ChunkedArray

source

pub fn tail(&self, length: Option<usize>) -> Selfwhere Self: Sized,

Get the tail of the ChunkedArray

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source

pub fn extend(&mut self, other: &Self)

Extend the memory backed by this array with the values from other.

Different from ChunkedArray::append which adds chunks to this ChunkedArray extend appends the data from other to the underlying PrimitiveArray and thus may cause a reallocation.

However if this does not cause a reallocation, the resulting data structure will not have any extra chunks and thus will yield faster queries.

Prefer extend over append when you want to do a query after a single append. For instance during online operations where you add n rows and rerun a query.

Prefer append over extend when you want to append many times before doing a query. For instance when you read in multiple files and when to store them in a single DataFrame. In the latter case finish the sequence of append operations with a rechunk.

source§

impl ChunkedArray<ListType>

source

pub fn full_null_with_dtype( name: &str, length: usize, inner_dtype: &DataType ) -> ListChunked

source§

impl<T: PolarsDataType> ChunkedArray<T>

source

pub fn is_null(&self) -> BooleanChunked

Get a mask of the null values.

source

pub fn is_not_null(&self) -> BooleanChunked

Get a mask of the valid values.

source§

impl<T> ChunkedArray<T>where ChunkedArray<T>: IntoSeries, T: PolarsFloatType, T::Native: Float + IsFloat + SubAssign + Pow<T::Native, Output = T::Native>,

source

pub fn rolling_apply_float<F>( &self, window_size: usize, f: F ) -> PolarsResult<Self>where F: FnMut(&mut ChunkedArray<T>) -> Option<T::Native>,

Available on crate feature rolling_window only.

Apply a rolling custom function. This is pretty slow because of dynamic dispatch.

source§

impl<T: PolarsNumericType> ChunkedArray<T>

source

pub fn tile(&self, n: usize) -> Self

source§

impl ChunkedArray<BinaryType>

source

pub unsafe fn to_utf8(&self) -> Utf8Chunked

Safety

Utf8 is not validated

source§

impl ChunkedArray<Utf8Type>

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source§

impl ChunkedArray<BooleanType>

source

pub fn all(&self) -> bool

Check if all values are true

source

pub fn any(&self) -> bool

Check if any value is true

source§

impl<T> ChunkedArray<T>where T: PolarsFloatType, T::Native: Float,

source

pub fn is_nan(&self) -> BooleanChunked

source

pub fn is_not_nan(&self) -> BooleanChunked

source

pub fn is_finite(&self) -> BooleanChunked

source

pub fn is_infinite(&self) -> BooleanChunked

source

pub fn none_to_nan(&self) -> Self

Convert missing values to NaN values.

source§

impl ChunkedArray<ListType>

source

pub fn par_iter(&self) -> impl ParallelIterator<Item = Option<Series>> + '_

source

pub fn par_iter_indexed( &mut self ) -> impl IndexedParallelIterator<Item = Option<Series>> + '_

source§

impl ChunkedArray<Utf8Type>

source

pub fn par_iter_indexed( &self ) -> impl IndexedParallelIterator<Item = Option<&str>>

source

pub fn par_iter(&self) -> impl ParallelIterator<Item = Option<&str>> + '_

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source

pub fn to_ndarray(&self) -> PolarsResult<ArrayView1<'_, T::Native>>

Available on crate feature ndarray only.

If data is aligned in a single chunk and has no Null values a zero copy view is returned as an ndarray

source§

impl ChunkedArray<ListType>

source

pub fn to_ndarray<N>(&self) -> PolarsResult<Array2<N::Native>>where N: PolarsNumericType,

Available on crate feature ndarray only.

If all nested Series have the same length, a 2 dimensional ndarray::Array is returned.

source§

impl<T> ChunkedArray<T>where T: PolarsDataType,

source

pub unsafe fn from_chunks(name: &str, chunks: Vec<ArrayRef>) -> Self

Create a new ChunkedArray from existing chunks.

Safety

The Arrow datatype of all chunks must match the PolarsDataType T.

source

pub unsafe fn with_chunks(&self, chunks: Vec<ArrayRef>) -> Self

Safety

The Arrow datatype of all chunks must match the PolarsDataType T.

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source

pub fn from_vec(name: &str, v: Vec<T::Native>) -> Self

Create a new ChunkedArray by taking ownership of the Vec. This operation is zero copy.

source

pub fn new_from_owned_with_null_bitmap( name: &str, values: Vec<T::Native>, buffer: Option<Bitmap> ) -> Self

Nullify values in slice with an existing null bitmap

source

pub unsafe fn mmap_slice(name: &str, values: &[T::Native]) -> Self

Create a temporary ChunkedArray from a slice.

Safety

The lifetime will be bound to the lifetime of the slice. This will not be checked by the borrowchecker.

source§

impl ChunkedArray<BooleanType>

source

pub unsafe fn mmap_slice( name: &str, values: &[u8], offset: usize, len: usize ) -> Self

Create a temporary ChunkedArray from a slice.

Safety

The lifetime will be bound to the lifetime of the slice. This will not be checked by the borrowchecker.

source§

impl ChunkedArray<ListType>

source

pub fn amortized_iter( &self ) -> AmortizedListIter<'_, impl Iterator<Item = Option<ArrayBox>> + '_>

This is an iterator over a ListChunked that save allocations. A Series is: 1. Arc<ChunkedArray> ChunkedArray is: 2. Vec< 3. ArrayRef>

The ArrayRef we indicated with 3. will be updated during iteration. The Series will be pinned in memory, saving an allocation for

  1. Arc<..>
  2. Vec<…>
Warning

Though memory safe in the sense that it will not read unowned memory, UB, or memory leaks this function still needs precautions. The returned should never be cloned or taken longer than a single iteration, as every call on next of the iterator will change the contents of that Series.

source

pub fn amortized_iter_with_name( &self, name: &str ) -> AmortizedListIter<'_, impl Iterator<Item = Option<ArrayBox>> + '_>

source

pub fn apply_amortized<'a, F>(&'a self, f: F) -> Selfwhere F: FnMut(UnstableSeries<'a>) -> Series,

Apply a closure F elementwise.

source

pub fn try_apply_amortized<'a, F>(&'a self, f: F) -> PolarsResult<Self>where F: FnMut(UnstableSeries<'a>) -> PolarsResult<Series>,

source§

impl ChunkedArray<ListType>

source

pub fn inner_dtype(&self) -> DataType

Get the inner data type of the list.

source

pub fn set_inner_dtype(&mut self, dtype: DataType)

source

pub fn set_fast_explode(&mut self)

source

pub fn _can_fast_explode(&self) -> bool

source

pub fn to_physical(&mut self, inner_dtype: DataType)

source

pub fn get_inner(&self) -> Series

Get the inner values as Series, ignoring the list offsets.

source

pub fn apply_to_inner( &self, func: &dyn Fn(Series) -> PolarsResult<Series> ) -> PolarsResult<ListChunked>

Ignore the list indices and apply func to the inner type as Series.

source§

impl<T> ChunkedArray<ObjectType<T>>where T: PolarsObject,

source

pub fn new_from_vec(name: &str, v: Vec<T>) -> Self

Available on crate feature object only.
source

pub fn new_empty(name: &str) -> Self

Available on crate feature object only.
source§

impl<T> ChunkedArray<ObjectType<T>>where T: PolarsObject,

source

pub unsafe fn get_object_unchecked( &self, index: usize ) -> Option<&dyn PolarsObjectSafe>

Available on crate feature object only.

Get a hold to an object that can be formatted or downcasted via the Any trait.

Safety

No bounds checks

source

pub fn get_object(&self, index: usize) -> Option<&dyn PolarsObjectSafe>

Available on crate feature object only.

Get a hold to an object that can be formatted or downcasted via the Any trait.

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType, Standard: Distribution<T::Native>,

source

pub fn init_rand(size: usize, null_density: f32, seed: Option<u64>) -> Self

Available on crate feature random only.
source§

impl<T> ChunkedArray<T>where T: PolarsDataType, ChunkedArray<T>: ChunkTake,

source

pub fn sample_n( &self, n: usize, with_replacement: bool, shuffle: bool, seed: Option<u64> ) -> PolarsResult<Self>

Available on crate feature random only.

Sample n datapoints from this ChunkedArray.

source

pub fn sample_frac( &self, frac: f64, with_replacement: bool, shuffle: bool, seed: Option<u64> ) -> PolarsResult<Self>

Available on crate feature random only.

Sample a fraction between 0.0-1.0 of this ChunkedArray.

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType, T::Native: Float,

source

pub fn rand_normal( name: &str, length: usize, mean: f64, std_dev: f64 ) -> PolarsResult<Self>

Available on crate feature random only.

Create ChunkedArray with samples from a Normal distribution.

source

pub fn rand_standard_normal(name: &str, length: usize) -> Self

Available on crate feature random only.

Create ChunkedArray with samples from a Standard Normal distribution.

source

pub fn rand_uniform(name: &str, length: usize, low: f64, high: f64) -> Self

Available on crate feature random only.

Create ChunkedArray with samples from a Uniform distribution.

source§

impl ChunkedArray<BooleanType>

source

pub fn rand_bernoulli(name: &str, length: usize, p: f64) -> PolarsResult<Self>

Available on crate feature random only.

Create ChunkedArray with samples from a Bernoulli distribution.

source§

impl<T: PolarsNumericType> ChunkedArray<T>

source

pub fn to_vec(&self) -> Vec<Option<T::Native>>

Convert to a Vec of Option<T::Native>.

source

pub fn to_vec_null_aware( &self ) -> Either<Vec<T::Native>, Vec<Option<T::Native>>>

Convert to a Vec but don’t return Option<T::Native> if there are no null values

source§

impl<T: PolarsDataType> ChunkedArray<T>

source

pub fn unset_fast_explode_list(&mut self)

source

pub fn is_sorted_flag(&self) -> IsSorted

source

pub fn set_sorted_flag(&mut self, sorted: IsSorted)

Set the ‘sorted’ bit meta info.

source

pub fn first_non_null(&self) -> Option<usize>

Get the index of the first non null value in this ChunkedArray.

source

pub fn last_non_null(&self) -> Option<usize>

Get the index of the last non null value in this ChunkedArray.

source

pub fn iter_validities( &self ) -> Map<Iter<'_, ArrayRef>, fn(_: &ArrayRef) -> Option<&Bitmap>>

Get the buffer of bits representing null values

source

pub fn has_validity(&self) -> bool

Return if any the chunks in this [ChunkedArray] have a validity bitmap. no bitmap means no null values.

source

pub fn shrink_to_fit(&mut self)

Shrink the capacity of this array to fit its length.

source

pub fn unpack_series_matching_type( &self, series: &Series ) -> PolarsResult<&ChunkedArray<T>>

Series to ChunkedArray<T>

source

pub fn chunk_id(&self) -> ChunkIdIter<'_>

Unique id representing the number of chunks

source

pub fn chunks(&self) -> &Vec<ArrayRef>

A reference to the chunks

source

pub unsafe fn chunks_mut(&mut self) -> &mut Vec<ArrayRef>

A mutable reference to the chunks

Safety

The caller must ensure to not change the DataType or length of any of the chunks.

source

pub fn is_optimal_aligned(&self) -> bool

Returns true if contains a single chunk and has no null values

source

pub fn null_count(&self) -> usize

Count the null values.

source

pub fn dtype(&self) -> &DataType

Get data type of ChunkedArray.

source

pub fn name(&self) -> &str

Name of the ChunkedArray.

source

pub fn ref_field(&self) -> &Field

Get a reference to the field.

source

pub fn rename(&mut self, name: &str)

Rename this ChunkedArray.

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType,

source

pub fn cont_slice(&self) -> PolarsResult<&[T::Native]>

Contiguous slice

source

pub fn data_views( &self ) -> impl Iterator<Item = &[T::Native]> + DoubleEndedIterator

Get slices of the underlying arrow data. NOTE: null values should be taken into account by the user of these slices as they are handled separately

source

pub fn into_no_null_iter( &self ) -> impl Iterator<Item = T::Native> + '_ + Send + Sync + ExactSizeIterator + DoubleEndedIterator + TrustedLen

source§

impl<T> ChunkedArray<T>where T: PolarsIntegerType, T::Native: AsU64 + FromPrimitive + Debug,

source

pub fn group_tuples_perfect( &self, max: usize, multithreaded: bool, group_capacity: usize ) -> GroupsProxy

source§

impl<T: PolarsNumericType> ChunkedArray<T>

source

pub fn new_vec(name: &str, v: Vec<T::Native>) -> Self

Specialization that prevents an allocation prefer this over ChunkedArray::new when you have a Vec<T::Native> and no null values.

source§

impl<T> ChunkedArray<T>where T: PolarsNumericType, ChunkedArray<T>: IntoSeries,

We cannot override the left hand side behaviour. So we create a trait LhsNumOps. This allows for 1.add(&Series)

source

pub fn lhs_sub<N: Num + NumCast>(&self, lhs: N) -> Self

Apply lhs - self

source

pub fn lhs_div<N: Num + NumCast>(&self, lhs: N) -> Self

Apply lhs / self

source

pub fn lhs_rem<N: Num + NumCast>(&self, lhs: N) -> Self

Apply lhs % self

Trait Implementations§

source§

impl Add<&ChunkedArray<BinaryType>> for &BinaryChunked

§

type Output = ChunkedArray<BinaryType>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl Add<&ChunkedArray<BooleanType>> for &BooleanChunked

§

type Output = ChunkedArray<UInt32Type>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl<T> Add<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl Add<&ChunkedArray<Utf8Type>> for &Utf8Chunked

§

type Output = ChunkedArray<Utf8Type>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl Add<ChunkedArray<BinaryType>> for BinaryChunked

§

type Output = ChunkedArray<BinaryType>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl Add<ChunkedArray<BooleanType>> for BooleanChunked

§

type Output = ChunkedArray<UInt32Type>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl<T> Add<ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl Add<ChunkedArray<Utf8Type>> for Utf8Chunked

§

type Output = ChunkedArray<Utf8Type>

The resulting type after applying the + operator.
source§

fn add(self, rhs: Self) -> Self::Output

Performs the + operation. Read more
source§

impl<T, N> Add<N> for &ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the + operator.
source§

fn add(self, rhs: N) -> Self::Output

Performs the + operation. Read more
source§

impl<T, N> Add<N> for ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the + operator.
source§

fn add(self, rhs: N) -> Self::Output

Performs the + operation. Read more
source§

impl<T> AggList for ChunkedArray<T>where T: PolarsNumericType, ChunkedArray<T>: IntoSeries,

source§

unsafe fn agg_list(&self, groups: &GroupsProxy) -> Series

Safety Read more
source§

impl<'a, T> AsMut<ChunkedArray<T>> for dyn SeriesTrait + 'awhere T: 'static + PolarsDataType,

source§

fn as_mut(&mut self) -> &mut ChunkedArray<T>

Converts this type into a mutable reference of the (usually inferred) input type.
source§

impl<T: PolarsDataType> AsRef<ChunkedArray<T>> for ChunkedArray<T>

source§

fn as_ref(&self) -> &ChunkedArray<T>

Converts this type into a shared reference of the (usually inferred) input type.
source§

impl<'a, T> AsRef<ChunkedArray<T>> for dyn SeriesTrait + 'awhere T: 'static + PolarsDataType,

source§

fn as_ref(&self) -> &ChunkedArray<T>

Converts this type into a shared reference of the (usually inferred) input type.
source§

impl<T: PolarsDataType> AsRefDataType for ChunkedArray<T>

source§

impl BitAnd<&ChunkedArray<BooleanType>> for &BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the & operator.
source§

fn bitand(self, rhs: Self) -> Self::Output

Performs the & operation. Read more
source§

impl<T> BitAnd<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsIntegerType, T::Native: BitAnd<Output = T::Native>,

§

type Output = ChunkedArray<T>

The resulting type after applying the & operator.
source§

fn bitand(self, rhs: Self) -> Self::Output

Performs the & operation. Read more
source§

impl BitAnd<ChunkedArray<BooleanType>> for BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the & operator.
source§

fn bitand(self, rhs: Self) -> Self::Output

Performs the & operation. Read more
source§

impl BitOr<&ChunkedArray<BooleanType>> for &BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the | operator.
source§

fn bitor(self, rhs: Self) -> Self::Output

Performs the | operation. Read more
source§

impl<T> BitOr<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsIntegerType, T::Native: BitOr<Output = T::Native>,

§

type Output = ChunkedArray<T>

The resulting type after applying the | operator.
source§

fn bitor(self, rhs: Self) -> Self::Output

Performs the | operation. Read more
source§

impl BitOr<ChunkedArray<BooleanType>> for BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the | operator.
source§

fn bitor(self, rhs: Self) -> Self::Output

Performs the | operation. Read more
source§

impl BitXor<&ChunkedArray<BooleanType>> for &BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the ^ operator.
source§

fn bitxor(self, rhs: Self) -> Self::Output

Performs the ^ operation. Read more
source§

impl<T> BitXor<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsIntegerType, T::Native: BitXor<Output = T::Native>,

§

type Output = ChunkedArray<T>

The resulting type after applying the ^ operator.
source§

fn bitxor(self, rhs: Self) -> Self::Output

Performs the ^ operation. Read more
source§

impl BitXor<ChunkedArray<BooleanType>> for BooleanChunked

§

type Output = ChunkedArray<BooleanType>

The resulting type after applying the ^ operator.
source§

fn bitxor(self, rhs: Self) -> Self::Output

Performs the ^ operation. Read more
source§

impl<T> ChunkAgg<<T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType, <T::Native as Simd>::Simd: Add<Output = <T::Native as Simd>::Simd> + Sum<T::Native> + SimdOrd<T::Native>,

source§

fn sum(&self) -> Option<T::Native>

Aggregate the sum of the ChunkedArray. Returns None if not implemented for T. If the array is empty, 0 is returned
source§

fn min(&self) -> Option<T::Native>

source§

fn max(&self) -> Option<T::Native>

Returns the maximum value in the array, according to the natural order. Returns None if the array is empty or only contains null values.
source§

fn mean(&self) -> Option<f64>

Returns the mean value in the array. Returns None if the array is empty or only contains null values.
source§

impl<T> ChunkAggSeries for ChunkedArray<T>where T: PolarsNumericType, <T::Native as Simd>::Simd: Add<Output = <T::Native as Simd>::Simd> + Sum<T::Native> + SimdOrd<T::Native>, ChunkedArray<T>: IntoSeries,

source§

fn sum_as_series(&self) -> Series

Get the sum of the ChunkedArray as a new Series of length 1.
source§

fn max_as_series(&self) -> Series

Get the max of the ChunkedArray as a new Series of length 1.
source§

fn min_as_series(&self) -> Series

Get the min of the ChunkedArray as a new Series of length 1.
source§

fn prod_as_series(&self) -> Series

Get the product of the ChunkedArray as a new Series of length 1.
source§

impl<T> ChunkAnyValue for ChunkedArray<T>where T: PolarsNumericType,

source§

unsafe fn get_any_value_unchecked(&self, index: usize) -> AnyValue<'_>

Get a single value. Beware this is slow. If you need to use this slightly performant, cast Categorical to UInt32 Read more
source§

fn get_any_value(&self, index: usize) -> PolarsResult<AnyValue<'_>>

Get a single value. Beware this is slow.
source§

impl<'a, T> ChunkApply<'a, <T as PolarsNumericType>::Native, <T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn apply_cast_numeric<F, S>(&self, f: F) -> ChunkedArray<S>where F: Fn(T::Native) -> S::Native + Copy, S: PolarsNumericType,

Apply a closure elementwise and cast to a Numeric ChunkedArray. This is fastest when the null check branching is more expensive than the closure application. Read more
source§

fn branch_apply_cast_numeric_no_null<F, S>(&self, f: F) -> ChunkedArray<S>where F: Fn(Option<T::Native>) -> S::Native, S: PolarsNumericType,

Apply a closure on optional values and cast to Numeric ChunkedArray without null values.
source§

fn apply<F>(&'a self, f: F) -> Selfwhere F: Fn(T::Native) -> T::Native + Copy,

Apply a closure elementwise. This is fastest when the null check branching is more expensive than the closure application. Often it is. Read more
source§

fn try_apply<F>(&'a self, f: F) -> PolarsResult<Self>where F: Fn(T::Native) -> PolarsResult<T::Native> + Copy,

source§

fn apply_on_opt<F>(&'a self, f: F) -> Selfwhere F: Fn(Option<T::Native>) -> Option<T::Native> + Copy,

Apply a closure elementwise including null values.
source§

fn apply_with_idx<F>(&'a self, f: F) -> Selfwhere F: Fn((usize, T::Native)) -> T::Native + Copy,

Apply a closure elementwise. The closure gets the index of the element as first argument.
source§

fn apply_with_idx_on_opt<F>(&'a self, f: F) -> Selfwhere F: Fn((usize, Option<T::Native>)) -> Option<T::Native> + Copy,

Apply a closure elementwise. The closure gets the index of the element as first argument.
source§

fn apply_to_slice<F, V>(&'a self, f: F, slice: &mut [V])where F: Fn(Option<T::Native>, &V) -> V,

Apply a closure elementwise and write results to a mutable slice.
source§

impl<T> ChunkApplyKernel<PrimitiveArray<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn apply_kernel( &self, f: &dyn Fn(&PrimitiveArray<T::Native>) -> ArrayRef ) -> Self

Apply kernel and return result as a new ChunkedArray.
source§

fn apply_kernel_cast<S>( &self, f: &dyn Fn(&PrimitiveArray<T::Native>) -> ArrayRef ) -> ChunkedArray<S>where S: PolarsDataType,

Apply a kernel that outputs an array of different type.
source§

impl<T> ChunkCast for ChunkedArray<T>where T: PolarsNumericType,

source§

fn cast(&self, data_type: &DataType) -> PolarsResult<Series>

Cast a [ChunkedArray] to [DataType]
source§

unsafe fn cast_unchecked(&self, data_type: &DataType) -> PolarsResult<Series>

Does not check if the cast is a valid one and may over/underflow Read more
source§

impl ChunkCompare<&ChunkedArray<BinaryType>> for BinaryChunked

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: &BinaryChunked) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: &BinaryChunked) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: &BinaryChunked) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: &BinaryChunked) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, rhs: &BinaryChunked) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, rhs: &BinaryChunked) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, rhs: &BinaryChunked) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, rhs: &BinaryChunked) -> BooleanChunked

Less than or equal comparison
source§

impl ChunkCompare<&ChunkedArray<BooleanType>> for BooleanChunked

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: &BooleanChunked) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: &BooleanChunked) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: &BooleanChunked) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: &BooleanChunked) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, rhs: &BooleanChunked) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, rhs: &BooleanChunked) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, rhs: &BooleanChunked) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, rhs: &BooleanChunked) -> BooleanChunked

Less than or equal comparison
source§

impl ChunkCompare<&ChunkedArray<ListType>> for ListChunked

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: &ListChunked) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: &ListChunked) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: &ListChunked) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: &ListChunked) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, _rhs: &ListChunked) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, _rhs: &ListChunked) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, _rhs: &ListChunked) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, _rhs: &ListChunked) -> BooleanChunked

Less than or equal comparison
source§

impl<T> ChunkCompare<&ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, rhs: &ChunkedArray<T>) -> BooleanChunked

Less than or equal comparison
source§

impl ChunkCompare<&ChunkedArray<Utf8Type>> for Utf8Chunked

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: &Utf8Chunked) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: &Utf8Chunked) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: &Utf8Chunked) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: &Utf8Chunked) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, rhs: &Utf8Chunked) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, rhs: &Utf8Chunked) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, rhs: &Utf8Chunked) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, rhs: &Utf8Chunked) -> BooleanChunked

Less than or equal comparison
source§

impl<T, Rhs> ChunkCompare<Rhs> for ChunkedArray<T>where T: PolarsNumericType, Rhs: ToPrimitive,

§

type Item = ChunkedArray<BooleanType>

source§

fn equal(&self, rhs: Rhs) -> BooleanChunked

Check for equality.
source§

fn equal_missing(&self, rhs: Rhs) -> BooleanChunked

Check for equality where None == None.
source§

fn not_equal(&self, rhs: Rhs) -> BooleanChunked

Check for inequality.
source§

fn not_equal_missing(&self, rhs: Rhs) -> BooleanChunked

Check for inequality where None == None.
source§

fn gt(&self, rhs: Rhs) -> BooleanChunked

Greater than comparison.
source§

fn gt_eq(&self, rhs: Rhs) -> BooleanChunked

Greater than or equal comparison.
source§

fn lt(&self, rhs: Rhs) -> BooleanChunked

Less than comparison.
source§

fn lt_eq(&self, rhs: Rhs) -> BooleanChunked

Less than or equal comparison
source§

impl<T> ChunkCumAgg<T> for ChunkedArray<T>where T: PolarsNumericType, ChunkedArray<T>: FromIterator<Option<T::Native>>,

Available on crate feature cum_agg only.
source§

fn cummax(&self, reverse: bool) -> ChunkedArray<T>

Get an array with the cumulative max computed at every element
source§

fn cummin(&self, reverse: bool) -> ChunkedArray<T>

Get an array with the cumulative min computed at every element
source§

fn cumsum(&self, reverse: bool) -> ChunkedArray<T>

Get an array with the cumulative sum computed at every element
source§

fn cumprod(&self, reverse: bool) -> ChunkedArray<T>

Get an array with the cumulative product computed at every element
source§

impl<T> ChunkExpandAtIndex<T> for ChunkedArray<T>where ChunkedArray<T>: ChunkFull<T::Native> + TakeRandom<Item = T::Native>, T: PolarsNumericType + PolarsDataType,

source§

fn new_from_index(&self, index: usize, length: usize) -> ChunkedArray<T>

Create a new ChunkedArray filled with values at that index.
source§

impl<T> ChunkFillNullValue<<T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn fill_null_with_values(&self, value: T::Native) -> PolarsResult<Self>

Replace None values with a give value T.
source§

impl<T> ChunkFilter<T> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn filter(&self, filter: &BooleanChunked) -> PolarsResult<ChunkedArray<T>>

Filter values in the ChunkedArray with a boolean mask. Read more
source§

impl<T> ChunkFull<<T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn full(name: &str, value: T::Native, length: usize) -> Self

Create a ChunkedArray with a single value.
source§

impl<T> ChunkFullNull for ChunkedArray<T>where T: PolarsNumericType,

source§

fn full_null(name: &str, length: usize) -> Self

source§

impl<T> ChunkPeaks for ChunkedArray<T>where T: PolarsNumericType,

source§

fn peak_max(&self) -> BooleanChunked

Get a boolean mask of the local maximum peaks.

source§

fn peak_min(&self) -> BooleanChunked

Get a boolean mask of the local minimum peaks.

source§

impl<T> ChunkQuantile<f64> for ChunkedArray<T>where T: PolarsIntegerType, T::Native: Ord,

source§

fn quantile( &self, quantile: f64, interpol: QuantileInterpolOptions ) -> PolarsResult<Option<f64>>

Aggregate a given quantile of the ChunkedArray. Returns None if the array is empty or only contains null values.
source§

fn median(&self) -> Option<f64>

Returns the mean value in the array. Returns None if the array is empty or only contains null values.
source§

impl<T> ChunkReverse for ChunkedArray<T>where T: PolarsNumericType,

source§

fn reverse(&self) -> ChunkedArray<T>

Return a reversed version of this array.
source§

impl<T> ChunkRollApply for ChunkedArray<T>where T: PolarsNumericType, Self: IntoSeries,

Available on crate feature rolling_window only.
source§

fn rolling_apply( &self, f: &dyn Fn(&Series) -> Series, options: RollingOptionsFixedWindow ) -> PolarsResult<Series>

Apply a rolling custom function. This is pretty slow because of dynamic dispatch.

source§

impl<'a, T> ChunkSet<'a, <T as PolarsNumericType>::Native, <T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn set_at_idx<I: IntoIterator<Item = IdxSize>>( &'a self, idx: I, value: Option<T::Native> ) -> PolarsResult<Self>

Set the values at indexes idx to some optional value Option<T>. Read more
source§

fn set_at_idx_with<I: IntoIterator<Item = IdxSize>, F>( &'a self, idx: I, f: F ) -> PolarsResult<Self>where F: Fn(Option<T::Native>) -> Option<T::Native>,

Set the values at indexes idx by applying a closure to these values. Read more
source§

fn set( &'a self, mask: &BooleanChunked, value: Option<T::Native> ) -> PolarsResult<Self>

Set the values where the mask evaluates to true to some optional value Option<T>. Read more
source§

impl<T> ChunkShift<T> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn shift(&self, periods: i64) -> ChunkedArray<T>

source§

impl<T> ChunkShiftFill<T, Option<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn shift_and_fill( &self, periods: i64, fill_value: Option<T::Native> ) -> ChunkedArray<T>

Shift the values by a given period and fill the parts that will be empty due to this operation with fill_value.
source§

impl<T> ChunkSort<T> for ChunkedArray<T>where T: PolarsIntegerType, T::Native: Default + Ord,

source§

fn arg_sort_multiple( &self, options: &SortMultipleOptions ) -> PolarsResult<IdxCa>

Panics

This function is very opinionated. We assume that all numeric Series are of the same type, if not it will panic

source§

fn sort_with(&self, options: SortOptions) -> ChunkedArray<T>

source§

fn sort(&self, descending: bool) -> ChunkedArray<T>

Returned a sorted ChunkedArray.
source§

fn arg_sort(&self, options: SortOptions) -> IdxCa

Retrieve the indexes needed to sort this array.
source§

impl<T> ChunkTake for ChunkedArray<T>where T: PolarsNumericType,

source§

unsafe fn take_unchecked<I, INulls>( &self, indices: TakeIdx<'_, I, INulls> ) -> Selfwhere Self: Sized, I: TakeIterator, INulls: TakeIteratorNulls,

Take values from ChunkedArray by index. Read more
source§

fn take<I, INulls>(&self, indices: TakeIdx<'_, I, INulls>) -> PolarsResult<Self>where Self: Sized, I: TakeIterator, INulls: TakeIteratorNulls,

Take values from ChunkedArray by index. Note that the iterator will be cloned, so prefer an iterator that takes the owned memory by reference.
source§

impl<T> ChunkUnique<T> for ChunkedArray<T>where T: PolarsIntegerType, T::Native: Hash + Eq + Ord, ChunkedArray<T>: IntoSeries,

source§

fn unique(&self) -> PolarsResult<Self>

Get unique values of a ChunkedArray
source§

fn arg_unique(&self) -> PolarsResult<IdxCa>

Get first index of the unique values in a ChunkedArray. This Vec is sorted.
source§

fn n_unique(&self) -> PolarsResult<usize>

Number of unique values in the ChunkedArray
source§

fn mode(&self) -> PolarsResult<Self>

Available on crate feature mode only.
The most occurring value(s). Can return multiple Values
source§

impl<T> ChunkVar<f64> for ChunkedArray<T>where T: PolarsIntegerType, <T::Native as Simd>::Simd: Add<Output = <T::Native as Simd>::Simd> + Sum<T::Native> + SimdOrd<T::Native>,

source§

fn var(&self, ddof: u8) -> Option<f64>

Compute the variance of this ChunkedArray/Series.
source§

fn std(&self, ddof: u8) -> Option<f64>

Compute the standard deviation of this ChunkedArray/Series.
source§

impl<T> ChunkZip<T> for ChunkedArray<T>where T: PolarsNumericType,

Available on crate feature zip_with only.
source§

fn zip_with( &self, mask: &BooleanChunked, other: &ChunkedArray<T> ) -> PolarsResult<ChunkedArray<T>>

Create a new ChunkedArray with values from self where the mask evaluates true and values from other where the mask evaluates false
source§

impl<T: PolarsDataType> Clone for ChunkedArray<T>

source§

fn clone(&self) -> Self

Returns a copy of the value. Read more
1.0.0 · source§

fn clone_from(&mut self, source: &Self)

Performs copy-assignment from source. Read more
source§

impl Debug for ChunkedArray<BooleanType>

source§

fn fmt(&self, f: &mut Formatter<'_>) -> Result

Formats the value using the given formatter. Read more
source§

impl<T> Debug for ChunkedArray<T>where T: PolarsNumericType,

source§

fn fmt(&self, f: &mut Formatter<'_>) -> Result

Formats the value using the given formatter. Read more
source§

impl<T: PolarsDataType> Default for ChunkedArray<T>

source§

fn default() -> Self

Returns the “default value” for a type. Read more
source§

impl<T> Div<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the / operator.
source§

fn div(self, rhs: Self) -> Self::Output

Performs the / operation. Read more
source§

impl<T> Div<ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the / operator.
source§

fn div(self, rhs: Self) -> Self::Output

Performs the / operation. Read more
source§

impl<T, N> Div<N> for &ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the / operator.
source§

fn div(self, rhs: N) -> Self::Output

Performs the / operation. Read more
source§

impl<T, N> Div<N> for ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the / operator.
source§

fn div(self, rhs: N) -> Self::Output

Performs the / operation. Read more
source§

impl<T: PolarsDataType> Drop for ChunkedArray<T>

Available on crate feature object only.
source§

fn drop(&mut self)

Executes the destructor for this type. Read more
source§

impl<T: PolarsNumericType> From<&[<T as PolarsNumericType>::Native]> for ChunkedArray<T>

source§

fn from(slice: &[T::Native]) -> Self

Converts to this type from the input type.
source§

impl<'a> From<&'a ChunkedArray<BooleanType>> for Vec<Option<bool>>

source§

fn from(ca: &'a BooleanChunked) -> Self

Converts to this type from the input type.
source§

impl<'a, T> From<&'a ChunkedArray<T>> for Vec<Option<T::Native>>where T: PolarsNumericType,

source§

fn from(ca: &'a ChunkedArray<T>) -> Self

Converts to this type from the input type.
source§

impl<'a> From<&'a ChunkedArray<UInt32Type>> for TakeIdx<'a, Dummy<usize>, Dummy<Option<usize>>>

Conversion from UInt32Chunked to Unchecked TakeIdx

source§

fn from(ca: &'a IdxCa) -> Self

Converts to this type from the input type.
source§

impl<'a> From<&'a ChunkedArray<Utf8Type>> for Vec<Option<&'a str>>

From trait

source§

fn from(ca: &'a Utf8Chunked) -> Self

Converts to this type from the input type.
source§

impl<T: PolarsNumericType> From<(&str, PrimitiveArray<<T as PolarsNumericType>::Native>)> for ChunkedArray<T>

source§

fn from(tpl: (&str, PrimitiveArray<T::Native>)) -> Self

Converts to this type from the input type.
source§

impl From<ChunkedArray<BooleanType>> for Vec<Option<bool>>

source§

fn from(ca: BooleanChunked) -> Self

Converts to this type from the input type.
source§

impl<T> From<ChunkedArray<T>> for Serieswhere T: PolarsDataType, ChunkedArray<T>: IntoSeries,

source§

fn from(ca: ChunkedArray<T>) -> Self

Converts to this type from the input type.
source§

impl From<ChunkedArray<Utf8Type>> for Vec<Option<String>>

source§

fn from(ca: Utf8Chunked) -> Self

Converts to this type from the input type.
source§

impl<T: PolarsNumericType> From<PrimitiveArray<<T as PolarsNumericType>::Native>> for ChunkedArray<T>

source§

fn from(a: PrimitiveArray<T::Native>) -> Self

Converts to this type from the input type.
source§

impl<T> FromIterator<(Vec<<T as PolarsNumericType>::Native, Global>, Option<Bitmap>)> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn from_iter<I: IntoIterator<Item = (Vec<T::Native>, Option<Bitmap>)>>( iter: I ) -> Self

Creates a value from an iterator. Read more
source§

impl<T> FromIterator<Option<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

FromIterator trait

source§

fn from_iter<I: IntoIterator<Item = Option<T::Native>>>(iter: I) -> Self

Creates a value from an iterator. Read more
source§

impl FromIterator<Option<bool>> for ChunkedArray<BooleanType>

source§

fn from_iter<I: IntoIterator<Item = Option<bool>>>(iter: I) -> Self

Creates a value from an iterator. Read more
source§

impl<T> FromIteratorReversed<Option<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn from_trusted_len_iter_rev<I: TrustedLen<Item = Option<T::Native>>>( iter: I ) -> Self

source§

impl<T> FromParallelIterator<Option<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn from_par_iter<I: IntoParallelIterator<Item = Option<T::Native>>>( iter: I ) -> Self

Creates an instance of the collection from the parallel iterator par_iter. Read more
source§

impl<T> FromTrustedLenIterator<Option<<T as PolarsNumericType>::Native>> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn from_iter_trusted_length<I: IntoIterator<Item = Option<T::Native>>>( iter: I ) -> Self

source§

impl FromTrustedLenIterator<Option<bool>> for ChunkedArray<BooleanType>

source§

fn from_iter_trusted_length<I: IntoIterator<Item = Option<bool>>>( iter: I ) -> Selfwhere I::IntoIter: TrustedLen,

source§

impl<T> IntoGroupsProxy for ChunkedArray<T>where T: PolarsNumericType, T::Native: NumCast,

source§

fn group_tuples( &self, multithreaded: bool, sorted: bool ) -> PolarsResult<GroupsProxy>

Create the tuples need for a groupby operation. * The first value in the tuple is the first index of the group. * The second value in the tuple is are the indexes of the groups including the first value.
source§

impl<'a, T> IntoIterator for &'a ChunkedArray<T>where T: PolarsNumericType,

§

type Item = Option<<T as PolarsNumericType>::Native>

The type of the elements being iterated over.
§

type IntoIter = Box<dyn PolarsIterator<Item = <&'a ChunkedArray<T> as IntoIterator>::Item> + 'a, Global>

Which kind of iterator are we turning this into?
source§

fn into_iter(self) -> Self::IntoIter

Creates an iterator from a value. Read more
source§

impl<T: PolarsDataType + 'static> IntoSeries for ChunkedArray<T>where SeriesWrap<ChunkedArray<T>>: SeriesTrait,

source§

fn into_series(self) -> Serieswhere Self: Sized,

source§

fn is_series() -> bool

source§

impl<'a, T> IntoTakeRandom<'a> for &'a ChunkedArray<T>where T: PolarsNumericType,

source§

impl<T> IsIn for ChunkedArray<T>where T: PolarsNumericType,

Available on crate feature is_in only.
source§

fn is_in(&self, other: &Series) -> PolarsResult<BooleanChunked>

Check if elements of this array are in the right Series, or List values of the right Series.
source§

impl<T> Mul<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the * operator.
source§

fn mul(self, rhs: Self) -> Self::Output

Performs the * operation. Read more
source§

impl<T> Mul<ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the * operator.
source§

fn mul(self, rhs: Self) -> Self::Output

Performs the * operation. Read more
source§

impl<T, N> Mul<N> for &ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the * operator.
source§

fn mul(self, rhs: N) -> Self::Output

Performs the * operation. Read more
source§

impl<T, N> Mul<N> for ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the * operator.
source§

fn mul(self, rhs: N) -> Self::Output

Performs the * operation. Read more
source§

impl NamedFrom<Range<i32>, Int32Type> for ChunkedArray<Int32Type>

source§

fn new(name: &str, range: Range<i32>) -> Self

Initialize by name and values.
source§

impl NamedFrom<Range<i64>, Int64Type> for ChunkedArray<Int64Type>

source§

fn new(name: &str, range: Range<i64>) -> Self

Initialize by name and values.
source§

impl NamedFrom<Range<u32>, UInt32Type> for ChunkedArray<UInt32Type>

source§

fn new(name: &str, range: Range<u32>) -> Self

Initialize by name and values.
source§

impl NamedFrom<Range<u64>, UInt64Type> for ChunkedArray<UInt64Type>

source§

fn new(name: &str, range: Range<u64>) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<String>]>> NamedFrom<T, [Option<String>]> for ChunkedArray<Utf8Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<Vec<u8>>]>> NamedFrom<T, [Option<Vec<u8, Global>>]> for ChunkedArray<BinaryType>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<bool>]>> NamedFrom<T, [Option<bool>]> for ChunkedArray<BooleanType>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<f32>]>> NamedFrom<T, [Option<f32>]> for ChunkedArray<Float32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<f64>]>> NamedFrom<T, [Option<f64>]> for ChunkedArray<Float64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<i32>]>> NamedFrom<T, [Option<i32>]> for ChunkedArray<Int32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<i64>]>> NamedFrom<T, [Option<i64>]> for ChunkedArray<Int64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<u32>]>> NamedFrom<T, [Option<u32>]> for ChunkedArray<UInt32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Option<u64>]>> NamedFrom<T, [Option<u64>]> for ChunkedArray<UInt64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[String]>> NamedFrom<T, [String]> for ChunkedArray<Utf8Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[Vec<u8>]>> NamedFrom<T, [Vec<u8, Global>]> for ChunkedArray<BinaryType>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[bool]>> NamedFrom<T, [bool]> for ChunkedArray<BooleanType>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[f32]>> NamedFrom<T, [f32]> for ChunkedArray<Float32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[f64]>> NamedFrom<T, [f64]> for ChunkedArray<Float64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[i32]>> NamedFrom<T, [i32]> for ChunkedArray<Int32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[i64]>> NamedFrom<T, [i64]> for ChunkedArray<Int64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[u32]>> NamedFrom<T, [u32]> for ChunkedArray<UInt32Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T: AsRef<[u64]>> NamedFrom<T, [u64]> for ChunkedArray<UInt64Type>

source§

fn new(name: &str, v: T) -> Self

Initialize by name and values.
source§

impl<T> NewChunkedArray<T, <T as PolarsNumericType>::Native> for ChunkedArray<T>where T: PolarsNumericType,

source§

fn from_iter_values( name: &str, it: impl Iterator<Item = T::Native> ) -> ChunkedArray<T>

Create a new ChunkedArray from an iterator.

source§

fn from_slice(name: &str, v: &[T::Native]) -> Self

source§

fn from_slice_options(name: &str, opt_v: &[Option<T::Native>]) -> Self

source§

fn from_iter_options( name: &str, it: impl Iterator<Item = Option<T::Native>> ) -> ChunkedArray<T>

Create a new ChunkedArray from an iterator.
source§

impl<T: NumOpsDispatchInner> NumOpsDispatch for ChunkedArray<T>

source§

impl<S: NumOpsDispatchCheckedInner> NumOpsDispatchChecked for ChunkedArray<S>

Available on crate feature checked_arithmetic only.
source§

fn checked_div(&self, rhs: &Series) -> PolarsResult<Series>

Checked integer division. Computes self / rhs, returning None if rhs == 0 or the division results in overflow.
source§

fn checked_div_num<T: ToPrimitive>(&self, rhs: T) -> PolarsResult<Series>

source§

impl<T> QuantileAggSeries for ChunkedArray<T>where T: PolarsIntegerType, T::Native: Ord, <T::Native as Simd>::Simd: Add<Output = <T::Native as Simd>::Simd> + Sum<T::Native> + SimdOrd<T::Native>,

source§

fn quantile_as_series( &self, quantile: f64, interpol: QuantileInterpolOptions ) -> PolarsResult<Series>

Get the quantile of the ChunkedArray as a new Series of length 1.
source§

fn median_as_series(&self) -> Series

Get the median of the ChunkedArray as a new Series of length 1.
source§

impl<T> Rem<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the % operator.
source§

fn rem(self, rhs: Self) -> Self::Output

Performs the % operation. Read more
source§

impl<T> Rem<ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the % operator.
source§

fn rem(self, rhs: Self) -> Self::Output

Performs the % operation. Read more
source§

impl<T, N> Rem<N> for &ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the % operator.
source§

fn rem(self, rhs: N) -> Self::Output

Performs the % operation. Read more
source§

impl<T, N> Rem<N> for ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the % operator.
source§

fn rem(self, rhs: N) -> Self::Output

Performs the % operation. Read more
source§

impl<T> RepeatBy for ChunkedArray<T>where T: PolarsNumericType,

Available on crate feature repeat_by only.
source§

fn repeat_by(&self, by: &IdxCa) -> PolarsResult<ListChunked>

Repeat the values n times, where n is determined by the values in by.
source§

impl<T> StrConcat for ChunkedArray<T>where T: PolarsNumericType, T::Native: Display,

Available on crate feature concat_str only.
source§

fn str_concat(&self, delimiter: &str) -> Utf8Chunked

Concat the values into a string array. Read more
source§

impl<T> Sub<&ChunkedArray<T>> for &ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the - operator.
source§

fn sub(self, rhs: Self) -> Self::Output

Performs the - operation. Read more
source§

impl<T> Sub<ChunkedArray<T>> for ChunkedArray<T>where T: PolarsNumericType,

§

type Output = ChunkedArray<T>

The resulting type after applying the - operator.
source§

fn sub(self, rhs: Self) -> Self::Output

Performs the - operation. Read more
source§

impl<T, N> Sub<N> for &ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the - operator.
source§

fn sub(self, rhs: N) -> Self::Output

Performs the - operation. Read more
source§

impl<T, N> Sub<N> for ChunkedArray<T>where T: PolarsNumericType, N: Num + ToPrimitive,

§

type Output = ChunkedArray<T>

The resulting type after applying the - operator.
source§

fn sub(self, rhs: N) -> Self::Output

Performs the - operation. Read more
source§

impl<'a, T> TakeRandom for &'a ChunkedArray<T>where T: PolarsNumericType,

§

type Item = <T as PolarsNumericType>::Native

source§

fn get(&self, index: usize) -> Option<Self::Item>

Get a nullable value by index. Read more
source§

unsafe fn get_unchecked(&self, index: usize) -> Option<Self::Item>

Get a value by index and ignore the null bit. Read more
source§

impl<T> TakeRandom for ChunkedArray<T>where T: PolarsNumericType,

§

type Item = <T as PolarsNumericType>::Native

source§

fn get(&self, index: usize) -> Option<Self::Item>

Get a nullable value by index. Read more
source§

unsafe fn get_unchecked(&self, index: usize) -> Option<Self::Item>

Get a value by index and ignore the null bit. Read more
source§

impl<T> VarAggSeries for ChunkedArray<T>where T: PolarsIntegerType, <T::Native as Simd>::Simd: Add<Output = <T::Native as Simd>::Simd> + Sum<T::Native> + SimdOrd<T::Native>,

source§

fn var_as_series(&self, ddof: u8) -> Series

Get the variance of the ChunkedArray as a new Series of length 1.
source§

fn std_as_series(&self, ddof: u8) -> Series

Get the standard deviation of the ChunkedArray as a new Series of length 1.

Auto Trait Implementations§

§

impl<T> !RefUnwindSafe for ChunkedArray<T>

§

impl<T> Send for ChunkedArray<T>

§

impl<T> Sync for ChunkedArray<T>

§

impl<T> Unpin for ChunkedArray<T>where T: Unpin,

§

impl<T> !UnwindSafe for ChunkedArray<T>

Blanket Implementations§

source§

impl<T> Any for Twhere T: 'static + ?Sized,

source§

fn type_id(&self) -> TypeId

Gets the TypeId of self. Read more
source§

impl<T> Borrow<T> for Twhere T: ?Sized,

source§

fn borrow(&self) -> &T

Immutably borrows from an owned value. Read more
source§

impl<T> BorrowMut<T> for Twhere T: ?Sized,

source§

fn borrow_mut(&mut self) -> &mut T

Mutably borrows from an owned value. Read more
source§

impl<T> DynClone for Twhere T: Clone,

source§

fn __clone_box(&self, _: Private) -> *mut ()

source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

source§

impl<T, U> Into<U> for Twhere U: From<T>,

source§

fn into(self) -> U

Calls U::from(self).

That is, this conversion is whatever the implementation of From<T> for U chooses to do.

source§

impl<T> Pointable for T

source§

const ALIGN: usize = mem::align_of::<T>()

The alignment of pointer.
§

type Init = T

The type for initializers.
source§

unsafe fn init(init: <T as Pointable>::Init) -> usize

Initializes a with the given initializer. Read more
source§

unsafe fn deref<'a>(ptr: usize) -> &'a T

Dereferences the given pointer. Read more
source§

unsafe fn deref_mut<'a>(ptr: usize) -> &'a mut T

Mutably dereferences the given pointer. Read more
source§

unsafe fn drop(ptr: usize)

Drops the object pointed to by the given pointer. Read more
source§

impl<T> ToOwned for Twhere T: Clone,

§

type Owned = T

The resulting type after obtaining ownership.
source§

fn to_owned(&self) -> T

Creates owned data from borrowed data, usually by cloning. Read more
source§

fn clone_into(&self, target: &mut T)

Uses borrowed data to replace owned data, usually by cloning. Read more
source§

impl<T, U> TryFrom<U> for Twhere U: Into<T>,

§

type Error = Infallible

The type returned in the event of a conversion error.
source§

fn try_from(value: U) -> Result<T, <T as TryFrom<U>>::Error>

Performs the conversion.
source§

impl<T, U> TryInto<U> for Twhere U: TryFrom<T>,

§

type Error = <U as TryFrom<T>>::Error

The type returned in the event of a conversion error.
source§

fn try_into(self) -> Result<U, <U as TryFrom<T>>::Error>

Performs the conversion.
source§

impl<V, T> VZip<V> for Twhere V: MultiLane<T>,

source§

fn vzip(self) -> V

source§

impl<T, Rhs, Output> NumOps<Rhs, Output> for Twhere T: Sub<Rhs, Output = Output> + Mul<Rhs, Output = Output> + Div<Rhs, Output = Output> + Add<Rhs, Output = Output> + Rem<Rhs, Output = Output>,