vortex_file/
file.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
use std::future::Future;
use std::ops::Range;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use futures::Stream;
use futures_util::{stream, FutureExt, StreamExt, TryFutureExt, TryStreamExt};
use itertools::Itertools;
use pin_project_lite::pin_project;
use vortex_array::stats::{Stat, StatsSet};
use vortex_array::stream::{ArrayStream, ArrayStreamAdapter};
use vortex_array::ContextRef;
use vortex_buffer::Buffer;
use vortex_dtype::{DType, Field, FieldMask, FieldPath};
use vortex_error::{vortex_err, VortexExpect, VortexResult};
use vortex_expr::transform::immediate_access::immediate_scope_access;
use vortex_expr::transform::simplify_typed::simplify_typed;
use vortex_expr::{ident, ExprRef};
use vortex_layout::{ExprEvaluator, LayoutReader};
use vortex_mask::Mask;
use vortex_scan::{RowMask, Scanner};

use crate::exec::ExecDriver;
use crate::io::IoDriver;
use crate::segments::channel::SegmentChannel;
use crate::{FileLayout, SplitBy};

/// A Vortex file ready for reading.
///
/// It is generic over the `IoDriver` implementation enabling us to swap out the I/O subsystem for
/// particular environments. For example, memory mapped files vs object-store. By remaining generic,
/// it allows us to support both `Send` and `?Send` I/O drivers.
pub struct VortexFile<I> {
    pub(crate) ctx: ContextRef,
    pub(crate) file_layout: FileLayout,
    pub(crate) io_driver: I,
    pub(crate) exec_driver: Arc<dyn ExecDriver>,
    pub(crate) split_by: SplitBy,
}

pub struct Scan {
    projection: ExprRef,
    filter: Option<ExprRef>,
    row_indices: Option<Buffer<u64>>,
}

impl Scan {
    pub fn all() -> Self {
        Self {
            projection: ident(),
            filter: None,
            row_indices: None,
        }
    }

    pub fn new(projection: ExprRef) -> Self {
        Self {
            projection,
            filter: None,
            row_indices: None,
        }
    }

    pub fn filtered(filter: ExprRef) -> Self {
        Self {
            projection: ident(),
            filter: Some(filter),
            row_indices: None,
        }
    }

    pub fn with_filter(mut self, filter: ExprRef) -> Self {
        self.filter = Some(filter);
        self
    }

    pub fn with_some_filter(mut self, filter: Option<ExprRef>) -> Self {
        self.filter = filter;
        self
    }

    pub fn with_projection(mut self, projection: ExprRef) -> Self {
        self.projection = projection;
        self
    }

    pub fn with_row_indices(mut self, row_indices: Buffer<u64>) -> Self {
        self.row_indices = Some(row_indices);
        self
    }

    /// Compute a mask of field paths referenced by this scan.
    pub fn field_mask(&self, scope_dtype: &DType) -> VortexResult<Vec<FieldMask>> {
        // TODO(joe): simplify this expr once
        let projection = simplify_typed(self.projection.clone(), scope_dtype)?;
        let filter = self
            .filter
            .clone()
            .map(|f| simplify_typed(f, scope_dtype))
            .transpose()?;

        let Some(struct_dtype) = scope_dtype.as_struct() else {
            return Ok(vec![FieldMask::All]);
        };

        let projection_mask = immediate_scope_access(&projection, struct_dtype)?;
        let filter_mask = filter
            .map(|f| immediate_scope_access(&f, struct_dtype))
            .transpose()?
            .unwrap_or_default();

        Ok(projection_mask
            .union(&filter_mask)
            .cloned()
            .map(|c| FieldMask::Prefix(FieldPath::from(Field::Name(c))))
            .collect_vec())
    }
}

/// Async implementation of Vortex File.
impl<I: IoDriver> VortexFile<I> {
    /// Returns the number of rows in the file.
    pub fn row_count(&self) -> u64 {
        self.file_layout.row_count()
    }

    /// Returns the DType of the file.
    pub fn dtype(&self) -> &DType {
        self.file_layout.dtype()
    }

    /// Returns the [`FileLayout`] of the file.
    pub fn file_layout(&self) -> &FileLayout {
        &self.file_layout
    }

