datafusion_expr/
planner.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! [`ContextProvider`] and [`ExprPlanner`] APIs to customize SQL query planning
19
20use std::fmt::Debug;
21use std::sync::Arc;
22
23use crate::expr::NullTreatment;
24#[cfg(feature = "sql")]
25use crate::logical_plan::LogicalPlan;
26use crate::{
27    AggregateUDF, Expr, GetFieldAccess, ScalarUDF, SortExpr, TableSource, WindowFrame,
28    WindowFunctionDefinition, WindowUDF,
29};
30use arrow::datatypes::{DataType, Field, FieldRef, SchemaRef};
31use datafusion_common::datatype::DataTypeExt;
32use datafusion_common::{
33    DFSchema, Result, TableReference, config::ConfigOptions,
34    file_options::file_type::FileType, not_impl_err,
35};
36#[cfg(feature = "sql")]
37use sqlparser::ast::{Expr as SQLExpr, Ident, ObjectName, TableAlias, TableFactor};
38
39/// Provides the `SQL` query planner meta-data about tables and
40/// functions referenced in SQL statements, without a direct dependency on the
41/// `datafusion` Catalog structures such as [`TableProvider`]
42///
43/// [`TableProvider`]: https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html
44pub trait ContextProvider {
45    /// Returns a table by reference, if it exists
46    fn get_table_source(&self, name: TableReference) -> Result<Arc<dyn TableSource>>;
47
48    /// Return the type of a file based on its extension (e.g. `.parquet`)
49    ///
50    /// This is used to plan `COPY` statements
51    fn get_file_type(&self, _ext: &str) -> Result<Arc<dyn FileType>> {
52        not_impl_err!("Registered file types are not supported")
53    }
54
55    /// Getter for a table function
56    fn get_table_function_source(
57        &self,
58        _name: &str,
59        _args: Vec<Expr>,
60    ) -> Result<Arc<dyn TableSource>> {
61        not_impl_err!("Table Functions are not supported")
62    }
63
64    /// Provides an intermediate table that is used to store the results of a CTE during execution
65    ///
66    /// CTE stands for "Common Table Expression"
67    ///
68    /// # Notes
69    /// We don't directly implement this in [`SqlToRel`] as implementing this function
70    /// often requires access to a table that contains
71    /// execution-related types that can't be a direct dependency
72    /// of the sql crate (for example [`CteWorkTable`]).
73    ///
74    /// The [`ContextProvider`] provides a way to "hide" this dependency.
75    ///
76    /// [`SqlToRel`]: https://docs.rs/datafusion/latest/datafusion/sql/planner/struct.SqlToRel.html
77    /// [`CteWorkTable`]: https://docs.rs/datafusion/latest/datafusion/datasource/cte_worktable/struct.CteWorkTable.html
78    fn create_cte_work_table(
79        &self,
80        _name: &str,
81        _schema: SchemaRef,
82    ) -> Result<Arc<dyn TableSource>> {
83        not_impl_err!("Recursive CTE is not implemented")
84    }
85
86    /// Return [`ExprPlanner`] extensions for planning expressions
87    fn get_expr_planners(&self) -> &[Arc<dyn ExprPlanner>] {
88        &[]
89    }
90
91    /// Return [`RelationPlanner`] extensions for planning table factors
92    #[cfg(feature = "sql")]
93    fn get_relation_planners(&self) -> &[Arc<dyn RelationPlanner>] {
94        &[]
95    }
96
97    /// Return [`TypePlanner`] extensions for planning data types
98    #[cfg(feature = "sql")]
99    fn get_type_planner(&self) -> Option<Arc<dyn TypePlanner>> {
100        None
101    }
102
103    /// Return the scalar function with a given name, if any
104    fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>>;
105
106    /// Return the aggregate function with a given name, if any
107    fn get_aggregate_meta(&self, name: &str) -> Option<Arc<AggregateUDF>>;
108
109    /// Return the window function with a given name, if any
110    fn get_window_meta(&self, name: &str) -> Option<Arc<WindowUDF>>;
111
112    /// Return the system/user-defined variable type, if any
113    ///
114    /// A user defined variable is typically accessed via `@var_name`
115    fn get_variable_type(&self, variable_names: &[String]) -> Option<DataType>;
116
117    /// Return metadata about a system/user-defined variable, if any.
118    ///
119    /// By default, this wraps [`Self::get_variable_type`] in an Arrow [`Field`]
120    /// with nullable set to `true` and no metadata. Implementations that can
121    /// provide richer information (such as nullability or extension metadata)
122    /// should override this method.
123    fn get_variable_field(&self, variable_names: &[String]) -> Option<FieldRef> {
124        self.get_variable_type(variable_names)
125            .map(|data_type| data_type.into_nullable_field_ref())
126    }
127
128    /// Return overall configuration options
129    fn options(&self) -> &ConfigOptions;
130
131    /// Return all scalar function names
132    fn udf_names(&self) -> Vec<String>;
133
134    /// Return all aggregate function names
135    fn udaf_names(&self) -> Vec<String>;
136
137    /// Return all window function names
138    fn udwf_names(&self) -> Vec<String>;
139}
140
141/// Customize planning of SQL AST expressions to [`Expr`]s
142pub trait ExprPlanner: Debug + Send + Sync {
143    /// Plan the binary operation between two expressions, returns original
144    /// BinaryExpr if not possible
145    fn plan_binary_op(
146        &self,
147        expr: RawBinaryExpr,
148        _schema: &DFSchema,
149    ) -> Result<PlannerResult<RawBinaryExpr>> {
150        Ok(PlannerResult::Original(expr))
151    }
152
153    /// Plan the field access expression, such as `foo.bar`
154    ///
155    /// returns original [`RawFieldAccessExpr`] if not possible
156    fn plan_field_access(
157        &self,
158        expr: RawFieldAccessExpr,
159        _schema: &DFSchema,
160    ) -> Result<PlannerResult<RawFieldAccessExpr>> {
161        Ok(PlannerResult::Original(expr))
162    }
163
164    /// Plan an array literal, such as `[1, 2, 3]`
165    ///
166    /// Returns original expression arguments if not possible
167    fn plan_array_literal(
168        &self,
169        exprs: Vec<Expr>,
170        _schema: &DFSchema,
171    ) -> Result<PlannerResult<Vec<Expr>>> {
172        Ok(PlannerResult::Original(exprs))
173    }
174
175    /// Plan a `POSITION` expression, such as `POSITION(<expr> in <expr>)`
176    ///
177    /// Returns original expression arguments if not possible
178    fn plan_position(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
179        Ok(PlannerResult::Original(args))
180    }
181
182    /// Plan a dictionary literal, such as `{ key: value, ...}`
183    ///
184    /// Returns original expression arguments if not possible
185    fn plan_dictionary_literal(
186        &self,
187        expr: RawDictionaryExpr,
188        _schema: &DFSchema,
189    ) -> Result<PlannerResult<RawDictionaryExpr>> {
190        Ok(PlannerResult::Original(expr))
191    }
192
193    /// Plan an extract expression, such as`EXTRACT(month FROM foo)`
194    ///
195    /// Returns original expression arguments if not possible
196    fn plan_extract(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
197        Ok(PlannerResult::Original(args))
198    }
199
200    /// Plan an substring expression, such as `SUBSTRING(<expr> [FROM <expr>] [FOR <expr>])`
201    ///
202    /// Returns original expression arguments if not possible
203    fn plan_substring(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
204        Ok(PlannerResult::Original(args))
205    }
206
207    /// Plans a struct literal, such as  `{'field1' : expr1, 'field2' : expr2, ...}`
208    ///
209    /// This function takes a vector of expressions and a boolean flag
210    /// indicating whether the struct uses the optional name
211    ///
212    /// Returns the original input expressions if planning is not possible.
213    fn plan_struct_literal(
214        &self,
215        args: Vec<Expr>,
216        _is_named_struct: bool,
217    ) -> Result<PlannerResult<Vec<Expr>>> {
218        Ok(PlannerResult::Original(args))
219    }
220
221    /// Plans an overlay expression, such as `overlay(str PLACING substr FROM pos [FOR count])`
222    ///
223    /// Returns original expression arguments if not possible
224    fn plan_overlay(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
225        Ok(PlannerResult::Original(args))
226    }
227
228    /// Plans a `make_map` expression, such as `make_map(key1, value1, key2, value2, ...)`
229    ///
230    /// Returns original expression arguments if not possible
231    fn plan_make_map(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
232        Ok(PlannerResult::Original(args))
233    }
234
235    /// Plans compound identifier such as `db.schema.table` for non-empty nested names
236    ///
237    /// # Note:
238    /// Currently compound identifier for outer query schema is not supported.
239    ///
240    /// Returns original expression if not possible
241    fn plan_compound_identifier(
242        &self,
243        _field: &Field,
244        _qualifier: Option<&TableReference>,
245        _nested_names: &[String],
246    ) -> Result<PlannerResult<Vec<Expr>>> {
247        not_impl_err!(
248            "Default planner compound identifier hasn't been implemented for ExprPlanner"
249        )
250    }
251
252    /// Plans `ANY` expression, such as `expr = ANY(array_expr)`
253    ///
254    /// Returns origin binary expression if not possible
255    fn plan_any(&self, expr: RawBinaryExpr) -> Result<PlannerResult<RawBinaryExpr>> {
256        Ok(PlannerResult::Original(expr))
257    }
258
259    /// Plans aggregate functions, such as `COUNT(<expr>)`
260    ///
261    /// Returns original expression arguments if not possible
262    fn plan_aggregate(
263        &self,
264        expr: RawAggregateExpr,
265    ) -> Result<PlannerResult<RawAggregateExpr>> {
266        Ok(PlannerResult::Original(expr))
267    }
268
269    /// Plans window functions, such as `COUNT(<expr>)`
270    ///
271    /// Returns original expression arguments if not possible
272    fn plan_window(&self, expr: RawWindowExpr) -> Result<PlannerResult<RawWindowExpr>> {
273        Ok(PlannerResult::Original(expr))
274    }
275}
276
277/// An operator with two arguments to plan
278///
279/// Note `left` and `right` are DataFusion [`Expr`]s but the `op` is the SQL AST
280/// operator.
281///
282/// This structure is used by [`ExprPlanner`] to plan operators with
283/// custom expressions.
284#[derive(Debug, Clone)]
285pub struct RawBinaryExpr {
286    #[cfg(not(feature = "sql"))]
287    pub op: datafusion_expr_common::operator::Operator,
288    #[cfg(feature = "sql")]
289    pub op: sqlparser::ast::BinaryOperator,
290    pub left: Expr,
291    pub right: Expr,
292}
293
294/// An expression with GetFieldAccess to plan
295///
296/// This structure is used by [`ExprPlanner`] to plan operators with
297/// custom expressions.
298#[derive(Debug, Clone)]
299pub struct RawFieldAccessExpr {
300    pub field_access: GetFieldAccess,
301    pub expr: Expr,
302}
303
304/// A Dictionary literal expression `{ key: value, ...}`
305///
306/// This structure is used by [`ExprPlanner`] to plan operators with
307/// custom expressions.
308#[derive(Debug, Clone)]
309pub struct RawDictionaryExpr {
310    pub keys: Vec<Expr>,
311    pub values: Vec<Expr>,
312}
313
314/// This structure is used by `AggregateFunctionPlanner` to plan operators with
315/// custom expressions.
316#[derive(Debug, Clone)]
317pub struct RawAggregateExpr {
318    pub func: Arc<AggregateUDF>,
319    pub args: Vec<Expr>,
320    pub distinct: bool,
321    pub filter: Option<Box<Expr>>,
322    pub order_by: Vec<SortExpr>,
323    pub null_treatment: Option<NullTreatment>,
324}
325
326/// This structure is used by `WindowFunctionPlanner` to plan operators with
327/// custom expressions.
328#[derive(Debug, Clone)]
329pub struct RawWindowExpr {
330    pub func_def: WindowFunctionDefinition,
331    pub args: Vec<Expr>,
332    pub partition_by: Vec<Expr>,
333    pub order_by: Vec<SortExpr>,
334    pub window_frame: WindowFrame,
335    pub filter: Option<Box<Expr>>,
336    pub null_treatment: Option<NullTreatment>,
337    pub distinct: bool,
338}
339
340/// Result of planning a raw expr with [`ExprPlanner`]
341#[derive(Debug, Clone)]
342pub enum PlannerResult<T> {
343    /// The raw expression was successfully planned as a new [`Expr`]
344    Planned(Expr),
345    /// The raw expression could not be planned, and is returned unmodified
346    Original(T),
347}
348
349/// Result of planning a relation with [`RelationPlanner`]
350#[cfg(feature = "sql")]
351#[derive(Debug, Clone)]
352pub struct PlannedRelation {
353    /// The logical plan for the relation
354    pub plan: LogicalPlan,
355    /// Optional table alias for the relation
356    pub alias: Option<TableAlias>,
357}
358
359#[cfg(feature = "sql")]
360impl PlannedRelation {
361    /// Create a new `PlannedRelation` with the given plan and alias
362    pub fn new(plan: LogicalPlan, alias: Option<TableAlias>) -> Self {
363        Self { plan, alias }
364    }
365}
366
367/// Result of attempting to plan a relation with extension planners
368#[cfg(feature = "sql")]
369#[derive(Debug)]
370pub enum RelationPlanning {
371    /// The relation was successfully planned by an extension planner
372    Planned(PlannedRelation),
373    /// No extension planner handled the relation, return it for default processing
374    Original(TableFactor),
375}
376
377/// Customize planning SQL table factors to [`LogicalPlan`]s.
378#[cfg(feature = "sql")]
379pub trait RelationPlanner: Debug + Send + Sync {
380    /// Plan a table factor into a [`LogicalPlan`].
381    ///
382    /// Returning [`RelationPlanning::Planned`] short-circuits further planning and uses the
383    /// provided plan. Returning [`RelationPlanning::Original`] allows the next registered planner,
384    /// or DataFusion's default logic, to handle the relation.
385    fn plan_relation(
386        &self,
387        relation: TableFactor,
388        context: &mut dyn RelationPlannerContext,
389    ) -> Result<RelationPlanning>;
390}
391
392/// Provides utilities for relation planners to interact with DataFusion's SQL
393/// planner.
394///
395/// This trait provides SQL planning utilities specific to relation planning,
396/// such as converting SQL expressions to logical expressions and normalizing
397/// identifiers. It uses composition to provide access to session context via
398/// [`ContextProvider`].
399#[cfg(feature = "sql")]
400pub trait RelationPlannerContext {
401    /// Provides access to the underlying context provider for reading session
402    /// configuration, accessing tables, functions, and other metadata.
403    fn context_provider(&self) -> &dyn ContextProvider;
404
405    /// Plans the specified relation through the full planner pipeline, starting
406    /// from the first registered relation planner.
407    fn plan(&mut self, relation: TableFactor) -> Result<LogicalPlan>;
408
409    /// Converts a SQL expression into a logical expression using the current
410    /// planner context.
411    fn sql_to_expr(&mut self, expr: SQLExpr, schema: &DFSchema) -> Result<Expr>;
412
413    /// Converts a SQL expression into a logical expression without DataFusion
414    /// rewrites.
415    fn sql_expr_to_logical_expr(
416        &mut self,
417        expr: SQLExpr,
418        schema: &DFSchema,
419    ) -> Result<Expr>;
420
421    /// Normalizes an identifier according to session settings.
422    fn normalize_ident(&self, ident: Ident) -> String;
423
424    /// Normalizes a SQL object name into a [`TableReference`].
425    fn object_name_to_table_reference(&self, name: ObjectName) -> Result<TableReference>;
426}
427
428/// Customize planning SQL types to DataFusion (Arrow) types.
429#[cfg(feature = "sql")]
430pub trait TypePlanner: Debug + Send + Sync {
431    /// Plan SQL [`sqlparser::ast::DataType`] to DataFusion [`DataType`]
432    ///
433    /// Returns None if not possible
434    fn plan_type(
435        &self,
436        _sql_type: &sqlparser::ast::DataType,
437    ) -> Result<Option<DataType>> {
438        Ok(None)
439    }
440}