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

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

Reply via email to