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
use std::any::Any;
use std::sync::Arc;

use arrow_schema::SchemaRef;
use async_trait::async_trait;
use datafusion::catalog::Session;
use datafusion::datasource::TableProvider;
use datafusion::prelude::*;
use datafusion_common::{Result as DFResult, ToDFSchema};
use datafusion_expr::utils::conjunction;
use datafusion_expr::{TableProviderFilterPushDown, TableType};
use datafusion_physical_expr::{create_physical_expr, EquivalenceProperties};
use datafusion_physical_plan::{ExecutionMode, ExecutionPlan, Partitioning, PlanProperties};
use itertools::Itertools;
use vortex_array::array::ChunkedArray;
use vortex_array::arrow::infer_schema;
use vortex_array::{ArrayDType as _, ArrayData};
use vortex_error::{VortexError, VortexExpect as _};
use vortex_expr::datafusion::convert_expr_to_vortex;
use vortex_expr::ExprRef;

use crate::can_be_pushed_down;
use crate::memory::exec::VortexScanExec;
use crate::memory::plans::{RowSelectorExec, TakeRowsExec};

/// A [`TableProvider`] that exposes an existing Vortex Array to the DataFusion SQL engine.
///
/// Only arrays that have a top-level [struct type](vortex_dtype::StructDType) can be exposed as
/// a table to DataFusion.
#[derive(Debug, Clone)]
pub struct VortexMemTable {
    array: ChunkedArray,
    schema_ref: SchemaRef,
    options: VortexMemTableOptions,
}

impl VortexMemTable {
    /// Build a new table provider from an existing [struct type](vortex_dtype::StructDType) array.
    ///
    /// # Panics
    ///
    /// Creation will panic if the provided array is not of `DType::Struct` type.
    pub fn new(array: ArrayData, options: VortexMemTableOptions) -> Self {
        let arrow_schema = infer_schema(array.dtype()).vortex_expect("schema is inferable");
        let schema_ref = SchemaRef::new(arrow_schema);

        let array = match ChunkedArray::try_from(array.clone()) {
            Ok(a) => a,
            _ => {
                let dtype = array.dtype().clone();
                ChunkedArray::try_new(vec![array], dtype)
                    .vortex_expect("Failed to wrap array as a ChunkedArray with 1 chunk")
            }
        };

        Self {
            array,
            schema_ref,
            options,
        }
    }
}

#[async_trait]
impl TableProvider for VortexMemTable {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema_ref)
    }

    fn table_type(&self) -> TableType {
        TableType::Base
    }

    /// Plan an array scan.
    ///
    /// Currently, projection pushdown is supported, but not filter pushdown.
    /// The array is flattened directly into the nearest Arrow-compatible encoding.
    async fn scan(
        &self,
        state: &dyn Session,
        projection: Option<&Vec<usize>>,
        filters: &[Expr],
        _limit: Option<usize>,
    ) -> DFResult<Arc<dyn ExecutionPlan>> {
        let output_projection: Vec<usize> = match projection {
            None => (0..self.schema_ref.fields().len()).collect(),
            Some(proj) => proj.clone(),
        };

        match conjunction(filters.to_vec()) {
            // If there is a filter expression, we execute in two phases, first performing a filter
            // on the input to get back row indices, and then taking the remaining struct columns
            // using the calculated indices from the filter.
            Some(expr) => {
                let df_schema = self.schema_ref.clone().to_dfschema()?;

                let filter_expr = create_physical_expr(&expr, &df_schema, state.execution_props())?;
                let filter_expr = convert_expr_to_vortex(filter_expr)?;

                make_filter_then_take_plan(
                    self.schema_ref.clone(),
                    filter_expr,
                    self.array.clone(),
                    output_projection,
                    state,
                )
            }

            // If no filters were pushed down, we materialize the entire StructArray into a
            // RecordBatch and let DataFusion process the entire query.
            _ => {
                let output_schema = Arc::new(
                    self.schema_ref
                        .project(output_projection.as_slice())
                        .map_err(VortexError::from)?,
                );
                let plan_properties = PlanProperties::new(
                    EquivalenceProperties::new(output_schema),
                    // non-pushdown scans execute in single partition, where the partition
                    // yields one RecordBatch per chunk in the input ChunkedArray
                    Partitioning::UnknownPartitioning(1),
                    ExecutionMode::Bounded,
                );

                Ok(Arc::new(VortexScanExec::try_new(
                    self.array.clone(),
                    output_projection,
                    plan_properties,
                )?))
            }
        }
    }

    fn supports_filters_pushdown(
        &self,
        filters: &[&Expr],
    ) -> DFResult<Vec<TableProviderFilterPushDown>> {
        // In the case the caller has configured this provider with filter pushdown disabled,
        // do not attempt to apply any filters at scan time.
        if !self.options.enable_pushdown {
            return Ok(filters
                .iter()
                .map(|_| TableProviderFilterPushDown::Unsupported)
                .collect());
        }

        filters
            .iter()
            .map(|expr| {
                if can_be_pushed_down(expr, self.schema().as_ref()) {
                    Ok(TableProviderFilterPushDown::Exact)
                } else {
                    Ok(TableProviderFilterPushDown::Unsupported)
                }
            })
            .try_collect()
    }
}

/// Optional configurations to pass when loading a [VortexMemTable].
#[derive(Debug, Clone)]
pub struct VortexMemTableOptions {
    pub enable_pushdown: bool,
}

impl Default for VortexMemTableOptions {
    fn default() -> Self {
        Self {
            enable_pushdown: true,
        }
    }
}

