From e63c3c9c4767ea39ab34ebfb67a9636872ab917c Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Sat, 10 Jan 2026 21:21:35 +0000 Subject: [PATCH 01/21] LayoutReader2 Signed-off-by: Nicholas Gates --- vortex-layout/src/lib.rs | 1 + vortex-layout/src/v2/mod.rs | 7 ++ vortex-layout/src/v2/optimizer.rs | 13 +++ vortex-layout/src/v2/reader.rs | 49 ++++++++++ vortex-layout/src/v2/readers/flat.rs | 112 ++++++++++++++++++++++ vortex-layout/src/v2/readers/mod.rs | 5 + vortex-layout/src/v2/readers/scalar_fn.rs | 94 ++++++++++++++++++ vortex-layout/src/v2/stream.rs | 34 +++++++ 8 files changed, 315 insertions(+) create mode 100644 vortex-layout/src/v2/mod.rs create mode 100644 vortex-layout/src/v2/optimizer.rs create mode 100644 vortex-layout/src/v2/reader.rs create mode 100644 vortex-layout/src/v2/readers/flat.rs create mode 100644 vortex-layout/src/v2/readers/mod.rs create mode 100644 vortex-layout/src/v2/readers/scalar_fn.rs create mode 100644 vortex-layout/src/v2/stream.rs diff --git a/vortex-layout/src/lib.rs b/vortex-layout/src/lib.rs index cc62398ef99..83be9af45a9 100644 --- a/vortex-layout/src/lib.rs +++ b/vortex-layout/src/lib.rs @@ -28,6 +28,7 @@ pub mod session; mod strategy; #[cfg(test)] mod test; +pub mod v2; pub mod vtable; pub type LayoutContext = VTableContext; diff --git a/vortex-layout/src/v2/mod.rs b/vortex-layout/src/v2/mod.rs new file mode 100644 index 00000000000..fd8a30068f6 --- /dev/null +++ b/vortex-layout/src/v2/mod.rs @@ -0,0 +1,7 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +mod optimizer; +pub mod reader; +pub mod readers; +pub mod stream; diff --git a/vortex-layout/src/v2/optimizer.rs b/vortex-layout/src/v2/optimizer.rs new file mode 100644 index 00000000000..db5567f7b13 --- /dev/null +++ b/vortex-layout/src/v2/optimizer.rs @@ -0,0 +1,13 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_error::VortexResult; + +use crate::v2::reader::LayoutReader2; +use crate::v2::reader::LayoutReader2Ref; + +impl dyn LayoutReader2 + '_ { + pub fn optimize(&self) -> VortexResult { + todo!() + } +} diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs new file mode 100644 index 00000000000..1aac4ded38c --- /dev/null +++ b/vortex-layout/src/v2/reader.rs @@ -0,0 +1,49 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Range; +use std::sync::Arc; + +use vortex_dtype::DType; +use vortex_error::VortexResult; + +use crate::v2::stream::SendableLayoutReaderStream; + +pub type LayoutReader2Ref = Arc; + +pub trait LayoutReader2: 'static + Send + Sync { + /// Returns the number of rows in the layout. + fn row_count(&self) -> u64; + + /// Returns the [`DType`] of the layout. + fn dtype(&self) -> &DType; + + /// Returns the number of child layouts. + fn nchildren(&self) -> usize; + + /// Returns the nth child reader of the layout. + fn child(&self, idx: usize) -> &LayoutReader2Ref; + + /// Execute the layout reader for the given range of data, returning a masked array stream. + fn execute(&self, row_range: Range) -> VortexResult; + + /// Attempt to reduce the layout reader to a more simple representation. + /// + /// Returns `Ok(None)` if no optimization is possible. + fn try_reduce(&self) -> VortexResult> { + _ = self; + Ok(None) + } + + /// Attempt to perform a reduction of the parent of this layout reader. + /// + /// Returns `Ok(None)` if no reduction is possible. + fn try_reduce_parent( + &self, + parent: &LayoutReader2Ref, + child_idx: usize, + ) -> VortexResult> { + _ = (self, parent, child_idx); + Ok(None) + } +} diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs new file mode 100644 index 00000000000..825c474b2a9 --- /dev/null +++ b/vortex-layout/src/v2/readers/flat.rs @@ -0,0 +1,112 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Range; + +use futures::future::BoxFuture; +use moka::future::FutureExt; +use vortex_array::ArrayRef; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; +use vortex_error::vortex_err; +use vortex_mask::Mask; + +use crate::layouts::SharedArrayFuture; +use crate::v2::reader::LayoutReader2; +use crate::v2::reader::LayoutReader2Ref; +use crate::v2::stream::LayoutReaderStream; +use crate::v2::stream::SendableLayoutReaderStream; + +pub struct FlatReader2 { + len: usize, + dtype: DType, + array_fut: SharedArrayFuture, +} + +impl LayoutReader2 for FlatReader2 { + fn row_count(&self) -> u64 { + self.len as u64 + } + + fn dtype(&self) -> &DType { + &self.dtype + } + + fn nchildren(&self) -> usize { + 0 + } + + fn child(&self, _idx: usize) -> &LayoutReader2Ref { + unreachable!() + } + + fn execute(&self, row_range: Range) -> VortexResult { + // We need to share the same array future + let array_fut = self.array_fut.clone(); + + let start = usize::try_from(row_range.start).map_err(|_| { + vortex_err!("Row range start {} is too large for usize", row_range.start) + })?; + let end = usize::try_from(row_range.end) + .map_err(|_| vortex_err!("Row range end {} is too large for usize", row_range.end))?; + + if start > self.len || end > self.len || start > end { + vortex_bail!( + "Row range {:?} is out of bounds for array of length {}", + row_range, + self.len + ); + } + + Ok(Box::new(FlatLayoutReaderStream { + array_fut, + offset: start, + remaining: end - start, + })) + } +} + +struct FlatLayoutReaderStream { + array_fut: SharedArrayFuture, + offset: usize, + remaining: usize, +} + +impl LayoutReaderStream for FlatLayoutReaderStream { + fn next_chunk_len(&self) -> Option { + if self.remaining == 0 { + None + } else { + Some(self.remaining) + } + } + + fn next_chunk( + &mut self, + selection: &Mask, + ) -> VortexResult>> { + if selection.len() > self.remaining { + vortex_bail!( + "Selection mask length {} exceeds remaining rows {}", + selection.len(), + self.remaining + ); + } + + let array_fut = self.array_fut.clone(); + let offset = self.offset; + let selection = selection.clone(); + + self.offset += selection.len(); + self.remaining -= selection.len(); + + Ok(async move { + let array = array_fut.await?; + let sliced_array = array.slice(offset..offset + selection.len()); + let selected_array = sliced_array.filter(selection.clone())?; + Ok(selected_array) + } + .boxed()) + } +} diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs new file mode 100644 index 00000000000..7239646c8dc --- /dev/null +++ b/vortex-layout/src/v2/readers/mod.rs @@ -0,0 +1,5 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +pub mod flat; +pub mod scalar_fn; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs new file mode 100644 index 00000000000..13bb7c2fad8 --- /dev/null +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -0,0 +1,94 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Range; + +use futures::future::BoxFuture; +use futures::future::try_join_all; +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::arrays::ScalarFnArray; +use vortex_array::expr::ScalarFn; +use vortex_array::optimizer::ArrayOptimizer; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_mask::Mask; + +use crate::v2::reader::LayoutReader2; +use crate::v2::reader::LayoutReader2Ref; +use crate::v2::stream::LayoutReaderStream; +use crate::v2::stream::SendableLayoutReaderStream; + +/// A [`LayoutReader2] for applying a scalar function to another layout. +pub struct ScalarFnReader { + scalar_fn: ScalarFn, + dtype: DType, + row_count: u64, + children: Vec, +} + +impl LayoutReader2 for ScalarFnReader { + fn row_count(&self) -> u64 { + self.row_count + } + + fn dtype(&self) -> &DType { + &self.dtype + } + + fn nchildren(&self) -> usize { + self.children.len() + } + + fn child(&self, idx: usize) -> &LayoutReader2Ref { + &self.children[idx] + } + + fn execute(&self, row_range: Range) -> VortexResult { + let input_streams = self + .children + .iter() + .map(|child| child.execute(row_range.clone())) + .collect::>>()?; + + Ok(Box::new(ScalarFnArrayStream { + scalar_fn: self.scalar_fn.clone(), + input_streams, + })) + } +} + +struct ScalarFnArrayStream { + scalar_fn: ScalarFn, + input_streams: Vec, +} + +impl LayoutReaderStream for ScalarFnArrayStream { + fn next_chunk_len(&self) -> Option { + self.input_streams + .iter() + .map(|s| s.next_chunk_len()) + .min() + .flatten() + } + + fn next_chunk( + &mut self, + selection: &Mask, + ) -> VortexResult>> { + let scalar_fn = self.scalar_fn.clone(); + let len = selection.true_count(); + let futs = self + .input_streams + .iter_mut() + .map(|s| s.next_chunk(selection)) + .collect::>>()?; + + Ok(Box::pin(async move { + let input_arrays = try_join_all(futs).await?; + let array = ScalarFnArray::try_new(scalar_fn, input_arrays, len)?.into_array(); + let array = array.optimize()?; + Ok(array) + })) + } +} diff --git a/vortex-layout/src/v2/stream.rs b/vortex-layout/src/v2/stream.rs new file mode 100644 index 00000000000..acb4105ef4a --- /dev/null +++ b/vortex-layout/src/v2/stream.rs @@ -0,0 +1,34 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use futures::future::BoxFuture; +use vortex_array::ArrayRef; +use vortex_error::VortexResult; +use vortex_mask::Mask; + +pub type SendableLayoutReaderStream = Box; + +/// A stream of data produced by a [`LayoutReader2`](crate::v2::reader::LayoutReader2). +/// +/// Layout readers are driven by requesting chunks of data using a given selection masks. +pub trait LayoutReaderStream { + /// Returns the length in rows of the next chunk in the stream. + /// + /// Returns [`None`] if the stream has ended. + fn next_chunk_len(&self) -> Option; + + /// Returns the next chunk of data given a selection mask of the requested length. + /// + /// The length of the provided selection mask must be `<=` the size returned from + /// [`LayoutReaderStream::next_chunk_len`]. + /// + /// The length of the returned chunk must be equal to the [`Mask::true_count`] of the selection + /// mask. + /// + /// The returned future has a `'static` lifetime allowing the calling to drive the stream + /// arbitrarily far without awaiting any data. + fn next_chunk( + &mut self, + selection: &Mask, + ) -> VortexResult>>; +} From ffd220bd1884c23d0ba893daed9112a8561e96fc Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Sat, 10 Jan 2026 21:38:52 +0000 Subject: [PATCH 02/21] LayoutReader2 Signed-off-by: Nicholas Gates --- vortex-layout/src/v2/readers/chunked.rs | 161 ++++++++++++++++++++++ vortex-layout/src/v2/readers/flat.rs | 6 + vortex-layout/src/v2/readers/mod.rs | 2 + vortex-layout/src/v2/readers/scalar_fn.rs | 6 + vortex-layout/src/v2/readers/struct_.rs | 99 +++++++++++++ vortex-layout/src/v2/stream.rs | 4 + 6 files changed, 278 insertions(+) create mode 100644 vortex-layout/src/v2/readers/chunked.rs create mode 100644 vortex-layout/src/v2/readers/struct_.rs diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs new file mode 100644 index 00000000000..58697e899eb --- /dev/null +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -0,0 +1,161 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Range; + +use futures::future::BoxFuture; +use futures::future::try_join_all; +use moka::future::FutureExt; +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::arrays::ChunkedArray; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; +use vortex_error::vortex_err; +use vortex_mask::Mask; + +use crate::v2::reader::LayoutReader2; +use crate::v2::reader::LayoutReader2Ref; +use crate::v2::stream::LayoutReaderStream; +use crate::v2::stream::SendableLayoutReaderStream; + +pub struct ChunkedReader2 { + row_count: u64, + dtype: DType, + chunks: Vec, +} + +impl LayoutReader2 for ChunkedReader2 { + fn row_count(&self) -> u64 { + self.row_count + } + + fn dtype(&self) -> &DType { + &self.dtype + } + + fn nchildren(&self) -> usize { + self.chunks.len() + } + + fn child(&self, idx: usize) -> &LayoutReader2Ref { + &self.chunks[idx] + } + + fn execute(&self, row_range: Range) -> VortexResult { + let mut remaining_start = row_range.start; + let mut remaining_end = row_range.end; + let mut streams = Vec::new(); + + for chunk in &self.chunks { + let chunk_row_count = chunk.row_count(); + + if remaining_start >= chunk_row_count { + // This chunk is before the requested range + remaining_start -= chunk_row_count; + remaining_end -= chunk_row_count; + continue; + } + + let start_in_chunk = remaining_start; + let end_in_chunk = if remaining_end <= chunk_row_count { + remaining_end + } else { + chunk_row_count + }; + + streams.push(chunk.execute(start_in_chunk..end_in_chunk)?); + + remaining_start = 0; + if remaining_end <= chunk_row_count { + break; + } else { + remaining_end -= chunk_row_count; + } + } + + Ok(Box::new(ChunkedReaderStream { + dtype: self.dtype.clone(), + chunks: streams, + })) + } +} + +struct ChunkedReaderStream { + dtype: DType, + chunks: Vec, +} + +impl LayoutReaderStream for ChunkedReaderStream { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn next_chunk_len(&self) -> Option { + self.chunks + .iter() + .map(|s| s.next_chunk_len()) + .find(|len| len.is_some()) + .flatten() + } + + fn next_chunk( + &mut self, + selection: &Mask, + ) -> VortexResult>> { + // Remove any chunks that are already exhausted + loop { + if self.chunks.is_empty() { + vortex_bail!("Early termination of chunked layout"); + } + if self.chunks[0].next_chunk_len().is_none() { + self.chunks.remove(0); + } else { + break; + } + } + + // Get the length of the next chunk + let mut next_len = self.chunks[0] + .next_chunk_len() + .ok_or_else(|| vortex_err!("Early termination of chunked layout"))?; + + if selection.len() <= next_len { + // The selection is smaller than the next chunk length, therefore we only need one chunk + return self.chunks[0].next_chunk(selection); + } + + // Otherwise, we need to gather from multiple chunks + let mut selection = selection.clone(); + let mut futs = vec![]; + while !selection.is_empty() { + if self.chunks.is_empty() { + vortex_bail!("Early termination of chunked layout"); + } + + // Slice off the right amount of selection for this chunk + let next_sel = selection.slice(..next_len); + selection = selection.slice(next_len..); + + let fut = self.chunks[0].next_chunk(&next_sel)?; + futs.push(fut); + + // Remove any chunks that are already exhausted + loop { + if self.chunks[0].next_chunk_len().is_none() { + self.chunks.remove(0); + } else { + break; + } + } + } + + let dtype = self.dtype.clone(); + Ok(async move { + let arrays = try_join_all(futs).await?; + Ok(ChunkedArray::try_new(arrays, dtype)?.into_array()) + } + .boxed()) + } +} diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 825c474b2a9..95aa5920c46 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -60,6 +60,7 @@ impl LayoutReader2 for FlatReader2 { } Ok(Box::new(FlatLayoutReaderStream { + dtype: self.dtype.clone(), array_fut, offset: start, remaining: end - start, @@ -68,12 +69,17 @@ impl LayoutReader2 for FlatReader2 { } struct FlatLayoutReaderStream { + dtype: DType, array_fut: SharedArrayFuture, offset: usize, remaining: usize, } impl LayoutReaderStream for FlatLayoutReaderStream { + fn dtype(&self) -> &DType { + &self.dtype + } + fn next_chunk_len(&self) -> Option { if self.remaining == 0 { None diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 7239646c8dc..6d020982f62 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -1,5 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +pub mod chunked; pub mod flat; pub mod scalar_fn; +pub mod struct_; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 13bb7c2fad8..5cf67c033eb 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -52,6 +52,7 @@ impl LayoutReader2 for ScalarFnReader { .collect::>>()?; Ok(Box::new(ScalarFnArrayStream { + dtype: self.dtype.clone(), scalar_fn: self.scalar_fn.clone(), input_streams, })) @@ -59,11 +60,16 @@ impl LayoutReader2 for ScalarFnReader { } struct ScalarFnArrayStream { + dtype: DType, scalar_fn: ScalarFn, input_streams: Vec, } impl LayoutReaderStream for ScalarFnArrayStream { + fn dtype(&self) -> &DType { + &self.dtype + } + fn next_chunk_len(&self) -> Option { self.input_streams .iter() diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs new file mode 100644 index 00000000000..38b832337d5 --- /dev/null +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -0,0 +1,99 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::ops::Range; + +use futures::future::BoxFuture; +use futures::future::try_join_all; +use moka::future::FutureExt; +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::arrays::StructArray; +use vortex_array::validity::Validity; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_mask::Mask; + +use crate::v2::reader::LayoutReader2; +use crate::v2::reader::LayoutReader2Ref; +use crate::v2::stream::LayoutReaderStream; +use crate::v2::stream::SendableLayoutReaderStream; + +pub struct StructReader2 { + row_count: u64, + dtype: DType, + fields: Vec, +} + +impl LayoutReader2 for StructReader2 { + fn row_count(&self) -> u64 { + self.row_count + } + + fn dtype(&self) -> &DType { + &self.dtype + } + + fn nchildren(&self) -> usize { + self.fields.len() + } + + fn child(&self, idx: usize) -> &LayoutReader2Ref { + &self.fields[idx] + } + + fn execute(&self, row_range: Range) -> VortexResult { + let field_streams = self + .fields + .iter() + .map(|field| field.execute(row_range.clone())) + .collect::>>()?; + + Ok(Box::new(StructReaderStream { + dtype: self.dtype.clone(), + fields: field_streams, + })) + } +} + +struct StructReaderStream { + dtype: DType, + fields: Vec, +} + +impl LayoutReaderStream for StructReaderStream { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn next_chunk_len(&self) -> Option { + self.fields + .iter() + .map(|s| s.next_chunk_len()) + .min() + .flatten() + } + + fn next_chunk( + &mut self, + selection: &Mask, + ) -> VortexResult>> { + let struct_fields = self.dtype.as_struct_fields().clone(); + let validity: Validity = self.dtype.nullability().into(); + let fields = self + .fields + .iter_mut() + .map(|s| s.next_chunk(selection)) + .collect::>>()?; + let len = selection.true_count(); + + Ok(async move { + let fields = try_join_all(fields).await?; + Ok( + StructArray::try_new_with_dtype(fields, struct_fields, len, validity.clone())? + .into_array(), + ) + } + .boxed()) + } +} diff --git a/vortex-layout/src/v2/stream.rs b/vortex-layout/src/v2/stream.rs index acb4105ef4a..7e57da7ba2b 100644 --- a/vortex-layout/src/v2/stream.rs +++ b/vortex-layout/src/v2/stream.rs @@ -3,6 +3,7 @@ use futures::future::BoxFuture; use vortex_array::ArrayRef; +use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_mask::Mask; @@ -12,6 +13,9 @@ pub type SendableLayoutReaderStream = Box &DType; + /// Returns the length in rows of the next chunk in the stream. /// /// Returns [`None`] if the stream has ended. From 80ecccc7a1413b434fe22a7de6948387366305c0 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Sun, 11 Jan 2026 10:26:10 +0000 Subject: [PATCH 03/21] LayoutReader2 Signed-off-by: Nicholas Gates --- vortex-layout/src/layout.rs | 16 ++++++++++++++++ vortex-layout/src/segments/source.rs | 5 +++++ vortex-layout/src/v2/reader.rs | 15 +++++++++++++++ vortex-layout/src/v2/readers/struct_.rs | 1 + vortex-layout/src/vtable.rs | 12 ++++++++++++ 5 files changed, 49 insertions(+) diff --git a/vortex-layout/src/layout.rs b/vortex-layout/src/layout.rs index 368d7c6bdc8..e22bf81f1a6 100644 --- a/vortex-layout/src/layout.rs +++ b/vortex-layout/src/layout.rs @@ -25,6 +25,8 @@ use crate::display::DisplayLayoutTree; use crate::display::display_tree_with_segment_sizes; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2::reader::LayoutReader2Ref; pub type LayoutId = ArcRef; @@ -76,6 +78,12 @@ pub trait Layout: 'static + Send + Sync + Debug + private::Sealed { segment_source: Arc, session: &VortexSession, ) -> VortexResult; + + fn new_reader2( + &self, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult; } pub trait IntoLayout { @@ -331,6 +339,14 @@ impl Layout for LayoutAdapter { ) -> VortexResult { V::new_reader(&self.0, name, segment_source, session) } + + fn new_reader2( + &self, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + V::new_reader2(&self.0, segment_source, session) + } } mod private { diff --git a/vortex-layout/src/segments/source.rs b/vortex-layout/src/segments/source.rs index a48a79b2889..3a5b3f84b99 100644 --- a/vortex-layout/src/segments/source.rs +++ b/vortex-layout/src/segments/source.rs @@ -1,6 +1,8 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::sync::Arc; + use futures::future::BoxFuture; use vortex_array::buffer::BufferHandle; use vortex_error::VortexResult; @@ -9,6 +11,9 @@ use crate::segments::SegmentId; /// Static future resolving to a segment byte buffer. pub type SegmentFuture = BoxFuture<'static, VortexResult>; +/// A reference-counted segment source. +pub type SegmentSourceRef = Arc; + /// A trait for providing segment data to a [`crate::LayoutReader`]. pub trait SegmentSource: 'static + Send + Sync { /// Request a segment, returning a future that will eventually resolve to the segment data. diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index 1aac4ded38c..6dad4890594 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -13,6 +13,14 @@ pub type LayoutReader2Ref = Arc; pub trait LayoutReader2: 'static + Send + Sync { /// Returns the number of rows in the layout. + /// + /// TODO(ngates): if we relaxed this to be a cardinality estimate, we could support arbitrary + /// data streams including joins, group bys, scans, etc. The problem is, invoking execute with + /// some row range becomes weird... + /// Perhaps we borrow DataFusion's style of partitioning where we ask the reader to partition + /// into `n` and then pass the partition index to execute? Or perhaps we just pass `n` to the + /// execute call and have the reader return all `n` partitions at once? That would also make + /// sharing cached resources a lot easier. fn row_count(&self) -> u64; /// Returns the [`DType`] of the layout. @@ -25,6 +33,13 @@ pub trait LayoutReader2: 'static + Send + Sync { fn child(&self, idx: usize) -> &LayoutReader2Ref; /// Execute the layout reader for the given range of data, returning a masked array stream. + /// + /// TODO(ngates): this bit feels weird to me. + /// It's odd that we don't know when a particular reader is done executing. Meaning we don't + /// have a good lifetime for cached resources. The returned reader stream _does_ have a good + /// lifetime for caching (the duration of the stream), so perhaps we just say that layout + /// readers should not hold data and instead each call to execute should make its own segment + /// requests? Assuming we can de-dupe also within a segment source, this seems reasonable. fn execute(&self, row_range: Range) -> VortexResult; /// Attempt to reduce the layout reader to a more simple representation. diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 38b832337d5..85c17855686 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -22,6 +22,7 @@ use crate::v2::stream::SendableLayoutReaderStream; pub struct StructReader2 { row_count: u64, dtype: DType, + // TODO(ngates): we should make this lazy? fields: Vec, } diff --git a/vortex-layout/src/vtable.rs b/vortex-layout/src/vtable.rs index 8012907693d..969fc02af0b 100644 --- a/vortex-layout/src/vtable.rs +++ b/vortex-layout/src/vtable.rs @@ -24,6 +24,8 @@ use crate::LayoutRef; use crate::children::LayoutChildren; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2::reader::LayoutReader2Ref; pub trait VTable: 'static + Sized + Send + Sync + Debug { type Layout: 'static + Send + Sync + Clone + Debug + Deref + IntoLayout; @@ -65,6 +67,16 @@ pub trait VTable: 'static + Sized + Send + Sync + Debug { session: &VortexSession, ) -> VortexResult; + /// Create a new v2 reader for the layout. + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + let _ = (layout, segment_source, session); + vortex_bail!("new_reader2 not implemented for this layout") + } + #[cfg(gpu_unstable)] /// Create a new reader for the layout that uses a gpu device fn new_gpu_reader( From 5d62d4d01e850c25d8187f9634b5551c3ee937e9 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 09:58:00 +0000 Subject: [PATCH 04/21] Add flag to run benchmarks under Samply Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + vortex-scan/Cargo.toml | 1 + vortex-scan/src/lib.rs | 1 + vortex-scan/src/v2/mod.rs | 5 ++ vortex-scan/src/v2/reader.rs | 53 +++++++++++++++++++ vortex-scan/src/v2/source.rs | 98 ++++++++++++++++++++++++++++++++++++ 6 files changed, 159 insertions(+) create mode 100644 vortex-scan/src/v2/mod.rs create mode 100644 vortex-scan/src/v2/reader.rs create mode 100644 vortex-scan/src/v2/source.rs diff --git a/Cargo.lock b/Cargo.lock index 47cb5e7e05a..83c4e454abb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10675,6 +10675,7 @@ version = "0.1.0" dependencies = [ "arrow-array 57.1.0", "arrow-schema 57.1.0", + "async-trait", "bit-vec 0.8.0", "futures", "itertools 0.14.0", diff --git a/vortex-scan/Cargo.toml b/vortex-scan/Cargo.toml index d78e92fa147..41961760041 100644 --- a/vortex-scan/Cargo.toml +++ b/vortex-scan/Cargo.toml @@ -29,6 +29,7 @@ vortex-mask = { workspace = true } vortex-metrics = { workspace = true } vortex-session = { workspace = true } +async-trait = { workspace = true } bit-vec = { workspace = true } futures = { workspace = true } itertools = { workspace = true } diff --git a/vortex-scan/src/lib.rs b/vortex-scan/src/lib.rs index 52eb0dd47bb..ed62775f7b0 100644 --- a/vortex-scan/src/lib.rs +++ b/vortex-scan/src/lib.rs @@ -26,5 +26,6 @@ pub mod gpu; mod repeated_scan; #[cfg(test)] mod test; +pub mod v2; pub use repeated_scan::RepeatedScan; diff --git a/vortex-scan/src/v2/mod.rs b/vortex-scan/src/v2/mod.rs new file mode 100644 index 00000000000..af92f452a33 --- /dev/null +++ b/vortex-scan/src/v2/mod.rs @@ -0,0 +1,5 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +pub mod reader; +pub mod source; diff --git a/vortex-scan/src/v2/reader.rs b/vortex-scan/src/v2/reader.rs new file mode 100644 index 00000000000..916b7379c47 --- /dev/null +++ b/vortex-scan/src/v2/reader.rs @@ -0,0 +1,53 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use async_trait::async_trait; +use vortex_array::ArrayRef; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_mask::Mask; + +/// A reader provides an interface for loading data from row-indexed layouts. +/// +/// Unlike a [`super::source::Source`], readers have a concrete row count allowing fixed +/// partitions over a known set of rows. Readers are driven by providing an input stream of +/// array data that can be used to provide arguments to parameterized filter and projection +/// expressions. +#[async_trait] +pub trait Reader: 'static + Send + Sync { + /// Get the data type of the layout being read. + fn dtype(&self) -> &DType; + + /// Returns the number of rows in the reader. + fn row_count(&self) -> u64; + + /// Creates a scan where an input stream is used to drive the output data. + /// + /// TODO(ngates): should this take a RowSelection? + async fn scan( + &self, + input_dtype: &DType, + row_offset: u64, + row_mask: Mask, + ) -> VortexResult; +} + +pub type ReaderScanRef = Box; + +/// A scan over a reader, producing output arrays given an input array to parameterize the filter +/// and projection expressions. +#[async_trait] +pub trait ReaderScan { + /// The data type of the returned data. + fn dtype(&self) -> &DType; + + /// The preferred maximum row count for the next batch. + /// + /// Returns [`None`] if there are no more batches. + fn next_batch_size(&mut self) -> Option; + + /// Returns the next batch of data given an input array. + /// + /// The returned batch must have the same number of rows as the input array. + async fn next_batch(&mut self, input: ArrayRef) -> VortexResult; +} diff --git a/vortex-scan/src/v2/source.rs b/vortex-scan/src/v2/source.rs new file mode 100644 index 00000000000..bfd7d6ee36e --- /dev/null +++ b/vortex-scan/src/v2/source.rs @@ -0,0 +1,98 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::any::Any; +use std::sync::Arc; + +use async_trait::async_trait; +use futures::stream::BoxStream; +use vortex_array::expr::Expression; +use vortex_array::stream::SendableArrayStream; +use vortex_dtype::DType; +use vortex_error::VortexResult; + +/// Create a Vortex source from serialized configuration. +/// +/// Providers can be registered with Vortex under a specific +#[async_trait(?Send)] +pub trait SourceProvider: 'static { + /// URI schemes handled by this source provider. + /// + /// TODO(ngates): this might not be the right way to plugin sources. + fn schemes(&self) -> &[&str]; + + /// Initialize a new source. + async fn init_source(&self, uri: String) -> VortexResult; + + /// Serialize a source split to bytes. + async fn serialize_split(&self, split: &dyn Split) -> VortexResult>; + + /// Deserialize a source split from bytes. + async fn deserialize_split(&self, data: &[u8]) -> VortexResult; +} + +/// A reference-counted source. +pub type SourceRef = Arc; + +/// A source represents a streamable dataset that can be scanned with projection and filter +/// expressions. Each scan produces splits that can be executed in parallel to read data. +/// Each split can be serialized for remote execution. +#[async_trait] +pub trait Source: 'static + Send + Sync { + /// Returns the dtype of the source. + fn dtype(&self) -> &DType; + + /// Returns an estimate of the row count of the source. + fn row_count_estimate(&self) -> Estimate; + + /// Returns a scan over the source. + async fn scan(&self, scan_request: ScanRequest) -> VortexResult; +} + +#[derive(Debug, Clone, Default)] +pub struct ScanRequest { + pub projection: Option, + pub filter: Option, + pub limit: Option, +} + +pub type SourceScanRef = Box; + +#[async_trait] +pub trait SourceScan: 'static + Send + Sync { + /// The returned dtype of the scan. + fn dtype(&self) -> &DType; + + /// An estimate of the remaining splits. + fn remaining_splits_estimate(&self) -> Estimate; + + /// Returns the next batch of splits to be processed. + /// + /// This should not return _more_ than the max_batch_size splits, but may return fewer. + async fn next_splits(&mut self, max_splits: usize) -> VortexResult>; +} + +pub type SplitStream = BoxStream<'static, VortexResult>; +pub type SplitRef = Arc; + +pub trait Split: 'static + Send + Sync { + /// Downcast the split to a concrete type. + fn as_any(&self) -> &dyn Any; + + /// Executes the split. + fn execute(&self) -> VortexResult; + + /// Returns an estimate of the row count for this split. + fn row_count_estimate(&self) -> Estimate; + + /// Returns an estimate of the byte size for this split. + fn byte_size_estimate(&self) -> Estimate; +} + +#[derive(Default)] +pub enum Estimate { + Exact(T), + UpperBound(T), + #[default] + Unknown, +} From f7f83289770879965eebb9eaa2b94484982fcac2 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 11:39:01 +0000 Subject: [PATCH 05/21] Add flag to run benchmarks under Samply Signed-off-by: Nicholas Gates --- vortex-scan/src/v2/reader.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/vortex-scan/src/v2/reader.rs b/vortex-scan/src/v2/reader.rs index 916b7379c47..6292933645e 100644 --- a/vortex-scan/src/v2/reader.rs +++ b/vortex-scan/src/v2/reader.rs @@ -1,6 +1,8 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::ops::Range; + use async_trait::async_trait; use vortex_array::ArrayRef; use vortex_dtype::DType; @@ -21,15 +23,8 @@ pub trait Reader: 'static + Send + Sync { /// Returns the number of rows in the reader. fn row_count(&self) -> u64; - /// Creates a scan where an input stream is used to drive the output data. - /// - /// TODO(ngates): should this take a RowSelection? - async fn scan( - &self, - input_dtype: &DType, - row_offset: u64, - row_mask: Mask, - ) -> VortexResult; + /// Creates a scan over the given row range of the reader. + async fn scan(&self, row_range: Range) -> VortexResult; } pub type ReaderScanRef = Box; @@ -48,6 +43,7 @@ pub trait ReaderScan { /// Returns the next batch of data given an input array. /// - /// The returned batch must have the same number of rows as the input array. - async fn next_batch(&mut self, input: ArrayRef) -> VortexResult; + /// The returned batch must have the same number of rows as the [`Mask::true_count`]. + /// The provided mask will have at most [`next_batch_size`] rows. + async fn next_batch(&mut self, mask: Mask) -> VortexResult; } From 64fbb5db19285955df23d459e09f2bf30e92f525 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 14:19:55 +0000 Subject: [PATCH 06/21] Scan API Signed-off-by: Nicholas Gates --- vortex-layout/src/layout.rs | 6 +- vortex-layout/src/v2/optimizer.rs | 8 +- vortex-layout/src/v2/reader.rs | 86 ++++++++++---------- vortex-layout/src/v2/readers/chunked.rs | 10 +-- vortex-layout/src/v2/readers/flat.rs | 8 +- vortex-layout/src/v2/readers/scalar_fn.rs | 12 +-- vortex-layout/src/v2/readers/struct_.rs | 10 +-- vortex-layout/src/v2/stream.rs | 2 +- vortex-layout/src/vtable.rs | 4 +- vortex-scan/src/lib.rs | 1 - vortex-scan/src/v2/mod.rs | 5 -- vortex-scan/src/v2/reader.rs | 49 ------------ vortex-scan/src/v2/source.rs | 98 ----------------------- 13 files changed, 74 insertions(+), 225 deletions(-) delete mode 100644 vortex-scan/src/v2/mod.rs delete mode 100644 vortex-scan/src/v2/reader.rs delete mode 100644 vortex-scan/src/v2/source.rs diff --git a/vortex-layout/src/layout.rs b/vortex-layout/src/layout.rs index e22bf81f1a6..79f5bd26c85 100644 --- a/vortex-layout/src/layout.rs +++ b/vortex-layout/src/layout.rs @@ -26,7 +26,7 @@ use crate::display::display_tree_with_segment_sizes; use crate::segments::SegmentId; use crate::segments::SegmentSource; use crate::segments::SegmentSourceRef; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::ReaderRef; pub type LayoutId = ArcRef; @@ -83,7 +83,7 @@ pub trait Layout: 'static + Send + Sync + Debug + private::Sealed { &self, segment_source: &SegmentSourceRef, session: &VortexSession, - ) -> VortexResult; + ) -> VortexResult; } pub trait IntoLayout { @@ -344,7 +344,7 @@ impl Layout for LayoutAdapter { &self, segment_source: &SegmentSourceRef, session: &VortexSession, - ) -> VortexResult { + ) -> VortexResult { V::new_reader2(&self.0, segment_source, session) } } diff --git a/vortex-layout/src/v2/optimizer.rs b/vortex-layout/src/v2/optimizer.rs index db5567f7b13..5c2686654e6 100644 --- a/vortex-layout/src/v2/optimizer.rs +++ b/vortex-layout/src/v2/optimizer.rs @@ -3,11 +3,11 @@ use vortex_error::VortexResult; -use crate::v2::reader::LayoutReader2; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; -impl dyn LayoutReader2 + '_ { - pub fn optimize(&self) -> VortexResult { +impl dyn Reader + '_ { + pub fn optimize(&self) -> VortexResult { todo!() } } diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index 6dad4890594..e8b56ceedb2 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -4,61 +4,63 @@ use std::ops::Range; use std::sync::Arc; +use async_trait::async_trait; +use vortex_array::ArrayRef; use vortex_dtype::DType; use vortex_error::VortexResult; +use vortex_mask::Mask; -use crate::v2::stream::SendableLayoutReaderStream; +pub type ReaderRef = Arc; -pub type LayoutReader2Ref = Arc; - -pub trait LayoutReader2: 'static + Send + Sync { - /// Returns the number of rows in the layout. - /// - /// TODO(ngates): if we relaxed this to be a cardinality estimate, we could support arbitrary - /// data streams including joins, group bys, scans, etc. The problem is, invoking execute with - /// some row range becomes weird... - /// Perhaps we borrow DataFusion's style of partitioning where we ask the reader to partition - /// into `n` and then pass the partition index to execute? Or perhaps we just pass `n` to the - /// execute call and have the reader return all `n` partitions at once? That would also make - /// sharing cached resources a lot easier. - fn row_count(&self) -> u64; - - /// Returns the [`DType`] of the layout. +/// A reader provides an interface for loading data from row-indexed layouts. +/// +/// Unlike a [`super::source::DataSource`], readers have a concrete row count allowing fixed +/// partitions over a known set of rows. Readers are driven by providing an input stream of +/// array data that can be used to provide arguments to parameterized filter and projection +/// expressions. +pub trait Reader: 'static + Send + Sync { + /// Get the data type of the layout being read. fn dtype(&self) -> &DType; - /// Returns the number of child layouts. - fn nchildren(&self) -> usize; - - /// Returns the nth child reader of the layout. - fn child(&self, idx: usize) -> &LayoutReader2Ref; - - /// Execute the layout reader for the given range of data, returning a masked array stream. - /// - /// TODO(ngates): this bit feels weird to me. - /// It's odd that we don't know when a particular reader is done executing. Meaning we don't - /// have a good lifetime for cached resources. The returned reader stream _does_ have a good - /// lifetime for caching (the duration of the stream), so perhaps we just say that layout - /// readers should not hold data and instead each call to execute should make its own segment - /// requests? Assuming we can de-dupe also within a segment source, this seems reasonable. - fn execute(&self, row_range: Range) -> VortexResult; + /// Returns the number of rows in the reader. + fn row_count(&self) -> u64; - /// Attempt to reduce the layout reader to a more simple representation. - /// - /// Returns `Ok(None)` if no optimization is possible. - fn try_reduce(&self) -> VortexResult> { - _ = self; + /// Reduces the reader, simplifying its internal structure if possible. + fn try_reduce(&self) -> VortexResult> { Ok(None) } - /// Attempt to perform a reduction of the parent of this layout reader. - /// - /// Returns `Ok(None)` if no reduction is possible. + /// Reduce the parent reader if possible, returning a new reader if successful. fn try_reduce_parent( &self, - parent: &LayoutReader2Ref, + parent: &ReaderRef, child_idx: usize, - ) -> VortexResult> { - _ = (self, parent, child_idx); + ) -> VortexResult> { + let _ = (parent, child_idx); Ok(None) } + + /// Creates a scan over the given row range of the reader. + fn scan(&self, row_range: Range) -> VortexResult; +} + +pub type ReaderScanRef = Box; + +/// A scan over a reader, producing output arrays given an input array to parameterize the filter +/// and projection expressions. +#[async_trait] +pub trait ReaderScan { + /// The data type of the returned data. + fn dtype(&self) -> &DType; + + /// The preferred maximum row count for the next batch. + /// + /// Returns [`None`] if there are no more batches. + fn next_batch_size(&mut self) -> Option; + + /// Returns the next batch of data given an input array. + /// + /// The returned batch must have the same number of rows as the [`Mask::true_count`]. + /// The provided mask will have at most [`next_batch_size`] rows. + async fn next_batch(&mut self, mask: Mask) -> VortexResult; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 58697e899eb..0350a3f38fd 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -15,18 +15,18 @@ use vortex_error::vortex_bail; use vortex_error::vortex_err; use vortex_mask::Mask; -use crate::v2::reader::LayoutReader2; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::stream::LayoutReaderStream; use crate::v2::stream::SendableLayoutReaderStream; pub struct ChunkedReader2 { row_count: u64, dtype: DType, - chunks: Vec, + chunks: Vec, } -impl LayoutReader2 for ChunkedReader2 { +impl Reader for ChunkedReader2 { fn row_count(&self) -> u64 { self.row_count } @@ -39,7 +39,7 @@ impl LayoutReader2 for ChunkedReader2 { self.chunks.len() } - fn child(&self, idx: usize) -> &LayoutReader2Ref { + fn child(&self, idx: usize) -> &ReaderRef { &self.chunks[idx] } diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 95aa5920c46..57ce3baeadb 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -13,8 +13,8 @@ use vortex_error::vortex_err; use vortex_mask::Mask; use crate::layouts::SharedArrayFuture; -use crate::v2::reader::LayoutReader2; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::stream::LayoutReaderStream; use crate::v2::stream::SendableLayoutReaderStream; @@ -24,7 +24,7 @@ pub struct FlatReader2 { array_fut: SharedArrayFuture, } -impl LayoutReader2 for FlatReader2 { +impl Reader for FlatReader2 { fn row_count(&self) -> u64 { self.len as u64 } @@ -37,7 +37,7 @@ impl LayoutReader2 for FlatReader2 { 0 } - fn child(&self, _idx: usize) -> &LayoutReader2Ref { + fn child(&self, _idx: usize) -> &ReaderRef { unreachable!() } diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 5cf67c033eb..3bf8e679924 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -14,20 +14,20 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_mask::Mask; -use crate::v2::reader::LayoutReader2; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::stream::LayoutReaderStream; use crate::v2::stream::SendableLayoutReaderStream; -/// A [`LayoutReader2] for applying a scalar function to another layout. +/// A [`Reader] for applying a scalar function to another layout. pub struct ScalarFnReader { scalar_fn: ScalarFn, dtype: DType, row_count: u64, - children: Vec, + children: Vec, } -impl LayoutReader2 for ScalarFnReader { +impl Reader for ScalarFnReader { fn row_count(&self) -> u64 { self.row_count } @@ -40,7 +40,7 @@ impl LayoutReader2 for ScalarFnReader { self.children.len() } - fn child(&self, idx: usize) -> &LayoutReader2Ref { + fn child(&self, idx: usize) -> &ReaderRef { &self.children[idx] } diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 85c17855686..bd1822fcbfa 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -14,8 +14,8 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_mask::Mask; -use crate::v2::reader::LayoutReader2; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::stream::LayoutReaderStream; use crate::v2::stream::SendableLayoutReaderStream; @@ -23,10 +23,10 @@ pub struct StructReader2 { row_count: u64, dtype: DType, // TODO(ngates): we should make this lazy? - fields: Vec, + fields: Vec, } -impl LayoutReader2 for StructReader2 { +impl Reader for StructReader2 { fn row_count(&self) -> u64 { self.row_count } @@ -39,7 +39,7 @@ impl LayoutReader2 for StructReader2 { self.fields.len() } - fn child(&self, idx: usize) -> &LayoutReader2Ref { + fn child(&self, idx: usize) -> &ReaderRef { &self.fields[idx] } diff --git a/vortex-layout/src/v2/stream.rs b/vortex-layout/src/v2/stream.rs index 7e57da7ba2b..b2cbf6c7cfa 100644 --- a/vortex-layout/src/v2/stream.rs +++ b/vortex-layout/src/v2/stream.rs @@ -9,7 +9,7 @@ use vortex_mask::Mask; pub type SendableLayoutReaderStream = Box; -/// A stream of data produced by a [`LayoutReader2`](crate::v2::reader::LayoutReader2). +/// A stream of data produced by a [`Reader`](crate::v2::reader::Reader). /// /// Layout readers are driven by requesting chunks of data using a given selection masks. pub trait LayoutReaderStream { diff --git a/vortex-layout/src/vtable.rs b/vortex-layout/src/vtable.rs index 969fc02af0b..a1aff45518c 100644 --- a/vortex-layout/src/vtable.rs +++ b/vortex-layout/src/vtable.rs @@ -25,7 +25,7 @@ use crate::children::LayoutChildren; use crate::segments::SegmentId; use crate::segments::SegmentSource; use crate::segments::SegmentSourceRef; -use crate::v2::reader::LayoutReader2Ref; +use crate::v2::reader::ReaderRef; pub trait VTable: 'static + Sized + Send + Sync + Debug { type Layout: 'static + Send + Sync + Clone + Debug + Deref + IntoLayout; @@ -72,7 +72,7 @@ pub trait VTable: 'static + Sized + Send + Sync + Debug { layout: &Self::Layout, segment_source: &SegmentSourceRef, session: &VortexSession, - ) -> VortexResult { + ) -> VortexResult { let _ = (layout, segment_source, session); vortex_bail!("new_reader2 not implemented for this layout") } diff --git a/vortex-scan/src/lib.rs b/vortex-scan/src/lib.rs index ed62775f7b0..52eb0dd47bb 100644 --- a/vortex-scan/src/lib.rs +++ b/vortex-scan/src/lib.rs @@ -26,6 +26,5 @@ pub mod gpu; mod repeated_scan; #[cfg(test)] mod test; -pub mod v2; pub use repeated_scan::RepeatedScan; diff --git a/vortex-scan/src/v2/mod.rs b/vortex-scan/src/v2/mod.rs deleted file mode 100644 index af92f452a33..00000000000 --- a/vortex-scan/src/v2/mod.rs +++ /dev/null @@ -1,5 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright the Vortex contributors - -pub mod reader; -pub mod source; diff --git a/vortex-scan/src/v2/reader.rs b/vortex-scan/src/v2/reader.rs deleted file mode 100644 index 6292933645e..00000000000 --- a/vortex-scan/src/v2/reader.rs +++ /dev/null @@ -1,49 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright the Vortex contributors - -use std::ops::Range; - -use async_trait::async_trait; -use vortex_array::ArrayRef; -use vortex_dtype::DType; -use vortex_error::VortexResult; -use vortex_mask::Mask; - -/// A reader provides an interface for loading data from row-indexed layouts. -/// -/// Unlike a [`super::source::Source`], readers have a concrete row count allowing fixed -/// partitions over a known set of rows. Readers are driven by providing an input stream of -/// array data that can be used to provide arguments to parameterized filter and projection -/// expressions. -#[async_trait] -pub trait Reader: 'static + Send + Sync { - /// Get the data type of the layout being read. - fn dtype(&self) -> &DType; - - /// Returns the number of rows in the reader. - fn row_count(&self) -> u64; - - /// Creates a scan over the given row range of the reader. - async fn scan(&self, row_range: Range) -> VortexResult; -} - -pub type ReaderScanRef = Box; - -/// A scan over a reader, producing output arrays given an input array to parameterize the filter -/// and projection expressions. -#[async_trait] -pub trait ReaderScan { - /// The data type of the returned data. - fn dtype(&self) -> &DType; - - /// The preferred maximum row count for the next batch. - /// - /// Returns [`None`] if there are no more batches. - fn next_batch_size(&mut self) -> Option; - - /// Returns the next batch of data given an input array. - /// - /// The returned batch must have the same number of rows as the [`Mask::true_count`]. - /// The provided mask will have at most [`next_batch_size`] rows. - async fn next_batch(&mut self, mask: Mask) -> VortexResult; -} diff --git a/vortex-scan/src/v2/source.rs b/vortex-scan/src/v2/source.rs deleted file mode 100644 index bfd7d6ee36e..00000000000 --- a/vortex-scan/src/v2/source.rs +++ /dev/null @@ -1,98 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright the Vortex contributors - -use std::any::Any; -use std::sync::Arc; - -use async_trait::async_trait; -use futures::stream::BoxStream; -use vortex_array::expr::Expression; -use vortex_array::stream::SendableArrayStream; -use vortex_dtype::DType; -use vortex_error::VortexResult; - -/// Create a Vortex source from serialized configuration. -/// -/// Providers can be registered with Vortex under a specific -#[async_trait(?Send)] -pub trait SourceProvider: 'static { - /// URI schemes handled by this source provider. - /// - /// TODO(ngates): this might not be the right way to plugin sources. - fn schemes(&self) -> &[&str]; - - /// Initialize a new source. - async fn init_source(&self, uri: String) -> VortexResult; - - /// Serialize a source split to bytes. - async fn serialize_split(&self, split: &dyn Split) -> VortexResult>; - - /// Deserialize a source split from bytes. - async fn deserialize_split(&self, data: &[u8]) -> VortexResult; -} - -/// A reference-counted source. -pub type SourceRef = Arc; - -/// A source represents a streamable dataset that can be scanned with projection and filter -/// expressions. Each scan produces splits that can be executed in parallel to read data. -/// Each split can be serialized for remote execution. -#[async_trait] -pub trait Source: 'static + Send + Sync { - /// Returns the dtype of the source. - fn dtype(&self) -> &DType; - - /// Returns an estimate of the row count of the source. - fn row_count_estimate(&self) -> Estimate; - - /// Returns a scan over the source. - async fn scan(&self, scan_request: ScanRequest) -> VortexResult; -} - -#[derive(Debug, Clone, Default)] -pub struct ScanRequest { - pub projection: Option, - pub filter: Option, - pub limit: Option, -} - -pub type SourceScanRef = Box; - -#[async_trait] -pub trait SourceScan: 'static + Send + Sync { - /// The returned dtype of the scan. - fn dtype(&self) -> &DType; - - /// An estimate of the remaining splits. - fn remaining_splits_estimate(&self) -> Estimate; - - /// Returns the next batch of splits to be processed. - /// - /// This should not return _more_ than the max_batch_size splits, but may return fewer. - async fn next_splits(&mut self, max_splits: usize) -> VortexResult>; -} - -pub type SplitStream = BoxStream<'static, VortexResult>; -pub type SplitRef = Arc; - -pub trait Split: 'static + Send + Sync { - /// Downcast the split to a concrete type. - fn as_any(&self) -> &dyn Any; - - /// Executes the split. - fn execute(&self) -> VortexResult; - - /// Returns an estimate of the row count for this split. - fn row_count_estimate(&self) -> Estimate; - - /// Returns an estimate of the byte size for this split. - fn byte_size_estimate(&self) -> Estimate; -} - -#[derive(Default)] -pub enum Estimate { - Exact(T), - UpperBound(T), - #[default] - Unknown, -} From 2b5cb33a740b9a0deebe00a654831b8a220ec71a Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 14:29:21 +0000 Subject: [PATCH 07/21] Scan API Signed-off-by: Nicholas Gates --- vortex-layout/src/v2/mod.rs | 1 - vortex-layout/src/v2/reader.rs | 31 +++++++++--------- vortex-layout/src/v2/readers/chunked.rs | 22 +++++-------- vortex-layout/src/v2/readers/flat.rs | 21 ++++--------- vortex-layout/src/v2/readers/scalar_fn.rs | 22 +++++-------- vortex-layout/src/v2/readers/struct_.rs | 22 +++++-------- vortex-layout/src/v2/stream.rs | 38 ----------------------- 7 files changed, 44 insertions(+), 113 deletions(-) delete mode 100644 vortex-layout/src/v2/stream.rs diff --git a/vortex-layout/src/v2/mod.rs b/vortex-layout/src/v2/mod.rs index fd8a30068f6..48e8c801039 100644 --- a/vortex-layout/src/v2/mod.rs +++ b/vortex-layout/src/v2/mod.rs @@ -4,4 +4,3 @@ mod optimizer; pub mod reader; pub mod readers; -pub mod stream; diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index e8b56ceedb2..6b6585a31f3 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -4,7 +4,7 @@ use std::ops::Range; use std::sync::Arc; -use async_trait::async_trait; +use futures::future::BoxFuture; use vortex_array::ArrayRef; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -41,26 +41,29 @@ pub trait Reader: 'static + Send + Sync { } /// Creates a scan over the given row range of the reader. - fn scan(&self, row_range: Range) -> VortexResult; + fn execute(&self, row_range: Range) -> VortexResult; } -pub type ReaderScanRef = Box; +pub type ReaderStreamRef = Box; -/// A scan over a reader, producing output arrays given an input array to parameterize the filter -/// and projection expressions. -#[async_trait] -pub trait ReaderScan { +pub trait ReaderStream: 'static + Send + Sync { /// The data type of the returned data. fn dtype(&self) -> &DType; - /// The preferred maximum row count for the next batch. + /// The preferred maximum row count for the next chunk. /// - /// Returns [`None`] if there are no more batches. - fn next_batch_size(&mut self) -> Option; + /// Returns [`None`] if there are no more chunks. + fn next_chunk_len(&self) -> Option; - /// Returns the next batch of data given an input array. + /// Returns the next chunk of data given an input array. /// - /// The returned batch must have the same number of rows as the [`Mask::true_count`]. - /// The provided mask will have at most [`next_batch_size`] rows. - async fn next_batch(&mut self, mask: Mask) -> VortexResult; + /// The returned chunk must have the same number of rows as the [`Mask::true_count`]. + /// The provided mask will have at most [`next_chunk_len`] rows. + /// + /// The returned future has a `'static` lifetime allowing the calling to drive the stream + /// arbitrarily far without awaiting any data. + fn next_chunk( + &mut self, + mask: &Mask, + ) -> VortexResult>>; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 0350a3f38fd..e4e5ca7088d 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -17,8 +17,8 @@ use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; -use crate::v2::stream::LayoutReaderStream; -use crate::v2::stream::SendableLayoutReaderStream; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; pub struct ChunkedReader2 { row_count: u64, @@ -27,23 +27,15 @@ pub struct ChunkedReader2 { } impl Reader for ChunkedReader2 { - fn row_count(&self) -> u64 { - self.row_count - } - fn dtype(&self) -> &DType { &self.dtype } - fn nchildren(&self) -> usize { - self.chunks.len() - } - - fn child(&self, idx: usize) -> &ReaderRef { - &self.chunks[idx] + fn row_count(&self) -> u64 { + self.row_count } - fn execute(&self, row_range: Range) -> VortexResult { + fn execute(&self, row_range: Range) -> VortexResult { let mut remaining_start = row_range.start; let mut remaining_end = row_range.end; let mut streams = Vec::new(); @@ -84,10 +76,10 @@ impl Reader for ChunkedReader2 { struct ChunkedReaderStream { dtype: DType, - chunks: Vec, + chunks: Vec, } -impl LayoutReaderStream for ChunkedReaderStream { +impl ReaderStream for ChunkedReaderStream { fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 57ce3baeadb..b4f0bf0c3e7 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -14,9 +14,8 @@ use vortex_mask::Mask; use crate::layouts::SharedArrayFuture; use crate::v2::reader::Reader; -use crate::v2::reader::ReaderRef; -use crate::v2::stream::LayoutReaderStream; -use crate::v2::stream::SendableLayoutReaderStream; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; pub struct FlatReader2 { len: usize, @@ -25,23 +24,15 @@ pub struct FlatReader2 { } impl Reader for FlatReader2 { - fn row_count(&self) -> u64 { - self.len as u64 - } - fn dtype(&self) -> &DType { &self.dtype } - fn nchildren(&self) -> usize { - 0 - } - - fn child(&self, _idx: usize) -> &ReaderRef { - unreachable!() + fn row_count(&self) -> u64 { + self.len as u64 } - fn execute(&self, row_range: Range) -> VortexResult { + fn execute(&self, row_range: Range) -> VortexResult { // We need to share the same array future let array_fut = self.array_fut.clone(); @@ -75,7 +66,7 @@ struct FlatLayoutReaderStream { remaining: usize, } -impl LayoutReaderStream for FlatLayoutReaderStream { +impl ReaderStream for FlatLayoutReaderStream { fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 3bf8e679924..009cfea7cf9 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -16,8 +16,8 @@ use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; -use crate::v2::stream::LayoutReaderStream; -use crate::v2::stream::SendableLayoutReaderStream; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; /// A [`Reader] for applying a scalar function to another layout. pub struct ScalarFnReader { @@ -28,23 +28,15 @@ pub struct ScalarFnReader { } impl Reader for ScalarFnReader { - fn row_count(&self) -> u64 { - self.row_count - } - fn dtype(&self) -> &DType { &self.dtype } - fn nchildren(&self) -> usize { - self.children.len() - } - - fn child(&self, idx: usize) -> &ReaderRef { - &self.children[idx] + fn row_count(&self) -> u64 { + self.row_count } - fn execute(&self, row_range: Range) -> VortexResult { + fn execute(&self, row_range: Range) -> VortexResult { let input_streams = self .children .iter() @@ -62,10 +54,10 @@ impl Reader for ScalarFnReader { struct ScalarFnArrayStream { dtype: DType, scalar_fn: ScalarFn, - input_streams: Vec, + input_streams: Vec, } -impl LayoutReaderStream for ScalarFnArrayStream { +impl ReaderStream for ScalarFnArrayStream { fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index bd1822fcbfa..97ddafa13e8 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -16,8 +16,8 @@ use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; -use crate::v2::stream::LayoutReaderStream; -use crate::v2::stream::SendableLayoutReaderStream; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; pub struct StructReader2 { row_count: u64, @@ -27,23 +27,15 @@ pub struct StructReader2 { } impl Reader for StructReader2 { - fn row_count(&self) -> u64 { - self.row_count - } - fn dtype(&self) -> &DType { &self.dtype } - fn nchildren(&self) -> usize { - self.fields.len() - } - - fn child(&self, idx: usize) -> &ReaderRef { - &self.fields[idx] + fn row_count(&self) -> u64 { + self.row_count } - fn execute(&self, row_range: Range) -> VortexResult { + fn execute(&self, row_range: Range) -> VortexResult { let field_streams = self .fields .iter() @@ -59,10 +51,10 @@ impl Reader for StructReader2 { struct StructReaderStream { dtype: DType, - fields: Vec, + fields: Vec, } -impl LayoutReaderStream for StructReaderStream { +impl ReaderStream for StructReaderStream { fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/stream.rs b/vortex-layout/src/v2/stream.rs deleted file mode 100644 index b2cbf6c7cfa..00000000000 --- a/vortex-layout/src/v2/stream.rs +++ /dev/null @@ -1,38 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright the Vortex contributors - -use futures::future::BoxFuture; -use vortex_array::ArrayRef; -use vortex_dtype::DType; -use vortex_error::VortexResult; -use vortex_mask::Mask; - -pub type SendableLayoutReaderStream = Box; - -/// A stream of data produced by a [`Reader`](crate::v2::reader::Reader). -/// -/// Layout readers are driven by requesting chunks of data using a given selection masks. -pub trait LayoutReaderStream { - /// Returns the [`DType`] of the data produced by the stream. - fn dtype(&self) -> &DType; - - /// Returns the length in rows of the next chunk in the stream. - /// - /// Returns [`None`] if the stream has ended. - fn next_chunk_len(&self) -> Option; - - /// Returns the next chunk of data given a selection mask of the requested length. - /// - /// The length of the provided selection mask must be `<=` the size returned from - /// [`LayoutReaderStream::next_chunk_len`]. - /// - /// The length of the returned chunk must be equal to the [`Mask::true_count`] of the selection - /// mask. - /// - /// The returned future has a `'static` lifetime allowing the calling to drive the stream - /// arbitrarily far without awaiting any data. - fn next_chunk( - &mut self, - selection: &Mask, - ) -> VortexResult>>; -} From 479cb8caaabe1f29f68b379fb33a887c6ea4dfd9 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 15:29:03 +0000 Subject: [PATCH 08/21] Scan API Signed-off-by: Nicholas Gates --- vortex-file/src/file.rs | 9 ++++ vortex-file/src/tests.rs | 68 ++++++++++++------------- vortex-layout/src/v2/readers/chunked.rs | 7 ++- vortex-scan/src/lib.rs | 1 + vortex-scan/src/v2/mod.rs | 4 ++ vortex-scan/src/v2/scan.rs | 68 +++++++++++++++++++++++++ 6 files changed, 121 insertions(+), 36 deletions(-) create mode 100644 vortex-scan/src/v2/mod.rs create mode 100644 vortex-scan/src/v2/scan.rs diff --git a/vortex-file/src/file.rs b/vortex-file/src/file.rs index b2161f042c7..10a30cdf0f2 100644 --- a/vortex-file/src/file.rs +++ b/vortex-file/src/file.rs @@ -27,6 +27,7 @@ use vortex_layout::segments::SegmentSource; use vortex_metrics::VortexMetrics; use vortex_scan::ScanBuilder; use vortex_scan::SplitBy; +use vortex_scan::v2::scan::ScanBuilder2; use vortex_session::VortexSession; use vortex_utils::aliases::hash_map::HashMap; @@ -103,6 +104,14 @@ impl VortexFile { ) } + pub fn scan2(&self) -> VortexResult { + let reader_ref = self + .footer + .layout() + .new_reader2(&self.segment_source, &self.session)?; + Ok(ScanBuilder2::new(reader_ref, self.session.clone())) + } + #[cfg(gpu_unstable)] pub fn gpu_scan( &self, diff --git a/vortex-file/src/tests.rs b/vortex-file/src/tests.rs index 2bcc0b73739..47020ac32f9 100644 --- a/vortex-file/src/tests.rs +++ b/vortex-file/src/tests.rs @@ -61,7 +61,7 @@ use vortex_io::session::RuntimeSession; use vortex_layout::session::LayoutSession; use vortex_metrics::VortexMetrics; use vortex_scalar::Scalar; -use vortex_scan::ScanBuilder; +use vortex_scan::v2::scan::ScanBuilder2; use vortex_session::VortexSession; use crate::OpenOptionsSessionExt; @@ -118,7 +118,7 @@ async fn test_read_simple() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .into_array_stream() .unwrap(); @@ -198,7 +198,7 @@ async fn test_round_trip_many_types() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .into_array_stream() .unwrap() @@ -284,7 +284,7 @@ async fn test_read_projection() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let array = file - .scan() + .scan2() .unwrap() .with_projection(select(["strings"], root())) .into_array_stream() @@ -306,7 +306,7 @@ async fn test_read_projection() { assert_arrays_eq!(actual.as_ref(), expected.as_ref()); let array = file - .scan() + .scan2() .unwrap() .with_projection(select(["numbers"], root())) .into_array_stream() @@ -355,7 +355,7 @@ async fn unequal_batches() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .into_array_stream() .unwrap(); @@ -415,7 +415,7 @@ async fn write_chunked() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .into_array_stream() .unwrap(); @@ -445,7 +445,7 @@ async fn test_empty_varbin_array_roundtrip() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let result = file - .scan() + .scan2() .unwrap() .into_array_stream() .unwrap() @@ -475,7 +475,7 @@ async fn issue_5385_filter_casted_column() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .with_filter(eq( cast( @@ -525,7 +525,7 @@ async fn filter_string() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .with_filter(eq(get_item("name", root()), lit("Joseph"))) .into_array_stream() @@ -574,7 +574,7 @@ async fn filter_or() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .with_filter(or( eq(get_item("name", root()), lit("Angela")), @@ -634,7 +634,7 @@ async fn filter_and() { .open_options() .open_buffer(buf) .unwrap() - .scan() + .scan2() .unwrap() .with_filter(and( gt(get_item("age", root()), lit(21)), @@ -688,7 +688,7 @@ async fn test_with_indices_simple() { // test no indices let actual_kept_array = file - .scan() + .scan2() .unwrap() .with_row_indices(Buffer::::empty()) .into_array_stream() @@ -704,7 +704,7 @@ async fn test_with_indices_simple() { let kept_indices = [0_u64, 3, 99, 100, 101, 399, 400, 401, 499]; let actual_kept_array = file - .scan() + .scan2() .unwrap() .with_row_indices(Buffer::from_iter(kept_indices)) .into_array_stream() @@ -724,7 +724,7 @@ async fn test_with_indices_simple() { // test all indices let actual_array = file - .scan() + .scan2() .unwrap() .with_row_indices((0u64..500).collect::>()) .into_array_stream() @@ -767,7 +767,7 @@ async fn test_with_indices_on_two_columns() { let kept_indices = [0_u64, 3, 7]; let array = file - .scan() + .scan2() .unwrap() .with_row_indices(Buffer::from_iter(kept_indices)) .into_array_stream() @@ -822,7 +822,7 @@ async fn test_with_indices_and_with_row_filter_simple() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let actual_kept_array = file - .scan() + .scan2() .unwrap() .with_filter(gt(get_item("numbers", root()), lit(50_i16))) .with_row_indices(Buffer::empty()) @@ -839,7 +839,7 @@ async fn test_with_indices_and_with_row_filter_simple() { let kept_indices = [0u64, 3, 99, 100, 101, 399, 400, 401, 499]; let actual_kept_array = file - .scan() + .scan2() .unwrap() .with_filter(gt(get_item("numbers", root()), lit(50_i16))) .with_row_indices(Buffer::from_iter(kept_indices)) @@ -862,7 +862,7 @@ async fn test_with_indices_and_with_row_filter_simple() { // test all indices let actual_array = file - .scan() + .scan2() .unwrap() .with_filter(gt(get_item("numbers", root()), lit(50_i16))) .with_row_indices((0..500).collect::>()) @@ -925,7 +925,7 @@ async fn filter_string_chunked() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let actual_array = file - .scan() + .scan2() .unwrap() .with_filter(eq(get_item("name", root()), lit("Joseph"))) .into_array_stream() @@ -1013,7 +1013,7 @@ async fn test_pruning_with_or() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let actual_array = file - .scan() + .scan2() .unwrap() .with_filter(or( lt_eq(get_item("letter", root()), lit("J")), @@ -1086,7 +1086,7 @@ async fn test_repeated_projection() { let file = SESSION.open_options().open_buffer(buf).unwrap(); let actual = file - .scan() + .scan2() .unwrap() .with_projection(select(["strings", "strings"], root())) .into_array_stream() @@ -1126,7 +1126,7 @@ async fn chunked_file() -> VortexResult { #[tokio::test] async fn basic_file_roundtrip() -> VortexResult<()> { let vxf = chunked_file().await?; - let result = vxf.scan()?.into_array_stream()?.read_all().await?; + let result = vxf.scan2()?.into_array_stream()?.read_all().await?; let expected = buffer![0i32, 1, 2, 3, 4, 5, 6, 7, 8].into_array(); assert_arrays_eq!(result.as_ref(), expected.as_ref()); @@ -1170,7 +1170,7 @@ async fn file_excluding_dtype() -> VortexResult<()> { async fn file_take() -> VortexResult<()> { let vxf = chunked_file().await?; let result = vxf - .scan()? + .scan2()? .with_row_indices(buffer![0, 1, 8]) .into_array_stream()? .read_all() @@ -1208,7 +1208,7 @@ async fn write_nullable_top_level_struct() { async fn round_trip( array: &dyn Array, - f: impl Fn(ScanBuilder) -> VortexResult>, + f: impl Fn(ScanBuilder2) -> VortexResult, ) -> VortexResult { let mut writer = vec![]; SESSION @@ -1225,7 +1225,7 @@ async fn round_trip( assert_eq!(vxf.dtype(), array.dtype()); assert_eq!(vxf.row_count(), array.len() as u64); - f(vxf.scan()?)?.into_array_stream()?.read_all().await + f(vxf.scan2()?)?.into_array_stream()?.read_all().await } #[tokio::test] @@ -1304,7 +1304,7 @@ async fn test_into_tokio_array_stream() -> VortexResult<()> { .await?; let file = SESSION.open_options().open_buffer(buf)?; - let stream = file.scan().unwrap().into_array_stream()?; + let stream = file.scan2().unwrap().into_array_stream()?; let array = stream.read_all().await?; assert_eq!(array.len(), 8); @@ -1326,7 +1326,7 @@ async fn test_array_stream_no_double_dict_encode() -> VortexResult<()> { .write(&mut buf, array.to_array_stream()) .await?; let file = SESSION.open_options().open_buffer(buf)?; - let read_array = file.scan()?.into_array_stream()?.read_all().await?; + let read_array = file.scan2()?.into_array_stream()?.read_all().await?; let dict = read_array .as_opt::() @@ -1354,7 +1354,7 @@ async fn test_writer_basic_push() -> VortexResult<()> { assert_eq!(summary.row_count(), 4); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 4); assert_eq!(result.dtype(), &dtype); @@ -1384,7 +1384,7 @@ async fn test_writer_multiple_pushes() -> VortexResult<()> { assert_eq!(summary.row_count(), 9); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 9); let numbers = result.to_struct().field_by_name("numbers")?.clone(); @@ -1415,7 +1415,7 @@ async fn test_writer_push_stream() -> VortexResult<()> { assert_eq!(summary.row_count(), 6); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 6); let numbers = result.to_struct().field_by_name("numbers")?.clone(); @@ -1476,7 +1476,7 @@ async fn test_writer_empty_chunks() -> VortexResult<()> { assert_eq!(summary.row_count(), 2); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 2); let numbers = result.to_struct().field_by_name("numbers")?.clone(); @@ -1511,7 +1511,7 @@ async fn test_writer_mixed_push_and_stream() -> VortexResult<()> { assert_eq!(summary.row_count(), 6); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 6); let numbers = result.to_struct().field_by_name("numbers")?.clone(); @@ -1548,7 +1548,7 @@ async fn test_writer_with_complex_types() -> VortexResult<()> { assert_eq!(footer.row_count(), 3); let file = SESSION.open_options().open_buffer(buf)?; - let result = file.scan()?.into_array_stream()?.read_all().await?; + let result = file.scan2()?.into_array_stream()?.read_all().await?; assert_eq!(result.len(), 3); assert_eq!(result.dtype(), &dtype); diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index e4e5ca7088d..244421dca36 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -10,6 +10,7 @@ use vortex_array::ArrayRef; use vortex_array::IntoArray; use vortex_array::arrays::ChunkedArray; use vortex_dtype::DType; +use vortex_error::VortexExpect; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_error::vortex_err; @@ -135,11 +136,13 @@ impl ReaderStream for ChunkedReaderStream { // Remove any chunks that are already exhausted loop { + if self.chunks.is_empty() { + vortex_bail!("Early termination of chunked layout"); + } if self.chunks[0].next_chunk_len().is_none() { self.chunks.remove(0); - } else { - break; } + next_len = self.chunks[0].next_chunk_len().vortex_expect("non-none"); } } diff --git a/vortex-scan/src/lib.rs b/vortex-scan/src/lib.rs index 52eb0dd47bb..ed62775f7b0 100644 --- a/vortex-scan/src/lib.rs +++ b/vortex-scan/src/lib.rs @@ -26,5 +26,6 @@ pub mod gpu; mod repeated_scan; #[cfg(test)] mod test; +pub mod v2; pub use repeated_scan::RepeatedScan; diff --git a/vortex-scan/src/v2/mod.rs b/vortex-scan/src/v2/mod.rs new file mode 100644 index 00000000000..4f05171404d --- /dev/null +++ b/vortex-scan/src/v2/mod.rs @@ -0,0 +1,4 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +pub mod scan; diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs new file mode 100644 index 00000000000..c806cf15e12 --- /dev/null +++ b/vortex-scan/src/v2/scan.rs @@ -0,0 +1,68 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_array::expr::Expression; +use vortex_array::expr::root; +use vortex_array::stream::SendableArrayStream; +use vortex_buffer::Buffer; +use vortex_error::VortexResult; +use vortex_layout::v2::reader::ReaderRef; +use vortex_session::VortexSession; + +use crate::Selection; + +pub struct ScanBuilder2 { + reader: ReaderRef, + projection: Expression, + filter: Option, + limit: Option, + row_selection: Selection, + session: VortexSession, +} + +impl ScanBuilder2 { + pub fn new(reader: ReaderRef, session: VortexSession) -> Self { + Self { + reader, + projection: root(), + filter: None, + limit: None, + row_selection: Selection::All, + session, + } + } + + pub fn with_filter(mut self, filter: Expression) -> Self { + self.filter = Some(filter); + self + } + + pub fn with_some_filter(mut self, filter: Option) -> Self { + self.filter = filter; + self + } + + pub fn with_projection(mut self, projection: Expression) -> Self { + self.projection = projection; + self + } + + pub fn with_limit(mut self, limit: u64) -> Self { + self.limit = Some(limit); + self + } + + pub fn with_row_selection(mut self, row_selection: Selection) -> Self { + self.row_selection = row_selection; + self + } + + pub fn with_row_indices(mut self, row_indices: Buffer) -> Self { + self.row_selection = Selection::IncludeByIndex(row_indices); + self + } + + pub fn into_array_stream(self) -> VortexResult { + todo!() + } +} From 9646a53274987ac1492f6c83661b3105eb579b72 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 15:36:12 +0000 Subject: [PATCH 09/21] Scan API Signed-off-by: Nicholas Gates --- vortex-scan/src/v2/scan.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index c806cf15e12..946514096bb 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -66,3 +66,5 @@ impl ScanBuilder2 { todo!() } } + +struct Scan {} From c0645ca1bfacd565f1b8b5fe4260aa1231357f88 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 14 Jan 2026 22:16:37 +0000 Subject: [PATCH 10/21] Scan API Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + vortex-array/src/expr/exprs/mod.rs | 2 + vortex-array/src/expr/exprs/stats.rs | 84 ++++++++++++++++ vortex-array/src/expr/stats/mod.rs | 2 +- vortex-array/src/expr/vtable.rs | 23 ++++- vortex-layout/src/v2/expression.rs | 52 ++++++++++ vortex-layout/src/v2/expressions/falsify.rs | 95 ++++++++++++++++++ vortex-layout/src/v2/expressions/mod.rs | 4 + vortex-layout/src/v2/matcher.rs | 16 +++ vortex-layout/src/v2/mod.rs | 3 + vortex-layout/src/v2/optimizer.rs | 7 +- vortex-layout/src/v2/reader.rs | 4 + vortex-layout/src/v2/readers/chunked.rs | 9 +- vortex-layout/src/v2/readers/constant.rs | 79 +++++++++++++++ vortex-layout/src/v2/readers/flat.rs | 9 +- vortex-layout/src/v2/readers/mod.rs | 2 + vortex-layout/src/v2/readers/scalar_fn.rs | 52 ++++++++++ vortex-layout/src/v2/readers/struct_.rs | 9 +- vortex-layout/src/v2/readers/zoned.rs | 104 ++++++++++++++++++++ vortex-scan/src/v2/scan.rs | 68 ++++++++++++- 20 files changed, 611 insertions(+), 14 deletions(-) create mode 100644 vortex-array/src/expr/exprs/stats.rs create mode 100644 vortex-layout/src/v2/expression.rs create mode 100644 vortex-layout/src/v2/expressions/falsify.rs create mode 100644 vortex-layout/src/v2/expressions/mod.rs create mode 100644 vortex-layout/src/v2/matcher.rs create mode 100644 vortex-layout/src/v2/readers/constant.rs create mode 100644 vortex-layout/src/v2/readers/zoned.rs diff --git a/Cargo.lock b/Cargo.lock index 11b61602873..c9462d2acdf 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10683,6 +10683,7 @@ version = "0.1.0" dependencies = [ "arrow-array 57.2.0", "arrow-schema 57.2.0", + "async-trait", "bit-vec 0.8.0", "futures", "itertools 0.14.0", diff --git a/vortex-array/src/expr/exprs/mod.rs b/vortex-array/src/expr/exprs/mod.rs index c606b53f5a0..c8a5776a131 100644 --- a/vortex-array/src/expr/exprs/mod.rs +++ b/vortex-array/src/expr/exprs/mod.rs @@ -17,6 +17,7 @@ pub(crate) mod operators; pub(crate) mod pack; pub(crate) mod root; pub(crate) mod select; +pub(crate) mod stats; pub use between::*; pub use binary::*; @@ -34,3 +35,4 @@ pub use operators::*; pub use pack::*; pub use root::*; pub use select::*; +pub use stats::*; diff --git a/vortex-array/src/expr/exprs/stats.rs b/vortex-array/src/expr/exprs/stats.rs new file mode 100644 index 00000000000..2afefa98232 --- /dev/null +++ b/vortex-array/src/expr/exprs/stats.rs @@ -0,0 +1,84 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_err; +use vortex_vector::Datum; +use vortex_vector::Scalar; + +use crate::Array; +use crate::ArrayRef; +use crate::IntoArray; +use crate::arrays::ConstantArray; +use crate::expr::Arity; +use crate::expr::ChildName; +use crate::expr::ExecutionArgs; +use crate::expr::ExprId; +use crate::expr::Expression; +use crate::expr::SimplifyCtx; +use crate::expr::VTable; +use crate::expr::VTableExt; +use crate::expr::stats::Stat; + +/// Creates a new expression that returns a minimum bound of its input. +pub fn statistic(stat: Stat, child: Expression) -> Expression { + Statistic.new_expr(stat, vec![child]) +} + +pub struct Statistic; + +impl VTable for Statistic { + type Options = Stat; + + fn id(&self) -> ExprId { + ExprId::from("statistic") + } + + fn arity(&self, _options: &Self::Options) -> Arity { + Arity::Exact(1) + } + + fn child_name(&self, _options: &Self::Options, _child_idx: usize) -> ChildName { + ChildName::from("input") + } + + fn return_dtype(&self, stat: &Stat, arg_dtypes: &[DType]) -> VortexResult { + stat.dtype(&arg_dtypes[0]) + .ok_or_else(|| { + vortex_err!( + "statistic {:?} not supported for dtype {:?}", + stat, + arg_dtypes[0] + ) + }) + // We make all statistics types nullable in case there is no reduction rule to handle + // the statistic expression. + .map(|dt| dt.as_nullable()) + } + + fn evaluate( + &self, + _stat: &Stat, + expr: &Expression, + scope: &ArrayRef, + ) -> VortexResult { + let return_dtype = expr.return_dtype(scope.dtype())?; + Ok(ConstantArray::new(vortex_scalar::Scalar::null(return_dtype), scope.len()).into_array()) + } + + fn execute(&self, _stat: &Stat, args: ExecutionArgs) -> VortexResult { + Ok(Datum::Scalar(Scalar::null(&args.return_dtype))) + } + + fn simplify( + &self, + _options: &Self::Options, + _expr: &Expression, + _ctx: &dyn SimplifyCtx, + ) -> VortexResult> { + // FIXME(ngates): we really want to implement a reduction rule for all arrays? But it's an array. + // And it's a reduction rule. How do we do this without reduce_parent on everything..? + Ok(None) + } +} diff --git a/vortex-array/src/expr/stats/mod.rs b/vortex-array/src/expr/stats/mod.rs index cba33e2743c..e01f807e21d 100644 --- a/vortex-array/src/expr/stats/mod.rs +++ b/vortex-array/src/expr/stats/mod.rs @@ -216,7 +216,7 @@ impl Stat { }) } - pub fn name(&self) -> &str { + pub const fn name(&self) -> &'static str { match self { Self::IsConstant => "is_constant", Self::IsSorted => "is_sorted", diff --git a/vortex-array/src/expr/vtable.rs b/vortex-array/src/expr/vtable.rs index 3e0efafb6ae..fc77fb4857f 100644 --- a/vortex-array/src/expr/vtable.rs +++ b/vortex-array/src/expr/vtable.rs @@ -72,7 +72,20 @@ pub trait VTable: 'static + Sized + Send + Sync { options: &Self::Options, expr: &Expression, f: &mut Formatter<'_>, - ) -> fmt::Result; + ) -> fmt::Result { + write!(f, "{}(", expr.id())?; + for (i, child) in expr.children().iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + child.fmt_sql(f)?; + } + let options = format!("{}", options); + if !options.is_empty() { + write!(f, ", options={}", options)?; + } + write!(f, ")") + } /// Compute the return [`DType`] of the expression if evaluated over the given input types. fn return_dtype(&self, options: &Self::Options, arg_dtypes: &[DType]) -> VortexResult; @@ -143,6 +156,14 @@ pub trait VTable: 'static + Sized + Send + Sync { Ok(None) } + /// Falsify the expression, returning a new expression that is true whenever the original + /// expression is guaranteed to be false via stats. + fn falsify(&self, options: &Self::Options, expr: &Expression) -> Option { + _ = options; + _ = expr; + None + } + /// See [`Expression::stat_falsification`]. fn stat_falsification( &self, diff --git a/vortex-layout/src/v2/expression.rs b/vortex-layout/src/v2/expression.rs new file mode 100644 index 00000000000..76f639f987f --- /dev/null +++ b/vortex-layout/src/v2/expression.rs @@ -0,0 +1,52 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::sync::Arc; + +use itertools::Itertools; +use vortex_array::expr::Expression; +use vortex_array::expr::Literal; +use vortex_array::expr::Root; +use vortex_error::VortexResult; + +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; +use crate::v2::readers::constant::ConstantReader; +use crate::v2::readers::scalar_fn::ScalarFnReader; + +impl dyn Reader + '_ { + /// Apply the expression to this reader, producing a new reader in constant time. + pub fn apply(self: Arc, expr: &Expression) -> VortexResult { + // If the expression is a root, return self. + if expr.is::() { + return Ok(self); + } + + // Manually convert literals to ConstantArray. + if let Some(scalar) = expr.as_opt::() { + return Ok(Arc::new(ConstantReader::new( + scalar.clone(), + self.row_count(), + ))); + } + + let row_count = self.row_count(); + + // Otherwise, collect the child readers. + let children: Vec<_> = expr + .children() + .iter() + .map(|e| self.clone().apply(e)) + .try_collect()?; + + // And wrap the scalar function up in an array. + let reader: ReaderRef = Arc::new(ScalarFnReader::try_new( + expr.scalar_fn().clone(), + children, + row_count, + )?); + + // Optimize the resulting reader. + reader.optimize() + } +} diff --git a/vortex-layout/src/v2/expressions/falsify.rs b/vortex-layout/src/v2/expressions/falsify.rs new file mode 100644 index 00000000000..56cfcf8a77c --- /dev/null +++ b/vortex-layout/src/v2/expressions/falsify.rs @@ -0,0 +1,95 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::fmt::Display; +use std::fmt::Formatter; + +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::arrays::ConstantArray; +use vortex_array::expr::Arity; +use vortex_array::expr::ChildName; +use vortex_array::expr::ExecutionArgs; +use vortex_array::expr::ExprId; +use vortex_array::expr::Expression; +use vortex_array::expr::VTable; +use vortex_dtype::DType; +use vortex_dtype::Nullability; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; +use vortex_vector::Datum; +use vortex_vector::Scalar; +use vortex_vector::bool::BoolScalar; + +/// An expression that evaluates to true when the predicate is provably false, without evaluating +/// it. +/// +/// Falsify typically reduces to operations over statistics expressions. For example, +/// the expression `falsify(col > 5)` may reduce to `col.max() <= 5`. +/// +/// If a falsify expression cannot be reduced, it evaluates to `false` for all inputs. +pub struct Falsify; + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub struct FalsifyOptions { + predicate: Expression, +} + +impl Display for FalsifyOptions { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "predicate={}", self.predicate) + } +} + +impl VTable for Falsify { + // FIXME(ngates): should the predicate be a child expression, or live like this in the options. + // It's a bit weird? Maybe it makes implementing the optimizer rules a little more fiddly? + // But it's weird to have a child expression that we know is never executed. + type Options = FalsifyOptions; + + fn id(&self) -> ExprId { + ExprId::from("falsify") + } + + fn arity(&self, _options: &Self::Options) -> Arity { + Arity::Exact(0) + } + + fn child_name(&self, _options: &Self::Options, _child_idx: usize) -> ChildName { + ChildName::from("predicate") + } + + fn fmt_sql( + &self, + _options: &Self::Options, + expr: &Expression, + f: &mut Formatter<'_>, + ) -> std::fmt::Result { + write!(f, "falsify(")?; + expr.child(0).fmt_sql(f)?; + write!(f, ")") + } + + fn return_dtype(&self, _options: &Self::Options, arg_dtypes: &[DType]) -> VortexResult { + if !arg_dtypes[0].is_boolean() { + vortex_bail!("falsify() requires a boolean argument"); + } + Ok(DType::Bool(Nullability::NonNullable)) + } + + // NOTE(ngates): do we prefer evaluate or execute semantics??? + fn evaluate( + &self, + _options: &Self::Options, + _expr: &Expression, + scope: &ArrayRef, + ) -> VortexResult { + // Unless falsify has been reduced by another expression, we cannot prove the predicate + // is false. Therefore, we return a constant false array. + Ok(ConstantArray::new(false, scope.len()).into_array()) + } + + fn execute(&self, _options: &Self::Options, _args: ExecutionArgs) -> VortexResult { + Ok(Datum::Scalar(Scalar::Bool(BoolScalar::new(Some(false))))) + } +} diff --git a/vortex-layout/src/v2/expressions/mod.rs b/vortex-layout/src/v2/expressions/mod.rs new file mode 100644 index 00000000000..41b859483a0 --- /dev/null +++ b/vortex-layout/src/v2/expressions/mod.rs @@ -0,0 +1,4 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +mod falsify; diff --git a/vortex-layout/src/v2/matcher.rs b/vortex-layout/src/v2/matcher.rs new file mode 100644 index 00000000000..dcd68f2dc1d --- /dev/null +++ b/vortex-layout/src/v2/matcher.rs @@ -0,0 +1,16 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use vortex_array::expr; + +use crate::v2::reader::Reader; +use crate::v2::readers::scalar_fn::ScalarFnReader; + +impl dyn Reader + '_ { + /// If this reader is a [`ScalarFnReader`], return its scalar function options + pub fn as_scalar_fn(&self) -> Option<&V::Options> { + self.as_any() + .downcast_ref::() + .and_then(|r| r.scalar_fn().as_opt::()) + } +} diff --git a/vortex-layout/src/v2/mod.rs b/vortex-layout/src/v2/mod.rs index 48e8c801039..2fe71dbffb7 100644 --- a/vortex-layout/src/v2/mod.rs +++ b/vortex-layout/src/v2/mod.rs @@ -1,6 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +mod expression; +mod expressions; +mod matcher; mod optimizer; pub mod reader; pub mod readers; diff --git a/vortex-layout/src/v2/optimizer.rs b/vortex-layout/src/v2/optimizer.rs index 5c2686654e6..343179037ac 100644 --- a/vortex-layout/src/v2/optimizer.rs +++ b/vortex-layout/src/v2/optimizer.rs @@ -1,13 +1,16 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::sync::Arc; + use vortex_error::VortexResult; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; impl dyn Reader + '_ { - pub fn optimize(&self) -> VortexResult { - todo!() + pub fn optimize(self: Arc) -> VortexResult { + // TODO(ngates): run the reduce rules + Ok(self) } } diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index 6b6585a31f3..589cbe8b322 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::any::Any; use std::ops::Range; use std::sync::Arc; @@ -19,6 +20,9 @@ pub type ReaderRef = Arc; /// array data that can be used to provide arguments to parameterized filter and projection /// expressions. pub trait Reader: 'static + Send + Sync { + /// Downcast the reader to a concrete type. + fn as_any(&self) -> &dyn Any; + /// Get the data type of the layout being read. fn dtype(&self) -> &DType; diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 244421dca36..7eaa90e388a 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::any::Any; use std::ops::Range; use futures::future::BoxFuture; @@ -21,13 +22,17 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; -pub struct ChunkedReader2 { +pub struct ChunkedReader { row_count: u64, dtype: DType, chunks: Vec, } -impl Reader for ChunkedReader2 { +impl Reader for ChunkedReader { + fn as_any(&self) -> &dyn Any { + self + } + fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs new file mode 100644 index 00000000000..a15676b712b --- /dev/null +++ b/vortex-layout/src/v2/readers/constant.rs @@ -0,0 +1,79 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::any::Any; +use std::ops::Range; + +use futures::future::BoxFuture; +use moka::future::FutureExt; +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::arrays::ConstantArray; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_mask::Mask; +use vortex_scalar::Scalar; + +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; + +pub struct ConstantReader { + scalar: Scalar, + row_count: u64, +} + +impl ConstantReader { + pub fn new(scalar: Scalar, row_count: u64) -> Self { + Self { scalar, row_count } + } +} + +impl Reader for ConstantReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn dtype(&self) -> &DType { + self.scalar.dtype() + } + + fn row_count(&self) -> u64 { + self.row_count + } + + fn execute(&self, row_range: Range) -> VortexResult { + let remaining = row_range.end.saturating_sub(row_range.start); + Ok(Box::new(ConstantReaderStream { + scalar: self.scalar.clone(), + remaining, + })) + } +} + +struct ConstantReaderStream { + scalar: Scalar, + remaining: u64, +} + +impl ReaderStream for ConstantReaderStream { + fn dtype(&self) -> &DType { + self.scalar.dtype() + } + + fn next_chunk_len(&self) -> Option { + if self.remaining == 0 { + None + } else { + Some(usize::try_from(self.remaining).unwrap_or(usize::MAX)) + } + } + + fn next_chunk( + &mut self, + mask: &Mask, + ) -> VortexResult>> { + let array = ConstantArray::new(self.scalar.clone(), mask.true_count()).into_array(); + Ok(async move { Ok(array) }.boxed()) + } +} diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index b4f0bf0c3e7..fefeec326d3 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::any::Any; use std::ops::Range; use futures::future::BoxFuture; @@ -17,13 +18,17 @@ use crate::v2::reader::Reader; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; -pub struct FlatReader2 { +pub struct FlatReader { len: usize, dtype: DType, array_fut: SharedArrayFuture, } -impl Reader for FlatReader2 { +impl Reader for FlatReader { + fn as_any(&self) -> &dyn Any { + self + } + fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 6d020982f62..6d354cb392d 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -2,6 +2,8 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors pub mod chunked; +pub mod constant; pub mod flat; pub mod scalar_fn; pub mod struct_; +mod zoned; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 009cfea7cf9..86e2348760f 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -1,14 +1,19 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::any::Any; use std::ops::Range; +use std::sync::Arc; use futures::future::BoxFuture; use futures::future::try_join_all; use vortex_array::ArrayRef; use vortex_array::IntoArray; use vortex_array::arrays::ScalarFnArray; +use vortex_array::expr::Expression; use vortex_array::expr::ScalarFn; +use vortex_array::expr::VTable; +use vortex_array::expr::VTableExt; use vortex_array::optimizer::ArrayOptimizer; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -27,7 +32,37 @@ pub struct ScalarFnReader { children: Vec, } +impl ScalarFnReader { + pub fn try_new( + scalar_fn: ScalarFn, + children: Vec, + row_count: u64, + ) -> VortexResult { + let dtype = scalar_fn.return_dtype( + &children + .iter() + .map(|c| c.dtype().clone()) + .collect::>(), + )?; + + Ok(Self { + scalar_fn, + dtype, + row_count, + children, + }) + } + + pub fn scalar_fn(&self) -> &ScalarFn { + &self.scalar_fn + } +} + impl Reader for ScalarFnReader { + fn as_any(&self) -> &dyn Any { + self + } + fn dtype(&self) -> &DType { &self.dtype } @@ -90,3 +125,20 @@ impl ReaderStream for ScalarFnArrayStream { })) } } + +pub trait ScalarFnReaderExt: VTable { + /// Creates a [`ScalarFnReader`] applying this scalar function to the given children. + fn new_reader( + &'static self, + options: Self::Options, + children: Vec, + row_count: u64, + ) -> VortexResult { + Ok(Arc::new(ScalarFnReader::try_new( + self.bind(options), + children.into(), + row_count, + )?)) + } +} +impl ScalarFnReaderExt for V {} diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 97ddafa13e8..d290065e16c 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::any::Any; use std::ops::Range; use futures::future::BoxFuture; @@ -19,14 +20,18 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; -pub struct StructReader2 { +pub struct StructReader { row_count: u64, dtype: DType, // TODO(ngates): we should make this lazy? fields: Vec, } -impl Reader for StructReader2 { +impl Reader for StructReader { + fn as_any(&self) -> &dyn Any { + self + } + fn dtype(&self) -> &DType { &self.dtype } diff --git a/vortex-layout/src/v2/readers/zoned.rs b/vortex-layout/src/v2/readers/zoned.rs new file mode 100644 index 00000000000..5fec47880ec --- /dev/null +++ b/vortex-layout/src/v2/readers/zoned.rs @@ -0,0 +1,104 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::any::Any; +use std::ops::Range; +use std::sync::Arc; + +use vortex_array::expr::GetItem; +use vortex_array::expr::Statistic; +use vortex_array::expr::stats::Stat; +use vortex_dtype::DType; +use vortex_dtype::FieldName; +use vortex_error::VortexResult; + +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; +use crate::v2::reader::ReaderStreamRef; +use crate::v2::readers::scalar_fn::ScalarFnReaderExt; + +pub struct ZonedReader { + data: ReaderRef, + zone_map: ReaderRef, + zone_len: usize, + present_stats: Arc<[Stat]>, +} + +impl Reader for ZonedReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn dtype(&self) -> &DType { + self.data.dtype() + } + + fn row_count(&self) -> u64 { + self.data.row_count() + } + + fn try_reduce_parent( + &self, + parent: &ReaderRef, + _child_idx: usize, + ) -> VortexResult> { + if let Some(stat) = parent.as_scalar_fn::() { + if !self.present_stats.contains(stat) { + return Ok(None); + } + + // We know the statistic is present; so we return a new reader that pulls the value + // from the zone map. + let zoned_statistic = GetItem.new_reader( + // FIXME(ngates): construct the field name properly + FieldName::from(stat.name()), + vec![self.zone_map.clone()], + self.zone_map.row_count(), + )?; + + // We now need to explode the zoned_statistic to match the data reader's row count. + // We do this based on the zone map's zone length. + let exploded_statistic = Arc::new(ZonedExpansionReader { + zoned: zoned_statistic, + zone_len: self.zone_len, + row_count: self.data.row_count(), + }); + + return Ok(Some(exploded_statistic)); + } + + Ok(None) + } + + fn execute(&self, row_range: Range) -> VortexResult { + // By default, a zoned reader is just a pass-through. + self.data.execute(row_range) + } +} + +/// A reader that expands zoned statistics to match the data rows. +/// This repeats each row of the zone map `zone_len` times. +/// TODO(ngates): we could use a RunEndReader + Slice to do this? +struct ZonedExpansionReader { + zoned: ReaderRef, + zone_len: usize, + row_count: u64, +} + +impl Reader for ZonedExpansionReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn dtype(&self) -> &DType { + self.zoned.dtype() + } + + fn row_count(&self) -> u64 { + self.row_count + } + + fn execute(&self, row_range: Range) -> VortexResult { + todo!() + } +} diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 946514096bb..834ce905e77 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -1,10 +1,19 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::ops::Range; +use std::pin::Pin; +use std::task::Context; +use std::task::Poll; + +use futures::Stream; +use vortex_array::ArrayRef; use vortex_array::expr::Expression; use vortex_array::expr::root; +use vortex_array::stream::ArrayStream; use vortex_array::stream::SendableArrayStream; use vortex_buffer::Buffer; +use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_layout::v2::reader::ReaderRef; use vortex_session::VortexSession; @@ -16,17 +25,20 @@ pub struct ScanBuilder2 { projection: Expression, filter: Option, limit: Option, - row_selection: Selection, + row_range: Range, + row_selection: Selection, // NOTE: applies to the selected row range. session: VortexSession, } impl ScanBuilder2 { pub fn new(reader: ReaderRef, session: VortexSession) -> Self { + let row_range = 0..reader.row_count(); Self { reader, projection: root(), filter: None, limit: None, + row_range, row_selection: Selection::All, session, } @@ -52,19 +64,67 @@ impl ScanBuilder2 { self } + pub fn with_row_range(mut self, row_range: Range) -> Self { + self.row_range = row_range; + self + } + + /// Sets the row selection to use the given selection (relative to the row range). pub fn with_row_selection(mut self, row_selection: Selection) -> Self { self.row_selection = row_selection; self } + /// Sets the row selection to include only the given row indices (relative to the row range). pub fn with_row_indices(mut self, row_indices: Buffer) -> Self { self.row_selection = Selection::IncludeByIndex(row_indices); self } - pub fn into_array_stream(self) -> VortexResult { - todo!() + pub fn into_array_stream(self) -> VortexResult { + let projection = self.projection.optimize_recursive(self.reader.dtype())?; + let filter = self + .filter + .map(|f| f.optimize_recursive(self.reader.dtype())) + .transpose()?; + + let dtype = projection.return_dtype(self.reader.dtype())?; + + // So we wrap the reader for filtering. + let filter_reader = filter.as_ref().map(|f| self.reader.apply(&f)).transpose()?; + let projection_reader = self.reader.apply(&projection)?; + + // And finally, we wrap the reader for pruning. + let pruning_reader = filter + .as_ref() + .map(|f| { + // TODO(ngates): wrap filter in `falsify` expression. + let f = f.falsify()?; + self.reader.apply(&f) + }) + .transpose()?; + + let reader_stream = self.reader.execute(self.row_range)?; + + Ok(Scan { dtype }) } } -struct Scan {} +struct Scan { + dtype: DType, + stream: SendableArrayStream, +} + +impl ArrayStream for Scan { + fn dtype(&self) -> &DType { + &self.dtype + } +} + +impl Stream for Scan { + type Item = VortexResult; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + todo!() + } +} From 33ea76df1470211f8770e9bf00e9dc3edbc6073d Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 15 Jan 2026 09:37:02 +0000 Subject: [PATCH 11/21] Merge Signed-off-by: Nicholas Gates --- vortex-layout/src/v2/readers/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 6d354cb392d..46d3c793376 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -6,4 +6,4 @@ pub mod constant; pub mod flat; pub mod scalar_fn; pub mod struct_; -mod zoned; +pub mod zoned; From 1074903bd4a94580019ef2a495eaa180e38891d9 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 5 Feb 2026 22:07:10 -0500 Subject: [PATCH 12/21] merge Signed-off-by: Nicholas Gates --- vortex-array/src/expr/exprs/stats.rs | 38 ++++++--- vortex-layout/src/v2/expressions/falsify.rs | 3 - vortex-layout/src/v2/mod.rs | 8 +- vortex-layout/src/v2/reader.rs | 39 +++++---- vortex-layout/src/v2/readers/chunked.rs | 5 +- vortex-layout/src/v2/readers/constant.rs | 47 +++++++++-- vortex-layout/src/v2/readers/flat.rs | 92 +++++++++++++-------- vortex-layout/src/v2/readers/mod.rs | 8 +- vortex-layout/src/v2/readers/scalar_fn.rs | 8 +- vortex-layout/src/v2/readers/struct_.rs | 33 ++++++-- vortex-layout/src/v2/readers/zoned.rs | 66 +++++++-------- vortex-scan/src/v2/scan.rs | 24 ++---- 12 files changed, 222 insertions(+), 149 deletions(-) diff --git a/vortex-array/src/expr/exprs/stats.rs b/vortex-array/src/expr/exprs/stats.rs index 2afefa98232..05763e5ee69 100644 --- a/vortex-array/src/expr/exprs/stats.rs +++ b/vortex-array/src/expr/exprs/stats.rs @@ -3,9 +3,9 @@ use vortex_dtype::DType; use vortex_error::VortexResult; +use vortex_error::vortex_bail; use vortex_error::vortex_err; -use vortex_vector::Datum; -use vortex_vector::Scalar; +use vortex_scalar::Scalar; use crate::Array; use crate::ArrayRef; @@ -19,7 +19,9 @@ use crate::expr::Expression; use crate::expr::SimplifyCtx; use crate::expr::VTable; use crate::expr::VTableExt; +use crate::expr::stats::Precision; use crate::expr::stats::Stat; +use crate::expr::stats::StatsProvider; /// Creates a new expression that returns a minimum bound of its input. pub fn statistic(stat: Stat, child: Expression) -> Expression { @@ -57,18 +59,28 @@ impl VTable for Statistic { .map(|dt| dt.as_nullable()) } - fn evaluate( - &self, - _stat: &Stat, - expr: &Expression, - scope: &ArrayRef, - ) -> VortexResult { - let return_dtype = expr.return_dtype(scope.dtype())?; - Ok(ConstantArray::new(vortex_scalar::Scalar::null(return_dtype), scope.len()).into_array()) - } + fn execute(&self, stat: &Stat, args: ExecutionArgs) -> VortexResult { + // FIXME(ngates): we should implement this as a reduction rule instead? + let Some(stat_dtype) = stat.dtype(args.inputs[0].dtype()) else { + vortex_bail!( + "Statistic {:?} not supported for dtype {:?}", + stat, + args.inputs[0].dtype() + ); + }; - fn execute(&self, _stat: &Stat, args: ExecutionArgs) -> VortexResult { - Ok(Datum::Scalar(Scalar::null(&args.return_dtype))) + Ok(match args.inputs[0].statistics().get(*stat) { + // TODO(ngates): do we care about precision here? Possibly we should configure in the + // options of the expression. + Some(Precision::Exact(v)) => { + // We have an exact value for the statistic; so we return a constant array + // with that value. + ConstantArray::new(v, args.row_count).into_array() + } + None | Some(Precision::Inexact(_)) => { + ConstantArray::new(Scalar::null(stat_dtype), args.row_count).into_array() + } + }) } fn simplify( diff --git a/vortex-layout/src/v2/expressions/falsify.rs b/vortex-layout/src/v2/expressions/falsify.rs index 56cfcf8a77c..23235ca7971 100644 --- a/vortex-layout/src/v2/expressions/falsify.rs +++ b/vortex-layout/src/v2/expressions/falsify.rs @@ -17,9 +17,6 @@ use vortex_dtype::DType; use vortex_dtype::Nullability; use vortex_error::VortexResult; use vortex_error::vortex_bail; -use vortex_vector::Datum; -use vortex_vector::Scalar; -use vortex_vector::bool::BoolScalar; /// An expression that evaluates to true when the predicate is provably false, without evaluating /// it. diff --git a/vortex-layout/src/v2/mod.rs b/vortex-layout/src/v2/mod.rs index 2fe71dbffb7..c3d440a1cc9 100644 --- a/vortex-layout/src/v2/mod.rs +++ b/vortex-layout/src/v2/mod.rs @@ -1,9 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -mod expression; -mod expressions; -mod matcher; -mod optimizer; +// mod expression; +// mod expressions; +// mod matcher; +// mod optimizer; pub mod reader; pub mod readers; diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index 589cbe8b322..efe7958343d 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -7,18 +7,18 @@ use std::sync::Arc; use futures::future::BoxFuture; use vortex_array::ArrayRef; +use vortex_array::MaskFuture; +use vortex_array::expr::Expression; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_mask::Mask; pub type ReaderRef = Arc; /// A reader provides an interface for loading data from row-indexed layouts. /// -/// Unlike a [`super::source::DataSource`], readers have a concrete row count allowing fixed -/// partitions over a known set of rows. Readers are driven by providing an input stream of -/// array data that can be used to provide arguments to parameterized filter and projection -/// expressions. +/// Readers have a concrete row count allowing fixed partitions over a known set of rows. Readers +/// are driven by asking for the next chunk size, before providing a [`MaskFuture`] that resolves +/// into a mask of that length. pub trait Reader: 'static + Send + Sync { /// Downcast the reader to a concrete type. fn as_any(&self) -> &dyn Any; @@ -29,20 +29,9 @@ pub trait Reader: 'static + Send + Sync { /// Returns the number of rows in the reader. fn row_count(&self) -> u64; - /// Reduces the reader, simplifying its internal structure if possible. - fn try_reduce(&self) -> VortexResult> { - Ok(None) - } - - /// Reduce the parent reader if possible, returning a new reader if successful. - fn try_reduce_parent( - &self, - parent: &ReaderRef, - child_idx: usize, - ) -> VortexResult> { - let _ = (parent, child_idx); - Ok(None) - } + /// Apply an expression to the reader, returning a new reader that will execute the expression + /// on top of the current reader. + fn apply(&self, expression: &Expression) -> VortexResult; /// Creates a scan over the given row range of the reader. fn execute(&self, row_range: Range) -> VortexResult; @@ -59,6 +48,13 @@ pub trait ReaderStream: 'static + Send + Sync { /// Returns [`None`] if there are no more chunks. fn next_chunk_len(&self) -> Option; + /// Skip the next `n` rows of the stream. + /// + /// # Panics + /// + /// Panics if `n` is greater than the number of rows remaining in the stream.. + fn skip(&mut self, n: usize); + /// Returns the next chunk of data given an input array. /// /// The returned chunk must have the same number of rows as the [`Mask::true_count`]. @@ -68,6 +64,9 @@ pub trait ReaderStream: 'static + Send + Sync { /// arbitrarily far without awaiting any data. fn next_chunk( &mut self, - mask: &Mask, + mask: MaskFuture, + // TODO(ngates): it would be good to pass an object here that has some lifetime, and this + // object is required to construct segment futures. That way the implementation is forced + // to construct segment futures in this call in order to satisfy the static result lifetime. ) -> VortexResult>>; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 7eaa90e388a..4014820c9b1 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -9,13 +9,14 @@ use futures::future::try_join_all; use moka::future::FutureExt; use vortex_array::ArrayRef; use vortex_array::IntoArray; +use vortex_array::MaskFuture; use vortex_array::arrays::ChunkedArray; +use vortex_array::expr::Expression; use vortex_dtype::DType; use vortex_error::VortexExpect; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_error::vortex_err; -use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; @@ -100,7 +101,7 @@ impl ReaderStream for ChunkedReaderStream { fn next_chunk( &mut self, - selection: &Mask, + selection: MaskFuture, ) -> VortexResult>> { // Remove any chunks that are already exhausted loop { diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index a15676b712b..06825d81288 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -3,29 +3,41 @@ use std::any::Any; use std::ops::Range; +use std::sync::Arc; use futures::future::BoxFuture; use moka::future::FutureExt; use vortex_array::ArrayRef; use vortex_array::IntoArray; +use vortex_array::MaskFuture; use vortex_array::arrays::ConstantArray; +use vortex_array::expr::Expression; +use vortex_array::expr::root; +use vortex_array::expr::transform::replace; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_mask::Mask; use vortex_scalar::Scalar; use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; pub struct ConstantReader { scalar: Scalar, row_count: u64, + + // Optional expression to apply to the constant value. + expression: Option, } impl ConstantReader { pub fn new(scalar: Scalar, row_count: u64) -> Self { - Self { scalar, row_count } + Self { + scalar, + row_count, + expression: None, + } } } @@ -42,11 +54,27 @@ impl Reader for ConstantReader { self.row_count } + fn apply(&self, expression: &Expression) -> VortexResult { + Ok(match &self.expression { + None => Arc::new(Self { + scalar: self.scalar.clone(), + row_count: self.row_count, + expression: Some(expression.clone()), + }), + Some(existing) => Arc::new(Self { + scalar: self.scalar.clone(), + row_count: self.row_count, + expression: Some(replace(existing.clone(), &root(), expression.clone())), + }), + }) + } + fn execute(&self, row_range: Range) -> VortexResult { let remaining = row_range.end.saturating_sub(row_range.start); Ok(Box::new(ConstantReaderStream { scalar: self.scalar.clone(), remaining, + expression: self.expression.clone(), })) } } @@ -54,6 +82,7 @@ impl Reader for ConstantReader { struct ConstantReaderStream { scalar: Scalar, remaining: u64, + expression: Option, } impl ReaderStream for ConstantReaderStream { @@ -71,9 +100,17 @@ impl ReaderStream for ConstantReaderStream { fn next_chunk( &mut self, - mask: &Mask, + mask: MaskFuture, ) -> VortexResult>> { - let array = ConstantArray::new(self.scalar.clone(), mask.true_count()).into_array(); - Ok(async move { Ok(array) }.boxed()) + let expression = self.expression.clone(); + Ok(async move { + let mask = mask.await?; + let mut array = ConstantArray::new(self.scalar.clone(), mask.true_count()).into_array(); + if let Some(e) = expression { + array = array.apply(&e)?; + } + Ok(array) + } + .boxed()) } } diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index fefeec326d3..7d098d7bb8a 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -3,25 +3,37 @@ use std::any::Any; use std::ops::Range; +use std::sync::Arc; +use futures::TryFutureExt; use futures::future::BoxFuture; +use futures::try_join; use moka::future::FutureExt; use vortex_array::ArrayRef; +use vortex_array::MaskFuture; +use vortex_array::expr::Expression; +use vortex_array::expr::root; +use vortex_array::expr::transform::replace; use vortex_dtype::DType; +use vortex_error::VortexError; use vortex_error::VortexResult; use vortex_error::vortex_bail; -use vortex_error::vortex_err; -use vortex_mask::Mask; +use vortex_error::vortex_panic; use crate::layouts::SharedArrayFuture; +use crate::segments::SegmentId; +use crate::segments::SegmentSourceRef; use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; pub struct FlatReader { len: usize, dtype: DType, - array_fut: SharedArrayFuture, + segment_id: SegmentId, + segment_source: SegmentSourceRef, + expression: Option, } impl Reader for FlatReader { @@ -37,30 +49,31 @@ impl Reader for FlatReader { self.len as u64 } - fn execute(&self, row_range: Range) -> VortexResult { - // We need to share the same array future - let array_fut = self.array_fut.clone(); - - let start = usize::try_from(row_range.start).map_err(|_| { - vortex_err!("Row range start {} is too large for usize", row_range.start) - })?; - let end = usize::try_from(row_range.end) - .map_err(|_| vortex_err!("Row range end {} is too large for usize", row_range.end))?; - - if start > self.len || end > self.len || start > end { - vortex_bail!( - "Row range {:?} is out of bounds for array of length {}", - row_range, - self.len - ); - } + fn apply(&self, expression: &Expression) -> VortexResult { + let new_dtype = expression.return_dtype(&self.dtype)?; + Ok(match &self.expression { + None => Arc::new(Self { + len: self.len, + dtype: new_dtype, + segment_id: self.segment_id.clone(), + segment_source: self.segment_source.clone(), + expression: Some(expression.clone()), + }), + Some(e) => { + let new_expr = replace(e.clone(), &root(), expression.clone()); + Arc::new(Self { + len: self.len, + dtype: new_dtype, + segment_id: self.segment_id.clone(), + segment_source: self.segment_source.clone(), + expression: Some(new_expr), + }) + } + }) + } - Ok(Box::new(FlatLayoutReaderStream { - dtype: self.dtype.clone(), - array_fut, - offset: start, - remaining: end - start, - })) + fn execute(&self, _row_range: Range) -> VortexResult { + todo!() } } @@ -84,29 +97,38 @@ impl ReaderStream for FlatLayoutReaderStream { } } + fn skip(&mut self, n: usize) { + if n > self.remaining { + vortex_panic!("Cannot skip {} rows, only {} remaining", n, self.remaining); + } + + self.offset += n; + self.remaining -= n; + } + fn next_chunk( &mut self, - selection: &Mask, + mask: MaskFuture, ) -> VortexResult>> { - if selection.len() > self.remaining { + if mask.len() > self.remaining { vortex_bail!( "Selection mask length {} exceeds remaining rows {}", - selection.len(), + mask.len(), self.remaining ); } let array_fut = self.array_fut.clone(); let offset = self.offset; - let selection = selection.clone(); + let mask = mask.clone(); - self.offset += selection.len(); - self.remaining -= selection.len(); + self.offset += mask.len(); + self.remaining -= mask.len(); Ok(async move { - let array = array_fut.await?; - let sliced_array = array.slice(offset..offset + selection.len()); - let selected_array = sliced_array.filter(selection.clone())?; + let (array, mask) = try_join!(array_fut.map_err(VortexError::from), mask)?; + let sliced_array = array.slice(offset..offset + mask.len())?; + let selected_array = sliced_array.filter(mask)?; Ok(selected_array) } .boxed()) diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 46d3c793376..473a4036874 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -1,9 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -pub mod chunked; -pub mod constant; +// pub mod chunked; +// pub mod constant; pub mod flat; -pub mod scalar_fn; +// pub mod scalar_fn; pub mod struct_; -pub mod zoned; +// pub mod zoned; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 86e2348760f..e421f3624a9 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -9,6 +9,7 @@ use futures::future::BoxFuture; use futures::future::try_join_all; use vortex_array::ArrayRef; use vortex_array::IntoArray; +use vortex_array::MaskFuture; use vortex_array::arrays::ScalarFnArray; use vortex_array::expr::Expression; use vortex_array::expr::ScalarFn; @@ -17,7 +18,6 @@ use vortex_array::expr::VTableExt; use vortex_array::optimizer::ArrayOptimizer; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; @@ -71,6 +71,10 @@ impl Reader for ScalarFnReader { self.row_count } + fn apply(&self, expression: &Expression) -> VortexResult { + todo!() + } + fn execute(&self, row_range: Range) -> VortexResult { let input_streams = self .children @@ -107,7 +111,7 @@ impl ReaderStream for ScalarFnArrayStream { fn next_chunk( &mut self, - selection: &Mask, + selection: MaskFuture, ) -> VortexResult>> { let scalar_fn = self.scalar_fn.clone(); let len = selection.true_count(); diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index d290065e16c..afdeee022bd 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -6,14 +6,16 @@ use std::ops::Range; use futures::future::BoxFuture; use futures::future::try_join_all; +use futures::try_join; use moka::future::FutureExt; use vortex_array::ArrayRef; use vortex_array::IntoArray; +use vortex_array::MaskFuture; use vortex_array::arrays::StructArray; +use vortex_array::expr::Expression; use vortex_array::validity::Validity; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_mask::Mask; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; @@ -40,6 +42,10 @@ impl Reader for StructReader { self.row_count } + fn apply(&self, _expression: &Expression) -> VortexResult { + todo!() + } + fn execute(&self, row_range: Range) -> VortexResult { let field_streams = self .fields @@ -72,25 +78,34 @@ impl ReaderStream for StructReaderStream { .flatten() } + fn skip(&mut self, n: usize) { + for field in &mut self.fields { + field.skip(n); + } + } + fn next_chunk( &mut self, - selection: &Mask, + selection: MaskFuture, ) -> VortexResult>> { let struct_fields = self.dtype.as_struct_fields().clone(); let validity: Validity = self.dtype.nullability().into(); let fields = self .fields .iter_mut() - .map(|s| s.next_chunk(selection)) + .map(|s| s.next_chunk(selection.clone())) .collect::>>()?; - let len = selection.true_count(); Ok(async move { - let fields = try_join_all(fields).await?; - Ok( - StructArray::try_new_with_dtype(fields, struct_fields, len, validity.clone())? - .into_array(), - ) + let fields = try_join_all(fields); + let (fields, mask) = try_join!(fields, selection)?; + Ok(StructArray::try_new_with_dtype( + fields, + struct_fields, + mask.true_count(), + validity.clone(), + )? + .into_array()) } .boxed()) } diff --git a/vortex-layout/src/v2/readers/zoned.rs b/vortex-layout/src/v2/readers/zoned.rs index 5fec47880ec..9ab25fd15c1 100644 --- a/vortex-layout/src/v2/readers/zoned.rs +++ b/vortex-layout/src/v2/readers/zoned.rs @@ -5,17 +5,15 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use vortex_array::expr::GetItem; -use vortex_array::expr::Statistic; +use vortex_array::expr::Expression; use vortex_array::expr::stats::Stat; use vortex_dtype::DType; -use vortex_dtype::FieldName; use vortex_error::VortexResult; +use vortex_error::vortex_bail; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStreamRef; -use crate::v2::readers::scalar_fn::ScalarFnReaderExt; pub struct ZonedReader { data: ReaderRef, @@ -37,37 +35,29 @@ impl Reader for ZonedReader { self.data.row_count() } - fn try_reduce_parent( - &self, - parent: &ReaderRef, - _child_idx: usize, - ) -> VortexResult> { - if let Some(stat) = parent.as_scalar_fn::() { - if !self.present_stats.contains(stat) { - return Ok(None); - } - - // We know the statistic is present; so we return a new reader that pulls the value - // from the zone map. - let zoned_statistic = GetItem.new_reader( - // FIXME(ngates): construct the field name properly - FieldName::from(stat.name()), - vec![self.zone_map.clone()], - self.zone_map.row_count(), - )?; - - // We now need to explode the zoned_statistic to match the data reader's row count. - // We do this based on the zone map's zone length. - let exploded_statistic = Arc::new(ZonedExpansionReader { - zoned: zoned_statistic, - zone_len: self.zone_len, - row_count: self.data.row_count(), - }); - - return Ok(Some(exploded_statistic)); - } - - Ok(None) + fn apply(&self, expression: &Expression) -> VortexResult { + // We need to apply the expression to both the data and the zone map. + let new_data = self.data.apply(expression)?; + let new_zone_map = self.zone_map.apply(expression)?; + + // We also need to update the present stats for the new zone map. + let new_present_stats = self + .present_stats + .iter() + .map(|stat| match stat { + Stat::Min => Stat::Min, + Stat::Max => Stat::Max, + Stat::NullCount => Stat::NullCount, + _ => vortex_bail!("Unsupported stat for zoned reader: {:?}", stat), + }) + .collect(); + + Ok(Arc::new(ZonedReader { + data: new_data, + zone_map: new_zone_map, + zone_len: self.zone_len, + present_stats: Arc::new(new_present_stats), + })) } fn execute(&self, row_range: Range) -> VortexResult { @@ -98,7 +88,11 @@ impl Reader for ZonedExpansionReader { self.row_count } - fn execute(&self, row_range: Range) -> VortexResult { + fn apply(&self, expression: &Expression) -> VortexResult { + todo!() + } + + fn execute(&self, _row_range: Range) -> VortexResult { todo!() } } diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 834ce905e77..8d00f5d7337 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -11,7 +11,6 @@ use vortex_array::ArrayRef; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::stream::ArrayStream; -use vortex_array::stream::SendableArrayStream; use vortex_buffer::Buffer; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -94,25 +93,18 @@ impl ScanBuilder2 { let filter_reader = filter.as_ref().map(|f| self.reader.apply(&f)).transpose()?; let projection_reader = self.reader.apply(&projection)?; - // And finally, we wrap the reader for pruning. - let pruning_reader = filter - .as_ref() - .map(|f| { - // TODO(ngates): wrap filter in `falsify` expression. - let f = f.falsify()?; - self.reader.apply(&f) - }) - .transpose()?; - - let reader_stream = self.reader.execute(self.row_range)?; - - Ok(Scan { dtype }) + Ok(Scan { + dtype, + filter_reader, + projection_reader, + }) } } struct Scan { dtype: DType, - stream: SendableArrayStream, + filter_reader: Option, + projection_reader: ReaderRef, } impl ArrayStream for Scan { @@ -124,7 +116,7 @@ impl ArrayStream for Scan { impl Stream for Scan { type Item = VortexResult; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { todo!() } } From 6599061fe5f573800481e6f5e09527f5e24ca282 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 09:15:40 -0500 Subject: [PATCH 13/21] merge Signed-off-by: Nicholas Gates --- vortex-layout/src/v2/mod.rs | 4 - vortex-layout/src/v2/reader.rs | 3 - vortex-layout/src/v2/readers/chunked.rs | 202 ++++++++++++++++------ vortex-layout/src/v2/readers/constant.rs | 38 ++-- vortex-layout/src/v2/readers/flat.rs | 123 ++++++++----- vortex-layout/src/v2/readers/mod.rs | 8 +- vortex-layout/src/v2/readers/scalar_fn.rs | 50 +++++- vortex-layout/src/v2/readers/struct_.rs | 103 +++++++++-- vortex-layout/src/v2/readers/zoned.rs | 78 ++++----- vortex-scan/src/v2/scan.rs | 107 +++++++++++- 10 files changed, 520 insertions(+), 196 deletions(-) diff --git a/vortex-layout/src/v2/mod.rs b/vortex-layout/src/v2/mod.rs index c3d440a1cc9..6b4185aaf5c 100644 --- a/vortex-layout/src/v2/mod.rs +++ b/vortex-layout/src/v2/mod.rs @@ -1,9 +1,5 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -// mod expression; -// mod expressions; -// mod matcher; -// mod optimizer; pub mod reader; pub mod readers; diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index efe7958343d..c9ee6122664 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -65,8 +65,5 @@ pub trait ReaderStream: 'static + Send + Sync { fn next_chunk( &mut self, mask: MaskFuture, - // TODO(ngates): it would be good to pass an object here that has some lifetime, and this - // object is required to construct segment futures. That way the implementation is forced - // to construct segment futures in this call in order to satisfy the static result lifetime. ) -> VortexResult>>; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 4014820c9b1..fc84dfca30d 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -2,7 +2,9 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use std::any::Any; +use std::collections::VecDeque; use std::ops::Range; +use std::sync::Arc; use futures::future::BoxFuture; use futures::future::try_join_all; @@ -13,22 +15,33 @@ use vortex_array::MaskFuture; use vortex_array::arrays::ChunkedArray; use vortex_array::expr::Expression; use vortex_dtype::DType; -use vortex_error::VortexExpect; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_error::vortex_err; +use vortex_error::vortex_panic; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +/// A reader over a sequence of chunk readers with the same dtype. pub struct ChunkedReader { row_count: u64, dtype: DType, chunks: Vec, } +impl ChunkedReader { + pub fn new(row_count: u64, dtype: DType, chunks: Vec) -> Self { + Self { + row_count, + dtype, + chunks, + } + } +} + impl Reader for ChunkedReader { fn as_any(&self) -> &dyn Any { self @@ -42,29 +55,42 @@ impl Reader for ChunkedReader { self.row_count } + fn apply(&self, expression: &Expression) -> VortexResult { + let new_chunks: Vec = self + .chunks + .iter() + .map(|chunk| chunk.apply(expression)) + .collect::>()?; + let new_dtype = expression.return_dtype(&self.dtype)?; + Ok(Arc::new(Self { + row_count: self.row_count, + dtype: new_dtype, + chunks: new_chunks, + })) + } + fn execute(&self, row_range: Range) -> VortexResult { + // Collect overlapping chunk readers (not streams — lazy construction). let mut remaining_start = row_range.start; let mut remaining_end = row_range.end; - let mut streams = Vec::new(); + let mut pending_chunks = VecDeque::new(); for chunk in &self.chunks { let chunk_row_count = chunk.row_count(); if remaining_start >= chunk_row_count { - // This chunk is before the requested range remaining_start -= chunk_row_count; remaining_end -= chunk_row_count; continue; } let start_in_chunk = remaining_start; - let end_in_chunk = if remaining_end <= chunk_row_count { - remaining_end - } else { - chunk_row_count - }; + let end_in_chunk = remaining_end.min(chunk_row_count); - streams.push(chunk.execute(start_in_chunk..end_in_chunk)?); + pending_chunks.push_back(PendingChunk { + reader: chunk.clone(), + row_range: start_in_chunk..end_in_chunk, + }); remaining_start = 0; if remaining_end <= chunk_row_count { @@ -76,14 +102,43 @@ impl Reader for ChunkedReader { Ok(Box::new(ChunkedReaderStream { dtype: self.dtype.clone(), - chunks: streams, + pending_chunks, + active_stream: None, })) } } +struct PendingChunk { + reader: ReaderRef, + row_range: Range, +} + +/// A stream that lazily constructs child streams as it advances through chunks. struct ChunkedReaderStream { dtype: DType, - chunks: Vec, + pending_chunks: VecDeque, + active_stream: Option, +} + +impl ChunkedReaderStream { + /// Ensure we have an active stream pointing at the next non-exhausted chunk. + fn ensure_active_stream(&mut self) -> VortexResult { + loop { + if let Some(ref stream) = self.active_stream { + if stream.next_chunk_len().is_some() { + return Ok(true); + } + // Current stream is exhausted, drop it. + self.active_stream = None; + } + + // Try to activate the next pending chunk. + let Some(pending) = self.pending_chunks.pop_front() else { + return Ok(false); + }; + self.active_stream = Some(pending.reader.execute(pending.row_range)?); + } + } } impl ReaderStream for ChunkedReaderStream { @@ -92,64 +147,103 @@ impl ReaderStream for ChunkedReaderStream { } fn next_chunk_len(&self) -> Option { - self.chunks - .iter() - .map(|s| s.next_chunk_len()) - .find(|len| len.is_some()) - .flatten() + if let Some(ref stream) = self.active_stream { + let len = stream.next_chunk_len(); + if len.is_some() { + return len; + } + } + // Peek at the next pending chunk's row count as a hint. + self.pending_chunks + .front() + .map(|c| usize::try_from(c.row_range.end - c.row_range.start).unwrap_or(usize::MAX)) } - fn next_chunk( - &mut self, - selection: MaskFuture, - ) -> VortexResult>> { - // Remove any chunks that are already exhausted - loop { - if self.chunks.is_empty() { - vortex_bail!("Early termination of chunked layout"); + fn skip(&mut self, mut n: usize) { + while n > 0 { + // Try to skip within the active stream. + if let Some(ref mut stream) = self.active_stream { + if let Some(chunk_len) = stream.next_chunk_len() { + if n <= chunk_len { + stream.skip(n); + return; + } + stream.skip(chunk_len); + n -= chunk_len; + } + self.active_stream = None; } - if self.chunks[0].next_chunk_len().is_none() { - self.chunks.remove(0); + + // Skip entire pending chunks without constructing streams. + let Some(pending) = self.pending_chunks.front() else { + vortex_panic!("Cannot skip {} more rows, no chunks remaining", n); + }; + let chunk_rows = usize::try_from(pending.row_range.end - pending.row_range.start) + .unwrap_or(usize::MAX); + if n >= chunk_rows { + self.pending_chunks.pop_front(); + n -= chunk_rows; } else { - break; + // Partial skip — construct the stream and skip within it. + // We just checked front() is Some, so pop_front() is guaranteed. + match self.pending_chunks.pop_front() { + Some(pending) => { + let mut stream = match pending.reader.execute(pending.row_range) { + Ok(s) => s, + Err(e) => vortex_panic!("failed to execute chunk reader: {e}"), + }; + stream.skip(n); + self.active_stream = Some(stream); + } + None => vortex_panic!("pending chunk disappeared during skip"), + } + return; } } + } + + fn next_chunk( + &mut self, + mask: MaskFuture, + ) -> VortexResult>> { + if !self.ensure_active_stream()? { + vortex_bail!("No more chunks in chunked reader stream"); + } - // Get the length of the next chunk - let mut next_len = self.chunks[0] + let stream = self + .active_stream + .as_mut() + .ok_or_else(|| vortex_err!("Active stream missing after ensure"))?; + let next_len = stream .next_chunk_len() - .ok_or_else(|| vortex_err!("Early termination of chunked layout"))?; + .ok_or_else(|| vortex_err!("Active stream unexpectedly exhausted"))?; - if selection.len() <= next_len { - // The selection is smaller than the next chunk length, therefore we only need one chunk - return self.chunks[0].next_chunk(selection); + if mask.len() <= next_len { + return stream.next_chunk(mask); } - // Otherwise, we need to gather from multiple chunks - let mut selection = selection.clone(); - let mut futs = vec![]; - while !selection.is_empty() { - if self.chunks.is_empty() { - vortex_bail!("Early termination of chunked layout"); + // Mask spans multiple chunks — gather results from each. + let mut remaining_mask = mask; + let mut futs = Vec::new(); + + while !remaining_mask.is_empty() { + if !self.ensure_active_stream()? { + vortex_bail!("Ran out of chunks while processing mask"); } - // Slice off the right amount of selection for this chunk - let next_sel = selection.slice(..next_len); - selection = selection.slice(next_len..); + let stream = self + .active_stream + .as_mut() + .ok_or_else(|| vortex_err!("Active stream missing after ensure"))?; + let chunk_len = stream + .next_chunk_len() + .ok_or_else(|| vortex_err!("Active stream unexpectedly exhausted"))?; - let fut = self.chunks[0].next_chunk(&next_sel)?; - futs.push(fut); + let take = chunk_len.min(remaining_mask.len()); + let chunk_mask = remaining_mask.slice(0..take); + remaining_mask = remaining_mask.slice(take..remaining_mask.len()); - // Remove any chunks that are already exhausted - loop { - if self.chunks.is_empty() { - vortex_bail!("Early termination of chunked layout"); - } - if self.chunks[0].next_chunk_len().is_none() { - self.chunks.remove(0); - } - next_len = self.chunks[0].next_chunk_len().vortex_expect("non-none"); - } + futs.push(stream.next_chunk(chunk_mask)?); } let dtype = self.dtype.clone(); diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index 06825d81288..d22bac4e4db 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -16,6 +16,7 @@ use vortex_array::expr::root; use vortex_array::expr::transform::replace; use vortex_dtype::DType; use vortex_error::VortexResult; +use vortex_error::vortex_panic; use vortex_scalar::Scalar; use crate::v2::reader::Reader; @@ -23,11 +24,10 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +/// A reader that produces constant values. pub struct ConstantReader { scalar: Scalar, row_count: u64, - - // Optional expression to apply to the constant value. expression: Option, } @@ -55,18 +55,15 @@ impl Reader for ConstantReader { } fn apply(&self, expression: &Expression) -> VortexResult { - Ok(match &self.expression { - None => Arc::new(Self { - scalar: self.scalar.clone(), - row_count: self.row_count, - expression: Some(expression.clone()), - }), - Some(existing) => Arc::new(Self { - scalar: self.scalar.clone(), - row_count: self.row_count, - expression: Some(replace(existing.clone(), &root(), expression.clone())), - }), - }) + let new_expr = match &self.expression { + None => expression.clone(), + Some(existing) => replace(existing.clone(), &root(), expression.clone()), + }; + Ok(Arc::new(Self { + scalar: self.scalar.clone(), + row_count: self.row_count, + expression: Some(new_expr), + })) } fn execute(&self, row_range: Range) -> VortexResult { @@ -98,14 +95,25 @@ impl ReaderStream for ConstantReaderStream { } } + fn skip(&mut self, n: usize) { + let n = n as u64; + if n > self.remaining { + vortex_panic!("Cannot skip {} rows, only {} remaining", n, self.remaining); + } + self.remaining -= n; + } + fn next_chunk( &mut self, mask: MaskFuture, ) -> VortexResult>> { + let scalar = self.scalar.clone(); let expression = self.expression.clone(); + self.remaining = self.remaining.saturating_sub(mask.len() as u64); + Ok(async move { let mask = mask.await?; - let mut array = ConstantArray::new(self.scalar.clone(), mask.true_count()).into_array(); + let mut array = ConstantArray::new(scalar, mask.true_count()).into_array(); if let Some(e) = expression { array = array.apply(&e)?; } diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 7d098d7bb8a..732e5f5a6aa 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -5,22 +5,20 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use futures::TryFutureExt; use futures::future::BoxFuture; -use futures::try_join; use moka::future::FutureExt; use vortex_array::ArrayRef; +use vortex_array::Canonical; +use vortex_array::IntoArray; use vortex_array::MaskFuture; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::expr::transform::replace; use vortex_dtype::DType; -use vortex_error::VortexError; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_error::vortex_panic; -use crate::layouts::SharedArrayFuture; use crate::segments::SegmentId; use crate::segments::SegmentSourceRef; use crate::v2::reader::Reader; @@ -28,6 +26,10 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +/// A leaf reader that reads a single flat segment. +/// +/// The segment source handles caching — multiple streams reading the same segment will share +/// the underlying I/O through the segment source's cache. pub struct FlatReader { len: usize, dtype: DType, @@ -36,6 +38,23 @@ pub struct FlatReader { expression: Option, } +impl FlatReader { + pub fn new( + len: usize, + dtype: DType, + segment_id: SegmentId, + segment_source: SegmentSourceRef, + ) -> Self { + Self { + len, + dtype, + segment_id, + segment_source, + expression: None, + } + } +} + impl Reader for FlatReader { fn as_any(&self) -> &dyn Any { self @@ -50,41 +69,56 @@ impl Reader for FlatReader { } fn apply(&self, expression: &Expression) -> VortexResult { - let new_dtype = expression.return_dtype(&self.dtype)?; - Ok(match &self.expression { - None => Arc::new(Self { - len: self.len, - dtype: new_dtype, - segment_id: self.segment_id.clone(), - segment_source: self.segment_source.clone(), - expression: Some(expression.clone()), - }), - Some(e) => { - let new_expr = replace(e.clone(), &root(), expression.clone()); - Arc::new(Self { - len: self.len, - dtype: new_dtype, - segment_id: self.segment_id.clone(), - segment_source: self.segment_source.clone(), - expression: Some(new_expr), - }) - } - }) + let new_expr = match &self.expression { + None => expression.clone(), + Some(existing) => replace(existing.clone(), &root(), expression.clone()), + }; + let new_dtype = new_expr.return_dtype(&self.dtype)?; + Ok(Arc::new(Self { + len: self.len, + dtype: new_dtype, + segment_id: self.segment_id, + segment_source: self.segment_source.clone(), + expression: Some(new_expr), + })) } - fn execute(&self, _row_range: Range) -> VortexResult { - todo!() + fn execute(&self, row_range: Range) -> VortexResult { + let start = usize::try_from(row_range.start) + .map_err(|_| vortex_error::vortex_err!("row range start too large for usize"))?; + let end = usize::try_from(row_range.end) + .map_err(|_| vortex_error::vortex_err!("row range end too large for usize"))?; + + if start > self.len || end > self.len || start > end { + vortex_bail!( + "Row range {}..{} out of bounds for flat reader of length {}", + start, + end, + self.len + ); + } + + Ok(Box::new(FlatReaderStream { + dtype: self.dtype.clone(), + segment_id: self.segment_id, + segment_source: self.segment_source.clone(), + expression: self.expression.clone(), + offset: start, + remaining: end - start, + })) } } -struct FlatLayoutReaderStream { +struct FlatReaderStream { dtype: DType, - array_fut: SharedArrayFuture, + segment_id: SegmentId, + segment_source: SegmentSourceRef, + expression: Option, offset: usize, remaining: usize, } -impl ReaderStream for FlatLayoutReaderStream { +impl ReaderStream for FlatReaderStream { fn dtype(&self) -> &DType { &self.dtype } @@ -101,7 +135,6 @@ impl ReaderStream for FlatLayoutReaderStream { if n > self.remaining { vortex_panic!("Cannot skip {} rows, only {} remaining", n, self.remaining); } - self.offset += n; self.remaining -= n; } @@ -112,24 +145,36 @@ impl ReaderStream for FlatLayoutReaderStream { ) -> VortexResult>> { if mask.len() > self.remaining { vortex_bail!( - "Selection mask length {} exceeds remaining rows {}", + "Mask length {} exceeds remaining rows {}", mask.len(), self.remaining ); } - let array_fut = self.array_fut.clone(); + let segment_id = self.segment_id; + let segments = self.segment_source.clone(); let offset = self.offset; - let mask = mask.clone(); + let len = mask.len(); + let expression = self.expression.clone(); + + self.offset += len; + self.remaining -= len; - self.offset += mask.len(); - self.remaining -= mask.len(); + let dtype = self.dtype.clone(); Ok(async move { - let (array, mask) = try_join!(array_fut.map_err(VortexError::from), mask)?; - let sliced_array = array.slice(offset..offset + mask.len())?; - let selected_array = sliced_array.filter(mask)?; - Ok(selected_array) + // Await the mask first — if all-false, skip I/O entirely. + let mask = mask.await?; + if mask.true_count() == 0 { + return Ok(Canonical::empty(&dtype).into_array()); + } + + // Issue I/O only after confirming we have rows to read. + let _buffer = segments.request(segment_id).await?; + // TODO(ngates): decode buffer into array using ArrayParts/ArrayContext, + // then slice + filter + apply expression. + drop((offset, len, expression)); + todo!("decode segment buffer into array") } .boxed()) } diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 473a4036874..46d3c793376 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -1,9 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -// pub mod chunked; -// pub mod constant; +pub mod chunked; +pub mod constant; pub mod flat; -// pub mod scalar_fn; +pub mod scalar_fn; pub mod struct_; -// pub mod zoned; +pub mod zoned; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index e421f3624a9..07ee070c939 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -7,11 +7,14 @@ use std::sync::Arc; use futures::future::BoxFuture; use futures::future::try_join_all; +use itertools::Itertools; use vortex_array::ArrayRef; use vortex_array::IntoArray; use vortex_array::MaskFuture; use vortex_array::arrays::ScalarFnArray; use vortex_array::expr::Expression; +use vortex_array::expr::Literal; +use vortex_array::expr::Root; use vortex_array::expr::ScalarFn; use vortex_array::expr::VTable; use vortex_array::expr::VTableExt; @@ -23,8 +26,9 @@ use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +use crate::v2::readers::constant::ConstantReader; -/// A [`Reader] for applying a scalar function to another layout. +/// A [`Reader`] for applying a scalar function to child readers. pub struct ScalarFnReader { scalar_fn: ScalarFn, dtype: DType, @@ -72,7 +76,35 @@ impl Reader for ScalarFnReader { } fn apply(&self, expression: &Expression) -> VortexResult { - todo!() + // Treat this ScalarFnReader as a data source. Resolve Root to a clone of self, + // literals to constants, and wrap everything else in a new ScalarFnReader. + if expression.is::() { + return Ok(Arc::new(Self { + scalar_fn: self.scalar_fn.clone(), + dtype: self.dtype.clone(), + row_count: self.row_count, + children: self.children.clone(), + })); + } + + if let Some(scalar) = expression.as_opt::() { + return Ok(Arc::new(ConstantReader::new( + scalar.clone(), + self.row_count, + ))); + } + + let resolved_children: Vec = expression + .children() + .iter() + .map(|child| self.apply(child)) + .try_collect()?; + + Ok(Arc::new(Self::try_new( + expression.scalar_fn().clone(), + resolved_children, + self.row_count, + )?)) } fn execute(&self, row_range: Range) -> VortexResult { @@ -109,20 +141,26 @@ impl ReaderStream for ScalarFnArrayStream { .flatten() } + fn skip(&mut self, n: usize) { + for stream in &mut self.input_streams { + stream.skip(n); + } + } + fn next_chunk( &mut self, - selection: MaskFuture, + mask: MaskFuture, ) -> VortexResult>> { let scalar_fn = self.scalar_fn.clone(); - let len = selection.true_count(); let futs = self .input_streams .iter_mut() - .map(|s| s.next_chunk(selection)) + .map(|s| s.next_chunk(mask.clone())) .collect::>>()?; Ok(Box::pin(async move { let input_arrays = try_join_all(futs).await?; + let len = input_arrays.first().map(|a| a.len()).unwrap_or(0); let array = ScalarFnArray::try_new(scalar_fn, input_arrays, len)?.into_array(); let array = array.optimize()?; Ok(array) @@ -140,7 +178,7 @@ pub trait ScalarFnReaderExt: VTable { ) -> VortexResult { Ok(Arc::new(ScalarFnReader::try_new( self.bind(options), - children.into(), + children, row_count, )?)) } diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index afdeee022bd..5706ad1b04a 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -3,16 +3,21 @@ use std::any::Any; use std::ops::Range; +use std::sync::Arc; use futures::future::BoxFuture; use futures::future::try_join_all; use futures::try_join; +use itertools::Itertools; use moka::future::FutureExt; use vortex_array::ArrayRef; use vortex_array::IntoArray; use vortex_array::MaskFuture; use vortex_array::arrays::StructArray; use vortex_array::expr::Expression; +use vortex_array::expr::GetItem; +use vortex_array::expr::Literal; +use vortex_array::expr::Root; use vortex_array::validity::Validity; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -21,14 +26,78 @@ use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +use crate::v2::readers::constant::ConstantReader; +use crate::v2::readers::scalar_fn::ScalarFnReader; +/// A reader over a struct with named fields. pub struct StructReader { row_count: u64, dtype: DType, - // TODO(ngates): we should make this lazy? fields: Vec, } +impl StructReader { + pub fn new(row_count: u64, dtype: DType, fields: Vec) -> Self { + Self { + row_count, + dtype, + fields, + } + } + + /// Recursively resolve an expression through this struct, extracting fields where possible. + fn resolve_expr(&self, expr: &Expression) -> VortexResult { + // Root references this struct. + if expr.is::() { + return Ok(Arc::new(Self { + row_count: self.row_count, + dtype: self.dtype.clone(), + fields: self.fields.clone(), + })); + } + + // Literals become constant readers. + if let Some(scalar) = expr.as_opt::() { + return Ok(Arc::new(ConstantReader::new( + scalar.clone(), + self.row_count, + ))); + } + + // Recursively resolve all children through this struct. + let resolved_children: Vec = expr + .children() + .iter() + .map(|child| self.resolve_expr(child)) + .try_collect()?; + + // If this is GetItem and the resolved child is a StructReader, extract the field directly. + if let Some(field_name) = expr.as_opt::() { + debug_assert_eq!(resolved_children.len(), 1); + let child_reader = &resolved_children[0]; + if let Some(struct_reader) = child_reader.as_any().downcast_ref::() { + let struct_fields = struct_reader + .dtype + .as_struct_fields_opt() + .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))?; + let field_idx = struct_fields.find(field_name).ok_or_else(|| { + vortex_error::vortex_err!("Field '{}' not found in struct", field_name) + })?; + return Ok(struct_reader.fields[field_idx].clone()); + } + // Otherwise, the child is some other reader (e.g., ChunkedReader). Delegate apply. + return child_reader.apply(expr); + } + + // For any other scalar function, wrap the resolved children. + Ok(Arc::new(ScalarFnReader::try_new( + expr.scalar_fn().clone(), + resolved_children, + self.row_count, + )?)) + } +} + impl Reader for StructReader { fn as_any(&self) -> &dyn Any { self @@ -42,8 +111,8 @@ impl Reader for StructReader { self.row_count } - fn apply(&self, _expression: &Expression) -> VortexResult { - todo!() + fn apply(&self, expression: &Expression) -> VortexResult { + self.resolve_expr(expression) } fn execute(&self, row_range: Range) -> VortexResult { @@ -86,26 +155,32 @@ impl ReaderStream for StructReaderStream { fn next_chunk( &mut self, - selection: MaskFuture, + mask: MaskFuture, ) -> VortexResult>> { - let struct_fields = self.dtype.as_struct_fields().clone(); + let struct_fields = self + .dtype + .as_struct_fields_opt() + .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))? + .clone(); let validity: Validity = self.dtype.nullability().into(); let fields = self .fields .iter_mut() - .map(|s| s.next_chunk(selection.clone())) + .map(|s| s.next_chunk(mask.clone())) .collect::>>()?; Ok(async move { let fields = try_join_all(fields); - let (fields, mask) = try_join!(fields, selection)?; - Ok(StructArray::try_new_with_dtype( - fields, - struct_fields, - mask.true_count(), - validity.clone(), - )? - .into_array()) + let (fields, mask) = try_join!(fields, mask)?; + Ok( + StructArray::try_new_with_dtype( + fields, + struct_fields, + mask.true_count(), + validity, + )? + .into_array(), + ) } .boxed()) } diff --git a/vortex-layout/src/v2/readers/zoned.rs b/vortex-layout/src/v2/readers/zoned.rs index 9ab25fd15c1..3aeceed4ce4 100644 --- a/vortex-layout/src/v2/readers/zoned.rs +++ b/vortex-layout/src/v2/readers/zoned.rs @@ -9,12 +9,16 @@ use vortex_array::expr::Expression; use vortex_array::expr::stats::Stat; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_error::vortex_bail; use crate::v2::reader::Reader; use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStreamRef; +/// A reader that wraps a data reader with per-zone statistics for pruning. +/// +/// The zone map reader holds aggregate statistics (min, max, null_count) for each zone +/// of `zone_len` rows. During execute, zone statistics can be used to skip entire zones +/// that cannot match a filter expression. pub struct ZonedReader { data: ReaderRef, zone_map: ReaderRef, @@ -22,6 +26,22 @@ pub struct ZonedReader { present_stats: Arc<[Stat]>, } +impl ZonedReader { + pub fn new( + data: ReaderRef, + zone_map: ReaderRef, + zone_len: usize, + present_stats: Arc<[Stat]>, + ) -> Self { + Self { + data, + zone_map, + zone_len, + present_stats, + } + } +} + impl Reader for ZonedReader { fn as_any(&self) -> &dyn Any { self @@ -36,63 +56,23 @@ impl Reader for ZonedReader { } fn apply(&self, expression: &Expression) -> VortexResult { - // We need to apply the expression to both the data and the zone map. + // Apply the expression to the data reader. let new_data = self.data.apply(expression)?; - let new_zone_map = self.zone_map.apply(expression)?; - - // We also need to update the present stats for the new zone map. - let new_present_stats = self - .present_stats - .iter() - .map(|stat| match stat { - Stat::Min => Stat::Min, - Stat::Max => Stat::Max, - Stat::NullCount => Stat::NullCount, - _ => vortex_bail!("Unsupported stat for zoned reader: {:?}", stat), - }) - .collect(); + // TODO(ngates): transform the expression into zone-map-compatible pruning checks + // (e.g., `x > 5` becomes `max >= 5`). For now, we just pass through the data + // and keep the zone map as-is. Ok(Arc::new(ZonedReader { data: new_data, - zone_map: new_zone_map, + zone_map: self.zone_map.clone(), zone_len: self.zone_len, - present_stats: Arc::new(new_present_stats), + present_stats: self.present_stats.clone(), })) } fn execute(&self, row_range: Range) -> VortexResult { - // By default, a zoned reader is just a pass-through. + // TODO(ngates): use zone map to produce a pruning mask, then drive the data stream + // with zones that can't match the filter skipped. self.data.execute(row_range) } } - -/// A reader that expands zoned statistics to match the data rows. -/// This repeats each row of the zone map `zone_len` times. -/// TODO(ngates): we could use a RunEndReader + Slice to do this? -struct ZonedExpansionReader { - zoned: ReaderRef, - zone_len: usize, - row_count: u64, -} - -impl Reader for ZonedExpansionReader { - fn as_any(&self) -> &dyn Any { - self - } - - fn dtype(&self) -> &DType { - self.zoned.dtype() - } - - fn row_count(&self) -> u64 { - self.row_count - } - - fn apply(&self, expression: &Expression) -> VortexResult { - todo!() - } - - fn execute(&self, _row_range: Range) -> VortexResult { - todo!() - } -} diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 8d00f5d7337..2c52ca33e4e 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -7,7 +7,9 @@ use std::task::Context; use std::task::Poll; use futures::Stream; +use futures::future::BoxFuture; use vortex_array::ArrayRef; +use vortex_array::MaskFuture; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::stream::ArrayStream; @@ -15,6 +17,7 @@ use vortex_buffer::Buffer; use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_layout::v2::reader::ReaderRef; +use vortex_layout::v2::reader::ReaderStreamRef; use vortex_session::VortexSession; use crate::Selection; @@ -89,22 +92,36 @@ impl ScanBuilder2 { let dtype = projection.return_dtype(self.reader.dtype())?; - // So we wrap the reader for filtering. - let filter_reader = filter.as_ref().map(|f| self.reader.apply(&f)).transpose()?; + // Apply expressions to the reader tree. + let filter_reader = filter.as_ref().map(|f| self.reader.apply(f)).transpose()?; let projection_reader = self.reader.apply(&projection)?; + // Execute both readers over the row range to produce streams. + let filter_stream = filter_reader + .map(|r| r.execute(self.row_range.clone())) + .transpose()?; + let projection_stream = projection_reader.execute(self.row_range)?; + + // TODO(ngates): incorporate row_selection into the mask chain. + Ok(Scan { dtype, - filter_reader, - projection_reader, + filter_stream, + projection_stream, + pending: None, + limit: self.limit, + rows_produced: 0, }) } } struct Scan { dtype: DType, - filter_reader: Option, - projection_reader: ReaderRef, + filter_stream: Option, + projection_stream: ReaderStreamRef, + pending: Option>>, + limit: Option, + rows_produced: u64, } impl ArrayStream for Scan { @@ -116,7 +133,81 @@ impl ArrayStream for Scan { impl Stream for Scan { type Item = VortexResult; - fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - todo!() + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + + loop { + // Poll pending future if we have one. + if let Some(fut) = this.pending.as_mut() { + match fut.as_mut().poll(cx) { + Poll::Ready(result) => { + this.pending = None; + return match result { + Ok(array) => { + this.rows_produced += array.len() as u64; + Poll::Ready(Some(Ok(array))) + } + Err(e) => Poll::Ready(Some(Err(e))), + }; + } + Poll::Pending => return Poll::Pending, + } + } + + // Check limit. + if this.limit.is_some_and(|limit| this.rows_produced >= limit) { + return Poll::Ready(None); + } + + // Determine the next chunk size from the projection stream. + let proj_chunk_len = this.projection_stream.next_chunk_len(); + + // If a filter stream exists, synchronize chunk sizes. + let chunk_len = if let Some(filter_stream) = &this.filter_stream { + match (proj_chunk_len, filter_stream.next_chunk_len()) { + (Some(p), Some(f)) => Some(p.min(f)), + _ => None, + } + } else { + proj_chunk_len + }; + + let Some(mut chunk_len) = chunk_len else { + return Poll::Ready(None); + }; + + // Limit the chunk size to avoid exceeding the row limit. + if let Some(limit) = this.limit { + let remaining = usize::try_from(limit - this.rows_produced).unwrap_or(usize::MAX); + chunk_len = chunk_len.min(remaining); + if chunk_len == 0 { + return Poll::Ready(None); + } + } + + // Build the mask chain: if a filter exists, evaluate it first and convert the + // boolean result into a Mask for the projection stream. + let mask = if let Some(filter_stream) = &mut this.filter_stream { + let all_true = MaskFuture::new_true(chunk_len); + let filter_fut = match filter_stream.next_chunk(all_true) { + Ok(fut) => fut, + Err(e) => return Poll::Ready(Some(Err(e))), + }; + MaskFuture::new(chunk_len, async move { + let filter_result = filter_fut.await?; + filter_result.try_to_mask_fill_null_false() + }) + } else { + MaskFuture::new_true(chunk_len) + }; + + // Request the next projection chunk with the computed mask. + this.pending = Some(match this.projection_stream.next_chunk(mask) { + Ok(fut) => fut, + Err(e) => return Poll::Ready(Some(Err(e))), + }); + + // Loop back to poll the newly created future. + } } } From 7fed2d5026090f4c45d94c4f60bc4d9327a97bee Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 14:16:27 -0500 Subject: [PATCH 14/21] Draft release Signed-off-by: Nicholas Gates --- vortex-layout/src/layouts/chunked/mod.rs | 19 +++ vortex-layout/src/layouts/dict/mod.rs | 17 +++ vortex-layout/src/layouts/flat/mod.rs | 21 ++++ vortex-layout/src/layouts/struct_/mod.rs | 28 +++++ vortex-layout/src/layouts/zoned/mod.rs | 18 +++ vortex-layout/src/v2/readers/dict.rs | 144 +++++++++++++++++++++++ vortex-layout/src/v2/readers/flat.rs | 101 +++++++++++++--- vortex-layout/src/v2/readers/mod.rs | 1 + vortex-layout/src/v2/readers/struct_.rs | 43 ++++++- vortex-layout/src/vtable.rs | 2 +- vortex-scan/src/v2/scan.rs | 38 +++++- 11 files changed, 404 insertions(+), 28 deletions(-) create mode 100644 vortex-layout/src/v2/readers/dict.rs diff --git a/vortex-layout/src/layouts/chunked/mod.rs b/vortex-layout/src/layouts/chunked/mod.rs index 8c5c77b6f78..53a700aa640 100644 --- a/vortex-layout/src/layouts/chunked/mod.rs +++ b/vortex-layout/src/layouts/chunked/mod.rs @@ -6,6 +6,7 @@ pub mod writer; use std::sync::Arc; +use itertools::Itertools; use vortex_array::ArrayContext; use vortex_array::DeserializeMetadata; use vortex_array::EmptyMetadata; @@ -24,6 +25,9 @@ use crate::children::OwnedLayoutChildren; use crate::layouts::chunked::reader::ChunkedReader; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2; +use crate::v2::reader::ReaderRef; use crate::vtable; vtable!(Chunked); @@ -83,6 +87,21 @@ impl VTable for ChunkedVTable { ))) } + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + let chunks = (0..Self::nchildren(layout)) + .map(|i| Self::child(layout, i)?.new_reader2(segment_source, session)) + .try_collect()?; + Ok(Arc::new(v2::readers::chunked::ChunkedReader::new( + layout.row_count, + layout.dtype.clone(), + chunks, + ))) + } + fn build( _encoding: &Self::Encoding, dtype: &DType, diff --git a/vortex-layout/src/layouts/dict/mod.rs b/vortex-layout/src/layouts/dict/mod.rs index e24336e7d9a..7b5ff688b45 100644 --- a/vortex-layout/src/layouts/dict/mod.rs +++ b/vortex-layout/src/layouts/dict/mod.rs @@ -30,6 +30,9 @@ use crate::VTable; use crate::children::LayoutChildren; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2; +use crate::v2::reader::ReaderRef; use crate::vtable; vtable!(Dict); @@ -101,6 +104,20 @@ impl VTable for DictVTable { )?)) } + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + let values = layout.values.new_reader2(segment_source, session)?; + let codes = layout.codes.new_reader2(segment_source, session)?; + Ok(Arc::new(v2::readers::dict::DictReader::new( + Self::dtype(layout).clone(), + values, + codes, + ))) + } + fn build( _encoding: &Self::Encoding, dtype: &DType, diff --git a/vortex-layout/src/layouts/flat/mod.rs b/vortex-layout/src/layouts/flat/mod.rs index 3826fc5dc7d..890b8182398 100644 --- a/vortex-layout/src/layouts/flat/mod.rs +++ b/vortex-layout/src/layouts/flat/mod.rs @@ -10,6 +10,7 @@ use std::sync::Arc; use vortex_array::ArrayContext; use vortex_array::DeserializeMetadata; use vortex_array::ProstMetadata; +use vortex_array::session::ArraySessionExt; use vortex_buffer::ByteBuffer; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -27,6 +28,9 @@ use crate::children::LayoutChildren; use crate::layouts::flat::reader::FlatReader; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2; +use crate::v2::reader::ReaderRef; use crate::vtable; /// Check if inline array node is enabled. @@ -94,6 +98,23 @@ impl VTable for FlatVTable { ))) } + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + let len = usize::try_from(layout.row_count).unwrap_or(usize::MAX); + Ok(Arc::new(v2::readers::flat::FlatReader::new( + len, + layout.dtype.clone(), + layout.array_tree.clone(), + layout.segment_id, + segment_source.clone(), + layout.ctx.clone(), + session.arrays().registry().clone(), + ))) + } + fn build( _encoding: &Self::Encoding, dtype: &DType, diff --git a/vortex-layout/src/layouts/struct_/mod.rs b/vortex-layout/src/layouts/struct_/mod.rs index 2594e6723c0..a587e3b7a54 100644 --- a/vortex-layout/src/layouts/struct_/mod.rs +++ b/vortex-layout/src/layouts/struct_/mod.rs @@ -6,6 +6,7 @@ pub mod writer; use std::sync::Arc; +use itertools::Itertools; use reader::StructReader; use vortex_array::ArrayContext; use vortex_array::DeserializeMetadata; @@ -33,6 +34,9 @@ use crate::children::LayoutChildren; use crate::children::OwnedLayoutChildren; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2; +use crate::v2::reader::ReaderRef; use crate::vtable; vtable!(Struct); @@ -124,6 +128,30 @@ impl VTable for StructVTable { )?)) } + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + // Create validity reader from child 0 when nullable. + let validity = layout + .dtype + .is_nullable() + .then(|| Self::child(layout, 0)?.new_reader2(segment_source, session)) + .transpose()?; + let start_idx = if layout.dtype.is_nullable() { 1 } else { 0 }; + let nchildren = Self::nchildren(layout); + let fields = (start_idx..nchildren) + .map(|i| Self::child(layout, i)?.new_reader2(segment_source, session)) + .try_collect()?; + Ok(Arc::new(v2::readers::struct_::StructReader::new( + layout.row_count, + layout.dtype.clone(), + validity, + fields, + ))) + } + fn build( _encoding: &Self::Encoding, dtype: &DType, diff --git a/vortex-layout/src/layouts/zoned/mod.rs b/vortex-layout/src/layouts/zoned/mod.rs index 8cd63e3e429..d98ca79d8ee 100644 --- a/vortex-layout/src/layouts/zoned/mod.rs +++ b/vortex-layout/src/layouts/zoned/mod.rs @@ -38,6 +38,9 @@ use crate::layouts::zoned::reader::ZonedReader; use crate::layouts::zoned::zone_map::ZoneMap; use crate::segments::SegmentId; use crate::segments::SegmentSource; +use crate::segments::SegmentSourceRef; +use crate::v2; +use crate::v2::reader::ReaderRef; use crate::vtable; vtable!(Zoned); @@ -111,6 +114,21 @@ impl VTable for ZonedVTable { )?)) } + fn new_reader2( + layout: &Self::Layout, + segment_source: &SegmentSourceRef, + session: &VortexSession, + ) -> VortexResult { + let data = layout.child(0)?.new_reader2(segment_source, session)?; + let zones = layout.child(1)?.new_reader2(segment_source, session)?; + Ok(Arc::new(v2::readers::zoned::ZonedReader::new( + data, + zones, + layout.zone_len, + layout.present_stats.clone(), + ))) + } + fn build( _encoding: &Self::Encoding, dtype: &DType, diff --git a/vortex-layout/src/v2/readers/dict.rs b/vortex-layout/src/v2/readers/dict.rs new file mode 100644 index 00000000000..908ed410f8d --- /dev/null +++ b/vortex-layout/src/v2/readers/dict.rs @@ -0,0 +1,144 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::any::Any; +use std::ops::Range; +use std::sync::Arc; + +use futures::FutureExt; +use futures::future::BoxFuture; +use futures::future::Shared; +use vortex_array::ArrayRef; +use vortex_array::IntoArray; +use vortex_array::MaskFuture; +use vortex_array::arrays::DictArray; +use vortex_array::expr::Expression; +use vortex_array::expr::root; +use vortex_array::expr::transform::replace; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_err; + +use crate::v2::reader::Reader; +use crate::v2::reader::ReaderRef; +use crate::v2::reader::ReaderStream; +use crate::v2::reader::ReaderStreamRef; + +type SharedVortexResult = Result>; + +/// A reader that reconstructs dict-encoded arrays from separate values and codes readers. +/// +/// The values reader holds the dictionary (typically a small flat array), while the codes +/// reader holds integer indices into the dictionary (row-aligned with the parent layout). +pub struct DictReader { + dtype: DType, + values: ReaderRef, + codes: ReaderRef, + expression: Option, +} + +impl DictReader { + pub fn new(dtype: DType, values: ReaderRef, codes: ReaderRef) -> Self { + Self { + dtype, + values, + codes, + expression: None, + } + } +} + +impl Reader for DictReader { + fn as_any(&self) -> &dyn Any { + self + } + + fn dtype(&self) -> &DType { + &self.dtype + } + + fn row_count(&self) -> u64 { + self.codes.row_count() + } + + fn apply(&self, expression: &Expression) -> VortexResult { + // Dict encoding is transparent — expressions are applied after reconstruction. + // We compose the expression and apply it after creating the DictArray. + let new_expr = match &self.expression { + None => expression.clone(), + Some(existing) => replace(existing.clone(), &root(), expression.clone()), + }; + let new_dtype = new_expr.return_dtype(&self.dtype)?; + Ok(Arc::new(Self { + dtype: new_dtype, + values: self.values.clone(), + codes: self.codes.clone(), + expression: Some(new_expr), + })) + } + + fn execute(&self, row_range: Range) -> VortexResult { + // Read the full dictionary values once and share across all chunks. + let values_row_count = self.values.row_count(); + let mut values_stream = self.values.execute(0..values_row_count)?; + let values_len = values_stream + .next_chunk_len() + .ok_or_else(|| vortex_err!("Dict values stream is empty"))?; + let values_mask = MaskFuture::new_true(values_len); + let values_fut: Shared>> = values_stream + .next_chunk(values_mask)? + .map(|r| r.map_err(Arc::new)) + .boxed() + .shared(); + + let codes_stream = self.codes.execute(row_range)?; + + Ok(Box::new(DictReaderStream { + dtype: self.dtype.clone(), + codes_stream, + values_fut, + expression: self.expression.clone(), + })) + } +} + +struct DictReaderStream { + dtype: DType, + codes_stream: ReaderStreamRef, + values_fut: Shared>>, + expression: Option, +} + +impl ReaderStream for DictReaderStream { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn next_chunk_len(&self) -> Option { + self.codes_stream.next_chunk_len() + } + + fn skip(&mut self, n: usize) { + self.codes_stream.skip(n); + } + + fn next_chunk( + &mut self, + mask: MaskFuture, + ) -> VortexResult>> { + let values_fut = self.values_fut.clone(); + let codes_fut = self.codes_stream.next_chunk(mask)?; + let expression = self.expression.clone(); + + Ok(async move { + let values = values_fut.await.map_err(|e| vortex_err!("{e}"))?; + let codes = codes_fut.await?; + let mut array = DictArray::try_new(codes, values)?.into_array(); + if let Some(expr) = expression { + array = array.apply(&expr)?; + } + Ok(array) + } + .boxed()) + } +} diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 732e5f5a6aa..a71e2f1b00f 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -5,18 +5,26 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; +use futures::FutureExt; use futures::future::BoxFuture; -use moka::future::FutureExt; +use futures::future::Shared; +use vortex_array::ArrayContext; use vortex_array::ArrayRef; use vortex_array::Canonical; use vortex_array::IntoArray; use vortex_array::MaskFuture; +use vortex_array::buffer::BufferHandle; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::expr::transform::replace; +use vortex_array::serde::ArrayParts; +use vortex_array::session::ArrayRegistry; +use vortex_buffer::ByteBuffer; use vortex_dtype::DType; +use vortex_error::SharedVortexResult; use vortex_error::VortexResult; use vortex_error::vortex_bail; +use vortex_error::vortex_err; use vortex_error::vortex_panic; use crate::segments::SegmentId; @@ -26,6 +34,8 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; +type SharedSegmentFuture = Shared>>; + /// A leaf reader that reads a single flat segment. /// /// The segment source handles caching — multiple streams reading the same segment will share @@ -33,8 +43,12 @@ use crate::v2::reader::ReaderStreamRef; pub struct FlatReader { len: usize, dtype: DType, + decode_dtype: DType, + array_tree: Option, segment_id: SegmentId, segment_source: SegmentSourceRef, + ctx: ArrayContext, + registry: ArrayRegistry, expression: Option, } @@ -42,14 +56,21 @@ impl FlatReader { pub fn new( len: usize, dtype: DType, + array_tree: Option, segment_id: SegmentId, segment_source: SegmentSourceRef, + ctx: ArrayContext, + registry: ArrayRegistry, ) -> Self { Self { len, + decode_dtype: dtype.clone(), dtype, + array_tree, segment_id, segment_source, + ctx, + registry, expression: None, } } @@ -77,17 +98,21 @@ impl Reader for FlatReader { Ok(Arc::new(Self { len: self.len, dtype: new_dtype, + decode_dtype: self.decode_dtype.clone(), + array_tree: self.array_tree.clone(), segment_id: self.segment_id, segment_source: self.segment_source.clone(), + ctx: self.ctx.clone(), + registry: self.registry.clone(), expression: Some(new_expr), })) } fn execute(&self, row_range: Range) -> VortexResult { let start = usize::try_from(row_range.start) - .map_err(|_| vortex_error::vortex_err!("row range start too large for usize"))?; + .map_err(|_| vortex_err!("row range start too large for usize"))?; let end = usize::try_from(row_range.end) - .map_err(|_| vortex_error::vortex_err!("row range end too large for usize"))?; + .map_err(|_| vortex_err!("row range end too large for usize"))?; if start > self.len || end > self.len || start > end { vortex_bail!( @@ -98,11 +123,23 @@ impl Reader for FlatReader { ); } + // Request the segment once and share across all next_chunk calls. + let segment_fut = self + .segment_source + .request(self.segment_id) + .map(|r| r.map_err(Arc::new)) + .boxed() + .shared(); + Ok(Box::new(FlatReaderStream { dtype: self.dtype.clone(), - segment_id: self.segment_id, - segment_source: self.segment_source.clone(), + decode_dtype: self.decode_dtype.clone(), + segment_fut, + array_tree: self.array_tree.clone(), + ctx: self.ctx.clone(), + registry: self.registry.clone(), expression: self.expression.clone(), + row_count: self.len, offset: start, remaining: end - start, })) @@ -111,9 +148,13 @@ impl Reader for FlatReader { struct FlatReaderStream { dtype: DType, - segment_id: SegmentId, - segment_source: SegmentSourceRef, + decode_dtype: DType, + segment_fut: SharedSegmentFuture, + array_tree: Option, + ctx: ArrayContext, + registry: ArrayRegistry, expression: Option, + row_count: usize, offset: usize, remaining: usize, } @@ -151,30 +192,56 @@ impl ReaderStream for FlatReaderStream { ); } - let segment_id = self.segment_id; - let segments = self.segment_source.clone(); + let segment_fut = self.segment_fut.clone(); + let array_tree = self.array_tree.clone(); + let ctx = self.ctx.clone(); + let registry = self.registry.clone(); + let row_count = self.row_count; let offset = self.offset; let len = mask.len(); let expression = self.expression.clone(); + let dtype = self.dtype.clone(); + let decode_dtype = self.decode_dtype.clone(); self.offset += len; self.remaining -= len; - let dtype = self.dtype.clone(); - Ok(async move { // Await the mask first — if all-false, skip I/O entirely. + // TODO(ngates): should we race this against the segment future? let mask = mask.await?; if mask.true_count() == 0 { return Ok(Canonical::empty(&dtype).into_array()); } - // Issue I/O only after confirming we have rows to read. - let _buffer = segments.request(segment_id).await?; - // TODO(ngates): decode buffer into array using ArrayParts/ArrayContext, - // then slice + filter + apply expression. - drop((offset, len, expression)); - todo!("decode segment buffer into array") + // Await the shared segment future (I/O is issued once, shared across chunks). + let segment = segment_fut.await?; + let parts = if let Some(array_tree) = array_tree { + // Use the pre-stored flatbuffer from layout metadata combined with segment buffers. + ArrayParts::from_flatbuffer_and_segment(array_tree, segment)? + } else { + // Parse the flatbuffer from the segment itself. + ArrayParts::try_from(segment)? + }; + + let mut array = parts.decode(&decode_dtype, row_count, &ctx, ®istry)?; + + // Slice to the requested row range within the segment. + if offset > 0 || len < row_count { + array = array.slice(offset..offset + len)?; + } + + // Filter using the mask. + if !mask.all_true() { + array = array.filter(mask)?; + } + + // Apply any accumulated expression. + if let Some(expr) = expression { + array = array.apply(&expr)?; + } + + Ok(array) } .boxed()) } diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 46d3c793376..876acd44f44 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -3,6 +3,7 @@ pub mod chunked; pub mod constant; +pub mod dict; pub mod flat; pub mod scalar_fn; pub mod struct_; diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 5706ad1b04a..b6b2d626534 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -33,14 +33,21 @@ use crate::v2::readers::scalar_fn::ScalarFnReader; pub struct StructReader { row_count: u64, dtype: DType, + validity: Option, fields: Vec, } impl StructReader { - pub fn new(row_count: u64, dtype: DType, fields: Vec) -> Self { + pub fn new( + row_count: u64, + dtype: DType, + validity: Option, + fields: Vec, + ) -> Self { Self { row_count, dtype, + validity, fields, } } @@ -52,6 +59,7 @@ impl StructReader { return Ok(Arc::new(Self { row_count: self.row_count, dtype: self.dtype.clone(), + validity: self.validity.clone(), fields: self.fields.clone(), })); } @@ -116,6 +124,11 @@ impl Reader for StructReader { } fn execute(&self, row_range: Range) -> VortexResult { + let validity_stream = self + .validity + .as_ref() + .map(|v| v.execute(row_range.clone())) + .transpose()?; let field_streams = self .fields .iter() @@ -124,6 +137,7 @@ impl Reader for StructReader { Ok(Box::new(StructReaderStream { dtype: self.dtype.clone(), + validity: validity_stream, fields: field_streams, })) } @@ -131,6 +145,7 @@ impl Reader for StructReader { struct StructReaderStream { dtype: DType, + validity: Option, fields: Vec, } @@ -140,14 +155,23 @@ impl ReaderStream for StructReaderStream { } fn next_chunk_len(&self) -> Option { - self.fields + let field_min = self + .fields .iter() .map(|s| s.next_chunk_len()) .min() - .flatten() + .flatten(); + match (&self.validity, field_min) { + (Some(v), Some(f)) => v.next_chunk_len().map(|vl| vl.min(f)), + (Some(v), None) => v.next_chunk_len(), + (None, f) => f, + } } fn skip(&mut self, n: usize) { + if let Some(validity) = &mut self.validity { + validity.skip(n); + } for field in &mut self.fields { field.skip(n); } @@ -162,7 +186,12 @@ impl ReaderStream for StructReaderStream { .as_struct_fields_opt() .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))? .clone(); - let validity: Validity = self.dtype.nullability().into(); + let nullability = self.dtype.nullability(); + let validity_fut = self + .validity + .as_mut() + .map(|v| v.next_chunk(mask.clone())) + .transpose()?; let fields = self .fields .iter_mut() @@ -172,6 +201,12 @@ impl ReaderStream for StructReaderStream { Ok(async move { let fields = try_join_all(fields); let (fields, mask) = try_join!(fields, mask)?; + let validity = if let Some(validity_fut) = validity_fut { + let validity_array = validity_fut.await?; + Validity::Array(validity_array) + } else { + nullability.into() + }; Ok( StructArray::try_new_with_dtype( fields, diff --git a/vortex-layout/src/vtable.rs b/vortex-layout/src/vtable.rs index fe8f829b07b..54cd6772c7a 100644 --- a/vortex-layout/src/vtable.rs +++ b/vortex-layout/src/vtable.rs @@ -73,7 +73,7 @@ pub trait VTable: 'static + Sized + Send + Sync + Debug { session: &VortexSession, ) -> VortexResult { let _ = (layout, segment_source, session); - vortex_bail!("new_reader2 not implemented for this layout") + vortex_bail!("new_reader2 not implemented for layout: {:?}", layout) } /// Construct a new [`Layout`] from the provided parts. diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 2c52ca33e4e..87e360dcef6 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::ops::BitAnd; use std::ops::Range; use std::pin::Pin; use std::task::Context; @@ -97,13 +98,12 @@ impl ScanBuilder2 { let projection_reader = self.reader.apply(&projection)?; // Execute both readers over the row range to produce streams. + let row_offset = self.row_range.start; let filter_stream = filter_reader .map(|r| r.execute(self.row_range.clone())) .transpose()?; let projection_stream = projection_reader.execute(self.row_range)?; - // TODO(ngates): incorporate row_selection into the mask chain. - Ok(Scan { dtype, filter_stream, @@ -111,6 +111,8 @@ impl ScanBuilder2 { pending: None, limit: self.limit, rows_produced: 0, + row_selection: self.row_selection, + row_offset, }) } } @@ -122,6 +124,8 @@ struct Scan { pending: Option>>, limit: Option, rows_produced: u64, + row_selection: Selection, + row_offset: u64, } impl ArrayStream for Scan { @@ -185,8 +189,21 @@ impl Stream for Scan { } } - // Build the mask chain: if a filter exists, evaluate it first and convert the - // boolean result into a Mask for the projection stream. + // Compute the selection mask for this chunk's row range. + let chunk_row_range = this.row_offset..this.row_offset + chunk_len as u64; + let selection_mask = this.row_selection.row_mask(&chunk_row_range).mask().clone(); + + // If all rows are excluded by selection, skip this chunk entirely. + if selection_mask.all_false() { + if let Some(filter_stream) = &mut this.filter_stream { + filter_stream.skip(chunk_len); + } + this.projection_stream.skip(chunk_len); + this.row_offset += chunk_len as u64; + continue; + } + + // Build the mask chain: evaluate filter (if present), then AND with selection. let mask = if let Some(filter_stream) = &mut this.filter_stream { let all_true = MaskFuture::new_true(chunk_len); let filter_fut = match filter_stream.next_chunk(all_true) { @@ -195,12 +212,21 @@ impl Stream for Scan { }; MaskFuture::new(chunk_len, async move { let filter_result = filter_fut.await?; - filter_result.try_to_mask_fill_null_false() + let filter_mask = filter_result.try_to_mask_fill_null_false()?; + if selection_mask.all_true() { + Ok(filter_mask) + } else { + Ok((&filter_mask).bitand(&selection_mask)) + } }) - } else { + } else if selection_mask.all_true() { MaskFuture::new_true(chunk_len) + } else { + MaskFuture::ready(selection_mask) }; + this.row_offset += chunk_len as u64; + // Request the next projection chunk with the computed mask. this.pending = Some(match this.projection_stream.next_chunk(mask) { Ok(fut) => fut, From 0902a4b14eed41ad93cc0d8f055ec4d85a8bcde1 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 14:33:42 -0500 Subject: [PATCH 15/21] Do not build workspace to generate changelog Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 45cbba7871c..d8b67dc9557 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -35,11 +35,13 @@ use vortex::array::ArrayRef; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; use vortex::error::VortexError; +use vortex::error::VortexExpect; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; use vortex::layout::LayoutReader; use vortex::metrics::VortexMetrics; use vortex::scan::ScanBuilder; +use vortex::scan::v2::scan::ScanBuilder2; use vortex::session::VortexSession; use vortex_utils::aliases::dash_map::DashMap; use vortex_utils::aliases::dash_map::Entry; @@ -261,7 +263,14 @@ impl FileOpener for VortexOpener { } }; - let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); + // let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); + let mut scan_builder2 = ScanBuilder2::new( + vxf.footer() + .layout() + .new_reader2(&vxf.segment_source(), &session) + .vortex_expect("Failed to create footer"), + session.clone(), + ); if let Some(extensions) = file.extensions && let Some(vortex_plan) = extensions.downcast_ref::() From 9de162ec0b6544b6de93f0254f336151a6a41ead Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 14:47:20 -0500 Subject: [PATCH 16/21] Do not build workspace to generate changelog Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 44 +++++++++++++--------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index d8b67dc9557..9a9b7043adb 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -38,6 +38,7 @@ use vortex::error::VortexError; use vortex::error::VortexExpect; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; +use vortex::io::session::RuntimeSessionExt; use vortex::layout::LayoutReader; use vortex::metrics::VortexMetrics; use vortex::scan::ScanBuilder; @@ -264,7 +265,7 @@ impl FileOpener for VortexOpener { }; // let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); - let mut scan_builder2 = ScanBuilder2::new( + let mut scan_builder = ScanBuilder2::new( vxf.footer() .layout() .new_reader2(&vxf.segment_source(), &session) @@ -272,19 +273,19 @@ impl FileOpener for VortexOpener { session.clone(), ); - if let Some(extensions) = file.extensions - && let Some(vortex_plan) = extensions.downcast_ref::() - { - scan_builder = vortex_plan.apply_to_builder(scan_builder); - } + // if let Some(extensions) = file.extensions + // && let Some(vortex_plan) = extensions.downcast_ref::() + // { + // scan_builder = vortex_plan.apply_to_builder(scan_builder); + // } if let Some(file_range) = file.range { - scan_builder = apply_byte_range( - file_range, - file.object_meta.size, - vxf.row_count(), - scan_builder, - ); + // scan_builder = apply_byte_range( + // file_range, + // file.object_meta.size, + // vxf.row_count(), + // scan_builder, + // ); } let filter = filter @@ -326,17 +327,24 @@ impl FileOpener for VortexOpener { scan_builder = scan_builder.with_limit(limit); } + let handle = session.handle().clone(); let stream = scan_builder - .with_metrics(metrics) + // .with_metrics(metrics) .with_projection(scan_projection) .with_some_filter(filter) - .with_ordered(has_output_ordering) - .map(move |chunk| { + // .with_ordered(has_output_ordering) + .into_array_stream() + .vortex_expect("Failed to execute Vortex scan") + .then(move |chunk| { let mut ctx = session.create_execution_ctx(); - chunk.execute_record_batch(&stream_schema, &mut ctx) + let stream_schema = stream_schema.clone(); + handle.spawn_cpu(move || { + chunk + .vortex_expect("failed") + .execute_record_batch(&stream_schema, &mut ctx) + }) }) - .into_stream() - .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? + // .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? .map_ok(move |rb| { // We try and slice the stream into respecting datafusion's configured batch size. stream::iter( From 2f3316f181bbed191f580383466eb68ec179db9c Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 16:26:26 -0500 Subject: [PATCH 17/21] Do not build workspace to generate changelog Signed-off-by: Nicholas Gates --- vortex-array/src/lib.rs | 2 + vortex-layout/src/v2/reader.rs | 31 +--- vortex-layout/src/v2/readers/chunked.rs | 109 +++--------- vortex-layout/src/v2/readers/constant.rs | 33 ++-- vortex-layout/src/v2/readers/dict.rs | 47 +++-- vortex-layout/src/v2/readers/flat.rs | 108 ++++-------- vortex-layout/src/v2/readers/scalar_fn.rs | 101 ++++++++--- vortex-layout/src/v2/readers/struct_.rs | 170 ++++++++++++------ vortex-scan/src/api.rs | 9 +- vortex-scan/src/layout.rs | 6 + vortex-scan/src/selection.rs | 2 +- vortex-scan/src/v2/scan.rs | 202 +++++++++++++++++----- 12 files changed, 462 insertions(+), 358 deletions(-) diff --git a/vortex-array/src/lib.rs b/vortex-array/src/lib.rs index d4b2bd5e6fd..ea1ca2f54e3 100644 --- a/vortex-array/src/lib.rs +++ b/vortex-array/src/lib.rs @@ -16,6 +16,7 @@ use std::sync::LazyLock; pub use array::*; +pub use array_future::*; pub use canonical::*; pub use columnar::*; pub use context::*; @@ -31,6 +32,7 @@ pub mod accessor; #[doc(hidden)] pub mod aliases; mod array; +mod array_future; pub mod arrays; pub mod arrow; pub mod buffer; diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index c9ee6122664..1cf40f871d9 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -5,9 +5,7 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use futures::future::BoxFuture; -use vortex_array::ArrayRef; -use vortex_array::MaskFuture; +use vortex_array::ArrayFuture; use vortex_array::expr::Expression; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -17,8 +15,7 @@ pub type ReaderRef = Arc; /// A reader provides an interface for loading data from row-indexed layouts. /// /// Readers have a concrete row count allowing fixed partitions over a known set of rows. Readers -/// are driven by asking for the next chunk size, before providing a [`MaskFuture`] that resolves -/// into a mask of that length. +/// are driven by calling `next_chunk()` which returns an [`ArrayFuture`] with a known length. pub trait Reader: 'static + Send + Sync { /// Downcast the reader to a concrete type. fn as_any(&self) -> &dyn Any; @@ -39,31 +36,21 @@ pub trait Reader: 'static + Send + Sync { pub type ReaderStreamRef = Box; +/// A stream of array chunks from a reader. +/// +/// Each call to `next_chunk()` returns an [`ArrayFuture`] whose length is determined by +/// the reader (not the caller). Returns `None` when the stream is exhausted. pub trait ReaderStream: 'static + Send + Sync { /// The data type of the returned data. fn dtype(&self) -> &DType; - /// The preferred maximum row count for the next chunk. - /// - /// Returns [`None`] if there are no more chunks. - fn next_chunk_len(&self) -> Option; - /// Skip the next `n` rows of the stream. /// /// # Panics /// - /// Panics if `n` is greater than the number of rows remaining in the stream.. + /// Panics if `n` is greater than the number of rows remaining in the stream. fn skip(&mut self, n: usize); - /// Returns the next chunk of data given an input array. - /// - /// The returned chunk must have the same number of rows as the [`Mask::true_count`]. - /// The provided mask will have at most [`next_chunk_len`] rows. - /// - /// The returned future has a `'static` lifetime allowing the calling to drive the stream - /// arbitrarily far without awaiting any data. - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>>; + /// Returns the next chunk of data as an [`ArrayFuture`], or `None` if no more chunks. + fn next_chunk(&mut self) -> Option>; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index fc84dfca30d..fa9e76d2a26 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -6,18 +6,10 @@ use std::collections::VecDeque; use std::ops::Range; use std::sync::Arc; -use futures::future::BoxFuture; -use futures::future::try_join_all; -use moka::future::FutureExt; -use vortex_array::ArrayRef; -use vortex_array::IntoArray; -use vortex_array::MaskFuture; -use vortex_array::arrays::ChunkedArray; +use vortex_array::ArrayFuture; use vortex_array::expr::Expression; use vortex_dtype::DType; use vortex_error::VortexResult; -use vortex_error::vortex_bail; -use vortex_error::vortex_err; use vortex_error::vortex_panic; use crate::v2::reader::Reader; @@ -121,15 +113,12 @@ struct ChunkedReaderStream { } impl ChunkedReaderStream { - /// Ensure we have an active stream pointing at the next non-exhausted chunk. + /// Ensure we have an active stream pointing at a non-exhausted chunk. + /// Returns `Ok(true)` if a stream is ready, `Ok(false)` if no more chunks. fn ensure_active_stream(&mut self) -> VortexResult { loop { - if let Some(ref stream) = self.active_stream { - if stream.next_chunk_len().is_some() { - return Ok(true); - } - // Current stream is exhausted, drop it. - self.active_stream = None; + if self.active_stream.is_some() { + return Ok(true); } // Try to activate the next pending chunk. @@ -146,32 +135,15 @@ impl ReaderStream for ChunkedReaderStream { &self.dtype } - fn next_chunk_len(&self) -> Option { - if let Some(ref stream) = self.active_stream { - let len = stream.next_chunk_len(); - if len.is_some() { - return len; - } - } - // Peek at the next pending chunk's row count as a hint. - self.pending_chunks - .front() - .map(|c| usize::try_from(c.row_range.end - c.row_range.start).unwrap_or(usize::MAX)) - } - fn skip(&mut self, mut n: usize) { while n > 0 { // Try to skip within the active stream. if let Some(ref mut stream) = self.active_stream { - if let Some(chunk_len) = stream.next_chunk_len() { - if n <= chunk_len { - stream.skip(n); - return; - } - stream.skip(chunk_len); - n -= chunk_len; - } - self.active_stream = None; + // We don't know the stream's remaining length without next_chunk_len, + // so we try to skip and if there's no more data, move on. + // For skip, the contract says n must not exceed remaining, so we delegate. + stream.skip(n); + return; } // Skip entire pending chunks without constructing streams. @@ -185,7 +157,6 @@ impl ReaderStream for ChunkedReaderStream { n -= chunk_rows; } else { // Partial skip — construct the stream and skip within it. - // We just checked front() is Some, so pop_front() is guaranteed. match self.pending_chunks.pop_front() { Some(pending) => { let mut stream = match pending.reader.execute(pending.row_range) { @@ -202,55 +173,23 @@ impl ReaderStream for ChunkedReaderStream { } } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { - if !self.ensure_active_stream()? { - vortex_bail!("No more chunks in chunked reader stream"); - } - - let stream = self - .active_stream - .as_mut() - .ok_or_else(|| vortex_err!("Active stream missing after ensure"))?; - let next_len = stream - .next_chunk_len() - .ok_or_else(|| vortex_err!("Active stream unexpectedly exhausted"))?; - - if mask.len() <= next_len { - return stream.next_chunk(mask); - } - - // Mask spans multiple chunks — gather results from each. - let mut remaining_mask = mask; - let mut futs = Vec::new(); - - while !remaining_mask.is_empty() { - if !self.ensure_active_stream()? { - vortex_bail!("Ran out of chunks while processing mask"); + fn next_chunk(&mut self) -> Option> { + loop { + match self.ensure_active_stream() { + Ok(true) => {} + Ok(false) => return None, + Err(e) => return Some(Err(e)), } - let stream = self - .active_stream - .as_mut() - .ok_or_else(|| vortex_err!("Active stream missing after ensure"))?; - let chunk_len = stream - .next_chunk_len() - .ok_or_else(|| vortex_err!("Active stream unexpectedly exhausted"))?; + let stream = self.active_stream.as_mut()?; - let take = chunk_len.min(remaining_mask.len()); - let chunk_mask = remaining_mask.slice(0..take); - remaining_mask = remaining_mask.slice(take..remaining_mask.len()); - - futs.push(stream.next_chunk(chunk_mask)?); - } - - let dtype = self.dtype.clone(); - Ok(async move { - let arrays = try_join_all(futs).await?; - Ok(ChunkedArray::try_new(arrays, dtype)?.into_array()) + match stream.next_chunk() { + Some(result) => return Some(result), + None => { + // Current stream is exhausted, try next chunk. + self.active_stream = None; + } + } } - .boxed()) } } diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index d22bac4e4db..aee131538db 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -5,11 +5,8 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use futures::future::BoxFuture; -use moka::future::FutureExt; -use vortex_array::ArrayRef; +use vortex_array::ArrayFuture; use vortex_array::IntoArray; -use vortex_array::MaskFuture; use vortex_array::arrays::ConstantArray; use vortex_array::expr::Expression; use vortex_array::expr::root; @@ -87,14 +84,6 @@ impl ReaderStream for ConstantReaderStream { self.scalar.dtype() } - fn next_chunk_len(&self) -> Option { - if self.remaining == 0 { - None - } else { - Some(usize::try_from(self.remaining).unwrap_or(usize::MAX)) - } - } - fn skip(&mut self, n: usize) { let n = n as u64; if n > self.remaining { @@ -103,22 +92,22 @@ impl ReaderStream for ConstantReaderStream { self.remaining -= n; } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { + fn next_chunk(&mut self) -> Option> { + if self.remaining == 0 { + return None; + } + + let len = usize::try_from(self.remaining).unwrap_or(usize::MAX); let scalar = self.scalar.clone(); let expression = self.expression.clone(); - self.remaining = self.remaining.saturating_sub(mask.len() as u64); + self.remaining = 0; - Ok(async move { - let mask = mask.await?; - let mut array = ConstantArray::new(scalar, mask.true_count()).into_array(); + Some(Ok(ArrayFuture::new(len, async move { + let mut array = ConstantArray::new(scalar, len).into_array(); if let Some(e) = expression { array = array.apply(&e)?; } Ok(array) - } - .boxed()) + }))) } } diff --git a/vortex-layout/src/v2/readers/dict.rs b/vortex-layout/src/v2/readers/dict.rs index 908ed410f8d..bc70f93e710 100644 --- a/vortex-layout/src/v2/readers/dict.rs +++ b/vortex-layout/src/v2/readers/dict.rs @@ -8,14 +8,16 @@ use std::sync::Arc; use futures::FutureExt; use futures::future::BoxFuture; use futures::future::Shared; +use vortex_array::ArrayFuture; use vortex_array::ArrayRef; use vortex_array::IntoArray; -use vortex_array::MaskFuture; use vortex_array::arrays::DictArray; +use vortex_array::arrays::SharedArray; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::expr::transform::replace; use vortex_dtype::DType; +use vortex_error::VortexError; use vortex_error::VortexResult; use vortex_error::vortex_err; @@ -24,7 +26,7 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; -type SharedVortexResult = Result>; +type SharedVortexResult = Result>; /// A reader that reconstructs dict-encoded arrays from separate values and codes readers. /// @@ -81,15 +83,16 @@ impl Reader for DictReader { // Read the full dictionary values once and share across all chunks. let values_row_count = self.values.row_count(); let mut values_stream = self.values.execute(0..values_row_count)?; - let values_len = values_stream - .next_chunk_len() - .ok_or_else(|| vortex_err!("Dict values stream is empty"))?; - let values_mask = MaskFuture::new_true(values_len); - let values_fut: Shared>> = values_stream - .next_chunk(values_mask)? - .map(|r| r.map_err(Arc::new)) - .boxed() - .shared(); + let values_array_future = values_stream + .next_chunk() + .ok_or_else(|| vortex_err!("Dict values stream is empty"))??; + let values_fut: Shared>> = async move { + let array = values_array_future.await?; + Ok(SharedArray::new(array).into_array()) + } + .map(|r: VortexResult| r.map_err(Arc::new)) + .boxed() + .shared(); let codes_stream = self.codes.execute(row_range)?; @@ -114,31 +117,27 @@ impl ReaderStream for DictReaderStream { &self.dtype } - fn next_chunk_len(&self) -> Option { - self.codes_stream.next_chunk_len() - } - fn skip(&mut self, n: usize) { self.codes_stream.skip(n); } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { + fn next_chunk(&mut self) -> Option> { + let codes_future = match self.codes_stream.next_chunk()? { + Ok(f) => f, + Err(e) => return Some(Err(e)), + }; let values_fut = self.values_fut.clone(); - let codes_fut = self.codes_stream.next_chunk(mask)?; let expression = self.expression.clone(); + let len = codes_future.len(); - Ok(async move { + Some(Ok(ArrayFuture::new(len, async move { let values = values_fut.await.map_err(|e| vortex_err!("{e}"))?; - let codes = codes_fut.await?; + let codes = codes_future.await?; let mut array = DictArray::try_new(codes, values)?.into_array(); if let Some(expr) = expression { array = array.apply(&expr)?; } Ok(array) - } - .boxed()) + }))) } } diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index a71e2f1b00f..c4b8f892011 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -9,11 +9,8 @@ use futures::FutureExt; use futures::future::BoxFuture; use futures::future::Shared; use vortex_array::ArrayContext; +use vortex_array::ArrayFuture; use vortex_array::ArrayRef; -use vortex_array::Canonical; -use vortex_array::IntoArray; -use vortex_array::MaskFuture; -use vortex_array::buffer::BufferHandle; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::expr::transform::replace; @@ -34,7 +31,7 @@ use crate::v2::reader::ReaderRef; use crate::v2::reader::ReaderStream; use crate::v2::reader::ReaderStreamRef; -type SharedSegmentFuture = Shared>>; +type SharedArrayFuture = Shared>>; /// A leaf reader that reads a single flat segment. /// @@ -123,21 +120,30 @@ impl Reader for FlatReader { ); } - // Request the segment once and share across all next_chunk calls. - let segment_fut = self - .segment_source - .request(self.segment_id) - .map(|r| r.map_err(Arc::new)) - .boxed() - .shared(); + // Decode the array once and share the result across all next_chunk calls. + let segment_source = self.segment_source.clone(); + let segment_id = self.segment_id; + let array_tree = self.array_tree.clone(); + let decode_dtype = self.decode_dtype.clone(); + let row_count = self.len; + let ctx = self.ctx.clone(); + let registry = self.registry.clone(); + let array_fut = async move { + let segment = segment_source.request(segment_id).await?; + let parts = if let Some(array_tree) = array_tree { + ArrayParts::from_flatbuffer_and_segment(array_tree, segment)? + } else { + ArrayParts::try_from(segment)? + }; + parts.decode(&decode_dtype, row_count, &ctx, ®istry) + } + .map(|r| r.map_err(Arc::new)) + .boxed() + .shared(); Ok(Box::new(FlatReaderStream { dtype: self.dtype.clone(), - decode_dtype: self.decode_dtype.clone(), - segment_fut, - array_tree: self.array_tree.clone(), - ctx: self.ctx.clone(), - registry: self.registry.clone(), + array_fut, expression: self.expression.clone(), row_count: self.len, offset: start, @@ -148,11 +154,7 @@ impl Reader for FlatReader { struct FlatReaderStream { dtype: DType, - decode_dtype: DType, - segment_fut: SharedSegmentFuture, - array_tree: Option, - ctx: ArrayContext, - registry: ArrayRegistry, + array_fut: SharedArrayFuture, expression: Option, row_count: usize, offset: usize, @@ -164,14 +166,6 @@ impl ReaderStream for FlatReaderStream { &self.dtype } - fn next_chunk_len(&self) -> Option { - if self.remaining == 0 { - None - } else { - Some(self.remaining) - } - } - fn skip(&mut self, n: usize) { if n > self.remaining { vortex_panic!("Cannot skip {} rows, only {} remaining", n, self.remaining); @@ -180,69 +174,35 @@ impl ReaderStream for FlatReaderStream { self.remaining -= n; } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { - if mask.len() > self.remaining { - vortex_bail!( - "Mask length {} exceeds remaining rows {}", - mask.len(), - self.remaining - ); + fn next_chunk(&mut self) -> Option> { + if self.remaining == 0 { + return None; } - let segment_fut = self.segment_fut.clone(); - let array_tree = self.array_tree.clone(); - let ctx = self.ctx.clone(); - let registry = self.registry.clone(); + let array_fut = self.array_fut.clone(); let row_count = self.row_count; let offset = self.offset; - let len = mask.len(); + let len = self.remaining; let expression = self.expression.clone(); - let dtype = self.dtype.clone(); - let decode_dtype = self.decode_dtype.clone(); self.offset += len; - self.remaining -= len; - - Ok(async move { - // Await the mask first — if all-false, skip I/O entirely. - // TODO(ngates): should we race this against the segment future? - let mask = mask.await?; - if mask.true_count() == 0 { - return Ok(Canonical::empty(&dtype).into_array()); - } + self.remaining = 0; - // Await the shared segment future (I/O is issued once, shared across chunks). - let segment = segment_fut.await?; - let parts = if let Some(array_tree) = array_tree { - // Use the pre-stored flatbuffer from layout metadata combined with segment buffers. - ArrayParts::from_flatbuffer_and_segment(array_tree, segment)? - } else { - // Parse the flatbuffer from the segment itself. - ArrayParts::try_from(segment)? - }; - - let mut array = parts.decode(&decode_dtype, row_count, &ctx, ®istry)?; + Some(Ok(ArrayFuture::new(len, async move { + // Await the shared array future (decoded once, shared across chunks). + let mut array: ArrayRef = array_fut.await?; // Slice to the requested row range within the segment. if offset > 0 || len < row_count { array = array.slice(offset..offset + len)?; } - // Filter using the mask. - if !mask.all_true() { - array = array.filter(mask)?; - } - // Apply any accumulated expression. if let Some(expr) = expression { array = array.apply(&expr)?; } Ok(array) - } - .boxed()) + }))) } } diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 07ee070c939..e527a75cc16 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -5,12 +5,10 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use futures::future::BoxFuture; use futures::future::try_join_all; use itertools::Itertools; -use vortex_array::ArrayRef; +use vortex_array::ArrayFuture; use vortex_array::IntoArray; -use vortex_array::MaskFuture; use vortex_array::arrays::ScalarFnArray; use vortex_array::expr::Expression; use vortex_array::expr::Literal; @@ -114,10 +112,12 @@ impl Reader for ScalarFnReader { .map(|child| child.execute(row_range.clone())) .collect::>>()?; + let num_inputs = input_streams.len(); Ok(Box::new(ScalarFnArrayStream { dtype: self.dtype.clone(), scalar_fn: self.scalar_fn.clone(), input_streams, + input_buffers: vec![None; num_inputs], })) } } @@ -126,6 +126,20 @@ struct ScalarFnArrayStream { dtype: DType, scalar_fn: ScalarFn, input_streams: Vec, + input_buffers: Vec>, +} + +impl ScalarFnArrayStream { + /// Get the next ArrayFuture for an input stream, taking from the buffer first. + fn next_for_input( + stream: &mut ReaderStreamRef, + buffer: &mut Option, + ) -> Option> { + if let Some(buffered) = buffer.take() { + return Some(Ok(buffered)); + } + stream.next_chunk() + } } impl ReaderStream for ScalarFnArrayStream { @@ -133,38 +147,73 @@ impl ReaderStream for ScalarFnArrayStream { &self.dtype } - fn next_chunk_len(&self) -> Option { - self.input_streams - .iter() - .map(|s| s.next_chunk_len()) - .min() - .flatten() - } - fn skip(&mut self, n: usize) { - for stream in &mut self.input_streams { - stream.skip(n); + for (stream, buffer) in self + .input_streams + .iter_mut() + .zip(self.input_buffers.iter_mut()) + { + let mut remaining = n; + if let Some(buf) = buffer.take() { + if remaining < buf.len() { + *buffer = Some(buf.slice(remaining..buf.len())); + continue; + } + remaining -= buf.len(); + } + if remaining > 0 { + stream.skip(remaining); + } } } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { - let scalar_fn = self.scalar_fn.clone(); - let futs = self + fn next_chunk(&mut self) -> Option> { + // Collect an ArrayFuture for each input. + let mut all_futures: Vec = Vec::with_capacity(self.input_streams.len()); + + for (stream, buffer) in self .input_streams .iter_mut() - .map(|s| s.next_chunk(mask.clone())) - .collect::>>()?; + .zip(self.input_buffers.iter_mut()) + { + let future = match Self::next_for_input(stream, buffer) { + Some(Ok(f)) => f, + Some(Err(e)) => return Some(Err(e)), + None => return None, + }; + all_futures.push(future); + } + + if all_futures.is_empty() { + return None; + } + + // Find the minimum length. + let min_len = all_futures.iter().map(|f| f.len()).min().unwrap_or(0); + if min_len == 0 { + return None; + } + + // For inputs with len > min_len, buffer the remainder and slice. + let mut chunk_futures: Vec = Vec::with_capacity(all_futures.len()); + for (idx, future) in all_futures.into_iter().enumerate() { + if future.len() > min_len { + let remainder = future.slice(min_len..future.len()); + let chunk = future.slice(0..min_len); + self.input_buffers[idx] = Some(remainder); + chunk_futures.push(chunk); + } else { + chunk_futures.push(future); + } + } - Ok(Box::pin(async move { - let input_arrays = try_join_all(futs).await?; - let len = input_arrays.first().map(|a| a.len()).unwrap_or(0); - let array = ScalarFnArray::try_new(scalar_fn, input_arrays, len)?.into_array(); + let scalar_fn = self.scalar_fn.clone(); + Some(Ok(ArrayFuture::new(min_len, async move { + let input_arrays = try_join_all(chunk_futures).await?; + let array = ScalarFnArray::try_new(scalar_fn, input_arrays, min_len)?.into_array(); let array = array.optimize()?; Ok(array) - })) + }))) } } diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index b6b2d626534..cdaf247e55b 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -5,14 +5,10 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use futures::future::BoxFuture; use futures::future::try_join_all; -use futures::try_join; use itertools::Itertools; -use moka::future::FutureExt; -use vortex_array::ArrayRef; +use vortex_array::ArrayFuture; use vortex_array::IntoArray; -use vortex_array::MaskFuture; use vortex_array::arrays::StructArray; use vortex_array::expr::Expression; use vortex_array::expr::GetItem; @@ -135,10 +131,13 @@ impl Reader for StructReader { .map(|field| field.execute(row_range.clone())) .collect::>>()?; + let num_fields = field_streams.len(); Ok(Box::new(StructReaderStream { dtype: self.dtype.clone(), validity: validity_stream, fields: field_streams, + validity_buffer: None, + field_buffers: vec![None; num_fields], })) } } @@ -147,6 +146,38 @@ struct StructReaderStream { dtype: DType, validity: Option, fields: Vec, + validity_buffer: Option, + field_buffers: Vec>, +} + +impl StructReaderStream { + /// Get the next ArrayFuture for a child stream, taking from the buffer first. + fn next_for_child( + stream: &mut ReaderStreamRef, + buffer: &mut Option, + ) -> Option> { + if let Some(buffered) = buffer.take() { + return Some(Ok(buffered)); + } + stream.next_chunk() + } +} + +/// Skip `n` rows from an optional stream, consuming from the buffer first. +fn skip_child(stream: Option<&mut ReaderStreamRef>, buffer: &mut Option, n: usize) { + let mut remaining = n; + if let Some(buf) = buffer.take() { + if remaining < buf.len() { + *buffer = Some(buf.slice(remaining..buf.len())); + return; + } + remaining -= buf.len(); + } + if remaining > 0 + && let Some(stream) = stream + { + stream.skip(remaining); + } } impl ReaderStream for StructReaderStream { @@ -154,69 +185,98 @@ impl ReaderStream for StructReaderStream { &self.dtype } - fn next_chunk_len(&self) -> Option { - let field_min = self - .fields - .iter() - .map(|s| s.next_chunk_len()) - .min() - .flatten(); - match (&self.validity, field_min) { - (Some(v), Some(f)) => v.next_chunk_len().map(|vl| vl.min(f)), - (Some(v), None) => v.next_chunk_len(), - (None, f) => f, + fn skip(&mut self, n: usize) { + // Skip n rows from each child (validity + fields), consuming from buffers first. + skip_child(self.validity.as_mut(), &mut self.validity_buffer, n); + for (field_stream, field_buf) in self.fields.iter_mut().zip(self.field_buffers.iter_mut()) { + skip_child(Some(field_stream), field_buf, n); } } - fn skip(&mut self, n: usize) { - if let Some(validity) = &mut self.validity { - validity.skip(n); + fn next_chunk(&mut self) -> Option> { + // Collect an ArrayFuture for each child (validity + fields). + let mut all_futures: Vec = Vec::with_capacity(1 + self.fields.len()); + + // Validity + let has_validity = self.validity.is_some(); + if let Some(ref mut validity_stream) = self.validity { + let validity_future = + match Self::next_for_child(validity_stream, &mut self.validity_buffer) { + Some(Ok(f)) => f, + Some(Err(e)) => return Some(Err(e)), + None => return None, + }; + all_futures.push(validity_future); } - for field in &mut self.fields { - field.skip(n); + + // Fields + for (field_stream, field_buf) in self.fields.iter_mut().zip(self.field_buffers.iter_mut()) { + let field_future = match Self::next_for_child(field_stream, field_buf) { + Some(Ok(f)) => f, + Some(Err(e)) => return Some(Err(e)), + None => return None, + }; + all_futures.push(field_future); } - } - fn next_chunk( - &mut self, - mask: MaskFuture, - ) -> VortexResult>> { - let struct_fields = self + if all_futures.is_empty() { + return None; + } + + // Find the minimum length. + let min_len = all_futures.iter().map(|f| f.len()).min().unwrap_or(0); + if min_len == 0 { + return None; + } + + // For children with len > min_len, buffer the remainder and slice. + let mut chunk_futures: Vec = Vec::with_capacity(all_futures.len()); + + for (buf_idx, future) in all_futures.into_iter().enumerate() { + if future.len() > min_len { + // Buffer the remainder. + let remainder = future.slice(min_len..future.len()); + let chunk = future.slice(0..min_len); + + if buf_idx == 0 && has_validity { + self.validity_buffer = Some(remainder); + } else { + let field_idx = if has_validity { buf_idx - 1 } else { buf_idx }; + self.field_buffers[field_idx] = Some(remainder); + } + + chunk_futures.push(chunk); + } else { + chunk_futures.push(future); + } + } + + let struct_fields = match self .dtype .as_struct_fields_opt() - .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))? - .clone(); + .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype")) + { + Ok(f) => f.clone(), + Err(e) => return Some(Err(e)), + }; let nullability = self.dtype.nullability(); - let validity_fut = self - .validity - .as_mut() - .map(|v| v.next_chunk(mask.clone())) - .transpose()?; - let fields = self - .fields - .iter_mut() - .map(|s| s.next_chunk(mask.clone())) - .collect::>>()?; - Ok(async move { - let fields = try_join_all(fields); - let (fields, mask) = try_join!(fields, mask)?; - let validity = if let Some(validity_fut) = validity_fut { - let validity_array = validity_fut.await?; - Validity::Array(validity_array) + Some(Ok(ArrayFuture::new(min_len, async move { + // Split off validity future from field futures. + let arrays = try_join_all(chunk_futures).await?; + + let (validity, fields) = if has_validity { + let validity_array = arrays[0].clone(); + let fields = arrays[1..].to_vec(); + (Validity::Array(validity_array), fields) } else { - nullability.into() + (nullability.into(), arrays) }; + Ok( - StructArray::try_new_with_dtype( - fields, - struct_fields, - mask.true_count(), - validity, - )? - .into_array(), + StructArray::try_new_with_dtype(fields, struct_fields, min_len, validity)? + .into_array(), ) - } - .boxed()) + }))) } } diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index bbcfccba526..bc6115b5933 100644 --- a/vortex-scan/src/api.rs +++ b/vortex-scan/src/api.rs @@ -20,6 +20,7 @@ //! example which encodings it knows about. use std::any::Any; +use std::ops::Range; use std::sync::Arc; use async_trait::async_trait; @@ -30,6 +31,8 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_session::VortexSession; +use crate::Selection; + /// Create a Vortex source from serialized configuration. /// /// Providers can be registered with Vortex under a specific @@ -81,6 +84,10 @@ pub struct ScanRequest { pub filter: Option, /// Optional limit on the number of rows to scan. pub limit: Option, + /// Optional row range to scan within the data source. + pub row_range: Option>, + /// Row selection to apply within the row range. + pub row_selection: Selection, } /// A boxed data source scan. @@ -123,7 +130,7 @@ pub trait Split: 'static + Send { } /// An estimate that can be exact, an upper bound, or unknown. -#[derive(Default)] +#[derive(Debug, Clone, Default)] pub enum Estimate { /// The exact value. Exact(T), diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 183a023bd2d..50ce37598ae 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -65,6 +65,12 @@ impl DataSource for LayoutReaderDataSource { builder = builder.with_limit(limit); } + if let Some(row_range) = scan_request.row_range { + builder = builder.with_row_range(row_range); + } + + builder = builder.with_selection(scan_request.row_selection); + let scan = builder.prepare()?; let dtype = scan.dtype().clone(); let splits = scan.execute(None)?; diff --git a/vortex-scan/src/selection.rs b/vortex-scan/src/selection.rs index d2fb46d4bb0..7aee9b71c9d 100644 --- a/vortex-scan/src/selection.rs +++ b/vortex-scan/src/selection.rs @@ -12,7 +12,7 @@ use crate::row_mask::RowMask; /// A selection identifies a set of rows to include in the scan (in addition to applying any /// filter predicates). -#[derive(Default, Clone)] +#[derive(Debug, Default, Clone)] pub enum Selection { /// No selection, all rows are included. #[default] diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 87e360dcef6..e6be50e0335 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -9,8 +9,8 @@ use std::task::Poll; use futures::Stream; use futures::future::BoxFuture; +use vortex_array::ArrayFuture; use vortex_array::ArrayRef; -use vortex_array::MaskFuture; use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::stream::ArrayStream; @@ -19,6 +19,7 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_layout::v2::reader::ReaderRef; use vortex_layout::v2::reader::ReaderStreamRef; +use vortex_mask::Mask; use vortex_session::VortexSession; use crate::Selection; @@ -113,6 +114,8 @@ impl ScanBuilder2 { rows_produced: 0, row_selection: self.row_selection, row_offset, + filter_buffer: None, + projection_buffer: None, }) } } @@ -126,6 +129,80 @@ struct Scan { rows_produced: u64, row_selection: Selection, row_offset: u64, + filter_buffer: Option, + projection_buffer: Option, +} + +impl Scan { + /// Get the next projection chunk, consuming from the buffer first. + fn next_projection_chunk(&mut self) -> Option> { + if let Some(buffered) = self.projection_buffer.take() { + return Some(Ok(buffered)); + } + self.projection_stream.next_chunk() + } + + /// Get the next filter chunk, consuming from the buffer first. + fn next_filter_chunk( + stream: &mut ReaderStreamRef, + buffer: &mut Option, + ) -> Option> { + if let Some(buffered) = buffer.take() { + return Some(Ok(buffered)); + } + stream.next_chunk() + } + + /// Collect filter chunks covering exactly `n` rows. + /// Returns a vec of ArrayFutures whose total len == n. + fn collect_filter_chunks(&mut self, n: usize) -> Option>> { + let filter_stream = self.filter_stream.as_mut()?; + let mut chunks = Vec::new(); + let mut remaining = n; + + while remaining > 0 { + let chunk = match Self::next_filter_chunk(filter_stream, &mut self.filter_buffer) { + Some(Ok(f)) => f, + Some(Err(e)) => return Some(Err(e)), + None => { + return Some(Err(vortex_error::vortex_err!( + "Filter stream exhausted before covering {} rows", + n + ))); + } + }; + + if chunk.len() <= remaining { + remaining -= chunk.len(); + chunks.push(chunk); + } else { + // Buffer the remainder. + let used = chunk.slice(0..remaining); + self.filter_buffer = Some(chunk.slice(remaining..chunk.len())); + remaining = 0; + chunks.push(used); + } + } + + Some(Ok(chunks)) + } + + /// Skip `n` rows in the filter stream, consuming from the buffer first. + fn skip_filter(&mut self, n: usize) { + let mut remaining = n; + if let Some(buf) = self.filter_buffer.take() { + if remaining < buf.len() { + self.filter_buffer = Some(buf.slice(remaining..buf.len())); + return; + } + remaining -= buf.len(); + } + if remaining > 0 + && let Some(filter_stream) = &mut self.filter_stream + { + filter_stream.skip(remaining); + } + } } impl ArrayStream for Scan { @@ -163,75 +240,104 @@ impl Stream for Scan { return Poll::Ready(None); } - // Determine the next chunk size from the projection stream. - let proj_chunk_len = this.projection_stream.next_chunk_len(); - - // If a filter stream exists, synchronize chunk sizes. - let chunk_len = if let Some(filter_stream) = &this.filter_stream { - match (proj_chunk_len, filter_stream.next_chunk_len()) { - (Some(p), Some(f)) => Some(p.min(f)), - _ => None, - } - } else { - proj_chunk_len + // Get the next projection chunk. + let proj_future = match this.next_projection_chunk() { + Some(Ok(f)) => f, + Some(Err(e)) => return Poll::Ready(Some(Err(e))), + None => return Poll::Ready(None), }; - let Some(mut chunk_len) = chunk_len else { - return Poll::Ready(None); - }; + let mut chunk_len = proj_future.len(); - // Limit the chunk size to avoid exceeding the row limit. + // Apply limit: if remaining < chunk_len, slice the projection future. if let Some(limit) = this.limit { let remaining = usize::try_from(limit - this.rows_produced).unwrap_or(usize::MAX); - chunk_len = chunk_len.min(remaining); - if chunk_len == 0 { - return Poll::Ready(None); + if remaining < chunk_len { + // Buffer the remainder and use only what we need. + this.projection_buffer = Some(proj_future.slice(remaining..chunk_len)); + chunk_len = remaining; } } + let proj_future = if chunk_len < proj_future.len() { + proj_future.slice(0..chunk_len) + } else { + proj_future + }; + + if chunk_len == 0 { + return Poll::Ready(None); + } + // Compute the selection mask for this chunk's row range. let chunk_row_range = this.row_offset..this.row_offset + chunk_len as u64; let selection_mask = this.row_selection.row_mask(&chunk_row_range).mask().clone(); - // If all rows are excluded by selection, skip this chunk entirely. + // If all rows are excluded by selection, skip this chunk entirely (no I/O). if selection_mask.all_false() { - if let Some(filter_stream) = &mut this.filter_stream { - filter_stream.skip(chunk_len); - } - this.projection_stream.skip(chunk_len); + this.skip_filter(chunk_len); this.row_offset += chunk_len as u64; continue; } - // Build the mask chain: evaluate filter (if present), then AND with selection. - let mask = if let Some(filter_stream) = &mut this.filter_stream { - let all_true = MaskFuture::new_true(chunk_len); - let filter_fut = match filter_stream.next_chunk(all_true) { - Ok(fut) => fut, - Err(e) => return Poll::Ready(Some(Err(e))), + this.row_offset += chunk_len as u64; + + // Build the pending future: await projection, apply filter + selection. + if this.filter_stream.is_some() { + let filter_chunks = match this.collect_filter_chunks(chunk_len) { + Some(Ok(chunks)) => chunks, + Some(Err(e)) => return Poll::Ready(Some(Err(e))), + None => { + return Poll::Ready(Some(Err(vortex_error::vortex_err!( + "Filter stream missing" + )))); + } }; - MaskFuture::new(chunk_len, async move { - let filter_result = filter_fut.await?; - let filter_mask = filter_result.try_to_mask_fill_null_false()?; - if selection_mask.all_true() { - Ok(filter_mask) + + this.pending = Some(Box::pin(async move { + // Await filter chunks and combine into a single mask. + let mut filter_masks: Vec = Vec::with_capacity(filter_chunks.len()); + for filter_chunk in filter_chunks { + let filter_array = filter_chunk.await?; + filter_masks.push(filter_array.try_to_mask_fill_null_false()?); + } + + let filter_mask = if filter_masks.len() == 1 { + filter_masks + .into_iter() + .next() + .unwrap_or_else(|| Mask::new_true(0)) + } else { + Mask::concat(filter_masks.iter())? + }; + + // AND with selection mask. + let mask = if selection_mask.all_true() { + filter_mask + } else { + (&filter_mask).bitand(&selection_mask) + }; + + // Await projection. + let array = proj_future.await?; + + // Filter the projection array. + if mask.all_true() { + Ok(array) } else { - Ok((&filter_mask).bitand(&selection_mask)) + array.filter(mask) } - }) + })); } else if selection_mask.all_true() { - MaskFuture::new_true(chunk_len) + // No filter, no selection masking — just await projection. + this.pending = Some(Box::pin(proj_future)); } else { - MaskFuture::ready(selection_mask) - }; - - this.row_offset += chunk_len as u64; - - // Request the next projection chunk with the computed mask. - this.pending = Some(match this.projection_stream.next_chunk(mask) { - Ok(fut) => fut, - Err(e) => return Poll::Ready(Some(Err(e))), - }); + // No filter, but selection mask needs to be applied. + this.pending = Some(Box::pin(async move { + let array = proj_future.await?; + array.filter(selection_mask) + })); + } // Loop back to poll the newly created future. } From e5b4cbbfe4b15ed4079b77dd0016b517c1d7f3ea Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 16:26:33 -0500 Subject: [PATCH 18/21] Do not build workspace to generate changelog Signed-off-by: Nicholas Gates --- vortex-array/src/array_future.rs | 85 ++++++++++++++++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 vortex-array/src/array_future.rs diff --git a/vortex-array/src/array_future.rs b/vortex-array/src/array_future.rs new file mode 100644 index 00000000000..2cfcf25f67e --- /dev/null +++ b/vortex-array/src/array_future.rs @@ -0,0 +1,85 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::future::Future; +use std::ops::Range; +use std::sync::Arc; + +use futures::FutureExt; +use futures::TryFutureExt; +use futures::future::BoxFuture; +use futures::future::Shared; +use vortex_error::SharedVortexResult; +use vortex_error::VortexError; +use vortex_error::VortexResult; +use vortex_error::vortex_panic; + +use crate::ArrayRef; + +/// A future that resolves to an array with a known length. +#[derive(Clone)] +pub struct ArrayFuture { + inner: Shared>>, + len: usize, +} + +impl ArrayFuture { + /// Create a new `ArrayFuture` from a future that returns an array. + pub fn new(len: usize, fut: F) -> Self + where + F: Future> + Send + 'static, + { + Self { + inner: fut + .inspect(move |r| { + if let Ok(array) = r + && array.len() != len { + vortex_panic!("ArrayFuture created with future that returned array of incorrect length (expected {}, got {})", len, array.len()); + } + }) + .map_err(Arc::new) + .boxed() + .shared(), + len, + } + } + + /// Create an `ArrayFuture` from an already-resolved array. + pub fn ready(array: ArrayRef) -> Self { + let len = array.len(); + Self::new(len, async move { Ok(array) }) + } + + /// Returns the length of the array. + pub fn len(&self) -> usize { + self.len + } + + /// Returns true if the array is empty. + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + /// Create an `ArrayFuture` that resolves to a slice of the original array. + pub fn slice(&self, range: Range) -> Self { + let inner = self.inner.clone(); + let parent_len = self.len; + Self::new(range.len(), async move { + let array = inner.await?; + debug_assert!(range.end <= parent_len, "slice range out of bounds"); + let _ = parent_len; + array.slice(range) + }) + } +} + +impl Future for ArrayFuture { + type Output = VortexResult; + + fn poll( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll { + self.inner.poll_unpin(cx).map_err(VortexError::from) + } +} From bf17c5ade6bb5b1ff4e976723d99d7e595ba4496 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 18:22:41 -0500 Subject: [PATCH 19/21] Add display Signed-off-by: Nicholas Gates --- Cargo.lock | 1 - vortex-layout/src/v2/reader.rs | 15 +++++++++++++ vortex-layout/src/v2/readers/chunked.rs | 16 ++++++++++++++ vortex-layout/src/v2/readers/constant.rs | 10 +++++++++ vortex-layout/src/v2/readers/dict.rs | 23 +++++++++++++++++++ vortex-layout/src/v2/readers/flat.rs | 13 +++++++++++ vortex-layout/src/v2/readers/scalar_fn.rs | 16 ++++++++++++++ vortex-layout/src/v2/readers/struct_.rs | 27 +++++++++++++++++++++++ vortex-layout/src/v2/readers/zoned.rs | 22 ++++++++++++++++++ vortex-scalar/Cargo.toml | 1 - vortex-scan/src/v2/scan.rs | 11 ++++++++- 11 files changed, 152 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7176b893f72..695e9cacdad 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11005,7 +11005,6 @@ dependencies = [ "vortex-buffer", "vortex-dtype", "vortex-error", - "vortex-mask", "vortex-proto", "vortex-session", "vortex-utils", diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index 1cf40f871d9..c570f1d0820 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -2,9 +2,12 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use std::any::Any; +use std::fmt; +use std::fmt::Display; use std::ops::Range; use std::sync::Arc; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::expr::Expression; use vortex_dtype::DType; @@ -32,6 +35,18 @@ pub trait Reader: 'static + Send + Sync { /// Creates a scan over the given row range of the reader. fn execute(&self, row_range: Range) -> VortexResult; + + /// Build a tree representation of this reader for display purposes. + fn display_tree(&self) -> Tree; +} + +/// A convenience wrapper for displaying a reader tree. +pub struct DisplayReaderTree<'a>(pub &'a dyn Reader); + +impl Display for DisplayReaderTree<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0.display_tree()) + } } pub type ReaderStreamRef = Box; diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index fa9e76d2a26..0cf3e0b0d96 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -6,6 +6,7 @@ use std::collections::VecDeque; use std::ops::Range; use std::sync::Arc; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::expr::Expression; use vortex_dtype::DType; @@ -98,6 +99,21 @@ impl Reader for ChunkedReader { active_stream: None, })) } + + fn display_tree(&self) -> Tree { + let label = format!( + "Chunked({}, rows={}, chunks={})", + self.dtype, + self.row_count, + self.chunks.len() + ); + let mut tree = Tree::new(label); + for (i, chunk) in self.chunks.iter().enumerate() { + let child = chunk.display_tree(); + tree.push(Tree::new(format!("[{}]: {}", i, child.root)).with_leaves(child.leaves)); + } + tree + } } struct PendingChunk { diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index aee131538db..86201e26b7b 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -5,6 +5,7 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::IntoArray; use vortex_array::arrays::ConstantArray; @@ -71,6 +72,15 @@ impl Reader for ConstantReader { expression: self.expression.clone(), })) } + + fn display_tree(&self) -> Tree { + Tree::new(format!( + "Constant({}, rows={}, value={})", + self.scalar.dtype(), + self.row_count, + self.scalar + )) + } } struct ConstantReaderStream { diff --git a/vortex-layout/src/v2/readers/dict.rs b/vortex-layout/src/v2/readers/dict.rs index bc70f93e710..3a34cd7e70b 100644 --- a/vortex-layout/src/v2/readers/dict.rs +++ b/vortex-layout/src/v2/readers/dict.rs @@ -8,6 +8,7 @@ use std::sync::Arc; use futures::FutureExt; use futures::future::BoxFuture; use futures::future::Shared; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::ArrayRef; use vortex_array::IntoArray; @@ -103,6 +104,28 @@ impl Reader for DictReader { expression: self.expression.clone(), })) } + + fn display_tree(&self) -> Tree { + let mut label = format!("Dict({}, rows={}", self.dtype, self.codes.row_count()); + if let Some(expr) = &self.expression { + label.push_str(&format!(", expr={}", expr)); + } + label.push(')'); + + let mut tree = Tree::new(label); + + let values_child = self.values.display_tree(); + tree.push( + Tree::new(format!("values: {}", values_child.root)).with_leaves(values_child.leaves), + ); + + let codes_child = self.codes.display_tree(); + tree.push( + Tree::new(format!("codes: {}", codes_child.root)).with_leaves(codes_child.leaves), + ); + + tree + } } struct DictReaderStream { diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index c4b8f892011..4ad7aa4ecfb 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -8,6 +8,7 @@ use std::sync::Arc; use futures::FutureExt; use futures::future::BoxFuture; use futures::future::Shared; +use termtree::Tree; use vortex_array::ArrayContext; use vortex_array::ArrayFuture; use vortex_array::ArrayRef; @@ -150,6 +151,18 @@ impl Reader for FlatReader { remaining: end - start, })) } + + fn display_tree(&self) -> Tree { + let mut label = format!( + "Flat({}, rows={}, segment={}", + self.dtype, self.len, self.segment_id + ); + if let Some(expr) = &self.expression { + label.push_str(&format!(", expr={}", expr)); + } + label.push(')'); + Tree::new(label) + } } struct FlatReaderStream { diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index e527a75cc16..4d0fbdc0abd 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -7,6 +7,7 @@ use std::sync::Arc; use futures::future::try_join_all; use itertools::Itertools; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::IntoArray; use vortex_array::arrays::ScalarFnArray; @@ -120,6 +121,21 @@ impl Reader for ScalarFnReader { input_buffers: vec![None; num_inputs], })) } + + fn display_tree(&self) -> Tree { + let label = format!( + "ScalarFn({}, rows={}, fn={})", + self.dtype, self.row_count, self.scalar_fn + ); + let mut tree = Tree::new(label); + for (i, child) in self.children.iter().enumerate() { + let child_tree = child.display_tree(); + tree.push( + Tree::new(format!("[{}]: {}", i, child_tree.root)).with_leaves(child_tree.leaves), + ); + } + tree + } } struct ScalarFnArrayStream { diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index cdaf247e55b..12f423c7795 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -7,6 +7,7 @@ use std::sync::Arc; use futures::future::try_join_all; use itertools::Itertools; +use termtree::Tree; use vortex_array::ArrayFuture; use vortex_array::IntoArray; use vortex_array::arrays::StructArray; @@ -140,6 +141,32 @@ impl Reader for StructReader { field_buffers: vec![None; num_fields], })) } + + fn display_tree(&self) -> Tree { + let label = format!("Struct({}, rows={})", self.dtype, self.row_count); + let mut tree = Tree::new(label); + + // Add field children with names from the struct dtype. + if let Some(struct_fields) = self.dtype.as_struct_fields_opt() { + for (name, field_reader) in struct_fields.names().iter().zip(self.fields.iter()) { + let child = field_reader.display_tree(); + tree.push(Tree::new(format!("{}: {}", name, child.root)).with_leaves(child.leaves)); + } + } else { + for (i, field_reader) in self.fields.iter().enumerate() { + let child = field_reader.display_tree(); + tree.push(Tree::new(format!("[{}]: {}", i, child.root)).with_leaves(child.leaves)); + } + } + + // Add validity child if present. + if let Some(validity) = &self.validity { + let child = validity.display_tree(); + tree.push(Tree::new(format!("validity: {}", child.root)).with_leaves(child.leaves)); + } + + tree + } } struct StructReaderStream { diff --git a/vortex-layout/src/v2/readers/zoned.rs b/vortex-layout/src/v2/readers/zoned.rs index 3aeceed4ce4..56a4b04c84e 100644 --- a/vortex-layout/src/v2/readers/zoned.rs +++ b/vortex-layout/src/v2/readers/zoned.rs @@ -5,6 +5,7 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; +use termtree::Tree; use vortex_array::expr::Expression; use vortex_array::expr::stats::Stat; use vortex_dtype::DType; @@ -75,4 +76,25 @@ impl Reader for ZonedReader { // with zones that can't match the filter skipped. self.data.execute(row_range) } + + fn display_tree(&self) -> Tree { + let label = format!( + "Zoned({}, rows={}, zone_len={})", + self.data.dtype(), + self.data.row_count(), + self.zone_len + ); + let mut tree = Tree::new(label); + + let data_child = self.data.display_tree(); + tree.push(Tree::new(format!("data: {}", data_child.root)).with_leaves(data_child.leaves)); + + let zone_map_child = self.zone_map.display_tree(); + tree.push( + Tree::new(format!("zone_map: {}", zone_map_child.root)) + .with_leaves(zone_map_child.leaves), + ); + + tree + } } diff --git a/vortex-scalar/Cargo.toml b/vortex-scalar/Cargo.toml index d69c03f8679..d838c6bfdc4 100644 --- a/vortex-scalar/Cargo.toml +++ b/vortex-scalar/Cargo.toml @@ -27,7 +27,6 @@ prost = { workspace = true } vortex-buffer = { workspace = true } vortex-dtype = { workspace = true } vortex-error = { workspace = true } -vortex-mask = { workspace = true } vortex-proto = { workspace = true, features = ["scalar"] } vortex-session = { workspace = true } vortex-utils = { workspace = true } diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index e6be50e0335..837b748c4df 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -95,10 +95,19 @@ impl ScanBuilder2 { let dtype = projection.return_dtype(self.reader.dtype())?; // Apply expressions to the reader tree. - let filter_reader = filter.as_ref().map(|f| self.reader.apply(f)).transpose()?; let projection_reader = self.reader.apply(&projection)?; + let filter_reader = filter.as_ref().map(|f| self.reader.apply(f)).transpose()?; + + tracing::info!( + "Executing scan with:\nProjection:\n{}\nFilter:\n{}", + projection_reader.display_tree(), + filter_reader + .as_ref() + .map_or("None".to_string(), |f| f.display_tree().to_string()) + ); // Execute both readers over the row range to produce streams. + // TODO(ngates): we could partition this? let row_offset = self.row_range.start; let filter_stream = filter_reader .map(|r| r.execute(self.row_range.clone())) From 8aca46a6ee9d57123b8ea92b089687ccfc3d35e1 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 6 Feb 2026 20:36:11 -0500 Subject: [PATCH 20/21] Add display Signed-off-by: Nicholas Gates --- vortex-layout/src/v2/reader.rs | 2 +- vortex-layout/src/v2/readers/chunked.rs | 16 +++++----- vortex-layout/src/v2/readers/constant.rs | 6 ++-- vortex-layout/src/v2/readers/dict.rs | 13 ++++---- vortex-layout/src/v2/readers/flat.rs | 6 ++-- vortex-layout/src/v2/readers/scalar_fn.rs | 18 +++++------ vortex-layout/src/v2/readers/struct_.rs | 38 ++++++++++------------- vortex-scan/src/v2/scan.rs | 20 ++++++------ 8 files changed, 55 insertions(+), 64 deletions(-) diff --git a/vortex-layout/src/v2/reader.rs b/vortex-layout/src/v2/reader.rs index c570f1d0820..bf84cc14163 100644 --- a/vortex-layout/src/v2/reader.rs +++ b/vortex-layout/src/v2/reader.rs @@ -67,5 +67,5 @@ pub trait ReaderStream: 'static + Send + Sync { fn skip(&mut self, n: usize); /// Returns the next chunk of data as an [`ArrayFuture`], or `None` if no more chunks. - fn next_chunk(&mut self) -> Option>; + fn next_chunk(&mut self) -> VortexResult>; } diff --git a/vortex-layout/src/v2/readers/chunked.rs b/vortex-layout/src/v2/readers/chunked.rs index 0cf3e0b0d96..d08a8a4ff15 100644 --- a/vortex-layout/src/v2/readers/chunked.rs +++ b/vortex-layout/src/v2/readers/chunked.rs @@ -189,18 +189,18 @@ impl ReaderStream for ChunkedReaderStream { } } - fn next_chunk(&mut self) -> Option> { + fn next_chunk(&mut self) -> VortexResult> { loop { - match self.ensure_active_stream() { - Ok(true) => {} - Ok(false) => return None, - Err(e) => return Some(Err(e)), + if !self.ensure_active_stream()? { + return Ok(None); } - let stream = self.active_stream.as_mut()?; + let Some(stream) = self.active_stream.as_mut() else { + return Ok(None); + }; - match stream.next_chunk() { - Some(result) => return Some(result), + match stream.next_chunk()? { + Some(future) => return Ok(Some(future)), None => { // Current stream is exhausted, try next chunk. self.active_stream = None; diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index 86201e26b7b..04453de5709 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -102,9 +102,9 @@ impl ReaderStream for ConstantReaderStream { self.remaining -= n; } - fn next_chunk(&mut self) -> Option> { + fn next_chunk(&mut self) -> VortexResult> { if self.remaining == 0 { - return None; + return Ok(None); } let len = usize::try_from(self.remaining).unwrap_or(usize::MAX); @@ -112,7 +112,7 @@ impl ReaderStream for ConstantReaderStream { let expression = self.expression.clone(); self.remaining = 0; - Some(Ok(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, async move { let mut array = ConstantArray::new(scalar, len).into_array(); if let Some(e) = expression { array = array.apply(&e)?; diff --git a/vortex-layout/src/v2/readers/dict.rs b/vortex-layout/src/v2/readers/dict.rs index 3a34cd7e70b..e55f6b92a75 100644 --- a/vortex-layout/src/v2/readers/dict.rs +++ b/vortex-layout/src/v2/readers/dict.rs @@ -85,8 +85,8 @@ impl Reader for DictReader { let values_row_count = self.values.row_count(); let mut values_stream = self.values.execute(0..values_row_count)?; let values_array_future = values_stream - .next_chunk() - .ok_or_else(|| vortex_err!("Dict values stream is empty"))??; + .next_chunk()? + .ok_or_else(|| vortex_err!("Dict values stream is empty"))?; let values_fut: Shared>> = async move { let array = values_array_future.await?; Ok(SharedArray::new(array).into_array()) @@ -144,16 +144,15 @@ impl ReaderStream for DictReaderStream { self.codes_stream.skip(n); } - fn next_chunk(&mut self) -> Option> { - let codes_future = match self.codes_stream.next_chunk()? { - Ok(f) => f, - Err(e) => return Some(Err(e)), + fn next_chunk(&mut self) -> VortexResult> { + let Some(codes_future) = self.codes_stream.next_chunk()? else { + return Ok(None); }; let values_fut = self.values_fut.clone(); let expression = self.expression.clone(); let len = codes_future.len(); - Some(Ok(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, async move { let values = values_fut.await.map_err(|e| vortex_err!("{e}"))?; let codes = codes_future.await?; let mut array = DictArray::try_new(codes, values)?.into_array(); diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index 4ad7aa4ecfb..d7b4d68ec5d 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -187,9 +187,9 @@ impl ReaderStream for FlatReaderStream { self.remaining -= n; } - fn next_chunk(&mut self) -> Option> { + fn next_chunk(&mut self) -> VortexResult> { if self.remaining == 0 { - return None; + return Ok(None); } let array_fut = self.array_fut.clone(); @@ -201,7 +201,7 @@ impl ReaderStream for FlatReaderStream { self.offset += len; self.remaining = 0; - Some(Ok(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, async move { // Await the shared array future (decoded once, shared across chunks). let mut array: ArrayRef = array_fut.await?; diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 4d0fbdc0abd..841b412b669 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -150,9 +150,9 @@ impl ScalarFnArrayStream { fn next_for_input( stream: &mut ReaderStreamRef, buffer: &mut Option, - ) -> Option> { + ) -> VortexResult> { if let Some(buffered) = buffer.take() { - return Some(Ok(buffered)); + return Ok(Some(buffered)); } stream.next_chunk() } @@ -183,7 +183,7 @@ impl ReaderStream for ScalarFnArrayStream { } } - fn next_chunk(&mut self) -> Option> { + fn next_chunk(&mut self) -> VortexResult> { // Collect an ArrayFuture for each input. let mut all_futures: Vec = Vec::with_capacity(self.input_streams.len()); @@ -192,22 +192,20 @@ impl ReaderStream for ScalarFnArrayStream { .iter_mut() .zip(self.input_buffers.iter_mut()) { - let future = match Self::next_for_input(stream, buffer) { - Some(Ok(f)) => f, - Some(Err(e)) => return Some(Err(e)), - None => return None, + let Some(future) = Self::next_for_input(stream, buffer)? else { + return Ok(None); }; all_futures.push(future); } if all_futures.is_empty() { - return None; + return Ok(None); } // Find the minimum length. let min_len = all_futures.iter().map(|f| f.len()).min().unwrap_or(0); if min_len == 0 { - return None; + return Ok(None); } // For inputs with len > min_len, buffer the remainder and slice. @@ -224,7 +222,7 @@ impl ReaderStream for ScalarFnArrayStream { } let scalar_fn = self.scalar_fn.clone(); - Some(Ok(ArrayFuture::new(min_len, async move { + Ok(Some(ArrayFuture::new(min_len, async move { let input_arrays = try_join_all(chunk_futures).await?; let array = ScalarFnArray::try_new(scalar_fn, input_arrays, min_len)?.into_array(); let array = array.optimize()?; diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 12f423c7795..1b184928365 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -182,9 +182,9 @@ impl StructReaderStream { fn next_for_child( stream: &mut ReaderStreamRef, buffer: &mut Option, - ) -> Option> { + ) -> VortexResult> { if let Some(buffered) = buffer.take() { - return Some(Ok(buffered)); + return Ok(Some(buffered)); } stream.next_chunk() } @@ -220,40 +220,37 @@ impl ReaderStream for StructReaderStream { } } - fn next_chunk(&mut self) -> Option> { + fn next_chunk(&mut self) -> VortexResult> { // Collect an ArrayFuture for each child (validity + fields). let mut all_futures: Vec = Vec::with_capacity(1 + self.fields.len()); // Validity let has_validity = self.validity.is_some(); if let Some(ref mut validity_stream) = self.validity { - let validity_future = - match Self::next_for_child(validity_stream, &mut self.validity_buffer) { - Some(Ok(f)) => f, - Some(Err(e)) => return Some(Err(e)), - None => return None, - }; + let Some(validity_future) = + Self::next_for_child(validity_stream, &mut self.validity_buffer)? + else { + return Ok(None); + }; all_futures.push(validity_future); } // Fields for (field_stream, field_buf) in self.fields.iter_mut().zip(self.field_buffers.iter_mut()) { - let field_future = match Self::next_for_child(field_stream, field_buf) { - Some(Ok(f)) => f, - Some(Err(e)) => return Some(Err(e)), - None => return None, + let Some(field_future) = Self::next_for_child(field_stream, field_buf)? else { + return Ok(None); }; all_futures.push(field_future); } if all_futures.is_empty() { - return None; + return Ok(None); } // Find the minimum length. let min_len = all_futures.iter().map(|f| f.len()).min().unwrap_or(0); if min_len == 0 { - return None; + return Ok(None); } // For children with len > min_len, buffer the remainder and slice. @@ -278,17 +275,14 @@ impl ReaderStream for StructReaderStream { } } - let struct_fields = match self + let struct_fields = self .dtype .as_struct_fields_opt() - .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype")) - { - Ok(f) => f.clone(), - Err(e) => return Some(Err(e)), - }; + .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))? + .clone(); let nullability = self.dtype.nullability(); - Some(Ok(ArrayFuture::new(min_len, async move { + Ok(Some(ArrayFuture::new(min_len, async move { // Split off validity future from field futures. let arrays = try_join_all(chunk_futures).await?; diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 837b748c4df..3e68d93c35e 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -144,9 +144,9 @@ struct Scan { impl Scan { /// Get the next projection chunk, consuming from the buffer first. - fn next_projection_chunk(&mut self) -> Option> { + fn next_projection_chunk(&mut self) -> VortexResult> { if let Some(buffered) = self.projection_buffer.take() { - return Some(Ok(buffered)); + return Ok(Some(buffered)); } self.projection_stream.next_chunk() } @@ -155,9 +155,9 @@ impl Scan { fn next_filter_chunk( stream: &mut ReaderStreamRef, buffer: &mut Option, - ) -> Option> { + ) -> VortexResult> { if let Some(buffered) = buffer.take() { - return Some(Ok(buffered)); + return Ok(Some(buffered)); } stream.next_chunk() } @@ -171,14 +171,14 @@ impl Scan { while remaining > 0 { let chunk = match Self::next_filter_chunk(filter_stream, &mut self.filter_buffer) { - Some(Ok(f)) => f, - Some(Err(e)) => return Some(Err(e)), - None => { + Ok(Some(f)) => f, + Ok(None) => { return Some(Err(vortex_error::vortex_err!( "Filter stream exhausted before covering {} rows", n ))); } + Err(e) => return Some(Err(e)), }; if chunk.len() <= remaining { @@ -251,9 +251,9 @@ impl Stream for Scan { // Get the next projection chunk. let proj_future = match this.next_projection_chunk() { - Some(Ok(f)) => f, - Some(Err(e)) => return Poll::Ready(Some(Err(e))), - None => return Poll::Ready(None), + Ok(Some(f)) => f, + Ok(None) => return Poll::Ready(None), + Err(e) => return Poll::Ready(Some(Err(e))), }; let mut chunk_len = proj_future.len(); From b690791257a6d4f9e427cd58d32b301ea73829b3 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Sun, 8 Feb 2026 12:21:29 -0500 Subject: [PATCH 21/21] merge Signed-off-by: Nicholas Gates --- vortex-array/src/array_future.rs | 19 +++- vortex-layout/src/layouts/flat/mod.rs | 3 +- vortex-layout/src/v2/readers/constant.rs | 5 +- vortex-layout/src/v2/readers/dict.rs | 5 +- vortex-layout/src/v2/readers/flat.rs | 19 ++-- vortex-layout/src/v2/readers/mod.rs | 29 +++++ vortex-layout/src/v2/readers/scalar_fn.rs | 17 +-- vortex-layout/src/v2/readers/struct_.rs | 41 ++++---- vortex-scan/src/v2/scan.rs | 123 +++++++++++++++------- 9 files changed, 184 insertions(+), 77 deletions(-) diff --git a/vortex-array/src/array_future.rs b/vortex-array/src/array_future.rs index 2cfcf25f67e..94ee4a7b754 100644 --- a/vortex-array/src/array_future.rs +++ b/vortex-array/src/array_future.rs @@ -21,11 +21,12 @@ use crate::ArrayRef; pub struct ArrayFuture { inner: Shared>>, len: usize, + estimated_bytes: usize, } impl ArrayFuture { /// Create a new `ArrayFuture` from a future that returns an array. - pub fn new(len: usize, fut: F) -> Self + pub fn new(len: usize, estimated_bytes: usize, fut: F) -> Self where F: Future> + Send + 'static, { @@ -41,13 +42,14 @@ impl ArrayFuture { .boxed() .shared(), len, + estimated_bytes, } } /// Create an `ArrayFuture` from an already-resolved array. pub fn ready(array: ArrayRef) -> Self { let len = array.len(); - Self::new(len, async move { Ok(array) }) + Self::new(len, 0, async move { Ok(array) }) } /// Returns the length of the array. @@ -60,11 +62,22 @@ impl ArrayFuture { self.len == 0 } + /// Returns the estimated decoded byte size of the array. + pub fn estimated_bytes(&self) -> usize { + self.estimated_bytes + } + /// Create an `ArrayFuture` that resolves to a slice of the original array. pub fn slice(&self, range: Range) -> Self { let inner = self.inner.clone(); let parent_len = self.len; - Self::new(range.len(), async move { + let slice_estimated = if self.len > 0 { + usize::try_from(self.estimated_bytes as u128 * range.len() as u128 / self.len as u128) + .unwrap_or(usize::MAX) + } else { + 0 + }; + Self::new(range.len(), slice_estimated, async move { let array = inner.await?; debug_assert!(range.end <= parent_len, "slice range out of bounds"); let _ = parent_len; diff --git a/vortex-layout/src/layouts/flat/mod.rs b/vortex-layout/src/layouts/flat/mod.rs index 890b8182398..c6e792b2972 100644 --- a/vortex-layout/src/layouts/flat/mod.rs +++ b/vortex-layout/src/layouts/flat/mod.rs @@ -10,7 +10,6 @@ use std::sync::Arc; use vortex_array::ArrayContext; use vortex_array::DeserializeMetadata; use vortex_array::ProstMetadata; -use vortex_array::session::ArraySessionExt; use vortex_buffer::ByteBuffer; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -111,7 +110,7 @@ impl VTable for FlatVTable { layout.segment_id, segment_source.clone(), layout.ctx.clone(), - session.arrays().registry().clone(), + session.clone(), ))) } diff --git a/vortex-layout/src/v2/readers/constant.rs b/vortex-layout/src/v2/readers/constant.rs index 04453de5709..f729a50c4a1 100644 --- a/vortex-layout/src/v2/readers/constant.rs +++ b/vortex-layout/src/v2/readers/constant.rs @@ -70,6 +70,7 @@ impl Reader for ConstantReader { scalar: self.scalar.clone(), remaining, expression: self.expression.clone(), + estimated_bytes_per_row: super::estimated_decoded_bytes(self.scalar.dtype(), 1), })) } @@ -87,6 +88,7 @@ struct ConstantReaderStream { scalar: Scalar, remaining: u64, expression: Option, + estimated_bytes_per_row: usize, } impl ReaderStream for ConstantReaderStream { @@ -110,9 +112,10 @@ impl ReaderStream for ConstantReaderStream { let len = usize::try_from(self.remaining).unwrap_or(usize::MAX); let scalar = self.scalar.clone(); let expression = self.expression.clone(); + let estimated_bytes = len * self.estimated_bytes_per_row; self.remaining = 0; - Ok(Some(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, estimated_bytes, async move { let mut array = ConstantArray::new(scalar, len).into_array(); if let Some(e) = expression { array = array.apply(&e)?; diff --git a/vortex-layout/src/v2/readers/dict.rs b/vortex-layout/src/v2/readers/dict.rs index e55f6b92a75..5efdc3616c3 100644 --- a/vortex-layout/src/v2/readers/dict.rs +++ b/vortex-layout/src/v2/readers/dict.rs @@ -102,6 +102,7 @@ impl Reader for DictReader { codes_stream, values_fut, expression: self.expression.clone(), + estimated_bytes_per_row: super::estimated_decoded_bytes(&self.dtype, 1), })) } @@ -133,6 +134,7 @@ struct DictReaderStream { codes_stream: ReaderStreamRef, values_fut: Shared>>, expression: Option, + estimated_bytes_per_row: usize, } impl ReaderStream for DictReaderStream { @@ -151,8 +153,9 @@ impl ReaderStream for DictReaderStream { let values_fut = self.values_fut.clone(); let expression = self.expression.clone(); let len = codes_future.len(); + let estimated_bytes = len * self.estimated_bytes_per_row; - Ok(Some(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, estimated_bytes, async move { let values = values_fut.await.map_err(|e| vortex_err!("{e}"))?; let codes = codes_future.await?; let mut array = DictArray::try_new(codes, values)?.into_array(); diff --git a/vortex-layout/src/v2/readers/flat.rs b/vortex-layout/src/v2/readers/flat.rs index d7b4d68ec5d..fd8a14a3d75 100644 --- a/vortex-layout/src/v2/readers/flat.rs +++ b/vortex-layout/src/v2/readers/flat.rs @@ -16,7 +16,6 @@ use vortex_array::expr::Expression; use vortex_array::expr::root; use vortex_array::expr::transform::replace; use vortex_array::serde::ArrayParts; -use vortex_array::session::ArrayRegistry; use vortex_buffer::ByteBuffer; use vortex_dtype::DType; use vortex_error::SharedVortexResult; @@ -24,6 +23,7 @@ use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_error::vortex_err; use vortex_error::vortex_panic; +use vortex_session::VortexSession; use crate::segments::SegmentId; use crate::segments::SegmentSourceRef; @@ -46,7 +46,7 @@ pub struct FlatReader { segment_id: SegmentId, segment_source: SegmentSourceRef, ctx: ArrayContext, - registry: ArrayRegistry, + session: VortexSession, expression: Option, } @@ -58,7 +58,7 @@ impl FlatReader { segment_id: SegmentId, segment_source: SegmentSourceRef, ctx: ArrayContext, - registry: ArrayRegistry, + session: VortexSession, ) -> Self { Self { len, @@ -68,7 +68,7 @@ impl FlatReader { segment_id, segment_source, ctx, - registry, + session, expression: None, } } @@ -101,7 +101,7 @@ impl Reader for FlatReader { segment_id: self.segment_id, segment_source: self.segment_source.clone(), ctx: self.ctx.clone(), - registry: self.registry.clone(), + session: self.session.clone(), expression: Some(new_expr), })) } @@ -128,7 +128,7 @@ impl Reader for FlatReader { let decode_dtype = self.decode_dtype.clone(); let row_count = self.len; let ctx = self.ctx.clone(); - let registry = self.registry.clone(); + let session = self.session.clone(); let array_fut = async move { let segment = segment_source.request(segment_id).await?; let parts = if let Some(array_tree) = array_tree { @@ -136,7 +136,7 @@ impl Reader for FlatReader { } else { ArrayParts::try_from(segment)? }; - parts.decode(&decode_dtype, row_count, &ctx, ®istry) + parts.decode(&decode_dtype, row_count, &ctx, &session) } .map(|r| r.map_err(Arc::new)) .boxed() @@ -149,6 +149,7 @@ impl Reader for FlatReader { row_count: self.len, offset: start, remaining: end - start, + estimated_bytes_per_row: super::estimated_decoded_bytes(&self.dtype, 1), })) } @@ -172,6 +173,7 @@ struct FlatReaderStream { row_count: usize, offset: usize, remaining: usize, + estimated_bytes_per_row: usize, } impl ReaderStream for FlatReaderStream { @@ -197,11 +199,12 @@ impl ReaderStream for FlatReaderStream { let offset = self.offset; let len = self.remaining; let expression = self.expression.clone(); + let estimated_bytes = len * self.estimated_bytes_per_row; self.offset += len; self.remaining = 0; - Ok(Some(ArrayFuture::new(len, async move { + Ok(Some(ArrayFuture::new(len, estimated_bytes, async move { // Await the shared array future (decoded once, shared across chunks). let mut array: ArrayRef = array_fut.await?; diff --git a/vortex-layout/src/v2/readers/mod.rs b/vortex-layout/src/v2/readers/mod.rs index 876acd44f44..7bc328832c6 100644 --- a/vortex-layout/src/v2/readers/mod.rs +++ b/vortex-layout/src/v2/readers/mod.rs @@ -1,6 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use vortex_dtype::DType; +use vortex_dtype::DecimalType; + pub mod chunked; pub mod constant; pub mod dict; @@ -8,3 +11,29 @@ pub mod flat; pub mod scalar_fn; pub mod struct_; pub mod zoned; + +/// Estimates the decoded in-memory byte size for `n` elements of the given type. +/// +/// For fixed-width types this is exact. For variable-width types (strings, binary, lists) +/// a heuristic estimate is used. +fn estimated_decoded_bytes(dtype: &DType, n: usize) -> usize { + const VARIABLE_ELEMENT_ESTIMATE: usize = 64; + match dtype { + DType::Null => 0, + DType::Bool(_) => n, + DType::Primitive(ptype, _) => n * ptype.byte_width(), + DType::Decimal(decimal, _) => { + n * DecimalType::smallest_decimal_value_type(decimal).byte_width() + } + DType::Utf8(_) | DType::Binary(_) => n * VARIABLE_ELEMENT_ESTIMATE, + DType::List(elem_dtype, _) => n * 10 * estimated_decoded_bytes(elem_dtype, 1), + DType::FixedSizeList(elem_dtype, list_size, _) => { + n * estimated_decoded_bytes(elem_dtype, *list_size as usize) + } + DType::Struct(fields, _) => fields + .fields() + .map(|f| estimated_decoded_bytes(&f, n)) + .sum(), + DType::Extension(ext) => estimated_decoded_bytes(ext.storage_dtype(), n), + } +} diff --git a/vortex-layout/src/v2/readers/scalar_fn.rs b/vortex-layout/src/v2/readers/scalar_fn.rs index 841b412b669..7387f4dd29e 100644 --- a/vortex-layout/src/v2/readers/scalar_fn.rs +++ b/vortex-layout/src/v2/readers/scalar_fn.rs @@ -222,12 +222,17 @@ impl ReaderStream for ScalarFnArrayStream { } let scalar_fn = self.scalar_fn.clone(); - Ok(Some(ArrayFuture::new(min_len, async move { - let input_arrays = try_join_all(chunk_futures).await?; - let array = ScalarFnArray::try_new(scalar_fn, input_arrays, min_len)?.into_array(); - let array = array.optimize()?; - Ok(array) - }))) + let estimated_bytes: usize = chunk_futures.iter().map(|f| f.estimated_bytes()).sum(); + Ok(Some(ArrayFuture::new( + min_len, + estimated_bytes, + async move { + let input_arrays = try_join_all(chunk_futures).await?; + let array = ScalarFnArray::try_new(scalar_fn, input_arrays, min_len)?.into_array(); + let array = array.optimize()?; + Ok(array) + }, + ))) } } diff --git a/vortex-layout/src/v2/readers/struct_.rs b/vortex-layout/src/v2/readers/struct_.rs index 1b184928365..3e1840d6418 100644 --- a/vortex-layout/src/v2/readers/struct_.rs +++ b/vortex-layout/src/v2/readers/struct_.rs @@ -281,23 +281,28 @@ impl ReaderStream for StructReaderStream { .ok_or_else(|| vortex_error::vortex_err!("Expected struct dtype"))? .clone(); let nullability = self.dtype.nullability(); - - Ok(Some(ArrayFuture::new(min_len, async move { - // Split off validity future from field futures. - let arrays = try_join_all(chunk_futures).await?; - - let (validity, fields) = if has_validity { - let validity_array = arrays[0].clone(); - let fields = arrays[1..].to_vec(); - (Validity::Array(validity_array), fields) - } else { - (nullability.into(), arrays) - }; - - Ok( - StructArray::try_new_with_dtype(fields, struct_fields, min_len, validity)? - .into_array(), - ) - }))) + let estimated_bytes: usize = chunk_futures.iter().map(|f| f.estimated_bytes()).sum(); + + Ok(Some(ArrayFuture::new( + min_len, + estimated_bytes, + async move { + // Split off validity future from field futures. + let arrays = try_join_all(chunk_futures).await?; + + let (validity, fields) = if has_validity { + let validity_array = arrays[0].clone(); + let fields = arrays[1..].to_vec(); + (Validity::Array(validity_array), fields) + } else { + (nullability.into(), arrays) + }; + + Ok( + StructArray::try_new_with_dtype(fields, struct_fields, min_len, validity)? + .into_array(), + ) + }, + ))) } } diff --git a/vortex-scan/src/v2/scan.rs b/vortex-scan/src/v2/scan.rs index 3e68d93c35e..13b1959d4f2 100644 --- a/vortex-scan/src/v2/scan.rs +++ b/vortex-scan/src/v2/scan.rs @@ -1,6 +1,7 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors +use std::collections::VecDeque; use std::ops::BitAnd; use std::ops::Range; use std::pin::Pin; @@ -8,7 +9,9 @@ use std::task::Context; use std::task::Poll; use futures::Stream; +use futures::StreamExt; use futures::future::BoxFuture; +use futures::stream::FuturesOrdered; use vortex_array::ArrayFuture; use vortex_array::ArrayRef; use vortex_array::expr::Expression; @@ -32,6 +35,7 @@ pub struct ScanBuilder2 { row_range: Range, row_selection: Selection, // NOTE: applies to the selected row range. session: VortexSession, + byte_budget: usize, } impl ScanBuilder2 { @@ -45,6 +49,7 @@ impl ScanBuilder2 { row_range, row_selection: Selection::All, session, + byte_budget: 0, } } @@ -85,6 +90,14 @@ impl ScanBuilder2 { self } + /// Sets the byte budget for pipelining. When set to a non-zero value, the scan will + /// enqueue multiple chunks of work up to the given estimated byte budget, allowing I/O + /// and compute to overlap. + pub fn with_byte_budget(mut self, byte_budget: usize) -> Self { + self.byte_budget = byte_budget; + self + } + pub fn into_array_stream(self) -> VortexResult { let projection = self.projection.optimize_recursive(self.reader.dtype())?; let filter = self @@ -98,7 +111,7 @@ impl ScanBuilder2 { let projection_reader = self.reader.apply(&projection)?; let filter_reader = filter.as_ref().map(|f| self.reader.apply(f)).transpose()?; - tracing::info!( + tracing::debug!( "Executing scan with:\nProjection:\n{}\nFilter:\n{}", projection_reader.display_tree(), filter_reader @@ -118,13 +131,18 @@ impl ScanBuilder2 { dtype, filter_stream, projection_stream, - pending: None, + pipeline: FuturesOrdered::new(), + pipeline_bytes: VecDeque::new(), + bytes_in_flight: 0, + byte_budget: self.byte_budget, limit: self.limit, rows_produced: 0, + rows_enqueued: 0, row_selection: self.row_selection, row_offset, filter_buffer: None, projection_buffer: None, + exhausted: false, }) } } @@ -133,13 +151,18 @@ struct Scan { dtype: DType, filter_stream: Option, projection_stream: ReaderStreamRef, - pending: Option>>, + pipeline: FuturesOrdered>>, + pipeline_bytes: VecDeque, + bytes_in_flight: usize, + byte_budget: usize, limit: Option, rows_produced: u64, + rows_enqueued: u64, row_selection: Selection, row_offset: u64, filter_buffer: Option, projection_buffer: Option, + exhausted: bool, } impl Scan { @@ -226,33 +249,24 @@ impl Stream for Scan { fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); - loop { - // Poll pending future if we have one. - if let Some(fut) = this.pending.as_mut() { - match fut.as_mut().poll(cx) { - Poll::Ready(result) => { - this.pending = None; - return match result { - Ok(array) => { - this.rows_produced += array.len() as u64; - Poll::Ready(Some(Ok(array))) - } - Err(e) => Poll::Ready(Some(Err(e))), - }; - } - Poll::Pending => return Poll::Pending, - } - } - - // Check limit. - if this.limit.is_some_and(|limit| this.rows_produced >= limit) { - return Poll::Ready(None); + // === Fill phase === + // Enqueue chunks into the pipeline until we hit the byte budget or exhaust input. + while !this.exhausted + && (this.bytes_in_flight == 0 || this.bytes_in_flight < this.byte_budget) + { + // Check limit on rows enqueued. + if this.limit.is_some_and(|limit| this.rows_enqueued >= limit) { + this.exhausted = true; + break; } // Get the next projection chunk. let proj_future = match this.next_projection_chunk() { Ok(Some(f)) => f, - Ok(None) => return Poll::Ready(None), + Ok(None) => { + this.exhausted = true; + break; + } Err(e) => return Poll::Ready(Some(Err(e))), }; @@ -260,7 +274,7 @@ impl Stream for Scan { // Apply limit: if remaining < chunk_len, slice the projection future. if let Some(limit) = this.limit { - let remaining = usize::try_from(limit - this.rows_produced).unwrap_or(usize::MAX); + let remaining = usize::try_from(limit - this.rows_enqueued).unwrap_or(usize::MAX); if remaining < chunk_len { // Buffer the remainder and use only what we need. this.projection_buffer = Some(proj_future.slice(remaining..chunk_len)); @@ -275,7 +289,8 @@ impl Stream for Scan { }; if chunk_len == 0 { - return Poll::Ready(None); + this.exhausted = true; + break; } // Compute the selection mask for this chunk's row range. @@ -291,8 +306,12 @@ impl Stream for Scan { this.row_offset += chunk_len as u64; - // Build the pending future: await projection, apply filter + selection. - if this.filter_stream.is_some() { + // Estimate bytes for this chunk. + let proj_estimated = proj_future.estimated_bytes(); + let mut chunk_estimated = proj_estimated; + + // Build the future: await projection, apply filter + selection. + let fut: BoxFuture<'static, VortexResult> = if this.filter_stream.is_some() { let filter_chunks = match this.collect_filter_chunks(chunk_len) { Some(Ok(chunks)) => chunks, Some(Err(e)) => return Poll::Ready(Some(Err(e))), @@ -303,7 +322,12 @@ impl Stream for Scan { } }; - this.pending = Some(Box::pin(async move { + chunk_estimated += filter_chunks + .iter() + .map(|f| f.estimated_bytes()) + .sum::(); + + Box::pin(async move { // Await filter chunks and combine into a single mask. let mut filter_masks: Vec = Vec::with_capacity(filter_chunks.len()); for filter_chunk in filter_chunks { @@ -336,19 +360,42 @@ impl Stream for Scan { } else { array.filter(mask) } - })); + }) } else if selection_mask.all_true() { - // No filter, no selection masking — just await projection. - this.pending = Some(Box::pin(proj_future)); + Box::pin(proj_future) } else { - // No filter, but selection mask needs to be applied. - this.pending = Some(Box::pin(async move { + Box::pin(async move { let array = proj_future.await?; array.filter(selection_mask) - })); - } + }) + }; - // Loop back to poll the newly created future. + this.pipeline.push_back(fut); + this.pipeline_bytes.push_back(chunk_estimated); + this.bytes_in_flight += chunk_estimated; + this.rows_enqueued += chunk_len as u64; + } + + // === Drain phase === + if this.pipeline.is_empty() { + return Poll::Ready(None); + } + + match this.pipeline.poll_next_unpin(cx) { + Poll::Ready(Some(result)) => { + if let Some(estimated) = this.pipeline_bytes.pop_front() { + this.bytes_in_flight = this.bytes_in_flight.saturating_sub(estimated); + } + match result { + Ok(array) => { + this.rows_produced += array.len() as u64; + Poll::Ready(Some(Ok(array))) + } + Err(e) => Poll::Ready(Some(Err(e))), + } + } + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, } } }