    /// Performs a scan operation over the file.
    pub fn scan(&self, scan: Scan) -> VortexResult<impl ArrayStream + 'static + use<'_, I>> {
        let field_mask = scan.field_mask(self.dtype())?;
        let splits: Arc<[Range<u64>]> = self
            .split_by
            .splits(self.file_layout().root_layout(), &field_mask)?
            .into_iter()
            .collect_vec()
            .into();

        let row_masks = ArcIter::new(splits).filter_map(move |row_range| {
            let Some(row_indices) = &scan.row_indices else {
                // If there is no row indices filter, then take the whole range
                return Some(RowMask::new_valid_between(row_range.start, row_range.end));
            };

            // Otherwise, find the indices that are within the row range.
            if row_indices
                .first()
                .is_some_and(|&first| first >= row_range.end)
                || row_indices
                    .last()
                    .is_some_and(|&last| row_range.start >= last)
            {
                return None;
            }

            // For the given row range, find the indices that are within the row_indices.
            let start_idx = row_indices
                .binary_search(&row_range.start)
                .unwrap_or_else(|x| x);
            let end_idx = row_indices
                .binary_search(&row_range.end)
                .unwrap_or_else(|x| x);

            if start_idx == end_idx {
                // No rows in range
                return None;
            }

            // Construct a row mask for the range.
            let filter_mask = Mask::from_indices(
                usize::try_from(row_range.end - row_range.start)
                    .vortex_expect("Split ranges are within usize"),
                row_indices[start_idx..end_idx]
                    .iter()
                    .map(|&idx| {
                        usize::try_from(idx - row_range.start).vortex_expect("index within range")
                    })
                    .collect(),
            );
            Some(RowMask::new(filter_mask, row_range.start))
        });

        self.scan_with_masks(row_masks, scan.projection, scan.filter)
    }

    fn scan_with_masks<R>(
        &self,
        row_masks: R,
        projection: ExprRef,
        filter: Option<ExprRef>,
    ) -> VortexResult<impl ArrayStream + 'static + use<'_, I, R>>
    where
        R: Iterator<Item = RowMask> + Send + 'static,
    {
        let scanner = Arc::new(Scanner::new(self.dtype().clone(), projection, filter)?);

        let result_dtype = scanner.result_dtype().clone();

        // Set up a segment channel to collect segment requests from the execution stream.
        let segment_channel = SegmentChannel::new();

        // Create a single LayoutReader that is reused for the entire scan.
        let reader: Arc<dyn LayoutReader> = self
            .file_layout
            .root_layout()
            .reader(segment_channel.reader(), self.ctx.clone())?;

        // Now we give one end of the channel to the layout reader...
        let exec_stream = stream::iter(row_masks)
            .map(
                move |row_mask| match scanner.clone().range_scanner(row_mask) {
                    Ok(range_scan) => {
                        let reader = reader.clone();
                        async move {
                            range_scan
                                .evaluate_async(|row_mask, expr| {
                                    reader.evaluate_expr(row_mask, expr)
                                })
                                .await
                        }
                        .boxed()
                    }
                    Err(e) => futures::future::ready(Err(e)).boxed(),
                },
            )
            .boxed();
        let exec_stream = self.exec_driver.drive(exec_stream);

        // ...and the other end to the segment driver.
        let io_stream = self.io_driver.drive(segment_channel.into_stream());

        Ok(ArrayStreamAdapter::new(
            result_dtype,
            UnifiedDriverStream {
                exec_stream,
                io_stream,
            },
        ))
    }

    /// Resolves the requested statistics for the file.
    pub fn statistics(
        &self,
        field_paths: Arc<[FieldPath]>,
        stats: Arc<[Stat]>,
    ) -> VortexResult<impl Future<Output = VortexResult<Vec<StatsSet>>> + 'static + use<'_, I>>
    {
        // Set up a segment channel to collect segment requests from the execution stream.
        let segment_channel = SegmentChannel::new();

        // Create a single LayoutReader that is reused for the entire scan.
        let reader: Arc<dyn LayoutReader> = self
            .file_layout
            .root_layout()
            .reader(segment_channel.reader(), self.ctx.clone())?;

        let exec_future = async move { reader.evaluate_stats(field_paths, stats).await }.boxed();
        let io_stream = self.io_driver.drive(segment_channel.into_stream());

        Ok(UnifiedDriverFuture {
            exec_future,
            io_stream,
        })
    }
}

