From 470fbe96a4ac53dab8287a9e8968533eba33e580 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 06:49:18 -0500 Subject: [PATCH 01/57] Scan API Signed-off-by: Nicholas Gates --- vortex-scan/src/api.rs | 100 ++++++++++++++------- vortex-scan/src/layout.rs | 166 ++++++++++++++++++++++++----------- vortex-scan/src/selection.rs | 2 +- 3 files changed, 184 insertions(+), 84 deletions(-) diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index bbcfccba526..784a87f1ef5 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,27 +31,35 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_session::VortexSession; -/// Create a Vortex source from serialized configuration. +use crate::Selection; + +/// Opens a Vortex [`DataSource`] from a URI. /// -/// Providers can be registered with Vortex under a specific -#[async_trait(?Send)] -pub trait DataSourceProvider: 'static { - /// Attempt to initialize a new source. - /// - /// Returns `Ok(None)` if the provider cannot handle the given URI. - async fn initialize( +/// Configuration can be passed via the URI query parameters, similar to JDBC / ADBC. +/// Providers can be registered with the [`VortexSession`] to support additional URI schemes. +#[async_trait] +pub trait DataSourceOpener: 'static { + /// Attempt to open a new data source from a URI. + async fn open(&self, uri: String, session: &VortexSession) -> VortexResult; +} + +/// Supports deserialization of a Vortex [`DataSource`] on a remote worker. +#[async_trait] +pub trait DataSourceRemote: 'static { + /// Attempt to deserialize the source. + fn deserialize_data_source( &self, - uri: String, + data: &[u8], session: &VortexSession, - ) -> VortexResult>; + ) -> VortexResult; } /// A reference-counted data source. pub type DataSourceRef = Arc; /// A data source represents a streamable dataset that can be scanned with projection and filter -/// expressions. Each scan produces splits that can be executed (potentially in parallel) to read -/// data. Each split can be serialized for remote execution. +/// expressions. Each scan produces splits that can be executed in parallel to read data. Each +/// split can be serialized for remote execution. /// /// The DataSource may be used multiple times to create multiple scans, whereas each scan and each /// split of a scan can only be consumed once. @@ -62,14 +71,16 @@ pub trait DataSource: 'static + Send + Sync { /// 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; - - /// Serialize a split from this data source. - fn serialize_split(&self, split: &dyn Split) -> VortexResult>; + /// Serialize the [`DataSource`] to pass to a remote worker. + fn serialize(&self) -> VortexResult>> { + Ok(None) + } /// Deserialize a split that was previously serialized from a compatible data source. - fn deserialize_split(&self, data: &[u8]) -> VortexResult; + fn deserialize_split(&self, data: &[u8], session: &VortexSession) -> VortexResult; + + /// Returns a scan over the source. + async fn scan(&self, scan_request: ScanRequest) -> VortexResult; } /// A request to scan a data source. @@ -79,7 +90,13 @@ pub struct ScanRequest { pub projection: Option, /// Filter expression, `None` implies no filter. pub filter: Option, - /// Optional limit on the number of rows to scan. + /// The row range to read. + pub row_range: Option>, + /// A row selection to apply to the scan. The selection identifies rows within the specified + /// row range. + pub selection: Selection, + /// Optional limit on the number of rows returned by scan. Limits are applied after all + /// filtering and row selection. pub limit: Option, } @@ -112,24 +129,45 @@ pub trait Split: 'static + Send { /// Downcast the split to a concrete type. fn as_any(&self) -> &dyn Any; - /// Executes the split. - fn execute(self: Box) -> 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; + + /// Serialize this split for a remote worker. + fn serialize(&self) -> VortexResult>> { + Ok(None) + } + + /// Executes the split. + fn execute(self: Box) -> VortexResult; } /// An estimate that can be exact, an upper bound, or unknown. -#[derive(Default)] -pub enum Estimate { - /// The exact value. - Exact(T), - /// An upper bound on the value. - UpperBound(T), - /// The value is unknown. - #[default] - Unknown, +#[derive(Clone, Debug)] +pub struct Estimate { + /// The lower bound + pub lower: T, + /// The upper bound + pub upper: Option, +} + +impl Default for Estimate { + fn default() -> Self { + Self { + lower: T::default(), + upper: None, + } + } +} + +impl Estimate { + /// Creates an exact estimate. + pub fn exact(value: T) -> Self { + Self { + lower: value, + upper: Some(value), + } + } } diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 183a023bd2d..64ad1bae5e6 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -2,23 +2,19 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use std::any::Any; -use std::collections::VecDeque; +use std::ops::Range; use async_trait::async_trait; -use futures::StreamExt; -use futures::future::BoxFuture; -use futures::stream; -use vortex_array::ArrayRef; -use vortex_array::stream::ArrayStreamAdapter; +use vortex_array::expr::Expression; use vortex_array::stream::SendableArrayStream; use vortex_dtype::DType; -use vortex_error::VortexExpect; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_layout::LayoutReaderRef; use vortex_session::VortexSession; use crate::ScanBuilder; +use crate::Selection; use crate::api::DataSource; use crate::api::DataSourceScan; use crate::api::DataSourceScanRef; @@ -27,16 +23,34 @@ use crate::api::ScanRequest; use crate::api::Split; use crate::api::SplitRef; +/// The default number of rows per Scan API split. +const DEFAULT_SPLIT_SIZE: u64 = 100_000; + /// An implementation of a [`DataSource`] that reads data from a [`LayoutReaderRef`]. pub struct LayoutReaderDataSource { reader: LayoutReaderRef, session: VortexSession, + split_size: u64, } impl LayoutReaderDataSource { /// Creates a new [`LayoutReaderDataSource`]. pub fn new(reader: LayoutReaderRef, session: VortexSession) -> Self { - Self { reader, session } + Self { + reader, + session, + split_size: DEFAULT_SPLIT_SIZE, + } + } + + /// Sets the target number of rows per Scan API split. + /// + /// Each split drives a [`ScanBuilder`] over its row range, which internally handles + /// physical layout alignment and I/O pipelining. This controls the engine-level + /// parallelism granularity, not the I/O granularity. + pub fn with_split_size(mut self, split_size: u64) -> Self { + self.split_size = split_size; + self } } @@ -47,46 +61,49 @@ impl DataSource for LayoutReaderDataSource { } fn row_count_estimate(&self) -> Estimate { - Estimate::Exact(self.reader.row_count()) + Estimate::exact(self.reader.row_count()) } async fn scan(&self, scan_request: ScanRequest) -> VortexResult { - let mut builder = ScanBuilder::new(self.session.clone(), self.reader.clone()); + let total_rows = self.reader.row_count(); + let row_range = scan_request.row_range.unwrap_or(0..total_rows); - if let Some(projection) = scan_request.projection { - builder = builder.with_projection(projection); - } - - if let Some(filter) = scan_request.filter { - builder = builder.with_filter(filter); - } - - if let Some(limit) = scan_request.limit { - builder = builder.with_limit(limit); - } - - let scan = builder.prepare()?; - let dtype = scan.dtype().clone(); - let splits = scan.execute(None)?; + let dtype = if let Some(proj) = &scan_request.projection { + proj.return_dtype(self.reader.dtype())? + } else { + self.reader.dtype().clone() + }; Ok(Box::new(LayoutReaderScan { + reader: self.reader.clone(), + session: self.session.clone(), dtype, - splits: VecDeque::from_iter(splits), + projection: scan_request.projection, + filter: scan_request.filter, + limit: scan_request.limit, + selection: scan_request.selection, + next_row: row_range.start, + end_row: row_range.end, + split_size: self.split_size, })) } - fn serialize_split(&self, _split: &dyn Split) -> VortexResult> { - vortex_bail!("LayoutReader splits are not yet serializable"); - } - - fn deserialize_split(&self, _split: &[u8]) -> VortexResult { + fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { vortex_bail!("LayoutReader splits are not yet serializable"); } } struct LayoutReaderScan { + reader: LayoutReaderRef, + session: VortexSession, dtype: DType, - splits: VecDeque>>>, + projection: Option, + filter: Option, + limit: Option, + selection: Selection, + next_row: u64, + end_row: u64, + split_size: u64, } #[async_trait] @@ -96,22 +113,49 @@ impl DataSourceScan for LayoutReaderScan { } fn remaining_splits_estimate(&self) -> Estimate { - Estimate::Exact(self.splits.len()) + if self.next_row >= self.end_row { + return Estimate::exact(0); + } + let remaining_rows = self.end_row - self.next_row; + let splits = remaining_rows.div_ceil(self.split_size); + Estimate { + lower: 0, + upper: Some(usize::try_from(splits).unwrap_or(usize::MAX)), + } } async fn next_splits(&mut self, max_splits: usize) -> VortexResult> { - let n = std::cmp::min(max_splits, self.splits.len()); - - let mut splits = Vec::with_capacity(n); - for _ in 0..n { - let fut = self - .splits - .pop_front() - .vortex_expect("Checked length above ensures we have enough splits"); + let mut splits = Vec::new(); + + for _ in 0..max_splits { + if self.next_row >= self.end_row { + break; + } + + if self.limit.is_some_and(|limit| limit == 0) { + break; + } + + let split_end = (self.next_row + self.split_size).min(self.end_row); + let row_range = self.next_row..split_end; + let split_rows = split_end - self.next_row; + + let split_limit = self.limit; + if let Some(ref mut limit) = self.limit { + *limit = limit.saturating_sub(split_rows); + } + splits.push(Box::new(LayoutReaderSplit { - dtype: self.dtype.clone(), - fut, + reader: self.reader.clone(), + session: self.session.clone(), + projection: self.projection.clone(), + filter: self.filter.clone(), + limit: split_limit, + row_range, + selection: self.selection.clone(), }) as SplitRef); + + self.next_row = split_end; } Ok(splits) @@ -119,28 +163,46 @@ impl DataSourceScan for LayoutReaderScan { } struct LayoutReaderSplit { - dtype: DType, - fut: BoxFuture<'static, VortexResult>>, + reader: LayoutReaderRef, + session: VortexSession, + projection: Option, + filter: Option, + limit: Option, + row_range: Range, + selection: Selection, } -#[async_trait] impl Split for LayoutReaderSplit { fn as_any(&self) -> &dyn Any { self } fn execute(self: Box) -> VortexResult { - Ok(Box::pin(ArrayStreamAdapter::new( - self.dtype, - stream::once(self.fut).filter_map(|a| async move { a.transpose() }), - ))) + let mut builder = ScanBuilder::new(self.session, self.reader) + .with_row_range(self.row_range) + .with_selection(self.selection); + + if let Some(proj) = self.projection { + builder = builder.with_projection(proj); + } + if let Some(filter) = self.filter { + builder = builder.with_filter(filter); + } + if let Some(limit) = self.limit { + builder = builder.with_limit(limit); + } + + Ok(Box::pin(builder.into_array_stream()?)) } fn row_count_estimate(&self) -> Estimate { - Estimate::Unknown + Estimate { + lower: 0, + upper: Some(self.row_range.end - self.row_range.start), + } } fn byte_size_estimate(&self) -> Estimate { - Estimate::Unknown + Estimate::default() } } diff --git a/vortex-scan/src/selection.rs b/vortex-scan/src/selection.rs index d2fb46d4bb0..39f053c8ceb 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(Default, Clone, Debug)] pub enum Selection { /// No selection, all rows are included. #[default] From f9656153aac047a3c5ad07c0a4e5d1adfef7538e Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 07:24:51 -0500 Subject: [PATCH 02/57] Scan API Signed-off-by: Nicholas Gates --- .../src/persistent/access_plan.rs | 5 + vortex-datafusion/src/persistent/opener.rs | 152 +++++++++++++----- vortex-scan/src/layout.rs | 23 +++ 3 files changed, 141 insertions(+), 39 deletions(-) diff --git a/vortex-datafusion/src/persistent/access_plan.rs b/vortex-datafusion/src/persistent/access_plan.rs index 2a523ce8e16..9cf4a5824d6 100644 --- a/vortex-datafusion/src/persistent/access_plan.rs +++ b/vortex-datafusion/src/persistent/access_plan.rs @@ -23,6 +23,11 @@ impl VortexAccessPlan { } impl VortexAccessPlan { + /// Returns the selection, if one was set. + pub fn selection(&self) -> Option<&Selection> { + self.selection.as_ref() + } + /// Apply the plan to the scan's builder. pub fn apply_to_builder(&self, mut scan_builder: ScanBuilder) -> ScanBuilder where diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 15a2a81887a..56cbeeca23e 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -35,12 +35,17 @@ use vortex::array::ArrayRef; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; use vortex::error::VortexError; +use vortex::error::VortexResult; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; use vortex::layout::LayoutReader; use vortex::metrics::Label; use vortex::metrics::MetricsRegistry; use vortex::scan::ScanBuilder; +use vortex::scan::Selection; +use vortex::scan::api::DataSource as _; +use vortex::scan::api::ScanRequest; +use vortex::scan::layout::LayoutReaderDataSource; use vortex::session::VortexSession; use vortex_utils::aliases::dash_map::DashMap; use vortex_utils::aliases::dash_map::Entry; @@ -122,6 +127,7 @@ impl FileOpener for VortexOpener { let limit = self.limit; let layout_reader = self.layout_readers.clone(); let has_output_ordering = self.has_output_ordering; + let use_scan_api = std::env::var("VORTEX_USE_SCAN_API").is_ok_and(|v| v == "1"); let expr_convertor = self.expression_convertor.clone(); @@ -270,23 +276,7 @@ impl FileOpener for VortexOpener { } }; - let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); - - 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, - ); - } - + // Convert the DF filter to a Vortex expression. let filter = filter .and_then(|f| { // Verify that all filters we've accepted from DataFusion get pushed down. @@ -320,31 +310,116 @@ impl FileOpener for VortexOpener { }) .transpose()?; - if let Some(limit) = limit - && filter.is_none() - { - scan_builder = scan_builder.with_limit(limit); - } + let file_location = file.object_meta.location.clone(); - let stream = scan_builder - .with_metrics_registry(metrics_registry) - .with_projection(scan_projection) - .with_some_filter(filter) - .with_ordered(has_output_ordering) - .map(move |chunk| { - let mut ctx = session.create_execution_ctx(); - chunk.execute_record_batch(&stream_schema, &mut ctx) - }) - .into_stream() - .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? + // Both paths produce a boxed stream of VortexResult. + let chunk_stream = if use_scan_api { + // Scan API path: use LayoutReaderDataSource + ScanRequest. + let session = session.clone(); + let stream_schema = stream_schema.clone(); + + let mut selection = Selection::All; + if let Some(extensions) = &file.extensions + && let Some(vortex_plan) = extensions.downcast_ref::() + && let Some(sel) = vortex_plan.selection() + { + selection = sel.clone(); + } + + let row_range = file.range.map(|file_range| { + byte_range_to_row_range( + file_range.start as u64..file_range.end as u64, + vxf.row_count(), + file.object_meta.size, + ) + }); + + let scan_limit = if filter.is_none() { limit } else { None }; + + let scan_request = ScanRequest { + projection: Some(scan_projection), + filter, + row_range, + selection, + limit: scan_limit, + }; + + let data_source = LayoutReaderDataSource::new(layout_reader, session.clone()) + .with_some_metrics_registry(Some(metrics_registry)); + + let mut scan = data_source + .scan(scan_request) + .await + .map_err(|e| exec_datafusion_err!("Failed to create Vortex scan: {e}"))?; + + let splits = scan + .next_splits(usize::MAX) + .await + .map_err(|e| exec_datafusion_err!("Failed to get Vortex splits: {e}"))?; + + let streams: Vec<_> = splits + .into_iter() + .map(|split| split.execute()) + .collect::>>() + .map_err(|e| exec_datafusion_err!("Failed to execute Vortex split: {e}"))?; + + stream::iter(streams) + .flatten() + .map(move |result| { + result.and_then(|chunk| { + let mut ctx = session.create_execution_ctx(); + chunk.execute_record_batch(&stream_schema, &mut ctx) + }) + }) + .boxed() + } else { + // Direct ScanBuilder path (existing). + let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); + + 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, + ); + } + + if let Some(limit) = limit + && filter.is_none() + { + scan_builder = scan_builder.with_limit(limit); + } + + scan_builder + .with_metrics_registry(metrics_registry) + .with_projection(scan_projection) + .with_some_filter(filter) + .with_ordered(has_output_ordering) + .map(move |chunk| { + let mut ctx = session.create_execution_ctx(); + chunk.execute_record_batch(&stream_schema, &mut ctx) + }) + .into_stream() + .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? + .boxed() + }; + + // Shared post-processing: batch slicing, error mapping, leftover projection. + let stream = chunk_stream .map_ok(move |rb| { - // We try and slice the stream into respecting datafusion's configured batch size. + // Slice the stream to respect DataFusion's configured batch size. stream::iter( (0..rb.num_rows().div_ceil(batch_size * 2)) .flat_map(move |block_idx| { let offset = block_idx * batch_size * 2; - // If we have less than two batches worth of rows left, we keep them together as a single batch. if rb.num_rows() - offset < 2 * batch_size { let length = rb.num_rows() - offset; [Some(rb.slice(offset, length)), None].into_iter() @@ -359,10 +434,9 @@ impl FileOpener for VortexOpener { ) }) .map_err(move |e: VortexError| { - DataFusionError::External(Box::new(e.with_context(format!( - "Failed to read Vortex file: {}", - file.object_meta.location - )))) + DataFusionError::External(Box::new( + e.with_context(format!("Failed to read Vortex file: {file_location}",)), + )) }) .try_flatten() .map(move |batch| { diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 64ad1bae5e6..62b9b7b4a0b 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -3,6 +3,7 @@ use std::any::Any; use std::ops::Range; +use std::sync::Arc; use async_trait::async_trait; use vortex_array::expr::Expression; @@ -11,6 +12,7 @@ use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_layout::LayoutReaderRef; +use vortex_metrics::MetricsRegistry; use vortex_session::VortexSession; use crate::ScanBuilder; @@ -31,6 +33,7 @@ pub struct LayoutReaderDataSource { reader: LayoutReaderRef, session: VortexSession, split_size: u64, + metrics_registry: Option>, } impl LayoutReaderDataSource { @@ -40,6 +43,7 @@ impl LayoutReaderDataSource { reader, session, split_size: DEFAULT_SPLIT_SIZE, + metrics_registry: None, } } @@ -52,6 +56,18 @@ impl LayoutReaderDataSource { self.split_size = split_size; self } + + /// Sets the metrics registry for tracking scan performance. + pub fn with_metrics_registry(mut self, metrics: Arc) -> Self { + self.metrics_registry = Some(metrics); + self + } + + /// Optionally sets the metrics registry for tracking scan performance. + pub fn with_some_metrics_registry(mut self, metrics: Option>) -> Self { + self.metrics_registry = metrics; + self + } } #[async_trait] @@ -82,6 +98,7 @@ impl DataSource for LayoutReaderDataSource { filter: scan_request.filter, limit: scan_request.limit, selection: scan_request.selection, + metrics_registry: self.metrics_registry.clone(), next_row: row_range.start, end_row: row_range.end, split_size: self.split_size, @@ -101,6 +118,7 @@ struct LayoutReaderScan { filter: Option, limit: Option, selection: Selection, + metrics_registry: Option>, next_row: u64, end_row: u64, split_size: u64, @@ -153,6 +171,7 @@ impl DataSourceScan for LayoutReaderScan { limit: split_limit, row_range, selection: self.selection.clone(), + metrics_registry: self.metrics_registry.clone(), }) as SplitRef); self.next_row = split_end; @@ -170,6 +189,7 @@ struct LayoutReaderSplit { limit: Option, row_range: Range, selection: Selection, + metrics_registry: Option>, } impl Split for LayoutReaderSplit { @@ -191,6 +211,9 @@ impl Split for LayoutReaderSplit { if let Some(limit) = self.limit { builder = builder.with_limit(limit); } + if let Some(metrics) = self.metrics_registry { + builder = builder.with_metrics_registry(metrics); + } Ok(Box::pin(builder.into_array_stream()?)) } From ae39e1ab60e686772474543dab339fb83673ca47 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 08:55:36 -0500 Subject: [PATCH 03/57] Scan API Signed-off-by: Nicholas Gates --- vortex-scan/src/layout.rs | 15 ++++++++++----- vortex-scan/src/lib.rs | 1 + 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 62b9b7b4a0b..945e5fe58ff 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -25,9 +25,6 @@ use crate::api::ScanRequest; use crate::api::Split; use crate::api::SplitRef; -/// The default number of rows per Scan API split. -const DEFAULT_SPLIT_SIZE: u64 = 100_000; - /// An implementation of a [`DataSource`] that reads data from a [`LayoutReaderRef`]. pub struct LayoutReaderDataSource { reader: LayoutReaderRef, @@ -38,11 +35,16 @@ pub struct LayoutReaderDataSource { impl LayoutReaderDataSource { /// Creates a new [`LayoutReaderDataSource`]. + /// + /// By default, the entire scan is returned as a single split. This best preserves V1 + /// `ScanBuilder` behavior where one scan covers the full row range, allowing the internal + /// I/O pipeline and `SplitBy::Layout` chunking to operate without per-split overhead from + /// redundant expression resolution and layout tree traversal. pub fn new(reader: LayoutReaderRef, session: VortexSession) -> Self { Self { reader, session, - split_size: DEFAULT_SPLIT_SIZE, + split_size: u64::MAX, metrics_registry: None, } } @@ -154,7 +156,10 @@ impl DataSourceScan for LayoutReaderScan { break; } - let split_end = (self.next_row + self.split_size).min(self.end_row); + let split_end = self + .next_row + .saturating_add(self.split_size) + .min(self.end_row); let row_range = self.next_row..split_end; let split_rows = split_end - self.next_row; diff --git a/vortex-scan/src/lib.rs b/vortex-scan/src/lib.rs index d5d56e68e25..7c9f3320a91 100644 --- a/vortex-scan/src/lib.rs +++ b/vortex-scan/src/lib.rs @@ -23,6 +23,7 @@ mod scan_builder; pub use scan_builder::ScanBuilder; pub mod layout; +pub mod multi; mod repeated_scan; #[cfg(test)] mod test; From f41ccffe93b6238270b2fc76b35ee90f4e41f7d6 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 09:32:41 -0500 Subject: [PATCH 04/57] Scan API Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + vortex-duckdb/Cargo.toml | 1 + vortex-duckdb/src/scan.rs | 252 +++++++++++++++++++++++++++++--------- 3 files changed, 199 insertions(+), 55 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 578056b75b5..5871ac93a8b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10540,6 +10540,7 @@ dependencies = [ "anyhow", "async-compat", "async-fs", + "async-trait", "bindgen 0.72.1", "bitvec", "cbindgen", diff --git a/vortex-duckdb/Cargo.toml b/vortex-duckdb/Cargo.toml index d5613202e0e..06420c1e7d7 100644 --- a/vortex-duckdb/Cargo.toml +++ b/vortex-duckdb/Cargo.toml @@ -26,6 +26,7 @@ crate-type = ["staticlib", "cdylib", "rlib"] [dependencies] anyhow = { workspace = true } async-compat = { workspace = true } +async-trait = { workspace = true } async-fs = { workspace = true } bitvec = { workspace = true } custom-labels = { workspace = true } diff --git a/vortex-duckdb/src/scan.rs b/vortex-duckdb/src/scan.rs index b5382f68301..2d8030709a3 100644 --- a/vortex-duckdb/src/scan.rs +++ b/vortex-duckdb/src/scan.rs @@ -14,6 +14,7 @@ use std::task::Context; use std::task::Poll; use async_compat::Compat; +use async_trait::async_trait; use custom_labels::CURRENT_LABELSET; use futures::FutureExt; use futures::Stream; @@ -49,6 +50,12 @@ use vortex::file::VortexOpenOptions; use vortex::io::runtime::BlockingRuntime; use vortex::io::runtime::current::ThreadSafeIterator; use vortex::metrics::tracing::get_global_labels; +use vortex::scan::api::DataSource as _; +use vortex::scan::api::DataSourceRef; +use vortex::scan::api::ScanRequest; +use vortex::scan::layout::LayoutReaderDataSource; +use vortex::scan::multi::DataSourceFactory; +use vortex::scan::multi::MultiDataSource; use vortex::session::VortexSession; use vortex_utils::aliases::hash_set::HashSet; @@ -228,6 +235,37 @@ async fn open_file(url: Url, options: VortexOpenOptions) -> VortexResult, + object_cache: duckdb::ObjectCacheRef<'static>, +} + +#[async_trait] +impl DataSourceFactory for VortexFileFactory { + async fn open(&self) -> VortexResult> { + let cache = FooterCache::new(self.object_cache); + let entry = cache.entry(self.url.as_ref()); + let options = entry.apply_to_file(SESSION.open_options()); + let file = open_file(self.url.clone(), options).await?; + entry.put_if_absent(|| file.footer().clone()); + + if let Some(ref filter) = self.filter + && file.can_prune(filter)? + { + return Ok(None); + } + + let reader = file.layout_reader()?; + Ok(Some(Arc::new(LayoutReaderDataSource::new( + reader, + SESSION.clone(), + )))) + } +} + // taken from duckdb/common/constants.h COLUMN_IDENTIFIER_EMPTY // This is used by duckdb whenever there is no projection id in a logical_get node. // For some reason we cannot return an empty DataChunk and duckdb will look for the virtual column @@ -396,62 +434,28 @@ impl TableFunction for VortexTableFunction { let client_context = init_input.client_context()?; let object_cache = client_context.object_cache(); - let handle = RUNTIME.handle(); - let first_file = bind_data.first_file.clone(); - let scan_streams = stream::iter(bind_data.file_urls.clone()) - .enumerate() - .map(move |(idx, url)| { - let first_file = first_file.clone(); - let filter_expr = filter_expr.clone(); - let projection_expr = projection_expr.clone(); - let conversion_cache = Arc::new(ConversionCache::new(idx as u64)); - let object_cache = object_cache; - - handle - .spawn(async move { - let vxf = if idx == 0 { - // The first path from `file_paths` is skipped as - // the first file was already opened during bind. - Ok(first_file) - } else { - let cache = FooterCache::new(object_cache); - let entry = cache.entry(url.as_ref()); - let options = entry.apply_to_file(SESSION.open_options()); - let file = open_file(url.clone(), options).await?; - entry.put_if_absent(|| file.footer().clone()); - VortexResult::Ok(file) - }?; - - if let Some(ref filter) = filter_expr - && vxf.can_prune(filter)? - { - return Ok(None); - }; - - let scan = vxf - .scan()? - .with_some_filter(filter_expr) - .with_projection(projection_expr) - .with_ordered(false) - .map(move |split| Ok((split, conversion_cache.clone()))) - .into_stream()? - .boxed(); - - Ok(Some(scan)) - }) - .boxed() - }) - // Open up to num_workers * 2 files concurrently so we always have one ready to go. - .buffer_unordered(num_workers * 2) - .filter_map(|result| async move { result.transpose() }); + let use_scan_api = std::env::var("VORTEX_USE_SCAN_API").is_ok_and(|v| v == "1"); + + let iterator = if use_scan_api { + init_global_scan_api( + bind_data, + projection_expr, + filter_expr, + num_workers, + object_cache, + )? + } else { + init_global_direct( + bind_data, + projection_expr, + filter_expr, + num_workers, + object_cache, + )? + }; Ok(VortexGlobalData { - iterator: RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { - streams: scan_streams.boxed(), - streams_finished: false, - select_all: Default::default(), - max_concurrency: num_workers * 2, - }), + iterator, batch_id: AtomicU64::new(0), // TODO(joe): fetch this from somewhere??. ctx: ExecutionCtx::new(VortexSession::default()), @@ -545,6 +549,144 @@ impl TableFunction for VortexTableFunction { } } +type ScanIterator = ThreadSafeIterator)>>; + +/// Scan API path: creates a [`MultiDataSource`] from all files and lazily pulls splits, +/// executing each split as an independent stream for parallel consumption by DuckDB workers. +fn init_global_scan_api( + bind_data: &VortexBindData, + projection_expr: Expression, + filter_expr: Option, + num_workers: usize, + object_cache: duckdb::ObjectCacheRef<'static>, +) -> VortexResult { + let first_reader = bind_data.first_file.layout_reader()?; + let first_ds: DataSourceRef = + Arc::new(LayoutReaderDataSource::new(first_reader, SESSION.clone())); + + let factories: Vec> = bind_data.file_urls[1..] + .iter() + .map(|url| { + Arc::new(VortexFileFactory { + url: url.clone(), + filter: filter_expr.clone(), + object_cache, + }) as Arc + }) + .collect(); + + let multi_ds = + MultiDataSource::lazy(first_ds, factories, &SESSION).with_prefetch(num_workers * 2); + + let request = ScanRequest { + projection: Some(projection_expr), + filter: filter_expr, + ..Default::default() + }; + + let scan = RUNTIME.block_on(Compat::new(multi_ds.scan(request)))?; + let conversion_cache = Arc::new(ConversionCache::new(0)); + + // Lazily pull batches of splits, execute each into a stream, and feed into MultiScan. + let scan_streams = stream::unfold((scan, false), move |(mut scan, errored)| { + let cache = conversion_cache.clone(); + Compat::new(async move { + if errored { + return None; + } + match scan.next_splits(num_workers).await { + Ok(splits) if splits.is_empty() => None, + Ok(splits) => { + let streams: Vec>>> = splits + .into_iter() + .map(|split| { + let cache = cache.clone(); + let s = split.execute()?; + Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) + }) + .collect(); + Some((stream::iter(streams).boxed(), (scan, false))) + } + Err(e) => Some((stream::once(async { Err(e) }).boxed(), (scan, true))), + } + }) + }) + .flatten(); + + Ok(RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { + streams: scan_streams.boxed(), + streams_finished: false, + select_all: Default::default(), + max_concurrency: num_workers * 2, + })) +} + +/// Direct ScanBuilder path (existing behavior): opens files lazily via spawned tasks, +/// creates per-file scan streams, and drives them concurrently via [`MultiScan`]. +fn init_global_direct( + bind_data: &VortexBindData, + projection_expr: Expression, + filter_expr: Option, + num_workers: usize, + object_cache: duckdb::ObjectCacheRef<'static>, +) -> VortexResult { + let handle = RUNTIME.handle(); + let first_file = bind_data.first_file.clone(); + let scan_streams = stream::iter(bind_data.file_urls.clone()) + .enumerate() + .map(move |(idx, url)| { + let first_file = first_file.clone(); + let filter_expr = filter_expr.clone(); + let projection_expr = projection_expr.clone(); + let conversion_cache = Arc::new(ConversionCache::new(idx as u64)); + let object_cache = object_cache; + + handle + .spawn(async move { + let vxf = if idx == 0 { + // The first path from `file_paths` is skipped as + // the first file was already opened during bind. + Ok(first_file) + } else { + let cache = FooterCache::new(object_cache); + let entry = cache.entry(url.as_ref()); + let options = entry.apply_to_file(SESSION.open_options()); + let file = open_file(url.clone(), options).await?; + entry.put_if_absent(|| file.footer().clone()); + VortexResult::Ok(file) + }?; + + if let Some(ref filter) = filter_expr + && vxf.can_prune(filter)? + { + return Ok(None); + }; + + let scan = vxf + .scan()? + .with_some_filter(filter_expr) + .with_projection(projection_expr) + .with_ordered(false) + .map(move |split| Ok((split, conversion_cache.clone()))) + .into_stream()? + .boxed(); + + Ok(Some(scan)) + }) + .boxed() + }) + // Open up to num_workers * 2 files concurrently so we always have one ready to go. + .buffer_unordered(num_workers * 2) + .filter_map(|result| async move { result.transpose() }); + + Ok(RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { + streams: scan_streams.boxed(), + streams_finished: false, + select_all: Default::default(), + max_concurrency: num_workers * 2, + })) +} + struct MultiScan<'rt, T> { // A stream-of-streams of scan results. streams: BoxStream<'rt, VortexResult>>>, @@ -581,7 +723,7 @@ impl<'rt, T: 'rt> Stream for MultiScan<'rt, T> { // If all current streams returned `Poll::Pending`, then we try to fetch the next // stream to drive. The idea here is to ensure our executors are always busy with // CPU work by driving as many streams necessary to keep the I/O queues full. - if this.select_all.len() < this.max_concurrency { + if !this.streams_finished && this.select_all.len() < this.max_concurrency { match Pin::new(&mut this.streams).poll_next(cx) { Poll::Ready(Some(Ok(stream))) => { // Add the new stream to SelectAll, and continue the loop to poll it. From 23510e4ef64aab823c42fedb995bae3a40d888c1 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 09:32:46 -0500 Subject: [PATCH 05/57] Scan API Signed-off-by: Nicholas Gates --- vortex-scan/src/multi.rs | 322 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 322 insertions(+) create mode 100644 vortex-scan/src/multi.rs diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs new file mode 100644 index 00000000000..2eb8e34295b --- /dev/null +++ b/vortex-scan/src/multi.rs @@ -0,0 +1,322 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! A [`DataSource`] that combines multiple child data sources into a single scannable source. +//! +//! Splits from all children are interleaved, enabling parallel execution across files. +//! Children can be pre-opened (eager) or opened lazily via [`DataSourceFactory`] implementations, +//! with spawned prefetching to overlap file-opening I/O with split execution. + +use std::collections::VecDeque; +use std::sync::Arc; + +use async_trait::async_trait; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; +use vortex_error::vortex_err; +use vortex_io::runtime::Handle; +use vortex_io::runtime::Task; +use vortex_io::session::RuntimeSessionExt; +use vortex_session::VortexSession; + +use crate::api::DataSource; +use crate::api::DataSourceRef; +use crate::api::DataSourceScan; +use crate::api::DataSourceScanRef; +use crate::api::Estimate; +use crate::api::ScanRequest; +use crate::api::SplitRef; + +/// An async factory that produces a [`DataSource`]. +/// +/// Implementations handle engine-specific concerns like file opening, caching, and +/// statistics-based pruning. Returns `None` if the source should be skipped (e.g., pruned +/// based on file-level statistics). +#[async_trait] +pub trait DataSourceFactory: 'static + Send + Sync { + /// Opens the data source, or returns `None` if it should be skipped. + async fn open(&self) -> VortexResult>; +} + +/// Default number of deferred sources to open concurrently during scanning. +const DEFAULT_PREFETCH: usize = 8; + +/// A [`DataSource`] combining multiple children into a single scannable source. +/// +/// Children may be pre-opened or deferred via [`DataSourceFactory`]. During scanning, +/// deferred children are opened in the background using spawned tasks on the session's runtime, +/// keeping the I/O pipeline full while the engine processes splits from already-open sources. +pub struct MultiDataSource { + dtype: DType, + children: Vec, + handle: Handle, + prefetch: usize, +} + +enum MultiChild { + Opened(DataSourceRef), + Deferred(Arc), +} + +impl MultiDataSource { + /// Creates a multi-source from pre-opened data sources. + /// + /// Validates that all children share the same dtype. + pub fn try_new(children: Vec, session: &VortexSession) -> VortexResult { + let first = children + .first() + .ok_or_else(|| vortex_err!("MultiDataSource requires at least one child"))?; + let dtype = first.dtype().clone(); + + for (i, child) in children.iter().enumerate().skip(1) { + if child.dtype() != &dtype { + vortex_bail!( + "MultiDataSource dtype mismatch in child {}: expected {}, got {}", + i, + dtype, + child.dtype() + ); + } + } + + Ok(Self { + dtype, + children: children.into_iter().map(MultiChild::Opened).collect(), + handle: session.handle(), + prefetch: DEFAULT_PREFETCH, + }) + } + + /// Creates a multi-source with lazy opening. + /// + /// The first source must be pre-opened to determine the dtype (required by the sync + /// [`DataSource::dtype`] method). Remaining sources are opened lazily during scanning + /// via their factories, with dtype validated on open. + pub fn lazy( + first: DataSourceRef, + remaining: Vec>, + session: &VortexSession, + ) -> Self { + let dtype = first.dtype().clone(); + let mut children = Vec::with_capacity(1 + remaining.len()); + children.push(MultiChild::Opened(first)); + children.extend(remaining.into_iter().map(MultiChild::Deferred)); + + Self { + dtype, + children, + handle: session.handle(), + prefetch: DEFAULT_PREFETCH, + } + } + + /// Sets the number of deferred sources to open concurrently during scanning. + /// + /// Higher values overlap more file-opening I/O with split execution but use more memory + /// for in-flight metadata. Defaults to 8. + pub fn with_prefetch(mut self, prefetch: usize) -> Self { + self.prefetch = prefetch; + self + } +} + +#[async_trait] +impl DataSource for MultiDataSource { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn row_count_estimate(&self) -> Estimate { + let mut lower: u64 = 0; + let mut upper: Option = Some(0); + let mut has_deferred = false; + + for child in &self.children { + match child { + MultiChild::Opened(ds) => { + let est = ds.row_count_estimate(); + lower = lower.saturating_add(est.lower); + upper = match (upper, est.upper) { + (Some(a), Some(b)) => Some(a.saturating_add(b)), + _ => None, + }; + } + MultiChild::Deferred(_) => { + has_deferred = true; + } + } + } + + if has_deferred { + upper = None; + } + + Estimate { lower, upper } + } + + async fn scan(&self, scan_request: ScanRequest) -> VortexResult { + let mut ready = VecDeque::new(); + let mut deferred = VecDeque::new(); + + for child in &self.children { + match child { + MultiChild::Opened(ds) => ready.push_back(ds.clone()), + MultiChild::Deferred(factory) => deferred.push_back(factory.clone()), + } + } + + let remaining_limit = scan_request.limit; + + let mut scan = MultiDataSourceScan { + dtype: self.dtype.clone(), + request: scan_request, + current: None, + ready, + opening: VecDeque::new(), + deferred, + handle: self.handle.clone(), + prefetch: self.prefetch, + remaining_limit, + }; + + // Kick off initial prefetch of deferred sources. + scan.fill_pipeline(); + + Ok(Box::new(scan)) + } + + fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { + vortex_bail!("MultiDataSource splits are not yet serializable") + } +} + +struct MultiDataSourceScan { + dtype: DType, + request: ScanRequest, + /// Currently active child scan being drained. + current: Option, + /// Pre-opened sources ready to be scanned. + ready: VecDeque, + /// In-flight spawned opens. Tasks run on the session's runtime independently, + /// so they make progress even when we're not polling them. + opening: VecDeque>>>, + /// Remaining factories not yet spawned. + deferred: VecDeque>, + /// Runtime handle for spawning prefetch tasks. + handle: Handle, + /// Target number of in-flight + ready sources. + prefetch: usize, + /// Remaining row limit across all children. Decremented conservatively by each split's + /// upper row estimate. The engine enforces the exact limit at the stream level. + remaining_limit: Option, +} + +impl MultiDataSourceScan { + /// Spawns open tasks for deferred factories up to the prefetch target. + fn fill_pipeline(&mut self) { + while self.opening.len() + self.ready.len() < self.prefetch { + let Some(factory) = self.deferred.pop_front() else { + break; + }; + self.opening + .push_back(self.handle.spawn(async move { factory.open().await })); + } + } + + /// Gets the next ready data source, awaiting in-flight opens if needed. + async fn next_source(&mut self) -> VortexResult> { + loop { + if let Some(source) = self.ready.pop_front() { + return Ok(Some(source)); + } + + if let Some(task) = self.opening.pop_front() { + self.fill_pipeline(); + match task.await? { + Some(source) => return Ok(Some(source)), + None => continue, // pruned, try next + } + } + + return Ok(None); + } + } +} + +#[async_trait] +impl DataSourceScan for MultiDataSourceScan { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn remaining_splits_estimate(&self) -> Estimate { + let current_estimate = self + .current + .as_ref() + .map_or_else(|| Estimate::exact(0), |s| s.remaining_splits_estimate()); + + let remaining_sources = self.ready.len() + self.opening.len() + self.deferred.len(); + if remaining_sources == 0 { + return current_estimate; + } + + // With remaining sources whose split counts are unknown, we can only provide a lower bound. + Estimate { + lower: current_estimate.lower, + upper: None, + } + } + + async fn next_splits(&mut self, max_splits: usize) -> VortexResult> { + let mut splits = Vec::new(); + + while splits.len() < max_splits { + if self.remaining_limit.is_some_and(|l| l == 0) { + break; + } + + // Ensure we have a current scan. + if self.current.is_none() { + let Some(source) = self.next_source().await? else { + break; + }; + + if source.dtype() != &self.dtype { + vortex_bail!( + "MultiDataSource dtype mismatch: expected {}, got {}", + self.dtype, + source.dtype() + ); + } + + let mut child_request = self.request.clone(); + child_request.limit = self.remaining_limit; + self.current = Some(source.scan(child_request).await?); + } + + let scan = self + .current + .as_mut() + .ok_or_else(|| vortex_err!("expected active scan"))?; + + let child_splits = scan.next_splits(max_splits - splits.len()).await?; + + if child_splits.is_empty() { + self.current = None; + continue; + } + + if let Some(ref mut limit) = self.remaining_limit { + for split in &child_splits { + let est = split.row_count_estimate(); + *limit = limit.saturating_sub(est.upper.unwrap_or(est.lower)); + } + } + + splits.extend(child_splits); + } + + Ok(splits) + } +} From e925792010040a757fb329601facb772d9779465 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 09:35:00 -0500 Subject: [PATCH 06/57] Scan API Signed-off-by: Nicholas Gates --- .github/workflows/sql-benchmarks.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/sql-benchmarks.yml b/.github/workflows/sql-benchmarks.yml index 030953c100b..8eb1049ce33 100644 --- a/.github/workflows/sql-benchmarks.yml +++ b/.github/workflows/sql-benchmarks.yml @@ -187,6 +187,7 @@ jobs: if: matrix.remote_storage == null || github.event.pull_request.head.repo.fork == true shell: bash env: + VORTEX_USE_SCAN_API: "1" OTEL_SERVICE_NAME: "vortex-bench" OTEL_EXPORTER_OTLP_PROTOCOL: "http/protobuf" OTEL_EXPORTER_OTLP_ENDPOINT: "${{ (inputs.mode != 'pr' || github.event.pull_request.head.repo.fork == false) && secrets.OTEL_EXPORTER_OTLP_ENDPOINT || '' }}" @@ -200,6 +201,7 @@ jobs: if: matrix.remote_storage != null && (inputs.mode != 'pr' || github.event.pull_request.head.repo.fork == false) shell: bash env: + VORTEX_USE_SCAN_API: "1" AWS_REGION: "eu-west-1" OTEL_SERVICE_NAME: "vortex-bench" OTEL_EXPORTER_OTLP_PROTOCOL: "http/protobuf" From f8a7543fd65d75963aeed2592a33445e846d4155 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 10:10:43 -0500 Subject: [PATCH 07/57] Scan API Signed-off-by: Nicholas Gates --- .../bench_orchestrator/comparison/analyzer.py | 8 ++++++-- vortex-datafusion/src/persistent/opener.rs | 17 ++++++++++++++--- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/bench-orchestrator/bench_orchestrator/comparison/analyzer.py b/bench-orchestrator/bench_orchestrator/comparison/analyzer.py index 5fa05c40d4a..3ca6c5f3124 100644 --- a/bench-orchestrator/bench_orchestrator/comparison/analyzer.py +++ b/bench-orchestrator/bench_orchestrator/comparison/analyzer.py @@ -271,8 +271,12 @@ def compare_runs( # Pivot to get (query, engine, format) as rows, runs as columns pivot = combined.pivot_table(index=["query", "engine", "format"], columns="run", values="value", aggfunc="mean") + # Deduplicate labels while preserving order (two runs can share a label). + seen: set[str] = set() + unique_labels = [l for l in labels if l not in seen and not seen.add(l)] # type: ignore[func-returns-value] + # Reorder columns to match input order - pivot = pivot[[label for label in labels if label in pivot.columns]] + pivot = pivot[[label for label in unique_labels if label in pivot.columns]] # Compute ratios relative to baseline if baseline_label in pivot.columns: @@ -283,4 +287,4 @@ def compare_runs( else: result = pivot - return PivotComparison(df=result.reset_index(), baseline=baseline_label, columns=labels) + return PivotComparison(df=result.reset_index(), baseline=baseline_label, columns=unique_labels) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 56cbeeca23e..a10442bedfa 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::VortexResult; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; +use vortex::io::session::RuntimeSessionExt; use vortex::layout::LayoutReader; use vortex::metrics::Label; use vortex::metrics::MetricsRegistry; @@ -363,14 +364,24 @@ impl FileOpener for VortexOpener { .collect::>>() .map_err(|e| exec_datafusion_err!("Failed to execute Vortex split: {e}"))?; + // Spawn the Vortex-to-Arrow conversion onto a CPU thread so it doesn't + // block the polling thread. buffered(2) lets us overlap: while one + // chunk is being converted, the inner scan stream can drive I/O for + // the next chunk. + let handle = session.handle(); stream::iter(streams) .flatten() .map(move |result| { - result.and_then(|chunk| { - let mut ctx = session.create_execution_ctx(); - chunk.execute_record_batch(&stream_schema, &mut ctx) + let session = session.clone(); + let stream_schema = stream_schema.clone(); + handle.spawn_cpu(move || { + result.and_then(|chunk| { + let mut ctx = session.create_execution_ctx(); + chunk.execute_record_batch(&stream_schema, &mut ctx) + }) }) }) + .buffered(2) .boxed() } else { // Direct ScanBuilder path (existing). From d05997f6092468444bdaee7679b7004386b9b134 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 12:45:32 -0500 Subject: [PATCH 08/57] Scan API Signed-off-by: Nicholas Gates --- .../bench_orchestrator/storage/store.py | 10 +++++++--- vortex-datafusion/src/persistent/opener.rs | 13 ++++++++----- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/bench-orchestrator/bench_orchestrator/storage/store.py b/bench-orchestrator/bench_orchestrator/storage/store.py index 96847ac5df2..1703af1ad6f 100644 --- a/bench-orchestrator/bench_orchestrator/storage/store.py +++ b/bench-orchestrator/bench_orchestrator/storage/store.py @@ -96,9 +96,13 @@ def write_result(self, result: QueryResult) -> None: self._result_count += 1 def write_raw_json(self, json_line: str) -> None: - """Write a raw JSON line directly (from benchmark binary output).""" - if self._results_file: - self._results_file.write(json_line.strip() + "\n") + """Write a raw JSON line directly (from benchmark binary output). + + Non-JSON lines (e.g. DuckDB ASCII table output) are silently skipped. + """ + line = json_line.strip() + if self._results_file and line.startswith("{"): + self._results_file.write(line + "\n") self._results_file.flush() self._result_count += 1 diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index a10442bedfa..0b479f6fd09 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -364,11 +364,14 @@ impl FileOpener for VortexOpener { .collect::>>() .map_err(|e| exec_datafusion_err!("Failed to execute Vortex split: {e}"))?; - // Spawn the Vortex-to-Arrow conversion onto a CPU thread so it doesn't - // block the polling thread. buffered(2) lets us overlap: while one - // chunk is being converted, the inner scan stream can drive I/O for - // the next chunk. + // Spawn Vortex-to-Arrow conversion onto CPU threads so it doesn't + // block the polling thread. The inner scan stream already handles I/O + // and compute concurrency; this buffer just needs to be large enough + // that conversion doesn't stall the pipeline. let handle = session.handle(); + let num_workers = std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1); stream::iter(streams) .flatten() .map(move |result| { @@ -381,7 +384,7 @@ impl FileOpener for VortexOpener { }) }) }) - .buffered(2) + .buffered(num_workers) .boxed() } else { // Direct ScanBuilder path (existing). From 5f6123a6b977f2097ef420992876effec05c49c3 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 10 Feb 2026 16:40:23 -0500 Subject: [PATCH 09/57] Scan API Signed-off-by: Nicholas Gates --- bench-orchestrator/bench_orchestrator/comparison/analyzer.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/bench-orchestrator/bench_orchestrator/comparison/analyzer.py b/bench-orchestrator/bench_orchestrator/comparison/analyzer.py index 3ca6c5f3124..7ac62f188ff 100644 --- a/bench-orchestrator/bench_orchestrator/comparison/analyzer.py +++ b/bench-orchestrator/bench_orchestrator/comparison/analyzer.py @@ -272,8 +272,7 @@ def compare_runs( pivot = combined.pivot_table(index=["query", "engine", "format"], columns="run", values="value", aggfunc="mean") # Deduplicate labels while preserving order (two runs can share a label). - seen: set[str] = set() - unique_labels = [l for l in labels if l not in seen and not seen.add(l)] # type: ignore[func-returns-value] + unique_labels = list(dict.fromkeys(labels)) # Reorder columns to match input order pivot = pivot[[label for label in unique_labels if label in pivot.columns]] From c5e60a42f6637b07b1b51822c4a23d5fa9f9b507 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 10:56:08 -0500 Subject: [PATCH 10/57] Scan API Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 20 ++-- vortex-duckdb/src/scan.rs | 33 ++---- vortex-scan/src/api.rs | 17 ++- vortex-scan/src/layout.rs | 57 +++++----- vortex-scan/src/multi.rs | 115 ++++++++++++--------- 5 files changed, 120 insertions(+), 122 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 0b479f6fd09..87db49acefd 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -35,7 +35,6 @@ use vortex::array::ArrayRef; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; use vortex::error::VortexError; -use vortex::error::VortexResult; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; use vortex::io::session::RuntimeSessionExt; @@ -348,21 +347,22 @@ impl FileOpener for VortexOpener { let data_source = LayoutReaderDataSource::new(layout_reader, session.clone()) .with_some_metrics_registry(Some(metrics_registry)); - let mut scan = data_source + let scan = data_source .scan(scan_request) .await .map_err(|e| exec_datafusion_err!("Failed to create Vortex scan: {e}"))?; - let splits = scan - .next_splits(usize::MAX) + let streams: Vec<_> = scan + .splits() + .collect::>() .await - .map_err(|e| exec_datafusion_err!("Failed to get Vortex splits: {e}"))?; - - let streams: Vec<_> = splits .into_iter() - .map(|split| split.execute()) - .collect::>>() - .map_err(|e| exec_datafusion_err!("Failed to execute Vortex split: {e}"))?; + .map(|split_result| { + split_result.and_then(|split| split.execute()).map_err(|e| { + exec_datafusion_err!("Failed to execute Vortex split: {e}") + }) + }) + .collect::, _>>()?; // Spawn Vortex-to-Arrow conversion onto CPU threads so it doesn't // block the polling thread. The inner scan stream already handles I/O diff --git a/vortex-duckdb/src/scan.rs b/vortex-duckdb/src/scan.rs index 2d8030709a3..8d4f7d71109 100644 --- a/vortex-duckdb/src/scan.rs +++ b/vortex-duckdb/src/scan.rs @@ -587,31 +587,16 @@ fn init_global_scan_api( let scan = RUNTIME.block_on(Compat::new(multi_ds.scan(request)))?; let conversion_cache = Arc::new(ConversionCache::new(0)); - // Lazily pull batches of splits, execute each into a stream, and feed into MultiScan. - let scan_streams = stream::unfold((scan, false), move |(mut scan, errored)| { + // Lazily pull splits, execute each into a stream, and feed into MultiScan. + let scan_streams = scan.splits().map(move |split_result| { let cache = conversion_cache.clone(); - Compat::new(async move { - if errored { - return None; - } - match scan.next_splits(num_workers).await { - Ok(splits) if splits.is_empty() => None, - Ok(splits) => { - let streams: Vec>>> = splits - .into_iter() - .map(|split| { - let cache = cache.clone(); - let s = split.execute()?; - Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) - }) - .collect(); - Some((stream::iter(streams).boxed(), (scan, false))) - } - Err(e) => Some((stream::once(async { Err(e) }).boxed(), (scan, true))), - } - }) - }) - .flatten(); + let stream: VortexResult>> = (|| { + let split = split_result?; + let s = split.execute()?; + Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) + })(); + stream + }); Ok(RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { streams: scan_streams.boxed(), diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index 784a87f1ef5..d366a17be0b 100644 --- a/vortex-scan/src/api.rs +++ b/vortex-scan/src/api.rs @@ -33,6 +33,9 @@ use vortex_session::VortexSession; use crate::Selection; +/// A sendable stream of splits. +pub type SplitStream = BoxStream<'static, VortexResult>; + /// Opens a Vortex [`DataSource`] from a URI. /// /// Configuration can be passed via the URI query parameters, similar to JDBC / ADBC. @@ -104,23 +107,17 @@ pub struct ScanRequest { pub type DataSourceScanRef = Box; /// A data source scan produces splits that can be executed to read data from the source. -#[async_trait] pub trait DataSourceScan: 'static + Send { /// The returned dtype of the scan. fn dtype(&self) -> &DType; - /// An estimate of the remaining splits. - fn remaining_splits_estimate(&self) -> Estimate; + /// An estimate of the total number of splits. + fn splits_estimate(&self) -> Estimate; - /// Returns the next batch of splits to be processed. - /// - /// This should not return _more_ than `max_splits` splits, but may return fewer. - async fn next_splits(&mut self, max_splits: usize) -> VortexResult>; + /// Returns a stream of splits to be processed. + fn splits(self: Box) -> SplitStream; } -/// A stream of splits. -pub type SplitStream = BoxStream<'static, VortexResult>; - /// A reference-counted split. pub type SplitRef = Box; diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 945e5fe58ff..381a0d23474 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -6,6 +6,8 @@ use std::ops::Range; use std::sync::Arc; use async_trait::async_trait; +use futures::stream; +use futures::stream::StreamExt; use vortex_array::expr::Expression; use vortex_array::stream::SendableArrayStream; use vortex_dtype::DType; @@ -24,6 +26,7 @@ use crate::api::Estimate; use crate::api::ScanRequest; use crate::api::Split; use crate::api::SplitRef; +use crate::api::SplitStream; /// An implementation of a [`DataSource`] that reads data from a [`LayoutReaderRef`]. pub struct LayoutReaderDataSource { @@ -126,13 +129,12 @@ struct LayoutReaderScan { split_size: u64, } -#[async_trait] impl DataSourceScan for LayoutReaderScan { fn dtype(&self) -> &DType { &self.dtype } - fn remaining_splits_estimate(&self) -> Estimate { + fn splits_estimate(&self) -> Estimate { if self.next_row >= self.end_row { return Estimate::exact(0); } @@ -144,45 +146,44 @@ impl DataSourceScan for LayoutReaderScan { } } - async fn next_splits(&mut self, max_splits: usize) -> VortexResult> { - let mut splits = Vec::new(); - - for _ in 0..max_splits { - if self.next_row >= self.end_row { - break; + fn splits(self: Box) -> SplitStream { + stream::unfold(*self, |mut state| async move { + if state.next_row >= state.end_row { + return None; } - if self.limit.is_some_and(|limit| limit == 0) { - break; + if state.limit.is_some_and(|limit| limit == 0) { + return None; } - let split_end = self + let split_end = state .next_row - .saturating_add(self.split_size) - .min(self.end_row); - let row_range = self.next_row..split_end; - let split_rows = split_end - self.next_row; + .saturating_add(state.split_size) + .min(state.end_row); + let row_range = state.next_row..split_end; + let split_rows = split_end - state.next_row; - let split_limit = self.limit; - if let Some(ref mut limit) = self.limit { + let split_limit = state.limit; + if let Some(ref mut limit) = state.limit { *limit = limit.saturating_sub(split_rows); } - splits.push(Box::new(LayoutReaderSplit { - reader: self.reader.clone(), - session: self.session.clone(), - projection: self.projection.clone(), - filter: self.filter.clone(), + let split = Box::new(LayoutReaderSplit { + reader: state.reader.clone(), + session: state.session.clone(), + projection: state.projection.clone(), + filter: state.filter.clone(), limit: split_limit, row_range, - selection: self.selection.clone(), - metrics_registry: self.metrics_registry.clone(), - }) as SplitRef); + selection: state.selection.clone(), + metrics_registry: state.metrics_registry.clone(), + }) as SplitRef; - self.next_row = split_end; - } + state.next_row = split_end; - Ok(splits) + Some((Ok(split), state)) + }) + .boxed() } } diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 2eb8e34295b..489b6f06c2b 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -11,6 +11,8 @@ use std::collections::VecDeque; use std::sync::Arc; use async_trait::async_trait; +use futures::StreamExt; +use futures::stream; use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_error::vortex_bail; @@ -27,6 +29,7 @@ use crate::api::DataSourceScanRef; use crate::api::Estimate; use crate::api::ScanRequest; use crate::api::SplitRef; +use crate::api::SplitStream; /// An async factory that produces a [`DataSource`]. /// @@ -244,17 +247,16 @@ impl MultiDataSourceScan { } } -#[async_trait] impl DataSourceScan for MultiDataSourceScan { fn dtype(&self) -> &DType { &self.dtype } - fn remaining_splits_estimate(&self) -> Estimate { + fn splits_estimate(&self) -> Estimate { let current_estimate = self .current .as_ref() - .map_or_else(|| Estimate::exact(0), |s| s.remaining_splits_estimate()); + .map_or_else(|| Estimate::exact(0), |s| s.splits_estimate()); let remaining_sources = self.ready.len() + self.opening.len() + self.deferred.len(); if remaining_sources == 0 { @@ -268,55 +270,68 @@ impl DataSourceScan for MultiDataSourceScan { } } - async fn next_splits(&mut self, max_splits: usize) -> VortexResult> { - let mut splits = Vec::new(); - - while splits.len() < max_splits { - if self.remaining_limit.is_some_and(|l| l == 0) { - break; - } - - // Ensure we have a current scan. - if self.current.is_none() { - let Some(source) = self.next_source().await? else { - break; - }; - - if source.dtype() != &self.dtype { - vortex_bail!( - "MultiDataSource dtype mismatch: expected {}, got {}", - self.dtype, - source.dtype() - ); - } - - let mut child_request = self.request.clone(); - child_request.limit = self.remaining_limit; - self.current = Some(source.scan(child_request).await?); - } - - let scan = self - .current - .as_mut() - .ok_or_else(|| vortex_err!("expected active scan"))?; - - let child_splits = scan.next_splits(max_splits - splits.len()).await?; + fn splits(self: Box) -> SplitStream { + stream::unfold( + (Some(*self), None::), + |(mut state, mut current_stream)| async move { + loop { + // Try to pull from the current child's split stream. + if let Some(ref mut child_stream) = current_stream { + match child_stream.next().await { + Some(Ok(split)) => { + if let Some(ref mut s) = state + && let Some(ref mut limit) = s.remaining_limit + { + let est = split.row_count_estimate(); + *limit = limit.saturating_sub(est.upper.unwrap_or(est.lower)); + } + return Some((Ok(split), (state, current_stream))); + } + Some(Err(e)) => { + return Some((Err(e), (None, None))); + } + None => { + // Current child exhausted, move to next. + drop(current_stream.take()); + } + } + } + + let s = state.as_mut()?; + + if s.remaining_limit.is_some_and(|l| l == 0) { + return None; + } + + // Get the next data source. + let source = match s.next_source().await { + Ok(Some(source)) => source, + Ok(None) => return None, + Err(e) => return Some((Err(e), (None, None))), + }; - if child_splits.is_empty() { - self.current = None; - continue; - } + if source.dtype() != &s.dtype { + return Some(( + Err(vortex_err!( + "MultiDataSource dtype mismatch: expected {}, got {}", + s.dtype, + source.dtype() + )), + (None, None), + )); + } + + let mut child_request = s.request.clone(); + child_request.limit = s.remaining_limit; + let child_scan = match source.scan(child_request).await { + Ok(scan) => scan, + Err(e) => return Some((Err(e), (None, None))), + }; - if let Some(ref mut limit) = self.remaining_limit { - for split in &child_splits { - let est = split.row_count_estimate(); - *limit = limit.saturating_sub(est.upper.unwrap_or(est.lower)); + current_stream = Some(child_scan.splits()); } - } - - splits.extend(child_splits); - } - - Ok(splits) + }, + ) + .boxed() } } From 2d6c816568a4b32630714f4575f2adedb82ecbf5 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 11:29:54 -0500 Subject: [PATCH 11/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 19 +--- vortex-duckdb/src/scan.rs | 2 +- vortex-scan/src/api.rs | 9 +- vortex-scan/src/layout.rs | 118 +++++++++++---------- vortex-scan/src/multi.rs | 9 +- 5 files changed, 77 insertions(+), 80 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 87db49acefd..b69375c2ea8 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -30,6 +30,7 @@ use futures::StreamExt; use futures::TryStreamExt; use futures::stream; use object_store::path::Path; +use tokio_stream::StreamExt; use tracing::Instrument; use vortex::array::ArrayRef; use vortex::array::VortexSessionExecute; @@ -349,21 +350,8 @@ impl FileOpener for VortexOpener { let scan = data_source .scan(scan_request) - .await .map_err(|e| exec_datafusion_err!("Failed to create Vortex scan: {e}"))?; - let streams: Vec<_> = scan - .splits() - .collect::>() - .await - .into_iter() - .map(|split_result| { - split_result.and_then(|split| split.execute()).map_err(|e| { - exec_datafusion_err!("Failed to execute Vortex split: {e}") - }) - }) - .collect::, _>>()?; - // Spawn Vortex-to-Arrow conversion onto CPU threads so it doesn't // block the polling thread. The inner scan stream already handles I/O // and compute concurrency; this buffer just needs to be large enough @@ -372,8 +360,9 @@ impl FileOpener for VortexOpener { let num_workers = std::thread::available_parallelism() .map(|n| n.get()) .unwrap_or(1); - stream::iter(streams) - .flatten() + scan.splits() + .map(|split| split.and_then(|s| s.execute())) + .try_flatten() .map(move |result| { let session = session.clone(); let stream_schema = stream_schema.clone(); diff --git a/vortex-duckdb/src/scan.rs b/vortex-duckdb/src/scan.rs index 8d4f7d71109..45db265ca3d 100644 --- a/vortex-duckdb/src/scan.rs +++ b/vortex-duckdb/src/scan.rs @@ -584,7 +584,7 @@ fn init_global_scan_api( ..Default::default() }; - let scan = RUNTIME.block_on(Compat::new(multi_ds.scan(request)))?; + let scan = multi_ds.scan(request)?; let conversion_cache = Arc::new(ConversionCache::new(0)); // Lazily pull splits, execute each into a stream, and feed into MultiScan. diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index d366a17be0b..c4d4d1de4b3 100644 --- a/vortex-scan/src/api.rs +++ b/vortex-scan/src/api.rs @@ -66,12 +66,11 @@ pub type DataSourceRef = Arc; /// /// The DataSource may be used multiple times to create multiple scans, whereas each scan and each /// split of a scan can only be consumed once. -#[async_trait] pub trait DataSource: 'static + Send + Sync { /// Returns the dtype of the source. fn dtype(&self) -> &DType; - /// Returns an estimate of the row count of the source. + /// Returns an estimate of the row count of the un-filtered source. fn row_count_estimate(&self) -> Estimate; /// Serialize the [`DataSource`] to pass to a remote worker. @@ -83,7 +82,7 @@ pub trait DataSource: 'static + Send + Sync { fn deserialize_split(&self, data: &[u8], session: &VortexSession) -> VortexResult; /// Returns a scan over the source. - async fn scan(&self, scan_request: ScanRequest) -> VortexResult; + fn scan(&self, scan_request: ScanRequest) -> VortexResult; } /// A request to scan a data source. @@ -111,8 +110,8 @@ pub trait DataSourceScan: 'static + Send { /// The returned dtype of the scan. fn dtype(&self) -> &DType; - /// An estimate of the total number of splits. - fn splits_estimate(&self) -> Estimate; + /// Returns an estimate of the total number of splits the scan will produce. + fn split_count_estimate(&self) -> Estimate; /// Returns a stream of splits to be processed. fn splits(self: Box) -> SplitStream; diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 381a0d23474..8e4a3efb507 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -3,10 +3,12 @@ use std::any::Any; use std::ops::Range; +use std::pin::Pin; use std::sync::Arc; +use std::task::Context; +use std::task::Poll; -use async_trait::async_trait; -use futures::stream; +use futures::Stream; use futures::stream::StreamExt; use vortex_array::expr::Expression; use vortex_array::stream::SendableArrayStream; @@ -75,7 +77,6 @@ impl LayoutReaderDataSource { } } -#[async_trait] impl DataSource for LayoutReaderDataSource { fn dtype(&self) -> &DType { self.reader.dtype() @@ -85,7 +86,11 @@ impl DataSource for LayoutReaderDataSource { Estimate::exact(self.reader.row_count()) } - async fn scan(&self, scan_request: ScanRequest) -> VortexResult { + fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { + vortex_bail!("LayoutReader splits are not yet serializable"); + } + + fn scan(&self, scan_request: ScanRequest) -> VortexResult { let total_rows = self.reader.row_count(); let row_range = scan_request.row_range.unwrap_or(0..total_rows); @@ -109,10 +114,6 @@ impl DataSource for LayoutReaderDataSource { split_size: self.split_size, })) } - - fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { - vortex_bail!("LayoutReader splits are not yet serializable"); - } } struct LayoutReaderScan { @@ -134,56 +135,65 @@ impl DataSourceScan for LayoutReaderScan { &self.dtype } - fn splits_estimate(&self) -> Estimate { - if self.next_row >= self.end_row { - return Estimate::exact(0); + fn split_count_estimate(&self) -> Estimate { + let (lower, upper) = self.size_hint(); + Estimate { lower, upper } + } + + fn splits(self: Box) -> SplitStream { + (*self).boxed() + } +} + +impl Stream for LayoutReaderScan { + type Item = VortexResult; + + fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + + if this.next_row >= this.end_row { + return Poll::Ready(None); } - let remaining_rows = self.end_row - self.next_row; - let splits = remaining_rows.div_ceil(self.split_size); - Estimate { - lower: 0, - upper: Some(usize::try_from(splits).unwrap_or(usize::MAX)), + + if this.limit.is_some_and(|limit| limit == 0) { + return Poll::Ready(None); + } + + let split_end = this + .next_row + .saturating_add(this.split_size) + .min(this.end_row); + let row_range = this.next_row..split_end; + let split_rows = split_end - this.next_row; + + let split_limit = this.limit; + if let Some(ref mut limit) = this.limit { + *limit = limit.saturating_sub(split_rows); } + + let split = Box::new(LayoutReaderSplit { + reader: this.reader.clone(), + session: this.session.clone(), + projection: this.projection.clone(), + filter: this.filter.clone(), + limit: split_limit, + row_range, + selection: this.selection.clone(), + metrics_registry: this.metrics_registry.clone(), + }) as SplitRef; + + this.next_row = split_end; + + Poll::Ready(Some(Ok(split))) } - fn splits(self: Box) -> SplitStream { - stream::unfold(*self, |mut state| async move { - if state.next_row >= state.end_row { - return None; - } - - if state.limit.is_some_and(|limit| limit == 0) { - return None; - } - - let split_end = state - .next_row - .saturating_add(state.split_size) - .min(state.end_row); - let row_range = state.next_row..split_end; - let split_rows = split_end - state.next_row; - - let split_limit = state.limit; - if let Some(ref mut limit) = state.limit { - *limit = limit.saturating_sub(split_rows); - } - - let split = Box::new(LayoutReaderSplit { - reader: state.reader.clone(), - session: state.session.clone(), - projection: state.projection.clone(), - filter: state.filter.clone(), - limit: split_limit, - row_range, - selection: state.selection.clone(), - metrics_registry: state.metrics_registry.clone(), - }) as SplitRef; - - state.next_row = split_end; - - Some((Ok(split), state)) - }) - .boxed() + fn size_hint(&self) -> (usize, Option) { + if self.next_row >= self.end_row { + return (0, Some(0)); + } + let remaining_rows = self.end_row - self.next_row; + let splits = remaining_rows.div_ceil(self.split_size); + (0, Some(usize::try_from(splits).unwrap_or(usize::MAX))) } } diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 489b6f06c2b..937dbff77ea 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -124,7 +124,6 @@ impl MultiDataSource { } } -#[async_trait] impl DataSource for MultiDataSource { fn dtype(&self) -> &DType { &self.dtype @@ -158,7 +157,7 @@ impl DataSource for MultiDataSource { Estimate { lower, upper } } - async fn scan(&self, scan_request: ScanRequest) -> VortexResult { + fn scan(&self, scan_request: ScanRequest) -> VortexResult { let mut ready = VecDeque::new(); let mut deferred = VecDeque::new(); @@ -252,11 +251,11 @@ impl DataSourceScan for MultiDataSourceScan { &self.dtype } - fn splits_estimate(&self) -> Estimate { + fn split_count_estimate(&self) -> Estimate { let current_estimate = self .current .as_ref() - .map_or_else(|| Estimate::exact(0), |s| s.splits_estimate()); + .map_or_else(|| Estimate::exact(0), |s| s.split_count_estimate()); let remaining_sources = self.ready.len() + self.opening.len() + self.deferred.len(); if remaining_sources == 0 { @@ -323,7 +322,7 @@ impl DataSourceScan for MultiDataSourceScan { let mut child_request = s.request.clone(); child_request.limit = s.remaining_limit; - let child_scan = match source.scan(child_request).await { + let child_scan = match source.scan(child_request) { Ok(scan) => scan, Err(e) => return Some((Err(e), (None, None))), }; From 5a7e04751be7707c217774a09df6bc85eb3940dd Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 11:40:52 -0500 Subject: [PATCH 12/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index b69375c2ea8..5ee659c4d41 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -30,7 +30,6 @@ use futures::StreamExt; use futures::TryStreamExt; use futures::stream; use object_store::path::Path; -use tokio_stream::StreamExt; use tracing::Instrument; use vortex::array::ArrayRef; use vortex::array::VortexSessionExecute; From 07cb0d33ad57b8efd0e94d3f26585242f7f83cfd Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 14:27:07 -0500 Subject: [PATCH 13/57] DataFusion streaming Signed-off-by: Nicholas Gates --- Cargo.lock | 2 + benchmarks/datafusion-bench/Cargo.toml | 2 + benchmarks/datafusion-bench/src/main.rs | 97 ++++++++++ vortex-datafusion/src/lib.rs | 2 + vortex-datafusion/src/v2/exec.rs | 221 +++++++++++++++++++++ vortex-datafusion/src/v2/mod.rs | 14 ++ vortex-datafusion/src/v2/table.rs | 246 ++++++++++++++++++++++++ 7 files changed, 584 insertions(+) create mode 100644 vortex-datafusion/src/v2/exec.rs create mode 100644 vortex-datafusion/src/v2/mod.rs create mode 100644 vortex-datafusion/src/v2/table.rs diff --git a/Cargo.lock b/Cargo.lock index 5871ac93a8b..977cabb8114 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2327,6 +2327,7 @@ version = "0.1.0" dependencies = [ "anyhow", "arrow-ipc", + "async-trait", "clap", "custom-labels", "datafusion 52.1.0", @@ -2342,6 +2343,7 @@ dependencies = [ "parking_lot", "tokio", "url", + "vortex", "vortex-bench", "vortex-cuda", "vortex-datafusion", diff --git a/benchmarks/datafusion-bench/Cargo.toml b/benchmarks/datafusion-bench/Cargo.toml index 7ac28193188..2c3ddc4bce5 100644 --- a/benchmarks/datafusion-bench/Cargo.toml +++ b/benchmarks/datafusion-bench/Cargo.toml @@ -17,6 +17,7 @@ publish = false [dependencies] anyhow = { workspace = true } arrow-ipc.workspace = true +async-trait = { workspace = true } clap = { workspace = true, features = ["derive"] } custom-labels = { workspace = true } datafusion = { workspace = true, features = [ @@ -36,6 +37,7 @@ opentelemetry_sdk.workspace = true parking_lot = { workspace = true } tokio = { workspace = true, features = ["full"] } url = { workspace = true } +vortex = { workspace = true, features = ["object_store", "files", "tokio"] } vortex-bench = { workspace = true } vortex-cuda = { workspace = true, optional = true } vortex-datafusion = { workspace = true } diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 4ee078dd2dd..d1369ce4aad 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -5,6 +5,7 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::Instant; +use async_trait::async_trait; use clap::Parser; use clap::value_parser; use custom_labels::asynchronous::Label; @@ -26,6 +27,11 @@ use datafusion_physical_plan::collect; use futures::StreamExt; use parking_lot::Mutex; use tokio::fs::File; +use vortex::file::OpenOptionsSessionExt; +use vortex::scan::api::DataSourceRef; +use vortex::scan::layout::LayoutReaderDataSource; +use vortex::scan::multi::DataSourceFactory; +use vortex::scan::multi::MultiDataSource; use vortex_bench::Benchmark; use vortex_bench::BenchmarkArg; use vortex_bench::CompactionStrategy; @@ -33,6 +39,7 @@ use vortex_bench::Engine; use vortex_bench::Format; use vortex_bench::Opt; use vortex_bench::Opts; +use vortex_bench::SESSION; use vortex_bench::conversions::convert_parquet_directory_to_vortex; use vortex_bench::create_benchmark; use vortex_bench::create_output_writer; @@ -220,6 +227,10 @@ async fn main() -> anyhow::Result<()> { Ok(()) } +fn use_v2() -> bool { + std::env::var("VORTEX_DATAFUSION_V2").is_ok_and(|v| v == "1") +} + async fn register_benchmark_tables( session: &SessionContext, benchmark: &B, @@ -227,6 +238,9 @@ async fn register_benchmark_tables( ) -> anyhow::Result<()> { match format { Format::Arrow => register_arrow_tables(session, benchmark).await, + _ if use_v2() && matches!(format, Format::OnDiskVortex | Format::VortexCompact) => { + register_v2_tables(session, benchmark, format).await + } _ => { let benchmark_base = benchmark.data_url().join(&format!("{}/", format.name()))?; let file_format = format_to_df_format(format); @@ -265,6 +279,89 @@ async fn register_benchmark_tables( } } +/// A [`DataSourceFactory`] that lazily opens a single Vortex file. +struct VortexFileFactory { + path: PathBuf, +} + +#[async_trait] +impl DataSourceFactory for VortexFileFactory { + async fn open(&self) -> vortex::error::VortexResult> { + let file = SESSION.open_options().open_path(&self.path).await?; + let reader = file.layout_reader()?; + Ok(Some(Arc::new(LayoutReaderDataSource::new( + reader, + SESSION.clone(), + )))) + } +} + +/// Register tables using the V2 `VortexTable` + `MultiDataSource` path. +async fn register_v2_tables( + session: &SessionContext, + benchmark: &B, + format: Format, +) -> anyhow::Result<()> { + use vortex_datafusion::v2::VortexTable; + + let base_path = benchmark + .data_url() + .to_file_path() + .map_err(|_| anyhow::anyhow!("V2 path requires local file:// URL"))? + .join(format.name()); + + let dir_entries: Vec<_> = std::fs::read_dir(&base_path)?.collect::, _>>()?; + + for table in benchmark.table_specs().iter() { + let pattern = benchmark.pattern(table.name, format); + + // Find files matching this table's glob pattern. + let mut matching_paths: Vec = dir_entries + .iter() + .filter(|entry| { + let filename = entry.file_name(); + let filename_str = filename.to_str().unwrap_or(""); + match &pattern { + Some(p) => p.matches(filename_str), + None => filename_str == format!("{}.{}", table.name, format.ext()), + } + }) + .map(|entry| entry.path()) + .collect(); + matching_paths.sort(); + + anyhow::ensure!( + !matching_paths.is_empty(), + "no files found for table {}", + table.name + ); + + // Open the first file eagerly to get the dtype/schema. + let first_file = SESSION.open_options().open_path(&matching_paths[0]).await?; + let arrow_schema = Arc::new(first_file.dtype().to_arrow_schema()?); + let first_reader = first_file.layout_reader()?; + let first_source: DataSourceRef = + Arc::new(LayoutReaderDataSource::new(first_reader, SESSION.clone())); + + // Create lazy factories for remaining files. + let remaining: Vec> = matching_paths[1..] + .iter() + .map(|path| Arc::new(VortexFileFactory { path: path.clone() }) as _) + .collect(); + + let data_source: DataSourceRef = if remaining.is_empty() { + first_source + } else { + Arc::new(MultiDataSource::lazy(first_source, remaining, &SESSION)) + }; + + let table_provider = Arc::new(VortexTable::new(data_source, SESSION.clone(), arrow_schema)); + session.register_table(table.name, table_provider)?; + } + + Ok(()) +} + /// Load Arrow IPC files into in-memory DataFusion tables. async fn register_arrow_tables( session: &SessionContext, diff --git a/vortex-datafusion/src/lib.rs b/vortex-datafusion/src/lib.rs index 46a964e667a..5290af32e51 100644 --- a/vortex-datafusion/src/lib.rs +++ b/vortex-datafusion/src/lib.rs @@ -10,6 +10,8 @@ use vortex::expr::stats::Precision; mod convert; mod persistent; +/// V2 DataFusion integration using direct `TableProvider` + `ExecutionPlan` implementation. +pub mod v2; #[cfg(test)] mod tests; diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs new file mode 100644 index 00000000000..583c978045b --- /dev/null +++ b/vortex-datafusion/src/v2/exec.rs @@ -0,0 +1,221 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! [`VortexExec`] implements DataFusion's [`ExecutionPlan`] trait, mapping each Vortex split +//! to one DataFusion partition. + +use std::any::Any; +use std::fmt; +use std::fmt::Formatter; +use std::sync::Arc; + +use arrow_schema::SchemaRef; +use datafusion_common::DataFusionError; +use datafusion_common::Result as DFResult; +use datafusion_common::Statistics; +use datafusion_common::stats::Precision as DFPrecision; +use datafusion_execution::SendableRecordBatchStream; +use datafusion_execution::TaskContext; +use datafusion_physical_plan::DisplayAs; +use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_plan::ExecutionPlan; +use datafusion_physical_plan::Partitioning; +use datafusion_physical_plan::PlanProperties; +use datafusion_physical_plan::execution_plan::Boundedness; +use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::stream::RecordBatchStreamAdapter; +use futures::StreamExt; +use tokio::sync::Mutex; +use vortex::array::VortexSessionExecute; +use vortex::array::arrow::ArrowArrayExecutor; +use vortex::scan::api::Estimate; +use vortex::scan::api::SplitRef; +use vortex::session::VortexSession; + +/// A DataFusion [`ExecutionPlan`] that executes Vortex splits as partitions. +/// +/// Each partition corresponds to one Vortex [`vortex::scan::api::Split`]. The split is consumed +/// on first execute; re-executing the same partition returns an error. +pub struct VortexExec { + splits: Vec>>, + partition_stats: Vec, + session: VortexSession, + schema: SchemaRef, + properties: PlanProperties, +} + +impl VortexExec { + /// Creates a new [`VortexExec`] from a list of splits, output schema, and session. + /// + /// The provided arrow schema will be used to execute the array chunks into Arrow record + /// batches. It must be compatible with the schema of the splits, but no eager validation is + /// performed. + pub(crate) fn new(splits: Vec, schema: SchemaRef, session: VortexSession) -> Self { + let n = splits.len(); + let properties = PlanProperties::new( + datafusion_physical_expr::EquivalenceProperties::new(Arc::clone(&schema)), + Partitioning::UnknownPartitioning(n), + EmissionType::Incremental, + Boundedness::Bounded, + ); + + let partition_stats: Vec = splits + .iter() + .map(|split| Statistics { + num_rows: estimate_to_df_precision(&split.row_count_estimate()), + total_byte_size: estimate_to_df_precision(&split.byte_size_estimate()), + column_statistics: vec![], + }) + .collect(); + + Self { + splits: splits.into_iter().map(|s| Mutex::new(Some(s))).collect(), + partition_stats, + session, + schema, + properties, + } + } +} + +impl fmt::Debug for VortexExec { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + f.debug_struct("VortexExec") + .field("partitions", &self.splits.len()) + .field("schema", &self.schema) + .finish() + } +} + +impl DisplayAs for VortexExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + write!(f, "VortexExec: partitions={}", self.splits.len()) + } +} + +impl ExecutionPlan for VortexExec { + fn name(&self) -> &str { + "VortexExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DFResult> { + if !children.is_empty() { + return Err(DataFusionError::Internal( + "VortexExec is a leaf node and does not accept children".to_string(), + )); + } + Ok(self) + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> DFResult { + let split = { + let split_slot = self.splits.get(partition).ok_or_else(|| { + DataFusionError::Internal(format!( + "VortexExec: partition index {partition} out of range ({})", + self.splits.len() + )) + })?; + split_slot + .try_lock() + .map_err(|_| { + DataFusionError::Internal(format!( + "VortexExec: partition {partition} is already being executed" + )) + })? + .take() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "VortexExec: partition {partition} has already been executed" + )) + })? + }; + + let array_stream = split + .execute() + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + let schema = self.schema.clone(); + let session = self.session.clone(); + let stream = array_stream.map(move |result| { + // TODO(ngates): do I need to spawn this? + let mut ctx = session.create_execution_ctx(); + result + .and_then(|chunk| chunk.execute_record_batch(&schema, &mut ctx)) + .map_err(|e| DataFusionError::External(Box::new(e))) + }); + + Ok(Box::pin(RecordBatchStreamAdapter::new( + Arc::clone(&self.schema), + stream, + ))) + } + + fn partition_statistics(&self, partition: Option) -> DFResult { + match partition { + Some(idx) => { + let stats = self.partition_stats.get(idx).ok_or_else(|| { + DataFusionError::Internal(format!( + "VortexExec: partition index {idx} out of range ({})", + self.partition_stats.len() + )) + })?; + Ok(stats.clone()) + } + None => { + let mut total_rows: DFPrecision = DFPrecision::Absent; + let mut total_bytes: DFPrecision = DFPrecision::Absent; + for stats in &self.partition_stats { + total_rows = total_rows.add(&stats.num_rows); + total_bytes = total_bytes.add(&stats.total_byte_size); + } + Ok(Statistics { + num_rows: total_rows, + total_byte_size: total_bytes, + column_statistics: vec![], + }) + } + } + } + + fn try_swapping_with_projection( + &self, + _projection: &ProjectionExec, + ) -> DFResult>> { + // We can push down _all_ projections! Possibly... + Ok(None) + } +} + +/// Convert a Vortex [`Estimate`] to a DataFusion [`Precision`](DFPrecision). +fn estimate_to_df_precision(est: &Estimate) -> DFPrecision { + match est.upper { + Some(upper) if upper == est.lower => { + DFPrecision::Exact(usize::try_from(upper).unwrap_or(usize::MAX)) + } + Some(upper) => DFPrecision::Inexact(usize::try_from(upper).unwrap_or(usize::MAX)), + None if est.lower > 0 => { + DFPrecision::Inexact(usize::try_from(est.lower).unwrap_or(usize::MAX)) + } + None => DFPrecision::Absent, + } +} diff --git a/vortex-datafusion/src/v2/mod.rs b/vortex-datafusion/src/v2/mod.rs new file mode 100644 index 00000000000..a12468bcc7d --- /dev/null +++ b/vortex-datafusion/src/v2/mod.rs @@ -0,0 +1,14 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! An experimental implementation of the Vortex Scan API for DataFusion. +//! +//! This integration directly implements `TableProvider` + `ExecutionPlan`, bypassing DataFusion's +//! `FileFormat` abstraction. Instead, we prefer to build out Vortex's MultiFileDataSource in order +//! to provide the same level of functionality across all query engines. + +mod exec; +mod table; + +pub use exec::VortexExec; +pub use table::VortexTable; diff --git a/vortex-datafusion/src/v2/table.rs b/vortex-datafusion/src/v2/table.rs new file mode 100644 index 00000000000..a52eb1116eb --- /dev/null +++ b/vortex-datafusion/src/v2/table.rs @@ -0,0 +1,246 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! [`VortexTable`] implements DataFusion's [`TableProvider`] trait, providing a direct +//! integration between a Vortex [`DataSource`] and DataFusion's query engine. + +use std::any::Any; +use std::fmt; +use std::sync::Arc; + +use arrow_schema::SchemaRef; +use async_trait::async_trait; +use datafusion_catalog::Session; +use datafusion_catalog::TableProvider; +use datafusion_common::Result as DFResult; +use datafusion_expr::Expr; +use datafusion_expr::Operator as DFOperator; +use datafusion_expr::TableProviderFilterPushDown; +use datafusion_expr::TableType; +use datafusion_physical_plan::ExecutionPlan; +use futures::TryStreamExt; +use vortex::compute::LikeOptions; +use vortex::dtype::DType; +use vortex::dtype::Nullability; +use vortex::dtype::arrow::FromArrowType; +use vortex::expr::Binary; +use vortex::expr::Expression; +use vortex::expr::Like; +use vortex::expr::Operator; +use vortex::expr::VTableExt; +use vortex::expr::and_collect; +use vortex::expr::cast; +use vortex::expr::get_item; +use vortex::expr::is_null; +use vortex::expr::list_contains; +use vortex::expr::lit; +use vortex::expr::not; +use vortex::expr::pack; +use vortex::expr::root; +use vortex::scalar::Scalar; +use vortex::scan::api::DataSourceRef; +use vortex::scan::api::ScanRequest; +use vortex::session::VortexSession; + +use crate::convert::FromDataFusion; +use crate::v2::exec::VortexExec; + +fn vx_err(e: vortex::error::VortexError) -> datafusion_common::DataFusionError { + datafusion_common::DataFusionError::External(Box::new(e)) +} + +/// Try to convert a DataFusion logical [`Expr`] into a Vortex [`Expression`]. +/// +/// Returns `None` if the expression contains unsupported nodes. +fn try_convert_expr(expr: &Expr) -> Option { + match expr { + Expr::Column(col) => Some(get_item(col.name.clone(), root())), + Expr::Literal(value, _) => Some(lit(Scalar::from_df(value))), + Expr::BinaryExpr(binary) => { + let left = try_convert_expr(&binary.left)?; + let right = try_convert_expr(&binary.right)?; + let op = try_convert_operator(&binary.op)?; + Some(Binary.new_expr(op, [left, right])) + } + Expr::Not(child) => Some(not(try_convert_expr(child)?)), + Expr::IsNull(child) => Some(is_null(try_convert_expr(child)?)), + Expr::IsNotNull(child) => Some(not(is_null(try_convert_expr(child)?))), + Expr::Like(like) => { + let child = try_convert_expr(&like.expr)?; + let pattern = try_convert_expr(&like.pattern)?; + Some(Like.new_expr( + LikeOptions { + negated: like.negated, + case_insensitive: like.case_insensitive, + }, + [child, pattern], + )) + } + Expr::Cast(cast_expr) => { + let child = try_convert_expr(&cast_expr.expr)?; + let target = DType::from_arrow((&cast_expr.data_type, Nullability::Nullable)); + Some(cast(child, target)) + } + Expr::InList(in_list) => { + let value = try_convert_expr(&in_list.expr)?; + let scalars: Option> = in_list + .list + .iter() + .map(|e| match e { + Expr::Literal(v, _) => Some(Scalar::from_df(v)), + _ => None, + }) + .collect(); + let scalars = scalars?; + let first_dtype = scalars.first()?.dtype().clone(); + let list_scalar = Scalar::list(first_dtype, scalars, Nullability::Nullable); + let expr = list_contains(lit(list_scalar), value); + if in_list.negated { + Some(not(expr)) + } else { + Some(expr) + } + } + _ => None, + } +} + +fn try_convert_operator(op: &DFOperator) -> Option { + match op { + DFOperator::Eq => Some(Operator::Eq), + DFOperator::NotEq => Some(Operator::NotEq), + DFOperator::Lt => Some(Operator::Lt), + DFOperator::LtEq => Some(Operator::Lte), + DFOperator::Gt => Some(Operator::Gt), + DFOperator::GtEq => Some(Operator::Gte), + DFOperator::And => Some(Operator::And), + DFOperator::Or => Some(Operator::Or), + DFOperator::Plus => Some(Operator::Add), + DFOperator::Minus => Some(Operator::Sub), + DFOperator::Multiply => Some(Operator::Mul), + DFOperator::Divide => Some(Operator::Div), + _ => None, + } +} + +/// A DataFusion [`TableProvider`] backed by a Vortex [`DataSource`]. +/// +/// Maps each Vortex scan split to one DataFusion partition, letting DataFusion's scheduler +/// control concurrency. +pub struct VortexTable { + data_source: DataSourceRef, + session: VortexSession, + arrow_schema: SchemaRef, +} + +impl fmt::Debug for VortexTable { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("VortexTable") + .field("schema", &self.arrow_schema) + .finish() + } +} + +impl VortexTable { + /// Creates a new [`VortexTable`] from a Vortex data source and session. + /// + /// The Arrow schema will be used to emit the correct column names and types to DataFusion. + /// The Vortex DType of the data source should be compatible with this Arrow schema. + pub fn new( + data_source: DataSourceRef, + session: VortexSession, + arrow_schema: SchemaRef, + ) -> Self { + Self { + data_source, + session, + arrow_schema, + } + } +} + +#[async_trait] +impl TableProvider for VortexTable { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.arrow_schema) + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + fn supports_filters_pushdown( + &self, + filters: &[&Expr], + ) -> DFResult> { + Ok(filters + .iter() + .map(|expr| { + if try_convert_expr(expr).is_some() { + TableProviderFilterPushDown::Exact + } else { + TableProviderFilterPushDown::Inexact + } + }) + .collect()) + } + + async fn scan( + &self, + _state: &dyn Session, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + ) -> DFResult> { + // Build the projection expression and projected arrow schema. + let (vx_projection, projected_schema) = if let Some(indices) = projection { + let projected_fields: Vec<_> = indices + .iter() + .map(|&i| self.arrow_schema.field(i).clone()) + .collect(); + let projected_schema = Arc::new(arrow_schema::Schema::new(projected_fields)); + + let elements: Vec<(String, Expression)> = indices + .iter() + .map(|&i| { + let name = self.arrow_schema.field(i).name().clone(); + (name.clone(), get_item(name, root())) + }) + .collect(); + let expr = pack(elements, Nullability::NonNullable); + (Some(expr), projected_schema) + } else { + (None, Arc::clone(&self.arrow_schema)) + }; + + // Convert logical filter expressions to Vortex expressions. + let vx_filter = if !filters.is_empty() { + let vx_exprs: Vec = filters.iter().filter_map(try_convert_expr).collect(); + and_collect(vx_exprs) + } else { + None + }; + + // Build the scan request. + let scan_request = ScanRequest { + projection: vx_projection, + filter: vx_filter, + limit: limit.map(|l| u64::try_from(l).unwrap_or(u64::MAX)), + ..Default::default() + }; + + // Create the scan and collect splits. + let scan = self.data_source.scan(scan_request).map_err(vx_err)?; + let splits: Vec<_> = scan.splits().try_collect().await.map_err(vx_err)?; + + Ok(Arc::new(VortexExec::new( + splits, + projected_schema, + self.session.clone(), + ))) + } +} From b19d03bf31a3cf0065e2a99ac4499880e1edbdef Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 14:51:19 -0500 Subject: [PATCH 14/57] DataFusion streaming Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + vortex-file/Cargo.toml | 2 ++ vortex-file/src/lib.rs | 1 + 3 files changed, 4 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index 977cabb8114..f4e058b0160 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10664,6 +10664,7 @@ dependencies = [ "vortex-fsst", "vortex-io", "vortex-layout", + "vortex-mask", "vortex-metrics", "vortex-pco", "vortex-runend", diff --git a/vortex-file/Cargo.toml b/vortex-file/Cargo.toml index 9d32f9412e0..7e35ecfe5a9 100644 --- a/vortex-file/Cargo.toml +++ b/vortex-file/Cargo.toml @@ -46,6 +46,7 @@ vortex-flatbuffers = { workspace = true, features = ["file"] } vortex-fsst = { workspace = true } vortex-io = { workspace = true } vortex-layout = { workspace = true } +vortex-mask = { workspace = true } vortex-metrics = { workspace = true } vortex-pco = { workspace = true } vortex-runend = { workspace = true } @@ -62,6 +63,7 @@ vortex-zstd = { workspace = true, optional = true } tokio = { workspace = true, features = ["full"] } vortex-array = { workspace = true, features = ["_test-harness"] } vortex-io = { workspace = true, features = ["tokio"] } +vortex-layout = { workspace = true, features = ["_test-harness"] } vortex-scan = { workspace = true } [target.'cfg(unix)'.dev-dependencies] diff --git a/vortex-file/src/lib.rs b/vortex-file/src/lib.rs index 6ef1be9b58c..ca8ef9b16df 100644 --- a/vortex-file/src/lib.rs +++ b/vortex-file/src/lib.rs @@ -100,6 +100,7 @@ pub mod segments; mod strategy; #[cfg(test)] mod tests; +pub mod v2; mod writer; pub use file::*; From 811acd54b067d737acaff26314790fb7cce90f62 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 17:41:13 -0500 Subject: [PATCH 15/57] DataFusion streaming Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 60 +++-- vortex-datafusion/src/v2/exec.rs | 274 ++++++++++++++++++-- vortex-datafusion/src/v2/table.rs | 31 +++ vortex-file/src/v2/file_stats_reader.rs | 331 ++++++++++++++++++++++++ vortex-file/src/v2/mod.rs | 6 + vortex-scan/src/api.rs | 9 +- vortex-scan/src/layout.rs | 117 +++++++-- vortex-scan/src/multi.rs | 4 + vortex-scan/src/scan_builder.rs | 10 + 9 files changed, 766 insertions(+), 76 deletions(-) create mode 100644 vortex-file/src/v2/file_stats_reader.rs create mode 100644 vortex-file/src/v2/mod.rs diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index d1369ce4aad..2552637c7e5 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -27,7 +27,9 @@ use datafusion_physical_plan::collect; use futures::StreamExt; use parking_lot::Mutex; use tokio::fs::File; +use vortex::error::VortexExpect; use vortex::file::OpenOptionsSessionExt; +use vortex::file::v2::FileStatsLayoutReader; use vortex::scan::api::DataSourceRef; use vortex::scan::layout::LayoutReaderDataSource; use vortex::scan::multi::DataSourceFactory; @@ -288,7 +290,10 @@ struct VortexFileFactory { impl DataSourceFactory for VortexFileFactory { async fn open(&self) -> vortex::error::VortexResult> { let file = SESSION.open_options().open_path(&self.path).await?; - let reader = file.layout_reader()?; + let mut reader = file.layout_reader()?; + if let Some(stats) = file.file_stats().cloned() { + reader = Arc::new(FileStatsLayoutReader::new(reader, stats, SESSION.clone())); + } Ok(Some(Arc::new(LayoutReaderDataSource::new( reader, SESSION.clone(), @@ -302,46 +307,47 @@ async fn register_v2_tables( benchmark: &B, format: Format, ) -> anyhow::Result<()> { + use futures::TryStreamExt; use vortex_datafusion::v2::VortexTable; - let base_path = benchmark - .data_url() - .to_file_path() - .map_err(|_| anyhow::anyhow!("V2 path requires local file:// URL"))? - .join(format.name()); - - let dir_entries: Vec<_> = std::fs::read_dir(&base_path)?.collect::, _>>()?; + let benchmark_base = benchmark.data_url().join(&format!("{}/", format.name()))?; for table in benchmark.table_specs().iter() { let pattern = benchmark.pattern(table.name, format); - - // Find files matching this table's glob pattern. - let mut matching_paths: Vec = dir_entries - .iter() - .filter(|entry| { - let filename = entry.file_name(); - let filename_str = filename.to_str().unwrap_or(""); - match &pattern { - Some(p) => p.matches(filename_str), - None => filename_str == format!("{}.{}", table.name, format.ext()), - } - }) - .map(|entry| entry.path()) - .collect(); - matching_paths.sort(); + let table_url = ListingTableUrl::try_new(benchmark_base.clone(), pattern)?; + + // Use the same ListingTableUrl file discovery as v1. + let state = session.state(); + let store = state.runtime_env().object_store(table_url.object_store())?; + let mut file_metas: Vec<_> = table_url + .list_all_files(&state, store.as_ref(), format.ext()) + .await? + .try_collect() + .await?; + file_metas.sort_by(|a, b| a.location.cmp(&b.location)); anyhow::ensure!( - !matching_paths.is_empty(), + !file_metas.is_empty(), "no files found for table {}", table.name ); + // Convert object store paths to absolute local file paths. + // Object store locations are relative (e.g. "path/to/file.vortex"), so prepend "/". + let matching_paths: Vec = file_metas + .iter() + .map(|meta| PathBuf::from(format!("/{}", meta.location))) + .collect(); + // Open the first file eagerly to get the dtype/schema. let first_file = SESSION.open_options().open_path(&matching_paths[0]).await?; let arrow_schema = Arc::new(first_file.dtype().to_arrow_schema()?); - let first_reader = first_file.layout_reader()?; - let first_source: DataSourceRef = - Arc::new(LayoutReaderDataSource::new(first_reader, SESSION.clone())); + let first_source = VortexFileFactory { + path: matching_paths[0].clone(), + } + .open() + .await? + .vortex_expect("Missing first file"); // Create lazy factories for remaining files. let remaining: Vec> = matching_paths[1..] diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index 583c978045b..2016b552ae3 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -9,13 +9,21 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; +use arrow_schema::DataType; +use arrow_schema::Schema; use arrow_schema::SchemaRef; +use datafusion_common::ColumnStatistics; use datafusion_common::DataFusionError; use datafusion_common::Result as DFResult; use datafusion_common::Statistics; use datafusion_common::stats::Precision as DFPrecision; +use datafusion_common::tree_node::Transformed; +use datafusion_common::tree_node::TreeNode; +use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_execution::SendableRecordBatchStream; use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_plan::DisplayAs; use datafusion_physical_plan::DisplayFormatType; use datafusion_physical_plan::ExecutionPlan; @@ -23,26 +31,40 @@ use datafusion_physical_plan::Partitioning; use datafusion_physical_plan::PlanProperties; use datafusion_physical_plan::execution_plan::Boundedness; use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::expressions as df_expr; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use futures::StreamExt; use tokio::sync::Mutex; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; +use vortex::dtype::Nullability; +use vortex::expr::Expression; +use vortex::expr::get_item; +use vortex::expr::pack; +use vortex::expr::root; use vortex::scan::api::Estimate; use vortex::scan::api::SplitRef; use vortex::session::VortexSession; +use vortex_utils::aliases::hash_set::HashSet; + +use crate::convert::exprs::DefaultExpressionConvertor; +use crate::convert::exprs::ExpressionConvertor; /// A DataFusion [`ExecutionPlan`] that executes Vortex splits as partitions. /// /// Each partition corresponds to one Vortex [`vortex::scan::api::Split`]. The split is consumed /// on first execute; re-executing the same partition returns an error. pub struct VortexExec { - splits: Vec>>, + splits: Arc<[Mutex>]>, partition_stats: Vec, session: VortexSession, schema: SchemaRef, properties: PlanProperties, + /// An optional projection expression applied to each array chunk before Arrow conversion. + /// Populated by [`ExecutionPlan::try_swapping_with_projection`] when DataFusion pushes a + /// projection down into this scan node. + projection: Option, } impl VortexExec { @@ -70,11 +92,16 @@ impl VortexExec { .collect(); Self { - splits: splits.into_iter().map(|s| Mutex::new(Some(s))).collect(), + splits: splits + .into_iter() + .map(|s| Mutex::new(Some(s))) + .collect::>() + .into(), partition_stats, session, schema, properties, + projection: None, } } } @@ -84,13 +111,25 @@ impl fmt::Debug for VortexExec { f.debug_struct("VortexExec") .field("partitions", &self.splits.len()) .field("schema", &self.schema) + .field( + "projection", + &self.projection.as_ref().map(|e| format!("{}", e)), + ) .finish() } } impl DisplayAs for VortexExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - write!(f, "VortexExec: partitions={}", self.splits.len()) + write!( + f, + "VortexExec: partitions={}, projection={}", + self.splits.len(), + self.projection + .as_ref() + .map(|e| format!("{}", e)) + .unwrap_or_else(|| "*".to_string()) + ) } } @@ -156,13 +195,22 @@ impl ExecutionPlan for VortexExec { let schema = self.schema.clone(); let session = self.session.clone(); - let stream = array_stream.map(move |result| { - // TODO(ngates): do I need to spawn this? - let mut ctx = session.create_execution_ctx(); - result - .and_then(|chunk| chunk.execute_record_batch(&schema, &mut ctx)) - .map_err(|e| DataFusionError::External(Box::new(e))) - }); + let projection = self.projection.clone(); + let stream = array_stream + // Filter out empty arrays (e.g. from fully-pruned splits) before execution. + .filter(|result| std::future::ready(!matches!(result, Ok(arr) if arr.is_empty()))) + .map(move |result| { + let mut ctx = session.create_execution_ctx(); + result + .and_then(|chunk| { + let projected = match &projection { + Some(proj) => chunk.apply(proj)?, + None => chunk, + }; + projected.execute_record_batch(&schema, &mut ctx) + }) + .map_err(|e| DataFusionError::External(Box::new(e))) + }); Ok(Box::pin(RecordBatchStreamAdapter::new( Arc::clone(&self.schema), @@ -182,28 +230,214 @@ impl ExecutionPlan for VortexExec { Ok(stats.clone()) } None => { - let mut total_rows: DFPrecision = DFPrecision::Absent; - let mut total_bytes: DFPrecision = DFPrecision::Absent; + let mut num_rows: DFPrecision = DFPrecision::Absent; + let mut total_byte_size: DFPrecision = DFPrecision::Absent; for stats in &self.partition_stats { - total_rows = total_rows.add(&stats.num_rows); - total_bytes = total_bytes.add(&stats.total_byte_size); + num_rows = num_rows.add(&stats.num_rows); + total_byte_size = total_byte_size.add(&stats.total_byte_size); } + let column_statistics = + vec![ColumnStatistics::new_unknown(); self.schema.fields().len()]; Ok(Statistics { - num_rows: total_rows, - total_byte_size: total_bytes, - column_statistics: vec![], + num_rows, + total_byte_size, + column_statistics, }) } } } +} +impl VortexExec { + // FIXME(ngates): enable filter pushdown and implement try_swapping_with_filter fn try_swapping_with_projection( &self, - _projection: &ProjectionExec, + projection: &ProjectionExec, ) -> DFResult>> { - // We can push down _all_ projections! Possibly... - Ok(None) + tracing::info!( + "VortexExec: trying to swap with projection: {:#?} (current: {})", + projection, + self.projection + .as_ref() + .map(|e| format!("{}", e)) + .unwrap_or_else(|| "*".to_string()) + ); + + // Don't compose projections; if we already have one, let DataFusion handle it. + if self.projection.is_some() { + return Ok(None); + } + + let convertor = DefaultExpressionConvertor::default(); + let input_schema = self.schema.as_ref(); + + let mut scan_columns: Vec<(String, Expression)> = Vec::new(); + let mut scan_fields: Vec = Vec::new(); + let mut leftover_exprs: Vec<(Arc, String)> = Vec::new(); + let mut all_pushed = true; + let mut seen: HashSet = HashSet::new(); + + for proj_expr in projection.expr() { + let can_push = convertor.can_be_pushed_down(&proj_expr.expr, input_schema) + && !has_decimal_binary(&proj_expr.expr, input_schema); + + if can_push { + match convertor.convert(proj_expr.expr.as_ref()) { + Ok(vx_expr) => { + if seen.insert(proj_expr.alias.clone()) { + let output_schema = projection.schema(); + let field = output_schema + .field_with_name(&proj_expr.alias) + .map_err(|e| DataFusionError::Internal(e.to_string()))?; + scan_fields.push(field.clone()); + scan_columns.push((proj_expr.alias.clone(), vx_expr)); + } + + let idx = scan_fields + .iter() + .position(|f| f.name() == &proj_expr.alias) + .ok_or_else(|| { + DataFusionError::Internal(format!( + "field {} not found in scan schema", + proj_expr.alias + )) + })?; + leftover_exprs.push(( + Arc::new(df_expr::Column::new(&proj_expr.alias, idx)), + proj_expr.alias.clone(), + )); + } + Err(_) => { + all_pushed = false; + add_pass_through_columns( + &proj_expr.expr, + input_schema, + &mut scan_columns, + &mut scan_fields, + &mut seen, + )?; + let intermediate_schema = Schema::new(scan_fields.clone()); + let remapped = + remap_column_indices(proj_expr.expr.clone(), &intermediate_schema)?; + leftover_exprs.push((remapped, proj_expr.alias.clone())); + } + } + } else { + all_pushed = false; + add_pass_through_columns( + &proj_expr.expr, + input_schema, + &mut scan_columns, + &mut scan_fields, + &mut seen, + )?; + let intermediate_schema = Schema::new(scan_fields.clone()); + let remapped = remap_column_indices(proj_expr.expr.clone(), &intermediate_schema)?; + leftover_exprs.push((remapped, proj_expr.alias.clone())); + } + } + + if scan_columns.is_empty() { + return Ok(None); + } + + let scan_projection = pack(scan_columns, Nullability::NonNullable); + let scan_output_schema = Arc::new(Schema::new(scan_fields)); + + let new_properties = PlanProperties::new( + datafusion_physical_expr::EquivalenceProperties::new(Arc::clone(&scan_output_schema)), + Partitioning::UnknownPartitioning(self.splits.len()), + EmissionType::Incremental, + Boundedness::Bounded, + ); + + let new_exec = VortexExec { + splits: Arc::clone(&self.splits), + partition_stats: self.partition_stats.clone(), + session: self.session.clone(), + schema: scan_output_schema, + properties: new_properties, + projection: Some(scan_projection), + }; + + if all_pushed { + Ok(Some(Arc::new(new_exec))) + } else { + let new_exec = Arc::new(new_exec) as Arc; + let result = ProjectionExec::try_new(leftover_exprs, new_exec)?; + Ok(Some(Arc::new(result))) + } + } +} + +/// Add input columns required by a non-pushable expression to the scan projection. +fn add_pass_through_columns( + expr: &Arc, + input_schema: &Schema, + scan_columns: &mut Vec<(String, Expression)>, + scan_fields: &mut Vec, + seen: &mut HashSet, +) -> DFResult<()> { + for col in collect_columns(expr) { + if seen.insert(col.name().to_string()) { + let field = input_schema + .field_with_name(col.name()) + .map_err(|e| DataFusionError::Internal(e.to_string()))?; + scan_fields.push(field.clone()); + scan_columns.push((col.name().to_string(), get_item(col.name(), root()))); + } } + Ok(()) +} + +/// Remap column indices in a physical expression to match a new schema. +fn remap_column_indices( + expr: Arc, + new_schema: &Schema, +) -> DFResult> { + expr.transform(|node| { + if let Some(col) = node.as_any().downcast_ref::() { + let new_col = df_expr::Column::new_with_schema(col.name(), new_schema)?; + Ok(Transformed::yes(Arc::new(new_col) as Arc)) + } else { + Ok(Transformed::no(node)) + } + }) + .map(|result| result.data) +} + +/// Check if an expression tree contains decimal binary arithmetic that Vortex cannot handle. +/// +/// DataFusion assumes different decimal types can be coerced, but Vortex expects exact type +/// matches for binary operations. We avoid pushing these down. +fn has_decimal_binary(expr: &Arc, schema: &Schema) -> bool { + let mut found = false; + drop(expr.apply(|node| { + if let Some(binary) = node.as_any().downcast_ref::() + && binary.op().is_numerical_operators() + && let (Ok(l), Ok(r)) = ( + binary.left().data_type(schema), + binary.right().data_type(schema), + ) + && is_decimal(&l) + && is_decimal(&r) + { + found = true; + return Ok(TreeNodeRecursion::Stop); + } + Ok(TreeNodeRecursion::Continue) + })); + found +} + +fn is_decimal(dt: &DataType) -> bool { + matches!( + dt, + DataType::Decimal32(_, _) + | DataType::Decimal64(_, _) + | DataType::Decimal128(_, _) + | DataType::Decimal256(_, _) + ) } /// Convert a Vortex [`Estimate`] to a DataFusion [`Precision`](DFPrecision). diff --git a/vortex-datafusion/src/v2/table.rs b/vortex-datafusion/src/v2/table.rs index a52eb1116eb..ca7c2d7c445 100644 --- a/vortex-datafusion/src/v2/table.rs +++ b/vortex-datafusion/src/v2/table.rs @@ -12,7 +12,10 @@ use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion_catalog::Session; use datafusion_catalog::TableProvider; +use datafusion_common::ColumnStatistics; use datafusion_common::Result as DFResult; +use datafusion_common::Statistics; +use datafusion_common::stats::Precision; use datafusion_expr::Expr; use datafusion_expr::Operator as DFOperator; use datafusion_expr::TableProviderFilterPushDown; @@ -23,6 +26,7 @@ use vortex::compute::LikeOptions; use vortex::dtype::DType; use vortex::dtype::Nullability; use vortex::dtype::arrow::FromArrowType; +use vortex::encodings::fastlanes::bitpack_decompress::unpack_single; use vortex::expr::Binary; use vortex::expr::Expression; use vortex::expr::Like; @@ -243,4 +247,31 @@ impl TableProvider for VortexTable { self.session.clone(), ))) } + + fn statistics(&self) -> Option { + let row_count_est = self.data_source.row_count_estimate(); + let num_rows = match row_count_est.upper { + Some(upper) if row_count_est.lower == upper => usize::try_from(upper) + .map(Precision::Exact) + .unwrap_or_default(), + _ => Precision::Absent, + }; + + let byte_size_est = self.data_source.row_count_estimate(); + let total_byte_size = match byte_size_est.upper { + Some(upper) if byte_size_est.lower == upper => usize::try_from(upper) + .map(Precision::Exact) + .unwrap_or_default(), + _ => Precision::Absent, + }; + + let column_statistics = + vec![ColumnStatistics::new_unknown(); self.arrow_schema.fields.len()]; + + Some(Statistics { + num_rows, + total_byte_size, + column_statistics, + }) + } } diff --git a/vortex-file/src/v2/file_stats_reader.rs b/vortex-file/src/v2/file_stats_reader.rs new file mode 100644 index 00000000000..adb2e07616a --- /dev/null +++ b/vortex-file/src/v2/file_stats_reader.rs @@ -0,0 +1,331 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! A [`LayoutReader`] decorator that performs file-level stats pruning. +//! +//! If file-level statistics prove that a filter expression cannot match any rows in the file, +//! [`FileStatsLayoutReader`] short-circuits [`pruning_evaluation`](LayoutReader::pruning_evaluation) +//! by returning an all-false mask — avoiding all downstream I/O. + +use std::collections::BTreeSet; +use std::ops::Range; +use std::sync::Arc; + +use vortex_array::Columnar; +use vortex_array::MaskFuture; +use vortex_array::VortexSessionExecute; +use vortex_array::expr::Expression; +use vortex_array::expr::pruning::checked_pruning_expr; +use vortex_dtype::DType; +use vortex_dtype::Field; +use vortex_dtype::FieldMask; +use vortex_dtype::FieldPath; +use vortex_dtype::FieldPathSet; +use vortex_dtype::StructFields; +use vortex_error::VortexResult; +use vortex_layout::ArrayFuture; +use vortex_layout::LayoutReader; +use vortex_layout::LayoutReaderRef; +use vortex_mask::Mask; +use vortex_session::VortexSession; +use vortex_utils::aliases::dash_map::DashMap; +use vortex_utils::aliases::hash_map::HashMap; + +use crate::FileStatistics; +use crate::pruning::extract_relevant_file_stats_as_struct_row; + +/// A [`LayoutReader`] decorator that prunes entire files based on file-level statistics. +/// +/// This reader wraps an inner `LayoutReader` and intercepts `pruning_evaluation` calls. +/// When file-level stats prove that a filter expression is false for the entire file, +/// it returns an all-false mask immediately — avoiding all downstream I/O. +/// +/// Pruning results are cached per-expression since file-level stats are global +/// (the result is the same regardless of which row range is requested). +pub struct FileStatsLayoutReader { + child: LayoutReaderRef, + file_stats: FileStatistics, + struct_fields: StructFields, + available_stats: FieldPathSet, + session: VortexSession, + prune_cache: DashMap, +} + +impl FileStatsLayoutReader { + /// Creates a new `FileStatsLayoutReader` wrapping the given child reader. + /// + /// The `struct_fields` are derived from the child reader's dtype. If the dtype is not a + /// struct, the available stats will be empty and no pruning will occur. + /// + /// Pre-computes the set of available stat field paths from the struct fields and file stats. + pub fn new(child: LayoutReaderRef, file_stats: FileStatistics, session: VortexSession) -> Self { + let struct_fields = child + .dtype() + .as_struct_fields_opt() + .cloned() + .unwrap_or_default(); + + let available_stats = FieldPathSet::from_iter( + struct_fields + .names() + .iter() + .zip(file_stats.stats_sets().iter()) + .flat_map(|(name, stats)| { + stats.iter().map(|(stat, _)| { + FieldPath::from_iter([ + Field::Name(name.clone()), + Field::Name(stat.name().into()), + ]) + }) + }), + ); + + Self { + child, + file_stats, + struct_fields, + available_stats, + session, + prune_cache: DashMap::with_hasher(Default::default()), + } + } + + /// Evaluates whether the file can be fully pruned for the given expression. + /// + /// Returns `true` if file-level stats prove no rows can match, `false` otherwise. + fn evaluate_file_stats(&self, expr: &Expression) -> VortexResult { + let Some((predicate, required_stats)) = checked_pruning_expr(expr, &self.available_stats) + else { + return Ok(false); + }; + + let required_file_stats = HashMap::from_iter( + required_stats + .map() + .iter() + .map(|(path, stats)| (path.clone(), stats.clone())), + ); + + let Some(file_stats) = extract_relevant_file_stats_as_struct_row( + &required_file_stats, + self.file_stats.stats_sets(), + &self.struct_fields, + )? + else { + return Ok(false); + }; + + let mut ctx = self.session.create_execution_ctx(); + Ok( + match file_stats + .apply(&predicate)? + .execute::(&mut ctx)? + { + Columnar::Constant(s) => s.scalar().as_bool().value() == Some(true), + Columnar::Canonical(_) => false, + }, + ) + } +} + +impl LayoutReader for FileStatsLayoutReader { + fn name(&self) -> &Arc { + self.child.name() + } + + fn dtype(&self) -> &DType { + self.child.dtype() + } + + fn row_count(&self) -> u64 { + self.child.row_count() + } + + fn register_splits( + &self, + field_mask: &[FieldMask], + row_range: &Range, + splits: &mut BTreeSet, + ) -> VortexResult<()> { + self.child.register_splits(field_mask, row_range, splits) + } + + fn pruning_evaluation( + &self, + row_range: &Range, + expr: &Expression, + mask: Mask, + ) -> VortexResult { + // Check cache first with read-only lock. + if let Some(pruned) = self.prune_cache.get(expr) { + if *pruned { + return Ok(MaskFuture::ready(Mask::new_false(mask.len()))); + } + return self.child.pruning_evaluation(row_range, expr, mask); + } + + // Evaluate and cache. + let pruned = self.evaluate_file_stats(expr)?; + self.prune_cache.insert(expr.clone(), pruned); + + if pruned { + Ok(MaskFuture::ready(Mask::new_false(mask.len()))) + } else { + self.child.pruning_evaluation(row_range, expr, mask) + } + } + + fn filter_evaluation( + &self, + row_range: &Range, + expr: &Expression, + mask: MaskFuture, + ) -> VortexResult { + self.child.filter_evaluation(row_range, expr, mask) + } + + fn projection_evaluation( + &self, + row_range: &Range, + expr: &Expression, + mask: MaskFuture, + ) -> VortexResult { + self.child.projection_evaluation(row_range, expr, mask) + } +} + +#[cfg(test)] +mod tests { + #![allow(clippy::unwrap_used)] + + use std::sync::Arc; + use std::sync::LazyLock; + + use vortex_array::ArrayContext; + use vortex_array::IntoArray as _; + use vortex_array::arrays::StructArray; + use vortex_array::expr::get_item; + use vortex_array::expr::gt; + use vortex_array::expr::lit; + use vortex_array::expr::root; + use vortex_array::expr::session::ExprSession; + use vortex_array::expr::stats::Precision; + use vortex_array::expr::stats::Stat; + use vortex_array::session::ArraySession; + use vortex_array::stats::StatsSet; + use vortex_buffer::buffer; + use vortex_dtype::DType; + use vortex_dtype::Nullability; + use vortex_dtype::PType; + use vortex_error::VortexResult; + use vortex_io::runtime::single::block_on; + use vortex_io::session::RuntimeSession; + use vortex_layout::LayoutReader; + use vortex_layout::LayoutStrategy; + use vortex_layout::layouts::flat::writer::FlatLayoutStrategy; + use vortex_layout::layouts::table::TableStrategy; + use vortex_layout::segments::TestSegments; + use vortex_layout::sequence::SequenceId; + use vortex_layout::sequence::SequentialArrayStreamExt; + use vortex_layout::session::LayoutSession; + use vortex_mask::Mask; + use vortex_scalar::ScalarValue; + use vortex_session::VortexSession; + + use super::*; + + static SESSION: LazyLock = LazyLock::new(|| { + VortexSession::empty() + .with::() + .with::() + .with::() + .with::() + }); + + fn test_file_stats(min: i32, max: i32) -> FileStatistics { + let mut stats = StatsSet::default(); + stats.set(Stat::Min, Precision::exact(ScalarValue::from(min))); + stats.set(Stat::Max, Precision::exact(ScalarValue::from(max))); + FileStatistics::new( + Arc::from([stats]), + Arc::from([DType::Primitive(PType::I32, Nullability::NonNullable)]), + ) + } + + #[test] + fn pruning_when_filter_out_of_range() -> VortexResult<()> { + block_on(|handle| async { + let ctx = ArrayContext::empty(); + let segments = Arc::new(TestSegments::default()); + let (ptr, eof) = SequenceId::root().split(); + let struct_array = StructArray::from_fields( + [("col", buffer![1i32, 2, 3, 4, 5].into_array())].as_slice(), + )?; + let strategy = TableStrategy::new( + Arc::new(FlatLayoutStrategy::default()), + Arc::new(FlatLayoutStrategy::default()), + ); + let layout = strategy + .write_stream( + ctx, + segments.clone(), + struct_array.into_array().to_array_stream().sequenced(ptr), + eof, + handle, + ) + .await?; + + let child = layout.new_reader("".into(), segments, &SESSION)?; + + let reader = + FileStatsLayoutReader::new(child, test_file_stats(0, 100), SESSION.clone()); + + // col > 200 should be prunable since max is 100. + let expr = gt(get_item("col", root()), lit(200i32)); + let mask = Mask::new_true(5); + let result = reader.pruning_evaluation(&(0..5), &expr, mask)?.await?; + assert_eq!(result, Mask::new_false(5)); + + Ok(()) + }) + } + + #[test] + fn no_pruning_when_filter_in_range() -> VortexResult<()> { + block_on(|handle| async { + let ctx = ArrayContext::empty(); + let segments = Arc::new(TestSegments::default()); + let (ptr, eof) = SequenceId::root().split(); + let struct_array = StructArray::from_fields( + [("col", buffer![1i32, 2, 3, 4, 5].into_array())].as_slice(), + )?; + let strategy = TableStrategy::new( + Arc::new(FlatLayoutStrategy::default()), + Arc::new(FlatLayoutStrategy::default()), + ); + let layout = strategy + .write_stream( + ctx, + segments.clone(), + struct_array.into_array().to_array_stream().sequenced(ptr), + eof, + handle, + ) + .await?; + + let child = layout.new_reader("".into(), segments, &SESSION)?; + + let reader = + FileStatsLayoutReader::new(child, test_file_stats(0, 100), SESSION.clone()); + + // col > 50 should NOT be prunable since max is 100 (some rows could match). + let expr = gt(get_item("col", root()), lit(50i32)); + let mask = Mask::new_true(5); + let result = reader.pruning_evaluation(&(0..5), &expr, mask)?.await?; + // Should delegate to child, which returns the mask unchanged (struct reader doesn't prune). + assert_eq!(result, Mask::new_true(5)); + + Ok(()) + }) + } +} diff --git a/vortex-file/src/v2/mod.rs b/vortex-file/src/v2/mod.rs new file mode 100644 index 00000000000..988916b65d1 --- /dev/null +++ b/vortex-file/src/v2/mod.rs @@ -0,0 +1,6 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +mod file_stats_reader; + +pub use file_stats_reader::*; diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index c4d4d1de4b3..03bf63c1ab6 100644 --- a/vortex-scan/src/api.rs +++ b/vortex-scan/src/api.rs @@ -71,7 +71,14 @@ pub trait DataSource: 'static + Send + Sync { fn dtype(&self) -> &DType; /// Returns an estimate of the row count of the un-filtered source. - fn row_count_estimate(&self) -> Estimate; + fn row_count_estimate(&self) -> Estimate { + Estimate::default() + } + + /// Returns an estimate of the byte size of the un-filtered source. + fn byte_size_estimate(&self) -> Estimate { + Estimate::default() + } /// Serialize the [`DataSource`] to pass to a remote worker. fn serialize(&self) -> VortexResult>> { diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 8e4a3efb507..63f6f8dfd65 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -9,10 +9,16 @@ use std::task::Context; use std::task::Poll; use futures::Stream; +use futures::stream; use futures::stream::StreamExt; +use vortex_array::Canonical; +use vortex_array::IntoArray; use vortex_array::expr::Expression; +use vortex_array::expr::root; +use vortex_array::stream::ArrayStreamExt; use vortex_array::stream::SendableArrayStream; use vortex_dtype::DType; +use vortex_dtype::Nullability; use vortex_error::VortexResult; use vortex_error::vortex_bail; use vortex_layout::LayoutReaderRef; @@ -86,6 +92,10 @@ impl DataSource for LayoutReaderDataSource { Estimate::exact(self.reader.row_count()) } + fn byte_size_estimate(&self) -> Estimate { + Estimate::default() + } + fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { vortex_bail!("LayoutReader splits are not yet serializable"); } @@ -94,17 +104,37 @@ impl DataSource for LayoutReaderDataSource { let total_rows = self.reader.row_count(); let row_range = scan_request.row_range.unwrap_or(0..total_rows); - let dtype = if let Some(proj) = &scan_request.projection { - proj.return_dtype(self.reader.dtype())? - } else { - self.reader.dtype().clone() - }; + let projection = scan_request.projection.unwrap_or_else(root); + let dtype = projection.return_dtype(self.reader.dtype())?; + + // If the dtype is an empty struct, and there is no filter, we can return a special + // length-only scan. + if let DType::Struct(fields, Nullability::NonNullable) = &dtype + && fields.nfields() == 0 + && scan_request.filter.is_none() + { + let row_count = row_range.end - row_range.start; + let row_count = match scan_request.selection { + Selection::All => row_count, + Selection::IncludeByIndex(idx) => idx.len() as u64, + Selection::ExcludeByIndex(idx) => row_count - idx.len() as u64, + }; + + // Apply the limit. + let row_count = if let Some(limit) = scan_request.limit { + row_count.min(limit) + } else { + row_count + }; + + return Ok(Box::new(Empty { dtype, row_count })); + } Ok(Box::new(LayoutReaderScan { reader: self.reader.clone(), session: self.session.clone(), dtype, - projection: scan_request.projection, + projection, filter: scan_request.filter, limit: scan_request.limit, selection: scan_request.selection, @@ -120,7 +150,7 @@ struct LayoutReaderScan { reader: LayoutReaderRef, session: VortexSession, dtype: DType, - projection: Option, + projection: Expression, filter: Option, limit: Option, selection: Selection, @@ -200,7 +230,7 @@ impl Stream for LayoutReaderScan { struct LayoutReaderSplit { reader: LayoutReaderRef, session: VortexSession, - projection: Option, + projection: Expression, filter: Option, limit: Option, row_range: Range, @@ -213,35 +243,66 @@ impl Split for LayoutReaderSplit { self } + fn row_count_estimate(&self) -> Estimate { + Estimate { + lower: 0, + upper: Some(self.row_range.end - self.row_range.start), + } + } + + fn byte_size_estimate(&self) -> Estimate { + Estimate::default() + } + fn execute(self: Box) -> VortexResult { - let mut builder = ScanBuilder::new(self.session, self.reader) + let builder = ScanBuilder::new(self.session, self.reader) .with_row_range(self.row_range) - .with_selection(self.selection); - - if let Some(proj) = self.projection { - builder = builder.with_projection(proj); - } - if let Some(filter) = self.filter { - builder = builder.with_filter(filter); - } - if let Some(limit) = self.limit { - builder = builder.with_limit(limit); - } - if let Some(metrics) = self.metrics_registry { - builder = builder.with_metrics_registry(metrics); - } + .with_selection(self.selection) + .with_projection(self.projection) + .with_some_filter(self.filter) + .with_some_limit(self.limit) + .with_some_metrics_registry(self.metrics_registry); Ok(Box::pin(builder.into_array_stream()?)) } +} + +/// A scan that produces no data, only empty arrays with the correct row count. +struct Empty { + dtype: DType, + row_count: u64, +} + +impl DataSourceScan for Empty { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn split_count_estimate(&self) -> Estimate { + Estimate::exact(1) + } + + fn splits(self: Box) -> SplitStream { + stream::iter([Ok(self as _)]).boxed() + } +} + +impl Split for Empty { + fn as_any(&self) -> &dyn Any { + self + } fn row_count_estimate(&self) -> Estimate { - Estimate { - lower: 0, - upper: Some(self.row_range.end - self.row_range.start), - } + Estimate::exact(self.row_count) } fn byte_size_estimate(&self) -> Estimate { - Estimate::default() + Estimate::exact(0) + } + + fn execute(self: Box) -> VortexResult { + Ok(ArrayStreamExt::boxed( + Canonical::empty(&self.dtype).into_array().to_array_stream(), + )) } } diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 937dbff77ea..2ffcabfd5ee 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -31,6 +31,10 @@ use crate::api::ScanRequest; use crate::api::SplitRef; use crate::api::SplitStream; +// FIXME(ngates): this is the wrong abstraction. We should have a MultiFileDataSource in +// vortex-file that knows about files, and globs, and uses object store for file listing etc. +// It should also avoid building a layout tree at all when the dtype is an empty struct. + /// An async factory that produces a [`DataSource`]. /// /// Implementations handle engine-specific concerns like file opening, caching, and diff --git a/vortex-scan/src/scan_builder.rs b/vortex-scan/src/scan_builder.rs index 96bbac0db02..f0ebd866dcc 100644 --- a/vortex-scan/src/scan_builder.rs +++ b/vortex-scan/src/scan_builder.rs @@ -175,11 +175,21 @@ impl ScanBuilder { self } + pub fn with_some_metrics_registry(mut self, metrics: Option>) -> Self { + self.metrics_registry = metrics; + self + } + pub fn with_metrics_registry(mut self, metrics: Arc) -> Self { self.metrics_registry = Some(metrics); self } + pub fn with_some_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + pub fn with_limit(mut self, limit: u64) -> Self { self.limit = Some(limit); self From bbbaa62f969aa56863241656076817d51c11883d Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 21:09:55 -0500 Subject: [PATCH 16/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 1 + vortex-datafusion/src/v2/exec.rs | 4 +--- vortex-datafusion/src/v2/table.rs | 1 - vortex-scan/src/multi.rs | 8 ++++---- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 19136b387f5..379b1419f86 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -391,6 +391,7 @@ impl FileOpener for VortexOpener { let stream_schema = stream_schema.clone(); handle.spawn_cpu(move || { result.and_then(|chunk| { + tracing::info!("Chunk: {}", chunk.display_tree()); let mut ctx = session.create_execution_ctx(); chunk.execute_record_batch(&stream_schema, &mut ctx) }) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index 2016b552ae3..dcd227a9348 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -203,6 +203,7 @@ impl ExecutionPlan for VortexExec { let mut ctx = session.create_execution_ctx(); result .and_then(|chunk| { + tracing::info!("Chunk: {}", chunk.display_tree()); let projected = match &projection { Some(proj) => chunk.apply(proj)?, None => chunk, @@ -246,10 +247,7 @@ impl ExecutionPlan for VortexExec { } } } -} -impl VortexExec { - // FIXME(ngates): enable filter pushdown and implement try_swapping_with_filter fn try_swapping_with_projection( &self, projection: &ProjectionExec, diff --git a/vortex-datafusion/src/v2/table.rs b/vortex-datafusion/src/v2/table.rs index ca7c2d7c445..3aa0ed37936 100644 --- a/vortex-datafusion/src/v2/table.rs +++ b/vortex-datafusion/src/v2/table.rs @@ -26,7 +26,6 @@ use vortex::compute::LikeOptions; use vortex::dtype::DType; use vortex::dtype::Nullability; use vortex::dtype::arrow::FromArrowType; -use vortex::encodings::fastlanes::bitpack_decompress::unpack_single; use vortex::expr::Binary; use vortex::expr::Expression; use vortex::expr::Like; diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 2ffcabfd5ee..c2e59938257 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -161,6 +161,10 @@ impl DataSource for MultiDataSource { Estimate { lower, upper } } + fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { + vortex_bail!("MultiDataSource splits are not yet serializable") + } + fn scan(&self, scan_request: ScanRequest) -> VortexResult { let mut ready = VecDeque::new(); let mut deferred = VecDeque::new(); @@ -191,10 +195,6 @@ impl DataSource for MultiDataSource { Ok(Box::new(scan)) } - - fn deserialize_split(&self, _data: &[u8], _session: &VortexSession) -> VortexResult { - vortex_bail!("MultiDataSource splits are not yet serializable") - } } struct MultiDataSourceScan { From 717ebabfa8028fa2c5b8d5f26ba8a350816fc070 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Wed, 11 Feb 2026 22:25:47 -0500 Subject: [PATCH 17/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 3 +- vortex-datafusion/src/v2/exec.rs | 129 ++++++++++++++++----- 2 files changed, 100 insertions(+), 32 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 379b1419f86..156e38216b4 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -391,7 +391,6 @@ impl FileOpener for VortexOpener { let stream_schema = stream_schema.clone(); handle.spawn_cpu(move || { result.and_then(|chunk| { - tracing::info!("Chunk: {}", chunk.display_tree()); let mut ctx = session.create_execution_ctx(); chunk.execute_record_batch(&stream_schema, &mut ctx) }) @@ -428,7 +427,7 @@ impl FileOpener for VortexOpener { .with_metrics_registry(metrics_registry) .with_projection(scan_projection) .with_some_filter(filter) - .with_ordered(has_output_ordering) + // .with_ordered(has_output_ordering) .map(move |chunk| { let mut ctx = session.create_execution_ctx(); chunk.execute_record_batch(&stream_schema, &mut ctx) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index dcd227a9348..f5ba6ce7526 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -1,8 +1,9 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -//! [`VortexExec`] implements DataFusion's [`ExecutionPlan`] trait, mapping each Vortex split -//! to one DataFusion partition. +//! [`VortexExec`] implements DataFusion's [`ExecutionPlan`] trait, mapping Vortex splits +//! to DataFusion partitions. Multiple splits may be grouped into a single partition +//! via [`ExecutionPlan::repartitioned`] to match the target partition count. use std::any::Any; use std::fmt; @@ -16,6 +17,7 @@ use datafusion_common::ColumnStatistics; use datafusion_common::DataFusionError; use datafusion_common::Result as DFResult; use datafusion_common::Statistics; +use datafusion_common::config::ConfigOptions; use datafusion_common::stats::Precision as DFPrecision; use datafusion_common::tree_node::Transformed; use datafusion_common::tree_node::TreeNode; @@ -43,6 +45,7 @@ use vortex::expr::Expression; use vortex::expr::get_item; use vortex::expr::pack; use vortex::expr::root; +use vortex::io::session::RuntimeSessionExt; use vortex::scan::api::Estimate; use vortex::scan::api::SplitRef; use vortex::session::VortexSession; @@ -53,10 +56,12 @@ use crate::convert::exprs::ExpressionConvertor; /// A DataFusion [`ExecutionPlan`] that executes Vortex splits as partitions. /// -/// Each partition corresponds to one Vortex [`vortex::scan::api::Split`]. The split is consumed -/// on first execute; re-executing the same partition returns an error. +/// Each partition holds one or more Vortex [`vortex::scan::api::Split`]s whose streams are +/// chained sequentially on execute. Splits are consumed on first execute; re-executing the +/// same partition returns an error. pub struct VortexExec { - splits: Arc<[Mutex>]>, + /// Each partition holds one or more splits. Splits are consumed on first execute. + partitions: Arc<[Mutex>]>, partition_stats: Vec, session: VortexSession, schema: SchemaRef, @@ -92,9 +97,9 @@ impl VortexExec { .collect(); Self { - splits: splits + partitions: splits .into_iter() - .map(|s| Mutex::new(Some(s))) + .map(|s| Mutex::new(vec![s])) .collect::>() .into(), partition_stats, @@ -109,7 +114,7 @@ impl VortexExec { impl fmt::Debug for VortexExec { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { f.debug_struct("VortexExec") - .field("partitions", &self.splits.len()) + .field("partitions", &self.partitions.len()) .field("schema", &self.schema) .field( "projection", @@ -124,7 +129,7 @@ impl DisplayAs for VortexExec { write!( f, "VortexExec: partitions={}, projection={}", - self.splits.len(), + self.partitions.len(), self.projection .as_ref() .map(|e| format!("{}", e)) @@ -162,36 +167,101 @@ impl ExecutionPlan for VortexExec { Ok(self) } + fn repartitioned( + &self, + target_partitions: usize, + _config: &ConfigOptions, + ) -> DFResult>> { + // Only group splits when we have more partitions than the target. + if self.partitions.len() <= target_partitions { + return Ok(None); + } + + // Distribute old partitions round-robin into target_partitions groups, + // draining splits from each current partition and aggregating stats. + let mut grouped_splits: Vec> = + (0..target_partitions).map(|_| Vec::new()).collect(); + let mut grouped_stats: Vec = (0..target_partitions) + .map(|_| Statistics { + num_rows: DFPrecision::Absent, + total_byte_size: DFPrecision::Absent, + column_statistics: vec![], + }) + .collect(); + + for (i, (partition, stats)) in self + .partitions + .iter() + .zip(self.partition_stats.iter()) + .enumerate() + { + let group = i % target_partitions; + let mut guard = partition.try_lock().map_err(|_| { + DataFusionError::Internal( + "VortexExec: cannot repartition while partitions are being executed" + .to_string(), + ) + })?; + grouped_splits[group].extend(guard.drain(..)); + grouped_stats[group].num_rows = grouped_stats[group].num_rows.add(&stats.num_rows); + grouped_stats[group].total_byte_size = grouped_stats[group] + .total_byte_size + .add(&stats.total_byte_size); + } + + let properties = PlanProperties::new( + datafusion_physical_expr::EquivalenceProperties::new(Arc::clone(&self.schema)), + Partitioning::UnknownPartitioning(target_partitions), + EmissionType::Incremental, + Boundedness::Bounded, + ); + + Ok(Some(Arc::new(VortexExec { + partitions: grouped_splits + .into_iter() + .map(Mutex::new) + .collect::>() + .into(), + partition_stats: grouped_stats, + session: self.session.clone(), + schema: Arc::clone(&self.schema), + properties, + projection: self.projection.clone(), + }))) + } + fn execute( &self, partition: usize, _context: Arc, ) -> DFResult { - let split = { - let split_slot = self.splits.get(partition).ok_or_else(|| { + let splits = { + let partition_slot = self.partitions.get(partition).ok_or_else(|| { DataFusionError::Internal(format!( "VortexExec: partition index {partition} out of range ({})", - self.splits.len() + self.partitions.len() )) })?; - split_slot - .try_lock() - .map_err(|_| { - DataFusionError::Internal(format!( - "VortexExec: partition {partition} is already being executed" - )) - })? - .take() - .ok_or_else(|| { - DataFusionError::Internal(format!( - "VortexExec: partition {partition} has already been executed" - )) - })? + let mut guard = partition_slot.try_lock().map_err(|_| { + DataFusionError::Internal(format!( + "VortexExec: partition {partition} is already being executed" + )) + })?; + if guard.is_empty() { + return Err(DataFusionError::Internal(format!( + "VortexExec: partition {partition} has already been executed" + ))); + } + std::mem::take(&mut *guard) }; - let array_stream = split - .execute() + // Execute each split and chain their array streams sequentially. + let streams: Vec<_> = splits + .into_iter() + .map(|split| split.execute()) + .collect::, _>>() .map_err(|e| DataFusionError::External(Box::new(e)))?; + let array_stream = futures::stream::iter(streams).flatten(); let schema = self.schema.clone(); let session = self.session.clone(); @@ -203,7 +273,6 @@ impl ExecutionPlan for VortexExec { let mut ctx = session.create_execution_ctx(); result .and_then(|chunk| { - tracing::info!("Chunk: {}", chunk.display_tree()); let projected = match &projection { Some(proj) => chunk.apply(proj)?, None => chunk, @@ -344,13 +413,13 @@ impl ExecutionPlan for VortexExec { let new_properties = PlanProperties::new( datafusion_physical_expr::EquivalenceProperties::new(Arc::clone(&scan_output_schema)), - Partitioning::UnknownPartitioning(self.splits.len()), + Partitioning::UnknownPartitioning(self.partitions.len()), EmissionType::Incremental, Boundedness::Bounded, ); let new_exec = VortexExec { - splits: Arc::clone(&self.splits), + partitions: Arc::clone(&self.partitions), partition_stats: self.partition_stats.clone(), session: self.session.clone(), schema: scan_output_schema, From 0a9b5427c16dd11dc63c0e04c07c93fa894b9f85 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 12 Feb 2026 09:28:35 -0500 Subject: [PATCH 18/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/persistent/opener.rs | 4 +-- vortex-datafusion/src/v2/exec.rs | 42 ++++++++++++++-------- vortex-duckdb/src/scan.rs | 9 +++-- vortex-scan/src/api.rs | 8 +++-- vortex-scan/src/layout.rs | 42 ++++++++++++++++++---- vortex-scan/src/scan_builder.rs | 8 +++++ 6 files changed, 83 insertions(+), 30 deletions(-) diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 156e38216b4..3496feaa939 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -384,7 +384,7 @@ impl FileOpener for VortexOpener { .map(|n| n.get()) .unwrap_or(1); scan.splits() - .map(|split| split.and_then(|s| s.execute())) + .and_then(|s| s.execute()) .try_flatten() .map(move |result| { let session = session.clone(); @@ -427,7 +427,7 @@ impl FileOpener for VortexOpener { .with_metrics_registry(metrics_registry) .with_projection(scan_projection) .with_some_filter(filter) - // .with_ordered(has_output_ordering) + .with_ordered(has_output_ordering) .map(move |chunk| { let mut ctx = session.create_execution_ctx(); chunk.execute_record_batch(&stream_schema, &mut ctx) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index f5ba6ce7526..5a1a51349c6 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -37,6 +37,8 @@ use datafusion_physical_plan::expressions as df_expr; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use futures::StreamExt; +use futures::TryStreamExt; +use futures::stream; use tokio::sync::Mutex; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; @@ -255,32 +257,42 @@ impl ExecutionPlan for VortexExec { std::mem::take(&mut *guard) }; - // Execute each split and chain their array streams sequentially. - let streams: Vec<_> = splits - .into_iter() - .map(|split| split.execute()) - .collect::, _>>() - .map_err(|e| DataFusionError::External(Box::new(e)))?; - let array_stream = futures::stream::iter(streams).flatten(); - + // Chain split streams sequentially, prefetching the next split while consuming + // the current one. buffered(2) lets the next split's execute() future run + // concurrently with consumption of the current split's stream. let schema = self.schema.clone(); let session = self.session.clone(); let projection = self.projection.clone(); - let stream = array_stream - // Filter out empty arrays (e.g. from fully-pruned splits) before execution. + let handle = session.handle(); + let num_workers = std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1); + + let stream = stream::iter(splits) + .map(|split| split.execute()) + .buffered(2) + .try_flatten() + // Filter out empty arrays (e.g. from fully-pruned splits) before conversion. .filter(|result| std::future::ready(!matches!(result, Ok(arr) if arr.is_empty()))) + // Spawn Vortex-to-Arrow conversion onto CPU threads so it doesn't block the + // polling thread, matching the persistent path's behavior. .map(move |result| { - let mut ctx = session.create_execution_ctx(); - result - .and_then(|chunk| { + let session = session.clone(); + let schema = schema.clone(); + let projection = projection.clone(); + handle.spawn_cpu(move || { + let mut ctx = session.create_execution_ctx(); + result.and_then(|chunk| { let projected = match &projection { Some(proj) => chunk.apply(proj)?, None => chunk, }; projected.execute_record_batch(&schema, &mut ctx) }) - .map_err(|e| DataFusionError::External(Box::new(e))) - }); + }) + }) + .buffered(num_workers) + .map(|result| result.map_err(|e| DataFusionError::External(Box::new(e)))); Ok(Box::pin(RecordBatchStreamAdapter::new( Arc::clone(&self.schema), diff --git a/vortex-duckdb/src/scan.rs b/vortex-duckdb/src/scan.rs index 45db265ca3d..f5bb6044add 100644 --- a/vortex-duckdb/src/scan.rs +++ b/vortex-duckdb/src/scan.rs @@ -588,14 +588,13 @@ fn init_global_scan_api( let conversion_cache = Arc::new(ConversionCache::new(0)); // Lazily pull splits, execute each into a stream, and feed into MultiScan. - let scan_streams = scan.splits().map(move |split_result| { + let scan_streams = scan.splits().then(move |split_result| { let cache = conversion_cache.clone(); - let stream: VortexResult>> = (|| { + async move { let split = split_result?; - let s = split.execute()?; + let s = split.execute().await?; Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) - })(); - stream + } }); Ok(RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { diff --git a/vortex-scan/src/api.rs b/vortex-scan/src/api.rs index 03bf63c1ab6..ad4af7a38c7 100644 --- a/vortex-scan/src/api.rs +++ b/vortex-scan/src/api.rs @@ -24,6 +24,7 @@ use std::ops::Range; use std::sync::Arc; use async_trait::async_trait; +use futures::future::BoxFuture; use futures::stream::BoxStream; use vortex_array::expr::Expression; use vortex_array::stream::SendableArrayStream; @@ -143,8 +144,11 @@ pub trait Split: 'static + Send { Ok(None) } - /// Executes the split. - fn execute(self: Box) -> VortexResult; + /// Executes the split, returning a future that resolves to an array stream. + /// + /// This is async to allow split preparation (e.g. metadata reads) to happen concurrently + /// with consumption of the previous split's stream. + fn execute(self: Box) -> BoxFuture<'static, VortexResult>; } /// An estimate that can be exact, an upper bound, or unknown. diff --git a/vortex-scan/src/layout.rs b/vortex-scan/src/layout.rs index 63f6f8dfd65..985ce5e0ef4 100644 --- a/vortex-scan/src/layout.rs +++ b/vortex-scan/src/layout.rs @@ -9,18 +9,21 @@ use std::task::Context; use std::task::Poll; use futures::Stream; +use futures::future::BoxFuture; use futures::stream; use futures::stream::StreamExt; use vortex_array::Canonical; use vortex_array::IntoArray; use vortex_array::expr::Expression; use vortex_array::expr::root; +use vortex_array::stream::ArrayStreamAdapter; use vortex_array::stream::ArrayStreamExt; use vortex_array::stream::SendableArrayStream; use vortex_dtype::DType; use vortex_dtype::Nullability; use vortex_error::VortexResult; use vortex_error::vortex_bail; +use vortex_io::session::RuntimeSessionExt; use vortex_layout::LayoutReaderRef; use vortex_metrics::MetricsRegistry; use vortex_session::VortexSession; @@ -254,7 +257,8 @@ impl Split for LayoutReaderSplit { Estimate::default() } - fn execute(self: Box) -> VortexResult { + fn execute(self: Box) -> BoxFuture<'static, VortexResult> { + let handle = self.session.handle(); let builder = ScanBuilder::new(self.session, self.reader) .with_row_range(self.row_range) .with_selection(self.selection) @@ -263,7 +267,31 @@ impl Split for LayoutReaderSplit { .with_some_limit(self.limit) .with_some_metrics_registry(self.metrics_registry); - Ok(Box::pin(builder.into_array_stream()?)) + Box::pin(async move { + // NOTE(ngates): for now we replicate the behavior inside builder.into_stream() so + // that we can spawn the blocking phase as a future and give the caller more control + // over when a scan is planned vs executed. + let num_workers = std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1); + let concurrency = builder.concurrency() * num_workers; + let ordered = builder.ordered(); + let dtype = builder.dtype()?; + + let tasks = handle.spawn_blocking(move || builder.build()).await?; + + let stream = stream::iter(tasks); + let stream = if ordered { + stream.buffered(concurrency).boxed() + } else { + stream.buffer_unordered(concurrency).boxed() + }; + let stream = stream.filter_map(|chunk| async move { chunk.transpose() }); + + Ok(ArrayStreamExt::boxed(ArrayStreamAdapter::new( + dtype, stream, + ))) + }) } } @@ -300,9 +328,11 @@ impl Split for Empty { Estimate::exact(0) } - fn execute(self: Box) -> VortexResult { - Ok(ArrayStreamExt::boxed( - Canonical::empty(&self.dtype).into_array().to_array_stream(), - )) + fn execute(self: Box) -> BoxFuture<'static, VortexResult> { + Box::pin(async move { + Ok(ArrayStreamExt::boxed( + Canonical::empty(&self.dtype).into_array().to_array_stream(), + )) + }) } } diff --git a/vortex-scan/src/scan_builder.rs b/vortex-scan/src/scan_builder.rs index f0ebd866dcc..b955ecf800a 100644 --- a/vortex-scan/src/scan_builder.rs +++ b/vortex-scan/src/scan_builder.rs @@ -137,6 +137,10 @@ impl ScanBuilder { self } + pub fn ordered(&self) -> bool { + self.ordered + } + pub fn with_ordered(mut self, ordered: bool) -> Self { self.ordered = ordered; self @@ -167,6 +171,10 @@ impl ScanBuilder { self } + pub fn concurrency(&self) -> usize { + self.concurrency + } + /// The number of row splits to make progress on concurrently per-thread, must /// be greater than 0. pub fn with_concurrency(mut self, concurrency: usize) -> Self { From 25cac354a5786e4cd70a52633a979b792e6fae09 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 12 Feb 2026 11:23:59 -0500 Subject: [PATCH 19/57] DataFusion streaming Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 4 +- vortex-datafusion/src/persistent/format.rs | 1 - vortex-datafusion/src/v2/exec.rs | 62 +++++++++++++++------- vortex-scan/src/multi.rs | 11 ++-- 4 files changed, 53 insertions(+), 25 deletions(-) diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 2552637c7e5..2cd4178145c 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -340,8 +340,6 @@ async fn register_v2_tables( .collect(); // Open the first file eagerly to get the dtype/schema. - let first_file = SESSION.open_options().open_path(&matching_paths[0]).await?; - let arrow_schema = Arc::new(first_file.dtype().to_arrow_schema()?); let first_source = VortexFileFactory { path: matching_paths[0].clone(), } @@ -349,6 +347,8 @@ async fn register_v2_tables( .await? .vortex_expect("Missing first file"); + let arrow_schema = Arc::new(first_source.dtype().to_arrow_schema()?); + // Create lazy factories for remaining files. let remaining: Vec> = matching_paths[1..] .iter() diff --git a/vortex-datafusion/src/persistent/format.rs b/vortex-datafusion/src/persistent/format.rs index 7a64d0048d4..41bef233622 100644 --- a/vortex-datafusion/src/persistent/format.rs +++ b/vortex-datafusion/src/persistent/format.rs @@ -452,7 +452,6 @@ impl FileFormat for VortexFormat { } let total_byte_size = sum_of_column_byte_sizes.to_df(); - Ok(Statistics { num_rows: Precision::Exact( usize::try_from(row_count) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index 5a1a51349c6..ca8b7fae063 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -39,10 +39,13 @@ use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use futures::StreamExt; use futures::TryStreamExt; use futures::stream; +use tokio::io::AsyncWriteExt; use tokio::sync::Mutex; +use tracing::Instrument; use vortex::array::VortexSessionExecute; use vortex::array::arrow::ArrowArrayExecutor; use vortex::dtype::Nullability; +use vortex::error::VortexResult; use vortex::expr::Expression; use vortex::expr::get_item; use vortex::expr::pack; @@ -235,7 +238,7 @@ impl ExecutionPlan for VortexExec { fn execute( &self, partition: usize, - _context: Arc, + context: Arc, ) -> DFResult { let splits = { let partition_slot = self.partitions.get(partition).ok_or_else(|| { @@ -257,19 +260,22 @@ impl ExecutionPlan for VortexExec { std::mem::take(&mut *guard) }; - // Chain split streams sequentially, prefetching the next split while consuming - // the current one. buffered(2) lets the next split's execute() future run - // concurrently with consumption of the current split's stream. let schema = self.schema.clone(); let session = self.session.clone(); let projection = self.projection.clone(); - let handle = session.handle(); - let num_workers = std::thread::available_parallelism() - .map(|n| n.get()) - .unwrap_or(1); + + let batch_size = context.session_config().options().execution.batch_size; let stream = stream::iter(splits) - .map(|split| split.execute()) + .map(|split| { + split + .execute() + // TODO(ngates): rename execute -> prepare? Since it's the future that returns the stream? + .instrument(tracing::info_span!("VortexExec: preparing split")) + }) + // We prepare the next split's stream while consuming the current one. buffered(2) lets + // the next split's execute() future run concurrently with consumption of the current + // split's stream. .buffered(2) .try_flatten() // Filter out empty arrays (e.g. from fully-pruned splits) before conversion. @@ -280,18 +286,36 @@ impl ExecutionPlan for VortexExec { let session = session.clone(); let schema = schema.clone(); let projection = projection.clone(); - handle.spawn_cpu(move || { - let mut ctx = session.create_execution_ctx(); - result.and_then(|chunk| { - let projected = match &projection { - Some(proj) => chunk.apply(proj)?, - None => chunk, - }; - projected.execute_record_batch(&schema, &mut ctx) - }) + let mut ctx = session.create_execution_ctx(); + result.and_then(|chunk| { + let projected = match &projection { + Some(proj) => chunk.apply(proj)?, + None => chunk, + }; + projected.execute_record_batch(&schema, &mut ctx) }) }) - .buffered(num_workers) + .map_ok(move |rb| { + // Slice the stream to respect DataFusion's configured batch size. + stream::iter( + (0..rb.num_rows().div_ceil(batch_size * 2)) + .flat_map(move |block_idx| { + let offset = block_idx * batch_size * 2; + + if rb.num_rows() - offset < 2 * batch_size { + let length = rb.num_rows() - offset; + [Some(rb.slice(offset, length)), None].into_iter() + } else { + let first = rb.slice(offset, batch_size); + let second = rb.slice(offset + batch_size, batch_size); + [Some(first), Some(second)].into_iter() + } + }) + .flatten() + .map(VortexResult::Ok), + ) + }) + .try_flatten() .map(|result| result.map_err(|e| DataFusionError::External(Box::new(e)))); Ok(Box::pin(RecordBatchStreamAdapter::new( diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index c2e59938257..035b08a2be0 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -13,6 +13,7 @@ use std::sync::Arc; use async_trait::async_trait; use futures::StreamExt; use futures::stream; +use tracing::Instrument; use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_error::vortex_bail; @@ -47,7 +48,7 @@ pub trait DataSourceFactory: 'static + Send + Sync { } /// Default number of deferred sources to open concurrently during scanning. -const DEFAULT_PREFETCH: usize = 8; +const DEFAULT_PREFETCH: usize = usize::MAX; /// A [`DataSource`] combining multiple children into a single scannable source. /// @@ -225,8 +226,12 @@ impl MultiDataSourceScan { let Some(factory) = self.deferred.pop_front() else { break; }; - self.opening - .push_back(self.handle.spawn(async move { factory.open().await })); + self.opening.push_back(self.handle.spawn(async move { + factory + .open() + .instrument(tracing::info_span!("DataSourceFactory::open")) + .await + })); } } From 367baaac83cb34660bce322002674cb4dcd95481 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 12 Feb 2026 11:57:26 -0500 Subject: [PATCH 20/57] DataFusion streaming Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 23 +++++++++- vortex-scan/src/multi.rs | 59 ++++++++++++++++++++----- 2 files changed, 68 insertions(+), 14 deletions(-) diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 2cd4178145c..60777b2217b 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -25,9 +25,11 @@ use datafusion_bench::tracer::set_labels; use datafusion_physical_plan::ExecutionPlan; use datafusion_physical_plan::collect; use futures::StreamExt; +use object_store::ObjectStore; use parking_lot::Mutex; use tokio::fs::File; use vortex::error::VortexExpect; +use vortex::error::vortex_err; use vortex::file::OpenOptionsSessionExt; use vortex::file::v2::FileStatsLayoutReader; use vortex::scan::api::DataSourceRef; @@ -283,13 +285,24 @@ async fn register_benchmark_tables( /// A [`DataSourceFactory`] that lazily opens a single Vortex file. struct VortexFileFactory { + object_store: Arc, path: PathBuf, } #[async_trait] impl DataSourceFactory for VortexFileFactory { async fn open(&self) -> vortex::error::VortexResult> { - let file = SESSION.open_options().open_path(&self.path).await?; + let file = SESSION + .open_options() + .open_object_store( + &self.object_store, + self.path + .as_os_str() + .to_str() + .ok_or_else(|| vortex_err!("Invalid path"))?, + ) + .await?; + let mut reader = file.layout_reader()?; if let Some(stats) = file.file_stats().cloned() { reader = Arc::new(FileStatsLayoutReader::new(reader, stats, SESSION.clone())); @@ -341,6 +354,7 @@ async fn register_v2_tables( // Open the first file eagerly to get the dtype/schema. let first_source = VortexFileFactory { + object_store: store.clone(), path: matching_paths[0].clone(), } .open() @@ -352,7 +366,12 @@ async fn register_v2_tables( // Create lazy factories for remaining files. let remaining: Vec> = matching_paths[1..] .iter() - .map(|path| Arc::new(VortexFileFactory { path: path.clone() }) as _) + .map(|path| { + Arc::new(VortexFileFactory { + object_store: store.clone(), + path: path.clone(), + }) as _ + }) .collect(); let data_source: DataSourceRef = if remaining.is_empty() { diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 035b08a2be0..e6d40aca4d2 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -13,6 +13,7 @@ use std::sync::Arc; use async_trait::async_trait; use futures::StreamExt; use futures::stream; +use parking_lot::Mutex; use tracing::Instrument; use vortex_dtype::DType; use vortex_error::VortexResult; @@ -48,7 +49,7 @@ pub trait DataSourceFactory: 'static + Send + Sync { } /// Default number of deferred sources to open concurrently during scanning. -const DEFAULT_PREFETCH: usize = usize::MAX; +const DEFAULT_PREFETCH: usize = 8; /// A [`DataSource`] combining multiple children into a single scannable source. /// @@ -64,7 +65,10 @@ pub struct MultiDataSource { enum MultiChild { Opened(DataSourceRef), - Deferred(Arc), + Deferred { + factory: Arc, + cache: Arc>>, + }, } impl MultiDataSource { @@ -92,7 +96,9 @@ impl MultiDataSource { dtype, children: children.into_iter().map(MultiChild::Opened).collect(), handle: session.handle(), - prefetch: DEFAULT_PREFETCH, + prefetch: std::thread::available_parallelism() + .map(|v| v.get()) + .unwrap_or(DEFAULT_PREFETCH), }) } @@ -109,7 +115,10 @@ impl MultiDataSource { let dtype = first.dtype().clone(); let mut children = Vec::with_capacity(1 + remaining.len()); children.push(MultiChild::Opened(first)); - children.extend(remaining.into_iter().map(MultiChild::Deferred)); + children.extend(remaining.into_iter().map(|factory| MultiChild::Deferred { + factory, + cache: Arc::new(Mutex::new(None)), + })); Self { dtype, @@ -149,8 +158,18 @@ impl DataSource for MultiDataSource { _ => None, }; } - MultiChild::Deferred(_) => { - has_deferred = true; + MultiChild::Deferred { cache, .. } => { + let guard = cache.lock(); + if let Some(ds) = guard.as_ref() { + let est = ds.row_count_estimate(); + lower = lower.saturating_add(est.lower); + upper = match (upper, est.upper) { + (Some(a), Some(b)) => Some(a.saturating_add(b)), + _ => None, + }; + } else { + has_deferred = true; + } } } } @@ -173,7 +192,15 @@ impl DataSource for MultiDataSource { for child in &self.children { match child { MultiChild::Opened(ds) => ready.push_back(ds.clone()), - MultiChild::Deferred(factory) => deferred.push_back(factory.clone()), + MultiChild::Deferred { factory, cache } => { + let guard = cache.lock(); + if let Some(ds) = guard.as_ref() { + ready.push_back(ds.clone()); + } else { + drop(guard); + deferred.push_back((factory.clone(), cache.clone())); + } + } } } @@ -208,8 +235,11 @@ struct MultiDataSourceScan { /// In-flight spawned opens. Tasks run on the session's runtime independently, /// so they make progress even when we're not polling them. opening: VecDeque>>>, - /// Remaining factories not yet spawned. - deferred: VecDeque>, + /// Remaining factories not yet spawned, paired with their cache slots. + deferred: VecDeque<( + Arc, + Arc>>, + )>, /// Runtime handle for spawning prefetch tasks. handle: Handle, /// Target number of in-flight + ready sources. @@ -223,14 +253,19 @@ impl MultiDataSourceScan { /// Spawns open tasks for deferred factories up to the prefetch target. fn fill_pipeline(&mut self) { while self.opening.len() + self.ready.len() < self.prefetch { - let Some(factory) = self.deferred.pop_front() else { + let Some((factory, cache)) = self.deferred.pop_front() else { break; }; self.opening.push_back(self.handle.spawn(async move { - factory + let result = factory .open() .instrument(tracing::info_span!("DataSourceFactory::open")) - .await + .await?; + // Cache the opened source for reuse across scans. + if let Some(ref source) = result { + *cache.lock() = Some(source.clone()); + } + Ok(result) })); } } From 3b7d0ae6433cde254281ab0668e8f2d29d94e129 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 12 Feb 2026 18:20:27 -0500 Subject: [PATCH 21/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/v2/exec.rs | 2 - vortex-scan/src/multi.rs | 93 ++++++++++++++------------------ 2 files changed, 39 insertions(+), 56 deletions(-) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index ca8b7fae063..f14e61384b8 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -39,7 +39,6 @@ use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use futures::StreamExt; use futures::TryStreamExt; use futures::stream; -use tokio::io::AsyncWriteExt; use tokio::sync::Mutex; use tracing::Instrument; use vortex::array::VortexSessionExecute; @@ -50,7 +49,6 @@ use vortex::expr::Expression; use vortex::expr::get_item; use vortex::expr::pack; use vortex::expr::root; -use vortex::io::session::RuntimeSessionExt; use vortex::scan::api::Estimate; use vortex::scan::api::SplitRef; use vortex::session::VortexSession; diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index e6d40aca4d2..35ae327f7ae 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -6,6 +6,12 @@ //! Splits from all children are interleaved, enabling parallel execution across files. //! Children can be pre-opened (eager) or opened lazily via [`DataSourceFactory`] implementations, //! with spawned prefetching to overlap file-opening I/O with split execution. +//! +//! # Future Work +//! +//! This data source should evolve to support hive-style partitioning columns, different strategies +//! for unifying schemas, more flexible prefetching configurations, and more robust error handling +//! (e.g., skip failed sources instead of aborting the entire scan). use std::collections::VecDeque; use std::sync::Arc; @@ -33,10 +39,6 @@ use crate::api::ScanRequest; use crate::api::SplitRef; use crate::api::SplitStream; -// FIXME(ngates): this is the wrong abstraction. We should have a MultiFileDataSource in -// vortex-file that knows about files, and globs, and uses object store for file listing etc. -// It should also avoid building a layout tree at all when the dtype is an empty struct. - /// An async factory that produces a [`DataSource`]. /// /// Implementations handle engine-specific concerns like file opening, caching, and @@ -56,19 +58,19 @@ const DEFAULT_PREFETCH: usize = 8; /// Children may be pre-opened or deferred via [`DataSourceFactory`]. During scanning, /// deferred children are opened in the background using spawned tasks on the session's runtime, /// keeping the I/O pipeline full while the engine processes splits from already-open sources. +/// +/// Once a deferred child is successfully opened, it is promoted to [`MultiChild::Opened`] so +/// that subsequent scans reuse the opened source without re-opening. pub struct MultiDataSource { dtype: DType, - children: Vec, + children: Arc>>, handle: Handle, prefetch: usize, } enum MultiChild { Opened(DataSourceRef), - Deferred { - factory: Arc, - cache: Arc>>, - }, + Deferred(Arc), } impl MultiDataSource { @@ -94,7 +96,9 @@ impl MultiDataSource { Ok(Self { dtype, - children: children.into_iter().map(MultiChild::Opened).collect(), + children: Arc::new(Mutex::new( + children.into_iter().map(MultiChild::Opened).collect(), + )), handle: session.handle(), prefetch: std::thread::available_parallelism() .map(|v| v.get()) @@ -115,14 +119,11 @@ impl MultiDataSource { let dtype = first.dtype().clone(); let mut children = Vec::with_capacity(1 + remaining.len()); children.push(MultiChild::Opened(first)); - children.extend(remaining.into_iter().map(|factory| MultiChild::Deferred { - factory, - cache: Arc::new(Mutex::new(None)), - })); + children.extend(remaining.into_iter().map(MultiChild::Deferred)); Self { dtype, - children, + children: Arc::new(Mutex::new(children)), handle: session.handle(), prefetch: DEFAULT_PREFETCH, } @@ -148,7 +149,8 @@ impl DataSource for MultiDataSource { let mut upper: Option = Some(0); let mut has_deferred = false; - for child in &self.children { + let children = self.children.lock(); + for child in children.iter() { match child { MultiChild::Opened(ds) => { let est = ds.row_count_estimate(); @@ -158,18 +160,8 @@ impl DataSource for MultiDataSource { _ => None, }; } - MultiChild::Deferred { cache, .. } => { - let guard = cache.lock(); - if let Some(ds) = guard.as_ref() { - let est = ds.row_count_estimate(); - lower = lower.saturating_add(est.lower); - upper = match (upper, est.upper) { - (Some(a), Some(b)) => Some(a.saturating_add(b)), - _ => None, - }; - } else { - has_deferred = true; - } + MultiChild::Deferred(_) => { + has_deferred = true; } } } @@ -189,20 +181,14 @@ impl DataSource for MultiDataSource { let mut ready = VecDeque::new(); let mut deferred = VecDeque::new(); - for child in &self.children { + let children = self.children.lock(); + for (i, child) in children.iter().enumerate() { match child { MultiChild::Opened(ds) => ready.push_back(ds.clone()), - MultiChild::Deferred { factory, cache } => { - let guard = cache.lock(); - if let Some(ds) = guard.as_ref() { - ready.push_back(ds.clone()); - } else { - drop(guard); - deferred.push_back((factory.clone(), cache.clone())); - } - } + MultiChild::Deferred(factory) => deferred.push_back((i, factory.clone())), } } + drop(children); let remaining_limit = scan_request.limit; @@ -213,6 +199,7 @@ impl DataSource for MultiDataSource { ready, opening: VecDeque::new(), deferred, + children: Arc::clone(&self.children), handle: self.handle.clone(), prefetch: self.prefetch, remaining_limit, @@ -232,14 +219,12 @@ struct MultiDataSourceScan { current: Option, /// Pre-opened sources ready to be scanned. ready: VecDeque, - /// In-flight spawned opens. Tasks run on the session's runtime independently, - /// so they make progress even when we're not polling them. - opening: VecDeque>>>, - /// Remaining factories not yet spawned, paired with their cache slots. - deferred: VecDeque<( - Arc, - Arc>>, - )>, + /// In-flight spawned opens. Each task yields `(child_index, source)` on success. + opening: VecDeque>>>, + /// Remaining factories not yet spawned, paired with their child index. + deferred: VecDeque<(usize, Arc)>, + /// Shared children vec for promoting Deferred → Opened. + children: Arc>>, /// Runtime handle for spawning prefetch tasks. handle: Handle, /// Target number of in-flight + ready sources. @@ -253,19 +238,15 @@ impl MultiDataSourceScan { /// Spawns open tasks for deferred factories up to the prefetch target. fn fill_pipeline(&mut self) { while self.opening.len() + self.ready.len() < self.prefetch { - let Some((factory, cache)) = self.deferred.pop_front() else { + let Some((idx, factory)) = self.deferred.pop_front() else { break; }; self.opening.push_back(self.handle.spawn(async move { - let result = factory + let source = factory .open() .instrument(tracing::info_span!("DataSourceFactory::open")) .await?; - // Cache the opened source for reuse across scans. - if let Some(ref source) = result { - *cache.lock() = Some(source.clone()); - } - Ok(result) + Ok(source.map(|s| (idx, s))) })); } } @@ -280,7 +261,11 @@ impl MultiDataSourceScan { if let Some(task) = self.opening.pop_front() { self.fill_pipeline(); match task.await? { - Some(source) => return Ok(Some(source)), + Some((idx, source)) => { + // Promote Deferred → Opened so future scans reuse this source. + self.children.lock()[idx] = MultiChild::Opened(source.clone()); + return Ok(Some(source)); + } None => continue, // pruned, try next } } From ac06b32fb94ac33242cea7add432730ceabe06c0 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Thu, 12 Feb 2026 18:26:51 -0500 Subject: [PATCH 22/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/public-api.lock | 29 ++++++++++++ vortex-datafusion/src/v2/table.rs | 2 +- vortex-file/public-api.lock | 12 +++++ vortex-scan/public-api.lock | 78 +++++++++++++++++++++++-------- vortex-scan/src/multi.rs | 4 +- 5 files changed, 102 insertions(+), 23 deletions(-) diff --git a/vortex-datafusion/public-api.lock b/vortex-datafusion/public-api.lock index a107e615841..a632bb8b281 100644 --- a/vortex-datafusion/public-api.lock +++ b/vortex-datafusion/public-api.lock @@ -8,6 +8,34 @@ pub fn vortex_datafusion::metrics::VortexMetricsFinder::default() -> vortex_data impl datafusion_physical_plan::visitor::ExecutionPlanVisitor for vortex_datafusion::metrics::VortexMetricsFinder pub type vortex_datafusion::metrics::VortexMetricsFinder::Error = core::convert::Infallible pub fn vortex_datafusion::metrics::VortexMetricsFinder::pre_visit(&mut self, plan: &dyn datafusion_physical_plan::execution_plan::ExecutionPlan) -> core::result::Result +pub mod vortex_datafusion::v2 +pub struct vortex_datafusion::v2::VortexExec +impl core::fmt::Debug for vortex_datafusion::v2::VortexExec +pub fn vortex_datafusion::v2::VortexExec::fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result +impl datafusion_physical_plan::display::DisplayAs for vortex_datafusion::v2::VortexExec +pub fn vortex_datafusion::v2::VortexExec::fmt_as(&self, _t: datafusion_physical_plan::display::DisplayFormatType, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result +impl datafusion_physical_plan::execution_plan::ExecutionPlan for vortex_datafusion::v2::VortexExec +pub fn vortex_datafusion::v2::VortexExec::as_any(&self) -> &dyn core::any::Any +pub fn vortex_datafusion::v2::VortexExec::children(&self) -> alloc::vec::Vec<&alloc::sync::Arc> +pub fn vortex_datafusion::v2::VortexExec::execute(&self, partition: usize, context: alloc::sync::Arc) -> datafusion_common::error::Result +pub fn vortex_datafusion::v2::VortexExec::name(&self) -> &str +pub fn vortex_datafusion::v2::VortexExec::partition_statistics(&self, partition: core::option::Option) -> datafusion_common::error::Result +pub fn vortex_datafusion::v2::VortexExec::properties(&self) -> &datafusion_physical_plan::execution_plan::PlanProperties +pub fn vortex_datafusion::v2::VortexExec::repartitioned(&self, target_partitions: usize, _config: &datafusion_common::config::ConfigOptions) -> datafusion_common::error::Result>> +pub fn vortex_datafusion::v2::VortexExec::try_swapping_with_projection(&self, projection: &datafusion_physical_plan::projection::ProjectionExec) -> datafusion_common::error::Result>> +pub fn vortex_datafusion::v2::VortexExec::with_new_children(self: alloc::sync::Arc, children: alloc::vec::Vec>) -> datafusion_common::error::Result> +pub struct vortex_datafusion::v2::VortexTable +impl vortex_datafusion::v2::VortexTable +pub fn vortex_datafusion::v2::VortexTable::new(data_source: vortex_scan::api::DataSourceRef, session: vortex_session::VortexSession, arrow_schema: arrow_schema::schema::SchemaRef) -> Self +impl core::fmt::Debug for vortex_datafusion::v2::VortexTable +pub fn vortex_datafusion::v2::VortexTable::fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result +impl datafusion_catalog::table::TableProvider for vortex_datafusion::v2::VortexTable +pub fn vortex_datafusion::v2::VortexTable::as_any(&self) -> &dyn core::any::Any +pub fn vortex_datafusion::v2::VortexTable::scan<'life0, 'life1, 'life2, 'life3, 'async_trait>(&'life0 self, _state: &'life1 dyn datafusion_session::session::Session, projection: core::option::Option<&'life2 alloc::vec::Vec>, filters: &'life3 [datafusion_expr::expr::Expr], limit: core::option::Option) -> core::pin::Pin>> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait, 'life2: 'async_trait, 'life3: 'async_trait +pub fn vortex_datafusion::v2::VortexTable::schema(&self) -> arrow_schema::schema::SchemaRef +pub fn vortex_datafusion::v2::VortexTable::statistics(&self) -> core::option::Option +pub fn vortex_datafusion::v2::VortexTable::supports_filters_pushdown(&self, filters: &[&datafusion_expr::expr::Expr]) -> datafusion_common::error::Result> +pub fn vortex_datafusion::v2::VortexTable::table_type(&self) -> datafusion_expr::table_source::TableType pub struct vortex_datafusion::DefaultVortexReaderFactory impl vortex_datafusion::DefaultVortexReaderFactory pub fn vortex_datafusion::DefaultVortexReaderFactory::new(object_store: alloc::sync::Arc) -> Self @@ -18,6 +46,7 @@ pub fn vortex_datafusion::DefaultVortexReaderFactory::create_reader(&self, path: pub struct vortex_datafusion::VortexAccessPlan impl vortex_datafusion::VortexAccessPlan pub fn vortex_datafusion::VortexAccessPlan::apply_to_builder(&self, scan_builder: vortex_scan::scan_builder::ScanBuilder) -> vortex_scan::scan_builder::ScanBuilder where A: 'static + core::marker::Send +pub fn vortex_datafusion::VortexAccessPlan::selection(&self) -> core::option::Option<&vortex_scan::selection::Selection> impl vortex_datafusion::VortexAccessPlan pub fn vortex_datafusion::VortexAccessPlan::with_selection(self, selection: vortex_scan::selection::Selection) -> Self impl core::default::Default for vortex_datafusion::VortexAccessPlan diff --git a/vortex-datafusion/src/v2/table.rs b/vortex-datafusion/src/v2/table.rs index 3aa0ed37936..2812caca44e 100644 --- a/vortex-datafusion/src/v2/table.rs +++ b/vortex-datafusion/src/v2/table.rs @@ -126,7 +126,7 @@ fn try_convert_operator(op: &DFOperator) -> Option { } } -/// A DataFusion [`TableProvider`] backed by a Vortex [`DataSource`]. +/// A DataFusion [`TableProvider`] backed by a Vortex [`DataSourceRef`]. /// /// Maps each Vortex scan split to one DataFusion partition, letting DataFusion's scheduler /// control concurrency. diff --git a/vortex-file/public-api.lock b/vortex-file/public-api.lock index 256d6e1c569..47b6691df36 100644 --- a/vortex-file/public-api.lock +++ b/vortex-file/public-api.lock @@ -23,6 +23,18 @@ pub vortex_file::segments::RequestMetrics::individual_requests: vortex_metrics:: pub vortex_file::segments::RequestMetrics::num_requests_coalesced: vortex_metrics::histogram::Histogram impl vortex_file::segments::RequestMetrics pub fn vortex_file::segments::RequestMetrics::new(metrics_registry: &dyn vortex_metrics::MetricsRegistry, labels: alloc::vec::Vec) -> Self +pub mod vortex_file::v2 +pub struct vortex_file::v2::FileStatsLayoutReader +impl vortex_file::v2::FileStatsLayoutReader +pub fn vortex_file::v2::FileStatsLayoutReader::new(child: vortex_layout::reader::LayoutReaderRef, file_stats: vortex_file::FileStatistics, session: vortex_session::VortexSession) -> Self +impl vortex_layout::reader::LayoutReader for vortex_file::v2::FileStatsLayoutReader +pub fn vortex_file::v2::FileStatsLayoutReader::dtype(&self) -> &vortex_dtype::dtype::DType +pub fn vortex_file::v2::FileStatsLayoutReader::filter_evaluation(&self, row_range: &core::ops::range::Range, expr: &vortex_array::expr::expression::Expression, mask: vortex_array::mask_future::MaskFuture) -> vortex_error::VortexResult +pub fn vortex_file::v2::FileStatsLayoutReader::name(&self) -> &alloc::sync::Arc +pub fn vortex_file::v2::FileStatsLayoutReader::projection_evaluation(&self, row_range: &core::ops::range::Range, expr: &vortex_array::expr::expression::Expression, mask: vortex_array::mask_future::MaskFuture) -> vortex_error::VortexResult +pub fn vortex_file::v2::FileStatsLayoutReader::pruning_evaluation(&self, row_range: &core::ops::range::Range, expr: &vortex_array::expr::expression::Expression, mask: vortex_mask::Mask) -> vortex_error::VortexResult +pub fn vortex_file::v2::FileStatsLayoutReader::register_splits(&self, field_mask: &[vortex_dtype::field_mask::FieldMask], row_range: &core::ops::range::Range, splits: &mut alloc::collections::btree::set::BTreeSet) -> vortex_error::VortexResult<()> +pub fn vortex_file::v2::FileStatsLayoutReader::row_count(&self) -> u64 pub enum vortex_file::DeserializeStep pub vortex_file::DeserializeStep::Done(vortex_file::Footer) pub vortex_file::DeserializeStep::NeedFileSize diff --git a/vortex-scan/public-api.lock b/vortex-scan/public-api.lock index 515d1c01dad..97e06606b6c 100644 --- a/vortex-scan/public-api.lock +++ b/vortex-scan/public-api.lock @@ -1,15 +1,22 @@ pub mod vortex_scan pub mod vortex_scan::api -pub enum vortex_scan::api::Estimate -pub vortex_scan::api::Estimate::Exact(T) -pub vortex_scan::api::Estimate::Unknown -pub vortex_scan::api::Estimate::UpperBound(T) -impl core::default::Default for vortex_scan::api::Estimate -pub fn vortex_scan::api::Estimate::default() -> vortex_scan::api::Estimate +pub struct vortex_scan::api::Estimate +pub vortex_scan::api::Estimate::lower: T +pub vortex_scan::api::Estimate::upper: core::option::Option +impl vortex_scan::api::Estimate +pub fn vortex_scan::api::Estimate::exact(value: T) -> Self +impl core::clone::Clone for vortex_scan::api::Estimate +pub fn vortex_scan::api::Estimate::clone(&self) -> vortex_scan::api::Estimate +impl core::default::Default for vortex_scan::api::Estimate +pub fn vortex_scan::api::Estimate::default() -> Self +impl core::fmt::Debug for vortex_scan::api::Estimate +pub fn vortex_scan::api::Estimate::fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result pub struct vortex_scan::api::ScanRequest pub vortex_scan::api::ScanRequest::filter: core::option::Option pub vortex_scan::api::ScanRequest::limit: core::option::Option pub vortex_scan::api::ScanRequest::projection: core::option::Option +pub vortex_scan::api::ScanRequest::row_range: core::option::Option> +pub vortex_scan::api::ScanRequest::selection: vortex_scan::Selection impl core::clone::Clone for vortex_scan::api::ScanRequest pub fn vortex_scan::api::ScanRequest::clone(&self) -> vortex_scan::api::ScanRequest impl core::default::Default for vortex_scan::api::ScanRequest @@ -17,28 +24,37 @@ pub fn vortex_scan::api::ScanRequest::default() -> vortex_scan::api::ScanRequest impl core::fmt::Debug for vortex_scan::api::ScanRequest pub fn vortex_scan::api::ScanRequest::fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result pub trait vortex_scan::api::DataSource: 'static + core::marker::Send + core::marker::Sync -pub fn vortex_scan::api::DataSource::deserialize_split(&self, data: &[u8]) -> vortex_error::VortexResult +pub fn vortex_scan::api::DataSource::byte_size_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::api::DataSource::deserialize_split(&self, data: &[u8], session: &vortex_session::VortexSession) -> vortex_error::VortexResult pub fn vortex_scan::api::DataSource::dtype(&self) -> &vortex_dtype::dtype::DType pub fn vortex_scan::api::DataSource::row_count_estimate(&self) -> vortex_scan::api::Estimate -pub fn vortex_scan::api::DataSource::scan<'life0, 'async_trait>(&'life0 self, scan_request: vortex_scan::api::ScanRequest) -> core::pin::Pin> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait -pub fn vortex_scan::api::DataSource::serialize_split(&self, split: &dyn vortex_scan::api::Split) -> vortex_error::VortexResult> +pub fn vortex_scan::api::DataSource::scan(&self, scan_request: vortex_scan::api::ScanRequest) -> vortex_error::VortexResult +pub fn vortex_scan::api::DataSource::serialize(&self) -> vortex_error::VortexResult>> impl vortex_scan::api::DataSource for vortex_scan::layout::LayoutReaderDataSource -pub fn vortex_scan::layout::LayoutReaderDataSource::deserialize_split(&self, _split: &[u8]) -> vortex_error::VortexResult +pub fn vortex_scan::layout::LayoutReaderDataSource::byte_size_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::layout::LayoutReaderDataSource::deserialize_split(&self, _data: &[u8], _session: &vortex_session::VortexSession) -> vortex_error::VortexResult pub fn vortex_scan::layout::LayoutReaderDataSource::dtype(&self) -> &vortex_dtype::dtype::DType pub fn vortex_scan::layout::LayoutReaderDataSource::row_count_estimate(&self) -> vortex_scan::api::Estimate -pub fn vortex_scan::layout::LayoutReaderDataSource::scan<'life0, 'async_trait>(&'life0 self, scan_request: vortex_scan::api::ScanRequest) -> core::pin::Pin> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait -pub fn vortex_scan::layout::LayoutReaderDataSource::serialize_split(&self, _split: &dyn vortex_scan::api::Split) -> vortex_error::VortexResult> -pub trait vortex_scan::api::DataSourceProvider: 'static -pub fn vortex_scan::api::DataSourceProvider::initialize<'life0, 'life1, 'async_trait>(&'life0 self, uri: alloc::string::String, session: &'life1 vortex_session::VortexSession) -> core::pin::Pin>> + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait +pub fn vortex_scan::layout::LayoutReaderDataSource::scan(&self, scan_request: vortex_scan::api::ScanRequest) -> vortex_error::VortexResult +impl vortex_scan::api::DataSource for vortex_scan::multi::MultiDataSource +pub fn vortex_scan::multi::MultiDataSource::deserialize_split(&self, _data: &[u8], _session: &vortex_session::VortexSession) -> vortex_error::VortexResult +pub fn vortex_scan::multi::MultiDataSource::dtype(&self) -> &vortex_dtype::dtype::DType +pub fn vortex_scan::multi::MultiDataSource::row_count_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::multi::MultiDataSource::scan(&self, scan_request: vortex_scan::api::ScanRequest) -> vortex_error::VortexResult +pub trait vortex_scan::api::DataSourceOpener: 'static +pub fn vortex_scan::api::DataSourceOpener::open<'life0, 'life1, 'async_trait>(&'life0 self, uri: alloc::string::String, session: &'life1 vortex_session::VortexSession) -> core::pin::Pin> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait +pub trait vortex_scan::api::DataSourceRemote: 'static +pub fn vortex_scan::api::DataSourceRemote::deserialize_data_source(&self, data: &[u8], session: &vortex_session::VortexSession) -> vortex_error::VortexResult pub trait vortex_scan::api::DataSourceScan: 'static + core::marker::Send pub fn vortex_scan::api::DataSourceScan::dtype(&self) -> &vortex_dtype::dtype::DType -pub fn vortex_scan::api::DataSourceScan::next_splits<'life0, 'async_trait>(&'life0 mut self, max_splits: usize) -> core::pin::Pin>> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait -pub fn vortex_scan::api::DataSourceScan::remaining_splits_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::api::DataSourceScan::split_count_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::api::DataSourceScan::splits(self: alloc::boxed::Box) -> vortex_scan::api::SplitStream pub trait vortex_scan::api::Split: 'static + core::marker::Send pub fn vortex_scan::api::Split::as_any(&self) -> &dyn core::any::Any pub fn vortex_scan::api::Split::byte_size_estimate(&self) -> vortex_scan::api::Estimate -pub fn vortex_scan::api::Split::execute(self: alloc::boxed::Box) -> vortex_error::VortexResult +pub fn vortex_scan::api::Split::execute(self: alloc::boxed::Box) -> futures_core::future::BoxFuture<'static, vortex_error::VortexResult> pub fn vortex_scan::api::Split::row_count_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::api::Split::serialize(&self) -> vortex_error::VortexResult>> pub type vortex_scan::api::DataSourceRef = alloc::sync::Arc pub type vortex_scan::api::DataSourceScanRef = alloc::boxed::Box pub type vortex_scan::api::SplitRef = alloc::boxed::Box @@ -58,12 +74,28 @@ pub mod vortex_scan::layout pub struct vortex_scan::layout::LayoutReaderDataSource impl vortex_scan::layout::LayoutReaderDataSource pub fn vortex_scan::layout::LayoutReaderDataSource::new(reader: vortex_layout::reader::LayoutReaderRef, session: vortex_session::VortexSession) -> Self +pub fn vortex_scan::layout::LayoutReaderDataSource::with_metrics_registry(self, metrics: alloc::sync::Arc) -> Self +pub fn vortex_scan::layout::LayoutReaderDataSource::with_some_metrics_registry(self, metrics: core::option::Option>) -> Self +pub fn vortex_scan::layout::LayoutReaderDataSource::with_split_size(self, split_size: u64) -> Self impl vortex_scan::api::DataSource for vortex_scan::layout::LayoutReaderDataSource -pub fn vortex_scan::layout::LayoutReaderDataSource::deserialize_split(&self, _split: &[u8]) -> vortex_error::VortexResult +pub fn vortex_scan::layout::LayoutReaderDataSource::byte_size_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::layout::LayoutReaderDataSource::deserialize_split(&self, _data: &[u8], _session: &vortex_session::VortexSession) -> vortex_error::VortexResult pub fn vortex_scan::layout::LayoutReaderDataSource::dtype(&self) -> &vortex_dtype::dtype::DType pub fn vortex_scan::layout::LayoutReaderDataSource::row_count_estimate(&self) -> vortex_scan::api::Estimate -pub fn vortex_scan::layout::LayoutReaderDataSource::scan<'life0, 'async_trait>(&'life0 self, scan_request: vortex_scan::api::ScanRequest) -> core::pin::Pin> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait -pub fn vortex_scan::layout::LayoutReaderDataSource::serialize_split(&self, _split: &dyn vortex_scan::api::Split) -> vortex_error::VortexResult> +pub fn vortex_scan::layout::LayoutReaderDataSource::scan(&self, scan_request: vortex_scan::api::ScanRequest) -> vortex_error::VortexResult +pub mod vortex_scan::multi +pub struct vortex_scan::multi::MultiDataSource +impl vortex_scan::multi::MultiDataSource +pub fn vortex_scan::multi::MultiDataSource::lazy(first: vortex_scan::api::DataSourceRef, remaining: alloc::vec::Vec>, session: &vortex_session::VortexSession) -> Self +pub fn vortex_scan::multi::MultiDataSource::try_new(children: alloc::vec::Vec, session: &vortex_session::VortexSession) -> vortex_error::VortexResult +pub fn vortex_scan::multi::MultiDataSource::with_prefetch(self, prefetch: usize) -> Self +impl vortex_scan::api::DataSource for vortex_scan::multi::MultiDataSource +pub fn vortex_scan::multi::MultiDataSource::deserialize_split(&self, _data: &[u8], _session: &vortex_session::VortexSession) -> vortex_error::VortexResult +pub fn vortex_scan::multi::MultiDataSource::dtype(&self) -> &vortex_dtype::dtype::DType +pub fn vortex_scan::multi::MultiDataSource::row_count_estimate(&self) -> vortex_scan::api::Estimate +pub fn vortex_scan::multi::MultiDataSource::scan(&self, scan_request: vortex_scan::api::ScanRequest) -> vortex_error::VortexResult +pub trait vortex_scan::multi::DataSourceFactory: 'static + core::marker::Send + core::marker::Sync +pub fn vortex_scan::multi::DataSourceFactory::open<'life0, 'async_trait>(&'life0 self) -> core::pin::Pin>> + core::marker::Send + 'async_trait)>> where Self: 'async_trait, 'life0: 'async_trait pub mod vortex_scan::row_mask pub enum vortex_scan::Selection pub vortex_scan::Selection::All @@ -73,6 +105,8 @@ impl core::clone::Clone for vortex_scan::Selection pub fn vortex_scan::Selection::clone(&self) -> vortex_scan::Selection impl core::default::Default for vortex_scan::Selection pub fn vortex_scan::Selection::default() -> vortex_scan::Selection +impl core::fmt::Debug for vortex_scan::Selection +pub fn vortex_scan::Selection::fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result pub enum vortex_scan::SplitBy pub vortex_scan::SplitBy::Layout pub vortex_scan::SplitBy::RowCount(usize) @@ -103,10 +137,12 @@ pub fn vortex_scan::ScanBuilder::into_record_batc pub fn vortex_scan::ScanBuilder::into_record_batch_stream(self, schema: arrow_schema::schema::SchemaRef) -> vortex_error::VortexResult> + core::marker::Send + 'static> impl vortex_scan::ScanBuilder pub fn vortex_scan::ScanBuilder::build(self) -> vortex_error::VortexResult>>>> +pub fn vortex_scan::ScanBuilder::concurrency(&self) -> usize pub fn vortex_scan::ScanBuilder::dtype(&self) -> vortex_error::VortexResult pub fn vortex_scan::ScanBuilder::into_iter(self, runtime: &B) -> vortex_error::VortexResult> + 'static> pub fn vortex_scan::ScanBuilder::into_stream(self) -> vortex_error::VortexResult> + core::marker::Send + 'static + use> pub fn vortex_scan::ScanBuilder::map(self, map_fn: impl core::ops::function::Fn(A) -> vortex_error::VortexResult + 'static + core::marker::Send + core::marker::Sync) -> vortex_scan::ScanBuilder +pub fn vortex_scan::ScanBuilder::ordered(&self) -> bool pub fn vortex_scan::ScanBuilder::prepare(self) -> vortex_error::VortexResult> pub fn vortex_scan::ScanBuilder::session(&self) -> &vortex_session::VortexSession pub fn vortex_scan::ScanBuilder::with_concurrency(self, concurrency: usize) -> Self @@ -120,4 +156,6 @@ pub fn vortex_scan::ScanBuilder::with_row_offset(self, row_offset: u64) -> Se pub fn vortex_scan::ScanBuilder::with_row_range(self, row_range: core::ops::range::Range) -> Self pub fn vortex_scan::ScanBuilder::with_selection(self, selection: vortex_scan::Selection) -> Self pub fn vortex_scan::ScanBuilder::with_some_filter(self, filter: core::option::Option) -> Self +pub fn vortex_scan::ScanBuilder::with_some_limit(self, limit: core::option::Option) -> Self +pub fn vortex_scan::ScanBuilder::with_some_metrics_registry(self, metrics: core::option::Option>) -> Self pub fn vortex_scan::ScanBuilder::with_split_by(self, split_by: vortex_scan::SplitBy) -> Self diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 35ae327f7ae..6dd8fd58908 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -59,8 +59,8 @@ const DEFAULT_PREFETCH: usize = 8; /// deferred children are opened in the background using spawned tasks on the session's runtime, /// keeping the I/O pipeline full while the engine processes splits from already-open sources. /// -/// Once a deferred child is successfully opened, it is promoted to [`MultiChild::Opened`] so -/// that subsequent scans reuse the opened source without re-opening. +/// Once a deferred child is successfully opened, it is stored so that subsequent scans reuse the +/// opened source without re-opening. pub struct MultiDataSource { dtype: DType, children: Arc>>, From 270b24223ce9b4cc28098e5ad0167a31de7f22a7 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 12:42:58 -0500 Subject: [PATCH 23/57] DataFusion streaming Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + benchmarks/datafusion-bench/src/main.rs | 123 ++------ vortex-datafusion/src/lib.rs | 1 - vortex-datafusion/src/persistent/opener.rs | 154 +++------- vortex-file/Cargo.toml | 3 +- vortex-file/src/lib.rs | 2 + vortex-file/src/multi/builder.rs | 317 +++++++++++++++++++++ vortex-file/src/multi/glob.rs | 208 ++++++++++++++ vortex-file/src/multi/mod.rs | 22 ++ vortex-file/src/multi/source.rs | 144 ++++++++++ vortex-scan/src/multi.rs | 21 ++ 11 files changed, 783 insertions(+), 213 deletions(-) create mode 100644 vortex-file/src/multi/builder.rs create mode 100644 vortex-file/src/multi/glob.rs create mode 100644 vortex-file/src/multi/mod.rs create mode 100644 vortex-file/src/multi/source.rs diff --git a/Cargo.lock b/Cargo.lock index f960db924c0..3a637ab61fc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11164,6 +11164,7 @@ dependencies = [ "flatbuffers", "futures", "getrandom 0.3.4", + "glob", "itertools 0.14.0", "kanal", "object_store", diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 60777b2217b..fd02a4b25fd 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -5,7 +5,6 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::Instant; -use async_trait::async_trait; use clap::Parser; use clap::value_parser; use custom_labels::asynchronous::Label; @@ -25,17 +24,12 @@ use datafusion_bench::tracer::set_labels; use datafusion_physical_plan::ExecutionPlan; use datafusion_physical_plan::collect; use futures::StreamExt; -use object_store::ObjectStore; use parking_lot::Mutex; use tokio::fs::File; -use vortex::error::VortexExpect; -use vortex::error::vortex_err; -use vortex::file::OpenOptionsSessionExt; -use vortex::file::v2::FileStatsLayoutReader; +use vortex::file::multi::FileDiscovery; +use vortex::file::multi::MultiFileDataSourceBuilder; +use vortex::scan::api::DataSource as _; use vortex::scan::api::DataSourceRef; -use vortex::scan::layout::LayoutReaderDataSource; -use vortex::scan::multi::DataSourceFactory; -use vortex::scan::multi::MultiDataSource; use vortex_bench::Benchmark; use vortex_bench::BenchmarkArg; use vortex_bench::CompactionStrategy; @@ -231,8 +225,8 @@ async fn main() -> anyhow::Result<()> { Ok(()) } -fn use_v2() -> bool { - std::env::var("VORTEX_DATAFUSION_V2").is_ok_and(|v| v == "1") +fn use_scan_api() -> bool { + std::env::var("VORTEX_USE_SCAN_API").is_ok_and(|v| v == "1") } async fn register_benchmark_tables( @@ -242,7 +236,7 @@ async fn register_benchmark_tables( ) -> anyhow::Result<()> { match format { Format::Arrow => register_arrow_tables(session, benchmark).await, - _ if use_v2() && matches!(format, Format::OnDiskVortex | Format::VortexCompact) => { + _ if use_scan_api() && matches!(format, Format::OnDiskVortex | Format::VortexCompact) => { register_v2_tables(session, benchmark, format).await } _ => { @@ -283,102 +277,39 @@ async fn register_benchmark_tables( } } -/// A [`DataSourceFactory`] that lazily opens a single Vortex file. -struct VortexFileFactory { - object_store: Arc, - path: PathBuf, -} - -#[async_trait] -impl DataSourceFactory for VortexFileFactory { - async fn open(&self) -> vortex::error::VortexResult> { - let file = SESSION - .open_options() - .open_object_store( - &self.object_store, - self.path - .as_os_str() - .to_str() - .ok_or_else(|| vortex_err!("Invalid path"))?, - ) - .await?; - - let mut reader = file.layout_reader()?; - if let Some(stats) = file.file_stats().cloned() { - reader = Arc::new(FileStatsLayoutReader::new(reader, stats, SESSION.clone())); - } - Ok(Some(Arc::new(LayoutReaderDataSource::new( - reader, - SESSION.clone(), - )))) - } -} - -/// Register tables using the V2 `VortexTable` + `MultiDataSource` path. +/// Register tables using the V2 `VortexTable` + `MultiFileDataSource` path. async fn register_v2_tables( session: &SessionContext, benchmark: &B, format: Format, ) -> anyhow::Result<()> { - use futures::TryStreamExt; use vortex_datafusion::v2::VortexTable; let benchmark_base = benchmark.data_url().join(&format!("{}/", format.name()))?; for table in benchmark.table_specs().iter() { let pattern = benchmark.pattern(table.name, format); - let table_url = ListingTableUrl::try_new(benchmark_base.clone(), pattern)?; - - // Use the same ListingTableUrl file discovery as v1. - let state = session.state(); - let store = state.runtime_env().object_store(table_url.object_store())?; - let mut file_metas: Vec<_> = table_url - .list_all_files(&state, store.as_ref(), format.ext()) - .await? - .try_collect() - .await?; - file_metas.sort_by(|a, b| a.location.cmp(&b.location)); - - anyhow::ensure!( - !file_metas.is_empty(), - "no files found for table {}", - table.name - ); - - // Convert object store paths to absolute local file paths. - // Object store locations are relative (e.g. "path/to/file.vortex"), so prepend "/". - let matching_paths: Vec = file_metas - .iter() - .map(|meta| PathBuf::from(format!("/{}", meta.location))) - .collect(); - - // Open the first file eagerly to get the dtype/schema. - let first_source = VortexFileFactory { - object_store: store.clone(), - path: matching_paths[0].clone(), - } - .open() - .await? - .vortex_expect("Missing first file"); - - let arrow_schema = Arc::new(first_source.dtype().to_arrow_schema()?); - - // Create lazy factories for remaining files. - let remaining: Vec> = matching_paths[1..] - .iter() - .map(|path| { - Arc::new(VortexFileFactory { - object_store: store.clone(), - path: path.clone(), - }) as _ - }) - .collect(); + let table_url = ListingTableUrl::try_new(benchmark_base.clone(), pattern.clone())?; + let store = session + .state() + .runtime_env() + .object_store(table_url.object_store())?; + + let discovery = pattern + .map(FileDiscovery::Glob) + .unwrap_or(FileDiscovery::ListAll); + let multi_ds = MultiFileDataSourceBuilder::new( + SESSION.clone(), + store.clone(), + benchmark_base.as_str(), + ) + .with_prefix(table_url.prefix().clone()) + .with_discovery(discovery) + .build() + .await?; - let data_source: DataSourceRef = if remaining.is_empty() { - first_source - } else { - Arc::new(MultiDataSource::lazy(first_source, remaining, &SESSION)) - }; + let arrow_schema = Arc::new(multi_ds.dtype().to_arrow_schema()?); + let data_source: DataSourceRef = Arc::new(multi_ds); let table_provider = Arc::new(VortexTable::new(data_source, SESSION.clone(), arrow_schema)); session.register_table(table.name, table_provider)?; diff --git a/vortex-datafusion/src/lib.rs b/vortex-datafusion/src/lib.rs index c98eaca5920..133cee13c20 100644 --- a/vortex-datafusion/src/lib.rs +++ b/vortex-datafusion/src/lib.rs @@ -10,7 +10,6 @@ use vortex::expr::stats::Precision; mod convert; mod persistent; -/// V2 DataFusion integration using direct `TableProvider` + `ExecutionPlan` implementation. pub mod v2; #[cfg(test)] diff --git a/vortex-datafusion/src/persistent/opener.rs b/vortex-datafusion/src/persistent/opener.rs index 3496feaa939..fb013322c11 100644 --- a/vortex-datafusion/src/persistent/opener.rs +++ b/vortex-datafusion/src/persistent/opener.rs @@ -38,15 +38,10 @@ use vortex::array::arrow::ArrowArrayExecutor; use vortex::error::VortexError; use vortex::file::OpenOptionsSessionExt; use vortex::io::InstrumentedReadAt; -use vortex::io::session::RuntimeSessionExt; use vortex::layout::LayoutReader; use vortex::metrics::Label; use vortex::metrics::MetricsRegistry; use vortex::scan::ScanBuilder; -use vortex::scan::Selection; -use vortex::scan::api::DataSource as _; -use vortex::scan::api::ScanRequest; -use vortex::scan::layout::LayoutReaderDataSource; use vortex::session::VortexSession; use vortex_utils::aliases::dash_map::DashMap; use vortex_utils::aliases::dash_map::Entry; @@ -130,7 +125,6 @@ impl FileOpener for VortexOpener { let limit = self.limit; let layout_reader = self.layout_readers.clone(); let has_output_ordering = self.has_output_ordering; - let use_scan_api = std::env::var("VORTEX_USE_SCAN_API").is_ok_and(|v| v == "1"); let expr_convertor = self.expression_convertor.clone(); let projection_pushdown = self.projection_pushdown; @@ -300,7 +294,23 @@ impl FileOpener for VortexOpener { } }; - // Convert the DF filter to a Vortex expression. + let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); + + 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, + ); + } + let filter = filter .and_then(|f| { // Verify that all filters we've accepted from DataFusion get pushed down. @@ -334,118 +344,31 @@ impl FileOpener for VortexOpener { }) .transpose()?; - let file_location = file.object_meta.location.clone(); - - // Both paths produce a boxed stream of VortexResult. - let chunk_stream = if use_scan_api { - // Scan API path: use LayoutReaderDataSource + ScanRequest. - let session = session.clone(); - let stream_schema = stream_schema.clone(); - - let mut selection = Selection::All; - if let Some(extensions) = &file.extensions - && let Some(vortex_plan) = extensions.downcast_ref::() - && let Some(sel) = vortex_plan.selection() - { - selection = sel.clone(); - } - - let row_range = file.range.map(|file_range| { - byte_range_to_row_range( - file_range.start as u64..file_range.end as u64, - vxf.row_count(), - file.object_meta.size, - ) - }); - - let scan_limit = if filter.is_none() { limit } else { None }; - - let scan_request = ScanRequest { - projection: Some(scan_projection), - filter, - row_range, - selection, - limit: scan_limit, - }; - - let data_source = LayoutReaderDataSource::new(layout_reader, session.clone()) - .with_some_metrics_registry(Some(metrics_registry)); - - let scan = data_source - .scan(scan_request) - .map_err(|e| exec_datafusion_err!("Failed to create Vortex scan: {e}"))?; - - // Spawn Vortex-to-Arrow conversion onto CPU threads so it doesn't - // block the polling thread. The inner scan stream already handles I/O - // and compute concurrency; this buffer just needs to be large enough - // that conversion doesn't stall the pipeline. - let handle = session.handle(); - let num_workers = std::thread::available_parallelism() - .map(|n| n.get()) - .unwrap_or(1); - scan.splits() - .and_then(|s| s.execute()) - .try_flatten() - .map(move |result| { - let session = session.clone(); - let stream_schema = stream_schema.clone(); - handle.spawn_cpu(move || { - result.and_then(|chunk| { - let mut ctx = session.create_execution_ctx(); - chunk.execute_record_batch(&stream_schema, &mut ctx) - }) - }) - }) - .buffered(num_workers) - .boxed() - } else { - // Direct ScanBuilder path (existing). - let mut scan_builder = ScanBuilder::new(session.clone(), layout_reader); - - 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, - ); - } - - if let Some(limit) = limit - && filter.is_none() - { - scan_builder = scan_builder.with_limit(limit); - } - - scan_builder - .with_metrics_registry(metrics_registry) - .with_projection(scan_projection) - .with_some_filter(filter) - .with_ordered(has_output_ordering) - .map(move |chunk| { - let mut ctx = session.create_execution_ctx(); - chunk.execute_record_batch(&stream_schema, &mut ctx) - }) - .into_stream() - .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? - .boxed() - }; + if let Some(limit) = limit + && filter.is_none() + { + scan_builder = scan_builder.with_limit(limit); + } - // Shared post-processing: batch slicing, error mapping, leftover projection. - let stream = chunk_stream + let stream = scan_builder + .with_metrics_registry(metrics_registry) + .with_projection(scan_projection) + .with_some_filter(filter) + .with_ordered(has_output_ordering) + .map(move |chunk| { + let mut ctx = session.create_execution_ctx(); + chunk.execute_record_batch(&stream_schema, &mut ctx) + }) + .into_stream() + .map_err(|e| exec_datafusion_err!("Failed to create Vortex stream: {e}"))? .map_ok(move |rb| { - // Slice the stream to respect DataFusion's configured batch size. + // We try and slice the stream into respecting datafusion's configured batch size. stream::iter( (0..rb.num_rows().div_ceil(batch_size * 2)) .flat_map(move |block_idx| { let offset = block_idx * batch_size * 2; + // If we have less than two batches worth of rows left, we keep them together as a single batch. if rb.num_rows() - offset < 2 * batch_size { let length = rb.num_rows() - offset; [Some(rb.slice(offset, length)), None].into_iter() @@ -460,9 +383,10 @@ impl FileOpener for VortexOpener { ) }) .map_err(move |e: VortexError| { - DataFusionError::External(Box::new( - e.with_context(format!("Failed to read Vortex file: {file_location}",)), - )) + DataFusionError::External(Box::new(e.with_context(format!( + "Failed to read Vortex file: {}", + file.object_meta.location + )))) }) .try_flatten() .map(move |batch| { diff --git a/vortex-file/Cargo.toml b/vortex-file/Cargo.toml index 54c3bf0ff1d..72555eb5e43 100644 --- a/vortex-file/Cargo.toml +++ b/vortex-file/Cargo.toml @@ -22,6 +22,7 @@ bytes = { workspace = true } flatbuffers = { workspace = true } futures = { workspace = true, features = ["std", "async-await"] } getrandom_v03 = { workspace = true } # Needed to pickup the "wasm_js" feature for wasm targets from the workspace configuration +glob = { workspace = true, optional = true } itertools = { workspace = true } kanal = { workspace = true } object_store = { workspace = true, optional = true } @@ -74,7 +75,7 @@ vortex-cuda-macros = { workspace = true } workspace = true [features] -object_store = ["dep:object_store", "vortex-io/object_store", "tokio"] +object_store = ["dep:object_store", "dep:glob", "vortex-io/object_store", "tokio"] tokio = [ "dep:tokio", "vortex-error/tokio", diff --git a/vortex-file/src/lib.rs b/vortex-file/src/lib.rs index 64af6cb53e8..f1860db8fe1 100644 --- a/vortex-file/src/lib.rs +++ b/vortex-file/src/lib.rs @@ -93,6 +93,8 @@ mod counting; mod file; mod footer; +#[cfg(feature = "object_store")] +pub mod multi; mod open; mod pruning; mod read; diff --git a/vortex-file/src/multi/builder.rs b/vortex-file/src/multi/builder.rs new file mode 100644 index 00000000000..ae4d24fabf1 --- /dev/null +++ b/vortex-file/src/multi/builder.rs @@ -0,0 +1,317 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Builder for constructing a [`MultiFileDataSource`]. + +use std::sync::Arc; + +use glob::Pattern; +use object_store::ObjectStore; +use object_store::path::Path; +use tracing::Instrument; +use tracing::debug; +use tracing::info_span; +use vortex_array::expr::Expression; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; +use vortex_scan::multi::DataSourceFactory; +use vortex_scan::multi::MultiDataSource; +use vortex_session::VortexSession; + +use super::glob::expand_glob; +use super::glob::list_all; +use super::source::MultiFileDataSource; +use super::source::VortexFileFactory; +use super::source::data_source_from_file; +use crate::OpenOptionsSessionExt; +use crate::VortexOpenOptions; + +/// How to handle schema differences across files in a [`MultiFileDataSource`]. +#[derive(Debug, Clone, Default)] +pub enum SchemaResolution { + /// All files must have exactly the same [`DType`](vortex_dtype::DType). Error on mismatch. + #[default] + Exact, + /// Unify schemas: allow missing columns (filled with nulls) and compatible type upcasts. + /// + /// **Not yet implemented** — will return an error at build time. + Union, +} + +/// How files are discovered for a [`MultiFileDataSource`]. +pub enum FileDiscovery { + /// Explicit list of file paths (relative to the object store root). + Paths(Vec), + /// A glob pattern to expand against the object store (relative to the object store root). + Glob(Pattern), + /// List all files under the prefix that match the configured file extension. + ListAll, +} + +/// Builder for constructing a [`MultiFileDataSource`]. +/// +/// By default, all files in the object store are discovered (equivalent to a `*` glob). +/// Use [`with_paths`](Self::with_paths) or [`with_glob`](Self::with_glob) to restrict +/// which files are included. +/// +/// # Examples +/// +/// ```ignore +/// // Discover all files: +/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// .build() +/// .await?; +/// +/// // From a glob pattern: +/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// .with_glob(glob::Pattern::new("**/*.vortex")?) +/// .with_prefetch(16) +/// .build() +/// .await?; +/// +/// // From explicit paths: +/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// .with_paths(vec!["a.vortex".into(), "b.vortex".into()]) +/// .build() +/// .await?; +/// ``` +pub struct MultiFileDataSourceBuilder { + session: VortexSession, + object_store: Arc, + base_url: String, + /// The path prefix within the object store used for [`FileDiscovery::ListAll`]. + prefix: Path, + /// File extension filter for [`FileDiscovery::ListAll`] (e.g. `".vortex"`). + file_extension: String, + discovery: FileDiscovery, + schema_resolution: SchemaResolution, + open_options_fn: Arc VortexOpenOptions + Send + Sync>, + prefetch: Option, + filter: Option, + dtype: Option, +} + +impl MultiFileDataSourceBuilder { + /// Create a new builder from an object store and base URL prefix. + /// + /// The `base_url` is used for display/debug purposes. It should typically match the + /// location of the files (e.g. `"s3://bucket/data/"`). + pub fn new( + session: VortexSession, + object_store: Arc, + base_url: impl Into, + ) -> Self { + Self { + session, + object_store, + base_url: base_url.into(), + prefix: Path::from(""), + file_extension: ".vortex".to_string(), + discovery: FileDiscovery::ListAll, + schema_resolution: SchemaResolution::default(), + open_options_fn: Arc::new(|opts| opts), + prefetch: None, + filter: None, + dtype: None, + } + } + + /// Set the path prefix within the object store. + /// + /// This prefix is used by [`FileDiscovery::ListAll`] to scope file listing to a + /// subdirectory of the object store. For example, if the object store represents + /// `file:///` and the data lives at `/data/tables/`, set the prefix to `data/tables`. + pub fn with_prefix(mut self, prefix: impl Into) -> Self { + self.prefix = prefix.into(); + self + } + + /// Set the file extension filter for [`FileDiscovery::ListAll`]. + /// + /// Only files ending with this extension will be included. Defaults to `".vortex"`. + pub fn with_file_extension(mut self, ext: impl Into) -> Self { + self.file_extension = ext.into(); + self + } + + /// Set how files are discovered. + pub fn with_discovery(mut self, discovery: FileDiscovery) -> Self { + self.discovery = discovery; + self + } + + /// Set explicit file paths (relative to the object store root). + pub fn with_paths(self, paths: Vec) -> Self { + self.with_discovery(FileDiscovery::Paths(paths)) + } + + /// Discover files by expanding a glob pattern against the object store. + /// + /// The pattern is relative to the object store root + /// (e.g. `"data/**/*.vortex"`). Expansion happens eagerly during [`build`](Self::build). + pub fn with_glob(self, pattern: Pattern) -> Self { + self.with_discovery(FileDiscovery::Glob(pattern)) + } + + /// Set how schema differences across files should be handled. + pub fn with_schema_resolution(mut self, resolution: SchemaResolution) -> Self { + self.schema_resolution = resolution; + self + } + + /// Customize [`VortexOpenOptions`] applied to each file. + /// + /// Use this to configure segment caches, metrics registries, or other per-file options. + pub fn with_open_options( + mut self, + f: impl Fn(VortexOpenOptions) -> VortexOpenOptions + Send + Sync + 'static, + ) -> Self { + self.open_options_fn = Arc::new(f); + self + } + + /// Set the prefetch concurrency for lazy file opening. + /// + /// Higher values overlap more file-opening I/O with split execution but use more memory + /// for in-flight metadata. + /// + /// Defaults to [`std::thread::available_parallelism`]. + pub fn with_prefetch(mut self, prefetch: usize) -> Self { + self.prefetch = Some(prefetch); + self + } + + /// Set an explicit [`DType`] for the data source. + /// + /// When provided, no file needs to be eagerly opened to determine the schema — all files + /// are deferred and opened lazily during scanning. This is useful when the caller already + /// knows the schema (e.g. from a catalog or a prior scan). Combine with + /// [`with_open_options`](Self::with_open_options) to pass the dtype through to + /// [`VortexOpenOptions::with_dtype`](crate::VortexOpenOptions::with_dtype) on each file + /// open for additional I/O savings. + pub fn with_dtype(mut self, dtype: DType) -> Self { + self.dtype = Some(dtype); + self + } + + /// Set a filter expression for file-level pruning. + /// + /// Files whose statistics indicate they cannot match the filter will be skipped entirely. + /// When no explicit dtype is provided, the first file is always opened (to determine the + /// schema); deferred files may be skipped if their statistics prove the filter cannot match. + pub fn with_filter(mut self, filter: Expression) -> Self { + self.filter = Some(filter); + self + } + + /// Build the [`MultiFileDataSource`]. + /// + /// If a glob pattern was provided via [`with_glob`](Self::with_glob), it is expanded + /// eagerly against the object store. If a [`DType`] was provided via + /// [`with_dtype`](Self::with_dtype), all files are opened lazily during scanning. + /// Otherwise, the first file is opened eagerly to determine the schema. + pub async fn build(self) -> VortexResult { + async { + if matches!(self.schema_resolution, SchemaResolution::Union) { + vortex_bail!("SchemaResolution::Union is not yet implemented"); + } + + let discovery_kind = match &self.discovery { + FileDiscovery::Paths(p) => format!("paths({})", p.len()), + FileDiscovery::Glob(g) => format!("glob({})", g.as_str()), + FileDiscovery::ListAll => "list_all".to_string(), + }; + debug!( + base_url = %self.base_url, + discovery = %discovery_kind, + "building MultiFileDataSource" + ); + + let file_paths = match self.discovery { + FileDiscovery::Paths(ref paths) => paths.clone(), + FileDiscovery::Glob(ref pattern) => { + expand_glob(&self.object_store, pattern).await? + } + FileDiscovery::ListAll => { + list_all(&self.object_store, &self.prefix, &self.file_extension).await? + } + }; + + debug!( + base_url = %self.base_url, + file_count = file_paths.len(), + files = ?file_paths, + "discovered files" + ); + + if file_paths.is_empty() { + vortex_bail!( + "MultiFileDataSource requires at least one file (base_url: {})", + self.base_url + ); + } + + let file_count = file_paths.len(); + + let (dtype, inner) = if let Some(ref dtype) = self.dtype { + // DType provided externally — all files can be deferred. + let factories = self.make_factories(&file_paths); + let inner = MultiDataSource::all_deferred(dtype.clone(), factories, &self.session); + (dtype.clone(), inner) + } else { + // Open the first file eagerly to determine the dtype. + let first_path = &file_paths[0]; + debug!(path = %first_path, "opening first file eagerly for dtype"); + let first_options = (self.open_options_fn)(self.session.open_options()); + let first_file = first_options + .open_object_store(&self.object_store, first_path) + .await?; + + let dtype = first_file.dtype().clone(); + debug!(dtype = %dtype, "determined dtype from first file"); + let first_ds = data_source_from_file(&first_file, &self.session)?; + + let factories = self.make_factories(&file_paths[1..]); + let inner = MultiDataSource::lazy(first_ds, factories, &self.session); + (dtype, inner) + }; + + let inner = match self.prefetch { + Some(prefetch) => inner.with_prefetch(prefetch), + None => inner, + }; + + debug!( + base_url = %self.base_url, + file_count, + dtype = %dtype, + "built MultiFileDataSource" + ); + + Ok(MultiFileDataSource::new( + dtype, + inner, + self.base_url, + file_count, + )) + } + .instrument(info_span!("MultiFileDataSourceBuilder::build")) + .await + } + + fn make_factories(&self, paths: &[String]) -> Vec> { + paths + .iter() + .map(|path| { + Arc::new(VortexFileFactory { + object_store: self.object_store.clone(), + path: path.clone(), + filter: self.filter.clone(), + session: self.session.clone(), + open_options_fn: self.open_options_fn.clone(), + }) as Arc + }) + .collect() + } +} diff --git a/vortex-file/src/multi/glob.rs b/vortex-file/src/multi/glob.rs new file mode 100644 index 00000000000..e9e68691af7 --- /dev/null +++ b/vortex-file/src/multi/glob.rs @@ -0,0 +1,208 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! URL glob expansion for discovering Vortex files in object stores. +//! +//! Uses [`object_store::ObjectStore::list()`] with a computed prefix and client-side glob +//! filtering to discover files matching a pattern. + +use std::sync::Arc; + +use futures::StreamExt; +use glob::Pattern; +use object_store::ObjectStore; +use tracing::Instrument; +use tracing::debug; +use tracing::info_span; +use vortex_error::VortexResult; +use vortex_error::vortex_bail; + +/// List all files under `prefix` in the object store that end with `file_extension`, +/// returning sorted file paths relative to the store root. +pub(super) async fn list_all( + object_store: &Arc, + prefix: &object_store::path::Path, + file_extension: &str, +) -> VortexResult> { + let prefix_str = prefix.as_ref(); + async { + debug!(prefix = prefix_str, file_extension, "listing all files"); + + let mut paths: Vec = object_store + .list(Some(prefix)) + .filter_map(|result| async { + match result { + Ok(meta) => { + let path_str = meta.location.to_string(); + path_str.ends_with(file_extension).then_some(path_str) + } + Err(_) => None, + } + }) + .collect() + .await; + + paths.sort(); + debug!( + prefix = prefix_str, + file_extension, + file_count = paths.len(), + "listed all files" + ); + Ok(paths) + } + .instrument(info_span!("list_all", prefix = prefix_str, file_extension)) + .await +} + +/// Expand a glob pattern against an [`ObjectStore`], returning matching file paths relative +/// to the store root. +/// +/// The `glob_pattern` should be a path pattern (not a full URL) relative to the store root, +/// e.g. `"data/year=2024/**/*.vortex"`. +/// +/// # Algorithm +/// +/// 1. Find the first glob character (`*`, `?`, `[`) in the pattern. +/// 2. Use everything before it (up to the last `/`) as the list prefix. +/// 3. List objects with that prefix. +/// 4. Filter using [`glob::Pattern`] matching. +/// 5. Return sorted file paths. +pub(super) async fn expand_glob( + object_store: &Arc, + pattern: &Pattern, +) -> VortexResult> { + let glob_str = pattern.as_str(); + + async { + validate_glob(glob_str)?; + + let prefix = list_prefix(glob_str); + let prefix_path = object_store::path::Path::from(prefix); + + debug!(glob = glob_str, prefix, "expanding glob"); + + let mut paths: Vec = object_store + .list(Some(&prefix_path)) + .filter_map(|result| async { + match result { + Ok(meta) => { + let path_str = meta.location.to_string(); + pattern.matches(&path_str).then_some(path_str) + } + Err(_) => None, + } + }) + .collect() + .await; + + paths.sort(); + debug!(glob = glob_str, file_count = paths.len(), "expanded glob"); + Ok(paths) + } + .instrument(info_span!("expand_glob", glob = glob_str)) + .await +} + +/// Returns the list prefix for a path pattern containing glob characters. +/// +/// The prefix is the directory path up to the first glob character, which is used as the +/// `list()` prefix to narrow the object store listing. +/// +/// # Examples +/// +/// - `"path/to/file_*.txt"` → `"path/to/"` +/// - `"*.txt"` → `""` +/// - `"path/to/specific/file.txt"` → `"path/to/specific/"` +fn list_prefix(pattern: &str) -> &str { + let glob_pos = pattern.find(['*', '?', '[']).unwrap_or(pattern.len()); + + match pattern[..glob_pos].rfind('/') { + Some(slash_pos) => &pattern[..=slash_pos], + None => "", + } +} + +/// Validates that a glob pattern does not contain escaped glob characters. +fn validate_glob(pattern: &str) -> VortexResult<()> { + for escape_pattern in ["\\*", "\\?", "\\["] { + if pattern.contains(escape_pattern) { + vortex_bail!( + "Escaped glob characters are not allowed in patterns. Found '{}' in: {}", + escape_pattern, + pattern + ); + } + } + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_list_prefix_with_wildcard_in_filename() { + assert_eq!(list_prefix("folder/file*.txt"), "folder/"); + } + + #[test] + fn test_list_prefix_with_wildcard_in_directory() { + assert_eq!(list_prefix("folder/*/file.txt"), "folder/"); + } + + #[test] + fn test_list_prefix_nested_directories() { + assert_eq!(list_prefix("data/2023/*/logs/*.log"), "data/2023/"); + } + + #[test] + fn test_list_prefix_wildcard_at_root() { + assert_eq!(list_prefix("*.txt"), ""); + } + + #[test] + fn test_list_prefix_no_wildcards() { + assert_eq!( + list_prefix("folder/subfolder/file.txt"), + "folder/subfolder/" + ); + } + + #[test] + fn test_list_prefix_question_mark() { + assert_eq!(list_prefix("folder/file?.txt"), "folder/"); + } + + #[test] + fn test_list_prefix_bracket() { + assert_eq!(list_prefix("folder/file[abc].txt"), "folder/"); + } + + #[test] + fn test_list_prefix_empty() { + assert_eq!(list_prefix(""), ""); + } + + #[test] + fn test_validate_glob_valid() -> VortexResult<()> { + validate_glob("path/*.txt")?; + validate_glob("path/to/**/*.vortex")?; + Ok(()) + } + + #[test] + fn test_validate_glob_escaped_asterisk() { + assert!(validate_glob("path\\*.txt").is_err()); + } + + #[test] + fn test_validate_glob_escaped_question() { + assert!(validate_glob("path\\?.txt").is_err()); + } + + #[test] + fn test_validate_glob_escaped_bracket() { + assert!(validate_glob("path\\[test].txt").is_err()); + } +} diff --git a/vortex-file/src/multi/mod.rs b/vortex-file/src/multi/mod.rs new file mode 100644 index 00000000000..303551b30f6 --- /dev/null +++ b/vortex-file/src/multi/mod.rs @@ -0,0 +1,22 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Multi-file data source for scanning across multiple Vortex files. +//! +//! This module provides [`MultiFileDataSource`], a reusable, engine-agnostic data source that +//! discovers and opens multiple Vortex files, presenting them as a single [`DataSource`] +//! for scanning. It is analogous to DataFusion's `ListingTable`. +//! +//! Use [`MultiFileDataSourceBuilder`] to construct a `MultiFileDataSource` from an object store +//! and a set of file paths or a glob pattern. +//! +//! [`DataSource`]: vortex_scan::api::DataSource + +mod builder; +mod glob; +mod source; + +pub use builder::FileDiscovery; +pub use builder::MultiFileDataSourceBuilder; +pub use builder::SchemaResolution; +pub use source::MultiFileDataSource; diff --git a/vortex-file/src/multi/source.rs b/vortex-file/src/multi/source.rs new file mode 100644 index 00000000000..75819fb7784 --- /dev/null +++ b/vortex-file/src/multi/source.rs @@ -0,0 +1,144 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! A reusable, engine-agnostic multi-file [`DataSource`] for scanning across multiple Vortex files. +//! +//! [`MultiFileDataSource`] wraps a [`MultiDataSource`] and presents multiple Vortex files as a +//! single scannable data source. It is constructed via [`MultiFileDataSourceBuilder`]. +//! +//! # Future Work +//! +//! - **Hive-style partitioning**: Extract partition values from file paths (e.g. `year=2024/month=01/`) +//! and expose them as virtual columns. +//! - **Virtual columns**: `filename`, `file_row_number`, `file_index`. +//! - **Per-file statistics**: Merge column statistics across files for planner hints. + +use std::sync::Arc; + +use async_trait::async_trait; +use object_store::ObjectStore; +use tracing::debug; +use vortex_array::expr::Expression; +use vortex_dtype::DType; +use vortex_error::VortexResult; +use vortex_scan::api::DataSource; +use vortex_scan::api::DataSourceRef; +use vortex_scan::api::DataSourceScanRef; +use vortex_scan::api::Estimate; +use vortex_scan::api::ScanRequest; +use vortex_scan::api::SplitRef; +use vortex_scan::layout::LayoutReaderDataSource; +use vortex_scan::multi::DataSourceFactory; +use vortex_scan::multi::MultiDataSource; +use vortex_session::VortexSession; + +use crate::OpenOptionsSessionExt; +use crate::VortexFile; +use crate::v2::FileStatsLayoutReader; + +/// A [`DataSource`] that scans across multiple Vortex files, presenting them as a single source. +/// +/// Constructed via [`MultiFileDataSourceBuilder`](super::MultiFileDataSourceBuilder). +/// Internally delegates to [`MultiDataSource`] for scan orchestration, prefetching, and +/// split interleaving. +pub struct MultiFileDataSource { + dtype: DType, + inner: MultiDataSource, + base_url: String, + file_count: usize, +} + +impl MultiFileDataSource { + pub(super) fn new( + dtype: DType, + inner: MultiDataSource, + base_url: String, + file_count: usize, + ) -> Self { + Self { + dtype, + inner, + base_url, + file_count, + } + } + + /// Returns the base URL prefix for files in this data source. + pub fn base_url(&self) -> &str { + &self.base_url + } + + /// Returns the number of files in this data source. + pub fn file_count(&self) -> usize { + self.file_count + } +} + +impl DataSource for MultiFileDataSource { + fn dtype(&self) -> &DType { + &self.dtype + } + + fn row_count_estimate(&self) -> Estimate { + self.inner.row_count_estimate() + } + + fn deserialize_split(&self, data: &[u8], session: &VortexSession) -> VortexResult { + self.inner.deserialize_split(data, session) + } + + fn scan(&self, scan_request: ScanRequest) -> VortexResult { + self.inner.scan(scan_request) + } +} + +/// A [`DataSourceFactory`] that lazily opens a single Vortex file and wraps it in a +/// [`LayoutReaderDataSource`]. +/// +/// Handles statistics-based pruning via [`VortexFile::can_prune()`](crate::VortexFile::can_prune). +pub(super) struct VortexFileFactory { + pub(super) object_store: Arc, + pub(super) path: String, + pub(super) filter: Option, + pub(super) session: VortexSession, + pub(super) open_options_fn: + Arc crate::VortexOpenOptions + Send + Sync>, +} + +#[async_trait] +impl DataSourceFactory for VortexFileFactory { + async fn open(&self) -> VortexResult> { + debug!(path = %self.path, "opening vortex file"); + let options = (self.open_options_fn)(self.session.open_options()); + let file = options + .open_object_store(&self.object_store, &self.path) + .await?; + + if let Some(ref filter) = self.filter + && file.can_prune(filter)? + { + debug!(path = %self.path, "pruned file based on statistics"); + return Ok(None); + } + + let ds = data_source_from_file(&file, &self.session)?; + debug!(path = %self.path, "opened vortex file"); + Ok(Some(ds)) + } +} + +/// Create a [`DataSourceRef`] from a [`VortexFile`], wrapping with +/// [`FileStatsLayoutReader`] when file-level statistics are available. +pub(super) fn data_source_from_file( + file: &VortexFile, + session: &VortexSession, +) -> VortexResult { + let mut reader = file.layout_reader()?; + if let Some(stats) = file.file_stats().cloned() { + reader = Arc::new(FileStatsLayoutReader::new(reader, stats, session.clone())); + } + Ok(Arc::new(LayoutReaderDataSource::new( + reader, + session.clone(), + ))) +} diff --git a/vortex-scan/src/multi.rs b/vortex-scan/src/multi.rs index 6dd8fd58908..6d28aa299f8 100644 --- a/vortex-scan/src/multi.rs +++ b/vortex-scan/src/multi.rs @@ -129,6 +129,26 @@ impl MultiDataSource { } } + /// Creates a multi-source where all children are deferred. + /// + /// The dtype must be provided externally since there is no pre-opened source to infer it + /// from. This avoids eagerly opening any file when the schema is already known (e.g. from + /// a catalog or a prior scan). + pub fn all_deferred( + dtype: DType, + factories: Vec>, + session: &VortexSession, + ) -> Self { + let children = factories.into_iter().map(MultiChild::Deferred).collect(); + + Self { + dtype, + children: Arc::new(Mutex::new(children)), + handle: session.handle(), + prefetch: DEFAULT_PREFETCH, + } + } + /// Sets the number of deferred sources to open concurrently during scanning. /// /// Higher values overlap more file-opening I/O with split execution but use more memory @@ -212,6 +232,7 @@ impl DataSource for MultiDataSource { } } +#[allow(clippy::type_complexity)] struct MultiDataSourceScan { dtype: DType, request: ScanRequest, From 6aa5a0bfd0973f5650af0d2e00c6bcf256d328d0 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 12:57:42 -0500 Subject: [PATCH 24/57] DataFusion streaming Signed-off-by: Nicholas Gates --- vortex-datafusion/src/v2/exec.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/vortex-datafusion/src/v2/exec.rs b/vortex-datafusion/src/v2/exec.rs index f14e61384b8..fcd3a5a8c71 100644 --- a/vortex-datafusion/src/v2/exec.rs +++ b/vortex-datafusion/src/v2/exec.rs @@ -355,7 +355,7 @@ impl ExecutionPlan for VortexExec { &self, projection: &ProjectionExec, ) -> DFResult>> { - tracing::info!( + tracing::debug!( "VortexExec: trying to swap with projection: {:#?} (current: {})", projection, self.projection From 669cedb343d82519be8af2244aba877daf40c949 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 15:06:31 -0500 Subject: [PATCH 25/57] Scan API Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 28 +- vortex-duckdb/src/scan2.rs | 498 ++++++++++++++++++++++++ vortex-file/Cargo.toml | 1 + vortex-file/src/multi/builder.rs | 216 +++++----- vortex-file/src/multi/glob.rs | 145 +++---- vortex-file/src/multi/mod.rs | 1 + vortex-file/src/multi/source.rs | 30 +- 7 files changed, 714 insertions(+), 205 deletions(-) create mode 100644 vortex-duckdb/src/scan2.rs diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index fd02a4b25fd..7c0da3d9fc1 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -26,8 +26,10 @@ use datafusion_physical_plan::collect; use futures::StreamExt; use parking_lot::Mutex; use tokio::fs::File; +use vortex::error::vortex_err; use vortex::file::multi::FileDiscovery; -use vortex::file::multi::MultiFileDataSourceBuilder; +use vortex::file::multi::MultiFileDataSource; +use vortex::file::multi::Pattern; use vortex::scan::api::DataSource as _; use vortex::scan::api::DataSourceRef; use vortex_bench::Benchmark; @@ -295,18 +297,18 @@ async fn register_v2_tables( .runtime_env() .object_store(table_url.object_store())?; - let discovery = pattern - .map(FileDiscovery::Glob) - .unwrap_or(FileDiscovery::ListAll); - let multi_ds = MultiFileDataSourceBuilder::new( - SESSION.clone(), - store.clone(), - benchmark_base.as_str(), - ) - .with_prefix(table_url.prefix().clone()) - .with_discovery(discovery) - .build() - .await?; + let discovery = match pattern { + Some(p) => FileDiscovery::Glob(p), + None => FileDiscovery::Glob( + Pattern::new(&format!("*.{}", format.ext())) + .map_err(|e| vortex_err!("invalid pattern for table {}: {}", table.name, e))?, + ), + }; + let multi_ds = + MultiFileDataSource::builder(SESSION.clone(), store.clone(), benchmark_base.clone()) + .with_discovery(discovery) + .build() + .await?; let arrow_schema = Arc::new(multi_ds.dtype().to_arrow_schema()?); let data_source: DataSourceRef = Arc::new(multi_ds); diff --git a/vortex-duckdb/src/scan2.rs b/vortex-duckdb/src/scan2.rs new file mode 100644 index 00000000000..96a61e37097 --- /dev/null +++ b/vortex-duckdb/src/scan2.rs @@ -0,0 +1,498 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! Scan API implementation of the DuckDB `vortex_scan` table function. +//! +//! Uses [`MultiFileDataSource`] for file discovery and scanning via the Scan API. +//! Enabled by setting `VORTEX_USE_SCAN_API=1`. + +use std::ffi::CString; +use std::fmt; +use std::fmt::Debug; +use std::fmt::Formatter; +use std::sync::Arc; +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; + +use async_compat::Compat; +use custom_labels::CURRENT_LABELSET; +use futures::StreamExt; +use glob::Pattern; +use itertools::Itertools; +use num_traits::AsPrimitive; +use object_store::ObjectStore; +use object_store::local::LocalFileSystem; +use url::Url; +use vortex::VortexSessionDefault; +use vortex::array::ArrayRef; +use vortex::array::Canonical; +use vortex::array::ExecutionCtx; +use vortex::array::arrays::ScalarFnVTable; +use vortex::array::arrays::StructArray; +use vortex::array::arrays::StructVTable; +use vortex::array::optimizer::ArrayOptimizer; +use vortex::dtype::DType; +use vortex::dtype::FieldNames; +use vortex::error::VortexExpect; +use vortex::error::VortexResult; +use vortex::error::vortex_err; +use vortex::expr::Expression; +use vortex::expr::Pack; +use vortex::expr::and_collect; +use vortex::expr::col; +use vortex::expr::root; +use vortex::expr::select; +use vortex::file::multi::FileDiscovery; +use vortex::file::multi::MultiFileDataSource; +use vortex::io::runtime::BlockingRuntime; +use vortex::io::runtime::current::ThreadSafeIterator; +use vortex::metrics::tracing::get_global_labels; +use vortex::scan::api::DataSource as _; +use vortex::scan::api::DataSourceRef; +use vortex::scan::api::ScanRequest; +use vortex::session::VortexSession; +use vortex_utils::aliases::hash_set::HashSet; + +use crate::RUNTIME; +use crate::SESSION; +use crate::convert::try_from_bound_expression; +use crate::convert::try_from_table_filter; +use crate::duckdb; +use crate::duckdb::BindInput; +use crate::duckdb::BindResult; +use crate::duckdb::Cardinality; +use crate::duckdb::ClientContext; +use crate::duckdb::DataChunk; +use crate::duckdb::ExtractedValue; +use crate::duckdb::LogicalType; +use crate::duckdb::TableFunction; +use crate::duckdb::TableInitInput; +use crate::duckdb::VirtualColumnsResult; +use crate::exporter::ArrayExporter; +use crate::exporter::ConversionCache; +use crate::scan::MultiScan; +use crate::utils::object_store::s3_store; + +pub(crate) struct ScanApiBindData { + data_source: DataSourceRef, + filter_exprs: Vec, + column_names: Vec, + column_types: Vec, + max_threads: u64, + file_count: usize, + row_count_estimate: u64, +} + +impl Clone for ScanApiBindData { + fn clone(&self) -> Self { + Self { + data_source: self.data_source.clone(), + // filter_exprs are consumed once in `init_global`. + filter_exprs: vec![], + column_names: self.column_names.clone(), + column_types: self.column_types.clone(), + max_threads: self.max_threads, + file_count: self.file_count, + row_count_estimate: self.row_count_estimate, + } + } +} + +impl Debug for ScanApiBindData { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + f.debug_struct("ScanApiBindData") + .field("column_names", &self.column_names) + .field("column_types", &self.column_types) + .field("filter_exprs", &self.filter_exprs) + .field("file_count", &self.file_count) + .finish() + } +} + +pub(crate) struct ScanApiGlobalData { + iterator: ThreadSafeIterator)>>, + batch_id: AtomicU64, + ctx: ExecutionCtx, +} + +pub(crate) struct ScanApiLocalData { + iterator: ThreadSafeIterator)>>, + exporter: Option, + batch_id: Option, +} + +#[derive(Debug)] +pub(crate) struct ScanApiTableFunction; + +/// Parse a URL glob string into an object store, base URL, and file discovery. +/// +/// Supports S3 (`s3://bucket/path/*.vortex`) and local (`/path/*.vortex`) URLs. +fn parse_url_glob(url_glob: &str) -> VortexResult<(Arc, Url, FileDiscovery)> { + let is_s3 = url_glob.starts_with("s3://"); + + // Find the first glob character to split base URL from pattern. + let first_glob = url_glob.find(['*', '?', '[']); + + // Split at the last '/' before the first glob character. + let split_before = first_glob.unwrap_or(url_glob.len()); + let split_pos = url_glob[..split_before] + .rfind('/') + .map(|i| i + 1) + .unwrap_or(0); + let base_url_str = &url_glob[..split_pos]; + let pattern_str = &url_glob[split_pos..]; + + let pattern = Pattern::new(pattern_str) + .map_err(|e| vortex_err!("Invalid glob pattern '{}': {}", pattern_str, e))?; + + if is_s3 { + let base_url = Url::parse(base_url_str) + .map_err(|e| vortex_err!("Invalid S3 URL '{}': {}", base_url_str, e))?; + let bucket = base_url + .host_str() + .ok_or_else(|| vortex_err!("Missing bucket in S3 URL: {}", base_url_str))?; + Ok((s3_store(bucket)?, base_url, FileDiscovery::Glob(pattern))) + } else { + let path_str = base_url_str.strip_prefix("file://").unwrap_or(base_url_str); + let canonical = std::fs::canonicalize(path_str) + .map_err(|e| vortex_err!("Failed to resolve path '{}': {}", path_str, e))?; + let base_url = Url::from_directory_path(&canonical) + .map_err(|_| vortex_err!("Invalid directory path: {}", canonical.display()))?; + let store: Arc = Arc::new( + LocalFileSystem::new_with_prefix("/") + .map_err(|e| vortex_err!("Failed to create local filesystem: {}", e))?, + ); + Ok((store, base_url, FileDiscovery::Glob(pattern))) + } +} + +/// Extract column names and DuckDB types from a Vortex [`DType`]. +fn extract_schema_from_dtype(dtype: &DType) -> VortexResult<(Vec, Vec)> { + let struct_dtype = dtype + .as_struct_fields_opt() + .ok_or_else(|| vortex_err!("Vortex file must contain a struct array at the top level"))?; + + let mut column_names = Vec::new(); + let mut column_types = Vec::new(); + + for (field_name, field_dtype) in struct_dtype.names().iter().zip(struct_dtype.fields()) { + let logical_type = LogicalType::try_from(&field_dtype)?; + column_names.push(field_name.to_string()); + column_types.push(logical_type); + } + + Ok((column_names, column_types)) +} + +/// Creates a projection expression from the table initialization input. +fn extract_projection_expr(init: &TableInitInput) -> Expression { + let projection_ids = init.projection_ids().unwrap_or(&[]); + let column_ids = init.column_ids(); + + select( + projection_ids + .iter() + .map(|p| { + let idx: usize = p.as_(); + let val: usize = column_ids[idx].as_(); + val + }) + .map(|idx| { + init.bind_data() + .column_names + .get(idx) + .vortex_expect("prune idx in column names") + }) + .map(|s| Arc::from(s.as_str())) + .collect::(), + root(), + ) +} + +/// Creates a filter expression from the table filter set. +fn extract_table_filter_expr( + init: &TableInitInput, + column_ids: &[u64], +) -> VortexResult> { + let mut table_filter_exprs: HashSet = if let Some(filter) = init.table_filter_set() + { + filter + .into_iter() + .map(|(idx, ex)| { + let idx_u: usize = idx.as_(); + let col_idx: usize = column_ids[idx_u].as_(); + let name = init + .bind_data() + .column_names + .get(col_idx) + .vortex_expect("exists"); + try_from_table_filter( + &ex, + &col(name.as_str()), + init.bind_data().data_source.dtype(), + ) + }) + .collect::>>>()? + .unwrap_or_else(HashSet::new) + } else { + HashSet::new() + }; + + table_filter_exprs.extend(init.bind_data().filter_exprs.clone()); + Ok(and_collect(table_filter_exprs.into_iter().collect_vec())) +} + +// Taken from duckdb/common/constants.h COLUMN_IDENTIFIER_EMPTY +static EMPTY_COLUMN_IDX: u64 = 18446744073709551614; +static EMPTY_COLUMN_NAME: &str = ""; + +impl TableFunction for ScanApiTableFunction { + type BindData = ScanApiBindData; + type GlobalState = ScanApiGlobalData; + type LocalState = ScanApiLocalData; + + const PROJECTION_PUSHDOWN: bool = true; + const FILTER_PUSHDOWN: bool = true; + const FILTER_PRUNE: bool = true; + + fn parameters() -> Vec { + vec![LogicalType::varchar()] + } + + fn bind( + ctx: &ClientContext, + input: &BindInput, + result: &mut BindResult, + ) -> VortexResult { + let file_glob_string = input + .get_parameter(0) + .ok_or_else(|| vortex_err!("Missing file glob parameter"))?; + + let max_threads_cstr = CString::new("vortex_max_threads") + .map_err(|e| vortex_err!("Invalid setting name: {}", e))?; + let max_threads = ctx + .try_get_current_setting(&max_threads_cstr) + .and_then(|v| match v.as_ref().extract() { + ExtractedValue::UBigInt(val) => usize::try_from(val).ok(), + ExtractedValue::BigInt(val) if val > 0 => usize::try_from(val as u64).ok(), + _ => None, + }) + .unwrap_or_else(|| { + std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1) + }); + + tracing::trace!("running scan with max_threads {max_threads}"); + + let (object_store, base_url, discovery) = + parse_url_glob(&file_glob_string.as_ref().as_string())?; + + let multi_ds = RUNTIME.block_on(Compat::new( + MultiFileDataSource::builder(SESSION.clone(), object_store, base_url) + .with_discovery(discovery) + .with_prefetch(max_threads * 2) + .build(), + ))?; + + let (column_names, column_types) = extract_schema_from_dtype(multi_ds.dtype())?; + + for (column_name, column_type) in column_names.iter().zip(&column_types) { + result.add_result_column(column_name, column_type); + } + + let file_count = multi_ds.file_count(); + let estimate = multi_ds.row_count_estimate(); + let row_count_estimate = estimate.upper.unwrap_or(estimate.lower); + let data_source: DataSourceRef = Arc::new(multi_ds); + + Ok(ScanApiBindData { + data_source, + filter_exprs: vec![], + column_names, + column_types, + max_threads: max_threads as u64, + file_count, + row_count_estimate, + }) + } + + fn scan( + _client_context: &ClientContext, + _bind_data: &Self::BindData, + local_state: &mut Self::LocalState, + global_state: &mut Self::GlobalState, + chunk: &mut DataChunk, + ) -> VortexResult<()> { + loop { + if local_state.exporter.is_none() { + let Some(result) = local_state.iterator.next() else { + return Ok(()); + }; + + let (array_result, conversion_cache) = result?; + + let array_result = array_result.optimize_recursive()?; + let array_result = if let Some(array) = array_result.as_opt::() { + array.clone() + } else if let Some(array) = array_result.as_opt::() + && let Some(pack_options) = array.scalar_fn().as_opt::() + { + StructArray::new( + pack_options.names.clone(), + array.children(), + array.len(), + pack_options.nullability.into(), + ) + } else { + array_result + .execute::(&mut global_state.ctx)? + .into_struct() + }; + + local_state.exporter = Some(ArrayExporter::try_new( + &array_result, + &conversion_cache, + &mut global_state.ctx, + )?); + local_state.batch_id = Some(global_state.batch_id.fetch_add(1, Ordering::Relaxed)); + } + + let exporter = local_state + .exporter + .as_mut() + .vortex_expect("error: exporter missing"); + + let has_more_data = exporter.export(chunk)?; + + if !has_more_data { + local_state.exporter = None; + local_state.batch_id = None; + } else { + break; + } + } + + assert!(!chunk.is_empty()); + + Ok(()) + } + + fn init_global(init_input: &TableInitInput) -> VortexResult { + let bind_data = init_input.bind_data(); + let projection_expr = extract_projection_expr(init_input); + let filter_expr = extract_table_filter_expr(init_input, init_input.column_ids())?; + + tracing::trace!( + "Global init Vortex scan (v2) SELECT {} WHERE {}", + &projection_expr, + filter_expr + .as_ref() + .map_or_else(|| "true".to_string(), |f| f.to_string()) + ); + + #[expect(clippy::cast_possible_truncation, reason = "max_threads fits in usize")] + let num_workers = bind_data.max_threads as usize; + + let request = ScanRequest { + projection: Some(projection_expr), + filter: filter_expr, + ..Default::default() + }; + + let scan = bind_data.data_source.scan(request)?; + let conversion_cache = Arc::new(ConversionCache::new(0)); + + let scan_streams = scan.splits().then(move |split_result| { + let cache = conversion_cache.clone(); + async move { + let split = split_result?; + let s = split.execute().await?; + Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) + } + }); + + let iterator = RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { + streams: scan_streams.boxed(), + streams_finished: false, + select_all: Default::default(), + max_concurrency: num_workers * 2, + }); + + Ok(ScanApiGlobalData { + iterator, + batch_id: AtomicU64::new(0), + ctx: ExecutionCtx::new(VortexSession::default()), + }) + } + + fn init_local( + _init: &TableInitInput, + global: &mut Self::GlobalState, + ) -> VortexResult { + unsafe { + use custom_labels::sys; + + if sys::labelset_current().is_null() { + let ls = sys::labelset_new(0); + sys::labelset_replace(ls); + }; + } + + let global_labels = get_global_labels(); + + for (key, value) in global_labels { + CURRENT_LABELSET.set(key, value); + } + + Ok(ScanApiLocalData { + iterator: global.iterator.clone(), + exporter: None, + batch_id: None, + }) + } + + fn pushdown_complex_filter( + bind_data: &mut Self::BindData, + expr: &duckdb::Expression, + ) -> VortexResult { + let Some(expr) = try_from_bound_expression(expr)? else { + return Ok(false); + }; + bind_data.filter_exprs.push(expr); + Ok(false) + } + + fn cardinality(bind_data: &Self::BindData) -> Cardinality { + if bind_data.file_count == 1 { + Cardinality::Maximum(bind_data.row_count_estimate) + } else { + Cardinality::Estimate(bind_data.row_count_estimate) + } + } + + fn partition_data( + _bind_data: &Self::BindData, + _global_init_data: &mut Self::GlobalState, + local_init_data: &mut Self::LocalState, + ) -> VortexResult { + local_init_data + .batch_id + .ok_or_else(|| vortex_err!("batch id missing, no batches exported")) + } + + fn to_string(bind_data: &Self::BindData) -> Option> { + let mut result = Vec::new(); + result.push(("Function".to_string(), "Vortex Scan (v2)".to_string())); + result.push(("Files".to_string(), bind_data.file_count.to_string())); + if !bind_data.filter_exprs.is_empty() { + let mut filters = bind_data.filter_exprs.iter().map(|f| format!("{}", f)); + result.push(("Filters".to_string(), filters.join(" /\\\n"))); + } + Some(result) + } + + fn virtual_columns(_bind_data: &Self::BindData, result: &mut VirtualColumnsResult) { + result.register(EMPTY_COLUMN_IDX, EMPTY_COLUMN_NAME, &LogicalType::bool()); + } +} diff --git a/vortex-file/Cargo.toml b/vortex-file/Cargo.toml index 72555eb5e43..1711021d9a0 100644 --- a/vortex-file/Cargo.toml +++ b/vortex-file/Cargo.toml @@ -32,6 +32,7 @@ pin-project-lite = { workspace = true } tokio = { workspace = true, features = ["rt"], optional = true } tracing = { workspace = true } uuid = { workspace = true } # Needed to pickup the "js" feature for wasm targets from the workspace configuration +url = { workspace = true } vortex-alp = { workspace = true } vortex-array = { workspace = true } vortex-btrblocks = { workspace = true } diff --git a/vortex-file/src/multi/builder.rs b/vortex-file/src/multi/builder.rs index ae4d24fabf1..683dacbca19 100644 --- a/vortex-file/src/multi/builder.rs +++ b/vortex-file/src/multi/builder.rs @@ -8,19 +8,20 @@ use std::sync::Arc; use glob::Pattern; use object_store::ObjectStore; use object_store::path::Path; -use tracing::Instrument; use tracing::debug; -use tracing::info_span; +use url::Url; 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_scan::multi::DataSourceFactory; use vortex_scan::multi::MultiDataSource; use vortex_session::VortexSession; use super::glob::expand_glob; use super::glob::list_all; +use super::source::DiscoveredFile; use super::source::MultiFileDataSource; use super::source::VortexFileFactory; use super::source::data_source_from_file; @@ -40,6 +41,7 @@ pub enum SchemaResolution { } /// How files are discovered for a [`MultiFileDataSource`]. +#[derive(Debug)] pub enum FileDiscovery { /// Explicit list of file paths (relative to the object store root). Paths(Vec), @@ -79,11 +81,7 @@ pub enum FileDiscovery { pub struct MultiFileDataSourceBuilder { session: VortexSession, object_store: Arc, - base_url: String, - /// The path prefix within the object store used for [`FileDiscovery::ListAll`]. - prefix: Path, - /// File extension filter for [`FileDiscovery::ListAll`] (e.g. `".vortex"`). - file_extension: String, + base_url: Url, discovery: FileDiscovery, schema_resolution: SchemaResolution, open_options_fn: Arc VortexOpenOptions + Send + Sync>, @@ -92,22 +90,20 @@ pub struct MultiFileDataSourceBuilder { dtype: Option, } -impl MultiFileDataSourceBuilder { +impl MultiFileDataSource { /// Create a new builder from an object store and base URL prefix. /// /// The `base_url` is used for display/debug purposes. It should typically match the /// location of the files (e.g. `"s3://bucket/data/"`). - pub fn new( + pub fn builder( session: VortexSession, object_store: Arc, - base_url: impl Into, - ) -> Self { - Self { + base_url: Url, + ) -> MultiFileDataSourceBuilder { + MultiFileDataSourceBuilder { session, object_store, - base_url: base_url.into(), - prefix: Path::from(""), - file_extension: ".vortex".to_string(), + base_url, discovery: FileDiscovery::ListAll, schema_resolution: SchemaResolution::default(), open_options_fn: Arc::new(|opts| opts), @@ -116,25 +112,9 @@ impl MultiFileDataSourceBuilder { dtype: None, } } +} - /// Set the path prefix within the object store. - /// - /// This prefix is used by [`FileDiscovery::ListAll`] to scope file listing to a - /// subdirectory of the object store. For example, if the object store represents - /// `file:///` and the data lives at `/data/tables/`, set the prefix to `data/tables`. - pub fn with_prefix(mut self, prefix: impl Into) -> Self { - self.prefix = prefix.into(); - self - } - - /// Set the file extension filter for [`FileDiscovery::ListAll`]. - /// - /// Only files ending with this extension will be included. Defaults to `".vortex"`. - pub fn with_file_extension(mut self, ext: impl Into) -> Self { - self.file_extension = ext.into(); - self - } - +impl MultiFileDataSourceBuilder { /// Set how files are discovered. pub fn with_discovery(mut self, discovery: FileDiscovery) -> Self { self.discovery = discovery; @@ -211,102 +191,114 @@ impl MultiFileDataSourceBuilder { /// eagerly against the object store. If a [`DType`] was provided via /// [`with_dtype`](Self::with_dtype), all files are opened lazily during scanning. /// Otherwise, the first file is opened eagerly to determine the schema. + #[tracing::instrument(name = "MultiFileDataSourceBuilder::build", skip(self))] pub async fn build(self) -> VortexResult { - async { - if matches!(self.schema_resolution, SchemaResolution::Union) { - vortex_bail!("SchemaResolution::Union is not yet implemented"); - } + if matches!(self.schema_resolution, SchemaResolution::Union) { + vortex_bail!("SchemaResolution::Union is not yet implemented"); + } - let discovery_kind = match &self.discovery { - FileDiscovery::Paths(p) => format!("paths({})", p.len()), - FileDiscovery::Glob(g) => format!("glob({})", g.as_str()), - FileDiscovery::ListAll => "list_all".to_string(), - }; - debug!( - base_url = %self.base_url, - discovery = %discovery_kind, - "building MultiFileDataSource" - ); + let discovery_kind = match &self.discovery { + FileDiscovery::Paths(p) => format!("paths({})", p.len()), + FileDiscovery::Glob(g) => format!("glob({})", g.as_str()), + FileDiscovery::ListAll => "list_all".to_string(), + }; + debug!( + base_url = %self.base_url, + discovery = %discovery_kind, + "building MultiFileDataSource" + ); - let file_paths = match self.discovery { - FileDiscovery::Paths(ref paths) => paths.clone(), - FileDiscovery::Glob(ref pattern) => { - expand_glob(&self.object_store, pattern).await? - } - FileDiscovery::ListAll => { - list_all(&self.object_store, &self.prefix, &self.file_extension).await? - } - }; + debug!( + "Discovering files in {}: {:?}", + self.base_url, self.discovery + ); + let base_url_path = Path::from_url_path(self.base_url.path()) + .map_err(|e| vortex_err!("Invalid base_url '{}': {}", self.base_url, e))?; + let files = match self.discovery { + FileDiscovery::Paths(ref paths) => paths + .iter() + .map(|path| { + // FIXME(ngates): join path to the base_url_path. + DiscoveredFile { + path: path.clone(), + size: None, + } + }) + .collect(), + FileDiscovery::Glob(ref pattern) => { + expand_glob(&self.object_store, &base_url_path, pattern).await? + } + FileDiscovery::ListAll => list_all(&self.object_store, &base_url_path).await?, + }; - debug!( - base_url = %self.base_url, - file_count = file_paths.len(), - files = ?file_paths, - "discovered files" - ); + debug!( + base_url = %self.base_url, + file_count = files.len(), + files = ?files, + "discovered files" + ); - if file_paths.is_empty() { - vortex_bail!( - "MultiFileDataSource requires at least one file (base_url: {})", - self.base_url - ); - } + if files.is_empty() { + vortex_bail!( + "MultiFileDataSource requires at least one file (base_url: {})", + self.base_url + ); + } - let file_count = file_paths.len(); + let file_count = files.len(); - let (dtype, inner) = if let Some(ref dtype) = self.dtype { - // DType provided externally — all files can be deferred. - let factories = self.make_factories(&file_paths); - let inner = MultiDataSource::all_deferred(dtype.clone(), factories, &self.session); - (dtype.clone(), inner) - } else { - // Open the first file eagerly to determine the dtype. - let first_path = &file_paths[0]; - debug!(path = %first_path, "opening first file eagerly for dtype"); - let first_options = (self.open_options_fn)(self.session.open_options()); - let first_file = first_options - .open_object_store(&self.object_store, first_path) - .await?; + let (dtype, inner) = if let Some(ref dtype) = self.dtype { + // DType provided externally — all files can be deferred. + let factories = self.make_factories(&files); + let inner = MultiDataSource::all_deferred(dtype.clone(), factories, &self.session); + (dtype.clone(), inner) + } else { + let first = &files[0]; + debug!(path = %first.path, "opening first file eagerly for dtype"); + let mut first_options = (self.open_options_fn)(self.session.open_options()); + if let Some(size) = first.size { + first_options = first_options.with_file_size(size); + } + let first_file = first_options + .open_object_store(&self.object_store, &first.path) + .await?; - let dtype = first_file.dtype().clone(); - debug!(dtype = %dtype, "determined dtype from first file"); - let first_ds = data_source_from_file(&first_file, &self.session)?; + let dtype = first_file.dtype().clone(); + debug!(dtype = %dtype, "determined dtype from first file"); + let first_ds = data_source_from_file(&first_file, &self.session)?; - let factories = self.make_factories(&file_paths[1..]); - let inner = MultiDataSource::lazy(first_ds, factories, &self.session); - (dtype, inner) - }; + let factories = self.make_factories(&files[1..]); + let inner = MultiDataSource::lazy(first_ds, factories, &self.session); + (dtype, inner) + }; - let inner = match self.prefetch { - Some(prefetch) => inner.with_prefetch(prefetch), - None => inner, - }; + let inner = match self.prefetch { + Some(prefetch) => inner.with_prefetch(prefetch), + None => inner, + }; - debug!( - base_url = %self.base_url, - file_count, - dtype = %dtype, - "built MultiFileDataSource" - ); + debug!( + base_url = %self.base_url, + file_count, + dtype = %dtype, + "built MultiFileDataSource" + ); - Ok(MultiFileDataSource::new( - dtype, - inner, - self.base_url, - file_count, - )) - } - .instrument(info_span!("MultiFileDataSourceBuilder::build")) - .await + Ok(MultiFileDataSource::new( + dtype, + inner, + self.base_url.to_string(), + file_count, + )) } - fn make_factories(&self, paths: &[String]) -> Vec> { - paths + fn make_factories(&self, files: &[DiscoveredFile]) -> Vec> { + files .iter() - .map(|path| { + .map(|file| { Arc::new(VortexFileFactory { object_store: self.object_store.clone(), - path: path.clone(), + file: file.clone(), filter: self.filter.clone(), session: self.session.clone(), open_options_fn: self.open_options_fn.clone(), diff --git a/vortex-file/src/multi/glob.rs b/vortex-file/src/multi/glob.rs index e9e68691af7..15a7bf11933 100644 --- a/vortex-file/src/multi/glob.rs +++ b/vortex-file/src/multi/glob.rs @@ -9,54 +9,41 @@ use std::sync::Arc; use futures::StreamExt; +use futures::TryStreamExt; use glob::Pattern; use object_store::ObjectStore; -use tracing::Instrument; +use object_store::path::Path; use tracing::debug; -use tracing::info_span; use vortex_error::VortexResult; use vortex_error::vortex_bail; -/// List all files under `prefix` in the object store that end with `file_extension`, -/// returning sorted file paths relative to the store root. +use super::source::DiscoveredFile; + +/// List all files under `prefix` in the object store, returning sorted +/// [`DiscoveredFile`]s with path and size. +#[tracing::instrument(name = "list_all", skip(object_store))] pub(super) async fn list_all( object_store: &Arc, - prefix: &object_store::path::Path, - file_extension: &str, -) -> VortexResult> { - let prefix_str = prefix.as_ref(); - async { - debug!(prefix = prefix_str, file_extension, "listing all files"); - - let mut paths: Vec = object_store - .list(Some(prefix)) - .filter_map(|result| async { - match result { - Ok(meta) => { - let path_str = meta.location.to_string(); - path_str.ends_with(file_extension).then_some(path_str) - } - Err(_) => None, - } - }) - .collect() - .await; - - paths.sort(); - debug!( - prefix = prefix_str, - file_extension, - file_count = paths.len(), - "listed all files" - ); - Ok(paths) - } - .instrument(info_span!("list_all", prefix = prefix_str, file_extension)) - .await + prefix: &Path, +) -> VortexResult> { + debug!("listing all files"); + + let mut files: Vec = object_store + .list(Some(prefix)) + .map_ok(|meta| DiscoveredFile { + path: meta.location.to_string(), + size: Some(meta.size), + }) + .try_collect() + .await?; + + files.sort(); + debug!(file_count = files.len(), "listed all files"); + Ok(files) } -/// Expand a glob pattern against an [`ObjectStore`], returning matching file paths relative -/// to the store root. +/// Expand a glob pattern against an [`ObjectStore`], returning matching +/// [`DiscoveredFile`]s with path and size. /// /// The `glob_pattern` should be a path pattern (not a full URL) relative to the store root, /// e.g. `"data/year=2024/**/*.vortex"`. @@ -68,52 +55,68 @@ pub(super) async fn list_all( /// 3. List objects with that prefix. /// 4. Filter using [`glob::Pattern`] matching. /// 5. Return sorted file paths. +#[tracing::instrument(name = "expand_glob", skip(object_store))] pub(super) async fn expand_glob( object_store: &Arc, + base_url_path: &Path, pattern: &Pattern, -) -> VortexResult> { +) -> VortexResult> { let glob_str = pattern.as_str(); - async { - validate_glob(glob_str)?; - - let prefix = list_prefix(glob_str); - let prefix_path = object_store::path::Path::from(prefix); - - debug!(glob = glob_str, prefix, "expanding glob"); - - let mut paths: Vec = object_store - .list(Some(&prefix_path)) - .filter_map(|result| async { - match result { - Ok(meta) => { - let path_str = meta.location.to_string(); - pattern.matches(&path_str).then_some(path_str) - } - Err(_) => None, + validate_glob(glob_str)?; + + // Extract the static prefix from the glob pattern to narrow the listing. + let prefix = list_prefix(glob_str); + let listing_path = if prefix.is_empty() { + base_url_path.clone() + } else { + Path::from(format!( + "{}/{}", + base_url_path.as_ref().trim_end_matches('/'), + prefix.trim_end_matches('/') + )) + }; + let base_prefix = base_url_path.as_ref(); + + debug!(%base_url_path, %listing_path, "expanding glob"); + + let mut files: Vec = object_store + .list(Some(&listing_path)) + .filter_map(|result| async { + match result { + Ok(meta) => { + let path_str = meta.location.to_string(); + let relative = path_str + .strip_prefix(base_prefix) + .map(|s| s.trim_start_matches('/')) + .unwrap_or(&path_str); + pattern.matches(relative).then_some(DiscoveredFile { + path: path_str, + size: Some(meta.size), + }) } - }) - .collect() - .await; - - paths.sort(); - debug!(glob = glob_str, file_count = paths.len(), "expanded glob"); - Ok(paths) - } - .instrument(info_span!("expand_glob", glob = glob_str)) - .await + // FIXME(ngates): do not ignore errors + Err(_) => None, + } + }) + .collect() + .await; + + files.sort(); + debug!(file_count = files.len(), "expanded glob"); + Ok(files) } /// Returns the list prefix for a path pattern containing glob characters. /// -/// The prefix is the directory path up to the first glob character, which is used as the -/// `list()` prefix to narrow the object store listing. +/// The prefix is the directory path up to the first glob character, which is used to narrow +/// the `list()` call on the object store. /// /// # Examples /// -/// - `"path/to/file_*.txt"` → `"path/to/"` -/// - `"*.txt"` → `""` -/// - `"path/to/specific/file.txt"` → `"path/to/specific/"` +/// - `"path/to/file_*.txt"` -> `"path/to/"` +/// - `"*.txt"` -> `""` +/// - `"path/to/specific/file.txt"` -> `"path/to/specific/"` fn list_prefix(pattern: &str) -> &str { let glob_pos = pattern.find(['*', '?', '[']).unwrap_or(pattern.len()); diff --git a/vortex-file/src/multi/mod.rs b/vortex-file/src/multi/mod.rs index 303551b30f6..979348a1c3d 100644 --- a/vortex-file/src/multi/mod.rs +++ b/vortex-file/src/multi/mod.rs @@ -16,6 +16,7 @@ mod builder; mod glob; mod source; +pub use ::glob::Pattern; pub use builder::FileDiscovery; pub use builder::MultiFileDataSourceBuilder; pub use builder::SchemaResolution; diff --git a/vortex-file/src/multi/source.rs b/vortex-file/src/multi/source.rs index 75819fb7784..b6aa0314ad1 100644 --- a/vortex-file/src/multi/source.rs +++ b/vortex-file/src/multi/source.rs @@ -34,6 +34,7 @@ use vortex_session::VortexSession; use crate::OpenOptionsSessionExt; use crate::VortexFile; +use crate::VortexOpenOptions; use crate::v2::FileStatsLayoutReader; /// A [`DataSource`] that scans across multiple Vortex files, presenting them as a single source. @@ -95,34 +96,36 @@ impl DataSource for MultiFileDataSource { /// A [`DataSourceFactory`] that lazily opens a single Vortex file and wraps it in a /// [`LayoutReaderDataSource`]. /// -/// Handles statistics-based pruning via [`VortexFile::can_prune()`](crate::VortexFile::can_prune). +/// Handles statistics-based pruning via [`VortexFile::can_prune`]. pub(super) struct VortexFileFactory { pub(super) object_store: Arc, - pub(super) path: String, + pub(super) file: DiscoveredFile, pub(super) filter: Option, pub(super) session: VortexSession, - pub(super) open_options_fn: - Arc crate::VortexOpenOptions + Send + Sync>, + pub(super) open_options_fn: Arc VortexOpenOptions + Send + Sync>, } #[async_trait] impl DataSourceFactory for VortexFileFactory { async fn open(&self) -> VortexResult> { - debug!(path = %self.path, "opening vortex file"); - let options = (self.open_options_fn)(self.session.open_options()); + debug!(path = %self.file.path, "opening vortex file"); + let mut options = (self.open_options_fn)(self.session.open_options()); + if let Some(size) = self.file.size { + options = options.with_file_size(size); + } let file = options - .open_object_store(&self.object_store, &self.path) + .open_object_store(&self.object_store, &self.file.path) .await?; if let Some(ref filter) = self.filter && file.can_prune(filter)? { - debug!(path = %self.path, "pruned file based on statistics"); + debug!(path = %self.file.path, "pruned file based on statistics"); return Ok(None); } let ds = data_source_from_file(&file, &self.session)?; - debug!(path = %self.path, "opened vortex file"); + debug!(path = %self.file.path, "opened vortex file"); Ok(Some(ds)) } } @@ -142,3 +145,12 @@ pub(super) fn data_source_from_file( session.clone(), ))) } + +/// A file discovered during file listing, with its path and size in bytes. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] +pub(super) struct DiscoveredFile { + /// The file path relative to the object store root. + pub path: String, + /// The file size in bytes, if known. + pub size: Option, +} From 392f2638decd8f6213e2f98678eea56e1d7f582e Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 15:06:53 -0500 Subject: [PATCH 26/57] Scan API Signed-off-by: Nicholas Gates --- vortex-duckdb/src/scan2.rs | 498 ------------------------------------- 1 file changed, 498 deletions(-) delete mode 100644 vortex-duckdb/src/scan2.rs diff --git a/vortex-duckdb/src/scan2.rs b/vortex-duckdb/src/scan2.rs deleted file mode 100644 index 96a61e37097..00000000000 --- a/vortex-duckdb/src/scan2.rs +++ /dev/null @@ -1,498 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright the Vortex contributors - -//! Scan API implementation of the DuckDB `vortex_scan` table function. -//! -//! Uses [`MultiFileDataSource`] for file discovery and scanning via the Scan API. -//! Enabled by setting `VORTEX_USE_SCAN_API=1`. - -use std::ffi::CString; -use std::fmt; -use std::fmt::Debug; -use std::fmt::Formatter; -use std::sync::Arc; -use std::sync::atomic::AtomicU64; -use std::sync::atomic::Ordering; - -use async_compat::Compat; -use custom_labels::CURRENT_LABELSET; -use futures::StreamExt; -use glob::Pattern; -use itertools::Itertools; -use num_traits::AsPrimitive; -use object_store::ObjectStore; -use object_store::local::LocalFileSystem; -use url::Url; -use vortex::VortexSessionDefault; -use vortex::array::ArrayRef; -use vortex::array::Canonical; -use vortex::array::ExecutionCtx; -use vortex::array::arrays::ScalarFnVTable; -use vortex::array::arrays::StructArray; -use vortex::array::arrays::StructVTable; -use vortex::array::optimizer::ArrayOptimizer; -use vortex::dtype::DType; -use vortex::dtype::FieldNames; -use vortex::error::VortexExpect; -use vortex::error::VortexResult; -use vortex::error::vortex_err; -use vortex::expr::Expression; -use vortex::expr::Pack; -use vortex::expr::and_collect; -use vortex::expr::col; -use vortex::expr::root; -use vortex::expr::select; -use vortex::file::multi::FileDiscovery; -use vortex::file::multi::MultiFileDataSource; -use vortex::io::runtime::BlockingRuntime; -use vortex::io::runtime::current::ThreadSafeIterator; -use vortex::metrics::tracing::get_global_labels; -use vortex::scan::api::DataSource as _; -use vortex::scan::api::DataSourceRef; -use vortex::scan::api::ScanRequest; -use vortex::session::VortexSession; -use vortex_utils::aliases::hash_set::HashSet; - -use crate::RUNTIME; -use crate::SESSION; -use crate::convert::try_from_bound_expression; -use crate::convert::try_from_table_filter; -use crate::duckdb; -use crate::duckdb::BindInput; -use crate::duckdb::BindResult; -use crate::duckdb::Cardinality; -use crate::duckdb::ClientContext; -use crate::duckdb::DataChunk; -use crate::duckdb::ExtractedValue; -use crate::duckdb::LogicalType; -use crate::duckdb::TableFunction; -use crate::duckdb::TableInitInput; -use crate::duckdb::VirtualColumnsResult; -use crate::exporter::ArrayExporter; -use crate::exporter::ConversionCache; -use crate::scan::MultiScan; -use crate::utils::object_store::s3_store; - -pub(crate) struct ScanApiBindData { - data_source: DataSourceRef, - filter_exprs: Vec, - column_names: Vec, - column_types: Vec, - max_threads: u64, - file_count: usize, - row_count_estimate: u64, -} - -impl Clone for ScanApiBindData { - fn clone(&self) -> Self { - Self { - data_source: self.data_source.clone(), - // filter_exprs are consumed once in `init_global`. - filter_exprs: vec![], - column_names: self.column_names.clone(), - column_types: self.column_types.clone(), - max_threads: self.max_threads, - file_count: self.file_count, - row_count_estimate: self.row_count_estimate, - } - } -} - -impl Debug for ScanApiBindData { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - f.debug_struct("ScanApiBindData") - .field("column_names", &self.column_names) - .field("column_types", &self.column_types) - .field("filter_exprs", &self.filter_exprs) - .field("file_count", &self.file_count) - .finish() - } -} - -pub(crate) struct ScanApiGlobalData { - iterator: ThreadSafeIterator)>>, - batch_id: AtomicU64, - ctx: ExecutionCtx, -} - -pub(crate) struct ScanApiLocalData { - iterator: ThreadSafeIterator)>>, - exporter: Option, - batch_id: Option, -} - -#[derive(Debug)] -pub(crate) struct ScanApiTableFunction; - -/// Parse a URL glob string into an object store, base URL, and file discovery. -/// -/// Supports S3 (`s3://bucket/path/*.vortex`) and local (`/path/*.vortex`) URLs. -fn parse_url_glob(url_glob: &str) -> VortexResult<(Arc, Url, FileDiscovery)> { - let is_s3 = url_glob.starts_with("s3://"); - - // Find the first glob character to split base URL from pattern. - let first_glob = url_glob.find(['*', '?', '[']); - - // Split at the last '/' before the first glob character. - let split_before = first_glob.unwrap_or(url_glob.len()); - let split_pos = url_glob[..split_before] - .rfind('/') - .map(|i| i + 1) - .unwrap_or(0); - let base_url_str = &url_glob[..split_pos]; - let pattern_str = &url_glob[split_pos..]; - - let pattern = Pattern::new(pattern_str) - .map_err(|e| vortex_err!("Invalid glob pattern '{}': {}", pattern_str, e))?; - - if is_s3 { - let base_url = Url::parse(base_url_str) - .map_err(|e| vortex_err!("Invalid S3 URL '{}': {}", base_url_str, e))?; - let bucket = base_url - .host_str() - .ok_or_else(|| vortex_err!("Missing bucket in S3 URL: {}", base_url_str))?; - Ok((s3_store(bucket)?, base_url, FileDiscovery::Glob(pattern))) - } else { - let path_str = base_url_str.strip_prefix("file://").unwrap_or(base_url_str); - let canonical = std::fs::canonicalize(path_str) - .map_err(|e| vortex_err!("Failed to resolve path '{}': {}", path_str, e))?; - let base_url = Url::from_directory_path(&canonical) - .map_err(|_| vortex_err!("Invalid directory path: {}", canonical.display()))?; - let store: Arc = Arc::new( - LocalFileSystem::new_with_prefix("/") - .map_err(|e| vortex_err!("Failed to create local filesystem: {}", e))?, - ); - Ok((store, base_url, FileDiscovery::Glob(pattern))) - } -} - -/// Extract column names and DuckDB types from a Vortex [`DType`]. -fn extract_schema_from_dtype(dtype: &DType) -> VortexResult<(Vec, Vec)> { - let struct_dtype = dtype - .as_struct_fields_opt() - .ok_or_else(|| vortex_err!("Vortex file must contain a struct array at the top level"))?; - - let mut column_names = Vec::new(); - let mut column_types = Vec::new(); - - for (field_name, field_dtype) in struct_dtype.names().iter().zip(struct_dtype.fields()) { - let logical_type = LogicalType::try_from(&field_dtype)?; - column_names.push(field_name.to_string()); - column_types.push(logical_type); - } - - Ok((column_names, column_types)) -} - -/// Creates a projection expression from the table initialization input. -fn extract_projection_expr(init: &TableInitInput) -> Expression { - let projection_ids = init.projection_ids().unwrap_or(&[]); - let column_ids = init.column_ids(); - - select( - projection_ids - .iter() - .map(|p| { - let idx: usize = p.as_(); - let val: usize = column_ids[idx].as_(); - val - }) - .map(|idx| { - init.bind_data() - .column_names - .get(idx) - .vortex_expect("prune idx in column names") - }) - .map(|s| Arc::from(s.as_str())) - .collect::(), - root(), - ) -} - -/// Creates a filter expression from the table filter set. -fn extract_table_filter_expr( - init: &TableInitInput, - column_ids: &[u64], -) -> VortexResult> { - let mut table_filter_exprs: HashSet = if let Some(filter) = init.table_filter_set() - { - filter - .into_iter() - .map(|(idx, ex)| { - let idx_u: usize = idx.as_(); - let col_idx: usize = column_ids[idx_u].as_(); - let name = init - .bind_data() - .column_names - .get(col_idx) - .vortex_expect("exists"); - try_from_table_filter( - &ex, - &col(name.as_str()), - init.bind_data().data_source.dtype(), - ) - }) - .collect::>>>()? - .unwrap_or_else(HashSet::new) - } else { - HashSet::new() - }; - - table_filter_exprs.extend(init.bind_data().filter_exprs.clone()); - Ok(and_collect(table_filter_exprs.into_iter().collect_vec())) -} - -// Taken from duckdb/common/constants.h COLUMN_IDENTIFIER_EMPTY -static EMPTY_COLUMN_IDX: u64 = 18446744073709551614; -static EMPTY_COLUMN_NAME: &str = ""; - -impl TableFunction for ScanApiTableFunction { - type BindData = ScanApiBindData; - type GlobalState = ScanApiGlobalData; - type LocalState = ScanApiLocalData; - - const PROJECTION_PUSHDOWN: bool = true; - const FILTER_PUSHDOWN: bool = true; - const FILTER_PRUNE: bool = true; - - fn parameters() -> Vec { - vec![LogicalType::varchar()] - } - - fn bind( - ctx: &ClientContext, - input: &BindInput, - result: &mut BindResult, - ) -> VortexResult { - let file_glob_string = input - .get_parameter(0) - .ok_or_else(|| vortex_err!("Missing file glob parameter"))?; - - let max_threads_cstr = CString::new("vortex_max_threads") - .map_err(|e| vortex_err!("Invalid setting name: {}", e))?; - let max_threads = ctx - .try_get_current_setting(&max_threads_cstr) - .and_then(|v| match v.as_ref().extract() { - ExtractedValue::UBigInt(val) => usize::try_from(val).ok(), - ExtractedValue::BigInt(val) if val > 0 => usize::try_from(val as u64).ok(), - _ => None, - }) - .unwrap_or_else(|| { - std::thread::available_parallelism() - .map(|n| n.get()) - .unwrap_or(1) - }); - - tracing::trace!("running scan with max_threads {max_threads}"); - - let (object_store, base_url, discovery) = - parse_url_glob(&file_glob_string.as_ref().as_string())?; - - let multi_ds = RUNTIME.block_on(Compat::new( - MultiFileDataSource::builder(SESSION.clone(), object_store, base_url) - .with_discovery(discovery) - .with_prefetch(max_threads * 2) - .build(), - ))?; - - let (column_names, column_types) = extract_schema_from_dtype(multi_ds.dtype())?; - - for (column_name, column_type) in column_names.iter().zip(&column_types) { - result.add_result_column(column_name, column_type); - } - - let file_count = multi_ds.file_count(); - let estimate = multi_ds.row_count_estimate(); - let row_count_estimate = estimate.upper.unwrap_or(estimate.lower); - let data_source: DataSourceRef = Arc::new(multi_ds); - - Ok(ScanApiBindData { - data_source, - filter_exprs: vec![], - column_names, - column_types, - max_threads: max_threads as u64, - file_count, - row_count_estimate, - }) - } - - fn scan( - _client_context: &ClientContext, - _bind_data: &Self::BindData, - local_state: &mut Self::LocalState, - global_state: &mut Self::GlobalState, - chunk: &mut DataChunk, - ) -> VortexResult<()> { - loop { - if local_state.exporter.is_none() { - let Some(result) = local_state.iterator.next() else { - return Ok(()); - }; - - let (array_result, conversion_cache) = result?; - - let array_result = array_result.optimize_recursive()?; - let array_result = if let Some(array) = array_result.as_opt::() { - array.clone() - } else if let Some(array) = array_result.as_opt::() - && let Some(pack_options) = array.scalar_fn().as_opt::() - { - StructArray::new( - pack_options.names.clone(), - array.children(), - array.len(), - pack_options.nullability.into(), - ) - } else { - array_result - .execute::(&mut global_state.ctx)? - .into_struct() - }; - - local_state.exporter = Some(ArrayExporter::try_new( - &array_result, - &conversion_cache, - &mut global_state.ctx, - )?); - local_state.batch_id = Some(global_state.batch_id.fetch_add(1, Ordering::Relaxed)); - } - - let exporter = local_state - .exporter - .as_mut() - .vortex_expect("error: exporter missing"); - - let has_more_data = exporter.export(chunk)?; - - if !has_more_data { - local_state.exporter = None; - local_state.batch_id = None; - } else { - break; - } - } - - assert!(!chunk.is_empty()); - - Ok(()) - } - - fn init_global(init_input: &TableInitInput) -> VortexResult { - let bind_data = init_input.bind_data(); - let projection_expr = extract_projection_expr(init_input); - let filter_expr = extract_table_filter_expr(init_input, init_input.column_ids())?; - - tracing::trace!( - "Global init Vortex scan (v2) SELECT {} WHERE {}", - &projection_expr, - filter_expr - .as_ref() - .map_or_else(|| "true".to_string(), |f| f.to_string()) - ); - - #[expect(clippy::cast_possible_truncation, reason = "max_threads fits in usize")] - let num_workers = bind_data.max_threads as usize; - - let request = ScanRequest { - projection: Some(projection_expr), - filter: filter_expr, - ..Default::default() - }; - - let scan = bind_data.data_source.scan(request)?; - let conversion_cache = Arc::new(ConversionCache::new(0)); - - let scan_streams = scan.splits().then(move |split_result| { - let cache = conversion_cache.clone(); - async move { - let split = split_result?; - let s = split.execute().await?; - Ok(s.map(move |r| Ok((r?, cache.clone()))).boxed()) - } - }); - - let iterator = RUNTIME.block_on_stream_thread_safe(move |_| MultiScan { - streams: scan_streams.boxed(), - streams_finished: false, - select_all: Default::default(), - max_concurrency: num_workers * 2, - }); - - Ok(ScanApiGlobalData { - iterator, - batch_id: AtomicU64::new(0), - ctx: ExecutionCtx::new(VortexSession::default()), - }) - } - - fn init_local( - _init: &TableInitInput, - global: &mut Self::GlobalState, - ) -> VortexResult { - unsafe { - use custom_labels::sys; - - if sys::labelset_current().is_null() { - let ls = sys::labelset_new(0); - sys::labelset_replace(ls); - }; - } - - let global_labels = get_global_labels(); - - for (key, value) in global_labels { - CURRENT_LABELSET.set(key, value); - } - - Ok(ScanApiLocalData { - iterator: global.iterator.clone(), - exporter: None, - batch_id: None, - }) - } - - fn pushdown_complex_filter( - bind_data: &mut Self::BindData, - expr: &duckdb::Expression, - ) -> VortexResult { - let Some(expr) = try_from_bound_expression(expr)? else { - return Ok(false); - }; - bind_data.filter_exprs.push(expr); - Ok(false) - } - - fn cardinality(bind_data: &Self::BindData) -> Cardinality { - if bind_data.file_count == 1 { - Cardinality::Maximum(bind_data.row_count_estimate) - } else { - Cardinality::Estimate(bind_data.row_count_estimate) - } - } - - fn partition_data( - _bind_data: &Self::BindData, - _global_init_data: &mut Self::GlobalState, - local_init_data: &mut Self::LocalState, - ) -> VortexResult { - local_init_data - .batch_id - .ok_or_else(|| vortex_err!("batch id missing, no batches exported")) - } - - fn to_string(bind_data: &Self::BindData) -> Option> { - let mut result = Vec::new(); - result.push(("Function".to_string(), "Vortex Scan (v2)".to_string())); - result.push(("Files".to_string(), bind_data.file_count.to_string())); - if !bind_data.filter_exprs.is_empty() { - let mut filters = bind_data.filter_exprs.iter().map(|f| format!("{}", f)); - result.push(("Filters".to_string(), filters.join(" /\\\n"))); - } - Some(result) - } - - fn virtual_columns(_bind_data: &Self::BindData, result: &mut VirtualColumnsResult) { - result.register(EMPTY_COLUMN_IDX, EMPTY_COLUMN_NAME, &LogicalType::bool()); - } -} From a45bf71d616e9739dd83da4bd87ddca30084c261 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 15:07:00 -0500 Subject: [PATCH 27/57] Scan API Signed-off-by: Nicholas Gates --- Cargo.lock | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.lock b/Cargo.lock index 4d2d0a343a0..28de0fb5ad6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11214,6 +11214,7 @@ dependencies = [ "pin-project-lite", "tokio", "tracing", + "url", "uuid", "vortex-alp", "vortex-array", From 4aff18a659f65339a4267e42896459c236582372 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 16:27:35 -0500 Subject: [PATCH 28/57] Scan API Signed-off-by: Nicholas Gates --- vortex-file/src/filesystem/mod.rs | 45 ++++++++++++++++++++++ vortex-file/src/filesystem/object_store.rs | 0 vortex-file/src/filesystem/prefix.rs | 0 3 files changed, 45 insertions(+) create mode 100644 vortex-file/src/filesystem/mod.rs create mode 100644 vortex-file/src/filesystem/object_store.rs create mode 100644 vortex-file/src/filesystem/prefix.rs diff --git a/vortex-file/src/filesystem/mod.rs b/vortex-file/src/filesystem/mod.rs new file mode 100644 index 00000000000..8182004b055 --- /dev/null +++ b/vortex-file/src/filesystem/mod.rs @@ -0,0 +1,45 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! A filesystem abstraction for discovering and opening Vortex files. +//! +//! [`VortexFileSystem`] provides a storage-agnostic interface for listing files under a prefix +//! and opening them for reading. Implementations can target local filesystems, object stores, +//! or any other storage backend. + +use std::sync::Arc; + +use async_trait::async_trait; +use futures::stream::BoxStream; +use vortex_error::VortexResult; +use vortex_io::VortexReadAt; + +/// A file discovered during listing, with its path and optional size in bytes. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] +pub struct FileListing { + /// The file path (relative to the filesystem root). + pub path: String, + /// The file size in bytes, if known from the listing metadata. + pub size: Option, +} + +/// A storage-agnostic filesystem interface for discovering and reading Vortex files. +/// +/// Implementations handle the details of a particular storage backend (local disk, S3, GCS, etc.) +/// while consumers work through this uniform interface. +/// +/// # Future Work +/// +/// An `open_write` method will be added once [`VortexWrite`](vortex_io::VortexWrite) is +/// object-safe (it currently uses `impl Future` return types which prevent trait-object usage). +#[async_trait] +pub trait VortexFileSystem: Send + Sync + 'static { + /// List files whose paths start with `prefix`. + /// + /// Returns a stream of [`FileListing`] entries. The stream may yield entries in any order; + /// callers should sort if deterministic ordering is required. + fn list(&self, prefix: &str) -> BoxStream<'_, VortexResult>; + + /// Open a file for reading at the given path. + async fn open_read(&self, path: &str) -> VortexResult>; +} diff --git a/vortex-file/src/filesystem/object_store.rs b/vortex-file/src/filesystem/object_store.rs new file mode 100644 index 00000000000..e69de29bb2d diff --git a/vortex-file/src/filesystem/prefix.rs b/vortex-file/src/filesystem/prefix.rs new file mode 100644 index 00000000000..e69de29bb2d From fa7a431eb0e2084529263e9d9e926432b08563fb Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 16:27:39 -0500 Subject: [PATCH 29/57] Scan API Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 16 ++- vortex-file/src/filesystem/mod.rs | 13 ++- vortex-file/src/filesystem/object_store.rs | 61 +++++++++++ vortex-file/src/filesystem/prefix.rs | 69 +++++++++++++ vortex-file/src/lib.rs | 2 +- vortex-file/src/multi/builder.rs | 112 +++++++-------------- vortex-file/src/multi/glob.rs | 98 ++++++------------ vortex-file/src/multi/mod.rs | 4 +- vortex-file/src/multi/source.rs | 44 ++------ 9 files changed, 231 insertions(+), 188 deletions(-) diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 7c0da3d9fc1..00201e1d773 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -285,6 +285,8 @@ async fn register_v2_tables( benchmark: &B, format: Format, ) -> anyhow::Result<()> { + use vortex::file::filesystem::object_store::ObjectStoreFileSystem; + use vortex::io::session::RuntimeSessionExt; use vortex_datafusion::v2::VortexTable; let benchmark_base = benchmark.data_url().join(&format!("{}/", format.name()))?; @@ -297,6 +299,11 @@ async fn register_v2_tables( .runtime_env() .object_store(table_url.object_store())?; + let fs: Arc = + Arc::new(ObjectStoreFileSystem::new(store.clone(), SESSION.handle())); + let base_prefix = benchmark_base.path().trim_start_matches('/').to_string(); + let fs = fs.prefix(base_prefix); + let discovery = match pattern { Some(p) => FileDiscovery::Glob(p), None => FileDiscovery::Glob( @@ -304,11 +311,10 @@ async fn register_v2_tables( .map_err(|e| vortex_err!("invalid pattern for table {}: {}", table.name, e))?, ), }; - let multi_ds = - MultiFileDataSource::builder(SESSION.clone(), store.clone(), benchmark_base.clone()) - .with_discovery(discovery) - .build() - .await?; + let multi_ds = MultiFileDataSource::builder(SESSION.clone(), fs) + .with_discovery(discovery) + .build() + .await?; let arrow_schema = Arc::new(multi_ds.dtype().to_arrow_schema()?); let data_source: DataSourceRef = Arc::new(multi_ds); diff --git a/vortex-file/src/filesystem/mod.rs b/vortex-file/src/filesystem/mod.rs index 8182004b055..0fd8aa9cc18 100644 --- a/vortex-file/src/filesystem/mod.rs +++ b/vortex-file/src/filesystem/mod.rs @@ -3,10 +3,14 @@ //! A filesystem abstraction for discovering and opening Vortex files. //! -//! [`VortexFileSystem`] provides a storage-agnostic interface for listing files under a prefix +//! [`FileSystem`] provides a storage-agnostic interface for listing files under a prefix //! and opening them for reading. Implementations can target local filesystems, object stores, //! or any other storage backend. +#[cfg(feature = "object_store")] +pub mod object_store; +mod prefix; + use std::sync::Arc; use async_trait::async_trait; @@ -23,6 +27,9 @@ pub struct FileListing { pub size: Option, } +/// A reference-counted handle to a file system. +pub type FileSystemRef = Arc; + /// A storage-agnostic filesystem interface for discovering and reading Vortex files. /// /// Implementations handle the details of a particular storage backend (local disk, S3, GCS, etc.) @@ -33,12 +40,12 @@ pub struct FileListing { /// An `open_write` method will be added once [`VortexWrite`](vortex_io::VortexWrite) is /// object-safe (it currently uses `impl Future` return types which prevent trait-object usage). #[async_trait] -pub trait VortexFileSystem: Send + Sync + 'static { +pub trait FileSystem: Send + Sync + 'static { /// List files whose paths start with `prefix`. /// /// Returns a stream of [`FileListing`] entries. The stream may yield entries in any order; /// callers should sort if deterministic ordering is required. - fn list(&self, prefix: &str) -> BoxStream<'_, VortexResult>; + fn list(&self, prefix: Option<&str>) -> BoxStream<'_, VortexResult>; /// Open a file for reading at the given path. async fn open_read(&self, path: &str) -> VortexResult>; diff --git a/vortex-file/src/filesystem/object_store.rs b/vortex-file/src/filesystem/object_store.rs index e69de29bb2d..5a555e6da37 100644 --- a/vortex-file/src/filesystem/object_store.rs +++ b/vortex-file/src/filesystem/object_store.rs @@ -0,0 +1,61 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +//! [`FileSystem`] implementation backed by an [`ObjectStore`]. + +use std::sync::Arc; + +use async_trait::async_trait; +use futures::StreamExt; +use futures::stream::BoxStream; +use object_store::ObjectStore; +use object_store::path::Path; +use vortex_error::VortexResult; +use vortex_io::VortexReadAt; +use vortex_io::file::object_store::ObjectStoreSource; +use vortex_io::runtime::Handle; + +use crate::filesystem::FileListing; +use crate::filesystem::FileSystem; + +/// A [`FileSystem`] backed by an [`ObjectStore`]. +/// +// TODO(ngates): we could consider spawning a driver task inside this file system such that we can +// apply concurrency limits to the overall object store, rather than on a per-file basis. +pub struct ObjectStoreFileSystem { + store: Arc, + handle: Handle, +} + +impl ObjectStoreFileSystem { + /// Create a new filesystem backed by the given object store and runtime handle. + pub fn new(store: Arc, handle: Handle) -> Self { + Self { store, handle } + } +} + +#[async_trait] +impl FileSystem for ObjectStoreFileSystem { + fn list(&self, prefix: Option<&str>) -> BoxStream<'_, VortexResult> { + let path = prefix.map(Path::from); + self.store + .list(path.as_ref()) + .map(|result| { + result + .map(|meta| FileListing { + path: meta.location.to_string(), + size: Some(meta.size), + }) + .map_err(Into::into) + }) + .boxed() + } + + async fn open_read(&self, path: &str) -> VortexResult> { + Ok(Arc::new(ObjectStoreSource::new( + self.store.clone(), + path.into(), + self.handle.clone(), + ))) + } +} diff --git a/vortex-file/src/filesystem/prefix.rs b/vortex-file/src/filesystem/prefix.rs index e69de29bb2d..68048c85589 100644 --- a/vortex-file/src/filesystem/prefix.rs +++ b/vortex-file/src/filesystem/prefix.rs @@ -0,0 +1,69 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright the Vortex contributors + +use std::sync::Arc; + +use async_trait::async_trait; +use futures::StreamExt; +use futures::stream::BoxStream; +use vortex_error::VortexResult; +use vortex_io::VortexReadAt; + +use crate::filesystem::FileListing; +use crate::filesystem::FileSystem; +use crate::filesystem::FileSystemRef; + +/// A [`FileSystem`] decorator that roots all operations under a given prefix. +/// +/// Paths returned from [`list`](FileSystem::list) are relative to the prefix, and paths +/// passed to [`open_read`](FileSystem::open_read) are automatically prefixed. +pub struct PrefixFileSystem { + inner: FileSystemRef, + prefix: String, +} + +impl PrefixFileSystem { + pub fn new(inner: FileSystemRef, prefix: String) -> Self { + // Normalize to always have a trailing slash for clean concatenation. + let prefix = format!("{}/", prefix.trim_matches('/')); + Self { inner, prefix } + } +} + +#[async_trait] +impl FileSystem for PrefixFileSystem { + fn list(&self, prefix: Option<&str>) -> BoxStream<'_, VortexResult> { + let full_prefix = match prefix { + Some(suffix) => format!("{}{}", self.prefix, suffix.trim_start_matches('/')), + None => self.prefix.clone(), + }; + + let strip_prefix = self.prefix.clone(); + self.inner + .list(Some(&full_prefix)) + .map(move |result| { + result.map(|mut listing| { + listing.path = listing + .path + .strip_prefix(&strip_prefix) + .unwrap_or(&listing.path) + .to_string(); + listing + }) + }) + .boxed() + } + + async fn open_read(&self, path: &str) -> VortexResult> { + self.inner + .open_read(&format!("{}{}", self.prefix, path.trim_start_matches('/'))) + .await + } +} + +impl dyn FileSystem + '_ { + /// Create a new filesystem that applies the given prefix to all operations on this filesystem. + pub fn prefix(self: Arc, prefix: String) -> FileSystemRef { + Arc::new(PrefixFileSystem::new(self, prefix)) + } +} diff --git a/vortex-file/src/lib.rs b/vortex-file/src/lib.rs index f1860db8fe1..df23af21bae 100644 --- a/vortex-file/src/lib.rs +++ b/vortex-file/src/lib.rs @@ -92,8 +92,8 @@ mod counting; mod file; +pub mod filesystem; mod footer; -#[cfg(feature = "object_store")] pub mod multi; mod open; mod pruning; diff --git a/vortex-file/src/multi/builder.rs b/vortex-file/src/multi/builder.rs index 683dacbca19..33a5ecec06b 100644 --- a/vortex-file/src/multi/builder.rs +++ b/vortex-file/src/multi/builder.rs @@ -6,27 +6,23 @@ use std::sync::Arc; use glob::Pattern; -use object_store::ObjectStore; -use object_store::path::Path; use tracing::debug; -use url::Url; -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_scan::multi::DataSourceFactory; use vortex_scan::multi::MultiDataSource; use vortex_session::VortexSession; use super::glob::expand_glob; use super::glob::list_all; -use super::source::DiscoveredFile; use super::source::MultiFileDataSource; use super::source::VortexFileFactory; use super::source::data_source_from_file; use crate::OpenOptionsSessionExt; use crate::VortexOpenOptions; +use crate::filesystem::FileListing; +use crate::filesystem::FileSystemRef; /// How to handle schema differences across files in a [`MultiFileDataSource`]. #[derive(Debug, Clone, Default)] @@ -43,72 +39,72 @@ pub enum SchemaResolution { /// How files are discovered for a [`MultiFileDataSource`]. #[derive(Debug)] pub enum FileDiscovery { - /// Explicit list of file paths (relative to the object store root). + /// Explicit list of file paths. Paths(Vec), - /// A glob pattern to expand against the object store (relative to the object store root). + /// A glob pattern to expand against the filesystem. Glob(Pattern), - /// List all files under the prefix that match the configured file extension. + /// List all files in the filesystem. ListAll, } /// Builder for constructing a [`MultiFileDataSource`]. /// -/// By default, all files in the object store are discovered (equivalent to a `*` glob). +/// By default, all files are discovered by listing the filesystem (equivalent to `ListAll`). /// Use [`with_paths`](Self::with_paths) or [`with_glob`](Self::with_glob) to restrict /// which files are included. /// +/// To scope the data source to a subdirectory, wrap the filesystem with +/// [`FileSystem::prefix`](crate::filesystem::FileSystem::prefix) before passing it to the builder. +/// /// # Examples /// /// ```ignore -/// // Discover all files: -/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// // Discover all files under a prefix: +/// let fs = fs.prefix("data/".into()); +/// let ds = MultiFileDataSource::builder(session, fs) /// .build() /// .await?; /// /// // From a glob pattern: -/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// let fs = fs.prefix("data/".into()); +/// let ds = MultiFileDataSource::builder(session, fs) /// .with_glob(glob::Pattern::new("**/*.vortex")?) /// .with_prefetch(16) /// .build() /// .await?; /// /// // From explicit paths: -/// let ds = MultiFileDataSourceBuilder::new(session, object_store, "s3://bucket/data/") +/// let ds = MultiFileDataSource::builder(session, fs) /// .with_paths(vec!["a.vortex".into(), "b.vortex".into()]) /// .build() /// .await?; /// ``` pub struct MultiFileDataSourceBuilder { session: VortexSession, - object_store: Arc, - base_url: Url, + fs: FileSystemRef, discovery: FileDiscovery, schema_resolution: SchemaResolution, open_options_fn: Arc VortexOpenOptions + Send + Sync>, prefetch: Option, - filter: Option, dtype: Option, } impl MultiFileDataSource { - /// Create a new builder from an object store and base URL prefix. + /// Create a new builder from a filesystem. /// - /// The `base_url` is used for display/debug purposes. It should typically match the - /// location of the files (e.g. `"s3://bucket/data/"`). + /// To scope the data source to a subdirectory, wrap the filesystem with + /// [`FileSystem::prefix`](crate::filesystem::FileSystem::prefix). pub fn builder( session: VortexSession, - object_store: Arc, - base_url: Url, + fs: FileSystemRef, ) -> MultiFileDataSourceBuilder { MultiFileDataSourceBuilder { session, - object_store, - base_url, + fs, discovery: FileDiscovery::ListAll, schema_resolution: SchemaResolution::default(), open_options_fn: Arc::new(|opts| opts), prefetch: None, - filter: None, dtype: None, } } @@ -121,15 +117,15 @@ impl MultiFileDataSourceBuilder { self } - /// Set explicit file paths (relative to the object store root). + /// Set explicit file paths. pub fn with_paths(self, paths: Vec) -> Self { self.with_discovery(FileDiscovery::Paths(paths)) } - /// Discover files by expanding a glob pattern against the object store. + /// Discover files by expanding a glob pattern against the filesystem. /// - /// The pattern is relative to the object store root - /// (e.g. `"data/**/*.vortex"`). Expansion happens eagerly during [`build`](Self::build). + /// The pattern is relative to the filesystem root + /// (e.g. `"**/*.vortex"`). Expansion happens eagerly during [`build`](Self::build). pub fn with_glob(self, pattern: Pattern) -> Self { self.with_discovery(FileDiscovery::Glob(pattern)) } @@ -175,20 +171,10 @@ impl MultiFileDataSourceBuilder { self } - /// Set a filter expression for file-level pruning. - /// - /// Files whose statistics indicate they cannot match the filter will be skipped entirely. - /// When no explicit dtype is provided, the first file is always opened (to determine the - /// schema); deferred files may be skipped if their statistics prove the filter cannot match. - pub fn with_filter(mut self, filter: Expression) -> Self { - self.filter = Some(filter); - self - } - /// Build the [`MultiFileDataSource`]. /// /// If a glob pattern was provided via [`with_glob`](Self::with_glob), it is expanded - /// eagerly against the object store. If a [`DType`] was provided via + /// eagerly against the filesystem. If a [`DType`] was provided via /// [`with_dtype`](Self::with_dtype), all files are opened lazily during scanning. /// Otherwise, the first file is opened eagerly to determine the schema. #[tracing::instrument(name = "MultiFileDataSourceBuilder::build", skip(self))] @@ -203,46 +189,30 @@ impl MultiFileDataSourceBuilder { FileDiscovery::ListAll => "list_all".to_string(), }; debug!( - base_url = %self.base_url, discovery = %discovery_kind, "building MultiFileDataSource" ); - debug!( - "Discovering files in {}: {:?}", - self.base_url, self.discovery - ); - let base_url_path = Path::from_url_path(self.base_url.path()) - .map_err(|e| vortex_err!("Invalid base_url '{}': {}", self.base_url, e))?; let files = match self.discovery { FileDiscovery::Paths(ref paths) => paths .iter() - .map(|path| { - // FIXME(ngates): join path to the base_url_path. - DiscoveredFile { - path: path.clone(), - size: None, - } + .map(|path| FileListing { + path: path.clone(), + size: None, }) .collect(), - FileDiscovery::Glob(ref pattern) => { - expand_glob(&self.object_store, &base_url_path, pattern).await? - } - FileDiscovery::ListAll => list_all(&self.object_store, &base_url_path).await?, + FileDiscovery::Glob(ref pattern) => expand_glob(&self.fs, pattern).await?, + FileDiscovery::ListAll => list_all(&self.fs).await?, }; debug!( - base_url = %self.base_url, file_count = files.len(), files = ?files, "discovered files" ); if files.is_empty() { - vortex_bail!( - "MultiFileDataSource requires at least one file (base_url: {})", - self.base_url - ); + vortex_bail!("MultiFileDataSource requires at least one file"); } let file_count = files.len(); @@ -259,9 +229,8 @@ impl MultiFileDataSourceBuilder { if let Some(size) = first.size { first_options = first_options.with_file_size(size); } - let first_file = first_options - .open_object_store(&self.object_store, &first.path) - .await?; + let source = self.fs.open_read(&first.path).await?; + let first_file = first_options.open(source).await?; let dtype = first_file.dtype().clone(); debug!(dtype = %dtype, "determined dtype from first file"); @@ -278,28 +247,21 @@ impl MultiFileDataSourceBuilder { }; debug!( - base_url = %self.base_url, file_count, dtype = %dtype, "built MultiFileDataSource" ); - Ok(MultiFileDataSource::new( - dtype, - inner, - self.base_url.to_string(), - file_count, - )) + Ok(MultiFileDataSource::new(dtype, inner, file_count)) } - fn make_factories(&self, files: &[DiscoveredFile]) -> Vec> { + fn make_factories(&self, files: &[FileListing]) -> Vec> { files .iter() .map(|file| { Arc::new(VortexFileFactory { - object_store: self.object_store.clone(), + fs: self.fs.clone(), file: file.clone(), - filter: self.filter.clone(), session: self.session.clone(), open_options_fn: self.open_options_fn.clone(), }) as Arc diff --git a/vortex-file/src/multi/glob.rs b/vortex-file/src/multi/glob.rs index 15a7bf11933..5b8a7969aa8 100644 --- a/vortex-file/src/multi/glob.rs +++ b/vortex-file/src/multi/glob.rs @@ -1,106 +1,74 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright the Vortex contributors -//! URL glob expansion for discovering Vortex files in object stores. +//! Glob expansion for discovering Vortex files via a [`FileSystem`]. //! -//! Uses [`object_store::ObjectStore::list()`] with a computed prefix and client-side glob +//! Uses [`FileSystem::list()`] with a computed prefix and client-side glob //! filtering to discover files matching a pattern. -use std::sync::Arc; - -use futures::StreamExt; -use futures::TryStreamExt; use glob::Pattern; -use object_store::ObjectStore; -use object_store::path::Path; use tracing::debug; use vortex_error::VortexResult; use vortex_error::vortex_bail; -use super::source::DiscoveredFile; +use crate::filesystem::FileListing; +use crate::filesystem::FileSystemRef; + +/// List all files in the filesystem, returning sorted [`FileListing`]s. +#[tracing::instrument(name = "list_all", skip(fs))] +pub(super) async fn list_all(fs: &FileSystemRef) -> VortexResult> { + use futures::TryStreamExt; -/// List all files under `prefix` in the object store, returning sorted -/// [`DiscoveredFile`]s with path and size. -#[tracing::instrument(name = "list_all", skip(object_store))] -pub(super) async fn list_all( - object_store: &Arc, - prefix: &Path, -) -> VortexResult> { debug!("listing all files"); - let mut files: Vec = object_store - .list(Some(prefix)) - .map_ok(|meta| DiscoveredFile { - path: meta.location.to_string(), - size: Some(meta.size), - }) - .try_collect() - .await?; + let mut files: Vec = fs.list(None).try_collect().await?; files.sort(); debug!(file_count = files.len(), "listed all files"); Ok(files) } -/// Expand a glob pattern against an [`ObjectStore`], returning matching -/// [`DiscoveredFile`]s with path and size. +/// Expand a glob pattern against a [`FileSystem`], returning matching +/// [`FileListing`]s with path and size. /// -/// The `glob_pattern` should be a path pattern (not a full URL) relative to the store root, -/// e.g. `"data/year=2024/**/*.vortex"`. +/// The `pattern` is matched against file paths relative to the filesystem root +/// (e.g. `"**/*.vortex"`). /// /// # Algorithm /// /// 1. Find the first glob character (`*`, `?`, `[`) in the pattern. /// 2. Use everything before it (up to the last `/`) as the list prefix. -/// 3. List objects with that prefix. +/// 3. List files with that prefix. /// 4. Filter using [`glob::Pattern`] matching. /// 5. Return sorted file paths. -#[tracing::instrument(name = "expand_glob", skip(object_store))] +#[tracing::instrument(name = "expand_glob", skip(fs))] pub(super) async fn expand_glob( - object_store: &Arc, - base_url_path: &Path, + fs: &FileSystemRef, pattern: &Pattern, -) -> VortexResult> { +) -> VortexResult> { + use futures::TryStreamExt; + let glob_str = pattern.as_str(); validate_glob(glob_str)?; // Extract the static prefix from the glob pattern to narrow the listing. let prefix = list_prefix(glob_str); - let listing_path = if prefix.is_empty() { - base_url_path.clone() + let listing_prefix = if prefix.is_empty() { + None } else { - Path::from(format!( - "{}/{}", - base_url_path.as_ref().trim_end_matches('/'), - prefix.trim_end_matches('/') - )) + Some(prefix.trim_end_matches('/')) }; - let base_prefix = base_url_path.as_ref(); - - debug!(%base_url_path, %listing_path, "expanding glob"); - - let mut files: Vec = object_store - .list(Some(&listing_path)) - .filter_map(|result| async { - match result { - Ok(meta) => { - let path_str = meta.location.to_string(); - let relative = path_str - .strip_prefix(base_prefix) - .map(|s| s.trim_start_matches('/')) - .unwrap_or(&path_str); - pattern.matches(relative).then_some(DiscoveredFile { - path: path_str, - size: Some(meta.size), - }) - } - // FIXME(ngates): do not ignore errors - Err(_) => None, - } + + debug!(?listing_prefix, "expanding glob"); + + let mut files: Vec = fs + .list(listing_prefix) + .try_filter_map(|listing| async move { + Ok(pattern.matches(&listing.path).then_some(listing)) }) - .collect() - .await; + .try_collect() + .await?; files.sort(); debug!(file_count = files.len(), "expanded glob"); @@ -110,7 +78,7 @@ pub(super) async fn expand_glob( /// Returns the list prefix for a path pattern containing glob characters. /// /// The prefix is the directory path up to the first glob character, which is used to narrow -/// the `list()` call on the object store. +/// the `list()` call on the filesystem. /// /// # Examples /// diff --git a/vortex-file/src/multi/mod.rs b/vortex-file/src/multi/mod.rs index 979348a1c3d..5d717a2683f 100644 --- a/vortex-file/src/multi/mod.rs +++ b/vortex-file/src/multi/mod.rs @@ -7,8 +7,8 @@ //! discovers and opens multiple Vortex files, presenting them as a single [`DataSource`] //! for scanning. It is analogous to DataFusion's `ListingTable`. //! -//! Use [`MultiFileDataSourceBuilder`] to construct a `MultiFileDataSource` from an object store -//! and a set of file paths or a glob pattern. +//! Use [`MultiFileDataSourceBuilder`] to construct a `MultiFileDataSource` from a +//! [`FileSystem`](crate::filesystem::FileSystem) and a set of file paths or a glob pattern. //! //! [`DataSource`]: vortex_scan::api::DataSource diff --git a/vortex-file/src/multi/source.rs b/vortex-file/src/multi/source.rs index b6aa0314ad1..aad9e1be78c 100644 --- a/vortex-file/src/multi/source.rs +++ b/vortex-file/src/multi/source.rs @@ -16,9 +16,7 @@ use std::sync::Arc; use async_trait::async_trait; -use object_store::ObjectStore; use tracing::debug; -use vortex_array::expr::Expression; use vortex_dtype::DType; use vortex_error::VortexResult; use vortex_scan::api::DataSource; @@ -35,6 +33,8 @@ use vortex_session::VortexSession; use crate::OpenOptionsSessionExt; use crate::VortexFile; use crate::VortexOpenOptions; +use crate::filesystem::FileListing; +use crate::filesystem::FileSystemRef; use crate::v2::FileStatsLayoutReader; /// A [`DataSource`] that scans across multiple Vortex files, presenting them as a single source. @@ -45,30 +45,18 @@ use crate::v2::FileStatsLayoutReader; pub struct MultiFileDataSource { dtype: DType, inner: MultiDataSource, - base_url: String, file_count: usize, } impl MultiFileDataSource { - pub(super) fn new( - dtype: DType, - inner: MultiDataSource, - base_url: String, - file_count: usize, - ) -> Self { + pub(super) fn new(dtype: DType, inner: MultiDataSource, file_count: usize) -> Self { Self { dtype, inner, - base_url, file_count, } } - /// Returns the base URL prefix for files in this data source. - pub fn base_url(&self) -> &str { - &self.base_url - } - /// Returns the number of files in this data source. pub fn file_count(&self) -> usize { self.file_count @@ -98,9 +86,8 @@ impl DataSource for MultiFileDataSource { /// /// Handles statistics-based pruning via [`VortexFile::can_prune`]. pub(super) struct VortexFileFactory { - pub(super) object_store: Arc, - pub(super) file: DiscoveredFile, - pub(super) filter: Option, + pub(super) fs: FileSystemRef, + pub(super) file: FileListing, pub(super) session: VortexSession, pub(super) open_options_fn: Arc VortexOpenOptions + Send + Sync>, } @@ -113,16 +100,8 @@ impl DataSourceFactory for VortexFileFactory { if let Some(size) = self.file.size { options = options.with_file_size(size); } - let file = options - .open_object_store(&self.object_store, &self.file.path) - .await?; - - if let Some(ref filter) = self.filter - && file.can_prune(filter)? - { - debug!(path = %self.file.path, "pruned file based on statistics"); - return Ok(None); - } + let source = self.fs.open_read(&self.file.path).await?; + let file = options.open(source).await?; let ds = data_source_from_file(&file, &self.session)?; debug!(path = %self.file.path, "opened vortex file"); @@ -145,12 +124,3 @@ pub(super) fn data_source_from_file( session.clone(), ))) } - -/// A file discovered during file listing, with its path and size in bytes. -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] -pub(super) struct DiscoveredFile { - /// The file path relative to the object store root. - pub path: String, - /// The file size in bytes, if known. - pub size: Option, -} From fe6fbec30b1933f5b198e7c9fdbd029374d3eef8 Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Fri, 13 Feb 2026 16:45:58 -0500 Subject: [PATCH 30/57] Scan API Signed-off-by: Nicholas Gates --- benchmarks/datafusion-bench/src/main.rs | 3 ++ vortex-duckdb/src/duckdb/footer_cache.rs | 54 ++++++++---------------- vortex-duckdb/src/scan.rs | 52 +++++++++++++---------- vortex-file/src/footer/cache.rs | 47 +++++++++++++++++++++ vortex-file/src/footer/mod.rs | 2 + vortex-file/src/multi/builder.rs | 26 ++++++++++-- vortex-file/src/multi/glob.rs | 6 +-- vortex-file/src/multi/source.rs | 10 +++++ 8 files changed, 135 insertions(+), 65 deletions(-) create mode 100644 vortex-file/src/footer/cache.rs diff --git a/benchmarks/datafusion-bench/src/main.rs b/benchmarks/datafusion-bench/src/main.rs index 00201e1d773..d5386941685 100644 --- a/benchmarks/datafusion-bench/src/main.rs +++ b/benchmarks/datafusion-bench/src/main.rs @@ -285,11 +285,13 @@ async fn register_v2_tables( benchmark: &B, format: Format, ) -> anyhow::Result<()> { + use vortex::file::InMemoryFooterCache; use vortex::file::filesystem::object_store::ObjectStoreFileSystem; use vortex::io::session::RuntimeSessionExt; use vortex_datafusion::v2::VortexTable; let benchmark_base = benchmark.data_url().join(&format!("{}/", format.name()))?; + let footer_cache = Arc::new(InMemoryFooterCache::new()); for table in benchmark.table_specs().iter() { let pattern = benchmark.pattern(table.name, format); @@ -313,6 +315,7 @@ async fn register_v2_tables( }; let multi_ds = MultiFileDataSource::builder(SESSION.clone(), fs) .with_discovery(discovery) + .with_footer_cache(footer_cache.clone()) .build() .await?; diff --git a/vortex-duckdb/src/duckdb/footer_cache.rs b/vortex-duckdb/src/duckdb/footer_cache.rs index cdc7869e044..7e6ea0702f9 100644 --- a/vortex-duckdb/src/duckdb/footer_cache.rs +++ b/vortex-duckdb/src/duckdb/footer_cache.rs @@ -2,53 +2,35 @@ // SPDX-FileCopyrightText: Copyright the Vortex contributors use vortex::file::Footer; -use vortex::file::VortexOpenOptions; +use vortex::file::FooterCache; use crate::duckdb::ObjectCacheRef; -pub struct FooterCache<'a> { - object_cache: ObjectCacheRef<'a>, +/// A [`FooterCache`] backed by DuckDB's internal object cache. +pub struct DuckDbFooterCache { + object_cache: ObjectCacheRef<'static>, } -pub struct Entry<'a> { - object_cache: ObjectCacheRef<'a>, - key: String, - value: Option<&'a Footer>, -} - -impl Entry<'_> { - pub fn put_if_absent(self, value: impl FnOnce() -> Footer) { - if self.value.is_some() { - return; - } - self.object_cache.put(&self.key, value()); +impl DuckDbFooterCache { + pub fn new(object_cache: ObjectCacheRef<'static>) -> Self { + Self { object_cache } } - pub fn apply_to_file(&self, options: VortexOpenOptions) -> VortexOpenOptions { - if let Some(footer) = self.value { - options.with_footer(footer.clone()) - } else { - options - } + fn key(path: &str) -> String { + format!("vx_footer://{path}") } } -impl<'a> FooterCache<'a> { - pub fn new(object_cache: ObjectCacheRef<'a>) -> Self { - Self { object_cache } +impl FooterCache for DuckDbFooterCache { + fn get(&self, key: &str) -> Option