use crate::chunked_array::builder::get_bitmap;
use crate::prelude::*;
use arrow::{
array::{
ArrayRef, BooleanArray, Date64Array, Float32Array, Float64Array, Int16Array, Int32Array,
Int64Array, Int8Array, LargeStringArray, Time64NanosecondArray, UInt16Array, UInt32Array,
UInt64Array, UInt8Array,
},
buffer::Buffer,
datatypes::TimeUnit,
};
use itertools::Itertools;
use std::convert::TryFrom;
use std::iter::{Copied, Map};
use std::marker::PhantomData;
use std::sync::Arc;
pub mod ops;
#[macro_use]
pub mod arithmetic;
pub mod boolean;
pub mod builder;
pub mod cast;
pub mod comparison;
pub mod float;
pub mod iterator;
pub mod kernels;
#[cfg(feature = "ndarray")]
mod ndarray;
#[cfg(feature = "object")]
#[cfg_attr(docsrs, doc(cfg(feature = "object")))]
pub mod object;
#[cfg(feature = "random")]
#[cfg_attr(docsrs, doc(cfg(feature = "random")))]
mod random;
#[cfg(feature = "strings")]
#[cfg_attr(docsrs, doc(cfg(feature = "strings")))]
pub mod strings;
#[cfg(feature = "temporal")]
#[cfg_attr(docsrs, doc(cfg(feature = "temporal")))]
pub mod temporal;
pub mod upstream_traits;
use arrow::array::{
Array, ArrayData, Date32Array, DurationMillisecondArray, DurationNanosecondArray,
LargeListArray,
};
use crate::chunked_array::builder::categorical::RevMapping;
use crate::utils::{slice_offsets, CustomIterTools};
use arrow::util::bit_util::{get_bit, round_upto_power_of_2};
use polars_arrow::array::ValueSize;
use std::mem;
use std::ops::{Deref, DerefMut};
pub type ChunkIdIter<'a> = std::iter::Map<std::slice::Iter<'a, ArrayRef>, fn(&ArrayRef) -> usize>;
pub struct ChunkedArray<T> {
pub(crate) field: Arc<Field>,
pub(crate) chunks: Vec<ArrayRef>,
phantom: PhantomData<T>,
pub(crate) categorical_map: Option<Arc<RevMapping>>,
}
impl<T> ChunkedArray<T> {
pub fn array_data(&self) -> Vec<&ArrayData> {
self.chunks.iter().map(|arr| arr.data()).collect()
}
pub fn get_categorical_map(&self) -> Option<&Arc<RevMapping>> {
self.categorical_map.as_ref()
}
pub fn first_non_null(&self) -> Option<usize> {
if self.null_count() == self.len() {
None
} else if self.null_count() == 0 {
Some(0)
} else {
let mut offset = 0;
for (idx, (null_count, null_bit_buffer)) in self.null_bits().enumerate() {
if null_count == 0 {
return Some(offset);
} else {
let arr = &self.chunks[idx];
let null_bit_buffer = null_bit_buffer.as_ref().unwrap();
let bit_end = arr.offset() + arr.len();
let byte_start = std::cmp::min(round_upto_power_of_2(arr.offset(), 8), bit_end);
let data = null_bit_buffer.as_slice();
for i in arr.offset()..byte_start {
if get_bit(data, i) {
return Some(offset + i);
}
}
offset += arr.len()
}
}
None
}
}
pub fn null_bits(&self) -> impl Iterator<Item = (usize, Option<Buffer>)> + '_ {
self.chunks.iter().map(|arr| get_bitmap(arr.as_ref()))
}
pub(crate) unsafe fn unpack_series_matching_physical_type(
&self,
series: &Series,
) -> Result<&ChunkedArray<T>> {
let series_trait = &**series;
if self.dtype() == series.dtype() {
let ca = &*(series_trait as *const dyn SeriesTrait as *const ChunkedArray<T>);
Ok(ca)
} else {
use DataType::*;
match (self.dtype(), series.dtype()) {
(Int64, Date64) | (Int32, Date32) | (Int64, Duration(_)) | (Int64, Time64(_)) => {
let ca = &*(series_trait as *const dyn SeriesTrait as *const ChunkedArray<T>);
Ok(ca)
}
_ => Err(PolarsError::DataTypeMisMatch(
format!(
"cannot unpack series {:?} into matching type {:?}",
series,
self.dtype()
)
.into(),
)),
}
}
}
pub fn unpack_series_matching_type(&self, series: &Series) -> Result<&ChunkedArray<T>> {
if self.dtype() == series.dtype() {
unsafe { self.unpack_series_matching_physical_type(series) }
} else {
Err(PolarsError::DataTypeMisMatch(
format!(
"cannot unpack series {:?} into matching type {:?}",
series,
self.dtype()
)
.into(),
))
}
}
pub fn len(&self) -> usize {
self.chunks.iter().fold(0, |acc, arr| acc + arr.len())
}
pub fn is_empty(&self) -> bool {
self.len() == 0
}
pub fn chunk_id(&self) -> ChunkIdIter {
self.chunks.iter().map(|chunk| chunk.len())
}
pub fn chunks(&self) -> &Vec<ArrayRef> {
&self.chunks
}
pub fn is_optimal_aligned(&self) -> bool {
self.chunks.len() == 1 && self.null_count() == 0
}
pub fn null_count(&self) -> usize {
self.chunks.iter().map(|arr| arr.null_count()).sum()
}
pub fn limit(&self, num_elements: usize) -> Self {
self.slice(0, num_elements)
}
pub fn append_array(&mut self, other: ArrayRef) -> Result<()> {
if matches!(self.dtype(), DataType::Categorical) {
return Err(PolarsError::InvalidOperation(
"append_array not supported for categorical type".into(),
));
}
if self.field.data_type() == other.data_type() {
self.chunks.push(other);
Ok(())
} else {
Err(PolarsError::DataTypeMisMatch(
format!(
"cannot append array of type {:?} in array of type {:?}",
other.data_type(),
self.dtype()
)
.into(),
))
}
}
fn copy_with_chunks(&self, chunks: Vec<ArrayRef>) -> Self {
ChunkedArray {
field: self.field.clone(),
chunks,
phantom: PhantomData,
categorical_map: self.categorical_map.clone(),
}
}
pub fn slice(&self, offset: i64, length: usize) -> Self {
let (raw_offset, slice_len) = slice_offsets(offset, length, self.len());
let mut remaining_length = slice_len;
let mut remaining_offset = raw_offset;
let mut new_chunks = vec![];
for chunk in &self.chunks {
let chunk_len = chunk.len();
if remaining_offset > 0 && remaining_offset >= chunk_len {
remaining_offset -= chunk_len;
continue;
}
let take_len;
if remaining_length + remaining_offset > chunk_len {
take_len = chunk_len - remaining_offset;
} else {
take_len = remaining_length;
}
new_chunks.push(chunk.slice(remaining_offset, take_len));
remaining_length -= take_len;
remaining_offset = 0;
if remaining_length == 0 {
break;
}
}
self.copy_with_chunks(new_chunks)
}
pub fn is_null(&self) -> BooleanChunked {
if self.null_count() == 0 {
return BooleanChunked::full("is_null", false, self.len());
}
let chunks = self
.chunks
.iter()
.map(|arr| Arc::new((&**arr).is_null_mask()) as ArrayRef)
.collect_vec();
BooleanChunked::new_from_chunks("is_null", chunks)
}
pub fn is_not_null(&self) -> BooleanChunked {
if self.null_count() == 0 {
return BooleanChunked::full("is_not_null", true, self.len());
}
let chunks = self
.chunks
.iter()
.map(|arr| Arc::new((&**arr).is_not_null_mask()) as ArrayRef)
.collect_vec();
BooleanChunked::new_from_chunks("is_not_null", chunks)
}
pub fn dtype(&self) -> &DataType {
self.field.data_type()
}
#[inline]
pub(crate) fn index_to_chunked_index(&self, index: usize) -> (usize, usize) {
if self.chunks.len() == 1 {
return (0, index);
}
let mut index_remainder = index;
let mut current_chunk_idx = 0;
for chunk in &self.chunks {
let chunk_len = chunk.len();
if chunk_len > index_remainder {
break;
} else {
index_remainder -= chunk_len;
current_chunk_idx += 1;
}
}
(current_chunk_idx, index_remainder)
}
pub fn head(&self, length: Option<usize>) -> Self {
match length {
Some(len) => self.slice(0, std::cmp::min(len, self.len())),
None => self.slice(0, std::cmp::min(10, self.len())),
}
}
pub fn tail(&self, length: Option<usize>) -> Self {
let len = match length {
Some(len) => std::cmp::min(len, self.len()),
None => std::cmp::min(10, self.len()),
};
self.slice(-(len as i64), len)
}
pub fn append(&mut self, other: &Self)
where
Self: std::marker::Sized,
{
if matches!(self.dtype(), DataType::Categorical) && !self.is_empty() {
assert!(Arc::ptr_eq(
self.categorical_map.as_ref().unwrap(),
other.categorical_map.as_ref().unwrap()
));
}
if self.chunks.len() == 1 && self.is_empty() {
self.chunks = other.chunks.clone();
} else {
self.chunks.extend_from_slice(&other.chunks);
}
}
pub fn name(&self) -> &str {
self.field.name()
}
pub fn ref_field(&self) -> &Field {
&self.field
}
pub fn rename(&mut self, name: &str) {
self.field = Arc::new(Field::new(name, self.field.data_type().clone()))
}
}
impl<T> ChunkedArray<T>
where
T: PolarsDataType,
ChunkedArray<T>: ChunkOps,
{
pub(crate) fn match_chunks<I>(&self, chunk_id: I) -> Self
where
I: Iterator<Item = usize>,
{
debug_assert!(self.chunks.len() == 1);
let slice = |ca: &Self| {
let array = &ca.chunks[0];
let mut offset = 0;
let chunks = chunk_id
.map(|len| {
let out = array.slice(offset, len);
offset += len;
out
})
.collect();
Self::new_from_chunks(self.name(), chunks)
};
if self.chunks.len() != 1 {
let out = self.rechunk();
slice(&out)
} else {
slice(self)
}
}
}
impl<T> ChunkedArray<T>
where
T: PolarsDataType,
{
pub fn new_from_chunks(name: &str, chunks: Vec<ArrayRef>) -> Self {
let datatype = if matches!(T::get_dtype(), DataType::List(_)) {
if let Some(arr) = chunks.get(0) {
arr.data_type().into()
} else {
T::get_dtype()
}
} else {
T::get_dtype()
};
let field = Arc::new(Field::new(name, datatype));
ChunkedArray {
field,
chunks,
phantom: PhantomData,
categorical_map: None,
}
}
#[inline]
unsafe fn arr_to_any_value(&self, arr: &dyn Array, idx: usize) -> AnyValue {
if arr.is_null(idx) {
return AnyValue::Null;
}
macro_rules! downcast_and_pack {
($casttype:ident, $variant:ident) => {{
let arr = &*(arr as *const dyn Array as *const $casttype);
let v = arr.value_unchecked(idx);
AnyValue::$variant(v)
}};
}
macro_rules! downcast {
($casttype:ident) => {{
let arr = &*(arr as *const dyn Array as *const $casttype);
arr.value_unchecked(idx)
}};
}
match T::get_dtype() {
DataType::Utf8 => downcast_and_pack!(LargeStringArray, Utf8),
DataType::Boolean => downcast_and_pack!(BooleanArray, Boolean),
DataType::UInt8 => downcast_and_pack!(UInt8Array, UInt8),
DataType::UInt16 => downcast_and_pack!(UInt16Array, UInt16),
DataType::UInt32 => downcast_and_pack!(UInt32Array, UInt32),
DataType::UInt64 => downcast_and_pack!(UInt64Array, UInt64),
DataType::Int8 => downcast_and_pack!(Int8Array, Int8),
DataType::Int16 => downcast_and_pack!(Int16Array, Int16),
DataType::Int32 => downcast_and_pack!(Int32Array, Int32),
DataType::Int64 => downcast_and_pack!(Int64Array, Int64),
DataType::Float32 => downcast_and_pack!(Float32Array, Float32),
DataType::Float64 => downcast_and_pack!(Float64Array, Float64),
DataType::Date32 => downcast_and_pack!(Date32Array, Date32),
DataType::Date64 => downcast_and_pack!(Date64Array, Date64),
DataType::Time64(TimeUnit::Nanosecond) => {
let v = downcast!(Time64NanosecondArray);
AnyValue::Time64(v, TimeUnit::Nanosecond)
}
DataType::Duration(TimeUnit::Nanosecond) => {
let v = downcast!(DurationNanosecondArray);
AnyValue::Duration(v, TimeUnit::Nanosecond)
}
DataType::Duration(TimeUnit::Millisecond) => {
let v = downcast!(DurationMillisecondArray);
AnyValue::Duration(v, TimeUnit::Millisecond)
}
DataType::List(_) => {
let v = downcast!(LargeListArray);
let s = Series::try_from(("", v));
AnyValue::List(s.unwrap())
}
#[cfg(feature = "object")]
DataType::Object => AnyValue::Object(&"object"),
DataType::Categorical => {
let v = downcast!(UInt32Array);
AnyValue::Utf8(&self.categorical_map.as_ref().expect("should be set").get(v))
}
_ => unimplemented!(),
}
}
#[inline]
pub(crate) unsafe fn get_any_value_unchecked(&self, index: usize) -> AnyValue {
let (chunk_idx, idx) = self.index_to_chunked_index(index);
debug_assert!(chunk_idx < self.chunks.len());
let arr = &**self.chunks.get_unchecked(chunk_idx);
debug_assert!(idx < arr.len());
self.arr_to_any_value(arr, idx)
}
pub(crate) fn get_any_value(&self, index: usize) -> AnyValue {
let (chunk_idx, idx) = self.index_to_chunked_index(index);
let arr = &*self.chunks[chunk_idx];
assert!(idx < arr.len());
unsafe { self.arr_to_any_value(arr, idx) }
}
}
impl<T> ChunkedArray<T>
where
T: PolarsPrimitiveType,
{
pub fn new_from_aligned_vec(name: &str, v: AlignedVec<T::Native>) -> Self {
let arr = v.into_primitive_array::<T>(None);
Self::new_from_chunks(name, vec![Arc::new(arr)])
}
pub fn new_from_owned_with_null_bitmap(
name: &str,
values: AlignedVec<T::Native>,
buffer: Option<Buffer>,
) -> Self {
let arr = Arc::new(values.into_primitive_array::<T>(buffer));
ChunkedArray {
field: Arc::new(Field::new(name, T::get_dtype())),
chunks: vec![arr],
phantom: PhantomData,
categorical_map: None,
}
}
}
pub(crate) trait AsSinglePtr {
fn as_single_ptr(&mut self) -> Result<usize> {
Err(PolarsError::InvalidOperation(
"operation as_single_ptr not supported for this dtype".into(),
))
}
}
impl<T> AsSinglePtr for ChunkedArray<T>
where
T: PolarsNumericType,
{
fn as_single_ptr(&mut self) -> Result<usize> {
let mut ca = self.rechunk();
mem::swap(&mut ca, self);
let a = self.data_views().next().unwrap();
let ptr = a.as_ptr();
Ok(ptr as usize)
}
}
impl AsSinglePtr for BooleanChunked {}
impl AsSinglePtr for ListChunked {}
impl AsSinglePtr for Utf8Chunked {}
impl AsSinglePtr for CategoricalChunked {}
#[cfg(feature = "object")]
impl<T> AsSinglePtr for ObjectChunked<T> {}
impl<T> ChunkedArray<T>
where
T: PolarsNumericType,
{
pub fn cont_slice(&self) -> Result<&[T::Native]> {
if self.chunks.len() == 1 && self.chunks[0].null_count() == 0 {
Ok(self.downcast_iter().next().map(|arr| arr.values()).unwrap())
} else {
Err(PolarsError::NoSlice)
}
}
pub fn data_views(&self) -> impl Iterator<Item = &[T::Native]> + DoubleEndedIterator {
self.downcast_iter().map(|arr| arr.values())
}
#[allow(clippy::wrong_self_convention)]
pub fn into_no_null_iter(
&self,
) -> impl Iterator<Item = T::Native> + '_ + Send + Sync + ExactSizeIterator + DoubleEndedIterator
{
#[allow(clippy::map_clone)]
self.data_views()
.flatten()
.map(|v| *v)
.trust_my_length(self.len())
}
pub fn map<B, F>(&self, f: F) -> Result<Map<Copied<std::slice::Iter<T::Native>>, F>>
where
F: Fn(T::Native) -> B,
{
let slice = self.cont_slice()?;
Ok(slice.iter().copied().map(f))
}
pub fn map_null_checks<'a, B, F>(
&'a self,
f: F,
) -> Map<Box<dyn PolarsIterator<Item = Option<T::Native>> + 'a>, F>
where
F: Fn(Option<T::Native>) -> B,
{
self.into_iter().map(f)
}
pub fn fold<F, B>(&self, init: B, f: F) -> Result<B>
where
F: Fn(B, T::Native) -> B,
{
let slice = self.cont_slice()?;
Ok(slice.iter().copied().fold(init, f))
}
pub fn fold_null_checks<F, B>(&self, init: B, f: F) -> B
where
F: Fn(B, Option<T::Native>) -> B,
{
self.into_iter().fold(init, f)
}
}
impl<T> Clone for ChunkedArray<T> {
fn clone(&self) -> Self {
ChunkedArray {
field: self.field.clone(),
chunks: self.chunks.clone(),
phantom: PhantomData,
categorical_map: self.categorical_map.clone(),
}
}
}
impl<T> AsRef<ChunkedArray<T>> for ChunkedArray<T> {
fn as_ref(&self) -> &ChunkedArray<T> {
self
}
}
pub struct NoNull<T>(pub T);
impl Deref for CategoricalChunked {
type Target = UInt32Chunked;
fn deref(&self) -> &Self::Target {
let ptr = self as *const CategoricalChunked;
let ptr = ptr as *const UInt32Chunked;
unsafe { &*ptr }
}
}
impl DerefMut for CategoricalChunked {
fn deref_mut(&mut self) -> &mut Self::Target {
let ptr = self as *mut CategoricalChunked;
let ptr = ptr as *mut UInt32Chunked;
unsafe { &mut *ptr }
}
}
impl From<UInt32Chunked> for CategoricalChunked {
fn from(ca: UInt32Chunked) -> Self {
ca.cast().unwrap()
}
}
impl CategoricalChunked {
fn set_state<T>(mut self, other: &ChunkedArray<T>) -> Self {
self.categorical_map = other.categorical_map.clone();
self
}
}
impl ValueSize for ListChunked {
fn get_values_size(&self) -> usize {
self.chunks
.iter()
.fold(0usize, |acc, arr| acc + arr.get_values_size())
}
}
impl ValueSize for Utf8Chunked {
fn get_values_size(&self) -> usize {
self.chunks
.iter()
.fold(0usize, |acc, arr| acc + arr.get_values_size())
}
}
impl ListChunked {
pub fn inner_dtype(&self) -> DataType {
match self.dtype() {
DataType::List(dt) => dt.into(),
_ => unreachable!(),
}
}
}
#[cfg(test)]
pub(crate) mod test {
use crate::prelude::*;
pub(crate) fn get_chunked_array() -> Int32Chunked {
ChunkedArray::new_from_slice("a", &[1, 2, 3])
}
#[test]
fn test_sort() {
let a = Int32Chunked::new_from_slice("a", &[1, 9, 3, 2]);
let b = a
.sort(false)
.into_iter()
.map(|opt| opt.unwrap())
.collect::<Vec<_>>();
assert_eq!(b, [1, 2, 3, 9]);
let a = Utf8Chunked::new_from_slice("a", &["b", "a", "c"]);
let a = a.sort(false);
let b = a.into_iter().collect::<Vec<_>>();
assert_eq!(b, [Some("a"), Some("b"), Some("c")]);
}
#[test]
fn arithmetic() {
let s1 = get_chunked_array();
println!("{:?}", s1.chunks);
let s2 = &s1;
let s1 = &s1;
println!("{:?}", s1 + s2);
println!("{:?}", s1 - s2);
println!("{:?}", s1 * s2);
}
#[test]
fn iter() {
let s1 = get_chunked_array();
assert_eq!(s1.into_iter().fold(0, |acc, val| { acc + val.unwrap() }), 6)
}
#[test]
fn limit() {
let a = get_chunked_array();
let b = a.limit(2);
println!("{:?}", b);
assert_eq!(b.len(), 2)
}
#[test]
fn filter() {
let a = get_chunked_array();
let b = a
.filter(&BooleanChunked::new_from_slice(
"filter",
&[true, false, false],
))
.unwrap();
assert_eq!(b.len(), 1);
assert_eq!(b.into_iter().next(), Some(Some(1)));
}
#[test]
fn aggregates_numeric() {
let a = get_chunked_array();
assert_eq!(a.max(), Some(3));
assert_eq!(a.min(), Some(1));
assert_eq!(a.sum(), Some(6))
}
#[test]
fn take() {
let a = get_chunked_array();
let new = a.take([0usize, 1].iter().copied().into());
assert_eq!(new.len(), 2)
}
#[test]
fn get() {
let mut a = get_chunked_array();
assert_eq!(AnyValue::Int32(2), a.get_any_value(1));
a.append_array(a.chunks[0].clone()).unwrap();
assert_eq!(AnyValue::Int32(1), a.get_any_value(3));
}
#[test]
fn cast() {
let a = get_chunked_array();
let b = a.cast::<Int64Type>().unwrap();
assert_eq!(b.field.data_type(), &ArrowDataType::Int64)
}
fn assert_slice_equal<T>(ca: &ChunkedArray<T>, eq: &[T::Native])
where
ChunkedArray<T>: ChunkOps,
T: PolarsNumericType,
{
assert_eq!(
ca.into_iter().map(|opt| opt.unwrap()).collect::<Vec<_>>(),
eq
)
}
#[test]
fn slice() {
let mut first = UInt32Chunked::new_from_slice("first", &[0, 1, 2]);
let second = UInt32Chunked::new_from_slice("second", &[3, 4, 5]);
first.append(&second);
assert_slice_equal(&first.slice(0, 3), &[0, 1, 2]);
assert_slice_equal(&first.slice(0, 4), &[0, 1, 2, 3]);
assert_slice_equal(&first.slice(1, 4), &[1, 2, 3, 4]);
assert_slice_equal(&first.slice(3, 2), &[3, 4]);
assert_slice_equal(&first.slice(3, 3), &[3, 4, 5]);
assert_slice_equal(&first.slice(-3, 3), &[3, 4, 5]);
assert_slice_equal(&first.slice(-6, 6), &[0, 1, 2, 3, 4, 5]);
assert_eq!(first.slice(-7, 2).len(), 2);
assert_eq!(first.slice(-3, 4).len(), 3);
assert_eq!(first.slice(3, 4).len(), 3);
assert_eq!(first.slice(10, 4).len(), 0);
}
#[test]
fn sorting() {
let s = UInt32Chunked::new_from_slice("", &[9, 2, 4]);
let sorted = s.sort(false);
assert_slice_equal(&sorted, &[2, 4, 9]);
let sorted = s.sort(true);
assert_slice_equal(&sorted, &[9, 4, 2]);
let s: Utf8Chunked = ["b", "a", "z"].iter().collect();
let sorted = s.sort(false);
assert_eq!(
sorted.into_iter().collect::<Vec<_>>(),
&[Some("a"), Some("b"), Some("z")]
);
let sorted = s.sort(true);
assert_eq!(
sorted.into_iter().collect::<Vec<_>>(),
&[Some("z"), Some("b"), Some("a")]
);
let s: Utf8Chunked = [Some("b"), None, Some("z")].iter().copied().collect();
let sorted = s.sort(false);
assert_eq!(
sorted.into_iter().collect::<Vec<_>>(),
&[None, Some("b"), Some("z")]
);
}
#[test]
fn reverse() {
let s = UInt32Chunked::new_from_slice("", &[1, 2, 3]);
assert_slice_equal(&s.reverse(), &[3, 2, 1]);
let s = UInt32Chunked::new_from_opt_slice("", &[Some(1), None, Some(3)]);
assert_eq!(Vec::from(&s.reverse()), &[Some(3), None, Some(1)]);
let s = BooleanChunked::new_from_slice("", &[true, false]);
assert_eq!(Vec::from(&s.reverse()), &[Some(false), Some(true)]);
let s = Utf8Chunked::new_from_slice("", &["a", "b", "c"]);
assert_eq!(Vec::from(&s.reverse()), &[Some("c"), Some("b"), Some("a")]);
let s = Utf8Chunked::new_from_opt_slice("", &[Some("a"), None, Some("c")]);
assert_eq!(Vec::from(&s.reverse()), &[Some("c"), None, Some("a")]);
}
#[test]
fn test_null_sized_chunks() {
let mut s = Float64Chunked::new_from_slice("s", &Vec::<f64>::new());
s.append(&Float64Chunked::new_from_slice("s2", &[1., 2., 3.]));
dbg!(&s);
let s = Float64Chunked::new_from_slice("s", &Vec::<f64>::new());
dbg!(&s.into_iter().next());
}
#[test]
fn test_iter_categorical() {
let ca =
Utf8Chunked::new_from_opt_slice("", &[Some("foo"), None, Some("bar"), Some("ham")]);
let ca = ca.cast::<CategoricalType>().unwrap();
let v: Vec<_> = ca.into_iter().collect();
assert_eq!(v, &[Some(0), None, Some(1), Some(2)]);
}
}