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/23 17:27:48 UTC

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

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