impl VortexMemTableOptions {
    pub fn with_pushdown(mut self, enable_pushdown: bool) -> Self {
        self.enable_pushdown = enable_pushdown;
        self
    }
}

/// Construct an operator plan that executes in two stages.
///
/// The first plan stage only materializes the columns related to the provided set of filter
/// expressions. It evaluates the filters into a row selection.
///
/// The second stage receives the row selection above and dispatches a `take` on the remaining
/// columns.
fn make_filter_then_take_plan(
    schema: SchemaRef,
    filter_expr: ExprRef,
    chunked_array: ChunkedArray,
    output_projection: Vec<usize>,
    _session_state: &dyn Session,
) -> DFResult<Arc<dyn ExecutionPlan>> {
    let row_selector_op = Arc::new(RowSelectorExec::try_new(filter_expr, &chunked_array)?);

    Ok(Arc::new(TakeRowsExec::new(
        schema,
        &output_projection,
        row_selector_op,
        &chunked_array,
    )))
}

#[cfg(test)]
mod test {
    use arrow_array::cast::AsArray as _;
    use arrow_array::types::Int64Type;
    use arrow_schema::{DataType, Field, Schema};
    use datafusion::functions_aggregate::count::count_distinct;
    use datafusion::prelude::SessionContext;
    use datafusion_common::{Column, TableReference};
    use datafusion_expr::{and, col, lit, BinaryExpr, Expr, Operator};
    use vortex_array::array::{PrimitiveArray, StructArray, VarBinViewArray};
    use vortex_array::validity::Validity;
    use vortex_array::{ArrayData, IntoArrayData};

    use crate::memory::VortexMemTableOptions;
    use crate::{can_be_pushed_down, SessionContextExt as _};

    fn presidents_array() -> ArrayData {
        let names = VarBinViewArray::from_iter_str([
            "Washington",
            "Adams",
            "Jefferson",
            "Madison",
            "Monroe",
            "Adams",
        ]);
        let term_start = PrimitiveArray::from_vec(
            vec![1789u16, 1797, 1801, 1809, 1817, 1825],
            Validity::NonNullable,
        );

        StructArray::from_fields(&[
            ("president", names.into_array()),
            ("term_start", term_start.into_array()),
        ])
        .unwrap()
        .into_array()
    }

    #[tokio::test]
    #[cfg_attr(miri, ignore)]
    async fn test_datafusion_pushdown() {
        let ctx = SessionContext::new();

        let df = ctx.read_mem_vortex(presidents_array()).unwrap();

        let distinct_names = df
            .filter(col("term_start").gt_eq(lit(1795)))
            .unwrap()
            .aggregate(vec![], vec![count_distinct(col("president"))])
            .unwrap()
            .collect()
            .await
            .unwrap();

        assert_eq!(distinct_names.len(), 1);

        assert_eq!(
            *distinct_names[0]
                .column(0)
                .as_primitive::<Int64Type>()
                .values()
                .first()
                .unwrap(),
            4i64
        );
    }

    #[tokio::test]
    #[cfg_attr(miri, ignore)]
    async fn test_datafusion_no_pushdown() {
        let ctx = SessionContext::new();

        let df = ctx
            .read_mem_vortex_opts(
                presidents_array(),
                // Disable pushdown. We run this test to make sure that the naive codepath also
                // produces correct results and does not panic anywhere.
                VortexMemTableOptions::default().with_pushdown(false),
            )
            .unwrap();

        let distinct_names = df
            .filter(col("term_start").gt_eq(lit(1795)))
            .unwrap()
            .filter(col("term_start").lt(lit(2000)))
            .unwrap()
            .aggregate(vec![], vec![count_distinct(col("president"))])
            .unwrap()
            .collect()
            .await
            .unwrap();

        assert_eq!(distinct_names.len(), 1);

        assert_eq!(
            *distinct_names[0]
                .column(0)
                .as_primitive::<Int64Type>()
                .values()
                .first()
                .unwrap(),
            4i64
        );
    }

    #[test]
    fn test_can_be_pushed_down0() {
        let e = BinaryExpr {
            left: Box::new(
                Column {
                    relation: Some(TableReference::Bare {
                        table: "orders".into(),
                    }),
                    name: "o_orderstatus".to_string(),
                }
                .into(),
            ),
            op: Operator::Eq,
            right: Box::new(lit("F")),
        };
        let e = Expr::BinaryExpr(e);

        assert!(can_be_pushed_down(
            &e,
            &Schema::new(vec![Field::new("o_orderstatus", DataType::Utf8, true)])
        ));
    }

    #[test]
    fn test_can_be_pushed_down1() {
        let e = lit("hello");

        assert!(can_be_pushed_down(&e, &Schema::empty()));
    }

    #[test]
    fn test_can_be_pushed_down2() {
        let e = lit(3);

        assert!(can_be_pushed_down(&e, &Schema::empty()));
    }

    #[test]
    fn test_can_be_pushed_down3() {
        let e = BinaryExpr {
            left: Box::new(col("nums")),
            op: Operator::Modulo,
            right: Box::new(lit(5)),
        };
        let e = Expr::BinaryExpr(e);

        assert!(!can_be_pushed_down(
            &e,
            &Schema::new(vec![Field::new("nums", DataType::Int32, true)])
        ));
    }

    #[test]
    fn test_can_be_pushed_down4() {
        let e = and((col("a")).eq(lit(2u64)), col("b").eq(lit(true)));
        assert!(can_be_pushed_down(
            &e,
            &Schema::new(vec![
                Field::new("a", DataType::UInt64, true),
                Field::new("b", DataType::Boolean, true)
            ])
        ));
    }
}