You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/13 18:27:31 UTC

[GitHub] [arrow-datafusion] tustvold opened a new pull request, #2226: Morsel-driven Parallelism using rayon (#2199)

tustvold opened a new pull request, #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226

   # Which issue does this PR close?
   
   Closes #2199.
   
    # Rationale for this change
   
   See ticket
   
   # What changes are included in this PR?
   
   Adds a new datafusion-scheduler crate that provides an implementation of a push-based, morsel-driven scheduler based on rayon. More details, background, and discussion can be found in the proposal document [here](https://docs.google.com/document/d/1txX60thXn1tQO1ENNT8rwfU3cXLofa7ZccnvP4jD6AA), please feel free to comment there.
   
   I would describe the current implementation as an MVP, and there are definitely things that could be improved, but I would like to propose we get something merged and can then iterate on it further. My hope is that we can iteratively refine the implementation, and then once we are happy with it begin the process of refactoring the operators to better align with it.
   
   # Are there any user-facing changes?
   
   No, this is an entirely new crate, and does not require changes to any others.
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850811841


##########
datafusion/core/Cargo.toml:
##########
@@ -117,10 +117,6 @@ name = "scalar"
 harness = false
 name = "physical_plan"
 
-[[bench]]
-harness = false
-name = "parquet_query_sql"
-
 [[bench]]

Review Comment:
   That's the goal 😃



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850814519


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {

Review Comment:
   Pretty much 😃



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] andygrove commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
andygrove commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1105429313

   I am interested in this and would like to review but realistically, I won't be able to get to this until sometime next week.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851207785


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]
+    pub fn pipelines(&self) -> &[RoutablePipeline] {
+        &self.pipelines
+    }
+
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    pub fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    pub fn send_query_output(&self, output: ArrowResult<RecordBatch>) {
+        let _ = self.output.unbounded_send(output);
+    }
+
+    /// Returns the [`Spawner`] associated with this [`Query`]
+    pub fn spawner(&self) -> &Spawner {
+        &self.spawner
+    }
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple [`ExecutionPlan`], [`ExecGroup`] stores this state
+struct ExecGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`Query`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`Query`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+struct QueryBuilder {
+    task_context: Arc<TaskContext>,
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more [`ExecutionPlan`] to combine together into
+    /// a single [`ExecutionPipeline`]
+    exec_buffer: Option<ExecGroup>,
+}
+
+impl QueryBuilder {
+    fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        Self {
+            in_progress: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            exec_buffer: None,
+        }
+    }
+
+    /// Flush the current group of [`ExecutionPlan`] stored in `exec_buffer`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.exec_buffer.take().unwrap();
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the node to the current group of execution plan to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should

Review Comment:
   I've reworked the terminology to refer to operators, rather than plans. I think this should make it more clear?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856292834


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,439 @@
+// 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.
+
+use crate::query::Query;
+use crate::{is_worker, spawn_local, spawn_local_fifo, RoutablePipeline, Spawner};
+use arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Query, spawner: Spawner) -> QueryResults {
+    debug!("Spawning query: {:#?}", query);
+
+    let (sender, receiver) = mpsc::unbounded();
+    let query = Arc::new(QueryTask {
+        spawner,
+        pipelines: query.pipelines,
+        output: sender,
+    });
+
+    for (pipeline_idx, query_pipeline) in query.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            query.spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    QueryResults {
+        query,
+        inner: receiver,
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`QueryTask`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`QueryTask`] is dropped it signals completion of query execution
+    query: Arc<QueryTask>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = &self.query.pipelines[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    fn handle_error(&self, routable: &RoutablePipeline, error: DataFusionError) {
+        self.query.send_query_output(Err(error));
+        if let Some(link) = routable.output {
+            trace!(
+                "Closing pipeline: {:?}, partition: {}, due to error",
+                link,
+                self.waker.partition,
+            );
+
+            self.query.pipelines[link.pipeline]
+                .pipeline
+                .close(link.child, self.waker.partition);
+        }
+    }
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        let wake_count = self.waker.wake_count.load(Ordering::SeqCst);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = &self.query.pipelines;
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+
+                        let r = pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition);
+
+                        if let Err(e) = r {
+                            self.handle_error(routable, e);
+
+                            // Return without rescheduling this output again
+                            return;
+                        }
+                    }
+                    None => {
+                        trace!("Publishing batch to output");
+                        self.query.send_query_output(Ok(batch))
+                    }
+                }
+
+                // Reschedule this pipeline again
+                //
+                // We want to prioritise running tasks triggered by the most recent
+                // batch, so reschedule with FIFO ordering
+                //
+                // Note: We must schedule after we have routed the batch, otherwise
+                // we introduce a potential ordering race where the newly scheduled
+                // task runs before this task finishes routing the output
+                spawn_local_fifo(self);
+            }
+            Poll::Ready(Some(Err(e))) => {
+                trace!("Poll {:?}: Error: {:?}", self, e);
+                self.handle_error(routable, e)
+            }
+            Poll::Ready(None) => {
+                trace!("Poll {:?}: None", self);
+                match routable.output {
+                    Some(link) => {
+                        trace!("Closing pipeline: {:?}, partition: {}", link, partition);
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .close(link.child, partition)
+                    }
+                    None => self.query.finish(),
+                }
+            }
+            Poll::Pending => {
+                trace!("Poll {:?}: Pending", self);
+                // Attempt to reset the wake count with the value obtained prior
+                // to calling [`Pipeline::poll_partition`].
+                //
+                // If this fails it indicates a wakeup was received whilst executing
+                // [`Pipeline::poll_partition`] and we should reschedule the task
+                let reset = self.waker.wake_count.compare_exchange(
+                    wake_count,
+                    0,
+                    Ordering::SeqCst,
+                    Ordering::SeqCst,
+                );
+
+                if reset.is_err() {
+                    trace!("Wakeup triggered whilst polling: {:?}", self);
+                    spawn_local(self);
+                }
+            }
+        }
+    }
+}
+
+/// The result stream for a query
+///
+/// # Cancellation
+///
+/// Dropping this will cancel the inflight query
+pub struct QueryResults {
+    inner: mpsc::UnboundedReceiver<Option<Result<RecordBatch>>>,
+
+    /// Keep a reference to the [`QueryTask`] so it isn't dropped early
+    #[allow(unused)]
+    query: Arc<QueryTask>,
+}
+
+impl Stream for QueryResults {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        self.inner.poll_next_unpin(cx).map(Option::flatten)
+    }
+}
+
+/// The shared state of all [`Task`] created from the same [`Query`]
+#[derive(Debug)]
+struct QueryTask {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<Option<Result<RecordBatch>>>,
+}
+
+impl Drop for QueryTask {
+    fn drop(&mut self) {
+        debug!("Query dropped");
+    }
+}
+
+impl QueryTask {
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    fn send_query_output(&self, output: Result<RecordBatch>) {
+        let _ = self.output.unbounded_send(Some(output));
+    }
+
+    /// Mark this query as finished
+    fn finish(&self) {
+        let _ = self.output.unbounded_send(None);
+    }
+}
+
+struct TaskWaker {

Review Comment:
   `TaskWaker` implements `ArcWake` which handles the necessary shenanigans to translate to the `Waker` vtable. We could use `Waker` directly here, but it would require some rather exciting unsafe code :sweat_smile: 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] yjshen commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
yjshen commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r863799676


##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();
+//!
+//! let task = context.task_ctx();

Review Comment:
   The `task` is slightly misleading from the pipeline's `Task`.



