1818//! Defines the recursive query plan
1919
2020use std:: any:: Any ;
21+ use std:: mem:: take;
2122use std:: sync:: Arc ;
2223use std:: task:: { Context , Poll } ;
2324
2425use super :: work_table:: { ReservedBatches , WorkTable , WorkTableExec } ;
2526use crate :: execution_plan:: { Boundedness , EmissionType } ;
27+ use crate :: result_table:: ResultTable ;
2628use crate :: {
2729 metrics:: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ,
2830 PlanProperties , RecordBatchStream , SendableRecordBatchStream , Statistics ,
@@ -64,8 +66,9 @@ pub struct RecursiveQueryExec {
6466 static_term : Arc < dyn ExecutionPlan > ,
6567 /// The dynamic part (recursive term)
6668 recursive_term : Arc < dyn ExecutionPlan > ,
67- /// Distinction
6869 is_distinct : bool ,
70+ /// If is_distinct is true, holds the result table that saves all previous results
71+ result_table : Option < Arc < ResultTable > > ,
6972 /// Execution metrics
7073 metrics : ExecutionPlanMetricsSet ,
7174 /// Cache holding plan properties like equivalences, output partitioning etc.
@@ -77,13 +80,22 @@ impl RecursiveQueryExec {
7780 pub fn try_new (
7881 name : String ,
7982 static_term : Arc < dyn ExecutionPlan > ,
80- recursive_term : Arc < dyn ExecutionPlan > ,
83+ mut recursive_term : Arc < dyn ExecutionPlan > ,
8184 is_distinct : bool ,
8285 ) -> Result < Self > {
8386 // Each recursive query needs its own work table
8487 let work_table = Arc :: new ( WorkTable :: new ( ) ) ;
8588 // Use the same work table for both the WorkTableExec and the recursive term
86- let recursive_term = assign_work_table ( recursive_term, Arc :: clone ( & work_table) ) ?;
89+ recursive_term = assign_work_table ( recursive_term, Arc :: clone ( & work_table) ) ?;
90+ let result_table = if is_distinct {
91+ let result_table = Arc :: new ( ResultTable :: new ( ) ) ;
92+ // Use the same result table for both the ResultTableExec and the result term
93+ recursive_term =
94+ assign_work_table ( recursive_term, Arc :: clone ( & result_table) ) ?;
95+ Some ( result_table)
96+ } else {
97+ None
98+ } ;
8799 let cache = Self :: compute_properties ( static_term. schema ( ) ) ;
88100 Ok ( RecursiveQueryExec {
89101 name,
@@ -93,6 +105,7 @@ impl RecursiveQueryExec {
93105 work_table,
94106 metrics : ExecutionPlanMetricsSet :: new ( ) ,
95107 cache,
108+ result_table,
96109 } )
97110 }
98111
@@ -193,6 +206,7 @@ impl ExecutionPlan for RecursiveQueryExec {
193206 Ok ( Box :: pin ( RecursiveQueryStream :: new (
194207 context,
195208 Arc :: clone ( & self . work_table ) ,
209+ self . result_table . as_ref ( ) . map ( Arc :: clone) ,
196210 Arc :: clone ( & self . recursive_term ) ,
197211 static_stream,
198212 baseline_metrics,
@@ -237,16 +251,16 @@ impl DisplayAs for RecursiveQueryExec {
237251///
238252/// while batch := static_stream.next():
239253/// buffer.push(batch)
240- /// yield buffer
254+ /// yield batch
241255///
242256/// while buffer.len() > 0:
243257/// sender, receiver = Channel()
244- /// register_continuation (handle_name, receiver)
258+ /// register_work_table (handle_name, receiver)
245259/// sender.send(buffer.drain())
246260/// recursive_stream = recursive_term.execute()
247261/// while batch := recursive_stream.next():
248262/// buffer.append(batch)
249- /// yield buffer
263+ /// yield batch
250264///
251265struct RecursiveQueryStream {
252266 /// The context to be used for managing handlers & executing new tasks
@@ -268,6 +282,8 @@ struct RecursiveQueryStream {
268282 buffer : Vec < RecordBatch > ,
269283 /// Tracks the memory used by the buffer
270284 reservation : MemoryReservation ,
285+ /// The result table state, representing the table used for deduplication in case it is enabled
286+ results_table : Option < Arc < ResultTable > > ,
271287 // /// Metrics.
272288 _baseline_metrics : BaselineMetrics ,
273289}
@@ -277,6 +293,7 @@ impl RecursiveQueryStream {
277293 fn new (
278294 task_context : Arc < TaskContext > ,
279295 work_table : Arc < WorkTable > ,
296+ results_table : Option < Arc < ResultTable > > ,
280297 recursive_term : Arc < dyn ExecutionPlan > ,
281298 static_stream : SendableRecordBatchStream ,
282299 baseline_metrics : BaselineMetrics ,
@@ -294,6 +311,7 @@ impl RecursiveQueryStream {
294311 buffer : vec ! [ ] ,
295312 reservation,
296313 _baseline_metrics : baseline_metrics,
314+ results_table,
297315 }
298316 }
299317
@@ -327,11 +345,21 @@ impl RecursiveQueryStream {
327345 return Poll :: Ready ( None ) ;
328346 }
329347
348+ // Update the union table with the current buffer
349+ if self . results_table . is_some ( ) {
350+ // Note it's fine to take the memory reservation here, we are not cloning the underlying data,
351+ // and the result table is going to outlive the work table.
352+ let buffer = self . buffer . clone ( ) ;
353+ let reservation = self . reservation . take ( ) ;
354+ self . results_table
355+ . as_mut ( )
356+ . unwrap ( )
357+ . append ( buffer, reservation) ;
358+ }
359+
330360 // Update the work table with the current buffer
331- let reserved_batches = ReservedBatches :: new (
332- std:: mem:: take ( & mut self . buffer ) ,
333- self . reservation . take ( ) ,
334- ) ;
361+ let reserved_batches =
362+ ReservedBatches :: new ( take ( & mut self . buffer ) , self . reservation . take ( ) ) ;
335363 self . work_table . update ( reserved_batches) ;
336364
337365 // We always execute (and re-execute iteratively) the first partition.
@@ -345,9 +373,9 @@ impl RecursiveQueryStream {
345373 }
346374}
347375
348- fn assign_work_table (
376+ fn assign_work_table < T : Any + Send + Sync > (
349377 plan : Arc < dyn ExecutionPlan > ,
350- work_table : Arc < WorkTable > ,
378+ work_table : Arc < T > ,
351379) -> Result < Arc < dyn ExecutionPlan > > {
352380 let mut work_table_refs = 0 ;
353381 plan. transform_down ( |plan| {
@@ -380,7 +408,7 @@ fn assign_work_table(
380408fn reset_plan_states ( plan : Arc < dyn ExecutionPlan > ) -> Result < Arc < dyn ExecutionPlan > > {
381409 plan. transform_up ( |plan| {
382410 // WorkTableExec's states have already been updated correctly.
383- if plan. as_any ( ) . is :: < WorkTableExec > ( ) {
411+ if plan. as_any ( ) . is :: < WorkTableExec > ( ) || plan . as_any ( ) . is :: < ResultTable > ( ) {
384412 Ok ( Transformed :: no ( plan) )
385413 } else {
386414 let new_plan = Arc :: clone ( & plan) . reset_state ( ) ?;
0 commit comments