/// There is no `IntoIterator` for `Arc<[T]>` so to avoid copying into a Vec<T>, we define our own.
/// See <https://users.rust-lang.org/t/arc-to-owning-iterator/115190/11>.
struct ArcIter<T> {
    inner: Arc<[T]>,
    pos: usize,
}

impl<T> ArcIter<T> {
    fn new(inner: Arc<[T]>) -> Self {
        Self { inner, pos: 0 }
    }
}

impl<T: Clone> Iterator for ArcIter<T> {
    type Item = T;

    fn next(&mut self) -> Option<Self::Item> {
        (self.pos < self.inner.len()).then(|| {
            let item = self.inner[self.pos].clone();
            self.pos += 1;
            item
        })
    }
}

pin_project! {
    /// A [`Stream`] that drives the both the I/O stream and the execution stream concurrently.
    ///
    /// This is sort of like a `select!` implementation, but not quite.
    ///
    /// We can't use `futures::stream::select` because it requires both streams to terminate, and
    /// our I/O stream will never terminate.
    ///
    /// We can't use `futures::stream::zip` because it waits for boths streams to emit an item,
    /// but our execution stream may require multiple I/O operations to complete before it can
    /// return an item.
    struct UnifiedDriverStream<R, S> {
        #[pin]
        exec_stream: R,
        #[pin]
        io_stream: S,
    }
}

impl<T, R, S> Stream for UnifiedDriverStream<R, S>
where
    R: Stream<Item = VortexResult<T>>,
    S: Stream<Item = VortexResult<()>>,
{
    type Item = VortexResult<T>;

    fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        let mut this = self.project();
        loop {
            // If the exec stream is ready, then we can return the result.
            // If it's pending, then we try polling the I/O stream.
            if let Poll::Ready(r) = this.exec_stream.try_poll_next_unpin(cx) {
                return Poll::Ready(r);
            }

            match this.io_stream.as_mut().try_poll_next_unpin(cx) {
                // If the I/O stream made progress, it returns Ok.
                Poll::Ready(Some(Ok(()))) => {}
                // If the I/O stream failed, then propagate the error.
                Poll::Ready(Some(Err(result))) => {
                    return Poll::Ready(Some(Err(result)));
                }
                // Unexpected end of stream.
                Poll::Ready(None) => {
                    return Poll::Ready(Some(Err(vortex_err!("unexpected end of I/O stream"))));
                }
                // If the I/O stream is not ready, then we return Pending and wait for the next wakeup.
                Poll::Pending => return Poll::Pending,
            }
        }
    }
}

pin_project! {
    struct UnifiedDriverFuture<R, S> {
        #[pin]
        exec_future: R,
        #[pin]
        io_stream: S,
    }
}

impl<T, R, S> Future for UnifiedDriverFuture<R, S>
where
    R: Future<Output = VortexResult<T>>,
    S: Stream<Item = VortexResult<()>>,
{
    type Output = VortexResult<T>;

    fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
        let mut this = self.project();
        loop {
            // If the exec stream is ready, then we can return the result.
            // If it's pending, then we try polling the I/O stream.
            if let Poll::Ready(r) = this.exec_future.try_poll_unpin(cx) {
                return Poll::Ready(r);
            }

            match this.io_stream.as_mut().try_poll_next_unpin(cx) {
                // If the I/O stream made progress, it returns Ok.
                Poll::Ready(Some(Ok(()))) => {}
                // If the I/O stream failed, then propagate the error.
                Poll::Ready(Some(Err(result))) => {
                    return Poll::Ready(Err(result));
                }
                // Unexpected end of stream.
                Poll::Ready(None) => {
                    return Poll::Ready(Err(vortex_err!("unexpected end of I/O stream")));
                }
                // If the I/O stream is not ready, then we return Pending and wait for the next wakeup.
                Poll::Pending => return Poll::Pending,
            }
        }
    }
}