@@ -39,9 +39,7 @@ use crate::physical_plan::sort::SortExec;
3939use crate :: physical_plan:: udf;
4040use crate :: physical_plan:: windows:: WindowAggExec ;
4141use crate :: physical_plan:: { hash_utils, Partitioning } ;
42- use crate :: physical_plan:: {
43- AggregateExpr , ExecutionPlan , PhysicalExpr , PhysicalPlanner , WindowExpr ,
44- } ;
42+ use crate :: physical_plan:: { AggregateExpr , ExecutionPlan , PhysicalExpr , WindowExpr } ;
4543use crate :: prelude:: JoinType ;
4644use crate :: scalar:: ScalarValue ;
4745use crate :: sql:: utils:: { generate_sort_key, window_expr_common_partition_keys} ;
@@ -172,16 +170,51 @@ fn physical_name(e: &Expr, input_schema: &DFSchema) -> Result<String> {
172170 }
173171}
174172
173+ /// Physical query planner that converts a `LogicalPlan` to an
174+ /// `ExecutionPlan` suitable for execution.
175+ pub trait PhysicalPlanner {
176+ /// Create a physical plan from a logical plan
177+ fn create_physical_plan (
178+ & self ,
179+ logical_plan : & LogicalPlan ,
180+ ctx_state : & ExecutionContextState ,
181+ ) -> Result < Arc < dyn ExecutionPlan > > ;
182+
183+ /// Create a physical expression from a logical expression
184+ /// suitable for evaluation
185+ ///
186+ /// `expr`: the expression to convert
187+ ///
188+ /// `input_dfschema`: the logical plan schema for evaluating `e`
189+ ///
190+ /// `input_schema`: the physical schema for evaluating `e`
191+ fn create_physical_expr (
192+ & self ,
193+ expr : & Expr ,
194+ input_dfschema : & DFSchema ,
195+ input_schema : & Schema ,
196+ ctx_state : & ExecutionContextState ,
197+ ) -> Result < Arc < dyn PhysicalExpr > > ;
198+ }
199+
175200/// This trait exposes the ability to plan an [`ExecutionPlan`] out of a [`LogicalPlan`].
176201pub trait ExtensionPlanner {
177202 /// Create a physical plan for a [`UserDefinedLogicalNode`].
178- /// This errors when the planner knows how to plan the concrete implementation of `node`
179- /// but errors while doing so, and `None` when the planner does not know how to plan the `node`
180- /// and wants to delegate the planning to another [`ExtensionPlanner`].
203+ ///
204+ /// `input_dfschema`: the logical plan schema for the inputs to this node
205+ ///
206+ /// Returns an error when the planner knows how to plan the concrete
207+ /// implementation of `node` but errors while doing so.
208+ ///
209+ /// Returns `None` when the planner does not know how to plan the
210+ /// `node` and wants to delegate the planning to another
211+ /// [`ExtensionPlanner`].
181212 fn plan_extension (
182213 & self ,
214+ planner : & dyn PhysicalPlanner ,
183215 node : & dyn UserDefinedLogicalNode ,
184- inputs : & [ Arc < dyn ExecutionPlan > ] ,
216+ logical_inputs : & [ & LogicalPlan ] ,
217+ physical_inputs : & [ Arc < dyn ExecutionPlan > ] ,
185218 ctx_state : & ExecutionContextState ,
186219 ) -> Result < Option < Arc < dyn ExecutionPlan > > > ;
187220}
@@ -210,6 +243,30 @@ impl PhysicalPlanner for DefaultPhysicalPlanner {
210243 let plan = self . create_initial_plan ( logical_plan, ctx_state) ?;
211244 self . optimize_plan ( plan, ctx_state)
212245 }
246+
247+ /// Create a physical expression from a logical expression
248+ /// suitable for evaluation
249+ ///
250+ /// `e`: the expression to convert
251+ ///
252+ /// `input_dfschema`: the logical plan schema for evaluating `e`
253+ ///
254+ /// `input_schema`: the physical schema for evaluating `e`
255+ fn create_physical_expr (
256+ & self ,
257+ expr : & Expr ,
258+ input_dfschema : & DFSchema ,
259+ input_schema : & Schema ,
260+ ctx_state : & ExecutionContextState ,
261+ ) -> Result < Arc < dyn PhysicalExpr > > {
262+ DefaultPhysicalPlanner :: create_physical_expr (
263+ self ,
264+ expr,
265+ input_dfschema,
266+ input_schema,
267+ ctx_state,
268+ )
269+ }
213270}
214271
215272impl DefaultPhysicalPlanner {
@@ -721,7 +778,7 @@ impl DefaultPhysicalPlanner {
721778 ) ) )
722779 }
723780 LogicalPlan :: Extension { node } => {
724- let inputs = node
781+ let physical_inputs = node
725782 . inputs ( )
726783 . into_iter ( )
727784 . map ( |input_plan| self . create_initial_plan ( input_plan, ctx_state) )
@@ -733,7 +790,13 @@ impl DefaultPhysicalPlanner {
733790 if let Some ( plan) = maybe_plan {
734791 Ok ( Some ( plan) )
735792 } else {
736- planner. plan_extension ( node. as_ref ( ) , & inputs, ctx_state)
793+ planner. plan_extension (
794+ self ,
795+ node. as_ref ( ) ,
796+ & node. inputs ( ) ,
797+ & physical_inputs,
798+ ctx_state,
799+ )
737800 }
738801 } ,
739802 ) ?;
@@ -1644,8 +1707,10 @@ mod tests {
16441707 /// Create a physical plan for an extension node
16451708 fn plan_extension (
16461709 & self ,
1710+ _planner : & dyn PhysicalPlanner ,
16471711 _node : & dyn UserDefinedLogicalNode ,
1648- _inputs : & [ Arc < dyn ExecutionPlan > ] ,
1712+ _logical_inputs : & [ & LogicalPlan ] ,
1713+ _physical_inputs : & [ Arc < dyn ExecutionPlan > ] ,
16491714 _ctx_state : & ExecutionContextState ,
16501715 ) -> Result < Option < Arc < dyn ExecutionPlan > > > {
16511716 Ok ( Some ( Arc :: new ( NoOpExecutionPlan {
0 commit comments