Skip to main content

query/
datafusion.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Planner, QueryEngine implementations based on DataFusion.
16
17mod error;
18mod json_expr_planner;
19mod planner;
20
21use std::any::Any;
22use std::collections::HashMap;
23use std::sync::Arc;
24
25use async_trait::async_trait;
26use common_base::Plugins;
27use common_catalog::consts::is_readonly_schema;
28use common_error::ext::BoxedError;
29use common_function::function::FunctionContext;
30use common_function::function_factory::ScalarFunctionFactory;
31use common_query::{Output, OutputData, OutputMeta};
32use common_recordbatch::adapter::RecordBatchStreamAdapter;
33use common_recordbatch::{EmptyRecordBatchStream, SendableRecordBatchStream};
34use common_telemetry::tracing;
35use datafusion::catalog::TableFunction;
36use datafusion::dataframe::DataFrame;
37use datafusion::physical_plan::ExecutionPlan;
38use datafusion::physical_plan::analyze::AnalyzeExec;
39use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
40use datafusion_common::ResolvedTableReference;
41use datafusion_expr::{
42    AggregateUDF, DmlStatement, LogicalPlan as DfLogicalPlan, LogicalPlan, WindowUDF, WriteOp,
43};
44use datatypes::prelude::VectorRef;
45use datatypes::schema::Schema;
46use futures_util::StreamExt;
47use session::context::QueryContextRef;
48use snafu::{OptionExt, ResultExt, ensure};
49use sqlparser::ast::AnalyzeFormat;
50use table::TableRef;
51use table::requests::{DeleteRequest, InsertRequest};
52use tracing::Span;
53
54use crate::analyze::DistAnalyzeExec;
55pub use crate::datafusion::planner::DfContextProviderAdapter;
56use crate::dist_plan::{DistPlannerOptions, MergeScanLogicalPlan};
57use crate::error::{
58    CatalogSnafu, CreateRecordBatchSnafu, MissingTableMutationHandlerSnafu,
59    MissingTimestampColumnSnafu, QueryExecutionSnafu, Result, TableMutationSnafu,
60    TableNotFoundSnafu, TableReadOnlySnafu, UnsupportedExprSnafu,
61};
62use crate::executor::QueryExecutor;
63use crate::metrics::{
64    OnDone, QUERY_STAGE_ELAPSED, maybe_attach_region_watermark_metrics,
65    should_collect_region_watermark_from_query_ctx,
66};
67use crate::physical_wrapper::PhysicalPlanWrapperRef;
68use crate::planner::{DfLogicalPlanner, LogicalPlanner};
69use crate::query_engine::{DescribeResult, QueryEngineContext, QueryEngineState};
70use crate::{QueryEngine, metrics};
71
72/// Query parallelism hint key.
73/// This hint can be set in the query context to control the parallelism of the query execution.
74pub const QUERY_PARALLELISM_HINT: &str = "query_parallelism";
75
76/// Whether to fallback to the original plan when failed to push down.
77pub const QUERY_FALLBACK_HINT: &str = "query_fallback";
78
79pub struct DatafusionQueryEngine {
80    state: Arc<QueryEngineState>,
81    plugins: Plugins,
82}
83
84impl DatafusionQueryEngine {
85    pub fn new(state: Arc<QueryEngineState>, plugins: Plugins) -> Self {
86        Self { state, plugins }
87    }
88
89    #[tracing::instrument(skip_all)]
90    async fn exec_query_plan(
91        &self,
92        plan: LogicalPlan,
93        query_ctx: QueryContextRef,
94    ) -> Result<Output> {
95        let mut ctx = self.engine_context(query_ctx.clone());
96
97        // `create_physical_plan` will optimize logical plan internally
98        let physical_plan = self.create_physical_plan(&mut ctx, &plan).await?;
99        let optimized_physical_plan = self.optimize_physical_plan(&mut ctx, physical_plan)?;
100
101        let physical_plan = if let Some(wrapper) = self.plugins.get::<PhysicalPlanWrapperRef>() {
102            wrapper.wrap(optimized_physical_plan, query_ctx)
103        } else {
104            optimized_physical_plan
105        };
106
107        let stream = self.execute_stream(&ctx, &physical_plan)?;
108
109        Ok(Output::new(
110            OutputData::Stream(stream),
111            OutputMeta::new_with_plan(physical_plan),
112        ))
113    }
114
115    #[tracing::instrument(skip_all)]
116    async fn exec_dml_statement(
117        &self,
118        dml: DmlStatement,
119        query_ctx: QueryContextRef,
120    ) -> Result<Output> {
121        ensure!(
122            matches!(dml.op, WriteOp::Insert(_) | WriteOp::Delete),
123            UnsupportedExprSnafu {
124                name: format!("DML op {}", dml.op),
125            }
126        );
127
128        let _timer = QUERY_STAGE_ELAPSED
129            .with_label_values(&[dml.op.name()])
130            .start_timer();
131
132        let default_catalog = &query_ctx.current_catalog().to_owned();
133        let default_schema = &query_ctx.current_schema();
134        let table_name = dml.table_name.resolve(default_catalog, default_schema);
135        let table = self.find_table(&table_name, &query_ctx).await?;
136
137        let Output { data, meta } = self
138            .exec_query_plan((*dml.input).clone(), query_ctx.clone())
139            .await?;
140        let mut stream = match data {
141            OutputData::RecordBatches(batches) => batches.as_stream(),
142            OutputData::Stream(stream) => stream,
143            _ => unreachable!(),
144        };
145
146        let mut affected_rows = 0;
147        let mut insert_cost = 0;
148
149        while let Some(batch) = stream.next().await {
150            let batch = batch.context(CreateRecordBatchSnafu)?;
151            let column_vectors = batch
152                .column_vectors(&table_name.to_string(), table.schema())
153                .map_err(BoxedError::new)
154                .context(QueryExecutionSnafu)?;
155
156            match dml.op {
157                WriteOp::Insert(_) => {
158                    // We ignore the insert op.
159                    let output = self
160                        .insert(&table_name, column_vectors, query_ctx.clone())
161                        .await?;
162                    let (rows, cost) = output.extract_rows_and_cost();
163                    affected_rows += rows;
164                    insert_cost += cost;
165                }
166                WriteOp::Delete => {
167                    affected_rows += self
168                        .delete(&table_name, &table, column_vectors, query_ctx.clone())
169                        .await?;
170                }
171                _ => unreachable!("guarded by the 'ensure!' at the beginning"),
172            }
173        }
174        Ok(Output::new(
175            OutputData::AffectedRows(affected_rows),
176            OutputMeta::new(meta.plan, insert_cost),
177        ))
178    }
179
180    #[tracing::instrument(skip_all)]
181    async fn delete(
182        &self,
183        table_name: &ResolvedTableReference,
184        table: &TableRef,
185        column_vectors: HashMap<String, VectorRef>,
186        query_ctx: QueryContextRef,
187    ) -> Result<usize> {
188        let catalog_name = table_name.catalog.to_string();
189        let schema_name = table_name.schema.to_string();
190        let table_name = table_name.table.to_string();
191        let table_schema = table.schema();
192
193        ensure!(
194            !is_readonly_schema(&schema_name),
195            TableReadOnlySnafu { table: table_name }
196        );
197
198        let ts_column = table_schema
199            .timestamp_column()
200            .map(|x| &x.name)
201            .with_context(|| MissingTimestampColumnSnafu {
202                table_name: table_name.clone(),
203            })?;
204
205        let table_info = table.table_info();
206        let rowkey_columns = table_info
207            .meta
208            .row_key_column_names()
209            .collect::<Vec<&String>>();
210        let column_vectors = column_vectors
211            .into_iter()
212            .filter(|x| &x.0 == ts_column || rowkey_columns.contains(&&x.0))
213            .collect::<HashMap<_, _>>();
214
215        let request = DeleteRequest {
216            catalog_name,
217            schema_name,
218            table_name,
219            key_column_values: column_vectors,
220        };
221
222        self.state
223            .table_mutation_handler()
224            .context(MissingTableMutationHandlerSnafu)?
225            .delete(request, query_ctx)
226            .await
227            .context(TableMutationSnafu)
228    }
229
230    #[tracing::instrument(skip_all)]
231    async fn insert(
232        &self,
233        table_name: &ResolvedTableReference,
234        column_vectors: HashMap<String, VectorRef>,
235        query_ctx: QueryContextRef,
236    ) -> Result<Output> {
237        let catalog_name = table_name.catalog.to_string();
238        let schema_name = table_name.schema.to_string();
239        let table_name = table_name.table.to_string();
240
241        ensure!(
242            !is_readonly_schema(&schema_name),
243            TableReadOnlySnafu { table: table_name }
244        );
245
246        let request = InsertRequest {
247            catalog_name,
248            schema_name,
249            table_name,
250            columns_values: column_vectors,
251        };
252
253        self.state
254            .table_mutation_handler()
255            .context(MissingTableMutationHandlerSnafu)?
256            .insert(request, query_ctx)
257            .await
258            .context(TableMutationSnafu)
259    }
260
261    async fn find_table(
262        &self,
263        table_name: &ResolvedTableReference,
264        query_context: &QueryContextRef,
265    ) -> Result<TableRef> {
266        let catalog_name = table_name.catalog.as_ref();
267        let schema_name = table_name.schema.as_ref();
268        let table_name = table_name.table.as_ref();
269
270        self.state
271            .catalog_manager()
272            .table(catalog_name, schema_name, table_name, Some(query_context))
273            .await
274            .context(CatalogSnafu)?
275            .with_context(|| TableNotFoundSnafu { table: table_name })
276    }
277
278    #[tracing::instrument(skip_all)]
279    async fn create_physical_plan(
280        &self,
281        ctx: &mut QueryEngineContext,
282        logical_plan: &LogicalPlan,
283    ) -> Result<Arc<dyn ExecutionPlan>> {
284        /// Only print context on panic, to avoid cluttering logs.
285        ///
286        /// TODO(discord9): remove this once we catch the bug
287        #[derive(Debug)]
288        struct PanicLogger<'a> {
289            input_logical_plan: &'a LogicalPlan,
290            after_analyze: Option<LogicalPlan>,
291            after_optimize: Option<LogicalPlan>,
292            phy_plan: Option<Arc<dyn ExecutionPlan>>,
293        }
294        impl Drop for PanicLogger<'_> {
295            fn drop(&mut self) {
296                if std::thread::panicking() {
297                    common_telemetry::error!(
298                        "Panic while creating physical plan, input logical plan: {:?}, after analyze: {:?}, after optimize: {:?}, final physical plan: {:?}",
299                        self.input_logical_plan,
300                        self.after_analyze,
301                        self.after_optimize,
302                        self.phy_plan
303                    );
304                }
305            }
306        }
307
308        let mut logger = PanicLogger {
309            input_logical_plan: logical_plan,
310            after_analyze: None,
311            after_optimize: None,
312            phy_plan: None,
313        };
314
315        let _timer = metrics::CREATE_PHYSICAL_ELAPSED.start_timer();
316        let state = ctx.state();
317
318        common_telemetry::debug!("Create physical plan, input plan: {logical_plan}");
319
320        // special handle EXPLAIN plan
321        if matches!(logical_plan, DfLogicalPlan::Explain(_)) {
322            return state
323                .create_physical_plan(logical_plan)
324                .await
325                .map_err(Into::into);
326        }
327
328        // analyze first
329        let analyzed_plan = state.analyzer().execute_and_check(
330            logical_plan.clone(),
331            state.config_options(),
332            |_, _| {},
333        )?;
334
335        logger.after_analyze = Some(analyzed_plan.clone());
336
337        common_telemetry::debug!("Create physical plan, analyzed plan: {analyzed_plan}");
338
339        // skip optimize for MergeScan
340        let optimized_plan = if let DfLogicalPlan::Extension(ext) = &analyzed_plan
341            && ext.node.name() == MergeScanLogicalPlan::name()
342        {
343            analyzed_plan.clone()
344        } else {
345            state
346                .optimizer()
347                .optimize(analyzed_plan, state, |_, _| {})?
348        };
349
350        common_telemetry::debug!("Create physical plan, optimized plan: {optimized_plan}");
351        logger.after_optimize = Some(optimized_plan.clone());
352
353        let physical_plan = state
354            .query_planner()
355            .create_physical_plan(&optimized_plan, state)
356            .await?;
357
358        logger.phy_plan = Some(physical_plan.clone());
359        drop(logger);
360        Ok(physical_plan)
361    }
362
363    #[tracing::instrument(skip_all)]
364    fn optimize_physical_plan(
365        &self,
366        ctx: &mut QueryEngineContext,
367        plan: Arc<dyn ExecutionPlan>,
368    ) -> Result<Arc<dyn ExecutionPlan>> {
369        let _timer = metrics::OPTIMIZE_PHYSICAL_ELAPSED.start_timer();
370
371        // TODO(ruihang): `self.create_physical_plan()` already optimize the plan, check
372        // if we need to optimize it again here.
373        // let state = ctx.state();
374        // let config = state.config_options();
375
376        // skip optimize AnalyzeExec plan
377        let optimized_plan = if let Some(analyze_plan) = plan.as_any().downcast_ref::<AnalyzeExec>()
378        {
379            let format = if let Some(format) = ctx.query_ctx().explain_format()
380                && format.to_lowercase() == "json"
381            {
382                AnalyzeFormat::JSON
383            } else {
384                AnalyzeFormat::TEXT
385            };
386            // Sets the verbose flag of the query context.
387            // The MergeScanExec plan uses the verbose flag to determine whether to print the plan in verbose mode.
388            ctx.query_ctx().set_explain_verbose(analyze_plan.verbose());
389
390            Arc::new(DistAnalyzeExec::new(
391                analyze_plan.input().clone(),
392                analyze_plan.verbose(),
393                format,
394            ))
395            // let mut new_plan = analyze_plan.input().clone();
396            // for optimizer in state.physical_optimizers() {
397            //     new_plan = optimizer
398            //         .optimize(new_plan, config)
399            //         .context(DataFusionSnafu)?;
400            // }
401            // Arc::new(DistAnalyzeExec::new(new_plan))
402        } else {
403            plan
404            // let mut new_plan = plan;
405            // for optimizer in state.physical_optimizers() {
406            //     new_plan = optimizer
407            //         .optimize(new_plan, config)
408            //         .context(DataFusionSnafu)?;
409            // }
410            // new_plan
411        };
412
413        Ok(optimized_plan)
414    }
415}
416
417#[async_trait]
418impl QueryEngine for DatafusionQueryEngine {
419    fn as_any(&self) -> &dyn Any {
420        self
421    }
422
423    fn planner(&self) -> Arc<dyn LogicalPlanner> {
424        Arc::new(DfLogicalPlanner::new(self.state.clone()))
425    }
426
427    fn name(&self) -> &str {
428        "datafusion"
429    }
430
431    async fn describe(
432        &self,
433        plan: LogicalPlan,
434        _query_ctx: QueryContextRef,
435    ) -> Result<DescribeResult> {
436        Ok(DescribeResult { logical_plan: plan })
437    }
438
439    async fn execute(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result<Output> {
440        match plan {
441            LogicalPlan::Dml(dml) => self.exec_dml_statement(dml, query_ctx).await,
442            _ => self.exec_query_plan(plan, query_ctx).await,
443        }
444    }
445
446    /// Note in SQL queries, aggregate names are looked up using
447    /// lowercase unless the query uses quotes. For example,
448    ///
449    /// `SELECT MY_UDAF(x)...` will look for an aggregate named `"my_udaf"`
450    /// `SELECT "my_UDAF"(x)` will look for an aggregate named `"my_UDAF"`
451    ///
452    /// So it's better to make UDAF name lowercase when creating one.
453    fn register_aggregate_function(&self, func: AggregateUDF) {
454        self.state.register_aggr_function(func);
455    }
456
457    /// Register an scalar function.
458    /// Will override if the function with same name is already registered.
459    fn register_scalar_function(&self, func: ScalarFunctionFactory) {
460        self.state.register_scalar_function(func);
461    }
462
463    fn register_table_function(&self, func: Arc<TableFunction>) {
464        self.state.register_table_function(func);
465    }
466
467    fn register_window_function(&self, func: WindowUDF) {
468        self.state.register_window_function(func);
469    }
470
471    fn read_table(&self, table: TableRef) -> Result<DataFrame> {
472        self.state.read_table(table).map_err(Into::into)
473    }
474
475    fn engine_context(&self, query_ctx: QueryContextRef) -> QueryEngineContext {
476        let mut state = self.state.session_state();
477        state.config_mut().set_extension(query_ctx.clone());
478        // note that hints in "x-greptime-hints" is automatically parsed
479        // and set to query context's extension, so we can get it from query context.
480        if let Some(parallelism) = query_ctx.extension(QUERY_PARALLELISM_HINT) {
481            if let Ok(n) = parallelism.parse::<u64>() {
482                if n > 0 {
483                    let new_cfg = state.config().clone().with_target_partitions(n as usize);
484                    *state.config_mut() = new_cfg;
485                }
486            } else {
487                common_telemetry::warn!(
488                    "Failed to parse query_parallelism: {}, using default value",
489                    parallelism
490                );
491            }
492        }
493
494        // configure execution options
495        state.config_mut().options_mut().execution.time_zone =
496            Some(query_ctx.timezone().to_string());
497
498        // usually it's impossible to have both `set variable` set by sql client and
499        // hint in header by grpc client, so only need to deal with them separately
500        if query_ctx.configuration_parameter().allow_query_fallback() {
501            state
502                .config_mut()
503                .options_mut()
504                .extensions
505                .insert(DistPlannerOptions {
506                    allow_query_fallback: true,
507                });
508        } else if let Some(fallback) = query_ctx.extension(QUERY_FALLBACK_HINT) {
509            // also check the query context for fallback hint
510            // if it is set, we will enable the fallback
511            if fallback.to_lowercase().parse::<bool>().unwrap_or(false) {
512                state
513                    .config_mut()
514                    .options_mut()
515                    .extensions
516                    .insert(DistPlannerOptions {
517                        allow_query_fallback: true,
518                    });
519            }
520        }
521
522        state
523            .config_mut()
524            .options_mut()
525            .extensions
526            .insert(FunctionContext {
527                query_ctx: query_ctx.clone(),
528                state: self.engine_state().function_state(),
529            });
530
531        let config_options = state.config_options().clone();
532        let _ = state
533            .execution_props_mut()
534            .config_options
535            .insert(config_options);
536
537        QueryEngineContext::new(state, query_ctx)
538    }
539
540    fn engine_state(&self) -> &QueryEngineState {
541        &self.state
542    }
543}
544
545impl QueryExecutor for DatafusionQueryEngine {
546    #[tracing::instrument(skip_all)]
547    fn execute_stream(
548        &self,
549        ctx: &QueryEngineContext,
550        plan: &Arc<dyn ExecutionPlan>,
551    ) -> Result<SendableRecordBatchStream> {
552        let query_ctx = ctx.query_ctx();
553        let explain_verbose = query_ctx.explain_verbose();
554        let should_collect_region_watermark =
555            should_collect_region_watermark_from_query_ctx(&query_ctx)?;
556        let output_partitions = plan.properties().output_partitioning().partition_count();
557        if explain_verbose {
558            common_telemetry::info!("Executing query plan, output_partitions: {output_partitions}");
559        }
560
561        let exec_timer = metrics::EXEC_PLAN_ELAPSED.start_timer();
562        let task_ctx = ctx.build_task_ctx();
563        let span = Span::current();
564
565        match plan.properties().output_partitioning().partition_count() {
566            0 => {
567                let schema = Arc::new(
568                    Schema::try_from(plan.schema())
569                        .map_err(BoxedError::new)
570                        .context(QueryExecutionSnafu)?,
571                );
572                Ok(Box::pin(EmptyRecordBatchStream::new(schema)))
573            }
574            1 => {
575                let df_stream = plan.execute(0, task_ctx)?;
576                let mut stream = RecordBatchStreamAdapter::try_new_with_span(df_stream, span)
577                    .context(error::ConvertDfRecordBatchStreamSnafu)
578                    .map_err(BoxedError::new)
579                    .context(QueryExecutionSnafu)?;
580                stream.set_metrics2(plan.clone());
581                stream.set_explain_verbose(explain_verbose);
582                let stream = OnDone::new(Box::pin(stream), move || {
583                    let exec_cost = exec_timer.stop_and_record();
584                    if explain_verbose {
585                        common_telemetry::info!(
586                            "DatafusionQueryEngine execute 1 stream, cost: {:?}s",
587                            exec_cost,
588                        );
589                    }
590                });
591                Ok(maybe_attach_region_watermark_metrics(
592                    Box::pin(stream),
593                    plan.clone(),
594                    should_collect_region_watermark,
595                ))
596            }
597            _ => {
598                // merge into a single partition
599                let merged_plan = CoalescePartitionsExec::new(plan.clone());
600                // CoalescePartitionsExec must produce a single partition
601                assert_eq!(
602                    1,
603                    merged_plan
604                        .properties()
605                        .output_partitioning()
606                        .partition_count()
607                );
608                let df_stream = merged_plan.execute(0, task_ctx)?;
609                let mut stream = RecordBatchStreamAdapter::try_new_with_span(df_stream, span)
610                    .context(error::ConvertDfRecordBatchStreamSnafu)
611                    .map_err(BoxedError::new)
612                    .context(QueryExecutionSnafu)?;
613                stream.set_metrics2(plan.clone());
614                stream.set_explain_verbose(explain_verbose);
615                let stream = OnDone::new(Box::pin(stream), move || {
616                    let exec_cost = exec_timer.stop_and_record();
617                    if explain_verbose {
618                        common_telemetry::info!(
619                            "DatafusionQueryEngine execute {output_partitions} stream, cost: {:?}s",
620                            exec_cost
621                        );
622                    }
623                });
624                Ok(maybe_attach_region_watermark_metrics(
625                    Box::pin(stream),
626                    plan.clone(),
627                    should_collect_region_watermark,
628                ))
629            }
630        }
631    }
632}
633
634#[cfg(test)]
635mod tests {
636    use std::fmt;
637    use std::sync::Arc;
638    use std::sync::atomic::{AtomicUsize, Ordering};
639
640    use api::v1::SemanticType;
641    use arrow::array::{ArrayRef, UInt64Array};
642    use arrow_schema::SortOptions;
643    use catalog::RegisterTableRequest;
644    use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, NUMBERS_TABLE_ID};
645    use common_error::ext::BoxedError;
646    use common_recordbatch::{EmptyRecordBatchStream, SendableRecordBatchStream, util};
647    use datafusion::physical_plan::display::{DisplayAs, DisplayFormatType};
648    use datafusion::physical_plan::expressions::PhysicalSortExpr;
649    use datafusion::physical_plan::joins::{HashJoinExec, JoinOn, PartitionMode};
650    use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
651    use datafusion::physical_plan::{ExecutionPlan, PhysicalExpr};
652    use datafusion::prelude::{col, lit};
653    use datafusion_common::{JoinType, NullEquality};
654    use datafusion_physical_expr::expressions::Column;
655    use datatypes::prelude::ConcreteDataType;
656    use datatypes::schema::{ColumnSchema, SchemaRef};
657    use datatypes::vectors::{Helper, UInt32Vector, VectorRef};
658    use session::context::{QueryContext, QueryContextBuilder};
659    use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder, RegionMetadataRef};
660    use store_api::region_engine::{
661        PartitionRange, PrepareRequest, QueryScanContext, RegionScanner, ScannerProperties,
662    };
663    use store_api::storage::{RegionId, ScanRequest};
664    use table::table::numbers::{NUMBERS_TABLE_NAME, NumbersTable};
665    use table::table::scan::RegionScanExec;
666
667    use super::*;
668    use crate::options::QueryOptions;
669    use crate::parser::QueryLanguageParser;
670    use crate::part_sort::PartSortExec;
671    use crate::query_engine::{QueryEngineFactory, QueryEngineRef};
672
673    #[derive(Debug)]
674    struct RecordingScanner {
675        schema: SchemaRef,
676        metadata: RegionMetadataRef,
677        properties: ScannerProperties,
678        update_calls: Arc<AtomicUsize>,
679        last_filter_len: Arc<AtomicUsize>,
680    }
681
682    impl RecordingScanner {
683        fn new(
684            schema: SchemaRef,
685            metadata: RegionMetadataRef,
686            update_calls: Arc<AtomicUsize>,
687            last_filter_len: Arc<AtomicUsize>,
688        ) -> Self {
689            Self {
690                schema,
691                metadata,
692                properties: ScannerProperties::default(),
693                update_calls,
694                last_filter_len,
695            }
696        }
697    }
698
699    impl RegionScanner for RecordingScanner {
700        fn name(&self) -> &str {
701            "RecordingScanner"
702        }
703
704        fn properties(&self) -> &ScannerProperties {
705            &self.properties
706        }
707
708        fn schema(&self) -> SchemaRef {
709            self.schema.clone()
710        }
711
712        fn metadata(&self) -> RegionMetadataRef {
713            self.metadata.clone()
714        }
715
716        fn prepare(&mut self, request: PrepareRequest) -> std::result::Result<(), BoxedError> {
717            self.properties.prepare(request);
718            Ok(())
719        }
720
721        fn scan_partition(
722            &self,
723            _ctx: &QueryScanContext,
724            _metrics_set: &ExecutionPlanMetricsSet,
725            _partition: usize,
726        ) -> std::result::Result<SendableRecordBatchStream, BoxedError> {
727            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
728        }
729
730        fn has_predicate_without_region(&self) -> bool {
731            true
732        }
733
734        fn add_dyn_filter_to_predicate(
735            &mut self,
736            filter_exprs: Vec<Arc<dyn PhysicalExpr>>,
737        ) -> Vec<bool> {
738            self.update_calls.fetch_add(1, Ordering::Relaxed);
739            self.last_filter_len
740                .store(filter_exprs.len(), Ordering::Relaxed);
741            vec![true; filter_exprs.len()]
742        }
743
744        fn set_logical_region(&mut self, logical_region: bool) {
745            self.properties.set_logical_region(logical_region);
746        }
747    }
748
749    impl DisplayAs for RecordingScanner {
750        fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result {
751            write!(f, "RecordingScanner")
752        }
753    }
754
755    async fn create_test_engine() -> QueryEngineRef {
756        let catalog_manager = catalog::memory::new_memory_catalog_manager().unwrap();
757        let req = RegisterTableRequest {
758            catalog: DEFAULT_CATALOG_NAME.to_string(),
759            schema: DEFAULT_SCHEMA_NAME.to_string(),
760            table_name: NUMBERS_TABLE_NAME.to_string(),
761            table_id: NUMBERS_TABLE_ID,
762            table: NumbersTable::table(NUMBERS_TABLE_ID),
763        };
764        catalog_manager.register_table_sync(req).unwrap();
765
766        QueryEngineFactory::new(
767            catalog_manager,
768            None,
769            None,
770            None,
771            None,
772            false,
773            QueryOptions::default(),
774        )
775        .query_engine()
776    }
777
778    #[tokio::test]
779    async fn test_sql_to_plan() {
780        let engine = create_test_engine().await;
781        let sql = "select sum(number) from numbers limit 20";
782
783        let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
784        let plan = engine
785            .planner()
786            .plan(&stmt, QueryContext::arc())
787            .await
788            .unwrap();
789
790        assert_eq!(
791            plan.to_string(),
792            r#"Limit: skip=0, fetch=20
793  Projection: sum(numbers.number)
794    Aggregate: groupBy=[[]], aggr=[[sum(numbers.number)]]
795      TableScan: numbers"#
796        );
797    }
798
799    #[tokio::test]
800    async fn test_execute() {
801        let engine = create_test_engine().await;
802        let sql = "select sum(number) from numbers limit 20";
803
804        let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
805        let plan = engine
806            .planner()
807            .plan(&stmt, QueryContext::arc())
808            .await
809            .unwrap();
810
811        let output = engine.execute(plan, QueryContext::arc()).await.unwrap();
812
813        match output.data {
814            OutputData::Stream(recordbatch) => {
815                let numbers = util::collect(recordbatch).await.unwrap();
816                assert_eq!(1, numbers.len());
817                assert_eq!(numbers[0].num_columns(), 1);
818                assert_eq!(1, numbers[0].schema.num_columns());
819                assert_eq!(
820                    "sum(numbers.number)",
821                    numbers[0].schema.column_schemas()[0].name
822                );
823
824                let batch = &numbers[0];
825                assert_eq!(1, batch.num_columns());
826                assert_eq!(batch.column(0).len(), 1);
827
828                let expected = Arc::new(UInt64Array::from_iter_values([4950])) as ArrayRef;
829                assert_eq!(batch.column(0), &expected);
830            }
831            _ => unreachable!(),
832        }
833    }
834
835    #[tokio::test]
836    async fn test_read_table() {
837        let engine = create_test_engine().await;
838
839        let engine = engine
840            .as_any()
841            .downcast_ref::<DatafusionQueryEngine>()
842            .unwrap();
843        let query_ctx = Arc::new(QueryContextBuilder::default().build());
844        let table = engine
845            .find_table(
846                &ResolvedTableReference {
847                    catalog: "greptime".into(),
848                    schema: "public".into(),
849                    table: "numbers".into(),
850                },
851                &query_ctx,
852            )
853            .await
854            .unwrap();
855
856        let df = engine.read_table(table).unwrap();
857        let df = df
858            .select_columns(&["number"])
859            .unwrap()
860            .filter(col("number").lt(lit(10)))
861            .unwrap();
862        let batches = df.collect().await.unwrap();
863        assert_eq!(1, batches.len());
864        let batch = &batches[0];
865
866        assert_eq!(1, batch.num_columns());
867        assert_eq!(batch.column(0).len(), 10);
868
869        assert_eq!(
870            Helper::try_into_vector(batch.column(0)).unwrap(),
871            Arc::new(UInt32Vector::from_slice([0, 1, 2, 3, 4, 5, 6, 7, 8, 9])) as VectorRef
872        );
873    }
874
875    #[tokio::test]
876    async fn test_describe() {
877        let engine = create_test_engine().await;
878        let sql = "select sum(number) from numbers limit 20";
879
880        let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
881
882        let plan = engine
883            .planner()
884            .plan(&stmt, QueryContext::arc())
885            .await
886            .unwrap();
887
888        let DescribeResult { logical_plan } =
889            engine.describe(plan, QueryContext::arc()).await.unwrap();
890
891        let schema: Schema = logical_plan.schema().clone().try_into().unwrap();
892
893        assert_eq!(
894            schema.column_schemas()[0],
895            ColumnSchema::new(
896                "sum(numbers.number)",
897                ConcreteDataType::uint64_datatype(),
898                true
899            )
900        );
901        assert_eq!(
902            "Limit: skip=0, fetch=20\n  Projection: sum(numbers.number)\n    Aggregate: groupBy=[[]], aggr=[[sum(numbers.number)]]\n      TableScan: numbers",
903            format!("{}", logical_plan.display_indent())
904        );
905    }
906
907    #[tokio::test]
908    async fn test_topk_dynamic_filter_pushdown_reaches_region_scan() {
909        let engine = create_test_engine().await;
910        let engine = engine
911            .as_any()
912            .downcast_ref::<DatafusionQueryEngine>()
913            .unwrap();
914        let engine_ctx = engine.engine_context(QueryContext::arc());
915        let state = engine_ctx.state();
916
917        let schema = Arc::new(datatypes::schema::Schema::new(vec![ColumnSchema::new(
918            "ts",
919            ConcreteDataType::timestamp_millisecond_datatype(),
920            false,
921        )]));
922
923        let mut metadata_builder = RegionMetadataBuilder::new(RegionId::new(1024, 1));
924        metadata_builder
925            .push_column_metadata(ColumnMetadata {
926                column_schema: ColumnSchema::new(
927                    "ts",
928                    ConcreteDataType::timestamp_millisecond_datatype(),
929                    false,
930                )
931                .with_time_index(true),
932                semantic_type: SemanticType::Timestamp,
933                column_id: 1,
934            })
935            .primary_key(vec![]);
936        let metadata = Arc::new(metadata_builder.build().unwrap());
937
938        let update_calls = Arc::new(AtomicUsize::new(0));
939        let last_filter_len = Arc::new(AtomicUsize::new(0));
940        let scanner = Box::new(RecordingScanner::new(
941            schema,
942            metadata,
943            update_calls.clone(),
944            last_filter_len.clone(),
945        ));
946        let scan = Arc::new(RegionScanExec::new(scanner, ScanRequest::default(), None).unwrap());
947
948        let sort_expr = PhysicalSortExpr {
949            expr: Arc::new(Column::new("ts", 0)),
950            options: SortOptions {
951                descending: true,
952                ..Default::default()
953            },
954        };
955        let partition_ranges: Vec<Vec<PartitionRange>> = vec![vec![]];
956        let mut plan: Arc<dyn ExecutionPlan> =
957            Arc::new(PartSortExec::try_new(sort_expr, Some(3), partition_ranges, scan).unwrap());
958
959        for optimizer in state.physical_optimizers() {
960            plan = optimizer.optimize(plan, state.config_options()).unwrap();
961        }
962
963        assert!(update_calls.load(Ordering::Relaxed) > 0);
964        assert!(last_filter_len.load(Ordering::Relaxed) > 0);
965    }
966
967    #[tokio::test]
968    async fn test_join_dynamic_filter_pushdown_reaches_region_scan() {
969        let engine = create_test_engine().await;
970        let engine = engine
971            .as_any()
972            .downcast_ref::<DatafusionQueryEngine>()
973            .unwrap();
974        let engine_ctx = engine.engine_context(QueryContext::arc());
975        let state = engine_ctx.state();
976
977        assert!(
978            state
979                .config_options()
980                .optimizer
981                .enable_join_dynamic_filter_pushdown
982        );
983
984        let schema = Arc::new(datatypes::schema::Schema::new(vec![ColumnSchema::new(
985            "ts",
986            ConcreteDataType::timestamp_millisecond_datatype(),
987            false,
988        )]));
989
990        let mut left_metadata_builder = RegionMetadataBuilder::new(RegionId::new(2048, 1));
991        left_metadata_builder
992            .push_column_metadata(ColumnMetadata {
993                column_schema: ColumnSchema::new(
994                    "ts",
995                    ConcreteDataType::timestamp_millisecond_datatype(),
996                    false,
997                )
998                .with_time_index(true),
999                semantic_type: SemanticType::Timestamp,
1000                column_id: 1,
1001            })
1002            .primary_key(vec![]);
1003        let left_metadata = Arc::new(left_metadata_builder.build().unwrap());
1004
1005        let mut right_metadata_builder = RegionMetadataBuilder::new(RegionId::new(2048, 2));
1006        right_metadata_builder
1007            .push_column_metadata(ColumnMetadata {
1008                column_schema: ColumnSchema::new(
1009                    "ts",
1010                    ConcreteDataType::timestamp_millisecond_datatype(),
1011                    false,
1012                )
1013                .with_time_index(true),
1014                semantic_type: SemanticType::Timestamp,
1015                column_id: 1,
1016            })
1017            .primary_key(vec![]);
1018        let right_metadata = Arc::new(right_metadata_builder.build().unwrap());
1019
1020        let left_update_calls = Arc::new(AtomicUsize::new(0));
1021        let left_last_filter_len = Arc::new(AtomicUsize::new(0));
1022        let right_update_calls = Arc::new(AtomicUsize::new(0));
1023        let right_last_filter_len = Arc::new(AtomicUsize::new(0));
1024
1025        let left_scan = Arc::new(
1026            RegionScanExec::new(
1027                Box::new(RecordingScanner::new(
1028                    schema.clone(),
1029                    left_metadata,
1030                    left_update_calls.clone(),
1031                    left_last_filter_len.clone(),
1032                )),
1033                ScanRequest::default(),
1034                None,
1035            )
1036            .unwrap(),
1037        );
1038        let right_scan = Arc::new(
1039            RegionScanExec::new(
1040                Box::new(RecordingScanner::new(
1041                    schema,
1042                    right_metadata,
1043                    right_update_calls.clone(),
1044                    right_last_filter_len.clone(),
1045                )),
1046                ScanRequest::default(),
1047                None,
1048            )
1049            .unwrap(),
1050        );
1051
1052        let on: JoinOn = vec![(
1053            Arc::new(Column::new("ts", 0)) as Arc<dyn PhysicalExpr>,
1054            Arc::new(Column::new("ts", 0)) as Arc<dyn PhysicalExpr>,
1055        )];
1056
1057        let mut plan: Arc<dyn ExecutionPlan> = Arc::new(
1058            HashJoinExec::try_new(
1059                left_scan,
1060                right_scan,
1061                on,
1062                None,
1063                &JoinType::Inner,
1064                None,
1065                PartitionMode::CollectLeft,
1066                NullEquality::NullEqualsNull,
1067                false,
1068            )
1069            .unwrap(),
1070        );
1071
1072        for optimizer in state.physical_optimizers() {
1073            plan = optimizer.optimize(plan, state.config_options()).unwrap();
1074        }
1075
1076        assert!(left_update_calls.load(Ordering::Relaxed) > 0);
1077        assert_eq!(0, left_last_filter_len.load(Ordering::Relaxed));
1078        assert!(right_update_calls.load(Ordering::Relaxed) > 0);
1079        assert!(right_last_filter_len.load(Ordering::Relaxed) > 0);
1080    }
1081}