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: [email protected]

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

Reply via email to