@@ -15,15 +15,19 @@ use datafusion::{
1515 tree_node:: { Transformed , TreeNode } ,
1616 Statistics ,
1717 } ,
18+ config:: ConfigOptions ,
1819 error:: { DataFusionError , Result } ,
1920 execution:: { context:: SessionState , TaskContext } ,
2021 logical_expr:: { Extension , LogicalPlan } ,
2122 optimizer:: { optimizer:: Optimizer , OptimizerConfig , OptimizerRule } ,
2223 physical_expr:: EquivalenceProperties ,
2324 physical_plan:: {
2425 execution_plan:: { Boundedness , EmissionType } ,
26+ filter_pushdown:: {
27+ ChildPushdownResult , FilterPushdownPhase , FilterPushdownPropagation , PushedDown ,
28+ } ,
2529 metrics:: MetricsSet ,
26- DisplayAs , DisplayFormatType , ExecutionPlan , Partitioning , PlanProperties ,
30+ DisplayAs , DisplayFormatType , ExecutionPlan , Partitioning , PhysicalExpr , PlanProperties ,
2731 SendableRecordBatchStream ,
2832 } ,
2933 sql:: { sqlparser:: ast:: Statement , unparser:: Unparser } ,
@@ -162,6 +166,7 @@ pub struct VirtualExecutionPlan {
162166 executor : Arc < dyn SQLExecutor > ,
163167 props : PlanProperties ,
164168 statistics : Statistics ,
169+ filters : Vec < Arc < dyn PhysicalExpr > > ,
165170}
166171
167172impl VirtualExecutionPlan {
@@ -178,6 +183,7 @@ impl VirtualExecutionPlan {
178183 executor,
179184 props,
180185 statistics,
186+ filters : Vec :: new ( ) ,
181187 }
182188 }
183189
@@ -358,7 +364,8 @@ impl ExecutionPlan for VirtualExecutionPlan {
358364 _partition : usize ,
359365 _context : Arc < TaskContext > ,
360366 ) -> Result < SendableRecordBatchStream > {
361- self . executor . execute ( & self . final_sql ( ) ?, self . schema ( ) )
367+ self . executor
368+ . execute ( & self . final_sql ( ) ?, self . schema ( ) , & self . filters )
362369 }
363370
364371 fn properties ( & self ) -> & PlanProperties {
@@ -372,6 +379,36 @@ impl ExecutionPlan for VirtualExecutionPlan {
372379 fn metrics ( & self ) -> Option < MetricsSet > {
373380 self . executor . metrics ( )
374381 }
382+
383+ fn handle_child_pushdown_result (
384+ & self ,
385+ _phase : FilterPushdownPhase ,
386+ child_pushdown_result : ChildPushdownResult ,
387+ _config : & ConfigOptions ,
388+ ) -> Result < FilterPushdownPropagation < Arc < dyn ExecutionPlan > > > {
389+ let parent_filters: Vec < _ > = child_pushdown_result
390+ . clone ( )
391+ . parent_filters
392+ . into_iter ( )
393+ . map ( |f| f. filter )
394+ . collect ( ) ;
395+
396+ if parent_filters. is_empty ( ) {
397+ return Ok ( FilterPushdownPropagation {
398+ filters : vec ! [ ] ,
399+ updated_node : None ,
400+ } ) ;
401+ }
402+
403+ let filters_pushed_down = vec ! [ PushedDown :: Yes ; parent_filters. len( ) ] ;
404+ let mut node = self . clone ( ) ;
405+ node. filters = parent_filters;
406+
407+ Ok ( FilterPushdownPropagation {
408+ filters : filters_pushed_down,
409+ updated_node : Some ( Arc :: new ( node) ) ,
410+ } )
411+ }
375412}
376413
377414#[ cfg( test) ]
@@ -416,7 +453,12 @@ mod tests {
416453 Arc :: new ( unparser:: dialect:: DefaultDialect { } )
417454 }
418455
419- fn execute ( & self , _query : & str , _schema : SchemaRef ) -> Result < SendableRecordBatchStream > {
456+ fn execute (
457+ & self ,
458+ _query : & str ,
459+ _schema : SchemaRef ,
460+ _filters : & [ Arc < dyn PhysicalExpr > ] ,
461+ ) -> Result < SendableRecordBatchStream > {
420462 unimplemented ! ( )
421463 }
422464
0 commit comments