##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();
+//!
+//! let task = context.task_ctx();
+//! let stream = scheduler.schedule(plan, task).unwrap();
+//! let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+//! # }
+//! ```
+//!
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use crate::error::Result;
+use crate::execution::context::TaskContext;
+use crate::physical_plan::ExecutionPlan;
+
+use plan::{PipelinePlan, PipelinePlanner, RoutablePipeline};
+use task::{spawn_plan, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::ExecutionResults;
+
+mod pipeline;
+mod plan;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    #[cfg(test)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] that can be used to schedule [`ExecutionPlan`] on a dedicated thread pool
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`ExecutionResults`] that can be used to receive results as they are produced,
+    /// as a [`futures::Stream`] of [`RecordBatch`]
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<ExecutionResults> {
+        let plan = PipelinePlanner::new(plan, context).build()?;
+        Ok(self.schedule_plan(plan))
+    }
+
+    /// Schedule the provided [`PipelinePlan`] on this [`Scheduler`].
+    pub(crate) fn schedule_plan(&self, plan: PipelinePlan) -> ExecutionResults {
+        spawn_plan(plan, self.spawner())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {
+        *msg
+    } else if let Some(msg) = panic.downcast_ref::<String>() {
+        msg.as_str()
+    } else {
+        "UNKNOWN"
+    };
+
+    format!("worker {} panicked with: {}", worker, message)
+}
+
+/// Returns `true` if the current thread is a rayon worker thread
+///
+/// Note: if there are multiple rayon pools, this will return `true` if the current thread
+/// belongs to ANY rayon pool, even if this isn't a worker thread of a [`Scheduler`] instance
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,
+/// `spawn_local` will append to the front of the current worker's queue, workers pop tasks from
+/// the front of their queue, and steal tasks from the back of other workers queues
+///
+/// The effect is that tasks spawned using `spawn_local` will typically be prioritised in
+/// a LIFO order, however, this should not be relied upon
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,
+/// `spawn_local_fifo` will append to the back of the current worker's queue, workers pop tasks
+/// from the front of their queue, and steal tasks from the back of other workers queues
+///
+/// The effect is that tasks spawned using `spawn_local_fifo` will typically be prioritised
+/// in a FIFO order, however, this should not be relied upon
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {

Review Comment:
   Doc for the `Spawner`?  We could check the clippy by 
   
   ```
   cargo clippy --all-targets --features=scheduler --workspace -- -D warnings
   ```



##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();

Review Comment:
   nit: the above four lines are 1 space less of indent.



##########
datafusion/core/src/scheduler/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use crate::arrow::datatypes::SchemaRef;
+use crate::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use crate::error::Result;
+use crate::execution::context::TaskContext;
+use crate::physical_plan::expressions::PhysicalSortExpr;
+use crate::physical_plan::metrics::MetricsSet;
+use crate::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::scheduler::pipeline::Pipeline;
+use crate::scheduler::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.

Review Comment:
   👍



##########
datafusion/core/src/scheduler/pipeline/mod.rs:
##########
@@ -0,0 +1,110 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::error::Result;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Eager vs Lazy Execution
+///
+/// Whether computation is eagerly done on push, or lazily done on pull, is
+/// intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none if
+/// the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output.

Review Comment:
   `close` marks the end of input, and `poll_partition` flushes aggregate states to the output?



##########
datafusion/core/src/scheduler/pipeline/mod.rs:
##########
@@ -0,0 +1,110 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::error::Result;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Eager vs Lazy Execution
+///
+/// Whether computation is eagerly done on push, or lazily done on pull, is
+/// intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none if
+/// the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output.
+/// It would also be possible to flush once the partial aggregates reach a certain size
+///
+/// A partition-aware aggregation pipeline, which functions similarly to the above, but
+/// computes aggregations per input partition, before combining these prior to flush.
+///
+/// An async input pipeline, which has no inputs, and wakes the output partition
+/// whenever new data is available
+///
+/// A JIT compiled sequence of synchronous operators, that perform multiple operations
+/// from the physical plan as a single [`Pipeline`]. Parallelized implementations
+/// are also possible
+///
+pub trait Pipeline: Send + Sync + std::fmt::Debug {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()>;

Review Comment:
   I suppose the `child` comes from the `ExecutionPlan` children that provides input for the plan, but it's a little bit vague in the pipeline context, `upstream/downstream` are more meaningful names for pipelines?



##########
datafusion/core/src/scheduler/task.rs:
##########
@@ -0,0 +1,497 @@
+// 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.
+
+use crate::error::{DataFusionError, Result};
+use crate::physical_plan::stream::RecordBatchStreamAdapter;
+use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
+use crate::scheduler::{
+    is_worker, plan::PipelinePlan, spawn_local, spawn_local_fifo, RoutablePipeline,
+    Spawner,
+};
+use arrow::datatypes::SchemaRef;
+use arrow::error::ArrowError;
+use arrow::record_batch::RecordBatch;
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{ready, Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a [`PipelinePlan`] using the provided [`Spawner`]
+pub fn spawn_plan(plan: PipelinePlan, spawner: Spawner) -> ExecutionResults {
+    debug!("Spawning pipeline plan: {:#?}", plan);
+
+    let (senders, receivers) = (0..plan.output_partitions)
+        .map(|_| mpsc::unbounded())
+        .unzip::<_, _, Vec<_>, Vec<_>>();
+
+    let context = Arc::new(ExecutionContext {
+        spawner,
+        pipelines: plan.pipelines,
+        schema: plan.schema,
+        output: senders,
+    });
+
+    for (pipeline_idx, query_pipeline) in context.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            context.spawner.spawn(Task {
+                context: context.clone(),
+                waker: Arc::new(TaskWaker {
+                    context: Arc::downgrade(&context),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    let partitions = receivers
+        .into_iter()
+        .map(|receiver| ExecutionResultStream {
+            receiver: receiver,
+            context: context.clone(),
+        })
+        .collect();
+
+    ExecutionResults {
+        streams: partitions,
+        context,
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+pub struct Task {
+    /// Maintain a link to the [`ExecutionContext`] this is necessary to be able to
+    /// route the output of the partition to its destination
+    context: Arc<ExecutionContext>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = &self.context.pipelines[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    fn handle_error(
+        &self,
+        partition: usize,
+        routable: &RoutablePipeline,
+        error: DataFusionError,
+    ) {
+        self.context.send_query_output(partition, Err(error));
+        if let Some(link) = routable.output {
+            trace!(
+                "Closing pipeline: {:?}, partition: {}, due to error",
+                link,
+                self.waker.partition,
+            );
+
+            self.context.pipelines[link.pipeline]
+                .pipeline
+                .close(link.child, self.waker.partition);
+        }
+    }
+
+    /// Call [`Pipeline::poll_partition`], attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.context.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        let wake_count = self.waker.wake_count.load(Ordering::SeqCst);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = &self.context.pipelines;
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+
+                        let r = pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition);
+
+                        if let Err(e) = r {
+                            self.handle_error(partition, routable, e);
+
+                            // Return without rescheduling this output again
+                            return;
+                        }
+                    }
+                    None => {
+                        trace!("Publishing batch to output");
+                        self.context.send_query_output(partition, Ok(batch))
+                    }
+                }
+
+                // Reschedule this pipeline again
+                //
+                // We want to prioritise running tasks triggered by the most recent
+                // batch, so reschedule with FIFO ordering
+                //
+                // Note: We must schedule after we have routed the batch, otherwise
+                // we introduce a potential ordering race where the newly scheduled
+                // task runs before this task finishes routing the output
+                spawn_local_fifo(self);
+            }
+            Poll::Ready(Some(Err(e))) => {
+                trace!("Poll {:?}: Error: {:?}", self, e);
+                self.handle_error(partition, routable, e)
+            }
+            Poll::Ready(None) => {
+                trace!("Poll {:?}: None", self);
+                match routable.output {
+                    Some(link) => {
+                        trace!("Closing pipeline: {:?}, partition: {}", link, partition);
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .close(link.child, partition)
+                    }
+                    None => self.context.finish(partition),
+                }
+            }
+            Poll::Pending => {
+                trace!("Poll {:?}: Pending", self);
+                // Attempt to reset the wake count with the value obtained prior
+                // to calling [`Pipeline::poll_partition`].
+                //
+                // If this fails it indicates a wakeup was received whilst executing
+                // [`Pipeline::poll_partition`] and we should reschedule the task
+                let reset = self.waker.wake_count.compare_exchange(
+                    wake_count,
+                    0,
+                    Ordering::SeqCst,
+                    Ordering::SeqCst,
+                );
+
+                if reset.is_err() {
+                    trace!("Wakeup triggered whilst polling: {:?}", self);
+                    spawn_local(self);
+                }
+            }
+        }
+    }
+}
+
+/// The results of the execution of a query
+pub struct ExecutionResults {
+    /// [`ExecutionResultStream`] for each partition of this query
+    streams: Vec<ExecutionResultStream>,
+
+    /// Keep a reference to the [`ExecutionContext`] so it isn't dropped early
+    context: Arc<ExecutionContext>,
+}
+
+impl ExecutionResults {
+    /// Returns a [`SendableRecordBatchStream`] of this execution
+    ///
+    /// In the event of multiple output partitions, the output will be interleaved
+    pub fn stream(self) -> SendableRecordBatchStream {
+        let schema = self.context.schema.clone();
+        Box::pin(RecordBatchStreamAdapter::new(
+            schema,
+            futures::stream::select_all(self.streams),
+        ))
+    }
+
+    /// Returns a [`SendableRecordBatchStream`] for each partition of this execution
+    pub fn stream_partitioned(self) -> Vec<SendableRecordBatchStream> {
+        self.streams.into_iter().map(|s| Box::pin(s) as _).collect()
+    }
+}
+
+/// A result stream for the execution of a query
+struct ExecutionResultStream {
+    receiver: mpsc::UnboundedReceiver<Option<Result<RecordBatch>>>,
+
+    /// Keep a reference to the [`ExecutionContext`] so it isn't dropped early
+    context: Arc<ExecutionContext>,
+}
+
+impl Stream for ExecutionResultStream {
+    type Item = arrow::error::Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        let opt = ready!(self.receiver.poll_next_unpin(cx)).flatten();
+        Poll::Ready(opt.map(|r| r.map_err(|e| ArrowError::ExternalError(Box::new(e)))))
+    }
+}
+
+impl RecordBatchStream for ExecutionResultStream {
+    fn schema(&self) -> SchemaRef {
+        self.context.schema.clone()
+    }
+}
+
+/// The shared state of all [`Task`] created from the same [`PipelinePlan`]
+#[derive(Debug)]
+struct ExecutionContext {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// Schema of this plans output
+    pub schema: SchemaRef,
+
+    /// The output streams, per partition, for this query's execution
+    output: Vec<mpsc::UnboundedSender<Option<Result<RecordBatch>>>>,
+}
+
+impl Drop for ExecutionContext {
+    fn drop(&mut self) {
+        debug!("ExecutionContext dropped");
+    }
+}
+
+impl ExecutionContext {
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    fn is_cancelled(&self) -> bool {
+        self.output.iter().all(|x| x.is_closed())
+    }
+
+    /// Sends `output` to this query's output stream
+    fn send_query_output(&self, partition: usize, output: Result<RecordBatch>) {
+        let _ = self.output[partition].unbounded_send(Some(output));
+    }
+
+    /// Mark this partition as finished
+    fn finish(&self, partition: usize) {
+        let _ = self.output[partition].unbounded_send(None);
+    }
+}
+
+struct TaskWaker {
+    /// Store a weak reference to the [`ExecutionContext`] to avoid reference cycles if this
+    /// [`Waker`] is stored within a [`Pipeline`] owned by the [`ExecutionContext`]
+    context: Weak<ExecutionContext>,
+
+    /// A counter that stores the number of times this has been awoken
+    ///
+    /// A value > 0, implies the task is either in the ready queue or
+    /// currently being executed
+    ///
+    /// `TaskWaker::wake` always increments the `wake_count`, however, it only
+    /// re-enqueues the [`Task`] if the value prior to increment was 0
+    ///
+    /// This ensures that a given [`Task`] is not enqueued multiple times
+    ///
+    /// We store an integer, as opposed to a boolean, so that wake ups that
+    /// occur during [`Pipeline::poll_partition`] can be detected and handled
+    /// after it has finished executing
+    ///
+    wake_count: AtomicUsize,
+
+    /// The index of the pipeline within `query` to poll
+    pipeline: usize,
+
+    /// The partition of the pipeline within `query` to poll
+    partition: usize,

Review Comment:
   👍 Wake up the waker and re-enqueue the task.



##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();
+//!
+//! let task = context.task_ctx();
+//! let stream = scheduler.schedule(plan, task).unwrap();
+//! let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+//! # }
+//! ```
+//!
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use crate::error::Result;
+use crate::execution::context::TaskContext;
+use crate::physical_plan::ExecutionPlan;
+
+use plan::{PipelinePlan, PipelinePlanner, RoutablePipeline};
+use task::{spawn_plan, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::ExecutionResults;
+
+mod pipeline;
+mod plan;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    #[cfg(test)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] that can be used to schedule [`ExecutionPlan`] on a dedicated thread pool
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`ExecutionResults`] that can be used to receive results as they are produced,
+    /// as a [`futures::Stream`] of [`RecordBatch`]
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<ExecutionResults> {
+        let plan = PipelinePlanner::new(plan, context).build()?;
+        Ok(self.schedule_plan(plan))
+    }
+
+    /// Schedule the provided [`PipelinePlan`] on this [`Scheduler`].
+    pub(crate) fn schedule_plan(&self, plan: PipelinePlan) -> ExecutionResults {
+        spawn_plan(plan, self.spawner())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {
+        *msg
+    } else if let Some(msg) = panic.downcast_ref::<String>() {
+        msg.as_str()
+    } else {
+        "UNKNOWN"
+    };
+
+    format!("worker {} panicked with: {}", worker, message)
+}
+
+/// Returns `true` if the current thread is a rayon worker thread
+///
+/// Note: if there are multiple rayon pools, this will return `true` if the current thread
+/// belongs to ANY rayon pool, even if this isn't a worker thread of a [`Scheduler`] instance
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,
+/// `spawn_local` will append to the front of the current worker's queue, workers pop tasks from
+/// the front of their queue, and steal tasks from the back of other workers queues

Review Comment:
   👍 TIL



##########
datafusion/core/src/scheduler/pipeline/mod.rs:
##########
@@ -0,0 +1,110 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::error::Result;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Eager vs Lazy Execution
+///
+/// Whether computation is eagerly done on push, or lazily done on pull, is
+/// intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none if
+/// the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output.
+/// It would also be possible to flush once the partial aggregates reach a certain size

Review Comment:
   👍



##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();
+//!
+//! let task = context.task_ctx();
+//! let stream = scheduler.schedule(plan, task).unwrap();
+//! let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+//! # }
+//! ```
+//!
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use crate::error::Result;
+use crate::execution::context::TaskContext;
+use crate::physical_plan::ExecutionPlan;
+
+use plan::{PipelinePlan, PipelinePlanner, RoutablePipeline};
+use task::{spawn_plan, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::ExecutionResults;
+
+mod pipeline;
+mod plan;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    #[cfg(test)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] that can be used to schedule [`ExecutionPlan`] on a dedicated thread pool
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`ExecutionResults`] that can be used to receive results as they are produced,
+    /// as a [`futures::Stream`] of [`RecordBatch`]
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<ExecutionResults> {
+        let plan = PipelinePlanner::new(plan, context).build()?;
+        Ok(self.schedule_plan(plan))
+    }
+
+    /// Schedule the provided [`PipelinePlan`] on this [`Scheduler`].
+    pub(crate) fn schedule_plan(&self, plan: PipelinePlan) -> ExecutionResults {
+        spawn_plan(plan, self.spawner())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {
+        *msg
+    } else if let Some(msg) = panic.downcast_ref::<String>() {
+        msg.as_str()
+    } else {
+        "UNKNOWN"
+    };
+
+    format!("worker {} panicked with: {}", worker, message)
+}
+
+/// Returns `true` if the current thread is a rayon worker thread
+///
+/// Note: if there are multiple rayon pools, this will return `true` if the current thread
+/// belongs to ANY rayon pool, even if this isn't a worker thread of a [`Scheduler`] instance
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,
+/// `spawn_local` will append to the front of the current worker's queue, workers pop tasks from
+/// the front of their queue, and steal tasks from the back of other workers queues
+///
+/// The effect is that tasks spawned using `spawn_local` will typically be prioritised in
+/// a LIFO order, however, this should not be relied upon
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,
+/// `spawn_local_fifo` will append to the back of the current worker's queue, workers pop tasks
+/// from the front of their queue, and steal tasks from the back of other workers queues
+///
+/// The effect is that tasks spawned using `spawn_local_fifo` will typically be prioritised
+/// in a FIFO order, however, this should not be relied upon
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {
+    pool: Arc<ThreadPool>,
+}
+
+impl Spawner {
+    pub fn spawn(&self, task: Task) {
+        debug!("Spawning {:?} to any worker", task);
+        self.pool.spawn(move || task.do_work());
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use arrow::util::pretty::pretty_format_batches;
+    use std::ops::Range;
+    use std::panic::panic_any;
+
+    use futures::{StreamExt, TryStreamExt};
+    use log::info;
+    use rand::distributions::uniform::SampleUniform;
+    use rand::{thread_rng, Rng};
+
+    use crate::arrow::array::{ArrayRef, PrimitiveArray};
+    use crate::arrow::datatypes::{ArrowPrimitiveType, Float64Type, Int32Type};
+    use crate::arrow::record_batch::RecordBatch;
+    use crate::datasource::{MemTable, TableProvider};
+    use crate::physical_plan::displayable;
+    use crate::prelude::{SessionConfig, SessionContext};
+
+    use super::*;
+
+    fn generate_primitive<T, R>(
+        rng: &mut R,
+        len: usize,
+        valid_percent: f64,
+        range: Range<T::Native>,
+    ) -> ArrayRef
+    where
+        T: ArrowPrimitiveType,
+        T::Native: SampleUniform,
+        R: Rng,
+    {
+        Arc::new(PrimitiveArray::<T>::from_iter((0..len).map(|_| {
+            rng.gen_bool(valid_percent)
+                .then(|| rng.gen_range(range.clone()))
+        })))
+    }
+
+    fn generate_batch<R: Rng>(
+        rng: &mut R,
+        row_count: usize,
+        id_offset: i32,
+    ) -> RecordBatch {
+        let id_range = id_offset..(row_count as i32 + id_offset);
+        let a = generate_primitive::<Int32Type, _>(rng, row_count, 0.5, 0..1000);
+        let b = generate_primitive::<Float64Type, _>(rng, row_count, 0.5, 0. ..1000.);
+        let id = PrimitiveArray::<Int32Type>::from_iter_values(id_range);
+
+        RecordBatch::try_from_iter_with_nullable([
+            ("a", a, true),
+            ("b", b, true),
+            ("id", Arc::new(id), false),
+        ])
+        .unwrap()
+    }
+
+    const BATCHES_PER_PARTITION: usize = 20;
+    const ROWS_PER_BATCH: usize = 100;
+    const NUM_PARTITIONS: usize = 2;
+
+    fn make_batches() -> Vec<Vec<RecordBatch>> {
+        let mut rng = thread_rng();
+
+        let mut id_offset = 0;
+
+        (0..NUM_PARTITIONS)
+            .map(|_| {
+                (0..BATCHES_PER_PARTITION)
+                    .map(|_| {
+                        let batch = generate_batch(&mut rng, ROWS_PER_BATCH, id_offset);
+                        id_offset += ROWS_PER_BATCH as i32;
+                        batch
+                    })
+                    .collect()
+            })
+            .collect()
+    }
+
+    fn make_provider() -> Arc<dyn TableProvider> {
+        let batches = make_batches();
+        let schema = batches.first().unwrap().first().unwrap().schema();
+        Arc::new(MemTable::try_new(schema, make_batches()).unwrap())
+    }
+
+    fn init_logging() {
+        let _ = env_logger::builder().is_test(true).try_init();
+    }
+
+    #[tokio::test]
+    async fn test_simple() {
+        init_logging();
+
+        let scheduler = Scheduler::new(4);
+
+        let config = SessionConfig::new().with_target_partitions(4);
+        let context = SessionContext::with_config(config);
+
+        context.register_table("table1", make_provider()).unwrap();
+        context.register_table("table2", make_provider()).unwrap();
+
+        let queries = [
+            "select * from table1 order by id",
+            "select * from table1 where table1.a > 100 order by id",
+            "select distinct a from table1 where table1.b > 100 order by a",
+            "select * from table1 join table2 on table1.id = table2.id order by table1.id",
+            "select id from table1 union all select id from table2 order by id",
+            "select id from table1 union all select id from table2 where a > 100 order by id",
+            "select id, b from (select id, b from table1 union all select id, b from table2 where a > 100 order by id) as t where b > 10 order by id, b",
+            "select id, MIN(b), MAX(b), AVG(b) from table1 group by id order by id",
+            "select count(*) from table1 where table1.a > 4",

Review Comment:
   It would be beneficial if we could include one or several SQLs with their execution pipelines in the upper module doc.



##########
datafusion/core/src/scheduler/pipeline/mod.rs:
##########
@@ -0,0 +1,110 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::error::Result;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.

Review Comment:
   Pipeline inputs are identified by both a upstream index, and a partition index, whereas pipeline outputs are only
   identified by a partition index.



##########
datafusion/core/src/scheduler/plan.rs:
##########
@@ -0,0 +1,296 @@
+// 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.
+
+use arrow::datatypes::SchemaRef;
+use std::sync::Arc;
+
+use crate::error::Result;
+use crate::execution::context::TaskContext;
+use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use crate::physical_plan::repartition::RepartitionExec;
+use crate::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::scheduler::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`PipelinePlan`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`PipelinePlan`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`PipelinePlan`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct PipelinePlan {
+    /// Schema of this plans output
+    pub schema: SchemaRef,
+
+    /// Number of output partitions
+    pub output_partitions: usize,
+
+    /// Pipelines that comprise this plan
+    pub pipelines: Vec<RoutablePipeline>,
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple operators, [`OperatorGroup`] stores this state
+struct OperatorGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`PipelinePlan`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`PipelinePlan`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+pub struct PipelinePlanner {
+    task_context: Arc<TaskContext>,
+
+    /// The schema of this plan
+    schema: SchemaRef,
+
+    /// The number of output partitions of this plan
+    output_partitions: usize,
+
+    /// The current list of completed pipelines
+    completed: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more operators to combine
+    /// together into a single [`ExecutionPipeline`]
+    execution_operators: Option<OperatorGroup>,
+}
+
+impl PipelinePlanner {
+    pub fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        let schema = plan.schema();
+        let output_partitions = plan.output_partitioning().partition_count();
+        Self {
+            completed: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            execution_operators: None,
+            schema,
+            output_partitions,
+        }
+    }
+
+    /// Flush the current group of operators stored in `execution_operators`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.execution_operators.take().unwrap();
+        let node_idx = self.completed.len();
+        self.completed.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the operator to the current group of operators to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should

Review Comment:
   The policy will be best combined with a new push-aware ExecutionPlan API.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1099300739

   Ok as promised some benchmarks. It should be noted these come with some pretty big disclaimers:
   
   * Until we make changes to `ExecutionPlan`, the scheduler cannot introduce additional parallelism within a partition, as it is constrained by the current pull-based interface. Removing this will be a key performance unlock
   * The Parquet SQL benchmarks are massively dominated by the parquet performance, which may not be representative of all query workloads
   * Currently DataFusion uses `tokio::spawn_blocking` in the `tokio` case. Aside from this giving tokio more threads to play with, it also results in perfect thread-locality for the parquet decoder. I have therefore collected results with and without this enabled
   * My focus thus far has been to get something working, and not to squeeze out as much performance as possible, there is likely lots that could be improved
   
   That all being said, in like-for-like comparisons (i.e. without spawn blocking) we are actually seeing a slight performance improvement from the scheduler. I've not looked into why this is, but the only thing I can think of that might have improved performance is the switch to use rayon, everything else is either the same or would make it slower.
   
   * [Without Spawn Blocking.txt](https://github.com/apache/arrow-datafusion/files/8490374/Without.Spawn.Blocking.txt)
   * [With Spawn Blocking.txt](https://github.com/apache/arrow-datafusion/files/8490375/With.Spawn.Blocking.txt)
   
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851361531


##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,109 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Push vs Pull Execution
+///
+/// Whilst the interface exposed to the scheduler is push-based, in which member functions
+/// computation is performed is intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none none
+/// if the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output

Review Comment:
   Good shout, will include. Whilst enumerating the full list of possibilities is not the intent, this can act as a wishlist or sorts 😂



##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,109 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Push vs Pull Execution
+///
+/// Whilst the interface exposed to the scheduler is push-based, in which member functions
+/// computation is performed is intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none none
+/// if the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output

Review Comment:
   Good shout, will include. Whilst enumerating the full list of possibilities is not the intent, this can act as a wishlist of sorts 😂



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851364704


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition)

Review Comment:
   Yes, there should be plenty of parallelism to go around in most cases 😃



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851124299


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering

Review Comment:
   Yup, using relaxed ordering here is incorrect as the 0-1 transition needs to be ordered after the x-0 transition. Will fix



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856291150


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,276 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    pub pipelines: Vec<RoutablePipeline>,
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple operators, [`OperatorGroup`] stores this state
+struct OperatorGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`Query`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`Query`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+pub struct QueryBuilder {
+    task_context: Arc<TaskContext>,
+
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more operators to combine
+    /// together into a single [`ExecutionPipeline`]
+    execution_operators: Option<OperatorGroup>,
+}
+
+impl QueryBuilder {
+    pub fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        Self {
+            in_progress: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            execution_operators: None,
+        }
+    }
+
+    /// Flush the current group of operators stored in `execution_operators`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.execution_operators.take().unwrap();
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the operator to the current group of operators to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should
+        match self.execution_operators.as_mut() {
+            Some(buffer) => {
+                assert_eq!(parent, buffer.output, "QueryBuilder out of sync");
+                buffer.depth += 1;
+            }
+            None => {
+                self.execution_operators = Some(OperatorGroup {
+                    output: parent,
+                    root: plan,
+                    depth: 0,
+                })
+            }
+        }
+
+        match children.len() {
+            1 => {
+                // Enqueue the children with the parent of the `OperatorGroup`
+                self.to_visit
+                    .push((children.into_iter().next().unwrap(), parent))
+            }
+            _ => {
+                // We can only recursively group through nodes with a single child, therefore
+                // if this node has multiple children, we now need to flush the buffer and
+                // enqueue its children with this new pipeline as its parent
+                let node = self.flush_exec()?;
+                self.enqueue_children(children, node);
+            }
+        }
+
+        Ok(())
+    }
+
+    /// Add the given list of children to the stack of [`ExecutionPlan`] to visit
+    fn enqueue_children(
+        &mut self,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+        parent_node_idx: usize,
+    ) {
+        for (child_idx, child) in children.into_iter().enumerate() {
+            self.to_visit.push((
+                child,
+                Some(OutputLink {
+                    pipeline: parent_node_idx,
+                    child: child_idx,
+                }),
+            ))
+        }
+    }
+
+    /// Push a new [`RoutablePipeline`] and enqueue its children to be visited
+    fn push_pipeline(
+        &mut self,
+        node: RoutablePipeline,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) {
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(node);
+        self.enqueue_children(children, node_idx)
+    }
+
+    /// Push a new [`RepartitionPipeline`] first flushing any buffered [`OperatorGroup`]
+    fn push_repartition(
+        &mut self,
+        input: Partitioning,
+        output: Partitioning,
+        parent: Option<OutputLink>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<()> {
+        let parent = match &self.execution_operators {
+            Some(buffer) => {
+                assert_eq!(buffer.output, parent, "QueryBuilder out of sync");
+                Some(OutputLink {
+                    pipeline: self.flush_exec()?,
+                    child: 0, // Must be the only child
+                })
+            }
+            None => parent,
+        };
+
+        let node = Box::new(RepartitionPipeline::try_new(input, output)?);
+        self.push_pipeline(
+            RoutablePipeline {
+                pipeline: node,
+                output: parent,
+            },
+            children,
+        );
+        Ok(())
+    }
+
+    /// Visit an [`ExecutionPlan`] operator and add it to the [`Query`] being built
+    fn visit_operator(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        if let Some(repartition) = plan.as_any().downcast_ref::<RepartitionExec>() {

Review Comment:
   They aren't added into an `ExecutionPipeline` they are still a `Pipeline` technically...



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1098381176

   Good shout, I'll need to re-run them as some things have changed since then, will do first thing tomorrow. 👍
   
   The big ROI will be when operators start exploiting rayon and we simplify ExecutionPlan to match, but last time I ran we were getting some nice performance improvements despite the shimming costs


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850815904


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]
+    pub fn pipelines(&self) -> &[RoutablePipeline] {
+        &self.pipelines
+    }
+
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    pub fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    pub fn send_query_output(&self, output: ArrowResult<RecordBatch>) {
+        let _ = self.output.unbounded_send(output);
+    }
+
+    /// Returns the [`Spawner`] associated with this [`Query`]
+    pub fn spawner(&self) -> &Spawner {
+        &self.spawner
+    }
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple [`ExecutionPlan`], [`ExecGroup`] stores this state
+struct ExecGroup {

Review Comment:
   We aren't grouping pipelines, we're grouping operators here into pipelines. I'll try to update the docs to match.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850815433


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition)

Review Comment:
   Yes, which does create an interesting situation as the next output can't be polled until the last batch has been completely processed 🤔
   
   I think this will all shake out once we are consistently push-based



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] yjshen commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
yjshen commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r863796230


##########
datafusion/core/src/scheduler/mod.rs:
##########
@@ -0,0 +1,454 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();

Review Comment:
   nit: the above four lines are of 1 space less indent.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856534975


##########
datafusion/scheduler/Cargo.toml:
##########
@@ -0,0 +1,57 @@
+# 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.
+
+[package]
+name = "datafusion-scheduler"
+description = "Scheduling for DataFusion query engine"
+version = "7.0.0"
+homepage = "https://github.com/apache/arrow-datafusion"
+repository = "https://github.com/apache/arrow-datafusion"
+readme = "../README.md"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "query", "sql"]
+edition = "2021"
+rust-version = "1.58"
+
+[lib]
+name = "datafusion_scheduler"
+path = "src/lib.rs"
+
+[features]
+
+[dependencies]
+ahash = { version = "0.7", default-features = false }
+arrow = { version = "12" }
+async-trait = "0.1"
+datafusion = { path = "../core", version = "7.0.0" }

Review Comment:
   > Merge datafusion-scheduler with datafusion/core
   
   This is kind of what I was expecting (and then conditionally compile it based on a feature flag)



##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.
+///
+/// This in turn will allow for [`Pipeline`] implementations that are able to introduce
+/// parallelism beyond that expressed in their partitioning
+pub struct ExecutionPipeline {
+    proxied: Arc<dyn ExecutionPlan>,
+    inputs: Vec<Vec<Arc<Mutex<InputPartition>>>>,
+    outputs: Vec<Mutex<BoxStream<'static, ArrowResult<RecordBatch>>>>,
+}
+
+impl std::fmt::Debug for ExecutionPipeline {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let tree = debug_tree(self.proxied.as_ref());
+        f.debug_tuple("ExecutionNode").field(&tree).finish()
+    }
+}
+
+impl ExecutionPipeline {
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        depth: usize,
+    ) -> Result<Self> {
+        // The point in the plan at which to splice the plan graph
+        let mut splice_point = plan;
+        let mut parent_plans = Vec::with_capacity(depth.saturating_sub(1));
+        for _ in 0..depth {
+            let children = splice_point.children();
+            assert_eq!(
+                children.len(),
+                1,
+                "can only group through nodes with a single child"
+            );
+            parent_plans.push(splice_point);
+            splice_point = children.into_iter().next().unwrap();
+        }
+
+        // The children to replace with [`ProxyExecutionPlan`]
+        let children = splice_point.children();
+        let mut inputs = Vec::with_capacity(children.len());
+
+        // The spliced plan with its children replaced with [`ProxyExecutionPlan`]
+        let spliced = if !children.is_empty() {
+            let mut proxies: Vec<Arc<dyn ExecutionPlan>> =
+                Vec::with_capacity(children.len());
+
+            for child in children {
+                let count = child.output_partitioning().partition_count();
+
+                let mut child_inputs = Vec::with_capacity(count);
+                for _ in 0..count {
+                    child_inputs.push(Default::default())
+                }
+
+                inputs.push(child_inputs.clone());
+                proxies.push(Arc::new(ProxyExecutionPlan {
+                    inner: child,
+                    inputs: child_inputs,
+                }));
+            }
+
+            splice_point.with_new_children(proxies)?
+        } else {
+            splice_point.clone()
+        };
+
+        // Reconstruct the parent graph
+        let mut proxied = spliced;
+        for parent in parent_plans.into_iter().rev() {
+            proxied = parent.with_new_children(vec![proxied])?
+        }
+
+        // Construct the output streams
+        let output_count = proxied.output_partitioning().partition_count();
+        let outputs = (0..output_count)
+            .map(|x| {
+                let proxy_captured = proxied.clone();
+                let task_captured = task_context.clone();
+                let fut = async move {
+                    proxy_captured
+                        .execute(x, task_captured)
+                        .await
+                        .map_err(|e| ArrowError::ExternalError(Box::new(e)))
+                };
+
+                // Use futures::stream::once to handle operators that perform computation
+                // within `ExecutionPlan::execute`. If we evaluated these futures here
+                // we could potentially block indefinitely waiting for inputs that will
+                // never arrive as the query isn't scheduled yet
+                Mutex::new(futures::stream::once(fut).try_flatten().boxed())
+            })
+            .collect();
+
+        Ok(Self {
+            proxied,
+            inputs,
+            outputs,
+        })
+    }
+}
+
+impl Pipeline for ExecutionPipeline {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()> {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.buffer.push_back(input);
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+        Ok(())
+    }
+
+    fn close(&self, child: usize, partition: usize) {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.is_closed = true;
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn output_partitions(&self) -> usize {
+        self.outputs.len()
+    }
+
+    /// Poll an output partition, attempting to get its output
+    fn poll_partition(
+        &self,
+        cx: &mut Context<'_>,
+        partition: usize,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        self.outputs[partition]
+            .lock()
+            .poll_next_unpin(cx)
+            .map(|opt| opt.map(|r| r.map_err(Into::into)))
+    }
+}
+
+#[derive(Debug, Default)]
+struct InputPartition {
+    buffer: VecDeque<RecordBatch>,
+    wait_list: Vec<Waker>,

Review Comment:
   fiar enough



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r855283083


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {

Review Comment:
   This is now part of how the dispatch to separate pools functions and so is necessary



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1105643839

   I plan to review this carefully tomorrow morning


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856293325


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,276 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {

Review Comment:
   I like PipelinePlan



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] yjshen commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
yjshen commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1116248339

   Thanks @tustvold, I'll finish my review tomorrow.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851347286


##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,109 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Push vs Pull Execution
+///
+/// Whilst the interface exposed to the scheduler is push-based, in which member functions

Review Comment:
   ```suggestion
   /// Whilst the interface exposed to the scheduler is push-based, the order of member function
   ```



##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,324 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// stateless operators like filters, and stateful aggregations, and are better aligned
+/// with a push-based execution model. This in turn will allow for [`Pipeline`] implementations
+/// that are able to introduce parallelism beyond that expressed in their partitioning

Review Comment:
   I still don't understand this distinction stateful and non stateful operators.
   
   Maybe the difference is the "pipeline breaking" ability (the standard database term for operators that may not produce output until they have seen  some/all of their input)



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,386 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::{debug, error};
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// and is designed to decouple the execution parallelism from the parallelism expressed in
+/// the physical plan as partitions.
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {
+        *msg
+    } else if let Some(msg) = panic.downcast_ref::<String>() {
+        msg.as_str()
+    } else {
+        "UNKNOWN"
+    };
+
+    format!("worker {} panicked with: {}", worker, message)
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+///
+/// There is no guaranteed order of execution, as workers may steal at any time. However,

Review Comment:
   👍 



##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,109 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Push vs Pull Execution
+///
+/// Whilst the interface exposed to the scheduler is push-based, in which member functions
+/// computation is performed is intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none none
+/// if the operator is waiting on data from a different input partition
+///
+/// An aggregation pipeline which combines data from one or more input partitions into
+/// a single output partition. [`Pipeline::push`] would eagerly update the computed
+/// aggregates, and the final [`Pipeline::close`] trigger flushing these to the output

Review Comment:
   There is another related strategy for hash aggregation which is to do partial aggregation and when the hash table is full flush the output hash table and start again fresh (meaning many batch pushes produce some output, but some pushes would produce output - the content of the hash table).
   
   Perhaps this is similar to merge-pipeline from a scheduler perspective



##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,109 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+/// This is not intended as an eventual replacement for the physical plan representation
+/// within DataFusion, [`ExecutionPlan`], but rather a generic interface that
+/// parts of the physical plan are "compiled" into by the scheduler.
+///
+/// # Push vs Pull Execution
+///
+/// Whilst the interface exposed to the scheduler is push-based, in which member functions
+/// computation is performed is intentionally left as an implementation detail of the [`Pipeline`]
+///
+/// This allows flexibility to support the following different patterns, and potentially more:
+///
+/// An eager, push-based pipeline, that processes a batch synchronously in [`Pipeline::push`]
+/// and immediately wakes the corresponding output partition.
+///
+/// A parallel, push-based pipeline, that enqueues the processing of a batch to the rayon
+/// thread pool in [`Pipeline::push`], and wakes the corresponding output partition when
+/// the job completes. Order and non-order preserving variants are possible
+///
+/// A merge pipeline which combines data from one or more input partitions into one or
+/// more output partitions. [`Pipeline::push`] adds data to an input buffer, and wakes
+/// any output partitions that may now be able to make progress. This may be none none

Review Comment:
   ```suggestion
   /// any output partitions that may now be able to make progress. This may be none
   ```



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851360139


##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,324 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// stateless operators like filters, and stateful aggregations, and are better aligned
+/// with a push-based execution model. This in turn will allow for [`Pipeline`] implementations
+/// that are able to introduce parallelism beyond that expressed in their partitioning

Review Comment:
   Yes, I shall reword to use pipeline breaker instead of stateful 👍



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850809762


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()

Review Comment:
   I'd rather keep the check cheap, as it will be called fairly frequently, it's mainly intended as a sanity check



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] andygrove commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
andygrove commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r859354319


##########
datafusion/scheduler/Cargo.toml:
##########
@@ -0,0 +1,57 @@
+# 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.
+
+[package]
+name = "datafusion-scheduler"
+description = "Scheduling for DataFusion query engine"
+version = "7.0.0"
+homepage = "https://github.com/apache/arrow-datafusion"
+repository = "https://github.com/apache/arrow-datafusion"
+readme = "../README.md"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "query", "sql"]
+edition = "2021"
+rust-version = "1.58"
+
+[lib]
+name = "datafusion_scheduler"
+path = "src/lib.rs"
+
+[features]
+
+[dependencies]
+ahash = { version = "0.7", default-features = false }
+arrow = { version = "12" }
+async-trait = "0.1"
+datafusion = { path = "../core", version = "7.0.0" }

Review Comment:
   I would rather see the scheduler added in core than see physical plans (ExecutionPlan) move to datafusion-common



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1116847798

   I'll also try and give it another read tomorrow ~ 12:00 UTC


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856478974


##########
datafusion/scheduler/Cargo.toml:
##########
@@ -0,0 +1,57 @@
+# 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.
+
+[package]
+name = "datafusion-scheduler"
+description = "Scheduling for DataFusion query engine"
+version = "7.0.0"
+homepage = "https://github.com/apache/arrow-datafusion"
+repository = "https://github.com/apache/arrow-datafusion"
+readme = "../README.md"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "query", "sql"]
+edition = "2021"
+rust-version = "1.58"
+
+[lib]
+name = "datafusion_scheduler"
+path = "src/lib.rs"
+
+[features]
+
+[dependencies]
+ahash = { version = "0.7", default-features = false }
+arrow = { version = "12" }
+async-trait = "0.1"
+datafusion = { path = "../core", version = "7.0.0" }

Review Comment:
   So, I'm not sure how to do this without introducing a circular dependency. Currently ExecutionPlan is defined in datafusion/core. Options I can see are:
   
   * Move ExecutionPlan into datafusion-common, this would require also moving the metrics
   * Merge datafusion-scheduler with datafusion/core
   
   I'm not sure which would be preferable?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1105438482

   No rush at all, I am away much of next week, and this is a bit of a chonker, so happy to wait and give people adequate time to review 👍


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1117360252

   > Can we provide some diagrams to illustrate how a tree of ExecutionPlan is break-up into pipelines and how pipelines are run to finish?
   
   I volunteer to help create ascii art for this. If you want to meet  @tustvold and we can work some examples I can ascii-ify them 😎    


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r864881475


##########
datafusion/core/src/lib.rs:
##########
@@ -218,6 +218,8 @@ pub mod physical_optimizer;
 pub mod physical_plan;
 pub mod prelude;
 pub mod scalar;
+#[cfg(feature = "scheduler")]

Review Comment:
   👍 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850812580


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,

Review Comment:
   This replicates the current logic in RelartitionExec and is a simple way to avoid deadlocks in circular plans. I figured if it was fine before, it was fine for now and we can sort out back pressure at a later date.
   
   Will add comments



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850764524


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool

Review Comment:
   ```suggestion
       /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
   ```



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan

Review Comment:
   ```suggestion
   /// A [`Scheduler`] creates and manages a pool of dedicated worker threads on which
   /// query execution is scheduled. This is based on the idea of [Morsel-Driven Parallelism]
   /// and is designed to decouple execution parallelism from the parallelism expressed in 
   /// the physical plan as `partitions`. 
   ```



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()

Review Comment:
   should it maybe check the thread name as well in case some other part of the system is using rayon??



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering

Review Comment:
   I think it would help here to explain what guarantees that "FIFO" tasks gives -- is it best effort or can code rely on it?



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {
+    pool: Arc<ThreadPool>,
+}
+
+impl Spawner {
+    pub fn spawn(&self, task: Task) {
+        debug!("Spawning {:?} to any worker", task);
+        self.pool.spawn(move || task.do_work());
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use arrow::util::pretty::pretty_format_batches;
+    use std::ops::Range;
+
+    use futures::TryStreamExt;
+    use log::info;
+    use rand::distributions::uniform::SampleUniform;
+    use rand::{thread_rng, Rng};
+
+    use datafusion::arrow::array::{ArrayRef, PrimitiveArray};
+    use datafusion::arrow::datatypes::{ArrowPrimitiveType, Float64Type, Int32Type};
+    use datafusion::arrow::record_batch::RecordBatch;
+    use datafusion::datasource::{MemTable, TableProvider};
+    use datafusion::physical_plan::displayable;
+    use datafusion::prelude::{SessionConfig, SessionContext};
+
+    use super::*;
+
+    fn generate_primitive<T, R>(
+        rng: &mut R,
+        len: usize,
+        valid_percent: f64,
+        range: Range<T::Native>,
+    ) -> ArrayRef
+    where
+        T: ArrowPrimitiveType,
+        T::Native: SampleUniform,
+        R: Rng,
+    {
+        Arc::new(PrimitiveArray::<T>::from_iter((0..len).map(|_| {
+            rng.gen_bool(valid_percent)
+                .then(|| rng.gen_range(range.clone()))
+        })))
+    }
+
+    fn generate_batch<R: Rng>(
+        rng: &mut R,
+        row_count: usize,
+        id_offset: i32,
+    ) -> RecordBatch {
+        let id_range = id_offset..(row_count as i32 + id_offset);
+        let a = generate_primitive::<Int32Type, _>(rng, row_count, 0.5, 0..1000);
+        let b = generate_primitive::<Float64Type, _>(rng, row_count, 0.5, 0. ..1000.);
+        let id = PrimitiveArray::<Int32Type>::from_iter_values(id_range);
+
+        RecordBatch::try_from_iter_with_nullable([
+            ("a", a, true),
+            ("b", b, true),
+            ("id", Arc::new(id), false),
+        ])
+        .unwrap()
+    }
+
+    fn make_batches() -> Vec<Vec<RecordBatch>> {
+        let mut rng = thread_rng();
+
+        let batches_per_partition = 20;
+        let rows_per_batch = 100;
+        let num_partitions = 2;
+
+        let mut id_offset = 0;
+
+        (0..num_partitions)
+            .map(|_| {
+                (0..batches_per_partition)
+                    .map(|_| {
+                        let batch = generate_batch(&mut rng, rows_per_batch, id_offset);
+                        id_offset += rows_per_batch as i32;
+                        batch
+                    })
+                    .collect()
+            })
+            .collect()
+    }
+
+    fn make_provider() -> Arc<dyn TableProvider> {
+        let batches = make_batches();
+        let schema = batches.first().unwrap().first().unwrap().schema();
+        Arc::new(MemTable::try_new(schema, make_batches()).unwrap())
+    }
+
+    fn init_logging() {
+        let _ = env_logger::builder().is_test(true).try_init();
+    }
+
+    #[tokio::test]
+    async fn test_simple() {
+        init_logging();
+
+        let scheduler = Scheduler::new(4);
+
+        let config = SessionConfig::new().with_target_partitions(4);
+        let context = SessionContext::with_config(config);
+
+        context.register_table("table1", make_provider()).unwrap();
+        context.register_table("table2", make_provider()).unwrap();
+
+        let queries = [
+            "select * from table1 order by id",
+            "select * from table1 where table1.a > 100 order by id",
+            "select distinct a from table1 where table1.b > 100 order by a",
+            "select * from table1 join table2 on table1.id = table2.id order by table1.id",
+            "select id from table1 union all select id from table2 order by id",
+            "select id from table1 union all select id from table2 where a > 100 order by id",
+            "select id, b from (select id, b from table1 union all select id, b from table2 where a > 100 order by id) as t where b > 10 order by id, b",
+            "select id, MIN(b), MAX(b), AVG(b) from table1 group by id order by id",
+            "select count(*) from table1 where table1.a > 4",
+        ];
+
+        for sql in queries {
+            let task = context.task_ctx();
+
+            let query = context.sql(sql).await.unwrap();
+
+            let plan = query.create_physical_plan().await.unwrap();
+
+            info!("Plan: {}", displayable(plan.as_ref()).indent());
+
+            let stream = scheduler.schedule(plan, task).unwrap();

Review Comment:
   This is a neat way of demonstrating "end to end" test of this scheduler



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {

Review Comment:
   Describing the use of this struct would be helpful I think. Maybe explain why the wrapper around `Arc<ThreadPool>` is doing other than adding some indirection?



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,

Review Comment:
   I wonder if somewhere you could add a description (or a diagram -- 🙏 ) explaining how the concepts of `Pipeline` `Query`, `child` and `ExecutionPlan` and `OutputLink` are connected? It wasn't immediately obvious to me



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]
+    pub fn pipelines(&self) -> &[RoutablePipeline] {
+        &self.pipelines
+    }
+
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    pub fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    pub fn send_query_output(&self, output: ArrowResult<RecordBatch>) {
+        let _ = self.output.unbounded_send(output);
+    }
+
+    /// Returns the [`Spawner`] associated with this [`Query`]
+    pub fn spawner(&self) -> &Spawner {
+        &self.spawner
+    }
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple [`ExecutionPlan`], [`ExecGroup`] stores this state
+struct ExecGroup {

Review Comment:
   I didn't understand the need to add an extra layer of grouping within a pipeline -- I would have expected everything in a pipeline to run together, and if it needed to run separately it would run as a separate pipeline



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,

Review Comment:
   using an unbounded stream means there is no backpressure on execution, right? As in the query will run to completion as fast as possible, regardless of how fast the consumer consumes it. 
   
   Is there a reason this isn't a bounded receiver?



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]
+    pub fn pipelines(&self) -> &[RoutablePipeline] {
+        &self.pipelines
+    }
+
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    pub fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    pub fn send_query_output(&self, output: ArrowResult<RecordBatch>) {
+        let _ = self.output.unbounded_send(output);
+    }
+
+    /// Returns the [`Spawner`] associated with this [`Query`]
+    pub fn spawner(&self) -> &Spawner {
+        &self.spawner
+    }
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple [`ExecutionPlan`], [`ExecGroup`] stores this state
+struct ExecGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`Query`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`Query`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+struct QueryBuilder {
+    task_context: Arc<TaskContext>,
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more [`ExecutionPlan`] to combine together into
+    /// a single [`ExecutionPipeline`]
+    exec_buffer: Option<ExecGroup>,
+}
+
+impl QueryBuilder {
+    fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        Self {
+            in_progress: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            exec_buffer: None,
+        }
+    }
+
+    /// Flush the current group of [`ExecutionPlan`] stored in `exec_buffer`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.exec_buffer.take().unwrap();
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the node to the current group of execution plan to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should

Review Comment:
   Can you explain what the tradeoffs are? At first it would seem to me that the larger a pipeline the better (as there is more work that can be done and so the scheduler will have the greatest chance of putting all the threads to use?



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition)

Review Comment:
   Is it correct that this call may do a non trivial amount of work (aka this is the push / keep working on the batch in the same thread that produced it)?



##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,72 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+pub trait Pipeline: Send + Sync + std::fmt::Debug {
+    /// Push a [`RecordBatch`] to the given input partition

Review Comment:
   is the expectation this returns immediately (as in the pipeline enqueues work to do that must then be polled until it is done)?
   
   Or does it immediately do the work in the current thread (what I would expect a "push" scheduler to do)



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {

Review Comment:
   To check my understanding:
   
   1. This code creates as many `Task`s  as there are "units of of work" -- aka (`pipeline`, `partition`) pairs
   2. There is a big free-for-all race where each Task is asked "do you have any output yet"
   3. If the answer to "yes" the scheduler calls "Pipeline::push` immediately with the output batch
   



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.

Review Comment:
   Can we add some commentary here about how the pipelines are chosen?



##########
datafusion/core/Cargo.toml:
##########
@@ -117,10 +117,6 @@ name = "scalar"
 harness = false
 name = "physical_plan"
 
-[[bench]]
-harness = false
-name = "parquet_query_sql"
-
 [[bench]]

Review Comment:
   Do you think the  scheduler should be an optional `feature` of the `core` (eventually?)



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;

Review Comment:
   Today I learned about `ArcWake` -- the number of specialized things in the Rust futures ecosystem boggles my mind



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]

Review Comment:
   ```suggestion
       /// Returns a list of this queries [`RoutablePipeline`]
   ```



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering

Review Comment:
   I suggest starting with at least sequentially consistent  for coordination to be more resilient to (hard to debug) bugs unless there is some compelling performance measurement meaning that Relaxed ordering is better



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {
+    pool: Arc<ThreadPool>,
+}
+
+impl Spawner {
+    pub fn spawn(&self, task: Task) {
+        debug!("Spawning {:?} to any worker", task);
+        self.pool.spawn(move || task.do_work());
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use arrow::util::pretty::pretty_format_batches;
+    use std::ops::Range;
+
+    use futures::TryStreamExt;
+    use log::info;
+    use rand::distributions::uniform::SampleUniform;
+    use rand::{thread_rng, Rng};
+
+    use datafusion::arrow::array::{ArrayRef, PrimitiveArray};
+    use datafusion::arrow::datatypes::{ArrowPrimitiveType, Float64Type, Int32Type};
+    use datafusion::arrow::record_batch::RecordBatch;
+    use datafusion::datasource::{MemTable, TableProvider};
+    use datafusion::physical_plan::displayable;
+    use datafusion::prelude::{SessionConfig, SessionContext};
+
+    use super::*;
+
+    fn generate_primitive<T, R>(
+        rng: &mut R,
+        len: usize,
+        valid_percent: f64,
+        range: Range<T::Native>,
+    ) -> ArrayRef
+    where
+        T: ArrowPrimitiveType,
+        T::Native: SampleUniform,
+        R: Rng,
+    {
+        Arc::new(PrimitiveArray::<T>::from_iter((0..len).map(|_| {
+            rng.gen_bool(valid_percent)
+                .then(|| rng.gen_range(range.clone()))
+        })))
+    }
+
+    fn generate_batch<R: Rng>(
+        rng: &mut R,
+        row_count: usize,
+        id_offset: i32,
+    ) -> RecordBatch {
+        let id_range = id_offset..(row_count as i32 + id_offset);
+        let a = generate_primitive::<Int32Type, _>(rng, row_count, 0.5, 0..1000);
+        let b = generate_primitive::<Float64Type, _>(rng, row_count, 0.5, 0. ..1000.);
+        let id = PrimitiveArray::<Int32Type>::from_iter_values(id_range);
+
+        RecordBatch::try_from_iter_with_nullable([
+            ("a", a, true),
+            ("b", b, true),
+            ("id", Arc::new(id), false),
+        ])
+        .unwrap()
+    }
+
+    fn make_batches() -> Vec<Vec<RecordBatch>> {
+        let mut rng = thread_rng();
+
+        let batches_per_partition = 20;
+        let rows_per_batch = 100;
+        let num_partitions = 2;
+
+        let mut id_offset = 0;
+
+        (0..num_partitions)
+            .map(|_| {
+                (0..batches_per_partition)
+                    .map(|_| {
+                        let batch = generate_batch(&mut rng, rows_per_batch, id_offset);
+                        id_offset += rows_per_batch as i32;
+                        batch
+                    })
+                    .collect()
+            })
+            .collect()
+    }
+
+    fn make_provider() -> Arc<dyn TableProvider> {
+        let batches = make_batches();
+        let schema = batches.first().unwrap().first().unwrap().schema();
+        Arc::new(MemTable::try_new(schema, make_batches()).unwrap())
+    }
+
+    fn init_logging() {
+        let _ = env_logger::builder().is_test(true).try_init();
+    }
+
+    #[tokio::test]
+    async fn test_simple() {
+        init_logging();
+
+        let scheduler = Scheduler::new(4);
+
+        let config = SessionConfig::new().with_target_partitions(4);
+        let context = SessionContext::with_config(config);
+
+        context.register_table("table1", make_provider()).unwrap();
+        context.register_table("table2", make_provider()).unwrap();
+
+        let queries = [
+            "select * from table1 order by id",
+            "select * from table1 where table1.a > 100 order by id",
+            "select distinct a from table1 where table1.b > 100 order by a",
+            "select * from table1 join table2 on table1.id = table2.id order by table1.id",
+            "select id from table1 union all select id from table2 order by id",
+            "select id from table1 union all select id from table2 where a > 100 order by id",
+            "select id, b from (select id, b from table1 union all select id, b from table2 where a > 100 order by id) as t where b > 10 order by id, b",
+            "select id, MIN(b), MAX(b), AVG(b) from table1 group by id order by id",
+            "select count(*) from table1 where table1.a > 4",
+        ];
+
+        for sql in queries {
+            let task = context.task_ctx();
+
+            let query = context.sql(sql).await.unwrap();
+
+            let plan = query.create_physical_plan().await.unwrap();
+
+            info!("Plan: {}", displayable(plan.as_ref()).indent());
+
+            let stream = scheduler.schedule(plan, task).unwrap();
+            let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+            let expected = query.collect().await.unwrap();
+
+            let total_expected = expected.iter().map(|x| x.num_rows()).sum::<usize>();
+            let total_scheduled = scheduled.iter().map(|x| x.num_rows()).sum::<usize>();
+            assert_eq!(total_expected, total_scheduled);
+
+            info!("Query \"{}\" produced {} rows", sql, total_expected);
+
+            let expected = pretty_format_batches(&expected).unwrap().to_string();
+            let scheduled = pretty_format_batches(&scheduled).unwrap().to_string();
+
+            assert_eq!(

Review Comment:
   `assert_batches_eq!`?



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {

Review Comment:
   If the existing operators were not `async` / `Future`s, would this code still look like a poll? Or would it do something else?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850818283


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {

Review Comment:
   Not sure, my hope is that most operators will use a simpler serial trait, and only a few will need to implement the Pipeline trait. I think some sort of async interface is unavoidable, and using the standard rust types for this seems sensible, but hopefully we can hide most of it away 😃



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850811404


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {
+    pool: Arc<ThreadPool>,
+}
+
+impl Spawner {
+    pub fn spawn(&self, task: Task) {
+        debug!("Spawning {:?} to any worker", task);
+        self.pool.spawn(move || task.do_work());
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use arrow::util::pretty::pretty_format_batches;
+    use std::ops::Range;
+
+    use futures::TryStreamExt;
+    use log::info;
+    use rand::distributions::uniform::SampleUniform;
+    use rand::{thread_rng, Rng};
+
+    use datafusion::arrow::array::{ArrayRef, PrimitiveArray};
+    use datafusion::arrow::datatypes::{ArrowPrimitiveType, Float64Type, Int32Type};
+    use datafusion::arrow::record_batch::RecordBatch;
+    use datafusion::datasource::{MemTable, TableProvider};
+    use datafusion::physical_plan::displayable;
+    use datafusion::prelude::{SessionConfig, SessionContext};
+
+    use super::*;
+
+    fn generate_primitive<T, R>(
+        rng: &mut R,
+        len: usize,
+        valid_percent: f64,
+        range: Range<T::Native>,
+    ) -> ArrayRef
+    where
+        T: ArrowPrimitiveType,
+        T::Native: SampleUniform,
+        R: Rng,
+    {
+        Arc::new(PrimitiveArray::<T>::from_iter((0..len).map(|_| {
+            rng.gen_bool(valid_percent)
+                .then(|| rng.gen_range(range.clone()))
+        })))
+    }
+
+    fn generate_batch<R: Rng>(
+        rng: &mut R,
+        row_count: usize,
+        id_offset: i32,
+    ) -> RecordBatch {
+        let id_range = id_offset..(row_count as i32 + id_offset);
+        let a = generate_primitive::<Int32Type, _>(rng, row_count, 0.5, 0..1000);
+        let b = generate_primitive::<Float64Type, _>(rng, row_count, 0.5, 0. ..1000.);
+        let id = PrimitiveArray::<Int32Type>::from_iter_values(id_range);
+
+        RecordBatch::try_from_iter_with_nullable([
+            ("a", a, true),
+            ("b", b, true),
+            ("id", Arc::new(id), false),
+        ])
+        .unwrap()
+    }
+
+    fn make_batches() -> Vec<Vec<RecordBatch>> {
+        let mut rng = thread_rng();
+
+        let batches_per_partition = 20;
+        let rows_per_batch = 100;
+        let num_partitions = 2;
+
+        let mut id_offset = 0;
+
+        (0..num_partitions)
+            .map(|_| {
+                (0..batches_per_partition)
+                    .map(|_| {
+                        let batch = generate_batch(&mut rng, rows_per_batch, id_offset);
+                        id_offset += rows_per_batch as i32;
+                        batch
+                    })
+                    .collect()
+            })
+            .collect()
+    }
+
+    fn make_provider() -> Arc<dyn TableProvider> {
+        let batches = make_batches();
+        let schema = batches.first().unwrap().first().unwrap().schema();
+        Arc::new(MemTable::try_new(schema, make_batches()).unwrap())
+    }
+
+    fn init_logging() {
+        let _ = env_logger::builder().is_test(true).try_init();
+    }
+
+    #[tokio::test]
+    async fn test_simple() {
+        init_logging();
+
+        let scheduler = Scheduler::new(4);
+
+        let config = SessionConfig::new().with_target_partitions(4);
+        let context = SessionContext::with_config(config);
+
+        context.register_table("table1", make_provider()).unwrap();
+        context.register_table("table2", make_provider()).unwrap();
+
+        let queries = [
+            "select * from table1 order by id",
+            "select * from table1 where table1.a > 100 order by id",
+            "select distinct a from table1 where table1.b > 100 order by a",
+            "select * from table1 join table2 on table1.id = table2.id order by table1.id",
+            "select id from table1 union all select id from table2 order by id",
+            "select id from table1 union all select id from table2 where a > 100 order by id",
+            "select id, b from (select id, b from table1 union all select id, b from table2 where a > 100 order by id) as t where b > 10 order by id, b",
+            "select id, MIN(b), MAX(b), AVG(b) from table1 group by id order by id",
+            "select count(*) from table1 where table1.a > 4",
+        ];
+
+        for sql in queries {
+            let task = context.task_ctx();
+
+            let query = context.sql(sql).await.unwrap();
+
+            let plan = query.create_physical_plan().await.unwrap();
+
+            info!("Plan: {}", displayable(plan.as_ref()).indent());
+
+            let stream = scheduler.schedule(plan, task).unwrap();
+            let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+            let expected = query.collect().await.unwrap();
+
+            let total_expected = expected.iter().map(|x| x.num_rows()).sum::<usize>();
+            let total_scheduled = scheduled.iter().map(|x| x.num_rows()).sum::<usize>();
+            assert_eq!(total_expected, total_scheduled);
+
+            info!("Query \"{}\" produced {} rows", sql, total_expected);
+
+            let expected = pretty_format_batches(&expected).unwrap().to_string();
+            let scheduled = pretty_format_batches(&scheduled).unwrap().to_string();
+
+            assert_eq!(

Review Comment:
   I think that requires one of the batches to not be formatted, and the other split on newlines, this seemed simpler?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850811404


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {
+    pool: Arc<ThreadPool>,
+}
+
+impl Spawner {
+    pub fn spawn(&self, task: Task) {
+        debug!("Spawning {:?} to any worker", task);
+        self.pool.spawn(move || task.do_work());
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use arrow::util::pretty::pretty_format_batches;
+    use std::ops::Range;
+
+    use futures::TryStreamExt;
+    use log::info;
+    use rand::distributions::uniform::SampleUniform;
+    use rand::{thread_rng, Rng};
+
+    use datafusion::arrow::array::{ArrayRef, PrimitiveArray};
+    use datafusion::arrow::datatypes::{ArrowPrimitiveType, Float64Type, Int32Type};
+    use datafusion::arrow::record_batch::RecordBatch;
+    use datafusion::datasource::{MemTable, TableProvider};
+    use datafusion::physical_plan::displayable;
+    use datafusion::prelude::{SessionConfig, SessionContext};
+
+    use super::*;
+
+    fn generate_primitive<T, R>(
+        rng: &mut R,
+        len: usize,
+        valid_percent: f64,
+        range: Range<T::Native>,
+    ) -> ArrayRef
+    where
+        T: ArrowPrimitiveType,
+        T::Native: SampleUniform,
+        R: Rng,
+    {
+        Arc::new(PrimitiveArray::<T>::from_iter((0..len).map(|_| {
+            rng.gen_bool(valid_percent)
+                .then(|| rng.gen_range(range.clone()))
+        })))
+    }
+
+    fn generate_batch<R: Rng>(
+        rng: &mut R,
+        row_count: usize,
+        id_offset: i32,
+    ) -> RecordBatch {
+        let id_range = id_offset..(row_count as i32 + id_offset);
+        let a = generate_primitive::<Int32Type, _>(rng, row_count, 0.5, 0..1000);
+        let b = generate_primitive::<Float64Type, _>(rng, row_count, 0.5, 0. ..1000.);
+        let id = PrimitiveArray::<Int32Type>::from_iter_values(id_range);
+
+        RecordBatch::try_from_iter_with_nullable([
+            ("a", a, true),
+            ("b", b, true),
+            ("id", Arc::new(id), false),
+        ])
+        .unwrap()
+    }
+
+    fn make_batches() -> Vec<Vec<RecordBatch>> {
+        let mut rng = thread_rng();
+
+        let batches_per_partition = 20;
+        let rows_per_batch = 100;
+        let num_partitions = 2;
+
+        let mut id_offset = 0;
+
+        (0..num_partitions)
+            .map(|_| {
+                (0..batches_per_partition)
+                    .map(|_| {
+                        let batch = generate_batch(&mut rng, rows_per_batch, id_offset);
+                        id_offset += rows_per_batch as i32;
+                        batch
+                    })
+                    .collect()
+            })
+            .collect()
+    }
+
+    fn make_provider() -> Arc<dyn TableProvider> {
+        let batches = make_batches();
+        let schema = batches.first().unwrap().first().unwrap().schema();
+        Arc::new(MemTable::try_new(schema, make_batches()).unwrap())
+    }
+
+    fn init_logging() {
+        let _ = env_logger::builder().is_test(true).try_init();
+    }
+
+    #[tokio::test]
+    async fn test_simple() {
+        init_logging();
+
+        let scheduler = Scheduler::new(4);
+
+        let config = SessionConfig::new().with_target_partitions(4);
+        let context = SessionContext::with_config(config);
+
+        context.register_table("table1", make_provider()).unwrap();
+        context.register_table("table2", make_provider()).unwrap();
+
+        let queries = [
+            "select * from table1 order by id",
+            "select * from table1 where table1.a > 100 order by id",
+            "select distinct a from table1 where table1.b > 100 order by a",
+            "select * from table1 join table2 on table1.id = table2.id order by table1.id",
+            "select id from table1 union all select id from table2 order by id",
+            "select id from table1 union all select id from table2 where a > 100 order by id",
+            "select id, b from (select id, b from table1 union all select id, b from table2 where a > 100 order by id) as t where b > 10 order by id, b",
+            "select id, MIN(b), MAX(b), AVG(b) from table1 group by id order by id",
+            "select count(*) from table1 where table1.a > 4",
+        ];
+
+        for sql in queries {
+            let task = context.task_ctx();
+
+            let query = context.sql(sql).await.unwrap();
+
+            let plan = query.create_physical_plan().await.unwrap();
+
+            info!("Plan: {}", displayable(plan.as_ref()).indent());
+
+            let stream = scheduler.schedule(plan, task).unwrap();
+            let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+            let expected = query.collect().await.unwrap();
+
+            let total_expected = expected.iter().map(|x| x.num_rows()).sum::<usize>();
+            let total_scheduled = scheduled.iter().map(|x| x.num_rows()).sum::<usize>();
+            assert_eq!(total_expected, total_scheduled);
+
+            info!("Query \"{}\" produced {} rows", sql, total_expected);
+
+            let expected = pretty_format_batches(&expected).unwrap().to_string();
+            let scheduled = pretty_format_batches(&scheduled).unwrap().to_string();
+
+            assert_eq!(

Review Comment:
   I think that requires one of the batches to not be formatted, and this seemed simpler?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851343859


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,

Review Comment:
   It is true that any plan with `ReparitionExec` will also potentially infinitely buffer the input. Maybe not ideal though



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,

Review Comment:
   It is true that any plan with `RepartitionExec` will also potentially infinitely buffer the input. Maybe not ideal though



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851344625


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,225 @@
+// 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.
+
+use crate::{is_worker, spawn_local, spawn_local_fifo, Query};
+use futures::task::ArcWake;
+use log::{debug, trace};
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Arc<Query>) {
+    debug!("Spawning query: {:#?}", query);
+
+    let spawner = query.spawner();
+
+    for (pipeline_idx, query_pipeline) in query.pipelines().iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`Query`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`Query`] is dropped it signals completion of query execution
+    query: Arc<Query>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = self.query.pipelines()[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        //
+        // We aren't using the wake count to synchronise other memory, and so can
+        // use relaxed memory ordering
+        let wake_count = self.waker.wake_count.load(Ordering::Relaxed);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = self.query.pipelines();
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition)

Review Comment:
   > Yes, which does create an interesting situation as the next output can't be polled until the last batch has been completely processed
   
   But this should be ok because all the other threads should be actively doing something if there is work to do, right?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856178334


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,381 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::{Query, QueryBuilder, RoutablePipeline};
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::QueryResults;
+
+mod pipeline;
+mod query;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// and is designed to decouple the execution parallelism from the parallelism expressed in
+/// the physical plan as partitions.
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// Queries scheduled on a [`Scheduler`] will run to completion even if the
+/// [`Scheduler`] is dropped
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {

Review Comment:
   Stylistically I recommend putting this structure (and its very nice documentation) at the top of this source file so people see it first. 
   
   I also think a doc example showing how to use it, would be very very helpful (so people didn't have to look at the unit test). Something like:
   
   ```rust
           let scheduler = Scheduler::new(4);
   
           let config = SessionConfig::new().with_target_partitions(4);
           let context = SessionContext::with_config(config);
           context.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?;
           let plan = context.sql("SELECT MIN(b) FROM example")
              .await
              .unwrap()
              .create_physical_plan()
              .unwrap()
   
           let stream = scheduler.schedule(plan, task).unwrap();
           let scheduled: Vec<_> = stream.try_collect().await.unwrap();
           let results = query.collect().await.unwrap();
   ```



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,381 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::{Query, QueryBuilder, RoutablePipeline};
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::QueryResults;
+
+mod pipeline;
+mod query;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// and is designed to decouple the execution parallelism from the parallelism expressed in
+/// the physical plan as partitions.
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// Queries scheduled on a [`Scheduler`] will run to completion even if the
+/// [`Scheduler`] is dropped
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced

Review Comment:
   ```suggestion
       /// Returns a [`QueryResults`] (stream of [`RecordBatch`]es) that can be used to receive results as they are produced
   ```



##########
datafusion/scheduler/Cargo.toml:
##########
@@ -0,0 +1,57 @@
+# 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.
+
+[package]
+name = "datafusion-scheduler"
+description = "Scheduling for DataFusion query engine"
+version = "7.0.0"
+homepage = "https://github.com/apache/arrow-datafusion"
+repository = "https://github.com/apache/arrow-datafusion"
+readme = "../README.md"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "query", "sql"]
+edition = "2021"
+rust-version = "1.58"
+
+[lib]
+name = "datafusion_scheduler"
+path = "src/lib.rs"
+
+[features]
+
+[dependencies]
+ahash = { version = "0.7", default-features = false }
+arrow = { version = "12" }
+async-trait = "0.1"
+datafusion = { path = "../core", version = "7.0.0" }

Review Comment:
   by structuring the crate this way it means `datafusions-scheduler` has to be brought in as its own crate. 
   
   I think it would be more consistent if datafusion-scheduler was an optional dependency of `datafusion` (aka `datafusion/core` like the `jit` module. That way users of the scheduler do not have to import the scheduler crate itself.



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,276 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    pub pipelines: Vec<RoutablePipeline>,
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple operators, [`OperatorGroup`] stores this state
+struct OperatorGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`Query`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`Query`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+pub struct QueryBuilder {
+    task_context: Arc<TaskContext>,
+
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more operators to combine
+    /// together into a single [`ExecutionPipeline`]
+    execution_operators: Option<OperatorGroup>,
+}
+
+impl QueryBuilder {
+    pub fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        Self {
+            in_progress: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            execution_operators: None,
+        }
+    }
+
+    /// Flush the current group of operators stored in `execution_operators`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.execution_operators.take().unwrap();
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the operator to the current group of operators to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should
+        match self.execution_operators.as_mut() {
+            Some(buffer) => {
+                assert_eq!(parent, buffer.output, "QueryBuilder out of sync");
+                buffer.depth += 1;
+            }
+            None => {
+                self.execution_operators = Some(OperatorGroup {
+                    output: parent,
+                    root: plan,
+                    depth: 0,
+                })
+            }
+        }
+
+        match children.len() {
+            1 => {
+                // Enqueue the children with the parent of the `OperatorGroup`
+                self.to_visit
+                    .push((children.into_iter().next().unwrap(), parent))
+            }
+            _ => {
+                // We can only recursively group through nodes with a single child, therefore
+                // if this node has multiple children, we now need to flush the buffer and
+                // enqueue its children with this new pipeline as its parent
+                let node = self.flush_exec()?;
+                self.enqueue_children(children, node);
+            }
+        }
+
+        Ok(())
+    }
+
+    /// Add the given list of children to the stack of [`ExecutionPlan`] to visit
+    fn enqueue_children(
+        &mut self,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+        parent_node_idx: usize,
+    ) {
+        for (child_idx, child) in children.into_iter().enumerate() {
+            self.to_visit.push((
+                child,
+                Some(OutputLink {
+                    pipeline: parent_node_idx,
+                    child: child_idx,
+                }),
+            ))
+        }
+    }
+
+    /// Push a new [`RoutablePipeline`] and enqueue its children to be visited
+    fn push_pipeline(
+        &mut self,
+        node: RoutablePipeline,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) {
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(node);
+        self.enqueue_children(children, node_idx)
+    }
+
+    /// Push a new [`RepartitionPipeline`] first flushing any buffered [`OperatorGroup`]
+    fn push_repartition(
+        &mut self,
+        input: Partitioning,
+        output: Partitioning,
+        parent: Option<OutputLink>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<()> {
+        let parent = match &self.execution_operators {
+            Some(buffer) => {
+                assert_eq!(buffer.output, parent, "QueryBuilder out of sync");
+                Some(OutputLink {
+                    pipeline: self.flush_exec()?,
+                    child: 0, // Must be the only child
+                })
+            }
+            None => parent,
+        };
+
+        let node = Box::new(RepartitionPipeline::try_new(input, output)?);
+        self.push_pipeline(
+            RoutablePipeline {
+                pipeline: node,
+                output: parent,
+            },
+            children,
+        );
+        Ok(())
+    }
+
+    /// Visit an [`ExecutionPlan`] operator and add it to the [`Query`] being built
+    fn visit_operator(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        if let Some(repartition) = plan.as_any().downcast_ref::<RepartitionExec>() {

Review Comment:
   Is it worth some commentary here that `RepartitionExec` and `CoalscePartitonsExec` are handled natively by the scheduler and thus not directly added into a `Pipeline`



##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and

Review Comment:
   This is very clear -- thank you for the writeup



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,381 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::{Query, QueryBuilder, RoutablePipeline};
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::QueryResults;
+
+mod pipeline;
+mod query;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// and is designed to decouple the execution parallelism from the parallelism expressed in
+/// the physical plan as partitions.
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// Queries scheduled on a [`Scheduler`] will run to completion even if the
+/// [`Scheduler`] is dropped
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<QueryResults> {
+        let query = QueryBuilder::new(plan, context).build()?;
+        Ok(self.schedule_query(query))
+    }
+
+    /// Schedule the provided [`Query`] on this [`Scheduler`].
+    pub(crate) fn schedule_query(&self, query: Query) -> QueryResults {
+        spawn_query(query, self.spawner())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {

Review Comment:
   I think i got in trouble in IOx by only handling &str and String -- what about trying to downcast to `&dyn Display` and `&dyn Debug` instead which would cover the &str and String cases as well



##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,276 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {

Review Comment:
   Editorially I find `Query` confusing in this context (as it is already used for other purposes in DataFusion and databases in general).
   
   It is confusing that we go from `sqlparser::Query --> LogicalPlan --> ExecutionPlan --> scheduler::Query`
   
   I suggest a name like `PipelineList`, `PipelinePlan` or`PushPipelines` would make this code easier to navigate for others



##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.
+///
+/// This in turn will allow for [`Pipeline`] implementations that are able to introduce
+/// parallelism beyond that expressed in their partitioning
+pub struct ExecutionPipeline {
+    proxied: Arc<dyn ExecutionPlan>,
+    inputs: Vec<Vec<Arc<Mutex<InputPartition>>>>,
+    outputs: Vec<Mutex<BoxStream<'static, ArrowResult<RecordBatch>>>>,
+}
+
+impl std::fmt::Debug for ExecutionPipeline {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let tree = debug_tree(self.proxied.as_ref());
+        f.debug_tuple("ExecutionNode").field(&tree).finish()
+    }
+}
+
+impl ExecutionPipeline {
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        depth: usize,
+    ) -> Result<Self> {
+        // The point in the plan at which to splice the plan graph
+        let mut splice_point = plan;
+        let mut parent_plans = Vec::with_capacity(depth.saturating_sub(1));
+        for _ in 0..depth {
+            let children = splice_point.children();
+            assert_eq!(
+                children.len(),
+                1,
+                "can only group through nodes with a single child"
+            );
+            parent_plans.push(splice_point);
+            splice_point = children.into_iter().next().unwrap();
+        }
+
+        // The children to replace with [`ProxyExecutionPlan`]
+        let children = splice_point.children();
+        let mut inputs = Vec::with_capacity(children.len());
+
+        // The spliced plan with its children replaced with [`ProxyExecutionPlan`]
+        let spliced = if !children.is_empty() {
+            let mut proxies: Vec<Arc<dyn ExecutionPlan>> =
+                Vec::with_capacity(children.len());
+
+            for child in children {
+                let count = child.output_partitioning().partition_count();
+
+                let mut child_inputs = Vec::with_capacity(count);
+                for _ in 0..count {
+                    child_inputs.push(Default::default())
+                }
+
+                inputs.push(child_inputs.clone());
+                proxies.push(Arc::new(ProxyExecutionPlan {
+                    inner: child,
+                    inputs: child_inputs,
+                }));
+            }
+
+            splice_point.with_new_children(proxies)?
+        } else {
+            splice_point.clone()
+        };
+
+        // Reconstruct the parent graph
+        let mut proxied = spliced;
+        for parent in parent_plans.into_iter().rev() {
+            proxied = parent.with_new_children(vec![proxied])?
+        }
+
+        // Construct the output streams
+        let output_count = proxied.output_partitioning().partition_count();
+        let outputs = (0..output_count)
+            .map(|x| {
+                let proxy_captured = proxied.clone();
+                let task_captured = task_context.clone();
+                let fut = async move {
+                    proxy_captured
+                        .execute(x, task_captured)
+                        .await
+                        .map_err(|e| ArrowError::ExternalError(Box::new(e)))
+                };
+
+                // Use futures::stream::once to handle operators that perform computation
+                // within `ExecutionPlan::execute`. If we evaluated these futures here
+                // we could potentially block indefinitely waiting for inputs that will
+                // never arrive as the query isn't scheduled yet
+                Mutex::new(futures::stream::once(fut).try_flatten().boxed())
+            })
+            .collect();
+
+        Ok(Self {
+            proxied,
+            inputs,
+            outputs,
+        })
+    }
+}
+
+impl Pipeline for ExecutionPipeline {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()> {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.buffer.push_back(input);
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+        Ok(())
+    }
+
+    fn close(&self, child: usize, partition: usize) {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.is_closed = true;
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn output_partitions(&self) -> usize {
+        self.outputs.len()
+    }
+
+    /// Poll an output partition, attempting to get its output
+    fn poll_partition(
+        &self,
+        cx: &mut Context<'_>,
+        partition: usize,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        self.outputs[partition]
+            .lock()
+            .poll_next_unpin(cx)
+            .map(|opt| opt.map(|r| r.map_err(Into::into)))
+    }
+}
+
+#[derive(Debug, Default)]
+struct InputPartition {
+    buffer: VecDeque<RecordBatch>,
+    wait_list: Vec<Waker>,

Review Comment:
   Maybe calling it `wake_list` would be better to align with the list of wakers
   
   ```suggestion
       wake_list: Vec<Waker>,
   ```



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,439 @@
+// 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.
+
+use crate::query::Query;
+use crate::{is_worker, spawn_local, spawn_local_fifo, RoutablePipeline, Spawner};
+use arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Query, spawner: Spawner) -> QueryResults {
+    debug!("Spawning query: {:#?}", query);
+
+    let (sender, receiver) = mpsc::unbounded();
+    let query = Arc::new(QueryTask {
+        spawner,
+        pipelines: query.pipelines,
+        output: sender,
+    });
+
+    for (pipeline_idx, query_pipeline) in query.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            query.spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    QueryResults {
+        query,
+        inner: receiver,
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`QueryTask`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`QueryTask`] is dropped it signals completion of query execution
+    query: Arc<QueryTask>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = &self.query.pipelines[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    fn handle_error(&self, routable: &RoutablePipeline, error: DataFusionError) {
+        self.query.send_query_output(Err(error));
+        if let Some(link) = routable.output {
+            trace!(
+                "Closing pipeline: {:?}, partition: {}, due to error",
+                link,
+                self.waker.partition,
+            );
+
+            self.query.pipelines[link.pipeline]
+                .pipeline
+                .close(link.child, self.waker.partition);
+        }
+    }
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        let wake_count = self.waker.wake_count.load(Ordering::SeqCst);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = &self.query.pipelines;
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+
+                        let r = pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition);
+
+                        if let Err(e) = r {
+                            self.handle_error(routable, e);
+
+                            // Return without rescheduling this output again
+                            return;
+                        }
+                    }
+                    None => {
+                        trace!("Publishing batch to output");
+                        self.query.send_query_output(Ok(batch))
+                    }
+                }
+
+                // Reschedule this pipeline again
+                //
+                // We want to prioritise running tasks triggered by the most recent
+                // batch, so reschedule with FIFO ordering
+                //
+                // Note: We must schedule after we have routed the batch, otherwise
+                // we introduce a potential ordering race where the newly scheduled
+                // task runs before this task finishes routing the output
+                spawn_local_fifo(self);
+            }
+            Poll::Ready(Some(Err(e))) => {
+                trace!("Poll {:?}: Error: {:?}", self, e);
+                self.handle_error(routable, e)
+            }
+            Poll::Ready(None) => {
+                trace!("Poll {:?}: None", self);
+                match routable.output {
+                    Some(link) => {
+                        trace!("Closing pipeline: {:?}, partition: {}", link, partition);
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .close(link.child, partition)
+                    }
+                    None => self.query.finish(),
+                }
+            }
+            Poll::Pending => {
+                trace!("Poll {:?}: Pending", self);
+                // Attempt to reset the wake count with the value obtained prior
+                // to calling [`Pipeline::poll_partition`].
+                //
+                // If this fails it indicates a wakeup was received whilst executing
+                // [`Pipeline::poll_partition`] and we should reschedule the task
+                let reset = self.waker.wake_count.compare_exchange(
+                    wake_count,
+                    0,
+                    Ordering::SeqCst,
+                    Ordering::SeqCst,
+                );
+
+                if reset.is_err() {
+                    trace!("Wakeup triggered whilst polling: {:?}", self);
+                    spawn_local(self);
+                }
+            }
+        }
+    }
+}
+
+/// The result stream for a query
+///
+/// # Cancellation
+///
+/// Dropping this will cancel the inflight query
+pub struct QueryResults {

Review Comment:
   ```suggestion
   pub struct ExecutionResults {
   ```
   
   Or something like
   
   ```suggestion
   pub struct PipelinePlanResults {
   ```
   
   (related to comment above)



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,439 @@
+// 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.
+
+use crate::query::Query;
+use crate::{is_worker, spawn_local, spawn_local_fifo, RoutablePipeline, Spawner};
+use arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Query, spawner: Spawner) -> QueryResults {
+    debug!("Spawning query: {:#?}", query);
+
+    let (sender, receiver) = mpsc::unbounded();
+    let query = Arc::new(QueryTask {
+        spawner,
+        pipelines: query.pipelines,
+        output: sender,
+    });
+
+    for (pipeline_idx, query_pipeline) in query.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            query.spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    QueryResults {
+        query,
+        inner: receiver,
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`QueryTask`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`QueryTask`] is dropped it signals completion of query execution
+    query: Arc<QueryTask>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = &self.query.pipelines[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    fn handle_error(&self, routable: &RoutablePipeline, error: DataFusionError) {
+        self.query.send_query_output(Err(error));
+        if let Some(link) = routable.output {
+            trace!(
+                "Closing pipeline: {:?}, partition: {}, due to error",
+                link,
+                self.waker.partition,
+            );
+
+            self.query.pipelines[link.pipeline]
+                .pipeline
+                .close(link.child, self.waker.partition);
+        }
+    }
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        let wake_count = self.waker.wake_count.load(Ordering::SeqCst);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = &self.query.pipelines;
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+
+                        let r = pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition);
+
+                        if let Err(e) = r {
+                            self.handle_error(routable, e);
+
+                            // Return without rescheduling this output again
+                            return;
+                        }
+                    }
+                    None => {
+                        trace!("Publishing batch to output");
+                        self.query.send_query_output(Ok(batch))
+                    }
+                }
+
+                // Reschedule this pipeline again
+                //
+                // We want to prioritise running tasks triggered by the most recent
+                // batch, so reschedule with FIFO ordering
+                //
+                // Note: We must schedule after we have routed the batch, otherwise
+                // we introduce a potential ordering race where the newly scheduled
+                // task runs before this task finishes routing the output
+                spawn_local_fifo(self);
+            }
+            Poll::Ready(Some(Err(e))) => {
+                trace!("Poll {:?}: Error: {:?}", self, e);
+                self.handle_error(routable, e)
+            }
+            Poll::Ready(None) => {
+                trace!("Poll {:?}: None", self);
+                match routable.output {
+                    Some(link) => {
+                        trace!("Closing pipeline: {:?}, partition: {}", link, partition);
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .close(link.child, partition)
+                    }
+                    None => self.query.finish(),
+                }
+            }
+            Poll::Pending => {
+                trace!("Poll {:?}: Pending", self);
+                // Attempt to reset the wake count with the value obtained prior
+                // to calling [`Pipeline::poll_partition`].
+                //
+                // If this fails it indicates a wakeup was received whilst executing
+                // [`Pipeline::poll_partition`] and we should reschedule the task
+                let reset = self.waker.wake_count.compare_exchange(
+                    wake_count,
+                    0,
+                    Ordering::SeqCst,
+                    Ordering::SeqCst,
+                );
+
+                if reset.is_err() {
+                    trace!("Wakeup triggered whilst polling: {:?}", self);
+                    spawn_local(self);
+                }
+            }
+        }
+    }
+}
+
+/// The result stream for a query
+///
+/// # Cancellation
+///
+/// Dropping this will cancel the inflight query
+pub struct QueryResults {
+    inner: mpsc::UnboundedReceiver<Option<Result<RecordBatch>>>,
+
+    /// Keep a reference to the [`QueryTask`] so it isn't dropped early
+    #[allow(unused)]
+    query: Arc<QueryTask>,
+}
+
+impl Stream for QueryResults {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        self.inner.poll_next_unpin(cx).map(Option::flatten)
+    }
+}
+
+/// The shared state of all [`Task`] created from the same [`Query`]
+#[derive(Debug)]
+struct QueryTask {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<Option<Result<RecordBatch>>>,
+}
+
+impl Drop for QueryTask {
+    fn drop(&mut self) {
+        debug!("Query dropped");
+    }
+}
+
+impl QueryTask {
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    fn send_query_output(&self, output: Result<RecordBatch>) {
+        let _ = self.output.unbounded_send(Some(output));
+    }
+
+    /// Mark this query as finished
+    fn finish(&self) {
+        let _ = self.output.unbounded_send(None);
+    }
+}
+
+struct TaskWaker {

Review Comment:
   Since we already use the futures `Waker` elsewhere in this PR, I wonder if we can use the same here?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856535693


##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.
+///
+/// This in turn will allow for [`Pipeline`] implementations that are able to introduce
+/// parallelism beyond that expressed in their partitioning
+pub struct ExecutionPipeline {
+    proxied: Arc<dyn ExecutionPlan>,
+    inputs: Vec<Vec<Arc<Mutex<InputPartition>>>>,
+    outputs: Vec<Mutex<BoxStream<'static, ArrowResult<RecordBatch>>>>,
+}
+
+impl std::fmt::Debug for ExecutionPipeline {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let tree = debug_tree(self.proxied.as_ref());
+        f.debug_tuple("ExecutionNode").field(&tree).finish()
+    }
+}
+
+impl ExecutionPipeline {
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        depth: usize,
+    ) -> Result<Self> {
+        // The point in the plan at which to splice the plan graph
+        let mut splice_point = plan;
+        let mut parent_plans = Vec::with_capacity(depth.saturating_sub(1));
+        for _ in 0..depth {
+            let children = splice_point.children();
+            assert_eq!(
+                children.len(),
+                1,
+                "can only group through nodes with a single child"
+            );
+            parent_plans.push(splice_point);
+            splice_point = children.into_iter().next().unwrap();
+        }
+
+        // The children to replace with [`ProxyExecutionPlan`]
+        let children = splice_point.children();
+        let mut inputs = Vec::with_capacity(children.len());
+
+        // The spliced plan with its children replaced with [`ProxyExecutionPlan`]
+        let spliced = if !children.is_empty() {
+            let mut proxies: Vec<Arc<dyn ExecutionPlan>> =
+                Vec::with_capacity(children.len());
+
+            for child in children {
+                let count = child.output_partitioning().partition_count();
+
+                let mut child_inputs = Vec::with_capacity(count);
+                for _ in 0..count {
+                    child_inputs.push(Default::default())
+                }
+
+                inputs.push(child_inputs.clone());
+                proxies.push(Arc::new(ProxyExecutionPlan {
+                    inner: child,
+                    inputs: child_inputs,
+                }));
+            }
+
+            splice_point.with_new_children(proxies)?
+        } else {
+            splice_point.clone()
+        };
+
+        // Reconstruct the parent graph
+        let mut proxied = spliced;
+        for parent in parent_plans.into_iter().rev() {
+            proxied = parent.with_new_children(vec![proxied])?
+        }
+
+        // Construct the output streams
+        let output_count = proxied.output_partitioning().partition_count();
+        let outputs = (0..output_count)
+            .map(|x| {
+                let proxy_captured = proxied.clone();
+                let task_captured = task_context.clone();
+                let fut = async move {
+                    proxy_captured
+                        .execute(x, task_captured)
+                        .await
+                        .map_err(|e| ArrowError::ExternalError(Box::new(e)))
+                };
+
+                // Use futures::stream::once to handle operators that perform computation
+                // within `ExecutionPlan::execute`. If we evaluated these futures here
+                // we could potentially block indefinitely waiting for inputs that will
+                // never arrive as the query isn't scheduled yet
+                Mutex::new(futures::stream::once(fut).try_flatten().boxed())
+            })
+            .collect();
+
+        Ok(Self {
+            proxied,
+            inputs,
+            outputs,
+        })
+    }
+}
+
+impl Pipeline for ExecutionPipeline {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()> {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.buffer.push_back(input);
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+        Ok(())
+    }
+
+    fn close(&self, child: usize, partition: usize) {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.is_closed = true;
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn output_partitions(&self) -> usize {
+        self.outputs.len()
+    }
+
+    /// Poll an output partition, attempting to get its output
+    fn poll_partition(
+        &self,
+        cx: &mut Context<'_>,
+        partition: usize,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        self.outputs[partition]
+            .lock()
+            .poll_next_unpin(cx)
+            .map(|opt| opt.map(|r| r.map_err(Into::into)))
+    }
+}
+
+#[derive(Debug, Default)]
+struct InputPartition {
+    buffer: VecDeque<RecordBatch>,
+    wait_list: Vec<Waker>,

Review Comment:
   Those two links seem to point at a list called `waiters` (rather than `wait_list`) so not quite what is in this PR either :)
   
   I don't feel strongly about this



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856535099


##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.
+///
+/// This in turn will allow for [`Pipeline`] implementations that are able to introduce
+/// parallelism beyond that expressed in their partitioning
+pub struct ExecutionPipeline {
+    proxied: Arc<dyn ExecutionPlan>,
+    inputs: Vec<Vec<Arc<Mutex<InputPartition>>>>,
+    outputs: Vec<Mutex<BoxStream<'static, ArrowResult<RecordBatch>>>>,
+}
+
+impl std::fmt::Debug for ExecutionPipeline {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let tree = debug_tree(self.proxied.as_ref());
+        f.debug_tuple("ExecutionNode").field(&tree).finish()
+    }
+}
+
+impl ExecutionPipeline {
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        depth: usize,
+    ) -> Result<Self> {
+        // The point in the plan at which to splice the plan graph
+        let mut splice_point = plan;
+        let mut parent_plans = Vec::with_capacity(depth.saturating_sub(1));
+        for _ in 0..depth {
+            let children = splice_point.children();
+            assert_eq!(
+                children.len(),
+                1,
+                "can only group through nodes with a single child"
+            );
+            parent_plans.push(splice_point);
+            splice_point = children.into_iter().next().unwrap();
+        }
+
+        // The children to replace with [`ProxyExecutionPlan`]
+        let children = splice_point.children();
+        let mut inputs = Vec::with_capacity(children.len());
+
+        // The spliced plan with its children replaced with [`ProxyExecutionPlan`]
+        let spliced = if !children.is_empty() {
+            let mut proxies: Vec<Arc<dyn ExecutionPlan>> =
+                Vec::with_capacity(children.len());
+
+            for child in children {
+                let count = child.output_partitioning().partition_count();
+
+                let mut child_inputs = Vec::with_capacity(count);
+                for _ in 0..count {
+                    child_inputs.push(Default::default())
+                }
+
+                inputs.push(child_inputs.clone());
+                proxies.push(Arc::new(ProxyExecutionPlan {
+                    inner: child,
+                    inputs: child_inputs,
+                }));
+            }
+
+            splice_point.with_new_children(proxies)?
+        } else {
+            splice_point.clone()
+        };
+
+        // Reconstruct the parent graph
+        let mut proxied = spliced;
+        for parent in parent_plans.into_iter().rev() {
+            proxied = parent.with_new_children(vec![proxied])?
+        }
+
+        // Construct the output streams
+        let output_count = proxied.output_partitioning().partition_count();
+        let outputs = (0..output_count)
+            .map(|x| {
+                let proxy_captured = proxied.clone();
+                let task_captured = task_context.clone();
+                let fut = async move {
+                    proxy_captured
+                        .execute(x, task_captured)
+                        .await
+                        .map_err(|e| ArrowError::ExternalError(Box::new(e)))
+                };
+
+                // Use futures::stream::once to handle operators that perform computation
+                // within `ExecutionPlan::execute`. If we evaluated these futures here
+                // we could potentially block indefinitely waiting for inputs that will
+                // never arrive as the query isn't scheduled yet
+                Mutex::new(futures::stream::once(fut).try_flatten().boxed())
+            })
+            .collect();
+
+        Ok(Self {
+            proxied,
+            inputs,
+            outputs,
+        })
+    }
+}
+
+impl Pipeline for ExecutionPipeline {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()> {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.buffer.push_back(input);
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+        Ok(())
+    }
+
+    fn close(&self, child: usize, partition: usize) {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.is_closed = true;
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn output_partitions(&self) -> usize {
+        self.outputs.len()
+    }
+
+    /// Poll an output partition, attempting to get its output
+    fn poll_partition(
+        &self,
+        cx: &mut Context<'_>,
+        partition: usize,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        self.outputs[partition]
+            .lock()
+            .poll_next_unpin(cx)
+            .map(|opt| opt.map(|r| r.map_err(Into::into)))
+    }
+}
+
+#[derive(Debug, Default)]
+struct InputPartition {
+    buffer: VecDeque<RecordBatch>,
+    wait_list: Vec<Waker>,

Review Comment:
   fair enough



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] thinkharderdev commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1107665288

   I think there is an issue with filtering parquet sources (for some reason it is specific to parquet). PR'd a failing test case https://github.com/tustvold/arrow-datafusion/pull/38.
   
   Projection seems to work, but for some reason `FilterExec` results in an empty result. 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1116024213

   I think this is now ready, I'll leave it up for a few more days in case anyone else wants to take a look. Perhaps @yjshen or @houqp 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1098373737

   Can you please provide benchmark results as well? I am pretty sure you have them so I figure this is a light ask :)


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1117364244

   Thanks @tustvold  and @yjshen  -- I agree this is a great step forward. I expect we'll keep iterating but this is a good foundation I think


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850814077


##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,72 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+pub trait Pipeline: Send + Sync + std::fmt::Debug {
+    /// Push a [`RecordBatch`] to the given input partition

Review Comment:
   Either, currently RepartitionPipeline is push-based, whereas ExecutionPipeline has to be pull-based. This inconsistency may eventually go away though.
   
   Edit: I've added some further doc comments to the trait, PTAL



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850819477


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,
+}
+
+impl Drop for Query {
+    fn drop(&mut self) {
+        debug!("Query finished");
+    }
+}
+
+impl Query {
+    /// Creates a new [`Query`] from the provided [`ExecutionPlan`], returning
+    /// an [`mpsc::UnboundedReceiver`] that can be used to receive the results
+    /// of this query's execution
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        spawner: Spawner,
+    ) -> Result<(Query, mpsc::UnboundedReceiver<ArrowResult<RecordBatch>>)> {
+        QueryBuilder::new(plan, task_context).build(spawner)
+    }
+
+    /// Returns a list of this queries [`QueryPipeline`]
+    pub fn pipelines(&self) -> &[RoutablePipeline] {
+        &self.pipelines
+    }
+
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    pub fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    pub fn send_query_output(&self, output: ArrowResult<RecordBatch>) {
+        let _ = self.output.unbounded_send(output);
+    }
+
+    /// Returns the [`Spawner`] associated with this [`Query`]
+    pub fn spawner(&self) -> &Spawner {
+        &self.spawner
+    }
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple [`ExecutionPlan`], [`ExecGroup`] stores this state
+struct ExecGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`Query`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`Query`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+struct QueryBuilder {
+    task_context: Arc<TaskContext>,
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,
+
+    /// A list of [`ExecutionPlan`] still to visit, along with
+    /// where they should route their output
+    to_visit: Vec<(Arc<dyn ExecutionPlan>, Option<OutputLink>)>,
+
+    /// Stores one or more [`ExecutionPlan`] to combine together into
+    /// a single [`ExecutionPipeline`]
+    exec_buffer: Option<ExecGroup>,
+}
+
+impl QueryBuilder {
+    fn new(plan: Arc<dyn ExecutionPlan>, task_context: Arc<TaskContext>) -> Self {
+        Self {
+            in_progress: vec![],
+            to_visit: vec![(plan, None)],
+            task_context,
+            exec_buffer: None,
+        }
+    }
+
+    /// Flush the current group of [`ExecutionPlan`] stored in `exec_buffer`
+    /// into a single [`ExecutionPipeline]
+    fn flush_exec(&mut self) -> Result<usize> {
+        let group = self.exec_buffer.take().unwrap();
+        let node_idx = self.in_progress.len();
+        self.in_progress.push(RoutablePipeline {
+            pipeline: Box::new(ExecutionPipeline::new(
+                group.root,
+                self.task_context.clone(),
+                group.depth,
+            )?),
+            output: group.output,
+        });
+        Ok(node_idx)
+    }
+
+    /// Visit a non-special cased [`ExecutionPlan`]
+    fn visit_exec(
+        &mut self,
+        plan: Arc<dyn ExecutionPlan>,
+        parent: Option<OutputLink>,
+    ) -> Result<()> {
+        let children = plan.children();
+
+        // Add the node to the current group of execution plan to be combined
+        // into a single [`ExecutionPipeline`].
+        //
+        // TODO: More sophisticated policy, just because we can combine them doesn't mean we should

Review Comment:
   I don't have a good grasp on them yet, this just replicates the grouping of the current parallelism story. This is a note to investigate if this is actually a good thing 😃



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850811049


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering
+fn spawn_local_fifo(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn_fifo(|| task.do_work())
+}
+
+#[derive(Debug, Clone)]
+pub struct Spawner {

Review Comment:
   Purely indirection, it's a vestigial remnant of how shutdown used to work. Once I have that story fleshed out will revisit if this is still needed



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1105338672

   I'm marking this ready for review, I think there are definitely things that could be improved, but this PR is already quite large and I would quite like to get some feedback on it. Ideally I would like to get something in, so that we can start to iterate on this, and gather feedback from real-world workloads


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856456282


##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,439 @@
+// 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.
+
+use crate::query::Query;
+use crate::{is_worker, spawn_local, spawn_local_fifo, RoutablePipeline, Spawner};
+use arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a query using the provided [`Spawner`]
+pub fn spawn_query(query: Query, spawner: Spawner) -> QueryResults {
+    debug!("Spawning query: {:#?}", query);
+
+    let (sender, receiver) = mpsc::unbounded();
+    let query = Arc::new(QueryTask {
+        spawner,
+        pipelines: query.pipelines,
+        output: sender,
+    });
+
+    for (pipeline_idx, query_pipeline) in query.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            query.spawner.spawn(Task {
+                query: query.clone(),
+                waker: Arc::new(TaskWaker {
+                    query: Arc::downgrade(&query),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    QueryResults {
+        query,
+        inner: receiver,
+    }
+}
+
+/// A [`Task`] identifies an output partition within a given pipeline that may be able to
+/// make progress. The [`Scheduler`][super::Scheduler] maintains a list of outstanding
+/// [`Task`] and distributes them amongst its worker threads.
+///
+/// A [`Query`] is considered completed when it has no outstanding [`Task`]
+pub struct Task {
+    /// Maintain a link to the [`QueryTask`] this is necessary to be able to
+    /// route the output of the partition to its destination, and also because
+    /// when [`QueryTask`] is dropped it signals completion of query execution
+    query: Arc<QueryTask>,
+
+    /// A [`ArcWake`] that can be used to re-schedule this [`Task`] for execution
+    waker: Arc<TaskWaker>,
+}
+
+impl std::fmt::Debug for Task {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let output = &self.query.pipelines[self.waker.pipeline].output;
+
+        f.debug_struct("Task")
+            .field("pipeline", &self.waker.pipeline)
+            .field("partition", &self.waker.partition)
+            .field("output", &output)
+            .finish()
+    }
+}
+
+impl Task {
+    fn handle_error(&self, routable: &RoutablePipeline, error: DataFusionError) {
+        self.query.send_query_output(Err(error));
+        if let Some(link) = routable.output {
+            trace!(
+                "Closing pipeline: {:?}, partition: {}, due to error",
+                link,
+                self.waker.partition,
+            );
+
+            self.query.pipelines[link.pipeline]
+                .pipeline
+                .close(link.child, self.waker.partition);
+        }
+    }
+    /// Call [`Pipeline::poll_partition`] attempting to make progress on query execution
+    pub fn do_work(self) {
+        assert!(is_worker(), "Task::do_work called outside of worker pool");
+        if self.query.is_cancelled() {
+            return;
+        }
+
+        // Capture the wake count prior to calling [`Pipeline::poll_partition`]
+        // this allows us to detect concurrent wake ups and handle them correctly
+        let wake_count = self.waker.wake_count.load(Ordering::SeqCst);
+
+        let node = self.waker.pipeline;
+        let partition = self.waker.partition;
+
+        let waker = futures::task::waker_ref(&self.waker);
+        let mut cx = Context::from_waker(&*waker);
+
+        let pipelines = &self.query.pipelines;
+        let routable = &pipelines[node];
+        match routable.pipeline.poll_partition(&mut cx, partition) {
+            Poll::Ready(Some(Ok(batch))) => {
+                trace!("Poll {:?}: Ok: {}", self, batch.num_rows());
+                match routable.output {
+                    Some(link) => {
+                        trace!(
+                            "Publishing batch to pipeline {:?} partition {}",
+                            link,
+                            partition
+                        );
+
+                        let r = pipelines[link.pipeline]
+                            .pipeline
+                            .push(batch, link.child, partition);
+
+                        if let Err(e) = r {
+                            self.handle_error(routable, e);
+
+                            // Return without rescheduling this output again
+                            return;
+                        }
+                    }
+                    None => {
+                        trace!("Publishing batch to output");
+                        self.query.send_query_output(Ok(batch))
+                    }
+                }
+
+                // Reschedule this pipeline again
+                //
+                // We want to prioritise running tasks triggered by the most recent
+                // batch, so reschedule with FIFO ordering
+                //
+                // Note: We must schedule after we have routed the batch, otherwise
+                // we introduce a potential ordering race where the newly scheduled
+                // task runs before this task finishes routing the output
+                spawn_local_fifo(self);
+            }
+            Poll::Ready(Some(Err(e))) => {
+                trace!("Poll {:?}: Error: {:?}", self, e);
+                self.handle_error(routable, e)
+            }
+            Poll::Ready(None) => {
+                trace!("Poll {:?}: None", self);
+                match routable.output {
+                    Some(link) => {
+                        trace!("Closing pipeline: {:?}, partition: {}", link, partition);
+                        pipelines[link.pipeline]
+                            .pipeline
+                            .close(link.child, partition)
+                    }
+                    None => self.query.finish(),
+                }
+            }
+            Poll::Pending => {
+                trace!("Poll {:?}: Pending", self);
+                // Attempt to reset the wake count with the value obtained prior
+                // to calling [`Pipeline::poll_partition`].
+                //
+                // If this fails it indicates a wakeup was received whilst executing
+                // [`Pipeline::poll_partition`] and we should reschedule the task
+                let reset = self.waker.wake_count.compare_exchange(
+                    wake_count,
+                    0,
+                    Ordering::SeqCst,
+                    Ordering::SeqCst,
+                );
+
+                if reset.is_err() {
+                    trace!("Wakeup triggered whilst polling: {:?}", self);
+                    spawn_local(self);
+                }
+            }
+        }
+    }
+}
+
+/// The result stream for a query
+///
+/// # Cancellation
+///
+/// Dropping this will cancel the inflight query
+pub struct QueryResults {
+    inner: mpsc::UnboundedReceiver<Option<Result<RecordBatch>>>,
+
+    /// Keep a reference to the [`QueryTask`] so it isn't dropped early
+    #[allow(unused)]
+    query: Arc<QueryTask>,
+}
+
+impl Stream for QueryResults {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        self.inner.poll_next_unpin(cx).map(Option::flatten)
+    }
+}
+
+/// The shared state of all [`Task`] created from the same [`Query`]
+#[derive(Debug)]
+struct QueryTask {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<Option<Result<RecordBatch>>>,
+}
+
+impl Drop for QueryTask {
+    fn drop(&mut self) {
+        debug!("Query dropped");
+    }
+}
+
+impl QueryTask {
+    /// Returns `true` if this query has been dropped, specifically if the
+    /// stream returned by [`super::Scheduler::schedule`] has been dropped
+    fn is_cancelled(&self) -> bool {
+        self.output.is_closed()
+    }
+
+    /// Sends `output` to this query's output stream
+    fn send_query_output(&self, output: Result<RecordBatch>) {
+        let _ = self.output.unbounded_send(Some(output));
+    }
+
+    /// Mark this query as finished
+    fn finish(&self) {
+        let _ = self.output.unbounded_send(None);
+    }
+}
+
+struct TaskWaker {

Review Comment:
   lets not do that lol



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] xudong963 commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
xudong963 commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856941791


##########
datafusion/scheduler/src/plan.rs:
##########
@@ -0,0 +1,276 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+
+/// Identifies the [`Pipeline`] within the [`PipelinePlan`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`PipelinePlan`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`PipelinePlan`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct PipelinePlan {
+    pub pipelines: Vec<RoutablePipeline>,
+}
+
+/// When converting [`ExecutionPlan`] to [`Pipeline`] we may wish to group
+/// together multiple operators, [`OperatorGroup`] stores this state
+struct OperatorGroup {
+    /// Where to route the output of the eventual [`Pipeline`]
+    output: Option<OutputLink>,
+
+    /// The [`ExecutionPlan`] from which to start recursing
+    root: Arc<dyn ExecutionPlan>,
+
+    /// The number of times to recurse into the [`ExecutionPlan`]'s children
+    depth: usize,
+}
+
+/// A utility struct to assist converting from [`ExecutionPlan`] to [`PipelinePlan`]
+///
+/// The [`ExecutionPlan`] is visited in a depth-first fashion, gradually building
+/// up the [`RoutablePipeline`] for the [`PipelinePlan`]. As nodes are visited depth-first,
+/// a node is visited only after its parent has been.
+pub struct PipelinePlanner {
+    task_context: Arc<TaskContext>,
+
+    /// The current list of completed pipelines
+    in_progress: Vec<RoutablePipeline>,

Review Comment:
   `in_progress` with the annotation `completed` seems a little confusing 🤔



##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,411 @@
+// 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.
+
+//! A [`Scheduler`] maintains a pool of dedicated worker threads on which
+//! query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+//! and is designed to decouple the execution parallelism from the parallelism expressed in
+//! the physical plan as partitions.
+//!
+//! # Implementation
+//!
+//! When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+//! chunks called pipelines. Each pipeline may consist of one or more nodes from the
+//! [`ExecutionPlan`] tree.
+//!
+//! The scheduler then maintains a list of pending [`Task`], that identify a partition within
+//! a particular pipeline that may be able to make progress on some "morsel" of data. These
+//! [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+//! on a given "morsel" on the same thread that produced it.
+//!
+//! # Rayon
+//!
+//! Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+//! scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+//! structured concurrency primitives to express additional parallelism that may be exploited
+//! if there are idle threads available at runtime
+//!
+//! # Shutdown
+//!
+//! Queries scheduled on a [`Scheduler`] will run to completion even if the
+//! [`Scheduler`] is dropped
+//!
+//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+//! [rayon]: https://docs.rs/rayon/latest/rayon/
+//!
+//! # Example
+//!
+//! ```rust
+//! # use futures::TryStreamExt;
+//! # use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
+//! # use datafusion_scheduler::Scheduler;
+//!
+//! # #[tokio::main]
+//! # async fn main() {
+//! let scheduler = Scheduler::new(4);
+//! let config = SessionConfig::new().with_target_partitions(4);
+//! let context = SessionContext::with_config(config);
+//!
+//! context.register_csv("example", "../core/tests/example.csv", CsvReadOptions::new()).await.unwrap();
+//! let plan = context.sql("SELECT MIN(b) FROM example")
+//!     .await
+//!    .unwrap()
+//!    .create_physical_plan()
+//!    .await
+//!    .unwrap();
+//!
+//! let task = context.task_ctx();
+//! let stream = scheduler.schedule(plan, task).unwrap();
+//! let scheduled: Vec<_> = stream.try_collect().await.unwrap();
+//! # }
+//! ```
+//!
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::plan::{PipelinePlan, PipelinePlanner, RoutablePipeline};
+use crate::task::{spawn_plan, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::ExecutionResults;
+
+mod pipeline;
+mod plan;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]

Review Comment:
   If the method only used by tests, how about repace `#[allow(dead_code)]` with `#[cfg(test)]`?



##########
datafusion/scheduler/src/task.rs:
##########
@@ -0,0 +1,437 @@
+// 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.
+
+use crate::plan::PipelinePlan;
+use crate::{is_worker, spawn_local, spawn_local_fifo, RoutablePipeline, Spawner};
+use arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use futures::channel::mpsc;
+use futures::task::ArcWake;
+use futures::{Stream, StreamExt};
+use log::{debug, trace};
+use std::pin::Pin;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::{Arc, Weak};
+use std::task::{Context, Poll};
+
+/// Spawns a [`PipelinePlan`] using the provided [`Spawner`]
+pub fn spawn_plan(plan: PipelinePlan, spawner: Spawner) -> ExecutionResults {
+    debug!("Spawning pipeline plan: {:#?}", plan);
+
+    let (sender, receiver) = mpsc::unbounded();
+    let context = Arc::new(ExecutionContext {
+        spawner,
+        pipelines: plan.pipelines,
+        output: sender,
+    });
+
+    for (pipeline_idx, query_pipeline) in context.pipelines.iter().enumerate() {
+        for partition in 0..query_pipeline.pipeline.output_partitions() {
+            context.spawner.spawn(Task {
+                context: context.clone(),
+                waker: Arc::new(TaskWaker {
+                    context: Arc::downgrade(&context),
+                    wake_count: AtomicUsize::new(1),
+                    pipeline: pipeline_idx,
+                    partition,
+                }),
+            });
+        }
+    }
+
+    ExecutionResults {
+        context: context,

Review Comment:
   ```suggestion
           context,
   ```



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856480823


##########
datafusion/scheduler/src/pipeline/execution.rs:
##########
@@ -0,0 +1,330 @@
+// 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.
+
+use std::any::Any;
+use std::collections::VecDeque;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::error::ArrowError;
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+use parking_lot::Mutex;
+
+use datafusion::arrow::datatypes::SchemaRef;
+use datafusion::arrow::{error::Result as ArrowResult, record_batch::RecordBatch};
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::expressions::PhysicalSortExpr;
+use datafusion::physical_plan::metrics::MetricsSet;
+use datafusion::physical_plan::{
+    displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+use crate::pipeline::Pipeline;
+use crate::BoxStream;
+
+/// An [`ExecutionPipeline`] wraps a portion of an [`ExecutionPlan`] and
+/// converts it to the push-based [`Pipeline`] interface
+///
+/// Internally [`ExecutionPipeline`] is still pull-based which limits its parallelism
+/// to that of its output partitioning, however, it provides full compatibility with
+/// [`ExecutionPlan`] allowing full interoperability with the existing ecosystem
+///
+/// Longer term we will likely want to introduce new traits that differentiate between
+/// pipeline-able operators like filters, and pipeline-breakers like aggregations, and
+/// are better aligned with a push-based execution model.
+///
+/// This in turn will allow for [`Pipeline`] implementations that are able to introduce
+/// parallelism beyond that expressed in their partitioning
+pub struct ExecutionPipeline {
+    proxied: Arc<dyn ExecutionPlan>,
+    inputs: Vec<Vec<Arc<Mutex<InputPartition>>>>,
+    outputs: Vec<Mutex<BoxStream<'static, ArrowResult<RecordBatch>>>>,
+}
+
+impl std::fmt::Debug for ExecutionPipeline {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        let tree = debug_tree(self.proxied.as_ref());
+        f.debug_tuple("ExecutionNode").field(&tree).finish()
+    }
+}
+
+impl ExecutionPipeline {
+    pub fn new(
+        plan: Arc<dyn ExecutionPlan>,
+        task_context: Arc<TaskContext>,
+        depth: usize,
+    ) -> Result<Self> {
+        // The point in the plan at which to splice the plan graph
+        let mut splice_point = plan;
+        let mut parent_plans = Vec::with_capacity(depth.saturating_sub(1));
+        for _ in 0..depth {
+            let children = splice_point.children();
+            assert_eq!(
+                children.len(),
+                1,
+                "can only group through nodes with a single child"
+            );
+            parent_plans.push(splice_point);
+            splice_point = children.into_iter().next().unwrap();
+        }
+
+        // The children to replace with [`ProxyExecutionPlan`]
+        let children = splice_point.children();
+        let mut inputs = Vec::with_capacity(children.len());
+
+        // The spliced plan with its children replaced with [`ProxyExecutionPlan`]
+        let spliced = if !children.is_empty() {
+            let mut proxies: Vec<Arc<dyn ExecutionPlan>> =
+                Vec::with_capacity(children.len());
+
+            for child in children {
+                let count = child.output_partitioning().partition_count();
+
+                let mut child_inputs = Vec::with_capacity(count);
+                for _ in 0..count {
+                    child_inputs.push(Default::default())
+                }
+
+                inputs.push(child_inputs.clone());
+                proxies.push(Arc::new(ProxyExecutionPlan {
+                    inner: child,
+                    inputs: child_inputs,
+                }));
+            }
+
+            splice_point.with_new_children(proxies)?
+        } else {
+            splice_point.clone()
+        };
+
+        // Reconstruct the parent graph
+        let mut proxied = spliced;
+        for parent in parent_plans.into_iter().rev() {
+            proxied = parent.with_new_children(vec![proxied])?
+        }
+
+        // Construct the output streams
+        let output_count = proxied.output_partitioning().partition_count();
+        let outputs = (0..output_count)
+            .map(|x| {
+                let proxy_captured = proxied.clone();
+                let task_captured = task_context.clone();
+                let fut = async move {
+                    proxy_captured
+                        .execute(x, task_captured)
+                        .await
+                        .map_err(|e| ArrowError::ExternalError(Box::new(e)))
+                };
+
+                // Use futures::stream::once to handle operators that perform computation
+                // within `ExecutionPlan::execute`. If we evaluated these futures here
+                // we could potentially block indefinitely waiting for inputs that will
+                // never arrive as the query isn't scheduled yet
+                Mutex::new(futures::stream::once(fut).try_flatten().boxed())
+            })
+            .collect();
+
+        Ok(Self {
+            proxied,
+            inputs,
+            outputs,
+        })
+    }
+}
+
+impl Pipeline for ExecutionPipeline {
+    /// Push a [`RecordBatch`] to the given input partition
+    fn push(&self, input: RecordBatch, child: usize, partition: usize) -> Result<()> {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.buffer.push_back(input);
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+        Ok(())
+    }
+
+    fn close(&self, child: usize, partition: usize) {
+        let mut partition = self.inputs[child][partition].lock();
+        assert!(!partition.is_closed);
+
+        partition.is_closed = true;
+        for waker in partition.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn output_partitions(&self) -> usize {
+        self.outputs.len()
+    }
+
+    /// Poll an output partition, attempting to get its output
+    fn poll_partition(
+        &self,
+        cx: &mut Context<'_>,
+        partition: usize,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        self.outputs[partition]
+            .lock()
+            .poll_next_unpin(cx)
+            .map(|opt| opt.map(|r| r.map_err(Into::into)))
+    }
+}
+
+#[derive(Debug, Default)]
+struct InputPartition {
+    buffer: VecDeque<RecordBatch>,
+    wait_list: Vec<Waker>,

Review Comment:
   This is the "standard" nomenclature for a list of waiting tasks https://docs.rs/tokio/latest/src/tokio/sync/notify.rs.html#121 https://docs.rs/futures-util/0.3.21/src/futures_util/lock/mutex.rs.html#22
   
   I guess the idea is that the Waker is the representation of a waiting task, maybe??



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#issuecomment-1098953635

   Unfortunately I've discovered a bug with this implementation this morning, concerning concurrent wakeups. This could result in the same Task being scheduled concurrently, potentially leading to output reordering. I need to think a bit more on how to handle this. I have updated the description accordingly


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850810063


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool
+fn spawn_local(task: Task) {
+    // Verify is a worker thread to avoid creating a global pool
+    assert!(is_worker(), "must be called from a worker");
+    rayon::spawn(|| task.do_work())
+}
+
+/// Spawn a [`Task`] onto the local workers thread pool with fifo ordering

Review Comment:
   Best effort, will add link to rayon doc



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850814077


##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,72 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+pub trait Pipeline: Send + Sync + std::fmt::Debug {
+    /// Push a [`RecordBatch`] to the given input partition

Review Comment:
   Either, currently RepartitionPipeline is push-based, whereas ExecutionPipeline has to be pull-based. This inconsistency may eventually go away though



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] Dandandan commented on a diff in pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850605009


##########
datafusion/scheduler/src/pipeline/repartition.rs:
##########
@@ -0,0 +1,222 @@
+// 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.
+
+use std::collections::VecDeque;
+use std::sync::Arc;
+use std::task::{Context, Poll, Waker};
+
+use arrow::array::ArrayRef;
+use parking_lot::Mutex;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::{DataFusionError, Result};
+use datafusion::physical_plan::{Partitioning, PhysicalExpr};
+
+use crate::pipeline::Pipeline;
+use crate::ArrowResult;
+
+/// A [`Pipeline`] that can repartition its input
+#[derive(Debug)]
+pub struct RepartitionPipeline {
+    output: Partitioning,
+    state: Mutex<RepartitionState>,
+}
+
+impl RepartitionPipeline {
+    /// Create a new [`RepartitionPipeline`] with the given `input` and `output` partitioning
+    pub fn new(input: Partitioning, output: Partitioning) -> Self {
+        let input_count = input.partition_count();
+        assert_ne!(input_count, 0);
+
+        let num_partitions = match output {
+            Partitioning::RoundRobinBatch(num_partitions) => num_partitions,
+            Partitioning::Hash(_, num_partitions) => num_partitions,
+            Partitioning::UnknownPartitioning(_) => unreachable!(),
+        };
+        assert_ne!(num_partitions, 0);
+
+        let state = Mutex::new(RepartitionState {
+            next_idx: 0,
+            hash_buffer: vec![],
+            partition_closed: vec![false; input_count],
+            input_closed: false,
+            output_buffers: (0..num_partitions).map(|_| Default::default()).collect(),
+        });
+
+        Self { output, state }
+    }
+}
+
+struct RepartitionState {
+    next_idx: usize,
+    hash_buffer: Vec<u64>,
+    partition_closed: Vec<bool>,
+    input_closed: bool,
+    output_buffers: Vec<OutputBuffer>,
+}
+
+impl std::fmt::Debug for RepartitionState {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        f.debug_struct("RepartitionState")
+            .field("next_idx", &self.next_idx)
+            .field("partition_closed", &self.partition_closed)
+            .field("input_closed", &self.input_closed)
+            .finish()
+    }
+}
+
+impl RepartitionState {
+    fn push_batch(&mut self, partition: usize, batch: RecordBatch) {
+        let buffer = &mut self.output_buffers[partition];
+
+        buffer.batches.push_back(batch);
+
+        for waker in buffer.wait_list.drain(..) {
+            waker.wake()
+        }
+    }
+
+    fn hash_batch(
+        &mut self,
+        exprs: &[Arc<dyn PhysicalExpr>],
+        input: RecordBatch,
+    ) -> Result<()> {
+        let arrays = exprs

Review Comment:
   By now we have three (similar) copies of this code (at least datafusion / ballista and now the scheduler that I know of).
   I think we should extract this at some time to some common place (datafusion or even arrow-rs).



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Morsel-driven Parallelism using rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r849777965


##########
datafusion/scheduler/benches/parquet_query_sql.rs:
##########
@@ -220,17 +230,42 @@ fn criterion_benchmark(c: &mut Criterion) {
             continue;
         }
 
-        let query = query.as_str();
-        c.bench_function(query, |b| {
+        c.bench_function(&format!("tokio: {}", query), |b| {
             b.iter(|| {
-                let context = context.clone();
-                rt.block_on(async move {
-                    let query = context.sql(query).await.unwrap();
+                let query = query.clone();
+                let mut context = context.clone();
+                let (sender, mut receiver) = futures::channel::mpsc::unbounded();
+
+                // Spawn work to a separate tokio thread pool

Review Comment:
   This makes the performance comparison more fair, as the dispatch to another thread pool is necessary in both approaches, as you don't want to run long-running query workloads on the same tokio pool as say network IO, and can have a non-negligible impact on the simpler queries.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r850814077


##########
datafusion/scheduler/src/pipeline/mod.rs:
##########
@@ -0,0 +1,72 @@
+// 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.
+
+use std::task::{Context, Poll};
+
+use arrow::record_batch::RecordBatch;
+
+use crate::ArrowResult;
+
+pub mod execution;
+pub mod repartition;
+
+/// A push-based interface used by the scheduler to drive query execution
+///
+/// A pipeline processes data from one or more input partitions, producing output
+/// to one or more output partitions. As a [`Pipeline`] may drawn on input from
+/// more than one upstream [`Pipeline`], input partitions are identified by both
+/// a child index, and a partition index, whereas output partitions are only
+/// identified by a partition index.
+///
+pub trait Pipeline: Send + Sync + std::fmt::Debug {
+    /// Push a [`RecordBatch`] to the given input partition

Review Comment:
   Either, currently RepartitionPipeline is push-based, whereas ExecutionPipeline has to be pull-based. This inconsistency may eventually go away though.
   
   I've added some further doc comments to the trait



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851343138


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,275 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::{BoxStream, StreamExt};
+use log::debug;
+
+use datafusion::arrow::error::Result as ArrowResult;
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::Query;
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+mod pipeline;
+mod query;
+mod task;
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// which decouples execution parallelism from the parallelism expressed in the physical plan
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// TBC
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` threads in its thread pool
+    pub fn new(num_threads: usize) -> Self {
+        let pool = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .thread_name(|idx| format!("df-worker-{}", idx))
+            .build()
+            .unwrap();
+
+        Self {
+            pool: Arc::new(pool),
+        }
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<BoxStream<'static, ArrowResult<RecordBatch>>> {
+        let (query, receiver) = Query::new(plan, context, self.spawner())?;
+        spawn_query(Arc::new(query));
+        Ok(receiver.boxed())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Returns `true` if the current thread is a worker thread
+fn is_worker() -> bool {
+    rayon::current_thread_index().is_some()

Review Comment:
   maybe a comment to that effect? 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r851364087


##########
datafusion/scheduler/src/query.rs:
##########
@@ -0,0 +1,337 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::channel::mpsc;
+use log::debug;
+
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
+use datafusion::physical_plan::repartition::RepartitionExec;
+use datafusion::physical_plan::{ExecutionPlan, Partitioning};
+
+use crate::pipeline::{
+    execution::ExecutionPipeline, repartition::RepartitionPipeline, Pipeline,
+};
+use crate::{ArrowResult, Spawner};
+
+/// Identifies the [`Pipeline`] within the [`Query`] to route output to
+#[derive(Debug, Clone, Copy, PartialEq)]
+pub struct OutputLink {
+    /// The index of the [`Pipeline`] in [`Query`] to route output to
+    pub pipeline: usize,
+
+    /// The child of the [`Pipeline`] to route output to
+    pub child: usize,
+}
+
+/// Combines a [`Pipeline`] with an [`OutputLink`] identifying where to send its output
+#[derive(Debug)]
+pub struct RoutablePipeline {
+    /// The pipeline that produces data
+    pub pipeline: Box<dyn Pipeline>,
+
+    /// Where to send output the output of `pipeline`
+    ///
+    /// If `None`, the output should be sent to the query output
+    pub output: Option<OutputLink>,
+}
+
+/// [`Query`] is the scheduler's representation of the [`ExecutionPlan`] passed to
+/// [`super::Scheduler::schedule`]. It combines the list of [Pipeline`] with the information
+/// necessary to route output from one stage to the next
+#[derive(Debug)]
+pub struct Query {
+    /// Spawner for this query
+    spawner: Spawner,
+
+    /// List of pipelines that belong to this query, pipelines are addressed
+    /// based on their index within this list
+    pipelines: Vec<RoutablePipeline>,
+
+    /// The output stream for this query's execution
+    output: mpsc::UnboundedSender<ArrowResult<RecordBatch>>,

Review Comment:
   That is my interpretation of https://github.com/apache/arrow-datafusion/blob/master/datafusion/core/src/physical_plan/repartition.rs#L192
   
   FWIW the scheduler may be less likely to run into this than the current-DF, because of the best-effort LIFO task ordering, but we probably should have some back-pressure mechanism eventually



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb merged pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
alamb merged PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856290226


##########
datafusion/scheduler/src/lib.rs:
##########
@@ -0,0 +1,381 @@
+// 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.
+
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use log::{debug, error};
+
+use datafusion::error::Result;
+use datafusion::execution::context::TaskContext;
+use datafusion::physical_plan::ExecutionPlan;
+
+use crate::query::{Query, QueryBuilder, RoutablePipeline};
+use crate::task::{spawn_query, Task};
+
+use rayon::{ThreadPool, ThreadPoolBuilder};
+
+pub use task::QueryResults;
+
+mod pipeline;
+mod query;
+mod task;
+
+/// Builder for a [`Scheduler`]
+#[derive(Debug)]
+pub struct SchedulerBuilder {
+    inner: ThreadPoolBuilder,
+}
+
+impl SchedulerBuilder {
+    /// Create a new [`SchedulerConfig`] with the provided number of threads
+    pub fn new(num_threads: usize) -> Self {
+        let builder = ThreadPoolBuilder::new()
+            .num_threads(num_threads)
+            .panic_handler(|p| error!("{}", format_worker_panic(p)))
+            .thread_name(|idx| format!("df-worker-{}", idx));
+
+        Self { inner: builder }
+    }
+
+    /// Registers a custom panic handler
+    ///
+    /// Used by tests
+    #[allow(dead_code)]
+    fn panic_handler<H>(self, panic_handler: H) -> Self
+    where
+        H: Fn(Box<dyn std::any::Any + Send>) + Send + Sync + 'static,
+    {
+        Self {
+            inner: self.inner.panic_handler(panic_handler),
+        }
+    }
+
+    /// Build a new [`Scheduler`]
+    fn build(self) -> Scheduler {
+        Scheduler {
+            pool: Arc::new(self.inner.build().unwrap()),
+        }
+    }
+}
+
+/// A [`Scheduler`] maintains a pool of dedicated worker threads on which
+/// query execution can be scheduled. This is based on the idea of [Morsel-Driven Parallelism]
+/// and is designed to decouple the execution parallelism from the parallelism expressed in
+/// the physical plan as partitions.
+///
+/// # Implementation
+///
+/// When provided with an [`ExecutionPlan`] the [`Scheduler`] first breaks it up into smaller
+/// chunks called pipelines. Each pipeline may consist of one or more nodes from the
+/// [`ExecutionPlan`] tree.
+///
+/// The scheduler then maintains a list of pending [`Task`], that identify a partition within
+/// a particular pipeline that may be able to make progress on some "morsel" of data. These
+/// [`Task`] are then scheduled on the worker pool, with a preference for scheduling work
+/// on a given "morsel" on the same thread that produced it.
+///
+/// # Rayon
+///
+/// Under-the-hood these [`Task`] are scheduled by [rayon], which is a lightweight, work-stealing
+/// scheduler optimised for CPU-bound workloads. Pipelines may exploit this fact, and use [rayon]'s
+/// structured concurrency primitives to express additional parallelism that may be exploited
+/// if there are idle threads available at runtime
+///
+/// # Shutdown
+///
+/// Queries scheduled on a [`Scheduler`] will run to completion even if the
+/// [`Scheduler`] is dropped
+///
+/// [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
+/// [rayon]: https://docs.rs/rayon/latest/rayon/
+///
+pub struct Scheduler {
+    pool: Arc<ThreadPool>,
+}
+
+impl Scheduler {
+    /// Create a new [`Scheduler`] with `num_threads` new threads in a dedicated thread pool
+    pub fn new(num_threads: usize) -> Self {
+        SchedulerBuilder::new(num_threads).build()
+    }
+
+    /// Schedule the provided [`ExecutionPlan`] on this [`Scheduler`].
+    ///
+    /// Returns a [`BoxStream`] that can be used to receive results as they are produced
+    pub fn schedule(
+        &self,
+        plan: Arc<dyn ExecutionPlan>,
+        context: Arc<TaskContext>,
+    ) -> Result<QueryResults> {
+        let query = QueryBuilder::new(plan, context).build()?;
+        Ok(self.schedule_query(query))
+    }
+
+    /// Schedule the provided [`Query`] on this [`Scheduler`].
+    pub(crate) fn schedule_query(&self, query: Query) -> QueryResults {
+        spawn_query(query, self.spawner())
+    }
+
+    fn spawner(&self) -> Spawner {
+        Spawner {
+            pool: self.pool.clone(),
+        }
+    }
+}
+
+/// Formats a panic message for a worker
+fn format_worker_panic(panic: Box<dyn std::any::Any + Send>) -> String {
+    let maybe_idx = rayon::current_thread_index();
+    let worker: &dyn std::fmt::Display = match &maybe_idx {
+        Some(idx) => idx,
+        None => &"UNKNOWN",
+    };
+
+    let message = if let Some(msg) = panic.downcast_ref::<&str>() {

Review Comment:
   I don't think downcast_ref lets you cast to different types from the original Any, so I don't think this would work unless someone actually panic'ed with a `&'static dyn Display` type (which would be hard to construct if even possible)



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #2226: Introduce new optional scheduler, using Morsel-driven Parallelism + rayon (#2199)

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2226:
URL: https://github.com/apache/arrow-datafusion/pull/2226#discussion_r856290635


##########
datafusion/scheduler/Cargo.toml:
##########
@@ -0,0 +1,57 @@
+# 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.
+
+[package]
+name = "datafusion-scheduler"
+description = "Scheduling for DataFusion query engine"
+version = "7.0.0"
+homepage = "https://github.com/apache/arrow-datafusion"
+repository = "https://github.com/apache/arrow-datafusion"
+readme = "../README.md"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "query", "sql"]
+edition = "2021"
+rust-version = "1.58"
+
+[lib]
+name = "datafusion_scheduler"
+path = "src/lib.rs"
+
+[features]
+
+[dependencies]
+ahash = { version = "0.7", default-features = false }
+arrow = { version = "12" }
+async-trait = "0.1"
+datafusion = { path = "../core", version = "7.0.0" }

Review Comment:
   Sounds good - this will also help down the line if we want to stabilise things :+1:



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org