vortex_layout/layouts/chunked/
eval_expr.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
use async_trait::async_trait;
use futures::future::{ready, try_join_all};
use futures::FutureExt;
use vortex_array::array::{ChunkedArray, ConstantArray};
use vortex_array::{ArrayDType, ArrayData, Canonical, IntoArrayData, IntoArrayVariant};
use vortex_error::VortexResult;
use vortex_expr::pruning::PruningPredicate;
use vortex_expr::ExprRef;
use vortex_scalar::Scalar;
use vortex_scan::RowMask;

use crate::layouts::chunked::reader::ChunkedReader;
use crate::reader::LayoutScanExt;
use crate::ExprEvaluator;

#[async_trait(?Send)]
impl ExprEvaluator for ChunkedReader {
    async fn evaluate_expr(
        self: &Self,
        row_mask: RowMask,
        expr: ExprRef,
    ) -> VortexResult<ArrayData> {
        // Compute the result dtype of the expression.
        let dtype = expr
            .evaluate(&Canonical::empty(self.dtype())?.into_array())?
            .dtype()
            .clone();

        // First we need to compute the pruning mask
        let pruning_predicate = PruningPredicate::try_new(&expr);
        let pruning_mask = if let Some(predicate) = pruning_predicate {
            // If the expression is prune-able, then fetch the stats table
            if let Some(stats_table) = self.stats_table().await? {
                predicate
                    .evaluate(stats_table.array())?
                    .map(|mask| mask.into_bool())
                    .transpose()?
                    .map(|mask| mask.boolean_buffer())
            } else {
                None
            }
        } else {
            None
        };

        // Now we set up futures to evaluate each chunk at the same time
        let mut chunks = Vec::with_capacity(self.nchunks());

        let mut row_offset = 0;
        for chunk_idx in 0..self.nchunks() {
            let chunk_reader = self.child(chunk_idx)?;

            // Figure out the row range of the chunk
            let chunk_len = chunk_reader.layout().row_count();
            let chunk_range = row_offset..row_offset + chunk_len;
            row_offset += chunk_len;

            // Try to skip the chunk based on the row-mask
            if row_mask.is_disjoint(chunk_range.clone()) {
                continue;
            }

            // If the pruning mask tells us the chunk is pruned (i.e. the expr is ALL false),
            // then we can just return a constant array.
            if let Some(pruning_mask) = &pruning_mask {
                if pruning_mask.value(chunk_idx) {
                    let false_array = ConstantArray::new(
                        Scalar::bool(false, dtype.nullability()),
                        row_mask.true_count(),
                    );
                    chunks.push(ready(Ok(false_array.into_array())).boxed_local());
                    continue;
                }
            }

            // Otherwise, we need to read it. So we set up a mask for the chunk range.
            let chunk_mask = row_mask
                .slice(chunk_range.start, chunk_range.end)?
                .shift(chunk_range.start)?;

            let expr = expr.clone();
            chunks.push(chunk_reader.evaluate_expr(chunk_mask, expr).boxed_local());
        }

        // Wait for all chunks to be evaluated
        let chunks = try_join_all(chunks).await?;

        Ok(ChunkedArray::try_new(chunks, dtype)?.into_array())
    }
}

#[cfg(test)]
mod test {
    use std::sync::Arc;

    use futures::executor::block_on;
    use vortex_array::array::{BoolArray, ChunkedArray, ConstantArray};
    use vortex_array::{ArrayLen, IntoArrayData, IntoArrayVariant};
    use vortex_buffer::buffer;
    use vortex_dtype::Nullability::NonNullable;
    use vortex_dtype::{DType, PType};
    use vortex_error::VortexExpect;
    use vortex_expr::{gt, lit, Identity};
    use vortex_scan::RowMask;

    use crate::layouts::chunked::writer::ChunkedLayoutWriter;
    use crate::segments::test::TestSegments;
    use crate::strategies::LayoutWriterExt;
    use crate::LayoutData;

    /// Create a chunked layout with three chunks of primitive arrays.
    fn chunked_layout() -> (Arc<TestSegments>, LayoutData) {
        let mut segments = TestSegments::default();
        let layout = ChunkedLayoutWriter::new(
            &DType::Primitive(PType::I32, NonNullable),
            Default::default(),
        )
        .push_all(
            &mut segments,
            [
                Ok(buffer![1, 2, 3].into_array()),
                Ok(buffer![4, 5, 6].into_array()),
                Ok(buffer![7, 8, 9].into_array()),
            ],
        )
        .unwrap();
        (Arc::new(segments), layout)
    }

    #[test]
    fn test_chunked_scan() {
        block_on(async {
            let (segments, layout) = chunked_layout();

            let result = layout
                .reader(segments, Default::default())
                .unwrap()
                .evaluate_expr(
                    RowMask::new_valid_between(0, layout.row_count()),
                    Identity::new_expr(),
                )
                .await
                .unwrap()
                .into_primitive()
                .unwrap();

            assert_eq!(result.len(), 9);
            assert_eq!(result.as_slice::<i32>(), &[1, 2, 3, 4, 5, 6, 7, 8, 9]);
        })
    }

    #[test]
    fn test_chunked_pruning_mask() {
        block_on(async {
            let (segments, layout) = chunked_layout();
            let row_count = layout.row_count();
            let reader = layout.reader(segments, Default::default()).unwrap();

            // Choose a prune-able expression
            let expr = gt(Identity::new_expr(), lit(7));

            let result = reader
                .evaluate_expr(RowMask::new_valid_between(0, row_count), expr.clone())
                .await
                .unwrap();
            let result = ChunkedArray::try_from(result).unwrap();

            // Now we ensure that the pruned chunks are ConstantArrays, instead of having been
            // evaluated.
            assert_eq!(result.nchunks(), 3);
            ConstantArray::try_from(result.chunk(0).unwrap())
                .vortex_expect("Expected first chunk to be pruned");
            ConstantArray::try_from(result.chunk(1).unwrap())
                .vortex_expect("Expected second chunk to be pruned");
            BoolArray::try_from(result.chunk(2).unwrap())
                .vortex_expect("Expected third chunk to be evaluated");
        })
    }
}