diff --git a/bench-vortex/benches/compress.rs b/bench-vortex/benches/compress.rs index 91bd8c0f2d..bb764e133b 100644 --- a/bench-vortex/benches/compress.rs +++ b/bench-vortex/benches/compress.rs @@ -34,7 +34,7 @@ use vortex::error::VortexResult; use vortex::file::{LayoutContext, LayoutDeserializer, VortexFileWriter, VortexReadBuilder}; use vortex::sampling_compressor::compressors::fsst::FSSTCompressor; use vortex::sampling_compressor::{SamplingCompressor, ALL_ENCODINGS_CONTEXT}; -use vortex::{ArrayDType, ArrayData, IntoArrayData, IntoCanonical}; +use vortex::{ArrayDType, ArrayData, IntoArrayData, IntoArrayVariant}; use crate::tokio_runtime::TOKIO_RUNTIME; @@ -146,7 +146,7 @@ fn vortex_decompress_read(runtime: &Runtime, buf: Buffer) -> VortexResult = chunked_array .chunks() - .map(|chunk| -> PyResult { Ok(chunk.into_arrow()?) }) + .map(|chunk| -> PyResult { Ok(chunk.into_inferred_arrow()?) }) .collect::>>()?; if chunks.is_empty() { return Err(PyValueError::new_err("No chunks in array")); @@ -140,7 +140,7 @@ impl PyArray { } else { Ok(vortex .clone() - .into_arrow()? + .into_inferred_arrow()? .into_data() .to_pyarrow(py)? .into_bound(py)) diff --git a/vortex-array/src/array/constant/compute/boolean.rs b/vortex-array/src/array/constant/compute/boolean.rs index c93b37305b..656465dfd2 100644 --- a/vortex-array/src/array/constant/compute/boolean.rs +++ b/vortex-array/src/array/constant/compute/boolean.rs @@ -107,6 +107,8 @@ mod test { #[case(BoolArray::from_iter([Some(true), Some(false), Some(true), Some(false)].into_iter()).into_array(), ConstantArray::new(true, 4).into_array())] fn test_and(#[case] lhs: ArrayData, #[case] rhs: ArrayData) { + let rr = and(&lhs, &rhs); + println!("{:?}", rr); let r = and(&lhs, &rhs).unwrap().into_bool().unwrap().into_array(); let v0 = scalar_at(&r, 0).unwrap().as_bool().value(); diff --git a/vortex-array/src/array/varbin/canonical.rs b/vortex-array/src/array/varbin/canonical.rs index b8245fb11f..a9fb9f1e81 100644 --- a/vortex-array/src/array/varbin/canonical.rs +++ b/vortex-array/src/array/varbin/canonical.rs @@ -1,7 +1,7 @@ use arrow_array::ArrayRef; use arrow_schema::DataType; use vortex_dtype::DType; -use vortex_error::VortexResult; +use vortex_error::{vortex_bail, VortexResult}; use crate::array::varbin::arrow::varbin_to_arrow; use crate::array::varbin::VarBinArray; @@ -23,10 +23,21 @@ impl IntoCanonical for VarBinArray { VarBinViewArray::try_from(ArrayData::from_arrow(array, nullable)).map(Canonical::VarBinView) } - fn into_arrow(self) -> VortexResult { + fn into_arrow(self, data_type: &DataType) -> VortexResult { // Specialized implementation of `into_arrow` for VarBin since it has a direct // Arrow representation. - varbin_to_arrow(&self) + let array_ref = varbin_to_arrow(&self)?; + + // Note, arrow::cast clones the array + Ok(match data_type { + DataType::Binary | DataType::LargeBinary | DataType::Utf8 | DataType::LargeUtf8 => { + array_ref + } + DataType::Utf8View | DataType::BinaryView => { + arrow_cast::cast(array_ref.as_ref(), data_type)? + } + _ => vortex_bail!("Unsupported data type: {:?}", data_type), + }) } } diff --git a/vortex-array/src/arrow/datum.rs b/vortex-array/src/arrow/datum.rs index b0d0ba08a6..ab7582bb33 100644 --- a/vortex-array/src/arrow/datum.rs +++ b/vortex-array/src/arrow/datum.rs @@ -1,9 +1,10 @@ use arrow_array::{Array, ArrayRef, Datum as ArrowDatum}; use vortex_error::VortexError; +use crate::arrow::infer_data_type; use crate::compute::slice; use crate::stats::{ArrayStatistics, Stat}; -use crate::{ArrayData, IntoCanonical}; +use crate::{ArrayDType, ArrayData, IntoCanonical}; /// A wrapper around a generic Arrow array that can be used as a Datum in Arrow compute. pub struct Datum { @@ -15,18 +16,19 @@ impl TryFrom for Datum { type Error = VortexError; fn try_from(array: ArrayData) -> Result { + let data_type = infer_data_type(array.dtype())?; if array .statistics() .get_as::(Stat::IsConstant) .unwrap_or_default() { Ok(Self { - array: slice(array, 0, 1)?.into_arrow()?, + array: slice(array, 0, 1)?.into_arrow(&data_type)?, is_scalar: true, }) } else { Ok(Self { - array: array.into_arrow()?, + array: array.into_arrow(&data_type)?, is_scalar: false, }) } diff --git a/vortex-array/src/arrow/dtype.rs b/vortex-array/src/arrow/dtype.rs index d5582c4890..e3a5bce1be 100644 --- a/vortex-array/src/arrow/dtype.rs +++ b/vortex-array/src/arrow/dtype.rs @@ -12,6 +12,7 @@ use std::sync::Arc; +use arrow_array::ArrayRef; use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaBuilder, SchemaRef}; use itertools::Itertools; use vortex_datetime_dtype::arrow::{make_arrow_temporal_dtype, make_temporal_ext_dtype}; @@ -20,6 +21,7 @@ use vortex_dtype::{DType, Nullability, PType, StructDType}; use vortex_error::{vortex_bail, vortex_err, VortexResult}; use crate::arrow::{FromArrowType, TryFromArrowType}; +use crate::{ArrayDType, ArrayData, IntoCanonical}; impl TryFromArrowType<&DataType> for PType { fn try_from_arrow(value: &DataType) -> VortexResult { @@ -184,6 +186,13 @@ pub fn infer_data_type(dtype: &DType) -> VortexResult { }) } +impl ArrayData { + pub fn into_inferred_arrow(self) -> VortexResult { + let data_type = infer_data_type(self.dtype())?; + self.into_arrow(&data_type) + } +} + #[cfg(test)] mod test { use arrow_schema::{DataType, Field, FieldRef, Fields, Schema}; diff --git a/vortex-array/src/arrow/record_batch.rs b/vortex-array/src/arrow/record_batch.rs index 25332b127f..397791bdc1 100644 --- a/vortex-array/src/arrow/record_batch.rs +++ b/vortex-array/src/arrow/record_batch.rs @@ -1,12 +1,12 @@ -use arrow_array::cast::as_struct_array; use arrow_array::RecordBatch; +use arrow_schema::Schema; use itertools::Itertools; use vortex_error::{vortex_err, VortexError, VortexResult}; use crate::array::StructArray; use crate::arrow::FromArrowArray; use crate::validity::Validity; -use crate::{ArrayData, IntoArrayData, IntoArrayVariant, IntoCanonical}; +use crate::{ArrayData, IntoArrayData, IntoArrayVariant}; impl TryFrom for ArrayData { type Error = VortexError; @@ -45,12 +45,23 @@ impl TryFrom for RecordBatch { } } +#[allow(dead_code)] +impl StructArray { + fn into_record_batch(self) -> RecordBatch { + todo!() + // RecordBatch::from(self) + } + + fn into_record_batch_with_schema(self, _schema: &Schema) -> RecordBatch { + todo!() + } +} + +// TODO: remove this impl TryFrom for RecordBatch { type Error = VortexError; - fn try_from(value: StructArray) -> VortexResult { - let array_ref = value.into_canonical()?.into_arrow()?; - let struct_array = as_struct_array(array_ref.as_ref()); - Ok(Self::from(struct_array)) + fn try_from(_value: StructArray) -> VortexResult { + todo!() } } diff --git a/vortex-array/src/canonical.rs b/vortex-array/src/canonical.rs index 8c6fddf98c..ed3f9ae77d 100644 --- a/vortex-array/src/canonical.rs +++ b/vortex-array/src/canonical.rs @@ -4,14 +4,15 @@ use std::sync::Arc; use arrow_array::types::*; use arrow_array::{ - Array, ArrayRef, ArrowPrimitiveType, BooleanArray as ArrowBoolArray, Date32Array, Date64Array, - NullArray as ArrowNullArray, PrimitiveArray as ArrowPrimitiveArray, + new_null_array, Array, ArrayRef, ArrowPrimitiveType, BooleanArray as ArrowBoolArray, + Date32Array, Date64Array, PrimitiveArray as ArrowPrimitiveArray, StructArray as ArrowStructArray, Time32MillisecondArray, Time32SecondArray, Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, }; use arrow_buffer::ScalarBuffer; -use arrow_schema::{Field, FieldRef, Fields}; +use arrow_schema::{DataType, Field, FieldRef, Fields}; +use itertools::Itertools; use vortex_datetime_dtype::{is_temporal_ext_type, TemporalMetadata, TimeUnit}; use vortex_dtype::{DType, NativePType, PType}; use vortex_error::{vortex_bail, VortexError, VortexResult}; @@ -70,25 +71,25 @@ impl Canonical { /// Scalar arrays such as Bool and Primitive canonical arrays should convert with /// zero copies, while more complex variants such as Struct may require allocations if its child /// arrays require decompression. - pub fn into_arrow(self) -> VortexResult { - Ok(match self { - Canonical::Null(a) => null_to_arrow(a)?, - Canonical::Bool(a) => bool_to_arrow(a)?, - Canonical::Primitive(a) => primitive_to_arrow(a)?, - Canonical::Struct(a) => struct_to_arrow(a)?, - Canonical::List(a) => list_to_arrow(a)?, - Canonical::VarBinView(a) => varbinview_as_arrow(&a), + pub fn into_arrow(self, data_type: &DataType) -> VortexResult { + match self { + Canonical::Null(a) => null_to_arrow(a, data_type), + Canonical::Bool(a) => bool_to_arrow(a, data_type), + Canonical::Primitive(a) => primitive_to_arrow(a, data_type), + Canonical::Struct(a) => struct_to_arrow(a, data_type), + Canonical::List(a) => list_to_arrow(a, data_type), + Canonical::VarBinView(a) => varbinview_to_arrow(&a, data_type), Canonical::Extension(a) => { if is_temporal_ext_type(a.id()) { - temporal_to_arrow(TemporalArray::try_from(a.into_array())?)? + temporal_to_arrow(TemporalArray::try_from(a.into_array())?) } else { // Convert storage array directly into arrow, losing type information // that will let us round-trip. // TODO(aduffy): https://github.com/spiraldb/vortex/issues/1167 - a.storage().into_arrow()? + a.storage().into_arrow(data_type) } } - }) + } } } @@ -156,21 +157,27 @@ impl Canonical { } } -fn null_to_arrow(null_array: NullArray) -> VortexResult { - Ok(Arc::new(ArrowNullArray::new(null_array.len()))) +fn null_to_arrow(null_array: NullArray, data_type: &DataType) -> VortexResult { + Ok(new_null_array(data_type, null_array.len())) } -fn bool_to_arrow(bool_array: BoolArray) -> VortexResult { +fn bool_to_arrow(bool_array: BoolArray, data_type: &DataType) -> VortexResult { + debug_assert_eq!(data_type, &DataType::Boolean); Ok(Arc::new(ArrowBoolArray::new( bool_array.boolean_buffer(), bool_array.logical_validity().to_null_buffer()?, ))) } -fn primitive_to_arrow(primitive_array: PrimitiveArray) -> VortexResult { +fn primitive_to_arrow( + primitive_array: PrimitiveArray, + data_type: &DataType, +) -> VortexResult { fn as_arrow_array_primitive( array: &PrimitiveArray, + data_type: &DataType, ) -> VortexResult>> { + debug_assert_eq!(data_type, &T::DATA_TYPE); Ok(Arc::new(ArrowPrimitiveArray::new( ScalarBuffer::::new(array.buffer().clone().into_arrow(), 0, array.len()), array.logical_validity().to_null_buffer()?, @@ -178,29 +185,39 @@ fn primitive_to_arrow(primitive_array: PrimitiveArray) -> VortexResult } Ok(match primitive_array.ptype() { - PType::U8 => as_arrow_array_primitive::(&primitive_array)?, - PType::U16 => as_arrow_array_primitive::(&primitive_array)?, - PType::U32 => as_arrow_array_primitive::(&primitive_array)?, - PType::U64 => as_arrow_array_primitive::(&primitive_array)?, - PType::I8 => as_arrow_array_primitive::(&primitive_array)?, - PType::I16 => as_arrow_array_primitive::(&primitive_array)?, - PType::I32 => as_arrow_array_primitive::(&primitive_array)?, - PType::I64 => as_arrow_array_primitive::(&primitive_array)?, - PType::F16 => as_arrow_array_primitive::(&primitive_array)?, - PType::F32 => as_arrow_array_primitive::(&primitive_array)?, - PType::F64 => as_arrow_array_primitive::(&primitive_array)?, + // TODO: verfiy that data_type is ignored not in debug + PType::U8 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::U16 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::U32 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::U64 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::I8 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::I16 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::I32 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::I64 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::F16 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::F32 => as_arrow_array_primitive::(&primitive_array, data_type)?, + PType::F64 => as_arrow_array_primitive::(&primitive_array, data_type)?, }) } -fn struct_to_arrow(struct_array: StructArray) -> VortexResult { - let field_arrays = struct_array - .names() +// TODO: think about top level struct array nullability, can you specify the nullability of the +// struct arrow array +fn struct_to_arrow(struct_array: StructArray, data_type: &DataType) -> VortexResult { + let target_fields = match data_type { + DataType::Struct(fields) => fields, + _ => vortex_bail!("Expected DataType::Struct, got {:?}", data_type), + }; + + let field_arrays = target_fields .iter() - .zip(struct_array.children()) - .map(|(name, f)| { - f.into_canonical() - .map_err(|err| err.with_context(format!("Failed to canonicalize field {}", name))) - .and_then(|c| c.into_arrow()) + .zip_eq(struct_array.children()) + .map(|(field, arr)| { + arr.into_canonical() + .map_err(|err| { + err.with_context(format!("Failed to canonicalize field {}", field.name())) + }) + // TODO: fix me + .and_then(|c| c.into_arrow(field.data_type())) }) .collect::>>()?; @@ -235,8 +252,23 @@ fn struct_to_arrow(struct_array: StructArray) -> VortexResult { } } +pub(crate) fn varbinview_to_arrow( + var_bin_view: &VarBinViewArray, + data_type: &DataType, +) -> VortexResult { + let arrow_arr = varbinview_as_arrow(var_bin_view); + // Note, arrow cast clones the array + Ok(match data_type { + DataType::Binary | DataType::LargeBinary | DataType::Utf8 | DataType::LargeUtf8 => { + arrow_cast::cast(arrow_arr.as_ref(), data_type)? + } + DataType::Utf8View | DataType::BinaryView => arrow_arr, + _ => vortex_bail!("Unsupported data type: {:?}", data_type), + }) +} + // TODO(joe): unify with varbin -fn list_to_arrow(list: ListArray) -> VortexResult { +fn list_to_arrow(list: ListArray, data_type: &DataType) -> VortexResult { let offsets = list .offsets() .into_primitive() @@ -253,7 +285,7 @@ fn list_to_arrow(list: ListArray) -> VortexResult { .map_err(|err| err.with_context("Failed to cast offsets to PrimitiveArray of i32"))?, }; - let values = list.elements().into_arrow()?; + let values = list.elements().into_arrow(data_type)?; let field_ref = FieldRef::new(Field::new_list_field( values.data_type().clone(), @@ -366,11 +398,11 @@ fn temporal_to_arrow(temporal_array: TemporalArray) -> VortexResult { pub trait IntoCanonical { fn into_canonical(self) -> VortexResult; - fn into_arrow(self) -> VortexResult + fn into_arrow(self, data_type: &DataType) -> VortexResult where Self: Sized, { - self.into_canonical()?.into_arrow() + self.into_canonical()?.into_arrow(data_type) } } @@ -379,7 +411,7 @@ pub trait IntoCanonical { pub trait IntoCanonicalVTable { fn into_canonical(&self, array: ArrayData) -> VortexResult; - fn into_arrow(&self, array: ArrayData) -> VortexResult; + fn into_arrow(&self, array: ArrayData, data_type: &DataType) -> VortexResult; } /// Implement the [IntoCanonicalVTable] for all encodings with arrays implementing [IntoCanonical]. @@ -396,8 +428,8 @@ where Ok(canonical) } - fn into_arrow(&self, array: ArrayData) -> VortexResult { - E::Array::try_from(array)?.into_arrow() + fn into_arrow(&self, array: ArrayData, data_type: &DataType) -> VortexResult { + E::Array::try_from(array)?.into_arrow(data_type) } } @@ -468,8 +500,8 @@ impl IntoCanonical for ArrayData { self.encoding().into_canonical(self) } - fn into_arrow(self) -> VortexResult { - self.encoding().into_arrow(self) + fn into_arrow(self, data_type: &DataType) -> VortexResult { + self.encoding().into_arrow(self, data_type) } } @@ -534,9 +566,9 @@ mod test { use arrow_schema::{DataType, Field}; use crate::array::{PrimitiveArray, SparseArray, StructArray}; - use crate::arrow::FromArrowArray; + use crate::arrow::{infer_data_type, FromArrowArray}; use crate::validity::Validity; - use crate::{ArrayData, IntoArrayData, IntoCanonical}; + use crate::{ArrayDType, ArrayData, IntoArrayData, IntoCanonical}; #[test] fn test_canonicalize_nested_struct() { @@ -569,8 +601,9 @@ mod test { ]) .unwrap(); + let data_type = infer_data_type(nested_struct_array.dtype()).unwrap(); let arrow_struct = nested_struct_array - .into_arrow() + .into_arrow(&data_type) .unwrap() .as_any() .downcast_ref::() @@ -637,10 +670,11 @@ mod test { ); let vortex_struct = ArrayData::from_arrow(&arrow_struct, true); + let data_type = infer_data_type(vortex_struct.dtype()).unwrap(); assert_eq!( &arrow_struct, - vortex_struct.into_arrow().unwrap().as_struct() + vortex_struct.into_arrow(&data_type).unwrap().as_struct() ); } @@ -660,7 +694,11 @@ mod test { ); let vortex_list = ArrayData::from_arrow(&arrow_list, true); + let data_type = infer_data_type(vortex_list.dtype()).unwrap(); - assert_eq!(&arrow_list, vortex_list.into_arrow().unwrap().as_list()); + assert_eq!( + &arrow_list, + vortex_list.into_arrow(&data_type).unwrap().as_list() + ); } } diff --git a/vortex-array/src/compute/boolean.rs b/vortex-array/src/compute/boolean.rs index 2bab7d22f6..fc03187bce 100644 --- a/vortex-array/src/compute/boolean.rs +++ b/vortex-array/src/compute/boolean.rs @@ -2,6 +2,7 @@ use std::sync::Arc; use arrow_array::cast::AsArray; use arrow_array::ArrayRef; +use arrow_schema::DataType; use vortex_dtype::DType; use vortex_error::{vortex_bail, vortex_err, VortexError, VortexResult}; @@ -168,11 +169,11 @@ pub(crate) fn arrow_boolean( let nullable = lhs.dtype().is_nullable() || rhs.dtype().is_nullable(); let lhs = Canonical::Bool(lhs.into_bool()?) - .into_arrow()? + .into_arrow(&DataType::Boolean)? .as_boolean() .clone(); let rhs = Canonical::Bool(rhs.into_bool()?) - .into_arrow()? + .into_arrow(&DataType::Boolean)? .as_boolean() .clone(); diff --git a/vortex-array/src/compute/filter.rs b/vortex-array/src/compute/filter.rs index 030f8195b5..7495c9f473 100644 --- a/vortex-array/src/compute/filter.rs +++ b/vortex-array/src/compute/filter.rs @@ -8,7 +8,7 @@ use vortex_dtype::{DType, Nullability}; use vortex_error::{vortex_bail, vortex_err, VortexError, VortexExpect, VortexResult}; use crate::array::{BoolArray, ConstantArray}; -use crate::arrow::FromArrowArray; +use crate::arrow::{infer_data_type, FromArrowArray}; use crate::compute::scalar_at; use crate::encoding::Encoding; use crate::stats::ArrayStatistics; @@ -107,7 +107,9 @@ fn filter_impl(array: &ArrayData, mask: FilterMask) -> VortexResult { array.encoding().id(), ); - let array_ref = array.clone().into_arrow()?; + let data_type = infer_data_type(array.dtype())?; + + let array_ref = array.clone().into_arrow(&data_type)?; let mask_array = BooleanArray::new(mask.to_boolean_buffer()?, None); let filtered = arrow_select::filter::filter(array_ref.as_ref(), &mask_array)?; diff --git a/vortex-array/src/encoding/opaque.rs b/vortex-array/src/encoding/opaque.rs index b25c0763cd..fcd07968b8 100644 --- a/vortex-array/src/encoding/opaque.rs +++ b/vortex-array/src/encoding/opaque.rs @@ -3,6 +3,7 @@ use std::fmt::{Debug, Display, Formatter}; use std::sync::Arc; use arrow_array::ArrayRef; +use arrow_schema::DataType; use vortex_error::{vortex_bail, vortex_panic, VortexResult}; use crate::compute::ComputeVTable; @@ -49,7 +50,7 @@ impl IntoCanonicalVTable for OpaqueEncoding { ) } - fn into_arrow(&self, _array: ArrayData) -> VortexResult { + fn into_arrow(&self, _array: ArrayData, _data_type: &DataType) -> VortexResult { vortex_bail!( "OpaqueEncoding: into_arrow cannot be called for opaque array ({})", self.0 diff --git a/vortex-datafusion/src/memory/plans.rs b/vortex-datafusion/src/memory/plans.rs index a416fc6ff1..b86f97a77a 100644 --- a/vortex-datafusion/src/memory/plans.rs +++ b/vortex-datafusion/src/memory/plans.rs @@ -21,7 +21,7 @@ use pin_project::pin_project; use vortex_array::array::ChunkedArray; use vortex_array::arrow::FromArrowArray; use vortex_array::compute::take; -use vortex_array::{ArrayData, IntoArrayVariant, IntoCanonical}; +use vortex_array::{ArrayData, IntoArrayVariant}; use vortex_dtype::field::Field; use vortex_error::{vortex_err, vortex_panic, VortexError}; use vortex_expr::ExprRef; @@ -160,7 +160,7 @@ impl Stream for RowIndicesStream { .conjunction_expr .evaluate(vortex_struct.as_ref()) .map_err(|e| DataFusionError::External(e.into()))? - .into_arrow()?; + .into_inferred_arrow()?; // Convert the `selection` BooleanArray into a UInt64Array of indices. let selection_indices = selection @@ -348,7 +348,7 @@ where // We should find a way to avoid decoding the filter columns and only decode the other // columns, then stitch the StructArray back together from those. let projected_for_output = chunk.project(this.output_projection)?; - let decoded = take(projected_for_output, &row_indices)?.into_arrow()?; + let decoded = take(projected_for_output, &row_indices)?.into_inferred_arrow()?; // Send back a single record batch of the decoded data. let output_batch = RecordBatch::from(decoded.as_struct()); diff --git a/vortex-datafusion/src/persistent/statistics.rs b/vortex-datafusion/src/persistent/statistics.rs index 3113a755be..10ebf1d864 100644 --- a/vortex-datafusion/src/persistent/statistics.rs +++ b/vortex-datafusion/src/persistent/statistics.rs @@ -7,14 +7,13 @@ use datafusion_expr::Accumulator; use vortex_array::array::StructArray; use vortex_array::stats::Stat; use vortex_array::variants::StructArrayTrait as _; -use vortex_array::IntoCanonical; use vortex_error::VortexResult; pub fn array_to_col_statistics(array: &StructArray) -> VortexResult { let mut stats = ColumnStatistics::new_unknown(); if let Some(null_count_array) = array.field_by_name(Stat::NullCount.name()) { - let array = null_count_array.into_arrow()?; + let array = null_count_array.into_inferred_arrow()?; let array = array.as_primitive::(); let null_count = array.iter().map(|v| v.unwrap_or_default()).sum::(); @@ -22,7 +21,7 @@ pub fn array_to_col_statistics(array: &StructArray) -> VortexResult VortexResult VortexResult> { match array.field_by_name(Stat::UncompressedSizeInBytes.name()) { None => Ok(None), Some(array) => { - let array = array.into_arrow()?; + let array = array.into_inferred_arrow()?; let array = array.as_primitive::(); let uncompressed_size = array.iter().map(|v| v.unwrap_or_default()).sum::();