|
2 | 2 | // SPDX-FileCopyrightText: Copyright the Vortex contributors |
3 | 3 |
|
4 | 4 | use std::path::PathBuf; |
| 5 | +use std::sync::Arc; |
5 | 6 | use std::sync::atomic::AtomicU64; |
6 | 7 | use std::sync::atomic::Ordering::SeqCst; |
7 | | -use std::sync::*; |
8 | 8 |
|
9 | | -use atomic::AtomicBool; |
10 | 9 | use bitvec::macros::internal::funty::Fundamental; |
11 | | -use crossbeam_queue::SegQueue; |
| 10 | +use vortex::ToCanonical; |
12 | 11 | use vortex::dtype::FieldNames; |
13 | 12 | use vortex::error::{VortexExpect, VortexResult, vortex_bail, vortex_err}; |
14 | 13 | use vortex::expr::{ExprRef, and, and_collect, lit, root, select}; |
15 | 14 | use vortex::file::{VortexFile, VortexOpenOptions}; |
| 15 | +use vortex::scan::{MultiScan, MultiScanIterator}; |
16 | 16 |
|
17 | 17 | use crate::convert::{try_from_bound_expression, try_from_table_filter}; |
18 | 18 | use crate::duckdb::{ |
19 | 19 | BindInput, BindResult, DataChunk, Expression, LogicalType, TableFunction, TableInitInput, |
20 | 20 | }; |
21 | | -use crate::exporter::ArrayIteratorExporter; |
| 21 | +use crate::exporter::{ArrayExporter, ConversionCache}; |
22 | 22 |
|
23 | 23 | pub struct VortexBindData { |
24 | 24 | first_file: VortexFile, |
@@ -54,17 +54,16 @@ impl std::fmt::Debug for VortexBindData { |
54 | 54 | } |
55 | 55 |
|
56 | 56 | pub struct VortexGlobalData { |
57 | | - file_paths: SegQueue<PathBuf>, |
58 | | - is_first_file_processed: AtomicBool, |
59 | | - /// We currently use a conversion cache to cache converted arrays, this id is used to |
60 | | - /// ensure that each cache created has a unique id used to name those arrays |
61 | | - conversion_cache_id: AtomicU64, |
62 | | - filter_expr: Option<ExprRef>, |
63 | | - projection_expr: ExprRef, |
| 57 | + multi_scan: MultiScan, |
| 58 | + cache_id: AtomicU64, |
64 | 59 | } |
65 | 60 |
|
66 | 61 | pub struct VortexLocalData { |
67 | | - exporter: Option<ArrayIteratorExporter>, |
| 62 | + multi_scan_iterator: MultiScanIterator, |
| 63 | + exporter: Option<ArrayExporter>, |
| 64 | + |
| 65 | + // TODO(Alex): replace with global conversion cache |
| 66 | + conversion_cache: ConversionCache, |
68 | 67 | } |
69 | 68 |
|
70 | 69 | #[derive(Debug)] |
@@ -148,16 +147,6 @@ fn extract_table_filter_expr( |
148 | 147 | Ok(Some(filter_expr)) |
149 | 148 | } |
150 | 149 |
|
151 | | -/// Creates a lock-free queue populated with file paths from bind data. |
152 | | -fn create_file_paths_queue(bind_data: &VortexBindData) -> SegQueue<PathBuf> { |
153 | | - let file_paths = SegQueue::new(); |
154 | | - // Skip the first file as it is opened during bind. |
155 | | - for path in bind_data.file_paths.iter().skip(1) { |
156 | | - file_paths.push(path.clone()); |
157 | | - } |
158 | | - file_paths |
159 | | -} |
160 | | - |
161 | 150 | impl TableFunction for VortexTableFunction { |
162 | 151 | type BindData = VortexBindData; |
163 | 152 | type GlobalState = VortexGlobalData; |
@@ -210,82 +199,92 @@ impl TableFunction for VortexTableFunction { |
210 | 199 | } |
211 | 200 |
|
212 | 201 | fn scan( |
213 | | - bind_data: &Self::BindData, |
| 202 | + _bind_data: &Self::BindData, |
214 | 203 | local_state: &mut Self::LocalState, |
215 | 204 | global_state: &mut Self::GlobalState, |
216 | 205 | chunk: &mut DataChunk, |
217 | 206 | ) -> VortexResult<()> { |
218 | | - let exporter_for_file = |
219 | | - |file: &VortexFile, id: u64| -> VortexResult<ArrayIteratorExporter> { |
220 | | - let array_iterator = file |
221 | | - .scan()? |
222 | | - .with_projection(global_state.projection_expr.clone()) |
223 | | - .with_some_filter(global_state.filter_expr.clone()) |
224 | | - .into_array_iter() |
225 | | - .map_err(|e| vortex_err!("Failed to create array iterator: {}", e))?; |
226 | | - |
227 | | - Ok(ArrayIteratorExporter::new(Box::new(array_iterator), id)) |
228 | | - }; |
229 | | - |
230 | 207 | loop { |
231 | 208 | if local_state.exporter.is_none() { |
232 | | - if !global_state.is_first_file_processed.swap(true, SeqCst) { |
233 | | - let cache_id = global_state.conversion_cache_id.fetch_add(1, SeqCst); |
234 | | - local_state.exporter = |
235 | | - Some(exporter_for_file(&bind_data.first_file, cache_id)?); |
236 | | - } |
237 | | - // Retrieve a file path from the shared lock-free queue. |
238 | | - else if let Some(file_path) = global_state.file_paths.pop() { |
239 | | - let file = VortexOpenOptions::file() |
240 | | - .open_blocking(&file_path) |
241 | | - .map_err(|e| vortex_err!("Failed to open Vortex file: {}", e))?; |
242 | | - |
243 | | - let cache_id = global_state.conversion_cache_id.fetch_add(1, SeqCst); |
244 | | - local_state.exporter = Some(exporter_for_file(&file, cache_id)?); |
245 | | - } else { |
246 | | - // If the exporter is None and there are no more files to process, signal that the scan finished. |
247 | | - chunk.set_len(0); |
| 209 | + let Some(array_result) = local_state.multi_scan_iterator.next() else { |
248 | 210 | return Ok(()); |
249 | | - } |
| 211 | + }; |
| 212 | + |
| 213 | + // TODO(Alex): replace with global conversion cache |
| 214 | + local_state.conversion_cache = |
| 215 | + ConversionCache::new(global_state.cache_id.fetch_add(1, SeqCst)); |
| 216 | + |
| 217 | + local_state.exporter = Some(ArrayExporter::try_new( |
| 218 | + &array_result?.to_struct()?, |
| 219 | + &mut local_state.conversion_cache, |
| 220 | + )?); |
250 | 221 | } |
251 | 222 |
|
252 | | - let Some(ref mut exporter) = local_state.exporter else { |
253 | | - vortex_bail!("ArrayIteratorExporter is not set") |
254 | | - }; |
| 223 | + let exporter = local_state |
| 224 | + .exporter |
| 225 | + .as_mut() |
| 226 | + .vortex_expect("exporter should exist"); |
255 | 227 |
|
256 | | - let is_data_left_to_scan = !exporter |
257 | | - .export(chunk) |
258 | | - .map_err(|e| vortex_err!("Failed to export data: {}", e))?; |
| 228 | + let has_more_data = exporter.export(chunk)?; |
259 | 229 |
|
260 | | - if is_data_left_to_scan { |
| 230 | + if !has_more_data { |
| 231 | + // This exporter is fully consumed. |
261 | 232 | local_state.exporter = None; |
262 | 233 | } else { |
263 | | - assert!(!chunk.is_empty()); |
264 | | - return Ok(()); |
| 234 | + break; |
265 | 235 | } |
266 | 236 | } |
| 237 | + |
| 238 | + assert!(!chunk.is_empty()); |
| 239 | + |
| 240 | + Ok(()) |
267 | 241 | } |
268 | 242 |
|
269 | 243 | fn init_global(init_input: &TableInitInput<Self>) -> VortexResult<Self::GlobalState> { |
270 | 244 | let bind_data = init_input.bind_data(); |
271 | | - let file_paths = create_file_paths_queue(bind_data); |
272 | 245 | let projection_expr = extract_projection_expr(init_input); |
273 | 246 | let filter_expr = extract_table_filter_expr(init_input, init_input.column_ids())?; |
| 247 | + let is_first_file_queued = Arc::new(std::sync::atomic::AtomicBool::new(false)); |
| 248 | + |
| 249 | + let closures = bind_data.file_paths.clone().into_iter().map(move |path| { |
| 250 | + let first_file = bind_data.first_file.clone(); |
| 251 | + let filter_expr = filter_expr.clone(); |
| 252 | + let projection_expr = projection_expr.clone(); |
| 253 | + let is_first_file_queued = is_first_file_queued.clone(); |
| 254 | + |
| 255 | + move || { |
| 256 | + let file = if !is_first_file_queued.swap(true, SeqCst) { |
| 257 | + // The first path from `file_paths` is skipped as the first |
| 258 | + // file was already opened during bind. |
| 259 | + first_file |
| 260 | + } else { |
| 261 | + VortexOpenOptions::file() |
| 262 | + .open_blocking(&path) |
| 263 | + .vortex_expect("Failed to open Vortex file") |
| 264 | + }; |
| 265 | + |
| 266 | + file.scan() |
| 267 | + .vortex_expect("Failed to create scan builder") |
| 268 | + .with_some_filter(filter_expr) |
| 269 | + .with_projection(projection_expr) |
| 270 | + } |
| 271 | + }); |
274 | 272 |
|
275 | 273 | Ok(VortexGlobalData { |
276 | | - file_paths, |
277 | | - is_first_file_processed: AtomicBool::new(false), |
278 | | - conversion_cache_id: AtomicU64::new(0), |
279 | | - filter_expr, |
280 | | - projection_expr, |
| 274 | + multi_scan: MultiScan::new().with_scan_builders(closures), |
| 275 | + cache_id: AtomicU64::new(0), |
281 | 276 | }) |
282 | 277 | } |
283 | 278 |
|
284 | 279 | fn init_local( |
285 | 280 | _init: &TableInitInput<Self>, |
286 | | - _global: &mut Self::GlobalState, |
| 281 | + global: &mut Self::GlobalState, |
287 | 282 | ) -> VortexResult<Self::LocalState> { |
288 | | - Ok(VortexLocalData { exporter: None }) |
| 283 | + Ok(VortexLocalData { |
| 284 | + multi_scan_iterator: global.multi_scan.new_scan_iterator(), |
| 285 | + exporter: None, |
| 286 | + conversion_cache: ConversionCache::new(global.cache_id.fetch_add(1, SeqCst)), |
| 287 | + }) |
289 | 288 | } |
290 | 289 |
|
291 | 290 | fn pushdown_complex_filter( |
|
0 commit comments