matthewgapp commented on code in PR #8840:
URL: https://github.com/apache/arrow-datafusion/pull/8840#discussion_r1468204446


##########
datafusion/physical-plan/src/recursive_query.rs:
##########
@@ -0,0 +1,373 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Defines the recursive query plan
+
+use std::any::Any;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+use super::expressions::PhysicalSortExpr;
+use super::metrics::BaselineMetrics;
+use super::RecordBatchStream;
+use super::{
+    metrics::{ExecutionPlanMetricsSet, MetricsSet},
+    work_table::{WorkTable, WorkTableExec},
+    SendableRecordBatchStream, Statistics,
+};
+
+use arrow::datatypes::SchemaRef;
+use arrow::record_batch::RecordBatch;
+use datafusion_common::tree_node::{Transformed, TreeNode};
+use datafusion_common::{not_impl_err, DataFusionError, Result};
+use datafusion_execution::TaskContext;
+use datafusion_physical_expr::Partitioning;
+use futures::{ready, Stream, StreamExt};
+
+use crate::{DisplayAs, DisplayFormatType, ExecutionPlan};
+
+/// Recursive query execution plan.
+///
+/// This plan has two components: a base part (the static term) and
+/// a dynamic part (the recursive term). The execution will start from
+/// the base, and as long as the previous iteration produced at least
+/// a single new row (taking care of the distinction) the recursive
+/// part will be continuously executed.
+///
+/// Before each execution of the dynamic part, the rows from the previous
+/// iteration will be available in a "working table" (not a real table,
+/// can be only accessed using a continuance operation).
+///
+/// Note that there won't be any limit or checks applied to detect
+/// an infinite recursion, so it is up to the planner to ensure that
+/// it won't happen.
+#[derive(Debug)]
+pub struct RecursiveQueryExec {
+    /// Name of the query handler
+    name: String,
+    /// The working table of cte
+    work_table: Arc<WorkTable>,
+    /// The base part (static term)
+    static_term: Arc<dyn ExecutionPlan>,
+    /// The dynamic part (recursive term)
+    recursive_term: Arc<dyn ExecutionPlan>,
+    /// Distinction
+    is_distinct: bool,
+    /// Execution metrics
+    metrics: ExecutionPlanMetricsSet,
+}
+
+impl RecursiveQueryExec {
+    /// Create a new RecursiveQueryExec
+    pub fn try_new(
+        name: String,
+        static_term: Arc<dyn ExecutionPlan>,
+        recursive_term: Arc<dyn ExecutionPlan>,
+        is_distinct: bool,
+    ) -> Result<Self> {
+        // Each recursive query needs its own work table
+        let work_table = Arc::new(WorkTable::new());
+        // Use the same work table for both the WorkTableExec and the 
recursive term
+        let recursive_term = assign_work_table(recursive_term, 
work_table.clone())?;
+        Ok(RecursiveQueryExec {
+            name,
+            static_term,
+            recursive_term,
+            is_distinct,
+            work_table,
+            metrics: ExecutionPlanMetricsSet::new(),
+        })
+    }
+}
+
+impl ExecutionPlan for RecursiveQueryExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.static_term.schema()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.static_term.clone(), self.recursive_term.clone()]
+    }
+
+    // Distribution on a recursive query is really tricky to handle.
+    // For now, we are going to use a single partition but in the
+    // future we might find a better way to handle this.
+    fn output_partitioning(&self) -> Partitioning {
+        Partitioning::UnknownPartitioning(1)
+    }
+
+    // TODO: control these hints and see whether we can
+    // infer some from the child plans (static/recurisve terms).
+    fn maintains_input_order(&self) -> Vec<bool> {
+        vec![false, false]
+    }
+
+    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
+        vec![false, false]
+    }
+
+    fn required_input_distribution(&self) -> 
Vec<datafusion_physical_expr::Distribution> {
+        vec![
+            datafusion_physical_expr::Distribution::SinglePartition,
+            datafusion_physical_expr::Distribution::SinglePartition,
+        ]
+    }
+
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(RecursiveQueryExec {
+            name: self.name.clone(),
+            static_term: children[0].clone(),
+            recursive_term: children[1].clone(),
+            is_distinct: self.is_distinct,
+            work_table: self.work_table.clone(),
+            metrics: self.metrics.clone(),
+        }))
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        // TODO: we might be able to handle multiple partitions in the future.
+        if partition != 0 {
+            return Err(DataFusionError::Internal(format!(
+                "RecursiveQueryExec got an invalid partition {} (expected 0)",
+                partition
+            )));
+        }
+
+        let static_stream = self.static_term.execute(partition, 
context.clone())?;
+        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
+        Ok(Box::pin(RecursiveQueryStream::new(
+            context,
+            self.work_table.clone(),
+            self.recursive_term.clone(),
+            static_stream,
+            baseline_metrics,
+        )))
+    }
+
+    fn metrics(&self) -> Option<MetricsSet> {
+        Some(self.metrics.clone_inner())
+    }
+
+    fn statistics(&self) -> Result<Statistics> {
+        Ok(Statistics::new_unknown(&self.schema()))
+    }
+}
+
+impl DisplayAs for RecursiveQueryExec {
+    fn fmt_as(
+        &self,
+        t: DisplayFormatType,
+        f: &mut std::fmt::Formatter,
+    ) -> std::fmt::Result {
+        match t {
+            DisplayFormatType::Default | DisplayFormatType::Verbose => {
+                write!(f, "RecursiveQueryExec: is_distinct={}", 
self.is_distinct)

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to