vortex_scan/lib.rs
1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright the Vortex contributors
3
4#![deny(missing_docs)]
5
6//! The Vortex Scan API implements an abstract table scan interface that can be used to
7//! read data from various data sources.
8//!
9//! It supports arbitrary projection expressions, filter expressions, and limit pushdown as well
10//! as mechanisms for parallel and distributed execution via partitions.
11//!
12//! The API is currently under development and may change in future releases, however we hope to
13//! stabilize into stable C ABI for use within foreign language bindings.
14//!
15//! If you are looking to scan Vortex files or layouts, the Vortex implementation of the Scan API
16//! can be found in the `vortex-layout` crate.
17//!
18//! ## Open Issues
19//!
20//! * We probably want to make the DataSource serializable as well, so that we can share
21//! source-level state with workers, separately from partition serialization.
22//! * We should add a way for the client to negotiate capabilities with the data source, for
23//! example which encodings it knows about.
24
25pub mod row_mask;
26pub mod selection;
27
28use std::any::Any;
29use std::ops::Range;
30use std::sync::Arc;
31
32use async_trait::async_trait;
33use futures::stream::BoxStream;
34use selection::Selection;
35use vortex_array::dtype::DType;
36use vortex_array::dtype::FieldPath;
37use vortex_array::expr::Expression;
38use vortex_array::expr::root;
39use vortex_array::expr::stats::Precision;
40use vortex_array::stats::StatsSet;
41use vortex_array::stream::SendableArrayStream;
42use vortex_error::VortexResult;
43use vortex_error::vortex_bail;
44use vortex_session::VortexSession;
45
46/// A sendable stream of partitions.
47pub type PartitionStream = BoxStream<'static, VortexResult<PartitionRef>>;
48
49/// Opens a Vortex [`DataSource`] from a URI.
50///
51/// Configuration can be passed via the URI query parameters, similar to JDBC / ADBC.
52/// Providers can be registered with the [`VortexSession`] to support additional URI schemes.
53#[async_trait]
54pub trait DataSourceOpener: 'static {
55 /// Attempt to open a new data source from a URI.
56 async fn open(&self, uri: String, session: &VortexSession) -> VortexResult<DataSourceRef>;
57}
58
59/// Supports deserialization of a Vortex [`DataSource`] on a remote worker.
60#[async_trait]
61pub trait DataSourceRemote: 'static {
62 /// Attempt to deserialize the source.
63 fn deserialize_data_source(
64 &self,
65 data: &[u8],
66 session: &VortexSession,
67 ) -> VortexResult<DataSourceRef>;
68}
69
70/// A reference-counted data source.
71pub type DataSourceRef = Arc<dyn DataSource>;
72
73/// A data source represents a streamable dataset that can be scanned with projection and filter
74/// expressions. Each scan produces partitions that can be executed in parallel to read data. Each
75/// partition can be serialized for remote execution.
76///
77/// The DataSource may be used multiple times to create multiple scans, whereas each scan and each
78/// partition of a scan can only be consumed once.
79#[async_trait]
80pub trait DataSource: 'static + Send + Sync {
81 /// Returns the dtype of the source.
82 fn dtype(&self) -> &DType;
83
84 /// Returns an estimate of the row count of the un-filtered source.
85 fn row_count(&self) -> Option<Precision<u64>> {
86 None
87 }
88
89 /// Returns an estimate of the byte size of the un-filtered source.
90 fn byte_size(&self) -> Option<Precision<u64>> {
91 None
92 }
93
94 /// Serialize the [`DataSource`] to pass to a remote worker.
95 fn serialize(&self) -> VortexResult<Option<Vec<u8>>> {
96 Ok(None)
97 }
98
99 /// Deserialize a partition that was previously serialized from a compatible data source.
100 fn deserialize_partition(
101 &self,
102 data: &[u8],
103 session: &VortexSession,
104 ) -> VortexResult<PartitionRef> {
105 let _ = (data, session);
106 vortex_bail!("DataSource does not support deserialization")
107 }
108
109 /// Returns a scan over the source.
110 async fn scan(&self, scan_request: ScanRequest) -> VortexResult<DataSourceScanRef>;
111
112 /// Returns the statistics for a given field.
113 async fn field_statistics(&self, field_path: &FieldPath) -> VortexResult<StatsSet>;
114}
115
116/// A request to scan a data source.
117#[derive(Debug, Clone)]
118pub struct ScanRequest {
119 /// Projection expression. Defaults to `root()` which returns all columns.
120 pub projection: Expression,
121 /// Filter expression, `None` implies no filter.
122 pub filter: Option<Expression>,
123 /// The row range to read.
124 pub row_range: Option<Range<u64>>,
125 /// A row selection to apply to the scan. The selection identifies rows within the specified
126 /// row range.
127 pub selection: Selection,
128 /// Whether the scan should preserve row order. If false, the scan may produce rows in any
129 /// order, for example to enable parallel execution across partitions.
130 pub ordered: bool,
131 /// Optional limit on the number of rows returned by scan. Limits are applied after all
132 /// filtering and row selection.
133 pub limit: Option<u64>,
134}
135
136impl Default for ScanRequest {
137 fn default() -> Self {
138 Self {
139 projection: root(),
140 filter: None,
141 row_range: None,
142 selection: Selection::default(),
143 ordered: false,
144 limit: None,
145 }
146 }
147}
148
149/// A boxed data source scan.
150pub type DataSourceScanRef = Box<dyn DataSourceScan>;
151
152/// A data source scan produces partitions that can be executed to read data from the source.
153pub trait DataSourceScan: 'static + Send {
154 /// The returned dtype of the scan.
155 fn dtype(&self) -> &DType;
156
157 /// Returns an estimate of the total number of partitions the scan will produce.
158 fn partition_count(&self) -> Option<Precision<usize>>;
159
160 /// Returns a stream of partitions to be processed.
161 fn partitions(self: Box<Self>) -> PartitionStream;
162}
163
164/// A reference-counted partition.
165pub type PartitionRef = Box<dyn Partition>;
166
167/// A partition represents a unit of work that can be executed to produce a stream of arrays.
168pub trait Partition: 'static + Send {
169 /// Downcast the partition to a concrete type.
170 fn as_any(&self) -> &dyn Any;
171
172 /// Returns an estimate of the row count for this partition.
173 fn row_count(&self) -> Option<Precision<u64>>;
174
175 /// Returns an estimate of the byte size for this partition.
176 fn byte_size(&self) -> Option<Precision<u64>>;
177
178 /// Serialize this partition for a remote worker.
179 fn serialize(&self) -> VortexResult<Option<Vec<u8>>> {
180 Ok(None)
181 }
182
183 /// Executes the partition, returning an array stream.
184 ///
185 /// This method must be fast. The returned stream should be lazy — all non-trivial work
186 /// (I/O, decoding, filtering) must be deferred to when the stream is polled. Expensive
187 /// operations should be spawned onto the runtime to enable parallel execution across
188 /// threads.
189 fn execute(self: Box<Self>) -> VortexResult<SendableArrayStream>;
190}