diff --git a/azure-pipelines.yml b/azure-pipelines.yml index fe8eb58cfc5..0d7c1077a06 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -65,7 +65,7 @@ jobs: tokio-codec: [] tokio-executor: - current-thread - - threadpool + - thread-pool tokio-io: - util tokio-sync: @@ -94,6 +94,13 @@ jobs: - tokio-no-features - tokio-with-net +# Run loom tests +- template: ci/azure-loom.yml + parameters: + rust: beta + crates: + - tokio-executor + # Try cross compiling - template: ci/azure-cross-compile.yml parameters: diff --git a/ci/azure-loom.yml b/ci/azure-loom.yml new file mode 100644 index 00000000000..c57bc732327 --- /dev/null +++ b/ci/azure-loom.yml @@ -0,0 +1,18 @@ +jobs: +- job: loom + displayName: Loom tests + pool: + vmImage: ubuntu-16.04 + + steps: + - template: azure-install-rust.yml + parameters: + rust_version: ${{ parameters.rust }} + + - ${{ each crate in parameters.crates }}: + - script: RUSTFLAGS="--cfg loom" cargo test --lib --release + env: + LOOM_MAX_PREEMPTIONS: 2 + CI: 'True' + displayName: test ${{ crate }} + workingDirectory: $(Build.SourcesDirectory)/${{ crate }} diff --git a/tokio-executor/Cargo.toml b/tokio-executor/Cargo.toml index df38d750955..7a1d15d5e44 100644 --- a/tokio-executor/Cargo.toml +++ b/tokio-executor/Cargo.toml @@ -23,40 +23,31 @@ categories = ["concurrency", "asynchronous"] [features] blocking = ["tokio-sync", "lazy_static"] current-thread = ["crossbeam-channel"] -threadpool = [ - "tokio-sync", - "crossbeam-deque", - "crossbeam-queue", - "crossbeam-utils", - "futures-core-preview", - "num_cpus", - "lazy_static", - "slab", -] +thread-pool = ["num_cpus"] [dependencies] +futures-util-preview = { version = "=0.3.0-alpha.19", features = ["channel"] } tokio-sync = { version = "=0.2.0-alpha.6", optional = true, path = "../tokio-sync" } tracing = { version = "0.1.5", optional = true } -futures-util-preview = { version = "=0.3.0-alpha.19", features = ["channel"] } # current-thread dependencies crossbeam-channel = { version = "0.3.8", optional = true } # threadpool dependencies -crossbeam-deque = { version = "0.7.0", optional = true } -crossbeam-queue = { version = "0.1.0", optional = true } -crossbeam-utils = { version = "0.6.4", optional = true } -futures-core-preview = { version = "=0.3.0-alpha.19", optional = true } num_cpus = { version = "1.2", optional = true } + +# blocking +futures-core-preview = { version = "=0.3.0-alpha.19", optional = true } lazy_static = { version = "1", optional = true } -slab = { version = "0.4.1", optional = true } [dev-dependencies] tokio = { version = "=0.2.0-alpha.6", path = "../tokio" } +tokio-sync = { version = "=0.2.0-alpha.6", path = "../tokio-sync" } tokio-test = { version = "=0.2.0-alpha.6", path = "../tokio-test" } futures-core-preview = "=0.3.0-alpha.19" +loom = { version = "0.2.9", features = ["futures", "checkpoint"] } rand = "0.7" [package.metadata.docs.rs] diff --git a/tokio-executor/benches/blocking.rs b/tokio-executor/benches/blocking.rs deleted file mode 100644 index ceb5631853e..00000000000 --- a/tokio-executor/benches/blocking.rs +++ /dev/null @@ -1,133 +0,0 @@ -#![cfg(feature = "broken")] -#![feature(test)] -#![warn(rust_2018_idioms)] - -extern crate test; - -const ITER: usize = 1_000; - -mod blocking { - use super::*; - use futures::future::*; - use tokio_executor::threadpool::{blocking, Builder}; - - #[bench] - fn cpu_bound(b: &mut test::Bencher) { - let pool = Builder::new().pool_size(2).max_blocking(20).build(); - - b.iter(|| { - let count_down = Arc::new(CountDown::new(ITER)); - - for _ in 0..ITER { - let count_down = count_down.clone(); - - pool.spawn(lazy(move || { - poll_fn(|| blocking(|| perform_complex_computation()).map_err(|_| panic!())) - .and_then(move |_| { - // Do something with the value - count_down.dec(); - Ok(()) - }) - })); - } - - count_down.wait(); - }) - } -} - -mod message_passing { - use super::*; - use futures::future::*; - use futures::sync::oneshot; - use tokio_executor::threadpool::Builder; - - #[bench] - fn cpu_bound(b: &mut test::Bencher) { - let pool = Builder::new().pool_size(2).max_blocking(20).build(); - - let blocking = threadpool::ThreadPool::new(20); - - b.iter(|| { - let count_down = Arc::new(CountDown::new(ITER)); - - for _ in 0..ITER { - let count_down = count_down.clone(); - let blocking = blocking.clone(); - - pool.spawn(lazy(move || { - // Create a channel to receive the return value. - let (tx, rx) = oneshot::channel(); - - // Spawn a task on the blocking thread pool to process the - // computation. - blocking.execute(move || { - let res = perform_complex_computation(); - tx.send(res).unwrap(); - }); - - rx.and_then(move |_| { - count_down.dec(); - Ok(()) - }) - .map_err(|_| panic!()) - })); - } - - count_down.wait(); - }) - } -} - -fn perform_complex_computation() -> usize { - use rand::*; - - // Simulate a CPU heavy computation - let mut rng = rand::thread_rng(); - rng.gen() -} - -// Util for waiting until the tasks complete - -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::*; -use std::sync::*; - -struct CountDown { - rem: AtomicUsize, - mutex: Mutex<()>, - condvar: Condvar, -} - -impl CountDown { - fn new(rem: usize) -> Self { - CountDown { - rem: AtomicUsize::new(rem), - mutex: Mutex::new(()), - condvar: Condvar::new(), - } - } - - fn dec(&self) { - let prev = self.rem.fetch_sub(1, AcqRel); - - if prev != 1 { - return; - } - - let _lock = self.mutex.lock().unwrap(); - self.condvar.notify_all(); - } - - fn wait(&self) { - let mut lock = self.mutex.lock().unwrap(); - - loop { - if self.rem.load(Acquire) == 0 { - return; - } - - lock = self.condvar.wait(lock).unwrap(); - } - } -} diff --git a/tokio-executor/benches/thread_pool.rs b/tokio-executor/benches/thread_pool.rs new file mode 100644 index 00000000000..003c7ca971e --- /dev/null +++ b/tokio-executor/benches/thread_pool.rs @@ -0,0 +1,161 @@ +#![feature(test)] + +extern crate test; + +use tokio_executor::thread_pool::{Builder, Spawner, ThreadPool}; +use tokio_sync::oneshot; + +use std::future::Future; +use std::pin::Pin; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering::Relaxed; +use std::sync::{mpsc, Arc}; +use std::task::{Context, Poll}; + +struct Backoff(usize); + +impl Future for Backoff { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { + if self.0 == 0 { + Poll::Ready(()) + } else { + self.0 -= 1; + cx.waker().wake_by_ref(); + Poll::Pending + } + } +} + +const NUM_THREADS: usize = 6; + +#[bench] +fn spawn_many(b: &mut test::Bencher) { + const NUM_SPAWN: usize = 10_000; + + let threadpool = Builder::new().num_threads(NUM_THREADS).build(); + + let (tx, rx) = mpsc::sync_channel(1000); + let rem = Arc::new(AtomicUsize::new(0)); + + b.iter(|| { + rem.store(NUM_SPAWN, Relaxed); + + for _ in 0..NUM_SPAWN { + let tx = tx.clone(); + let rem = rem.clone(); + + threadpool.spawn(async move { + if 1 == rem.fetch_sub(1, Relaxed) { + tx.send(()).unwrap(); + } + }); + } + + let _ = rx.recv().unwrap(); + }); +} + +#[bench] +fn yield_many(b: &mut test::Bencher) { + const NUM_YIELD: usize = 1_000; + const TASKS_PER_CPU: usize = 50; + + let threadpool = Builder::new().num_threads(NUM_THREADS).build(); + + let tasks = TASKS_PER_CPU * num_cpus::get_physical(); + let (tx, rx) = mpsc::sync_channel(tasks); + + b.iter(move || { + for _ in 0..tasks { + let tx = tx.clone(); + + threadpool.spawn(async move { + let backoff = Backoff(NUM_YIELD); + backoff.await; + tx.send(()).unwrap(); + }); + } + + for _ in 0..tasks { + let _ = rx.recv().unwrap(); + } + }); +} + +#[bench] +fn ping_pong(b: &mut test::Bencher) { + const NUM_PINGS: usize = 1_000; + + let threadpool = Builder::new().num_threads(NUM_THREADS).build(); + + let (done_tx, done_rx) = mpsc::sync_channel(1000); + let rem = Arc::new(AtomicUsize::new(0)); + + b.iter(|| { + let done_tx = done_tx.clone(); + let rem = rem.clone(); + rem.store(NUM_PINGS, Relaxed); + + let spawner = threadpool.spawner().clone(); + + threadpool.spawn(async move { + for _ in 0..NUM_PINGS { + let rem = rem.clone(); + let done_tx = done_tx.clone(); + + let spawner2 = spawner.clone(); + + spawner.spawn(async move { + let (tx1, rx1) = oneshot::channel(); + let (tx2, rx2) = oneshot::channel(); + + spawner2.spawn(async move { + rx1.await.unwrap(); + tx2.send(()).unwrap(); + }); + + tx1.send(()).unwrap(); + rx2.await.unwrap(); + + if 1 == rem.fetch_sub(1, Relaxed) { + done_tx.send(()).unwrap(); + } + }); + } + }); + + done_rx.recv().unwrap(); + }); +} + +#[bench] +fn chained_spawn(b: &mut test::Bencher) { + const ITER: usize = 1_000; + + let threadpool = Builder::new().num_threads(NUM_THREADS).build(); + + fn iter(spawner: Spawner, done_tx: mpsc::SyncSender<()>, n: usize) { + if n == 0 { + done_tx.send(()).unwrap(); + } else { + let s2 = spawner.clone(); + spawner.spawn(async move { + iter(s2, done_tx, n - 1); + }); + } + } + + let (done_tx, done_rx) = mpsc::sync_channel(1000); + + b.iter(move || { + let done_tx = done_tx.clone(); + let spawner = threadpool.spawner().clone(); + threadpool.spawn(async move { + iter(spawner, done_tx, ITER); + }); + + done_rx.recv().unwrap(); + }); +} diff --git a/tokio-executor/benches/threadpool.rs b/tokio-executor/benches/threadpool.rs deleted file mode 100644 index 56617a59fd7..00000000000 --- a/tokio-executor/benches/threadpool.rs +++ /dev/null @@ -1,161 +0,0 @@ -#![cfg(feature = "broken")] -#![feature(test)] -#![warn(rust_2018_idioms)] - -extern crate test; - -const NUM_SPAWN: usize = 10_000; -const NUM_YIELD: usize = 1_000; -const TASKS_PER_CPU: usize = 50; - -mod threadpool { - use futures::{future, task, Async}; - use num_cpus; - use std::sync::atomic::AtomicUsize; - use std::sync::atomic::Ordering::SeqCst; - use std::sync::{mpsc, Arc}; - use tokio_executor::threadpool::*; - - #[bench] - fn spawn_many(b: &mut test::Bencher) { - let threadpool = ThreadPool::new(); - - let (tx, rx) = mpsc::sync_channel(10); - let rem = Arc::new(AtomicUsize::new(0)); - - b.iter(move || { - rem.store(super::NUM_SPAWN, SeqCst); - - for _ in 0..super::NUM_SPAWN { - let tx = tx.clone(); - let rem = rem.clone(); - - threadpool.spawn(future::lazy(move || { - if 1 == rem.fetch_sub(1, SeqCst) { - tx.send(()).unwrap(); - } - - Ok(()) - })); - } - - let _ = rx.recv().unwrap(); - }); - } - - #[bench] - fn yield_many(b: &mut test::Bencher) { - let threadpool = ThreadPool::new(); - let tasks = super::TASKS_PER_CPU * num_cpus::get(); - - let (tx, rx) = mpsc::sync_channel(tasks); - - b.iter(move || { - for _ in 0..tasks { - let mut rem = super::NUM_YIELD; - let tx = tx.clone(); - - threadpool.spawn(future::poll_fn(move || { - rem -= 1; - - if rem == 0 { - tx.send(()).unwrap(); - Ok(Async::Ready(())) - } else { - // Notify the current task - task::current().notify(); - - // Not ready - Ok(Async::NotReady) - } - })); - } - - for _ in 0..tasks { - let _ = rx.recv().unwrap(); - } - }); - } -} - -// In this case, CPU pool completes the benchmark faster, but this is due to how -// CpuPool currently behaves, starving other futures. This completes the -// benchmark quickly but results in poor runtime characteristics for a thread -// pool. -// -// See rust-lang-nursery/futures-rs#617 -// -mod cpupool { - use futures::future::{self, Executor}; - use futures::{task, Async}; - use futures_cpupool::*; - use num_cpus; - use std::sync::atomic::AtomicUsize; - use std::sync::atomic::Ordering::SeqCst; - use std::sync::{mpsc, Arc}; - - #[bench] - fn spawn_many(b: &mut test::Bencher) { - let pool = CpuPool::new(num_cpus::get()); - - let (tx, rx) = mpsc::sync_channel(10); - let rem = Arc::new(AtomicUsize::new(0)); - - b.iter(move || { - rem.store(super::NUM_SPAWN, SeqCst); - - for _ in 0..super::NUM_SPAWN { - let tx = tx.clone(); - let rem = rem.clone(); - - pool.execute(future::lazy(move || { - if 1 == rem.fetch_sub(1, SeqCst) { - tx.send(()).unwrap(); - } - - Ok(()) - })) - .ok() - .unwrap(); - } - - let _ = rx.recv().unwrap(); - }); - } - - #[bench] - fn yield_many(b: &mut test::Bencher) { - let pool = CpuPool::new(num_cpus::get()); - let tasks = super::TASKS_PER_CPU * num_cpus::get(); - - let (tx, rx) = mpsc::sync_channel(tasks); - - b.iter(move || { - for _ in 0..tasks { - let mut rem = super::NUM_YIELD; - let tx = tx.clone(); - - pool.execute(future::poll_fn(move || { - rem -= 1; - - if rem == 0 { - tx.send(()).unwrap(); - Ok(Async::Ready(())) - } else { - // Notify the current task - task::current().notify(); - - // Not ready - Ok(Async::NotReady) - } - })) - .ok() - .unwrap(); - } - - for _ in 0..tasks { - let _ = rx.recv().unwrap(); - } - }); - } -} diff --git a/tokio-executor/benches/threadpool_depth.rs b/tokio-executor/benches/threadpool_depth.rs deleted file mode 100644 index e6d9183d3c0..00000000000 --- a/tokio-executor/benches/threadpool_depth.rs +++ /dev/null @@ -1,72 +0,0 @@ -#![cfg(feature = "broken")] -#![feature(test)] -#![warn(rust_2018_idioms)] - -extern crate test; - -const ITER: usize = 20_000; - -mod us { - use futures::future; - use std::sync::mpsc; - use tokio_executor::threadpool::*; - - #[bench] - fn chained_spawn(b: &mut test::Bencher) { - let threadpool = ThreadPool::new(); - - fn spawn(pool_tx: Sender, res_tx: mpsc::Sender<()>, n: usize) { - if n == 0 { - res_tx.send(()).unwrap(); - } else { - let pool_tx2 = pool_tx.clone(); - pool_tx - .spawn(future::lazy(move || { - spawn(pool_tx2, res_tx, n - 1); - Ok(()) - })) - .unwrap(); - } - } - - b.iter(move || { - let (res_tx, res_rx) = mpsc::channel(); - - spawn(threadpool.sender().clone(), res_tx, super::ITER); - res_rx.recv().unwrap(); - }); - } -} - -mod cpupool { - use futures::future::{self, Executor}; - use futures_cpupool::*; - use num_cpus; - use std::sync::mpsc; - - #[bench] - fn chained_spawn(b: &mut test::Bencher) { - let pool = CpuPool::new(num_cpus::get()); - - fn spawn(pool: CpuPool, res_tx: mpsc::Sender<()>, n: usize) { - if n == 0 { - res_tx.send(()).unwrap(); - } else { - let pool2 = pool.clone(); - pool.execute(future::lazy(move || { - spawn(pool2, res_tx, n - 1); - Ok(()) - })) - .ok() - .unwrap(); - } - } - - b.iter(move || { - let (res_tx, res_rx) = mpsc::channel(); - - spawn(pool.clone(), res_tx, super::ITER); - res_rx.recv().unwrap(); - }); - } -} diff --git a/tokio-executor/src/global.rs b/tokio-executor/src/global.rs index 81ea0af880a..2dfe35ac960 100644 --- a/tokio-executor/src/global.rs +++ b/tokio-executor/src/global.rs @@ -1,4 +1,7 @@ -use super::{Executor, SpawnError}; +#[cfg(feature = "thread-pool")] +use crate::thread_pool::ThreadPool; +use crate::{Executor, SpawnError}; + use std::cell::Cell; use std::future::Future; use std::pin::Pin; @@ -37,17 +40,18 @@ impl DefaultExecutor { #[inline] fn with_current R, R>(f: F) -> Option { - EXECUTOR.with( - |current_executor| match current_executor.replace(State::Active) { - State::Ready(executor_ptr) => { - let executor = unsafe { &mut *executor_ptr }; - let result = f(executor); - current_executor.set(State::Ready(executor_ptr)); - Some(result) - } - State::Empty | State::Active => None, - }, - ) + EXECUTOR.with(|current_executor| match current_executor.get() { + State::Ready(executor_ptr) => { + let executor = unsafe { &mut *executor_ptr }; + Some(f(executor)) + } + #[cfg(feature = "thread-pool")] + State::ThreadPool(threadpool_ptr) => { + let mut thread_pool = unsafe { &*threadpool_ptr }; + Some(f(&mut thread_pool)) + } + State::Empty => None, + }) } } @@ -55,10 +59,13 @@ impl DefaultExecutor { enum State { // default executor not defined Empty, - // default executor is defined and ready to be used + + // default executor is a thread pool instance. + #[cfg(feature = "thread-pool")] + ThreadPool(*const ThreadPool), + + // default executor is set to a custom executor. Ready(*mut dyn Executor), - // default executor is currently active (used to detect recursive calls) - Active, } thread_local! { @@ -132,7 +139,26 @@ pub fn spawn(future: T) where T: Future + Send + 'static, { - DefaultExecutor::current().spawn(Box::pin(future)).unwrap() + EXECUTOR.with(|current_executor| match current_executor.get() { + State::Ready(executor_ptr) => { + let executor = unsafe { &mut *executor_ptr }; + executor.spawn(Box::pin(future)).unwrap(); + } + #[cfg(feature = "thread-pool")] + State::ThreadPool(threadpool_ptr) => { + let thread_pool = unsafe { &*threadpool_ptr }; + thread_pool.spawn_background(future); + } + State::Empty => panic!("must be called from the context of Tokio runtime"), + }) +} + +#[cfg(feature = "thread-pool")] +pub(crate) fn with_threadpool(thread_pool: &ThreadPool, f: F) -> R +where + F: FnOnce() -> R, +{ + with_state(State::ThreadPool(thread_pool as *const ThreadPool), f) } /// Set the default executor for the duration of the closure @@ -143,9 +169,24 @@ pub fn with_default(executor: &mut T, f: F) -> R where T: Executor, F: FnOnce() -> R, +{ + // While scary, this is safe. The function takes a + // `&mut Executor`, which guarantees that the reference lives for the + // duration of `with_default`. + // + // Because we are always clearing the TLS value at the end of the + // function, we can cast the reference to 'static which thread-local + // cells require. + let executor = unsafe { hide_lt(executor as &mut _ as *mut _) }; + with_state(State::Ready(executor), f) +} + +fn with_state(state: State, f: F) -> R +where + F: FnOnce() -> R, { EXECUTOR.with(|cell| { - let was = cell.get(); + let was = cell.replace(State::Empty); // Ensure that the executor is removed from the thread-local context // when leaving the scope. This handles cases that involve panicking. @@ -159,16 +200,15 @@ where let _reset = Reset(cell, was); - // While scary, this is safe. The function takes a - // `&mut Executor`, which guarantees that the reference lives for the - // duration of `with_default`. - // - // Because we are always clearing the TLS value at the end of the - // function, we can cast the reference to 'static which thread-local - // cells require. - let executor = unsafe { hide_lt(executor as &mut _ as *mut _) }; + if let State::Ready(executor) = state { + let executor = unsafe { &mut *executor }; - cell.set(State::Ready(executor)); + if executor.status().is_err() { + panic!("executor not active; is this because `with_default` is called with `DefaultExecutor`?"); + } + } + + cell.set(state); f() }) @@ -183,7 +223,7 @@ unsafe fn hide_lt<'a>(p: *mut (dyn Executor + 'a)) -> *mut (dyn Executor + 'stat #[cfg(test)] mod tests { - use super::{with_default, DefaultExecutor, Executor}; + use super::{with_default, DefaultExecutor}; #[test] fn default_executor_is_send_and_sync() { @@ -193,12 +233,11 @@ mod tests { } #[test] + #[should_panic] fn nested_default_executor_status() { let _enter = super::super::enter().unwrap(); let mut executor = DefaultExecutor::current(); - let result = with_default(&mut executor, || DefaultExecutor::current().status()); - - assert!(result.err().unwrap().is_shutdown()) + let _result = with_default(&mut executor, || ()); } } diff --git a/tokio-executor/src/lib.rs b/tokio-executor/src/lib.rs index a1c00377274..b5e0bf471d6 100644 --- a/tokio-executor/src/lib.rs +++ b/tokio-executor/src/lib.rs @@ -60,6 +60,17 @@ //! [`DefaultExecutor`]: struct.DefaultExecutor.html //! [`Park`]: park/index.html //! [`Future::poll`]: https://doc.rust-lang.org/std/future/trait.Future.html#tymethod.poll + +#[cfg(all(test, loom))] +macro_rules! thread_local { + ($($tts:tt)+) => { loom::thread_local!{ $($tts)+ } } +} + +// At the top due to macros +#[cfg(test)] +#[macro_use] +mod tests; + #[cfg(any(feature = "current-thread", feature = "threadpool"))] #[macro_use] mod tracing; @@ -68,8 +79,13 @@ mod enter; mod error; mod executor; mod global; +mod loom; pub mod park; +#[cfg(feature = "thread-pool")] +mod task; mod typed; +#[cfg(feature = "thread-pool")] +mod util; #[cfg(feature = "blocking")] pub mod blocking; @@ -77,8 +93,8 @@ pub mod blocking; #[cfg(feature = "current-thread")] pub mod current_thread; -#[cfg(feature = "threadpool")] -pub mod threadpool; +#[cfg(feature = "thread-pool")] +pub mod thread_pool; pub use crate::enter::{enter, exit, Enter, EnterError}; pub use crate::error::SpawnError; diff --git a/tokio-executor/src/loom/mod.rs b/tokio-executor/src/loom/mod.rs new file mode 100644 index 00000000000..0336513552a --- /dev/null +++ b/tokio-executor/src/loom/mod.rs @@ -0,0 +1,25 @@ +//! Stub out the necessary APIs to model with loom. + +#[cfg(not(all(test, loom)))] +pub(crate) mod std; + +#[cfg(all(test, loom))] +pub(crate) mod std { + pub(crate) use loom::{alloc, cell, sync, thread}; + + pub(crate) mod rand { + pub(crate) fn seed() -> u64 { + 1 + } + } + + pub(crate) mod sys { + pub(crate) fn num_cpus() -> usize { + 2 + } + } +} + +pub(crate) use self::std::sync; +#[cfg(feature = "thread-pool")] +pub(crate) use self::std::{alloc, cell, rand, sys, thread}; diff --git a/tokio-executor/src/loom/std/atomic_u32.rs b/tokio-executor/src/loom/std/atomic_u32.rs new file mode 100644 index 00000000000..0128ab2b3d4 --- /dev/null +++ b/tokio-executor/src/loom/std/atomic_u32.rs @@ -0,0 +1,44 @@ +use std::cell::UnsafeCell; +use std::fmt; +use std::ops::Deref; + +/// `AtomicU32` providing an additional `load_unsync` function. +pub(crate) struct AtomicU32 { + inner: UnsafeCell, +} + +unsafe impl Send for AtomicU32 {} +unsafe impl Sync for AtomicU32 {} + +impl AtomicU32 { + pub(crate) fn new(val: u32) -> AtomicU32 { + let inner = UnsafeCell::new(std::sync::atomic::AtomicU32::new(val)); + AtomicU32 { inner } + } + + /// Perform an unsynchronized load. + /// + /// # Safety + /// + /// All mutations must have happened before the unsynchronized load. + /// Additionally, there must be no concurrent mutations. + pub(crate) unsafe fn unsync_load(&self) -> u32 { + *(*self.inner.get()).get_mut() + } +} + +impl Deref for AtomicU32 { + type Target = std::sync::atomic::AtomicU32; + + fn deref(&self) -> &Self::Target { + // safety: it is always safe to access `&self` fns on the inner value as + // we never perform unsafe mutations. + unsafe { &*self.inner.get() } + } +} + +impl fmt::Debug for AtomicU32 { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + self.deref().fmt(fmt) + } +} diff --git a/tokio-executor/src/loom/std/atomic_usize.rs b/tokio-executor/src/loom/std/atomic_usize.rs new file mode 100644 index 00000000000..e9db9a41df7 --- /dev/null +++ b/tokio-executor/src/loom/std/atomic_usize.rs @@ -0,0 +1,45 @@ +use std::cell::UnsafeCell; +use std::fmt; +use std::ops::Deref; + +/// `AtomicUsize` providing an additional `load_unsync` function. +pub(crate) struct AtomicUsize { + inner: UnsafeCell, +} + +unsafe impl Send for AtomicUsize {} +unsafe impl Sync for AtomicUsize {} + +impl AtomicUsize { + pub(crate) fn new(val: usize) -> AtomicUsize { + let inner = UnsafeCell::new(std::sync::atomic::AtomicUsize::new(val)); + AtomicUsize { inner } + } + + /// Perform an unsynchronized load. + /// + /// # Safety + /// + /// All mutations must have happened before the unsynchronized load. + /// Additionally, there must be no concurrent mutations. + #[cfg(feature = "thread-pool")] + pub(crate) unsafe fn unsync_load(&self) -> usize { + *(*self.inner.get()).get_mut() + } +} + +impl Deref for AtomicUsize { + type Target = std::sync::atomic::AtomicUsize; + + fn deref(&self) -> &Self::Target { + // safety: it is always safe to access `&self` fns on the inner value as + // we never perform unsafe mutations. + unsafe { &*self.inner.get() } + } +} + +impl fmt::Debug for AtomicUsize { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + self.deref().fmt(fmt) + } +} diff --git a/tokio-executor/src/loom/std/causal_cell.rs b/tokio-executor/src/loom/std/causal_cell.rs new file mode 100644 index 00000000000..8247788d507 --- /dev/null +++ b/tokio-executor/src/loom/std/causal_cell.rs @@ -0,0 +1,48 @@ +use std::cell::UnsafeCell; + +pub(crate) struct CausalCell(UnsafeCell); + +#[derive(Default)] +pub(crate) struct CausalCheck(()); + +impl CausalCell { + pub(crate) fn new(data: T) -> CausalCell { + CausalCell(UnsafeCell::new(data)) + } + + pub(crate) fn with(&self, f: F) -> R + where + F: FnOnce(*const T) -> R, + { + f(self.0.get()) + } + + pub(crate) fn with_unchecked(&self, f: F) -> R + where + F: FnOnce(*const T) -> R, + { + f(self.0.get()) + } + + pub(crate) fn check(&self) {} + + pub(crate) fn with_deferred(&self, f: F) -> (R, CausalCheck) + where + F: FnOnce(*const T) -> R, + { + (f(self.0.get()), CausalCheck::default()) + } + + pub(crate) fn with_mut(&self, f: F) -> R + where + F: FnOnce(*mut T) -> R, + { + f(self.0.get()) + } +} + +impl CausalCheck { + pub(crate) fn check(self) {} + + pub(crate) fn join(&mut self, _other: CausalCheck) {} +} diff --git a/tokio-executor/src/loom/std/mod.rs b/tokio-executor/src/loom/std/mod.rs new file mode 100644 index 00000000000..850399fe63a --- /dev/null +++ b/tokio-executor/src/loom/std/mod.rs @@ -0,0 +1,77 @@ +#[cfg(feature = "thread-pool")] +mod atomic_u32; +mod atomic_usize; +#[cfg(feature = "thread-pool")] +mod causal_cell; + +#[cfg(feature = "thread-pool")] +pub(crate) mod alloc { + #[derive(Debug)] + pub(crate) struct Track { + value: T, + } + + impl Track { + pub(crate) fn new(value: T) -> Track { + Track { value } + } + + pub(crate) fn get_mut(&mut self) -> &mut T { + &mut self.value + } + + pub(crate) fn into_inner(self) -> T { + self.value + } + } +} + +#[cfg(feature = "thread-pool")] +pub(crate) mod cell { + pub(crate) use super::causal_cell::{CausalCell, CausalCheck}; +} + +#[cfg(feature = "thread-pool")] +pub(crate) mod rand { + use std::collections::hash_map::RandomState; + use std::hash::{BuildHasher, Hash, Hasher}; + use std::sync::atomic::AtomicU32; + use std::sync::atomic::Ordering::Relaxed; + + static COUNTER: AtomicU32 = AtomicU32::new(1); + + pub(crate) fn seed() -> u64 { + let rand_state = RandomState::new(); + + let mut hasher = rand_state.build_hasher(); + + // Hash some unique-ish data to generate some new state + COUNTER.fetch_add(1, Relaxed).hash(&mut hasher); + + // Get the seed + hasher.finish() + } +} + +pub(crate) mod sync { + pub(crate) use std::sync::{Arc, Condvar, Mutex}; + + pub(crate) mod atomic { + #[cfg(feature = "thread-pool")] + pub(crate) use crate::loom::std::atomic_u32::AtomicU32; + pub(crate) use crate::loom::std::atomic_usize::AtomicUsize; + + #[cfg(feature = "thread-pool")] + pub(crate) use std::sync::atomic::{fence, spin_loop_hint, AtomicPtr}; + } +} + +#[cfg(feature = "thread-pool")] +pub(crate) mod sys { + pub(crate) fn num_cpus() -> usize { + usize::max(1, num_cpus::get_physical()) + } +} + +#[cfg(feature = "thread-pool")] +pub(crate) use std::thread; diff --git a/tokio-executor/src/park/mod.rs b/tokio-executor/src/park/mod.rs new file mode 100644 index 00000000000..16c78da81c8 --- /dev/null +++ b/tokio-executor/src/park/mod.rs @@ -0,0 +1,140 @@ +//! Abstraction over blocking and unblocking the current thread. +//! +//! Provides an abstraction over blocking the current thread. This is similar to +//! the park / unpark constructs provided by [`std`] but made generic. This +//! allows embedding custom functionality to perform when the thread is blocked. +//! +//! A blocked [`Park`][p] instance is unblocked by calling [`unpark`] on its +//! [`Unpark`][up] handle. +//! +//! The [`ParkThread`] struct implements [`Park`][p] using +//! [`thread::park`][`std`] to put the thread to sleep. The Tokio reactor also +//! implements park, but uses [`mio::Poll`][mio] to block the thread instead. +//! +//! The [`Park`][p] trait is composable. A timer implementation might decorate a +//! [`Park`][p] implementation by checking if any timeouts have elapsed after +//! the inner [`Park`][p] implementation unblocks. +//! +//! # Model +//! +//! Conceptually, each [`Park`][p] instance has an associated token, which is +//! initially not present: +//! +//! * The [`park`] method blocks the current thread unless or until the token +//! is available, at which point it atomically consumes the token. +//! * The [`unpark`] method atomically makes the token available if it wasn't +//! already. +//! +//! Some things to note: +//! +//! * If [`unpark`] is called before [`park`], the next call to [`park`] will +//! **not** block the thread. +//! * **Spurious** wakeups are permitted, i.e., the [`park`] method may unblock +//! even if [`unpark`] was not called. +//! * [`park_timeout`] does the same as [`park`] but allows specifying a maximum +//! time to block the thread for. +//! +//! [`std`]: https://doc.rust-lang.org/std/thread/fn.park.html +//! [`thread::park`]: https://doc.rust-lang.org/std/thread/fn.park.html +//! [`ParkThread`]: struct.ParkThread.html +//! [p]: trait.Park.html +//! [`park`]: trait.Park.html#tymethod.park +//! [`park_timeout`]: trait.Park.html#tymethod.park_timeout +//! [`unpark`]: trait.Unpark.html#tymethod.unpark +//! [up]: trait.Unpark.html +//! [mio]: https://docs.rs/mio/0.6/mio/struct.Poll.html + +mod thread; +pub use self::thread::{ParkError, ParkThread, UnparkThread}; + +use std::sync::Arc; +use std::time::Duration; + +/// Block the current thread. +/// +/// See [module documentation][mod] for more details. +/// +/// [mod]: ../index.html +pub trait Park { + /// Unpark handle type for the `Park` implementation. + type Unpark: Unpark; + + /// Error returned by `park` + type Error; + + /// Get a new `Unpark` handle associated with this `Park` instance. + fn unpark(&self) -> Self::Unpark; + + /// Block the current thread unless or until the token is available. + /// + /// A call to `park` does not guarantee that the thread will remain blocked + /// forever, and callers should be prepared for this possibility. This + /// function may wakeup spuriously for any reason. + /// + /// See [module documentation][mod] for more details. + /// + /// # Panics + /// + /// This function **should** not panic, but ultimately, panics are left as + /// an implementation detail. Refer to the documentation for the specific + /// `Park` implementation + /// + /// [mod]: ../index.html + fn park(&mut self) -> Result<(), Self::Error>; + + /// Park the current thread for at most `duration`. + /// + /// This function is the same as `park` but allows specifying a maximum time + /// to block the thread for. + /// + /// Same as `park`, there is no guarantee that the thread will remain + /// blocked for any amount of time. Spurious wakeups are permitted for any + /// reason. + /// + /// See [module documentation][mod] for more details. + /// + /// # Panics + /// + /// This function **should** not panic, but ultimately, panics are left as + /// an implementation detail. Refer to the documentation for the specific + /// `Park` implementation + /// + /// [mod]: ../index.html + fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error>; +} + +/// Unblock a thread blocked by the associated [`Park`] instance. +/// +/// See [module documentation][mod] for more details. +/// +/// [mod]: ../index.html +/// [`Park`]: trait.Park.html +pub trait Unpark: Sync + Send + 'static { + /// Unblock a thread that is blocked by the associated `Park` handle. + /// + /// Calling `unpark` atomically makes available the unpark token, if it is + /// not already available. + /// + /// See [module documentation][mod] for more details. + /// + /// # Panics + /// + /// This function **should** not panic, but ultimately, panics are left as + /// an implementation detail. Refer to the documentation for the specific + /// `Unpark` implementation + /// + /// [mod]: ../index.html + fn unpark(&self); +} + +impl Unpark for Box { + fn unpark(&self) { + (**self).unpark() + } +} + +impl Unpark for Arc { + fn unpark(&self) { + (**self).unpark() + } +} diff --git a/tokio-executor/src/park.rs b/tokio-executor/src/park/thread.rs similarity index 58% rename from tokio-executor/src/park.rs rename to tokio-executor/src/park/thread.rs index 86876654bfe..1b9b2c563d9 100644 --- a/tokio-executor/src/park.rs +++ b/tokio-executor/src/park/thread.rs @@ -1,146 +1,14 @@ -//! Abstraction over blocking and unblocking the current thread. -//! -//! Provides an abstraction over blocking the current thread. This is similar to -//! the park / unpark constructs provided by [`std`] but made generic. This -//! allows embedding custom functionality to perform when the thread is blocked. -//! -//! A blocked [`Park`][p] instance is unblocked by calling [`unpark`] on its -//! [`Unpark`][up] handle. -//! -//! The [`ParkThread`] struct implements [`Park`][p] using -//! [`thread::park`][`std`] to put the thread to sleep. The Tokio reactor also -//! implements park, but uses [`mio::Poll`][mio] to block the thread instead. -//! -//! The [`Park`][p] trait is composable. A timer implementation might decorate a -//! [`Park`][p] implementation by checking if any timeouts have elapsed after -//! the inner [`Park`][p] implementation unblocks. -//! -//! # Model -//! -//! Conceptually, each [`Park`][p] instance has an associated token, which is -//! initially not present: -//! -//! * The [`park`] method blocks the current thread unless or until the token -//! is available, at which point it atomically consumes the token. -//! * The [`unpark`] method atomically makes the token available if it wasn't -//! already. -//! -//! Some things to note: -//! -//! * If [`unpark`] is called before [`park`], the next call to [`park`] will -//! **not** block the thread. -//! * **Spurious** wakeups are permitted, i.e., the [`park`] method may unblock -//! even if [`unpark`] was not called. -//! * [`park_timeout`] does the same as [`park`] but allows specifying a maximum -//! time to block the thread for. -//! -//! [`std`]: https://doc.rust-lang.org/std/thread/fn.park.html -//! [`thread::park`]: https://doc.rust-lang.org/std/thread/fn.park.html -//! [`ParkThread`]: struct.ParkThread.html -//! [p]: trait.Park.html -//! [`park`]: trait.Park.html#tymethod.park -//! [`park_timeout`]: trait.Park.html#tymethod.park_timeout -//! [`unpark`]: trait.Unpark.html#tymethod.unpark -//! [up]: trait.Unpark.html -//! [mio]: https://docs.rs/mio/0.6/mio/struct.Poll.html +use crate::loom::sync::atomic::AtomicUsize; +use crate::loom::sync::{Arc, Condvar, Mutex}; +use crate::park::{Park, Unpark}; use std::marker::PhantomData; use std::mem; use std::rc::Rc; -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::sync::{Arc, Condvar, Mutex}; +use std::sync::atomic::Ordering; use std::task::{RawWaker, RawWakerVTable, Waker}; use std::time::Duration; -/// Block the current thread. -/// -/// See [module documentation][mod] for more details. -/// -/// [mod]: ../index.html -pub trait Park { - /// Unpark handle type for the `Park` implementation. - type Unpark: Unpark; - - /// Error returned by `park` - type Error; - - /// Get a new `Unpark` handle associated with this `Park` instance. - fn unpark(&self) -> Self::Unpark; - - /// Block the current thread unless or until the token is available. - /// - /// A call to `park` does not guarantee that the thread will remain blocked - /// forever, and callers should be prepared for this possibility. This - /// function may wakeup spuriously for any reason. - /// - /// See [module documentation][mod] for more details. - /// - /// # Panics - /// - /// This function **should** not panic, but ultimately, panics are left as - /// an implementation detail. Refer to the documentation for the specific - /// `Park` implementation - /// - /// [mod]: ../index.html - fn park(&mut self) -> Result<(), Self::Error>; - - /// Park the current thread for at most `duration`. - /// - /// This function is the same as `park` but allows specifying a maximum time - /// to block the thread for. - /// - /// Same as `park`, there is no guarantee that the thread will remain - /// blocked for any amount of time. Spurious wakeups are permitted for any - /// reason. - /// - /// See [module documentation][mod] for more details. - /// - /// # Panics - /// - /// This function **should** not panic, but ultimately, panics are left as - /// an implementation detail. Refer to the documentation for the specific - /// `Park` implementation - /// - /// [mod]: ../index.html - fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error>; -} - -/// Unblock a thread blocked by the associated [`Park`] instance. -/// -/// See [module documentation][mod] for more details. -/// -/// [mod]: ../index.html -/// [`Park`]: trait.Park.html -pub trait Unpark: Sync + Send + 'static { - /// Unblock a thread that is blocked by the associated `Park` handle. - /// - /// Calling `unpark` atomically makes available the unpark token, if it is - /// not already available. - /// - /// See [module documentation][mod] for more details. - /// - /// # Panics - /// - /// This function **should** not panic, but ultimately, panics are left as - /// an implementation detail. Refer to the documentation for the specific - /// `Unpark` implementation - /// - /// [mod]: ../index.html - fn unpark(&self); -} - -impl Unpark for Box { - fn unpark(&self) { - (**self).unpark() - } -} - -impl Unpark for Arc { - fn unpark(&self) { - (**self).unpark() - } -} - /// Blocks the current thread using a condition variable. /// /// Implements the [`Park`] functionality by using a condition variable. An @@ -353,7 +221,7 @@ impl Unpark for UnparkThread { } } -static VTABLE: RawWakerVTable = RawWakerVTable::new(clone, wake, wake_by_ref, drop); +static VTABLE: RawWakerVTable = RawWakerVTable::new(clone, wake, wake_by_ref, drop_waker); impl UnparkThread { pub(crate) fn into_waker(self) -> Waker { @@ -377,6 +245,10 @@ unsafe fn clone(raw: *const ()) -> RawWaker { unparker_to_raw_waker(unparker) } +unsafe fn drop_waker(raw: *const ()) { + let _ = Inner::from_raw(raw); +} + unsafe fn wake(raw: *const ()) { let unparker = Inner::from_raw(raw); unparker.unpark(); diff --git a/tokio-executor/src/task/core.rs b/tokio-executor/src/task/core.rs new file mode 100644 index 00000000000..9c4e813eb5f --- /dev/null +++ b/tokio-executor/src/task/core.rs @@ -0,0 +1,153 @@ +use crate::loom::alloc::Track; +use crate::loom::cell::CausalCell; +use crate::task::raw::{self, Vtable}; +use crate::task::state::State; +use crate::task::waker::waker_ref; +use crate::task::Schedule; + +use std::cell::UnsafeCell; +use std::future::Future; +use std::mem::MaybeUninit; +use std::pin::Pin; +use std::ptr::{self, NonNull}; +use std::task::{Context, Poll, Waker}; + +/// The task cell. Contains the components of the task. +/// +/// It is critical for `Header` to be the first field as the task structure will +/// be referenced by both *mut Cell and *mut Header. +#[repr(C)] +pub(super) struct Cell { + /// Hot task state data + pub(super) header: Header, + + /// Either the future or output, depending on the execution stage. + pub(super) core: Core, + + /// Cold data + pub(super) trailer: Trailer, +} + +/// The core of the task. +/// +/// Holds the future or output, depending on the stage of execution. +pub(super) struct Core { + stage: Stage, +} + +/// Crate public as this is also needed by the pool. +#[repr(C)] +pub(crate) struct Header { + /// Task state + pub(super) state: State, + + /// Pointer to the executor owned by the task + pub(super) executor: CausalCell>>, + + /// Pointer to next task, used for misc task linked lists. + pub(crate) queue_next: UnsafeCell<*const Header>, + + /// Pointer to the next task in the ownership list. + pub(crate) owned_next: UnsafeCell>>>, + + /// Pointer to the previous task in the ownership list. + pub(crate) owned_prev: UnsafeCell>>>, + + /// Table of function pointers for executing actions on the task. + pub(super) vtable: &'static Vtable, + + /// Used by loom to track the causality of the future. Without loom, this is + /// unit. + pub(super) future_causality: CausalCell<()>, +} + +/// Cold data is stored after the future. +pub(super) struct Trailer { + /// Consumer task waiting on completion of this task. + pub(super) waker: CausalCell>>, +} + +/// Either the future or the output. +enum Stage { + Running(Track), + Finished(Track>), + Consumed, +} + +impl Cell { + /// Allocate a new task cell, containing the header, trailer, and core + /// structures. + pub(super) fn new(future: T, state: State) -> Box> { + Box::new(Cell { + header: Header { + state, + executor: CausalCell::new(None), + queue_next: UnsafeCell::new(ptr::null()), + owned_next: UnsafeCell::new(None), + owned_prev: UnsafeCell::new(None), + vtable: raw::vtable::(), + future_causality: CausalCell::new(()), + }, + core: Core { + stage: Stage::Running(Track::new(future)), + }, + trailer: Trailer { + waker: CausalCell::new(MaybeUninit::new(None)), + }, + }) + } +} + +impl Core { + pub(super) fn transition_to_consumed(&mut self) { + self.stage = Stage::Consumed + } + + pub(super) fn poll(&mut self, header: &Header) -> Poll + where + S: Schedule, + { + let res = { + let future = match &mut self.stage { + Stage::Running(tracked) => tracked.get_mut(), + _ => unreachable!("unexpected stage"), + }; + + // The future is pinned within the task. The above state transition + // has ensured the safety of this action. + let future = unsafe { Pin::new_unchecked(future) }; + + // The waker passed into the `poll` function does not require a ref + // count increment. + let waker_ref = waker_ref::(header); + let mut cx = Context::from_waker(&*waker_ref); + + future.poll(&mut cx) + }; + + if res.is_ready() { + self.stage = Stage::Consumed; + } + + res + } + + pub(super) fn store_output(&mut self, output: super::Result) { + self.stage = Stage::Finished(Track::new(output)); + } + + pub(super) unsafe fn read_output(&mut self, dst: *mut Track>) { + use std::mem; + + dst.write(match mem::replace(&mut self.stage, Stage::Consumed) { + Stage::Finished(output) => output, + _ => unreachable!("unexpected state"), + }); + } +} + +impl Header { + pub(super) fn executor(&self) -> Option> { + unsafe { self.executor.with(|ptr| *ptr) } + } +} diff --git a/tokio-executor/src/task/error.rs b/tokio-executor/src/task/error.rs new file mode 100644 index 00000000000..17d3911ca12 --- /dev/null +++ b/tokio-executor/src/task/error.rs @@ -0,0 +1,48 @@ +use std::any::Any; +use std::fmt; + +/// Task failed to execute to completion. +pub struct Error { + repr: Repr, +} + +enum Repr { + Cancelled, + Panic(Box), +} + +impl Error { + /// Create a new `cancelled` error + pub fn cancelled() -> Error { + Error { + repr: Repr::Cancelled, + } + } + + /// Create a new `panic` error + pub fn panic(err: Box) -> Error { + Error { + repr: Repr::Panic(err), + } + } +} + +impl fmt::Display for Error { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + match &self.repr { + Repr::Cancelled => write!(fmt, "cancelled"), + Repr::Panic(_) => write!(fmt, "panic"), + } + } +} + +impl fmt::Debug for Error { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + match &self.repr { + Repr::Cancelled => write!(fmt, "task::Error::Cancelled"), + Repr::Panic(_) => write!(fmt, "task::Error::Panic(...)"), + } + } +} + +impl std::error::Error for Error {} diff --git a/tokio-executor/src/task/harness.rs b/tokio-executor/src/task/harness.rs new file mode 100644 index 00000000000..15b3b023d87 --- /dev/null +++ b/tokio-executor/src/task/harness.rs @@ -0,0 +1,546 @@ +use crate::loom::alloc::Track; +use crate::loom::cell::CausalCheck; +use crate::task::core::{Cell, Core, Header, Trailer}; +use crate::task::state::Snapshot; +use crate::task::{Error, Schedule, Task}; + +use std::future::Future; +use std::mem::{ManuallyDrop, MaybeUninit}; +use std::ptr::NonNull; +use std::task::{Poll, Waker}; + +/// Typed raw task handle +pub(super) struct Harness { + cell: NonNull>, +} + +impl Harness +where + T: Future, + S: 'static, +{ + pub(super) unsafe fn from_raw(ptr: *mut ()) -> Harness { + debug_assert!(!ptr.is_null()); + + let cell = NonNull::new_unchecked(ptr as *mut Cell); + Harness { cell } + } + + fn header(&self) -> &Header { + unsafe { &self.cell.as_ref().header } + } + + fn trailer(&self) -> &Trailer { + unsafe { &self.cell.as_ref().trailer } + } + + fn core(&mut self) -> &mut Core { + unsafe { &mut self.cell.as_mut().core } + } +} + +impl Harness +where + T: Future, + S: Schedule, +{ + /// Poll the inner future. + /// + /// All necessary state checks and transitions are performed. + /// + /// Panics raised while polling the future are handled. + /// + /// Returns `true` if the task needs to be scheduled again + pub(super) fn poll(mut self, executor: NonNull) -> bool { + use std::panic; + + // Transition the task to the running state. + let res = self.header().state.transition_to_running(); + + if res.is_canceled() { + // The task was concurrently canceled. + self.do_cancel(res); + return false; + } + + let join_interest = res.is_join_interested(); + debug_assert!(join_interest || !res.has_join_waker()); + + // Get the cell components + let cell = unsafe { &mut self.cell.as_mut() }; + let header = &cell.header; + let core = &mut cell.core; + + // If the task's executor pointer is not yet set, then set it here. This + // is safe because a) this is the only time the value is set. b) at this + // point, there are no outstanding wakers which might access the + // field concurrently. + if header.executor().is_none() { + unsafe { + // We don't want the destructor to run because we don't really + // own the task here. + let task = ManuallyDrop::new(Task::from_raw(header.into())); + // Call the scheduler's bind callback + executor.as_ref().bind(&task); + header.executor.with_mut(|ptr| *ptr = Some(executor)); + } + } + + // The transition to `Running` done above ensures that a lock on the + // future has been obtained. This also ensures the `*mut T` pointer + // contains the future (as opposed to the output) and is initialized. + + let res = header.future_causality.with_mut(|_| { + panic::catch_unwind(panic::AssertUnwindSafe(|| { + struct Guard<'a, T: Future> { + core: &'a mut Core, + polled: bool, + } + + impl Drop for Guard<'_, T> { + fn drop(&mut self) { + if !self.polled { + self.core.transition_to_consumed(); + } + } + } + + let mut guard = Guard { + core, + polled: false, + }; + + let res = guard.core.poll(header); + + // prevent the guard from dropping the future + guard.polled = true; + + res + })) + }); + + match res { + Ok(Poll::Ready(out)) => { + self.complete(executor, join_interest, Ok(out)); + false + } + Ok(Poll::Pending) => { + let res = self.header().state.transition_to_idle(); + + if res.is_canceled() { + self.do_cancel(res); + false + } else { + res.is_notified() + } + } + Err(err) => { + self.complete(executor, join_interest, Err(Error::panic(err))); + false + } + } + } + + pub(super) unsafe fn drop_task(mut self) { + let might_drop_join_waker_on_release = self.might_drop_join_waker_on_release(); + + // Read the join waker cell just to have it + let (join_waker, check) = self.read_join_waker(); + + // transition the task to released + let res = self.header().state.release_task(); + + assert!(res.is_terminal(), "state = {:?}", res); + + if might_drop_join_waker_on_release && !res.is_join_interested() { + debug_assert!(res.has_join_waker()); + + // Its our responsibility to drop the waker + check.check(); + let _ = join_waker.assume_init(); + } + + if res.is_final_ref() { + self.dealloc(); + } + } + + unsafe fn dealloc(self) { + // Check causality + self.header().executor.with_mut(|_| {}); + self.header().future_causality.with_mut(|_| {}); + self.trailer().waker.with_mut(|_| { + // we can't check the contents of this cell as it is considered + // "uninitialized" data at this point. + }); + + drop(Box::from_raw(self.cell.as_ptr())); + } + + // ===== join handle ===== + + pub(super) unsafe fn read_output( + mut self, + dst: *mut Track>, + state: Snapshot, + ) { + if state.is_canceled() { + dst.write(Track::new(Err(Error::cancelled()))); + } else { + self.core().read_output(dst); + } + + // Before transitioning the state, the waker must be read. It is + // possible that, after the transition, we are responsible for dropping + // the waker but before the waker can be read from the struct, the + // struct is deallocated. + let (waker, check) = self.read_join_waker(); + + // The operation counts as dropping the join handle + let res = self.header().state.complete_join_handle(); + + if res.is_released() { + // We are responsible for freeing the waker handle + check.check(); + drop(waker.assume_init()); + } + + if res.is_final_ref() { + self.dealloc(); + } + } + + pub(super) fn store_join_waker(&self, waker: &Waker) -> Snapshot { + unsafe { + self.trailer().waker.with_mut(|ptr| { + (*ptr).as_mut_ptr().replace(Some(waker.clone())); + }); + } + + let res = self.header().state.store_join_waker(); + + if res.is_complete() || res.is_canceled() { + // Drop the waker here + self.trailer() + .waker + .with_mut(|ptr| unsafe { *(*ptr).as_mut_ptr() = None }); + } + + res + } + + pub(super) fn swap_join_waker(&self, waker: &Waker, prev: Snapshot) -> Snapshot { + unsafe { + let will_wake = self + .trailer() + .waker + .with(|ptr| (*(*ptr).as_ptr()).as_ref().unwrap().will_wake(waker)); + + if will_wake { + return prev; + } + + // Acquire the lock + let state = self.header().state.unset_waker(); + + if state.is_active() { + return self.store_join_waker(waker); + } + + state + } + } + + pub(super) fn drop_join_handle_slow(mut self) { + unsafe { + // Before transitioning the state, the waker must be read. It is + // possible that, after the transition, we are responsible for dropping + // the waker but before the waker can be read from the struct, the + // struct is deallocated. + let (waker, check) = self.read_join_waker(); + + // The operation counts as dropping the join handle + let res = match self.header().state.drop_join_handle_slow() { + Ok(res) => res, + Err(res) => { + // The task output must be read & dropped + debug_assert!(!(res.is_complete() && res.is_canceled())); + + if res.is_complete() { + self.core().transition_to_consumed(); + } + + self.header().state.complete_join_handle() + } + }; + + if !(res.is_complete() | res.is_canceled()) || res.is_released() { + // We are responsible for freeing the waker handle + check.check(); + drop(waker.assume_init()); + } + + if res.is_final_ref() { + self.dealloc(); + } + } + } + + // ===== waker behavior ===== + + pub(super) fn wake_by_val(self) { + self.wake_by_ref(); + self.drop_waker(); + } + + pub(super) fn wake_by_local_ref(&self) { + self.wake_by_ref(); + } + + pub(super) fn wake_by_ref(&self) { + if self.header().state.transition_to_notified() { + unsafe { + let executor = match self.header().executor.with(|ptr| *ptr) { + Some(executor) => executor, + None => panic!("executor should be set"), + }; + + S::schedule(executor.as_ref(), self.to_task()); + } + } + } + + pub(super) fn drop_waker(self) { + if self.header().state.ref_dec() { + unsafe { + self.dealloc(); + } + } + } + + /// Cancel the task. + /// + /// `from_queue` signals the caller is cancelling the task after popping it + /// from the queue. This indicates "polling" capability. + pub(super) fn cancel(self, from_queue: bool) { + let res = if from_queue { + self.header().state.transition_to_canceled_from_queue() + } else { + match self.header().state.transition_to_canceled_from_list() { + Some(res) => res, + None => return, + } + }; + + self.do_cancel(res); + } + + fn do_cancel(mut self, res: Snapshot) { + use std::panic; + + debug_assert!(!res.is_complete()); + + let cell = unsafe { &mut self.cell.as_mut() }; + let header = &cell.header; + let core = &mut cell.core; + + // Since we transitioned the task state to `canceled`, it won't ever be + // polled again. We are now responsible for all cleanup. + // + // We have to drop the future + // + header.future_causality.with_mut(|_| { + // Guard against potential panics in the drop handler + let _ = panic::catch_unwind(panic::AssertUnwindSafe(|| { + // Drop the future + core.transition_to_consumed(); + })); + }); + + // If there is a join waker, we must notify it so it can observe the + // task was canceled. + if res.is_join_interested() && res.has_join_waker() { + // Notify the join handle. The transition to cancelled obtained a + // lock on the waker cell. + unsafe { + self.wake_join(); + } + + // Also track that we might be responsible for releasing the waker. + self.set_might_drop_join_waker_on_release(); + } + + // The `RELEASED` flag is not set yet. + assert!(!res.is_final_ref()); + + // This **can** be null if the task is being cancelled before it was + // ever polled. + let bound_executor = unsafe { self.header().executor.with(|ptr| *ptr) }; + + unsafe { + let task = self.to_task(); + + if let Some(executor) = bound_executor { + executor.as_ref().release(task); + } else { + // Just drop the task. This will release / deallocate memory. + drop(task); + } + } + } + + // ====== internal ====== + + fn complete( + mut self, + executor: NonNull, + join_interest: bool, + output: super::Result, + ) { + if join_interest { + // Store the output. The future has already been dropped + self.core().store_output(output); + } + + let bound_executor = unsafe { self.header().executor.with(|ptr| *ptr) }; + + // Handle releasing the task. First, check if the current + // executor is the one that is bound to the task: + if Some(executor) == bound_executor { + unsafe { + // perform a local release + let task = ManuallyDrop::new(self.to_task()); + executor.as_ref().release_local(&task); + + if self.transition_to_released(join_interest).is_final_ref() { + self.dealloc(); + } + } + } else { + let res = self.transition_to_complete(join_interest); + assert!(!res.is_final_ref()); + + if res.has_join_waker() { + // The release step happens later once the task has migrated back to + // the worker that owns it. At that point, the releaser **may** also + // be responsible for dropping. This fact must be tracked until + // the release step happens. + self.set_might_drop_join_waker_on_release(); + } + + unsafe { + let task = self.to_task(); + + let executor = match bound_executor { + Some(executor) => executor, + None => panic!("executor should be set"), + }; + + executor.as_ref().release(task); + } + } + } + + /// Return `true` if the task structure should be deallocated + fn transition_to_complete(&mut self, join_interest: bool) -> Snapshot { + let res = self.header().state.transition_to_complete(); + + self.notify_join_handle(join_interest, res); + + // Transition to complete last to ensure freeing does + // not happen until the above work is done. + res + } + + /// Return `true` if the task structure should be deallocated + fn transition_to_released(&mut self, join_interest: bool) -> Snapshot { + if join_interest { + let res1 = self.transition_to_complete(join_interest); + + // At this point, the join waker may not be changed. Once we perform + // `release_task` we may no longer read from the struct but we + // **may** be responsible for dropping the waker. We do an + // optimistic read here. + let (join_waker, check) = unsafe { self.read_join_waker() }; + + let res2 = self.header().state.release_task(); + + if res1.has_join_waker() && !res2.is_join_interested() { + debug_assert!(res2.has_join_waker()); + + // Its our responsibility to drop the waker + check.check(); + unsafe { + drop(join_waker.assume_init()); + } + } + + res2 + } else { + self.header().state.transition_to_released() + } + } + + fn notify_join_handle(&mut self, join_interest: bool, res: Snapshot) { + if join_interest { + if !res.is_join_interested() { + debug_assert!(!res.has_join_waker()); + + // The join handle dropped interest before we could release + // the output. We are now responsible for releasing the + // output. + self.core().transition_to_consumed(); + } else if res.has_join_waker() { + if res.is_canceled() { + // The join handle will set the output to Cancelled without + // attempting to read the output. We must drop it here. + self.core().transition_to_consumed(); + } + + // Notify the join handle. The previous transition obtains the + // lock on the waker cell. + unsafe { + self.wake_join(); + } + } + } + } + + fn might_drop_join_waker_on_release(&self) -> bool { + unsafe { + let next = *self.header().queue_next.get() as usize; + next & 1 == 1 + } + } + + fn set_might_drop_join_waker_on_release(&self) { + unsafe { + debug_assert!( + (*self.header().queue_next.get()).is_null(), + "the task's queue_next field must be null when releasing" + ); + + *self.header().queue_next.get() = 1 as *const _; + } + } + + unsafe fn wake_join(&self) { + // LOOM: ensure we can make this call + self.trailer().waker.check(); + self.trailer().waker.with_unchecked(|ptr| { + (*(*ptr).as_ptr()) + .as_ref() + .expect("waker missing") + .wake_by_ref(); + }); + } + + unsafe fn read_join_waker(&mut self) -> (MaybeUninit>, CausalCheck) { + self.trailer().waker.with_deferred(|ptr| ptr.read()) + } + + unsafe fn to_task(&self) -> Task { + let ptr = self.cell.as_ptr() as *mut Header; + Task::from_raw(NonNull::new_unchecked(ptr)) + } +} diff --git a/tokio-executor/src/task/join.rs b/tokio-executor/src/task/join.rs new file mode 100644 index 00000000000..bce0faa4c71 --- /dev/null +++ b/tokio-executor/src/task/join.rs @@ -0,0 +1,74 @@ +use crate::loom::alloc::Track; +use crate::task::raw::RawTask; + +use std::future::Future; +use std::marker::PhantomData; +use std::pin::Pin; +use std::task::{Context, Poll}; + +pub(crate) struct JoinHandle { + raw: Option>, + _p: PhantomData, +} + +impl JoinHandle { + pub(super) fn new(raw: RawTask) -> JoinHandle { + JoinHandle { + raw: Some(raw), + _p: PhantomData, + } + } +} + +impl Unpin for JoinHandle {} + +impl Future for JoinHandle { + type Output = super::Result; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + use std::mem::MaybeUninit; + + // Raw should always be set + let raw = self.raw.as_ref().unwrap(); + + // Load the current task state + let mut state = raw.header().state.load(); + + debug_assert!(state.is_join_interested()); + + if state.is_active() { + state = if state.has_join_waker() { + raw.swap_join_waker(cx.waker(), state) + } else { + raw.store_join_waker(cx.waker()) + }; + + if state.is_active() { + return Poll::Pending; + } + } + + let mut out = MaybeUninit::>::uninit(); + + unsafe { + // This could result in the task being freed. + raw.read_output(out.as_mut_ptr() as *mut (), state); + + self.raw = None; + + Poll::Ready(out.assume_init().into_inner()) + } + } +} + +impl Drop for JoinHandle { + fn drop(&mut self) { + if let Some(raw) = self.raw.take() { + if raw.header().state.drop_join_handle_fast() { + return; + } + + raw.drop_join_handle_slow(); + } + } +} diff --git a/tokio-executor/src/task/list.rs b/tokio-executor/src/task/list.rs new file mode 100644 index 00000000000..d9e0b44d487 --- /dev/null +++ b/tokio-executor/src/task/list.rs @@ -0,0 +1,70 @@ +use crate::task::{Header, Task}; + +use std::fmt; +use std::ptr::NonNull; + +pub(crate) struct OwnedList { + head: Option>>, +} + +impl OwnedList { + pub(crate) fn new() -> OwnedList { + OwnedList { head: None } + } + + pub(crate) fn insert(&mut self, task: &Task) { + unsafe { + debug_assert!((*task.header().owned_next.get()).is_none()); + debug_assert!((*task.header().owned_prev.get()).is_none()); + + let ptr = Some(task.header().into()); + + if let Some(next) = self.head { + debug_assert!((*next.as_ref().owned_prev.get()).is_none()); + *next.as_ref().owned_prev.get() = ptr; + } + + *task.header().owned_next.get() = self.head; + self.head = ptr; + } + } + + pub(crate) fn remove(&mut self, task: &Task) { + unsafe { + if let Some(next) = *task.header().owned_next.get() { + *next.as_ref().owned_prev.get() = *task.header().owned_prev.get(); + } + + if let Some(prev) = *task.header().owned_prev.get() { + *prev.as_ref().owned_next.get() = *task.header().owned_next.get(); + } else { + debug_assert_eq!(self.head, Some(task.header().into())); + self.head = *task.header().owned_next.get(); + } + } + } + + pub(crate) fn is_empty(&self) -> bool { + self.head.is_none() + } + + /// Transition all tasks in the list to canceled as part of the shutdown + /// process. + pub(crate) fn shutdown(&self) { + let mut curr = self.head; + + while let Some(task) = curr { + unsafe { + let vtable = task.as_ref().vtable; + (vtable.cancel)(task.as_ptr() as *mut (), false); + curr = *task.as_ref().owned_next.get(); + } + } + } +} + +impl fmt::Debug for OwnedList { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("OwnedList").finish() + } +} diff --git a/tokio-executor/src/task/mod.rs b/tokio-executor/src/task/mod.rs new file mode 100644 index 00000000000..dfc6628e7da --- /dev/null +++ b/tokio-executor/src/task/mod.rs @@ -0,0 +1,130 @@ +mod core; +pub(crate) use self::core::Header; + +mod error; +pub use self::error::Error; + +mod harness; + +mod join; +pub(crate) use self::join::JoinHandle; + +mod list; +pub(crate) use self::list::OwnedList; + +mod raw; + +mod stack; +pub(crate) use self::stack::TransferStack; + +mod state; +mod waker; + +/// Unit tests +#[cfg(test)] +mod tests; + +use self::raw::RawTask; + +use std::future::Future; +use std::ptr::NonNull; +use std::{fmt, mem}; + +/// An owned handle to the task, tracked by ref count +pub(crate) struct Task { + raw: RawTask, +} + +unsafe impl Send for Task {} + +/// Task result sent back +pub(crate) type Result = std::result::Result; + +pub(crate) trait Schedule: Send + Sync + Sized + 'static { + /// Bind a task to the executor. + /// + /// Guaranteed to be called from the thread that called `poll` on the task. + fn bind(&self, task: &Task); + + /// The task has completed work and is ready to be released. The scheduler + /// is free to drop it whenever. + fn release(&self, task: Task); + + /// The has been completed by the executor it was bound to. + fn release_local(&self, task: &Task); + + /// Schedule the task + fn schedule(&self, task: Task); +} + +/// Create a new task without an associated join handle +pub(crate) fn background(task: T) -> Task +where + T: Future + Send + 'static, + S: Schedule, +{ + let raw = RawTask::new_background(task); + Task { raw } +} + +/// Create a new task with an associated join handle +pub(crate) fn joinable(task: T) -> (Task, JoinHandle) +where + T: Future + Send + 'static, + S: Schedule, +{ + let raw = RawTask::new_joinable(task); + let task = Task { raw }; + let join = JoinHandle::new(raw); + + (task, join) +} + +impl Task { + pub(crate) unsafe fn from_raw(ptr: NonNull>) -> Task { + let raw = RawTask::from_raw(ptr); + Task { raw } + } + + pub(crate) fn header(&self) -> &Header { + self.raw.header() + } + + pub(crate) fn into_raw(self) -> NonNull> { + let raw = self.raw.into_raw(); + mem::forget(self); + raw + } +} + +impl Task { + /// Returns `self` when the task needs to be immediately re-scheduled + pub(crate) fn run(self, executor: NonNull) -> Option { + if unsafe { self.raw.poll(executor) } { + Some(self) + } else { + // Cleaning up the `Task` instance is done from within the poll + // function. + mem::forget(self); + None + } + } + + /// Pre-emptively cancel the task as part of the shutdown process. + pub(crate) fn shutdown(self) { + self.raw.cancel_from_queue(); + mem::forget(self); + } +} + +impl Drop for Task { + fn drop(&mut self) { + self.raw.drop_task(); + } +} + +impl fmt::Debug for Task { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Task").finish() + } +} diff --git a/tokio-executor/src/task/raw.rs b/tokio-executor/src/task/raw.rs new file mode 100644 index 00000000000..49fbb96cfa0 --- /dev/null +++ b/tokio-executor/src/task/raw.rs @@ -0,0 +1,190 @@ +use crate::loom::alloc::Track; +use crate::task::core::Cell; +use crate::task::harness::Harness; +use crate::task::state::{Snapshot, State}; +use crate::task::{Header, Schedule}; + +use std::future::Future; +use std::ptr::NonNull; +use std::task::Waker; + +/// Raw task handle +pub(super) struct RawTask { + ptr: NonNull>, +} + +pub(super) struct Vtable { + /// Poll the future + pub(super) poll: unsafe fn(*mut (), NonNull) -> bool, + + /// The task handle has been dropped and the join waker needs to be dropped + /// or the task struct needs to be deallocated + pub(super) drop_task: unsafe fn(*mut ()), + + /// Read the task output + pub(super) read_output: unsafe fn(*mut (), *mut (), Snapshot), + + /// Store the join handle's waker + /// + /// Returns a snapshot of the state **after** the transition + pub(super) store_join_waker: unsafe fn(*mut (), &Waker) -> Snapshot, + + /// Replace the join handle's waker + /// + /// Returns a snapshot of the state **after** the transition + pub(super) swap_join_waker: unsafe fn(*mut (), &Waker, Snapshot) -> Snapshot, + + /// The join handle has been dropped + pub(super) drop_join_handle_slow: unsafe fn(*mut ()), + + /// The task is being canceled + pub(super) cancel: unsafe fn(*mut (), bool), +} + +/// Get the vtable for the requested `T` and `S` generics. +pub(super) fn vtable() -> &'static Vtable { + &Vtable { + poll: poll::, + drop_task: drop_task::, + read_output: read_output::, + store_join_waker: store_join_waker::, + swap_join_waker: swap_join_waker::, + drop_join_handle_slow: drop_join_handle_slow::, + cancel: cancel::, + } +} + +impl RawTask { + pub(super) fn new_background(task: T) -> RawTask + where + T: Future + Send + 'static, + S: Schedule, + { + RawTask::new(task, State::new_background()) + } + + pub(super) fn new_joinable(task: T) -> RawTask + where + T: Future + Send + 'static, + S: Schedule, + { + RawTask::new(task, State::new_joinable()) + } + + fn new(task: T, state: State) -> RawTask + where + T: Future + Send + 'static, + S: Schedule, + { + let ptr = Box::into_raw(Cell::::new(task, state)); + let ptr = unsafe { NonNull::new_unchecked(ptr as *mut Header) }; + + RawTask { ptr } + } + + pub(super) unsafe fn from_raw(ptr: NonNull>) -> RawTask { + RawTask { ptr } + } + + /// Returns a reference to the task's meta structure. + /// + /// Safe as `Header` is `Sync`. + pub(super) fn header(&self) -> &Header { + unsafe { self.ptr.as_ref() } + } + + /// Returns a raw pointer to the task's meta structure. + pub(super) fn into_raw(self) -> NonNull> { + self.ptr + } + + /// Safety: mutual exclusion is required to call this function. + /// + /// Returns `true` if the task needs to be scheduled again. + pub(super) unsafe fn poll(self, executor: NonNull) -> bool { + // Get the vtable without holding a ref to the meta struct. This is done + // because a mutable reference to the task is passed into the poll fn. + let vtable = self.header().vtable; + + (vtable.poll)(self.ptr.as_ptr() as *mut (), executor) + } + + pub(super) fn drop_task(self) { + let vtable = self.header().vtable; + unsafe { + (vtable.drop_task)(self.ptr.as_ptr() as *mut ()); + } + } + + pub(super) unsafe fn read_output(self, dst: *mut (), state: Snapshot) { + let vtable = self.header().vtable; + (vtable.read_output)(self.ptr.as_ptr() as *mut (), dst, state); + } + + pub(super) fn store_join_waker(self, waker: &Waker) -> Snapshot { + let vtable = self.header().vtable; + unsafe { (vtable.store_join_waker)(self.ptr.as_ptr() as *mut (), waker) } + } + + pub(super) fn swap_join_waker(self, waker: &Waker, prev: Snapshot) -> Snapshot { + let vtable = self.header().vtable; + unsafe { (vtable.swap_join_waker)(self.ptr.as_ptr() as *mut (), waker, prev) } + } + + pub(super) fn drop_join_handle_slow(self) { + let vtable = self.header().vtable; + unsafe { (vtable.drop_join_handle_slow)(self.ptr.as_ptr() as *mut ()) } + } + + pub(super) fn cancel_from_queue(self) { + let vtable = self.header().vtable; + unsafe { (vtable.cancel)(self.ptr.as_ptr() as *mut (), true) } + } +} + +impl Clone for RawTask { + fn clone(&self) -> Self { + RawTask { ptr: self.ptr } + } +} + +impl Copy for RawTask {} + +unsafe fn poll(ptr: *mut (), executor: NonNull) -> bool { + let harness = Harness::::from_raw(ptr); + harness.poll(executor) +} + +unsafe fn drop_task(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); + harness.drop_task(); +} + +unsafe fn read_output(ptr: *mut (), dst: *mut (), state: Snapshot) { + let harness = Harness::::from_raw(ptr); + harness.read_output(dst as *mut Track>, state); +} + +unsafe fn store_join_waker(ptr: *mut (), waker: &Waker) -> Snapshot { + let harness = Harness::::from_raw(ptr); + harness.store_join_waker(waker) +} + +unsafe fn swap_join_waker( + ptr: *mut (), + waker: &Waker, + prev: Snapshot, +) -> Snapshot { + let harness = Harness::::from_raw(ptr); + harness.swap_join_waker(waker, prev) +} + +unsafe fn drop_join_handle_slow(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); + harness.drop_join_handle_slow() +} + +unsafe fn cancel(ptr: *mut (), from_queue: bool) { + let harness = Harness::::from_raw(ptr); + harness.cancel(from_queue) +} diff --git a/tokio-executor/src/task/stack.rs b/tokio-executor/src/task/stack.rs new file mode 100644 index 00000000000..9c981b4be76 --- /dev/null +++ b/tokio-executor/src/task/stack.rs @@ -0,0 +1,85 @@ +use crate::loom::sync::atomic::AtomicPtr; +use crate::task::{Header, Task}; + +use std::ptr::{self, NonNull}; +use std::sync::atomic::Ordering::{Acquire, Relaxed, Release}; + +/// Concurrent stack of tasks, used to pass ownership of a task from one worker +/// to another. +pub(crate) struct TransferStack { + head: AtomicPtr>, +} + +impl TransferStack { + pub(crate) fn new() -> TransferStack { + TransferStack { + head: AtomicPtr::new(ptr::null_mut()), + } + } + + pub(crate) fn push(&self, task: Task) { + unsafe { + let task = task.into_raw(); + + let next = (*task.as_ref().queue_next.get()) as usize; + + // At this point, the queue_next field may also be used to track + // whether or not the task must drop the join waker. + debug_assert_eq!(0, next & 1); + + // We don't care about any memory associated w/ setting the `head` + // field, just the current value. + let mut curr = self.head.load(Relaxed); + + loop { + *task.as_ref().queue_next.get() = (next | curr as usize) as *const _; + + let res = + self.head + .compare_exchange(curr, task.as_ptr() as *mut _, Release, Relaxed); + + match res { + Ok(_) => return, + Err(actual) => { + curr = actual; + } + } + } + } + } + + pub(crate) fn drain(&self) -> impl Iterator> { + struct Iter(*mut Header); + + impl Iterator for Iter { + type Item = Task; + + fn next(&mut self) -> Option> { + let task = NonNull::new(self.0)?; + + unsafe { + let next = *task.as_ref().queue_next.get() as usize; + + // remove the data bit + self.0 = (next & !1) as *mut _; + + Some(Task::from_raw(task)) + } + } + } + + impl Drop for Iter { + fn drop(&mut self) { + use std::process; + + if !self.0.is_null() { + // we have bugs + process::abort(); + } + } + } + + let ptr = self.head.swap(ptr::null_mut(), Acquire); + Iter(ptr) + } +} diff --git a/tokio-executor/src/task/state.rs b/tokio-executor/src/task/state.rs new file mode 100644 index 00000000000..4c4bbbb78b1 --- /dev/null +++ b/tokio-executor/src/task/state.rs @@ -0,0 +1,502 @@ +use crate::loom::sync::atomic::AtomicUsize; + +use std::fmt; +use std::sync::atomic::Ordering::{AcqRel, Acquire, Release}; +use std::usize; + +pub(super) struct State { + val: AtomicUsize, +} + +/// Current state value +#[derive(Copy, Clone)] +pub(super) struct Snapshot(usize); + +/// The task is currently being run. +const RUNNING: usize = 0b00_0001; + +/// The task has been notified by a waker. +const NOTIFIED: usize = 0b00_0010; + +/// The task is complete. +/// +/// Once this bit is set, it is never unset +const COMPLETE: usize = 0b00_0100; + +/// The primary task handle has been dropped. +const RELEASED: usize = 0b00_1000; + +/// The join handle is still around +const JOIN_INTEREST: usize = 0b01_0000; + +/// A join handle waker has been set +const JOIN_WAKER: usize = 0b10_0000; + +/// The task has been forcibly canceled. +const CANCELLED: usize = 0b100_0000; + +/// All bits +const LIFECYCLE_MASK: usize = + RUNNING | NOTIFIED | COMPLETE | RELEASED | JOIN_INTEREST | JOIN_WAKER | CANCELLED; + +/// Bits used by the waker ref count portion of the state. +/// +/// Ref counts only cover **wakers**. Other handles are tracked with other state +/// bits. +const WAKER_COUNT_MASK: usize = usize::MAX - LIFECYCLE_MASK; + +/// Number of positions to shift the ref count +const WAKER_COUNT_SHIFT: usize = WAKER_COUNT_MASK.count_zeros() as usize; + +/// One ref count +const WAKER_ONE: usize = 1 << WAKER_COUNT_SHIFT; + +/// Initial state +const INITIAL_STATE: usize = NOTIFIED; + +/// All transitions are performed via RMW operations. This establishes an +/// unambiguous modification order. +impl State { + /// Starts with a ref count of 1 + pub(super) fn new_background() -> State { + State { + val: AtomicUsize::new(INITIAL_STATE), + } + } + + /// Starts with a ref count of 2 + pub(super) fn new_joinable() -> State { + State { + val: AtomicUsize::new(INITIAL_STATE | JOIN_INTEREST), + } + } + + /// Load the current state, establishes `Acquire` ordering. + pub(super) fn load(&self) -> Snapshot { + Snapshot(self.val.load(Acquire)) + } + + /// Transition a task to the `Running` state. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn transition_to_running(&self) -> Snapshot { + const DELTA: usize = RUNNING | NOTIFIED; + + let prev = Snapshot(self.val.fetch_xor(DELTA, Acquire)); + debug_assert!(prev.is_notified()); + + if prev.is_running() { + // We were signalled to cancel + // + // Apply the state + let prev = self.val.fetch_or(CANCELLED, AcqRel); + return Snapshot(prev | CANCELLED); + } + + debug_assert!(!prev.is_running()); + + let next = Snapshot(prev.0 ^ DELTA); + + debug_assert!(next.is_running()); + debug_assert!(!next.is_notified()); + + next + } + + /// Transition the task from `Running` -> `Idle`. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn transition_to_idle(&self) -> Snapshot { + const DELTA: usize = RUNNING; + + let prev = Snapshot(self.val.fetch_xor(DELTA, AcqRel)); + + if !prev.is_running() { + // We were signaled to cancel. + // + // Apply the state + let prev = self.val.fetch_or(CANCELLED, AcqRel); + return Snapshot(prev | CANCELLED); + } + + let next = Snapshot(prev.0 ^ DELTA); + + debug_assert!(!next.is_running()); + + next + } + + /// Transition the task from `Running` -> `Complete`. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn transition_to_complete(&self) -> Snapshot { + const DELTA: usize = RUNNING | COMPLETE; + + let prev = Snapshot(self.val.fetch_xor(DELTA, AcqRel)); + + debug_assert!(!prev.is_complete()); + + let next = Snapshot(prev.0 ^ DELTA); + + debug_assert!(next.is_complete()); + + next + } + + /// Transition the task from `Running` -> `Released`. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn transition_to_released(&self) -> Snapshot { + const DELTA: usize = RUNNING | COMPLETE | RELEASED; + + let prev = Snapshot(self.val.fetch_xor(DELTA, AcqRel)); + + debug_assert!(prev.is_running()); + debug_assert!(!prev.is_complete()); + debug_assert!(!prev.is_released()); + + let next = Snapshot(prev.0 ^ DELTA); + + debug_assert!(!next.is_running()); + debug_assert!(next.is_complete()); + debug_assert!(next.is_released()); + + next + } + + /// Transition the task to the canceled state. + /// + /// Returns the snapshot of the state **after** the transition **if** the + /// transition was made successfully + /// + /// # States + /// + /// - Notifed: task may be in a queue, caller must not release. + /// - Running: cannot drop. The poll handle will handle releasing. + /// - Other prior states do not require cancellation. + /// + /// If the task has been notified, then it may still be in a queue. The + /// caller must not release the task. + pub(super) fn transition_to_canceled_from_queue(&self) -> Snapshot { + let prev = Snapshot(self.val.fetch_or(CANCELLED, AcqRel)); + + debug_assert!(!prev.is_complete()); + debug_assert!(!prev.is_running() || prev.is_notified()); + + Snapshot(prev.0 | CANCELLED) + } + + pub(super) fn transition_to_canceled_from_list(&self) -> Option { + let mut prev = self.load(); + + loop { + if !prev.is_active() { + return None; + } + + let mut next = prev; + + // Use the running flag to signal cancellation + if prev.is_running() { + next.0 -= RUNNING; + } else if prev.is_notified() { + next.0 += RUNNING; + } else { + next.0 |= CANCELLED; + } + + let res = self.val.compare_exchange(prev.0, next.0, AcqRel, Acquire); + + match res { + Ok(_) if next.is_canceled() => return Some(next), + Ok(_) => return None, + Err(actual) => { + prev = Snapshot(actual); + } + } + } + } + + /// Final transition to `Released`. Called when primary task handle is + /// dropped. This is roughly a "ref decrement" operation. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn release_task(&self) -> Snapshot { + use crate::loom::sync::atomic; + + const DELTA: usize = RELEASED; + + let prev = Snapshot(self.val.fetch_or(DELTA, Release)); + + debug_assert!(!prev.is_released()); + debug_assert!(prev.is_terminal(), "state = {:?}", prev); + + let next = Snapshot(prev.0 | DELTA); + + debug_assert!(next.is_released()); + + if next.is_final_ref() || (next.has_join_waker() && !next.is_join_interested()) { + // The final reference to the task was dropped, the caller must free the + // memory. Establish an acquire ordering. + atomic::fence(Acquire); + } + + next + } + + /// Transition the state to `Scheduled`. + /// + /// Returns `true` if the task needs to be submitted to the pool for + /// execution + pub(super) fn transition_to_notified(&self) -> bool { + const MASK: usize = RUNNING | NOTIFIED | COMPLETE | CANCELLED; + + let prev = self.val.fetch_or(NOTIFIED, Release); + prev & MASK == 0 + } + + /// Optimistically try to swap the state assuming the join handle is + /// __immediately__ dropped on spawn + pub(super) fn drop_join_handle_fast(&self) -> bool { + use std::sync::atomic::Ordering::Relaxed; + + // Relaxed is acceptable as if this function is called and succeeds, + // then nothing has been done w/ the join handle. + // + // The moment the join handle is used (polled), the `JOIN_WAKER` flag is + // set, at which point the CAS will fail. + // + // Given this, there is no risk if this operation is reordered. + self.val + .compare_exchange_weak( + INITIAL_STATE | JOIN_INTEREST, + INITIAL_STATE, + Relaxed, + Relaxed, + ) + .is_ok() + } + + /// The join handle has completed by reading the output + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn complete_join_handle(&self) -> Snapshot { + use crate::loom::sync::atomic; + + const DELTA: usize = JOIN_INTEREST; + + let prev = Snapshot(self.val.fetch_sub(DELTA, Release)); + + debug_assert!(prev.is_join_interested()); + + let next = Snapshot(prev.0 - DELTA); + + if !next.is_final_ref() { + return next; + } + + atomic::fence(Acquire); + + next + } + + /// The join handle is being dropped, this fails if the task has been + /// completed and the output must be dropped first then + /// `complete_join_handle` should be called. + /// + /// Returns a snapshot of the state **after** the transition. + pub(super) fn drop_join_handle_slow(&self) -> Result { + const MASK: usize = COMPLETE | CANCELLED; + + let mut prev = self.val.load(Acquire); + + loop { + // Once the complete bit is set, it is never unset. + if prev & MASK != 0 { + return Err(Snapshot(prev)); + } + + debug_assert!(prev & JOIN_INTEREST == JOIN_INTEREST); + + let next = (prev - JOIN_INTEREST) & !JOIN_WAKER; + + let res = self.val.compare_exchange(prev, next, AcqRel, Acquire); + + match res { + Ok(_) => { + return Ok(Snapshot(next)); + } + Err(actual) => { + prev = actual; + } + } + } + } + + /// Store the join waker. + pub(super) fn store_join_waker(&self) -> Snapshot { + use crate::loom::sync::atomic; + + const DELTA: usize = JOIN_WAKER; + + let prev = Snapshot(self.val.fetch_xor(DELTA, Release)); + + debug_assert!(!prev.has_join_waker()); + + let next = Snapshot(prev.0 ^ DELTA); + + debug_assert!(next.has_join_waker()); + + if next.is_complete() { + atomic::fence(Acquire); + } + + next + } + + pub(super) fn unset_waker(&self) -> Snapshot { + const MASK: usize = COMPLETE | CANCELLED; + + let mut prev = self.val.load(Acquire); + + loop { + // Once the `COMPLETE` bit is set, it is never unset + if prev & MASK != 0 { + return Snapshot(prev); + } + + debug_assert!(Snapshot(prev).has_join_waker()); + + let next = prev - JOIN_WAKER; + + let res = self.val.compare_exchange(prev, next, AcqRel, Acquire); + + match res { + Ok(_) => return Snapshot(next), + Err(actual) => { + prev = actual; + } + } + } + } + + pub(super) fn ref_inc(&self) { + use std::process; + use std::sync::atomic::Ordering::Relaxed; + + // Using a relaxed ordering is alright here, as knowledge of the + // original reference prevents other threads from erroneously deleting + // the object. + // + // As explained in the [Boost documentation][1], Increasing the + // reference counter can always be done with memory_order_relaxed: New + // references to an object can only be formed from an existing + // reference, and passing an existing reference from one thread to + // another must already provide any required synchronization. + // + // [1]: (www.boost.org/doc/libs/1_55_0/doc/html/atomic/usage_examples.html) + let prev = self.val.fetch_add(WAKER_ONE, Relaxed); + + // If the reference count overflowed, abort. + if prev > isize::max_value() as usize { + process::abort(); + } + } + + /// Returns `true` if the task should be released. + pub(super) fn ref_dec(&self) -> bool { + use crate::loom::sync::atomic; + + let prev = self.val.fetch_sub(WAKER_ONE, Release); + let next = Snapshot(prev - WAKER_ONE); + + if next.is_final_ref() { + atomic::fence(Acquire); + } + + next.is_final_ref() + } +} + +impl Snapshot { + pub(super) fn is_running(self) -> bool { + self.0 & RUNNING == RUNNING + } + + pub(super) fn is_notified(self) -> bool { + self.0 & NOTIFIED == NOTIFIED + } + + pub(super) fn is_released(self) -> bool { + self.0 & RELEASED == RELEASED + } + + pub(super) fn is_complete(self) -> bool { + self.0 & COMPLETE == COMPLETE + } + + pub(super) fn is_canceled(self) -> bool { + self.0 & CANCELLED == CANCELLED + } + + /// Used during normal runtime. + pub(super) fn is_active(self) -> bool { + self.0 & (COMPLETE | CANCELLED) == 0 + } + + /// Used before dropping the task + pub(super) fn is_terminal(self) -> bool { + // When both the notified & running flags are set, the task was canceled + // after being notified, before it was run. + // + // There is a race where: + // - The task state transitions to notified + // - The global queue is shutdown + // - The waker attempts to push into the global queue and fails. + // - The waker holds the last reference to the task, thus drops it. + // + // In this scenario, the cancelled bit will never get set. + !self.is_active() || (self.is_notified() && self.is_running()) + } + + pub(super) fn is_join_interested(self) -> bool { + self.0 & JOIN_INTEREST == JOIN_INTEREST + } + + pub(super) fn has_join_waker(self) -> bool { + self.0 & JOIN_WAKER == JOIN_WAKER + } + + pub(super) fn is_final_ref(self) -> bool { + const MASK: usize = WAKER_COUNT_MASK | RELEASED | JOIN_INTEREST; + + (self.0 & MASK) == RELEASED + } +} + +impl fmt::Debug for State { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + use std::sync::atomic::Ordering::SeqCst; + + let snapshot = Snapshot(self.val.load(SeqCst)); + + fmt.debug_struct("State") + .field("snapshot", &snapshot) + .finish() + } +} + +impl fmt::Debug for Snapshot { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Snapshot") + .field("is_running", &self.is_running()) + .field("is_notified", &self.is_notified()) + .field("is_released", &self.is_released()) + .field("is_complete", &self.is_complete()) + .field("is_canceled", &self.is_canceled()) + .field("is_join_interested", &self.is_join_interested()) + .field("has_join_waker", &self.has_join_waker()) + .field("is_final_ref", &self.is_final_ref()) + .finish() + } +} diff --git a/tokio-executor/src/task/tests/loom.rs b/tokio-executor/src/task/tests/loom.rs new file mode 100644 index 00000000000..dc2ae942c87 --- /dev/null +++ b/tokio-executor/src/task/tests/loom.rs @@ -0,0 +1,277 @@ +use crate::task; +use crate::tests::loom_schedule::LoomSchedule; + +use tokio_test::{assert_err, assert_ok}; + +use loom::future::block_on; +use loom::sync::atomic::AtomicBool; +use loom::sync::atomic::Ordering::{Acquire, Release}; +use loom::thread; +use std::future::Future; + +#[test] +fn create_drop_join_handle() { + loom::model(|| { + let (task, join_handle) = task::joinable(async { "hello" }); + + let schedule = LoomSchedule::new(); + let schedule = From::from(&schedule); + + let th = thread::spawn(move || { + drop(join_handle); + }); + + assert_none!(task.run(schedule)); + + th.join().unwrap(); + }); +} + +#[test] +fn poll_drop_handle_then_drop() { + use futures_util::future::poll_fn; + use std::pin::Pin; + use std::task::Poll; + + loom::model(|| { + let (task, mut join_handle) = task::joinable(async { "hello" }); + + let schedule = LoomSchedule::new(); + let schedule = From::from(&schedule); + + let th = thread::spawn(move || { + block_on(poll_fn(|cx| { + let _ = Pin::new(&mut join_handle).poll(cx); + Poll::Ready(()) + })); + }); + + assert_none!(task.run(schedule)); + + th.join().unwrap(); + }); +} + +#[test] +fn join_output() { + loom::model(|| { + let (task, join_handle) = task::joinable(async { "hello world" }); + + let schedule = LoomSchedule::new(); + let schedule = From::from(&schedule); + + let th = thread::spawn(move || { + let out = assert_ok!(block_on(join_handle)); + assert_eq!("hello world", out); + }); + + assert_none!(task.run(schedule)); + th.join().unwrap(); + }); +} + +#[test] +fn wake_by_ref() { + loom::model(|| { + let (task, join_handle) = task::joinable(gated(2, true, false)); + + let schedule = LoomSchedule::new(); + let schedule = &schedule; + schedule.push_task(task); + + let th = join_one_task(join_handle); + + work(schedule); + + assert_ok!(th.join().unwrap()); + }); +} + +#[test] +fn wake_by_val() { + loom::model(|| { + let (task, join_handle) = task::joinable(gated(2, true, true)); + + let schedule = LoomSchedule::new(); + let schedule = &schedule; + schedule.push_task(task); + + let th = join_one_task(join_handle); + + work(schedule); + + assert_ok!(th.join().unwrap()); + }); +} + +#[test] +fn release_remote() { + loom::model(|| { + let (task, join_handle) = task::joinable(gated(1, false, true)); + + let s1 = LoomSchedule::new(); + let s2 = LoomSchedule::new(); + + // Join handle + let th = join_one_task(join_handle); + + let task = match task.run(From::from(&s1)) { + Some(task) => task, + None => s1.recv().expect("released!"), + }; + + assert_none!(task.run(From::from(&s2))); + assert_none!(s1.recv()); + + assert_ok!(th.join().unwrap()); + }); +} + +#[test] +fn shutdown_task_before_poll() { + loom::model(|| { + let (task, join_handle) = task::joinable::<_, LoomSchedule>(async { "hello" }); + + let th = join_one_task(join_handle); + task.shutdown(); + + assert_err!(th.join().unwrap()); + }); +} + +#[test] +fn shutdown_from_list_after_poll() { + loom::model(|| { + let (task, join_handle) = task::joinable(gated(1, false, false)); + + let s1 = LoomSchedule::new(); + + let mut list = task::OwnedList::new(); + list.insert(&task); + + // Join handle + let th = join_two_tasks(join_handle); + + match task.run(From::from(&s1)) { + Some(task) => { + // always drain the list before calling shutdown on tasks + list.shutdown(); + + // The task was scheduled, drain it explicitly. + task.shutdown(); + } + None => { + list.shutdown(); + } + }; + + match s1.recv() { + Some(task) => task.shutdown(), + None => {} + } + + assert_err!(th.join().unwrap()); + }); +} + +#[test] +fn shutdown_from_queue_after_poll() { + loom::model(|| { + let (task, join_handle) = task::joinable(gated(1, false, false)); + + let s1 = LoomSchedule::new(); + + // Join handle + let th = join_two_tasks(join_handle); + + let task = match task.run(From::from(&s1)) { + Some(task) => task, + None => assert_some!(s1.recv()), + }; + + task.shutdown(); + + assert_err!(th.join().unwrap()); + }); +} + +fn gated(n: usize, complete_first_poll: bool, by_val: bool) -> impl Future { + use futures_util::future::poll_fn; + use std::sync::Arc; + use std::task::Poll; + + let gate = Arc::new(AtomicBool::new(false)); + let mut fired = false; + + poll_fn(move |cx| { + if !fired { + for _ in 0..n { + let gate = gate.clone(); + let waker = cx.waker().clone(); + thread::spawn(move || { + gate.store(true, Release); + + if by_val { + waker.wake() + } else { + waker.wake_by_ref(); + } + }); + } + + fired = true; + + if !complete_first_poll { + return Poll::Pending; + } + } + + if gate.load(Acquire) { + Poll::Ready("hello world") + } else { + Poll::Pending + } + }) +} + +fn work(schedule: &LoomSchedule) { + while let Some(task) = schedule.recv() { + let mut task = Some(task); + + while let Some(t) = task.take() { + task = t.run(From::from(schedule)); + } + } +} + +/// Spawn a thread to wait on the join handle. Uses a single task. +fn join_one_task(join_handle: T) -> loom::thread::JoinHandle { + thread::spawn(move || block_on(join_handle)) +} + +/// Spawn a thread to wait on the join handle using two tasks. First, poll the +/// join handle on the first task. If the join handle is not ready, then use a +/// second task to wait on it. +fn join_two_tasks( + join_handle: T, +) -> loom::thread::JoinHandle { + use futures_util::future::poll_fn; + use std::task::Poll; + + // Join handle + thread::spawn(move || { + let mut join_handle = Some(join_handle); + block_on(poll_fn(move |cx| { + use std::pin::Pin; + + let res = Pin::new(join_handle.as_mut().unwrap()).poll(cx); + + if res.is_ready() { + return res; + } + + // Yes, we are nesting + Poll::Ready(block_on(join_handle.take().unwrap())) + })) + }) +} diff --git a/tokio-executor/src/task/tests/mod.rs b/tokio-executor/src/task/tests/mod.rs new file mode 100644 index 00000000000..526e1e92162 --- /dev/null +++ b/tokio-executor/src/task/tests/mod.rs @@ -0,0 +1,5 @@ +#[cfg(loom)] +mod loom; + +#[cfg(not(loom))] +mod task; diff --git a/tokio-executor/src/task/tests/task.rs b/tokio-executor/src/task/tests/task.rs new file mode 100644 index 00000000000..6f6e0ede935 --- /dev/null +++ b/tokio-executor/src/task/tests/task.rs @@ -0,0 +1,644 @@ +use crate::task::{self, Header}; + +use crate::tests::backoff::*; +use crate::tests::mock_schedule::{mock, Mock}; +use crate::tests::track_drop::track_drop; + +use tokio::sync::oneshot; +use tokio_test::task::spawn; +use tokio_test::{assert_pending, assert_ready_err, assert_ready_ok}; + +use futures_util::future::poll_fn; +use std::sync::mpsc; + +#[test] +fn header_lte_cache_line() { + use std::mem::size_of; + + assert!(size_of::>() <= 8 * size_of::<*const ()>()); +} + +#[test] +fn create_complete_drop() { + let (tx, rx) = mpsc::channel(); + + let (task, did_drop) = track_drop(async move { + tx.send(1).unwrap(); + }); + + let task = task::background(task); + + let mock = mock().bind(&task).release_local(); + let mock = From::from(&mock); + + // Nothing is returned + assert!(task.run(mock).is_none()); + + // The message was sent + assert!(rx.try_recv().is_ok()); + + // The future & output were dropped. + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn create_yield_complete_drop() { + let (tx, rx) = mpsc::channel(); + + let (task, did_drop) = track_drop(async move { + backoff(1).await; + tx.send(1).unwrap(); + }); + + let task = task::background(task); + + let mock = mock().bind(&task).release_local(); + let mock = From::from(&mock); + + // Task is returned + let task = assert_some!(task.run(mock)); + + // The future was **not** dropped. + assert!(!did_drop.did_drop_future()); + + assert_none!(task.run(mock)); + + // The message was sent + assert!(rx.try_recv().is_ok()); + + // The future was dropped. + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn create_clone_yield_complete_drop() { + let (tx, rx) = mpsc::channel(); + + let (task, did_drop) = track_drop(async move { + backoff_clone(1).await; + tx.send(1).unwrap(); + }); + + let task = task::background(task); + + let mock = mock().bind(&task).release_local(); + let mock = From::from(&mock); + + // Task is returned + let task = assert_some!(task.run(mock)); + + // The future was **not** dropped. + assert!(!did_drop.did_drop_future()); + + assert_none!(task.run(mock)); + + // The message was sent + assert!(rx.try_recv().is_ok()); + + // The future was dropped. + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn create_wake_drop() { + let (tx, rx) = oneshot::channel(); + + let (task, did_drop) = track_drop(async move { rx.await }); + + let task = task::background(task); + + let mock = mock().bind(&task).schedule().release_local(); + + assert_none!(task.run(From::from(&mock))); + assert_none!(mock.next_pending_run()); + + // The future was **not** dropped. + assert!(!did_drop.did_drop_future()); + + tx.send("hello").unwrap(); + + let task = assert_some!(mock.next_pending_run()); + + assert_none!(task.run(From::from(&mock))); + + // The future was dropped. + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn notify_complete() { + use std::task::Poll::Ready; + + let (task, did_drop) = track_drop(async move { + poll_fn(|cx| { + cx.waker().wake_by_ref(); + Ready(()) + }) + .await; + }); + + let task = task::background(task); + + let mock = mock().bind(&task).release_local(); + let mock = From::from(&mock); + + assert_none!(task.run(mock)); + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn complete_on_second_schedule_obj() { + let (tx, rx) = mpsc::channel(); + + let (task, did_drop) = track_drop(async move { + backoff(1).await; + tx.send(1).unwrap(); + }); + + let task = task::background(task); + + let mock1 = mock(); + let mock2 = mock().bind(&task).release(); + + // Task is returned + let task = assert_some!(task.run(From::from(&mock2))); + + assert_none!(task.run(From::from(&mock1))); + + // The message was sent + assert!(rx.try_recv().is_ok()); + + // The future was dropped. + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); + + let _ = assert_some!(mock2.next_pending_drop()); +} + +#[test] +fn join_task_immediate_drop_handle() { + let (task, did_drop) = track_drop(async move { "hello".to_string() }); + + let (task, _) = task::joinable(task); + + let mock = mock().bind(&task).release_local(); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn join_task_immediate_complete_1() { + let (task, did_drop) = track_drop(async move { "hello".to_string() }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock = mock().bind(&task).release_local(); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + assert!(!handle.is_woken()); + + let out = assert_ready_ok!(handle.poll()); + assert_eq!(out.get_ref(), "hello"); + + drop(out); + + assert!(did_drop.did_drop_output()); +} + +#[test] +fn join_task_immediate_complete_2() { + let (task, did_drop) = track_drop(async move { "hello".to_string() }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock = mock().bind(&task).release_local(); + + assert_pending!(handle.poll()); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + assert!(handle.is_woken()); + + let out = assert_ready_ok!(handle.poll()); + assert_eq!(out.get_ref(), "hello"); + + drop(out); + + assert!(did_drop.did_drop_output()); +} + +#[test] +fn join_task_complete_later() { + let (task, did_drop) = track_drop(async move { + backoff(1).await; + "hello".to_string() + }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(async { handle.await }); + + let mock = mock().bind(&task).release_local(); + + let task = assert_some!(task.run(From::from(&mock))); + + assert!(!did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + + assert_pending!(handle.poll()); + + assert_none!(task.run(From::from(&mock))); + assert!(handle.is_woken()); + + let out = assert_ready_ok!(handle.poll()); + assert_eq!(out.get_ref(), "hello"); + + drop(out); + + assert!(did_drop.did_drop_output()); + + assert_eq!(1, handle.waker_ref_count()); +} + +#[test] +fn drop_join_after_poll() { + let (task, did_drop) = track_drop(async move { + backoff(1).await; + "hello".to_string() + }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(async { handle.await }); + + let mock = mock().bind(&task).release_local(); + + assert_pending!(handle.poll()); + drop(handle); + + let task = assert_some!(task.run(From::from(&mock))); + + assert!(!did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + + assert_none!(task.run(From::from(&mock))); + + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +fn join_handle_change_task_complete() { + use std::future::Future; + use std::pin::Pin; + + let (task, did_drop) = track_drop(async move { + backoff(1).await; + "hello".to_string() + }); + + let (task, mut handle) = task::joinable(task); + let mut t1 = spawn(poll_fn(|cx| Pin::new(&mut handle).poll(cx))); + + let mock = mock().bind(&task).release_local(); + + assert_pending!(t1.poll()); + drop(t1); + + let task = assert_some!(task.run(From::from(&mock))); + + let mut t2 = spawn(poll_fn(|cx| Pin::new(&mut handle).poll(cx))); + assert_pending!(t2.poll()); + + assert!(!did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + + assert_none!(task.run(From::from(&mock))); + + assert!(t2.is_woken()); + + let out = assert_ready_ok!(t2.poll()); + assert_eq!(out.get_ref(), "hello"); + + drop(out); + + assert!(did_drop.did_drop_output()); + + assert_eq!(1, t2.waker_ref_count()); +} + +#[test] +fn drop_handle_after_complete() { + let (task, did_drop) = track_drop(async move { "hello".to_string() }); + + let (task, handle) = task::joinable(task); + + let mock = mock().bind(&task).release_local(); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + + drop(handle); + + assert!(did_drop.did_drop_output()); +} + +#[test] +fn non_initial_task_state_drop_join_handle_without_polling() { + let (tx, rx) = oneshot::channel::<()>(); + + let (task, did_drop) = track_drop(async move { + rx.await.unwrap(); + "hello".to_string() + }); + + let (task, handle) = task::joinable(task); + + let mock = mock().bind(&task).schedule().release_local(); + + assert_none!(task.run(From::from(&mock))); + + drop(handle); + + assert!(!did_drop.did_drop_future()); + assert!(!did_drop.did_drop_output()); + + tx.send(()).unwrap(); + let task = assert_some!(mock.next_pending_run()); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); +} + +#[test] +#[cfg(not(miri))] +fn task_panic_background() { + let (task, did_drop) = track_drop(async move { + if true { + panic!() + } + "hello" + }); + + let task = task::background(task); + + let mock = mock().bind(&task).release_local(); + + assert!(task.run(From::from(&mock)).is_none()); + + assert!(did_drop.did_drop_future()); +} + +#[test] +#[cfg(not(miri))] +fn task_panic_join() { + let (task, did_drop) = track_drop(async move { + if true { + panic!() + } + "hello" + }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock = mock().bind(&task).release_local(); + + assert_pending!(handle.poll()); + + assert!(task.run(From::from(&mock)).is_none()); + assert!(did_drop.did_drop_future()); + assert!(handle.is_woken()); + + assert_ready_err!(handle.poll()); +} + +#[test] +fn complete_second_schedule_obj_before_join() { + let (tx, rx) = oneshot::channel(); + + let (task, did_drop) = track_drop(async move { rx.await.unwrap() }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock1 = mock(); + let mock2 = mock().bind(&task).schedule().release(); + + assert_pending!(handle.poll()); + + assert_none!(task.run(From::from(&mock2))); + + tx.send("hello").unwrap(); + + let task = assert_some!(mock2.next_pending_run()); + assert_none!(task.run(From::from(&mock1))); + assert!(did_drop.did_drop_future()); + + // The join handle was notified + assert!(handle.is_woken()); + + // Drop the task + let _ = assert_some!(mock2.next_pending_drop()); + + // Get the output + let out = assert_ready_ok!(handle.poll()); + assert_eq!(*out.get_ref(), "hello"); +} + +#[test] +fn complete_second_schedule_obj_after_join() { + let (tx, rx) = oneshot::channel(); + + let (task, did_drop) = track_drop(async move { rx.await.unwrap() }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock1 = mock(); + let mock2 = mock().bind(&task).schedule().release(); + + assert_pending!(handle.poll()); + + assert_none!(task.run(From::from(&mock2))); + + tx.send("hello").unwrap(); + + let task = assert_some!(mock2.next_pending_run()); + assert_none!(task.run(From::from(&mock1))); + assert!(did_drop.did_drop_future()); + + // The join handle was notified + assert!(handle.is_woken()); + + // Get the output + let out = assert_ready_ok!(handle.poll()); + assert_eq!(*out.get_ref(), "hello"); + + // Drop the task + let _ = assert_some!(mock2.next_pending_drop()); + + assert_eq!(1, handle.waker_ref_count()); +} + +#[test] +fn shutdown_from_list_before_notified() { + let (tx, rx) = oneshot::channel::<()>(); + let mut list = task::OwnedList::new(); + + let (task, did_drop) = track_drop(async move { rx.await }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + list.insert(&task); + + let mock = mock().bind(&task).release(); + + assert_pending!(handle.poll()); + assert_none!(task.run(From::from(&mock))); + + list.shutdown(); + assert!(did_drop.did_drop_future()); + + assert!(handle.is_woken()); + + let task = assert_some!(mock.next_pending_drop()); + drop(task); + + assert_ready_err!(handle.poll()); + + drop(tx); +} + +#[test] +fn shutdown_from_list_after_notified() { + let (tx, rx) = oneshot::channel::<()>(); + let mut list = task::OwnedList::new(); + + let (task, did_drop) = track_drop(async move { rx.await }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + list.insert(&task); + + let mock = mock().bind(&task).schedule().release(); + + assert_pending!(handle.poll()); + assert_none!(task.run(From::from(&mock))); + + tx.send(()).unwrap(); + + let task = assert_some!(mock.next_pending_run()); + + list.shutdown(); + + assert_none!(mock.next_pending_drop()); + + assert_none!(task.run(From::from(&mock))); + assert!(did_drop.did_drop_future()); + assert!(handle.is_woken()); + + let task = assert_some!(mock.next_pending_drop()); + drop(task); + + assert_ready_err!(handle.poll()); +} + +#[test] +fn shutdown_from_list_after_complete() { + let mut list = task::OwnedList::new(); + + let (task, did_drop) = track_drop(async move { + backoff(1).await; + "hello" + }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + list.insert(&task); + + let m1 = mock().bind(&task).release(); + let m2 = mock(); + + assert_pending!(handle.poll()); + let task = assert_some!(task.run(From::from(&m1))); + assert_none!(task.run(From::from(&m2))); + assert!(did_drop.did_drop_future()); + assert!(handle.is_woken()); + + list.shutdown(); + + let task = assert_some!(m1.next_pending_drop()); + drop(task); + + let out = assert_ready_ok!(handle.poll()); + assert_eq!(*out.get_ref(), "hello"); +} + +#[test] +fn shutdown_from_task_before_notified() { + let (tx, rx) = oneshot::channel::<()>(); + + let (task, did_drop) = track_drop(async move { rx.await }); + + let (task, handle) = task::joinable::<_, Mock>(task); + let mut handle = spawn(handle); + + assert_pending!(handle.poll()); + + task.shutdown(); + assert!(did_drop.did_drop_future()); + assert!(handle.is_woken()); + + assert_ready_err!(handle.poll()); + + drop(tx); +} + +#[test] +fn shutdown_from_task_after_notified() { + let (tx, rx) = oneshot::channel::<()>(); + + let (task, did_drop) = track_drop(async move { rx.await }); + + let (task, handle) = task::joinable(task); + let mut handle = spawn(handle); + + let mock = mock().bind(&task).schedule().release(); + + assert_pending!(handle.poll()); + assert_none!(task.run(From::from(&mock))); + + tx.send(()).unwrap(); + + let task = assert_some!(mock.next_pending_run()); + + task.shutdown(); + assert!(did_drop.did_drop_future()); + assert!(handle.is_woken()); + + let task = assert_some!(mock.next_pending_drop()); + drop(task); + + assert_ready_err!(handle.poll()); +} diff --git a/tokio-executor/src/task/waker.rs b/tokio-executor/src/task/waker.rs new file mode 100644 index 00000000000..c47f1366559 --- /dev/null +++ b/tokio-executor/src/task/waker.rs @@ -0,0 +1,107 @@ +use crate::task::harness::Harness; +use crate::task::{Header, Schedule}; + +use std::future::Future; +use std::marker::PhantomData; +use std::ops; +use std::task::{RawWaker, RawWakerVTable, Waker}; + +pub(super) struct WakerRef<'a, S: 'static> { + waker: Waker, + _p: PhantomData<&'a Header>, +} + +/// Returns a `WakerRef` which avoids having to pre-emptively increase the +/// refcount if there is no need to do so. +pub(super) fn waker_ref(meta: &Header) -> WakerRef<'_, S> +where + T: Future, + S: Schedule, +{ + let ptr = meta as *const _ as *const (); + + let vtable = &RawWakerVTable::new( + clone_waker::, + wake_unreachable, + wake_by_local_ref::, + noop, + ); + + let waker = unsafe { Waker::from_raw(RawWaker::new(ptr, vtable)) }; + + WakerRef { + waker, + _p: PhantomData, + } +} + +impl ops::Deref for WakerRef<'_, S> { + type Target = Waker; + + fn deref(&self) -> &Waker { + &self.waker + } +} + +unsafe fn clone_waker(ptr: *const ()) -> RawWaker +where + T: Future, + S: Schedule, +{ + let meta = ptr as *const Header; + (*meta).state.ref_inc(); + + let vtable = &RawWakerVTable::new( + clone_waker::, + wake_by_val::, + wake_by_ref::, + drop_waker::, + ); + + RawWaker::new(ptr, vtable) +} + +unsafe fn drop_waker(ptr: *const ()) +where + T: Future, + S: Schedule, +{ + let harness = Harness::::from_raw(ptr as *mut _); + harness.drop_waker(); +} + +// `wake()` cannot be called on the ref variaant. +unsafe fn wake_unreachable(_data: *const ()) { + unreachable!(); +} + +unsafe fn wake_by_val(ptr: *const ()) +where + T: Future, + S: Schedule, +{ + let harness = Harness::::from_raw(ptr as *mut _); + harness.wake_by_val(); +} + +// This function can only be called when on the runtime. +unsafe fn wake_by_local_ref(ptr: *const ()) +where + T: Future, + S: Schedule, +{ + let harness = Harness::::from_raw(ptr as *mut _); + harness.wake_by_local_ref(); +} + +// Wake without consuming the waker +unsafe fn wake_by_ref(ptr: *const ()) +where + T: Future, + S: Schedule, +{ + let harness = Harness::::from_raw(ptr as *mut _); + harness.wake_by_ref(); +} + +unsafe fn noop(_ptr: *const ()) {} diff --git a/tokio-executor/src/tests/backoff.rs b/tokio-executor/src/tests/backoff.rs new file mode 100644 index 00000000000..358ab2dad73 --- /dev/null +++ b/tokio-executor/src/tests/backoff.rs @@ -0,0 +1,32 @@ +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +pub(crate) struct Backoff(usize, bool); + +pub(crate) fn backoff(n: usize) -> impl Future { + Backoff(n, false) +} + +/// Back off, but clone the waker each time +pub(crate) fn backoff_clone(n: usize) -> impl Future { + Backoff(n, true) +} + +impl Future for Backoff { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + if self.0 == 0 { + return Poll::Ready(()); + } + + self.0 -= 1; + if self.1 { + cx.waker().clone().wake(); + } else { + cx.waker().wake_by_ref(); + } + Poll::Pending + } +} diff --git a/tokio-executor/src/tests/loom_oneshot.rs b/tokio-executor/src/tests/loom_oneshot.rs new file mode 100644 index 00000000000..c126fe479af --- /dev/null +++ b/tokio-executor/src/tests/loom_oneshot.rs @@ -0,0 +1,49 @@ +use loom::sync::Notify; + +use std::sync::{Arc, Mutex}; + +pub(crate) fn channel() -> (Sender, Receiver) { + let inner = Arc::new(Inner { + notify: Notify::new(), + value: Mutex::new(None), + }); + + let tx = Sender { + inner: inner.clone(), + }; + let rx = Receiver { inner }; + + (tx, rx) +} + +pub(crate) struct Sender { + inner: Arc>, +} + +pub(crate) struct Receiver { + inner: Arc>, +} + +struct Inner { + notify: Notify, + value: Mutex>, +} + +impl Sender { + pub(crate) fn send(self, value: T) { + *self.inner.value.lock().unwrap() = Some(value); + self.inner.notify.notify(); + } +} + +impl Receiver { + pub(crate) fn recv(self) -> T { + loop { + if let Some(v) = self.inner.value.lock().unwrap().take() { + return v; + } + + self.inner.notify.wait(); + } + } +} diff --git a/tokio-executor/src/tests/loom_schedule.rs b/tokio-executor/src/tests/loom_schedule.rs new file mode 100644 index 00000000000..d3f932284c2 --- /dev/null +++ b/tokio-executor/src/tests/loom_schedule.rs @@ -0,0 +1,51 @@ +use crate::task::{Schedule, Task}; + +use loom::sync::Notify; +use std::collections::VecDeque; +use std::sync::Mutex; + +pub(crate) struct LoomSchedule { + notify: Notify, + pending: Mutex>>>, +} + +impl LoomSchedule { + pub(crate) fn new() -> LoomSchedule { + LoomSchedule { + notify: Notify::new(), + pending: Mutex::new(VecDeque::new()), + } + } + + pub(crate) fn push_task(&self, task: Task) { + self.schedule(task); + } + + pub(crate) fn recv(&self) -> Option> { + loop { + if let Some(task) = self.pending.lock().unwrap().pop_front() { + return task; + } + + self.notify.wait(); + } + } +} + +impl Schedule for LoomSchedule { + fn bind(&self, _task: &Task) {} + + fn release(&self, task: Task) { + self.release_local(&task); + } + + fn release_local(&self, _task: &Task) { + self.pending.lock().unwrap().push_back(None); + self.notify.notify(); + } + + fn schedule(&self, task: Task) { + self.pending.lock().unwrap().push_back(Some(task)); + self.notify.notify(); + } +} diff --git a/tokio-executor/src/tests/mock_park.rs b/tokio-executor/src/tests/mock_park.rs new file mode 100644 index 00000000000..cd36fdf96dc --- /dev/null +++ b/tokio-executor/src/tests/mock_park.rs @@ -0,0 +1,66 @@ +#![allow(warnings)] + +use crate::park::{Park, Unpark}; + +use std::collections::HashMap; +use std::sync::atomic::{AtomicBool, Ordering::SeqCst}; +use std::sync::Arc; +use std::time::Duration; + +pub struct MockPark { + parks: HashMap>, +} + +#[derive(Clone)] +struct ParkImpl(Arc); + +struct Inner { + unparked: AtomicBool, +} + +impl MockPark { + pub fn new() -> MockPark { + MockPark { + parks: HashMap::new(), + } + } + + pub fn is_unparked(&self, index: usize) -> bool { + self.parks[&index].unparked.load(SeqCst) + } + + pub fn clear(&self, index: usize) { + self.parks[&index].unparked.store(false, SeqCst); + } + + pub fn mk_park(&mut self, index: usize) -> impl Park { + let inner = Arc::new(Inner { + unparked: AtomicBool::new(false), + }); + self.parks.insert(index, inner.clone()); + ParkImpl(inner) + } +} + +impl Park for ParkImpl { + type Unpark = ParkImpl; + type Error = (); + + fn unpark(&self) -> Self::Unpark { + self.clone() + } + + fn park(&mut self) -> Result<(), Self::Error> { + unimplemented!(); + } + + fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { + unimplemented!(); + } +} + +impl Unpark for ParkImpl { + fn unpark(&self) { + self.0.unparked.store(true, SeqCst); + } +} diff --git a/tokio-executor/src/tests/mock_schedule.rs b/tokio-executor/src/tests/mock_schedule.rs new file mode 100644 index 00000000000..6f1441b0880 --- /dev/null +++ b/tokio-executor/src/tests/mock_schedule.rs @@ -0,0 +1,131 @@ +#![allow(warnings)] + +use crate::task::{Header, Schedule, Task}; + +use std::collections::VecDeque; +use std::sync::Mutex; +use std::thread; + +pub(crate) struct Mock { + inner: Mutex, +} + +pub(crate) struct Noop; +pub(crate) static NOOP_SCHEDULE: Noop = Noop; + +struct Inner { + calls: VecDeque, + pending_run: VecDeque>, + pending_drop: VecDeque>, +} + +unsafe impl Send for Inner {} +unsafe impl Sync for Inner {} + +#[derive(Debug, Eq, PartialEq)] +enum Call { + Bind(*const Header), + Release, + ReleaseLocal, + Schedule, +} + +pub(crate) fn mock() -> Mock { + Mock { + inner: Mutex::new(Inner { + calls: VecDeque::new(), + pending_run: VecDeque::new(), + pending_drop: VecDeque::new(), + }), + } +} + +impl Mock { + pub(crate) fn bind(self, task: &Task) -> Self { + self.push(Call::Bind(task.header() as *const _)); + self + } + + pub(crate) fn release(self) -> Self { + self.push(Call::Release); + self + } + + pub(crate) fn release_local(self) -> Self { + self.push(Call::ReleaseLocal); + self + } + + pub(crate) fn schedule(self) -> Self { + self.push(Call::Schedule); + self + } + + pub(crate) fn next_pending_run(&self) -> Option> { + self.inner.lock().unwrap().pending_run.pop_front() + } + + pub(crate) fn next_pending_drop(&self) -> Option> { + self.inner.lock().unwrap().pending_drop.pop_front() + } + + fn push(&self, call: Call) { + self.inner.lock().unwrap().calls.push_back(call); + } + + fn next(&self, name: &str) -> Call { + self.inner + .lock() + .unwrap() + .calls + .pop_front() + .expect(&format!("received `{}`, but none expected", name)) + } +} + +impl Schedule for Mock { + fn bind(&self, task: &Task) { + match self.next("bind") { + Call::Bind(ptr) => { + assert!(ptr.eq(&(task.header() as *const _))); + } + call => panic!("expected `Bind`, was {:?}", call), + } + } + + fn release(&self, task: Task) { + match self.next("release") { + Call::Release => { + self.inner.lock().unwrap().pending_drop.push_back(task); + } + call => panic!("expected `Release`, was {:?}", call), + } + } + + fn release_local(&self, _task: &Task) { + assert_eq!(Call::ReleaseLocal, self.next("release_local")); + } + + fn schedule(&self, task: Task) { + self.inner.lock().unwrap().pending_run.push_back(task); + assert_eq!(Call::Schedule, self.next("schedule")); + } +} + +impl Drop for Mock { + fn drop(&mut self) { + if !thread::panicking() { + assert!(self.inner.lock().unwrap().calls.is_empty()); + } + } +} + +impl Schedule for Noop { + fn bind(&self, _task: &Task) {} + + fn release(&self, _task: Task) {} + + fn release_local(&self, _task: &Task) {} + + fn schedule(&self, _task: Task) {} +} diff --git a/tokio-executor/src/tests/mod.rs b/tokio-executor/src/tests/mod.rs new file mode 100644 index 00000000000..b287bcf29e9 --- /dev/null +++ b/tokio-executor/src/tests/mod.rs @@ -0,0 +1,40 @@ +//! Testing utilities + +#[cfg(not(loom))] +pub(crate) mod backoff; + +#[cfg(loom)] +pub(crate) mod loom_oneshot; + +#[cfg(loom)] +pub(crate) mod loom_schedule; + +#[cfg(not(loom))] +pub(crate) mod mock_park; + +pub(crate) mod mock_schedule; + +#[cfg(not(loom))] +pub(crate) mod track_drop; + +/// Panic if expression results in `None`. +#[macro_export] +macro_rules! assert_some { + ($e:expr) => {{ + match $e { + Some(v) => v, + _ => panic!("expected some, was none"), + } + }}; +} + +/// Panic if expression results in `Some`. +#[macro_export] +macro_rules! assert_none { + ($e:expr) => {{ + match $e { + Some(v) => panic!("expected none, was {:?}", v), + _ => {} + } + }}; +} diff --git a/tokio-executor/src/tests/track_drop.rs b/tokio-executor/src/tests/track_drop.rs new file mode 100644 index 00000000000..c3ded845f89 --- /dev/null +++ b/tokio-executor/src/tests/track_drop.rs @@ -0,0 +1,57 @@ +use std::future::Future; +use std::pin::Pin; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::Ordering::SeqCst; +use std::sync::Arc; +use std::task::{Context, Poll}; + +#[derive(Debug)] +pub(crate) struct TrackDrop(T, Arc); + +#[derive(Debug)] +pub(crate) struct DidDrop(Arc, Arc); + +pub(crate) fn track_drop( + future: T, +) -> (impl Future>, DidDrop) { + let did_drop_future = Arc::new(AtomicBool::new(false)); + let did_drop_output = Arc::new(AtomicBool::new(false)); + let did_drop = DidDrop(did_drop_future.clone(), did_drop_output.clone()); + + let future = async move { TrackDrop(future.await, did_drop_output) }; + + let future = TrackDrop(future, did_drop_future); + + (future, did_drop) +} + +impl TrackDrop { + pub(crate) fn get_ref(&self) -> &T { + &self.0 + } +} + +impl Future for TrackDrop { + type Output = T::Output; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let me = unsafe { Pin::map_unchecked_mut(self, |x| &mut x.0) }; + me.poll(cx) + } +} + +impl Drop for TrackDrop { + fn drop(&mut self) { + self.1.store(true, SeqCst); + } +} + +impl DidDrop { + pub(crate) fn did_drop_future(&self) -> bool { + self.0.load(SeqCst) + } + + pub(crate) fn did_drop_output(&self) -> bool { + self.1.load(SeqCst) + } +} diff --git a/tokio-executor/src/thread_pool/builder.rs b/tokio-executor/src/thread_pool/builder.rs new file mode 100644 index 00000000000..921b52d7c07 --- /dev/null +++ b/tokio-executor/src/thread_pool/builder.rs @@ -0,0 +1,253 @@ +use crate::loom::sync::Arc; +use crate::loom::sys::num_cpus; +use crate::loom::thread; +use crate::park::Park; +use crate::thread_pool::park::DefaultPark; +use crate::thread_pool::{shutdown, worker, Spawner, ThreadPool}; + +use std::{fmt, usize}; + +/// Builds a thread pool with custom configuration values. +pub struct Builder { + /// Number of threads to spawn + pool_size: usize, + + /// Thread name prefix + name_prefix: String, + + /// Thread stack size + stack_size: Option, + + /// Around worker callback + around_worker: Option>, +} + +type Callback = Box; + +impl Builder { + /// Returns a new thread pool builder initialized with default configuration + /// values. + pub fn new() -> Builder { + Builder { + pool_size: num_cpus(), + name_prefix: "tokio-runtime-worker-".to_string(), + stack_size: None, + around_worker: None, + } + } + + /// Set the number of threads running async tasks. + /// + /// This must be a number between 1 and 2,048 though it is advised to keep + /// this value on the smaller side. + /// + /// The default value is the number of cores available to the system. + /// + /// # Examples + /// + /// ``` + /// use tokio_executor::thread_pool::Builder; + /// + /// let thread_pool = Builder::new() + /// .num_threads(4) + /// .build(); + /// ``` + pub fn num_threads(&mut self, value: usize) -> &mut Self { + self.pool_size = value; + self + } + + /// Set name prefix of threads spawned by the scheduler + /// + /// Thread name prefix is used for generating thread names. For example, if + /// prefix is `my-pool-`, then threads in the pool will get names like + /// `my-pool-1` etc. + /// + /// If this configuration is not set, then the thread will use the system + /// default naming scheme. + /// + /// # Examples + /// + /// ``` + /// use tokio_executor::thread_pool::Builder; + /// + /// let thread_pool = Builder::new() + /// .name_prefix("my-pool-") + /// .build(); + /// ``` + pub fn name_prefix>(&mut self, val: S) -> &mut Self { + self.name_prefix = val.into(); + self + } + + /// Set the stack size (in bytes) for worker threads. + /// + /// The actual stack size may be greater than this value if the platform + /// specifies minimal stack size. + /// + /// The default stack size for spawned threads is 2 MiB, though this + /// particular stack size is subject to change in the future. + /// + /// # Examples + /// + /// ``` + /// use tokio_executor::thread_pool::Builder; + /// + /// let thread_pool = Builder::new() + /// .stack_size(32 * 1024) + /// .build(); + /// ``` + pub fn stack_size(&mut self, val: usize) -> &mut Self { + self.stack_size = Some(val); + self + } + + /// Execute function `f` on each worker thread. + /// + /// This function is provided a function that executes the worker and is + /// expected to call it, otherwise the worker thread will shutdown without + /// doing any work. + /// + /// # Examples + /// + /// ``` + /// use tokio_executor::thread_pool::Builder; + /// + /// let thread_pool = Builder::new() + /// .around_worker(|index, work| { + /// println!("worker {} is starting up", index); + /// work(); + /// println!("worker {} is shutting down", index); + /// }) + /// .build(); + /// ``` + pub fn around_worker(&mut self, f: F) -> &mut Self + where + F: Fn(usize, &mut dyn FnMut()) + Send + Sync + 'static, + { + self.around_worker = Some(Arc::new(Box::new(f))); + self + } + + /// Create the configured `ThreadPool`. + /// + /// The returned `ThreadPool` instance is ready to spawn tasks. + /// + /// # Examples + /// + /// ``` + /// use tokio_executor::thread_pool::Builder; + /// + /// let thread_pool = Builder::new() + /// .build(); + /// ``` + pub fn build(&self) -> ThreadPool { + self.build_with_park(|_| DefaultPark::new()) + } + + /// Create the configured `ThreadPool` with a custom `park` instances. + /// + /// The provided closure `build_park` is called once per worker and returns + /// a `Park` instance that is used by the worker to put itself to sleep. + pub fn build_with_park(&self, mut build_park: F) -> ThreadPool + where + F: FnMut(usize) -> P, + P: Park + Send + 'static, + { + let (shutdown_tx, shutdown_rx) = shutdown::channel(); + + let (pool, workers) = worker::create_set(self.pool_size, |i| BoxedPark::new(build_park(i))); + + // Spawn threads for each worker + for (idx, mut worker) in workers.into_iter().enumerate() { + let around_worker = self.around_worker.clone(); + let shutdown_tx = shutdown_tx.clone(); + + let mut th = thread::Builder::new().name(format!("{}{}", self.name_prefix, idx)); + + if let Some(stack) = self.stack_size { + th = th.stack_size(stack); + } + + let res = th.spawn(move || { + struct AbortOnPanic; + + impl Drop for AbortOnPanic { + fn drop(&mut self) { + if thread::panicking() { + eprintln!("[ERROR] unhandled panic in Tokio scheduler. This is a bug and should be reported."); + std::process::abort(); + } + } + } + + let _abort_on_panic = AbortOnPanic; + + if let Some(cb) = around_worker { + cb(idx, &mut || worker.run()); + } else { + worker.run(); + } + + // Worker must be dropped before the `shutdown_tx` + drop(worker); + + // Dropping the handle must happen __after__ the callback + drop(shutdown_tx); + }); + + if let Err(err) = res { + panic!("failed to spawn worker thread: {:?}", err); + } + } + + let spawner = Spawner::new(pool); + ThreadPool::from_parts(spawner, shutdown_rx) + } +} + +impl Default for Builder { + fn default() -> Builder { + Builder::new() + } +} + +impl fmt::Debug for Builder { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Builder") + .field("pool_size", &self.pool_size) + .field("name_prefix", &self.name_prefix) + .field("stack_size", &self.stack_size) + .finish() + } +} + +pub(crate) struct BoxedPark

{ + inner: P, +} + +impl

BoxedPark

{ + pub(crate) fn new(inner: P) -> Self { + BoxedPark { inner } + } +} + +impl

Park for BoxedPark

+where + P: Park, +{ + type Unpark = Box; + type Error = P::Error; + + fn unpark(&self) -> Self::Unpark { + Box::new(self.inner.unpark()) + } + + fn park(&mut self) -> Result<(), Self::Error> { + self.inner.park() + } + + fn park_timeout(&mut self, duration: std::time::Duration) -> Result<(), Self::Error> { + self.inner.park_timeout(duration) + } +} diff --git a/tokio-executor/src/thread_pool/current.rs b/tokio-executor/src/thread_pool/current.rs new file mode 100644 index 00000000000..dddad7601ab --- /dev/null +++ b/tokio-executor/src/thread_pool/current.rs @@ -0,0 +1,85 @@ +use crate::loom::sync::Arc; +use crate::park::Unpark; +use crate::thread_pool::{worker, Owned}; + +use std::cell::Cell; +use std::ptr; + +/// Tracks the current worker +#[derive(Debug)] +pub(super) struct Current { + inner: Inner, +} + +#[derive(Debug, Copy, Clone)] +struct Inner { + // thread-local variables cannot track generics. However, the current worker + // is only checked when `P` is already known, so the type can be figured out + // on demand. + workers: *const (), + idx: usize, +} + +// Pointer to the current worker info +thread_local!(static CURRENT_WORKER: Cell = Cell::new(Inner::new())); + +pub(super) fn set(pool: &Arc>, index: usize, f: F) -> R +where + F: FnOnce() -> R, + P: Unpark, +{ + CURRENT_WORKER.with(|cell| { + assert!(cell.get().workers.is_null()); + + struct Guard<'a>(&'a Cell); + + impl Drop for Guard<'_> { + fn drop(&mut self) { + self.0.set(Inner::new()); + } + } + + cell.set(Inner { + workers: pool.shared() as *const _ as *const (), + idx: index, + }); + + let _g = Guard(cell); + + f() + }) +} + +pub(super) fn get(f: F) -> R +where + F: FnOnce(&Current) -> R, +{ + CURRENT_WORKER.with(|cell| { + let current = Current { inner: cell.get() }; + f(¤t) + }) +} + +impl Current { + pub(super) fn as_member<'a, P>(&self, set: &'a worker::Set

) -> Option<&'a Owned

> + where + P: Unpark, + { + let inner = CURRENT_WORKER.with(|cell| cell.get()); + + if ptr::eq(inner.workers as *const _, set.shared().as_ptr()) { + Some(unsafe { &*set.owned()[inner.idx].get() }) + } else { + None + } + } +} + +impl Inner { + fn new() -> Inner { + Inner { + workers: ptr::null(), + idx: 0, + } + } +} diff --git a/tokio-executor/src/thread_pool/idle.rs b/tokio-executor/src/thread_pool/idle.rs new file mode 100644 index 00000000000..acf80df8799 --- /dev/null +++ b/tokio-executor/src/thread_pool/idle.rs @@ -0,0 +1,229 @@ +//! Coordinates idling workers + +use crate::loom::sync::atomic::AtomicUsize; +use crate::loom::sync::Mutex; + +use std::fmt; +use std::sync::atomic::Ordering::{self, AcqRel, Relaxed, SeqCst}; + +pub(super) struct Idle { + /// Tracks both the number of searching workers and the number of unparked + /// workers. + /// + /// Used as a fast-path to avoid acquiring the lock when needed. + state: AtomicUsize, + + /// Sleeping workers + sleepers: Mutex>, + + /// Total number of workers. + num_workers: usize, +} + +const UNPARK_SHIFT: usize = 16; +const UNPARK_MASK: usize = !SEARCH_MASK; +const SEARCH_MASK: usize = (1 << UNPARK_SHIFT) - 1; + +#[derive(Copy, Clone)] +struct State(usize); + +impl Idle { + pub(super) fn new(num_workers: usize) -> Idle { + let init = State::new(num_workers); + + Idle { + state: AtomicUsize::new(init.into()), + sleepers: Mutex::new(Vec::with_capacity(num_workers)), + num_workers, + } + } + + /// If there are no workers actively searching, returns the index of a + /// worker currently sleeping. + pub(super) fn worker_to_notify(&self) -> Option { + // If at least one worker is spinning, work being notified will + // eventully be found. A searching thread will find **some** work and + // notify another worker, eventually leading to our work being found. + // + // For this to happen, this load must happen before the thread + // transitioning `num_searching` to zero. Acquire / Relese does not + // provide sufficient guarantees, so this load is done with `SeqCst` and + // will pair with the `fetch_sub(1)` when transitioning out of + // searching. + if !self.notify_should_wakeup() { + return None; + } + + // Acquire the lock + let mut sleepers = self.sleepers.lock().unwrap(); + + // Check again, now that the lock is acquired + if !self.notify_should_wakeup() { + return None; + } + + // A worker should be woken up, atomically increment the number of + // searching workers as well as the number of unparked workers. + State::unpark_one(&self.state); + + // Get the worker to unpark + let ret = sleepers.pop(); + debug_assert!(ret.is_some()); + + ret + } + + /// Returns `true` if the worker needs to do a final check for submitted + /// work. + pub(super) fn transition_worker_to_parked(&self, worker: usize, is_searching: bool) -> bool { + // Acquire the lock + let mut sleepers = self.sleepers.lock().unwrap(); + + // Decrement the number of unparked threads + let ret = State::dec_num_unparked(&self.state, is_searching); + + // Track the sleeping worker + sleepers.push(worker); + + ret + } + + pub(super) fn transition_worker_to_searching(&self) -> bool { + // Using `Relaxed` ordering is acceptable here as it is just an + // optimization. This load has does not need to synchronize with + // anything, and the algorithm is correct no matter what the load + // returns (as in, it could return absolutely any `usize` value and the + // pool would be correct. + let state = State::load(&self.state, Relaxed); + if 2 * state.num_searching() >= self.num_workers { + return false; + } + + // It is possible for this routine to allow more than 50% of the workers + // to search. That is OK. Limiting searchers is only an optimization to + // prevent too much contention. + // + // At this point, we do not need a hard synchronization with `notify_work`, so `AcqRel` is sufficient. + State::inc_num_searching(&self.state, AcqRel); + true + } + + /// A lightweight transition from searching -> running. + /// + /// Returns `true` if this is the final searching worker. The caller + /// **must** notify a new worker. + pub(super) fn transition_worker_from_searching(&self) -> bool { + State::dec_num_searching(&self.state) + } + + /// Unpark a specific worker. This happens if tasks are submitted from + /// within the worker's park routine. + pub(super) fn unpark_worker_by_id(&self, worker_id: usize) { + let mut sleepers = self.sleepers.lock().unwrap(); + + for index in 0..sleepers.len() { + if sleepers[index] == worker_id { + sleepers.swap_remove(index); + + // Update the state accordingly whle the lock is held. + State::unpark_one(&self.state); + + return; + } + } + } + + /// Returns `true` if `worker_id` is contained in the sleep set + pub(super) fn is_parked(&self, worker_id: usize) -> bool { + let sleepers = self.sleepers.lock().unwrap(); + sleepers.contains(&worker_id) + } + + fn notify_should_wakeup(&self) -> bool { + let state = State::load(&self.state, SeqCst); + state.num_searching() == 0 && state.num_unparked() < self.num_workers + } +} + +impl State { + fn new(num_workers: usize) -> State { + // All workers start in the unparked state + let ret = State(num_workers << UNPARK_SHIFT); + debug_assert_eq!(num_workers, ret.num_unparked()); + debug_assert_eq!(0, ret.num_searching()); + ret + } + + fn load(cell: &AtomicUsize, ordering: Ordering) -> State { + State(cell.load(ordering)) + } + + fn unpark_one(cell: &AtomicUsize) { + cell.fetch_add(1 | (1 << UNPARK_SHIFT), SeqCst); + } + + fn inc_num_searching(cell: &AtomicUsize, ordering: Ordering) { + cell.fetch_add(1, ordering); + } + + /// Returns `true` if this is the final searching worker + fn dec_num_searching(cell: &AtomicUsize) -> bool { + let state = State(cell.fetch_sub(1, SeqCst)); + state.num_searching() == 1 + } + + /// Track a sleeping worker + /// + /// Returns `true` if this is the final searching worker. + fn dec_num_unparked(cell: &AtomicUsize, is_searching: bool) -> bool { + let mut dec = 1 << UNPARK_SHIFT; + + if is_searching { + dec += 1; + } + + let prev = State(cell.fetch_sub(dec, SeqCst)); + is_searching && prev.num_searching() == 1 + } + + /// Number of workers currently searching + fn num_searching(self) -> usize { + self.0 & SEARCH_MASK + } + + /// Number of workers currently unparked + fn num_unparked(self) -> usize { + (self.0 & UNPARK_MASK) >> UNPARK_SHIFT + } +} + +impl From for State { + fn from(src: usize) -> State { + State(src) + } +} + +impl From for usize { + fn from(src: State) -> usize { + src.0 + } +} + +impl fmt::Debug for State { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("worker::State") + .field("num_unparked", &self.num_unparked()) + .field("num_searching", &self.num_searching()) + .finish() + } +} + +#[test] +fn test_state() { + assert_eq!(0, UNPARK_MASK & SEARCH_MASK); + assert_eq!(0, !(UNPARK_MASK | SEARCH_MASK)); + + let state = State::new(10); + assert_eq!(10, state.num_unparked()); + assert_eq!(0, state.num_searching()); +} diff --git a/tokio-executor/src/thread_pool/join.rs b/tokio-executor/src/thread_pool/join.rs new file mode 100644 index 00000000000..ffa85eb3ec4 --- /dev/null +++ b/tokio-executor/src/thread_pool/join.rs @@ -0,0 +1,42 @@ +use crate::park::Unpark; +use crate::task; +use crate::thread_pool::Shared; + +use std::fmt; +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// An owned permission to join on a task (await its termination). +pub struct JoinHandle { + task: task::JoinHandle>>, +} + +impl JoinHandle +where + T: Send + 'static, +{ + pub(super) fn new(task: task::JoinHandle>>) -> JoinHandle { + JoinHandle { task } + } +} + +impl Future for JoinHandle +where + T: Send + 'static, +{ + type Output = task::Result; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + Pin::new(&mut self.task).poll(cx) + } +} + +impl fmt::Debug for JoinHandle +where + T: fmt::Debug, +{ + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("JoinHandle").finish() + } +} diff --git a/tokio-executor/src/thread_pool/mod.rs b/tokio-executor/src/thread_pool/mod.rs new file mode 100644 index 00000000000..767625550a0 --- /dev/null +++ b/tokio-executor/src/thread_pool/mod.rs @@ -0,0 +1,58 @@ +//! Threadpool + +mod builder; +pub use self::builder::Builder; + +mod current; + +mod idle; +use self::idle::Idle; + +mod join; +pub use self::join::JoinHandle; + +mod owned; +use self::owned::Owned; + +mod park; + +mod pool; +pub use self::pool::ThreadPool; + +mod queue; + +mod spawner; +pub use self::spawner::Spawner; + +mod set; + +mod shared; +use self::shared::Shared; + +mod shutdown; + +mod worker; + +/// Unit tests +#[cfg(test)] +mod tests; + +// Re-export `task::Error` +pub use crate::task::Error; + +// These exports are used in tests +#[cfg(test)] +#[allow(warnings)] +pub(crate) use self::worker::create_set as create_pool; + +pub(crate) type BoxFuture = + std::pin::Pin + Send + 'static>>; + +#[cfg(not(loom))] +const LOCAL_QUEUE_CAPACITY: usize = 256; + +// Shrink the size of the local queue when using loom. This shouldn't impact +// logic, but allows loom to test more edge cases in a reasonable a mount of +// time. +#[cfg(loom)] +const LOCAL_QUEUE_CAPACITY: usize = 2; diff --git a/tokio-executor/src/thread_pool/owned.rs b/tokio-executor/src/thread_pool/owned.rs new file mode 100644 index 00000000000..740ee0439ed --- /dev/null +++ b/tokio-executor/src/thread_pool/owned.rs @@ -0,0 +1,77 @@ +use crate::task::{self, Task}; +use crate::thread_pool::{queue, Shared}; +use crate::util::FastRand; + +use std::cell::Cell; + +/// Per-worker data accessible only by the thread driving the worker. +#[derive(Debug)] +pub(super) struct Owned { + /// Worker tick number. Used to schedule bookkeeping tasks every so often. + pub(super) tick: Cell, + + /// Caches the pool run state. + pub(super) is_running: Cell, + + /// `true` if the worker is currently searching for more work. + pub(super) is_searching: Cell, + + /// `true` when worker notification should be delayed. + /// + /// This is used to batch notifications triggered by the parker. + pub(super) defer_notification: Cell, + + /// `true` if a task was submitted while `defer_notification` was set + pub(super) did_submit_task: Cell, + + /// Fast random number generator + pub(super) rand: FastRand, + + /// Work queue + pub(super) work_queue: queue::Worker>, + + /// List of tasks owned by the worker + pub(super) owned_tasks: task::OwnedList>, +} + +impl

Owned

+where + P: 'static, +{ + pub(super) fn new(work_queue: queue::Worker>, rand: FastRand) -> Owned

{ + Owned { + tick: Cell::new(1), + is_running: Cell::new(true), + is_searching: Cell::new(false), + defer_notification: Cell::new(false), + did_submit_task: Cell::new(false), + rand, + work_queue, + owned_tasks: task::OwnedList::new(), + } + } + + /// Returns `true` if a worker should be notified + pub(super) fn submit_local(&self, task: Task>) -> bool { + let ret = self.work_queue.push(task); + + if self.defer_notification.get() { + self.did_submit_task.set(true); + false + } else { + ret + } + } + + pub(super) fn submit_local_yield(&self, task: Task>) { + self.work_queue.push_yield(task); + } + + pub(super) fn bind_task(&mut self, task: &Task>) { + self.owned_tasks.insert(task); + } + + pub(super) fn release_task(&mut self, task: &Task>) { + self.owned_tasks.remove(task); + } +} diff --git a/tokio-executor/src/thread_pool/park.rs b/tokio-executor/src/thread_pool/park.rs new file mode 100644 index 00000000000..bc3497684c2 --- /dev/null +++ b/tokio-executor/src/thread_pool/park.rs @@ -0,0 +1,182 @@ +use crate::loom::sync::atomic::AtomicUsize; +use crate::loom::sync::{Arc, Condvar, Mutex}; +use crate::park::{Park, Unpark}; + +use std::error::Error; +use std::fmt; +use std::sync::atomic::Ordering::SeqCst; +use std::time::Duration; + +/// Parks the thread. +#[derive(Debug)] +pub(crate) struct DefaultPark { + inner: Arc, +} + +/// Unparks threads that were parked by `DefaultPark`. +#[derive(Debug)] +pub(crate) struct DefaultUnpark { + inner: Arc, +} + +/// Error returned by [`ParkThread`] +/// +/// This currently is never returned, but might at some point in the future. +/// +/// [`ParkThread`]: struct.ParkThread.html +#[derive(Debug)] +pub(crate) struct ParkError { + _p: (), +} + +const EMPTY: usize = 0; +const PARKED: usize = 1; +const NOTIFIED: usize = 2; + +#[derive(Debug)] +struct Inner { + state: AtomicUsize, + lock: Mutex<()>, + cvar: Condvar, +} + +impl DefaultPark { + /// Creates a new `DefaultPark` instance. + pub(crate) fn new() -> DefaultPark { + DefaultPark { + inner: Arc::new(Inner { + state: AtomicUsize::new(EMPTY), + lock: Mutex::new(()), + cvar: Condvar::new(), + }), + } + } +} + +impl Park for DefaultPark { + type Unpark = DefaultUnpark; + type Error = ParkError; + + fn unpark(&self) -> Self::Unpark { + let inner = self.inner.clone(); + DefaultUnpark { inner } + } + + fn park(&mut self) -> Result<(), Self::Error> { + self.inner.park(None); + Ok(()) + } + + fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { + self.inner.park(Some(duration)); + Ok(()) + } +} + +impl Unpark for DefaultUnpark { + fn unpark(&self) { + self.inner.unpark(); + } +} + +impl fmt::Display for ParkError { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(fmt, "unknown park error") + } +} + +impl Error for ParkError {} + +impl Inner { + fn park(&self, timeout: Option) { + // If we were previously notified then we consume this notification and return quickly. + if self + .state + .compare_exchange(NOTIFIED, EMPTY, SeqCst, SeqCst) + .is_ok() + { + return; + } + + // If the timeout is zero, then there is no need to actually block. + if let Some(ref dur) = timeout { + if *dur == Duration::from_millis(0) { + return; + } + } + + // Otherwise we need to coordinate going to sleep. + let mut _m = self.lock.lock().unwrap(); + + match self.state.compare_exchange(EMPTY, PARKED, SeqCst, SeqCst) { + Ok(_) => {} + // Consume this notification to avoid spurious wakeups in the next park. + Err(NOTIFIED) => { + // We must read `state` here, even though we know it will be `NOTIFIED`. This is + // because `unpark` may have been called again since we read `NOTIFIED` in the + // `compare_exchange` above. We must perform an acquire operation that synchronizes + // with that `unpark` to observe any writes it made before the call to `unpark`. To + // do that we must read from the write it made to `state`. + let old = self.state.swap(EMPTY, SeqCst); + assert_eq!(old, NOTIFIED, "park state changed unexpectedly"); + return; + } + Err(n) => panic!("inconsistent park_timeout state: {}", n), + } + + match timeout { + None => { + loop { + // Block the current thread on the conditional variable. + _m = self.cvar.wait(_m).unwrap(); + + if self + .state + .compare_exchange(NOTIFIED, EMPTY, SeqCst, SeqCst) + .is_ok() + { + return; // got a notification + } + + // spurious wakeup, go back to sleep + } + } + Some(timeout) => { + // Wait with a timeout, and if we spuriously wake up or otherwise wake up from a + // notification we just want to unconditionally set `state` back to `EMPTY`, either + // consuming a notification or un-flagging ourselves as parked. + _m = self.cvar.wait_timeout(_m, timeout).unwrap().0; + + match self.state.swap(EMPTY, SeqCst) { + NOTIFIED => {} // got a notification + PARKED => {} // no notification + n => panic!("inconsistent park_timeout state: {}", n), + } + } + } + } + + fn unpark(&self) { + // To ensure the unparked thread will observe any writes we made before this call, we must + // perform a release operation that `park` can synchronize with. To do that we must write + // `NOTIFIED` even if `state` is already `NOTIFIED`. That is why this must be a swap rather + // than a compare-and-swap that returns if it reads `NOTIFIED` on failure. + match self.state.swap(NOTIFIED, SeqCst) { + EMPTY => return, // no one was waiting + NOTIFIED => return, // already unparked + PARKED => {} // gotta go wake someone up + n => panic!("inconsistent state in unpark: {}", n), + } + + // There is a period between when the parked thread sets `state` to `PARKED` (or last + // checked `state` in the case of a spurious wakeup) and when it actually waits on `cvar`. + // If we were to notify during this period it would be ignored and then when the parked + // thread went to sleep it would never wake up. Fortunately, it has `lock` locked at this + // stage so we can acquire `lock` to wait until it is ready to receive the notification. + // + // Releasing `lock` before the call to `notify_one` means that when the parked thread wakes + // it doesn't get woken only to have to wait for us to release `lock`. + drop(self.lock.lock()); + self.cvar.notify_one(); + } +} diff --git a/tokio-executor/src/thread_pool/pool.rs b/tokio-executor/src/thread_pool/pool.rs new file mode 100644 index 00000000000..9df368a880a --- /dev/null +++ b/tokio-executor/src/thread_pool/pool.rs @@ -0,0 +1,101 @@ +use crate::thread_pool::{shutdown, Builder, JoinHandle, Spawner}; +use crate::Executor; + +use std::fmt; +use std::future::Future; + +/// Work-stealing based thread pool for executing futures. +pub struct ThreadPool { + spawner: Spawner, + + /// Shutdown waiter + shutdown_rx: shutdown::Receiver, +} + +impl ThreadPool { + /// Create a new ThreadPool with default configuration + pub fn new() -> ThreadPool { + Builder::new().build() + } + + pub(super) fn from_parts(spawner: Spawner, shutdown_rx: shutdown::Receiver) -> ThreadPool { + ThreadPool { + spawner, + shutdown_rx, + } + } + + /// Returns reference to `Spawner`. + /// + /// The `Spawner` handle can be cloned and enables spawning tasks from other + /// threads. + pub fn spawner(&self) -> &Spawner { + &self.spawner + } + + /// Spawn a task + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + self.spawner.spawn(future) + } + + /// Spawn a task in the background + pub(crate) fn spawn_background(&self, future: F) + where + F: Future + Send + 'static, + { + self.spawner.spawn_background(future); + } + + /// Block the current thread waiting for the future to complete. + /// + /// The future will execute on the current thread, but all spawned tasks + /// will be executed on the thread pool. + pub fn block_on(&self, future: F) -> F::Output + where + F: Future, + { + crate::global::with_threadpool(self, || { + let mut enter = crate::enter().expect("attempting to block while on a Tokio executor"); + enter.block_on(future) + }) + } + + /// Shutdown the thread pool. + pub fn shutdown_now(&mut self) { + if self.spawner.workers().close() { + self.shutdown_rx.wait(); + } + } +} + +impl Default for ThreadPool { + fn default() -> ThreadPool { + ThreadPool::new() + } +} + +impl Executor for &ThreadPool { + fn spawn( + &mut self, + future: std::pin::Pin + Send>>, + ) -> Result<(), crate::SpawnError> { + ThreadPool::spawn_background(self, future); + Ok(()) + } +} + +impl fmt::Debug for ThreadPool { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("ThreadPool").finish() + } +} + +impl Drop for ThreadPool { + fn drop(&mut self) { + self.shutdown_now(); + } +} diff --git a/tokio-executor/src/thread_pool/queue/global.rs b/tokio-executor/src/thread_pool/queue/global.rs new file mode 100644 index 00000000000..2c98ffe929e --- /dev/null +++ b/tokio-executor/src/thread_pool/queue/global.rs @@ -0,0 +1,195 @@ +use crate::loom::sync::atomic::AtomicUsize; +use crate::loom::sync::Mutex; +use crate::task::{Header, Task}; + +use std::ptr::{self, NonNull}; +use std::sync::atomic::Ordering::{Acquire, Release}; +use std::usize; + +pub(super) struct Queue { + /// Pointers to the head and tail of the queue + pointers: Mutex>, + + /// Number of pending tasks in the queue. This helps prevent unnecessary + /// locking in the hot path. + /// + /// The LSB is a flag tracking whether or not the queue is open or not. + len: AtomicUsize, +} + +struct Pointers { + head: *const Header, + tail: *const Header, +} + +const CLOSED: usize = 1; +const MAX_LEN: usize = usize::MAX >> 1; + +impl Queue { + pub(super) fn new() -> Queue { + Queue { + pointers: Mutex::new(Pointers { + head: ptr::null(), + tail: ptr::null(), + }), + len: AtomicUsize::new(0), + } + } + + pub(super) fn is_empty(&self) -> bool { + self.len() == 0 + } + + pub(super) fn is_closed(&self) -> bool { + self.len.load(Acquire) & CLOSED == CLOSED + } + + /// Close the worker queue + pub(super) fn close(&self) -> bool { + // Acquire the lock + let _p = self.pointers.lock().unwrap(); + + let len = unsafe { + // Set the queue as closed. Because all mutations are synchronized by + // the mutex, a read followed by a write is acceptable. + self.len.unsync_load() + }; + + let ret = len & CLOSED == 0; + + self.len.store(len | CLOSED, Release); + + ret + } + + fn len(&self) -> usize { + self.len.load(Acquire) >> 1 + } + + pub(super) fn wait_for_unlocked(&self) { + // Acquire and release the lock immediately. This synchronizes the + // caller **after** all external waiters are done w/ the scheduler + // struct. + drop(self.pointers.lock().unwrap()); + } + + /// Push a value into the queue and call the closure **while still holding + /// the push lock** + pub(super) fn push(&self, task: Task, f: F) + where + F: FnOnce(Result<(), Task>), + { + unsafe { + // Acquire queue lock + let mut p = self.pointers.lock().unwrap(); + + // Check if the queue is closed. This must happen in the lock. + let len = self.len.unsync_load(); + if len & CLOSED == CLOSED { + f(Err(task)); + return; + } + + let task = task.into_raw(); + + // The next pointer should already be null + debug_assert!(get_next(task).is_null()); + + if let Some(tail) = NonNull::new(p.tail as *mut _) { + set_next(tail, task.as_ptr()); + } else { + p.head = task.as_ptr(); + } + + p.tail = task.as_ptr(); + + // Increment the count. + // + // All updates to the len atomic are guarded by the mutex. As such, + // a non-atomic load followed by a store is safe. + // + // We increment by 2 to avoid touching the shutdown flag + if (len >> 1) == MAX_LEN { + eprintln!("[ERROR] overflowed task counter. This is a bug and should be reported."); + std::process::abort(); + } + + self.len.store(len + 2, Release); + f(Ok(())); + } + } + + pub(super) fn push_batch(&self, batch_head: Task, batch_tail: Task, num: usize) { + unsafe { + let batch_head = batch_head.into_raw().as_ptr(); + let batch_tail = batch_tail.into_raw(); + + debug_assert!(get_next(batch_tail).is_null()); + + let mut p = self.pointers.lock().unwrap(); + + if let Some(tail) = NonNull::new(p.tail as *mut _) { + set_next(tail, batch_head); + } else { + p.head = batch_head; + } + + p.tail = batch_tail.as_ptr(); + + // Increment the count. + // + // All updates to the len atomic are guarded by the mutex. As such, + // a non-atomic load followed by a store is safe. + // + // Left shift by 1 to avoid touching the shutdown flag. + let len = self.len.unsync_load(); + + if (len >> 1) >= (MAX_LEN - num) { + std::process::abort(); + } + + self.len.store(len + (num << 1), Release); + } + } + + pub(super) fn pop(&self) -> Option> { + // Fast path, if len == 0, then there are no values + if self.is_empty() { + return None; + } + + unsafe { + let mut p = self.pointers.lock().unwrap(); + + // It is possible to hit null here if another thread poped the last + // task between us checking `len` and acquiring the lock. + let task = NonNull::new(p.head as *mut _)?; + + p.head = get_next(task); + + if p.head.is_null() { + p.tail = ptr::null(); + } + + set_next(task, ptr::null()); + + // Decrement the count. + // + // All updates to the len atomic are guarded by the mutex. As such, + // a non-atomic load followed by a store is safe. + // + // Decrement by 2 to avoid touching the shutdown flag + self.len.store(self.len.unsync_load() - 2, Release); + + Some(Task::from_raw(task)) + } + } +} + +unsafe fn get_next(meta: NonNull>) -> *const Header { + *meta.as_ref().queue_next.get() +} + +unsafe fn set_next(meta: NonNull>, val: *const Header) { + *meta.as_ref().queue_next.get() = val; +} diff --git a/tokio-executor/src/thread_pool/queue/inject.rs b/tokio-executor/src/thread_pool/queue/inject.rs new file mode 100644 index 00000000000..faa891f15ae --- /dev/null +++ b/tokio-executor/src/thread_pool/queue/inject.rs @@ -0,0 +1,36 @@ +use crate::loom::sync::Arc; +use crate::task::Task; +use crate::thread_pool::queue::Cluster; + +pub(crate) struct Inject { + cluster: Arc>, +} + +impl Inject { + pub(super) fn new(cluster: Arc>) -> Inject { + Inject { cluster } + } + + /// Push a value onto the queue + pub(crate) fn push(&self, task: Task, f: F) + where + F: FnOnce(Result<(), Task>), + { + self.cluster.global.push(task, f) + } + + /// Close the queue + /// + /// Returns `true` if the channel was closed. `false` indicates the pool was + /// previously closed. + pub(crate) fn close(&self) -> bool { + self.cluster.global.close() + } + + /// Wait for all locks on the queue to drop. + /// + /// This is done by locking w/o doing anything. + pub(crate) fn wait_for_unlocked(&self) { + self.cluster.global.wait_for_unlocked(); + } +} diff --git a/tokio-executor/src/thread_pool/queue/local.rs b/tokio-executor/src/thread_pool/queue/local.rs new file mode 100644 index 00000000000..4de397b9e41 --- /dev/null +++ b/tokio-executor/src/thread_pool/queue/local.rs @@ -0,0 +1,298 @@ +use crate::loom::cell::{CausalCell, CausalCheck}; +use crate::loom::sync::atomic::{self, AtomicU32}; +use crate::task::Task; +use crate::thread_pool::queue::global; +use crate::thread_pool::LOCAL_QUEUE_CAPACITY; + +use std::fmt; +use std::mem::MaybeUninit; +use std::ptr; +use std::sync::atomic::Ordering::{Acquire, Release}; + +pub(super) struct Queue { + /// Concurrently updated by many threads. + head: AtomicU32, + + /// Only updated by producer thread but read by many threads. + tail: AtomicU32, + + /// Elements + buffer: Box<[CausalCell>>]>, +} + +const MASK: usize = LOCAL_QUEUE_CAPACITY - 1; + +impl Queue { + pub(super) fn new() -> Queue { + debug_assert!(LOCAL_QUEUE_CAPACITY >= 2 && LOCAL_QUEUE_CAPACITY.is_power_of_two()); + + let mut buffer = Vec::with_capacity(LOCAL_QUEUE_CAPACITY); + + for _ in 0..LOCAL_QUEUE_CAPACITY { + buffer.push(CausalCell::new(MaybeUninit::uninit())); + } + + Queue { + head: AtomicU32::new(0), + tail: AtomicU32::new(0), + buffer: buffer.into(), + } + } +} + +impl Queue { + /// Push a task onto the local queue. + /// + /// This **must** be called by the producer thread. + pub(super) unsafe fn push(&self, mut task: Task, global: &global::Queue) { + loop { + let head = self.head.load(Acquire); + + // safety: this is the **only** thread that updates this cell. + let tail = self.tail.unsync_load(); + + if tail.wrapping_sub(head) < LOCAL_QUEUE_CAPACITY as u32 { + // Map the position to a slot index. + let idx = tail as usize & MASK; + + self.buffer[idx].with_mut(|ptr| { + // Write the task to the slot + ptr::write((*ptr).as_mut_ptr(), task); + }); + + // Make the task available + self.tail.store(tail.wrapping_add(1), Release); + + return; + } + + // The local buffer is full. Push a batch of work to the global + // queue. + match self.push_overflow(task, head, tail, global) { + Ok(_) => return, + // Lost the race, try again + Err(v) => task = v, + } + + atomic::spin_loop_hint(); + } + } + + /// Move a batch of tasks into the global queue. + /// + /// This will temporarily make some of the tasks unavailable to stealers. + /// Once `push_overflow` is done, a notification is sent out, so if other + /// workers "missed" some of the tasks during a steal, they will get + /// another opportunity. + #[inline(never)] + unsafe fn push_overflow( + &self, + task: Task, + head: u32, + tail: u32, + global: &global::Queue, + ) -> Result<(), Task> { + const BATCH_LEN: usize = LOCAL_QUEUE_CAPACITY / 2 + 1; + + let n = tail.wrapping_sub(head) / 2; + assert_eq!(n as usize, LOCAL_QUEUE_CAPACITY / 2, "queue is not full"); + + // Claim a bunch of tasks + // + // We are claiming the tasks **before** reading them out of the buffer. + // This is safe because only the **current** thread is able to push new + // tasks. + // + // There isn't really any need for memory ordering... Relaxed would + // work. This is because all tasks are pushed into the queue from the + // current thread (or memory has been acquired if the local queue handle + // moved). + let actual = self.head.compare_and_swap(head, head + n, Release); + if actual != head { + // We failed to claim the tasks, losing the race. Return out of + // this function and try the full `push` routine again. The queue + // may not be full anymore. + return Err(task); + } + + // link the tasks + for i in 0..n { + let j = i + 1; + + let i_idx = (i + head) as usize & MASK; + let j_idx = (j + head) as usize & MASK; + + // Get the next pointer + let next = if j == n { + // The last task in the local queue being moved + task.header() as *const _ + } else { + self.buffer[j_idx].with(|ptr| { + let value = (*ptr).as_ptr(); + (*value).header() as *const _ + }) + }; + + self.buffer[i_idx].with_mut(|ptr| { + let ptr = (*ptr).as_ptr(); + debug_assert!((*(*ptr).header().queue_next.get()).is_null()); + *(*ptr).header().queue_next.get() = next; + }); + } + + let head = self.buffer[head as usize & MASK].with(|ptr| ptr::read((*ptr).as_ptr())); + + // Push the tasks onto the global queue + global.push_batch(head, task, BATCH_LEN); + + Ok(()) + } + + /// Pop a task from the local queue. + /// + /// This **must** be called by the producer thread + pub(super) unsafe fn pop(&self) -> Option> { + loop { + let head = self.head.load(Acquire); + + // safety: this is the **only** thread that updates this cell. + let tail = self.tail.unsync_load(); + + if head == tail { + // queue is empty + return None; + } + + // Map the head position to a slot index. + let idx = head as usize & MASK; + + let task = self.buffer[idx].with(|ptr| { + // Tentatively read the task at the head position. Note that we + // have not yet claimed the task. + // + ptr::read(ptr) + }); + + // Attempt to claim the task read above. + let actual = self + .head + .compare_and_swap(head, head.wrapping_add(1), Release); + + if actual == head { + return Some(task.assume_init()); + } + + atomic::spin_loop_hint(); + } + } + + pub(super) fn is_empty(&self) -> bool { + let head = self.head.load(Acquire); + let tail = self.tail.load(Acquire); + + head == tail + } + + /// Steal half the tasks from self and place them into `dst`. + pub(super) unsafe fn steal(&self, dst: &Queue) -> Option> { + let dst_tail = dst.tail.unsync_load(); + + // Steal the tasks into `dst`'s buffer. This does not yet expose the + // tasks in `dst`. + let mut n = self.steal2(dst, dst_tail); + + if n == 0 { + // No tasks were stolen + return None; + } + + // We are returning a task here + n -= 1; + + let ret_pos = dst_tail.wrapping_add(n); + let ret_idx = ret_pos as usize & MASK; + + let ret = dst.buffer[ret_idx].with(|ptr| ptr::read((*ptr).as_ptr())); + + if n == 0 { + // The `dst` queue is empty, but a single task was stolen + return Some(ret); + } + + // Synchronize with stealers + let dst_head = dst.head.load(Acquire); + + assert!(dst_tail.wrapping_sub(dst_head) + n <= LOCAL_QUEUE_CAPACITY as u32); + + // Make the stolen items available to consumers + dst.tail.store(dst_tail.wrapping_add(n), Release); + + Some(ret) + } + + unsafe fn steal2(&self, dst: &Queue, dst_tail: u32) -> u32 { + loop { + let src_head = self.head.load(Acquire); + let src_tail = self.tail.load(Acquire); + + // Number of available tasks to steal + let n = src_tail.wrapping_sub(src_head); + let n = n - n / 2; + + if n == 0 { + return 0; + } + + if n > LOCAL_QUEUE_CAPACITY as u32 / 2 { + atomic::spin_loop_hint(); + // inconsistent, try again + continue; + } + + // Track CausalCell causality checks. The check is deferred until + // the compare_and_swap claims ownership of the tasks. + let mut check = CausalCheck::default(); + + for i in 0..n { + // Compute the positions + let src_pos = src_head.wrapping_add(i); + let dst_pos = dst_tail.wrapping_add(i); + + // Map to slots + let src_idx = src_pos as usize & MASK; + let dst_idx = dst_pos as usize & MASK; + + // Read the task + let (task, ch) = + self.buffer[src_idx].with_deferred(|ptr| ptr::read((*ptr).as_ptr())); + + check.join(ch); + + // Write the task to the new slot + dst.buffer[dst_idx].with_mut(|ptr| ptr::write((*ptr).as_mut_ptr(), task)); + } + + // Claim all of those tasks! + let actual = self + .head + .compare_and_swap(src_head, src_head.wrapping_add(n), Release); + + if actual == src_head { + check.check(); + return n; + } + + atomic::spin_loop_hint(); + } + } +} + +impl fmt::Debug for Queue { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("local::Queue") + .field("head", &self.head) + .field("tail", &self.tail) + .field("buffer", &"[...]") + .finish() + } +} diff --git a/tokio-executor/src/thread_pool/queue/mod.rs b/tokio-executor/src/thread_pool/queue/mod.rs new file mode 100644 index 00000000000..88633ee39e1 --- /dev/null +++ b/tokio-executor/src/thread_pool/queue/mod.rs @@ -0,0 +1,41 @@ +//! The threadpool's task queue system. + +mod global; +mod inject; +mod local; +mod worker; + +pub(crate) use self::inject::Inject; +pub(crate) use self::worker::Worker; + +use crate::loom::sync::Arc; + +pub(crate) fn build(workers: usize) -> Vec> { + let local: Vec<_> = (0..workers).map(|_| local::Queue::new()).collect(); + + let cluster = Arc::new(Cluster { + local: local.into_boxed_slice(), + global: global::Queue::new(), + }); + + (0..workers) + .map(|index| Worker::new(cluster.clone(), index)) + .collect() +} + +struct Cluster { + /// per-worker local queues + local: Box<[local::Queue]>, + global: global::Queue, +} + +impl Drop for Cluster { + fn drop(&mut self) { + // Drain all the queues + for queue in &self.local[..] { + while let Some(_) = unsafe { queue.pop() } {} + } + + while let Some(_) = self.global.pop() {} + } +} diff --git a/tokio-executor/src/thread_pool/queue/worker.rs b/tokio-executor/src/thread_pool/queue/worker.rs new file mode 100644 index 00000000000..d2248dc2c55 --- /dev/null +++ b/tokio-executor/src/thread_pool/queue/worker.rs @@ -0,0 +1,129 @@ +use crate::task::Task; +use crate::thread_pool::queue::{local, Cluster, Inject}; + +// Loom primitive +use crate::loom::sync::Arc; + +use std::cell::Cell; +use std::fmt; + +pub(crate) struct Worker { + cluster: Arc>, + index: u16, + /// Task to pop next + next: Cell>>, +} + +impl Worker { + pub(super) fn new(cluster: Arc>, index: usize) -> Worker { + Worker { + cluster, + index: index as u16, + next: Cell::new(None), + } + } + + pub(crate) fn injector(&self) -> Inject { + Inject::new(self.cluster.clone()) + } + + /// Returns `true` if the queue is closed + pub(crate) fn is_closed(&self) -> bool { + self.cluster.global.is_closed() + } + + /// Push to the local queue. + /// + /// If the local queue is full, the task is pushed onto the global queue. + /// + /// # Return + /// + /// Returns `true` if the pushed task can be stolen by another worker. + pub(crate) fn push(&self, task: Task) -> bool { + let prev = self.next.take(); + let ret = prev.is_some(); + + if let Some(prev) = prev { + // safety: we guarantee that only one thread pushes to this local + // queue at a time. + unsafe { + self.local().push(prev, &self.cluster.global); + } + } + + self.next.set(Some(task)); + + ret + } + + pub(crate) fn push_yield(&self, task: Task) { + unsafe { self.local().push(task, &self.cluster.global) } + } + + /// Pop a task checking the local queue first. + pub(crate) fn pop_local_first(&self) -> Option> { + self.local_pop().or_else(|| self.cluster.global.pop()) + } + + /// Pop a task checking the global queue first. + pub(crate) fn pop_global_first(&self) -> Option> { + self.cluster.global.pop().or_else(|| self.local_pop()) + } + + /// Steal from other local queues. + /// + /// `start` specifies the queue from which to start stealing. + pub(crate) fn steal(&self, start: usize) -> Option> { + let num_queues = self.cluster.local.len(); + + for i in 0..num_queues { + let i = (start + i) % num_queues; + + if i == self.index as usize { + continue; + } + + // safety: we own the dst queue + let ret = unsafe { self.cluster.local[i].steal(self.local()) }; + + if ret.is_some() { + return ret; + } + } + + None + } + + /// An approximation of whether or not the queue is empty. + pub(crate) fn is_empty(&self) -> bool { + for local_queue in &self.cluster.local[..] { + if !local_queue.is_empty() { + return false; + } + } + + self.cluster.global.is_empty() + } + + fn local_pop(&self) -> Option> { + if let Some(task) = self.next.take() { + return Some(task); + } + // safety: we guarantee that only one thread pushes to this local queue + // at a time. + unsafe { self.local().pop() } + } + + fn local(&self) -> &local::Queue { + &self.cluster.local[self.index as usize] + } +} + +impl fmt::Debug for Worker { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("queue::Worker") + .field("cluster", &"...") + .field("index", &self.index) + .finish() + } +} diff --git a/tokio-executor/src/thread_pool/set.rs b/tokio-executor/src/thread_pool/set.rs new file mode 100644 index 00000000000..3002223cd91 --- /dev/null +++ b/tokio-executor/src/thread_pool/set.rs @@ -0,0 +1,196 @@ +//! Putting a worker to sleep. +//! +//! - Attempt to spin. + +use crate::loom::rand::seed; +use crate::park::Unpark; +use crate::task::{self, Task}; +use crate::thread_pool::{current, queue, BoxFuture, Idle, JoinHandle, Owned, Shared}; +use crate::util::{CachePadded, FastRand}; +use crate::{Executor, SpawnError}; + +use std::cell::UnsafeCell; +use std::future::Future; + +pub(crate) struct Set

+where + P: 'static, +{ + /// Data accessible from all workers. + shared: Box<[Shared

]>, + + /// Data owned by the worker. + owned: Box<[UnsafeCell>>]>, + + /// Submit work to the pool while *not* currently on a worker thread. + inject: queue::Inject>, + + /// Coordinates idle workers + idle: Idle, +} + +unsafe impl Send for Set

{} +unsafe impl Sync for Set

{} + +impl

Set

+where + P: Unpark, +{ + /// Create a new worker set using the provided queues. + pub(crate) fn new(num_workers: usize, mut mk_unpark: F) -> Self + where + F: FnMut(usize) -> P, + { + assert!(num_workers > 0); + + let queues = queue::build(num_workers); + let inject = queues[0].injector(); + + let mut shared = Vec::with_capacity(queues.len()); + let mut owned = Vec::with_capacity(queues.len()); + + for (i, queue) in queues.into_iter().enumerate() { + let unpark = mk_unpark(i); + let rand = FastRand::new(seed()); + + shared.push(Shared::new(unpark)); + owned.push(UnsafeCell::new(CachePadded::new(Owned::new(queue, rand)))); + } + + Set { + shared: shared.into_boxed_slice(), + owned: owned.into_boxed_slice(), + inject, + idle: Idle::new(num_workers), + } + } + + fn inject_task(&self, task: Task>) { + self.inject.push(task, |res| { + if let Err(task) = res { + task.shutdown(); + + // There may be a worker, in the process of being shutdown, that is + // waiting for this task to be released, so we notify all workers + // just in case. + // + // Over aggressive, but the runtime is in the process of shutting + // down, so efficiency is not critical. + self.notify_all(); + } else { + self.notify_work(); + } + }); + } + + pub(super) fn notify_work(&self) { + if let Some(index) = self.idle.worker_to_notify() { + self.shared[index].unpark(); + } + } + + pub(super) fn notify_all(&self) { + for shared in &self.shared[..] { + shared.unpark(); + } + } + + pub(crate) fn spawn_background(&self, future: F) + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + let task = task::background(future); + self.schedule(task); + } + + pub(crate) fn schedule(&self, task: Task>) { + current::get(|current_worker| match current_worker.as_member(self) { + Some(worker) => { + if worker.submit_local(task) { + self.notify_work(); + } + } + None => { + self.inject_task(task); + } + }) + } + + pub(crate) fn set_container_ptr(&mut self) { + let ptr = self as *const _; + for shared in &mut self.shared[..] { + shared.set_container_ptr(ptr); + } + } + + /// Signal the pool is closed + /// + /// Returns `true` if the transition to closed is successful. `false` + /// indicates the pool was already closed. + pub(crate) fn close(&self) -> bool { + if self.inject.close() { + self.notify_all(); + true + } else { + false + } + } + + pub(crate) fn len(&self) -> usize { + self.shared.len() + } + + pub(super) fn index_of(&self, shared: &Shared

) -> usize { + use std::mem; + + let size = mem::size_of::>(); + + ((shared as *const _ as usize) - (&self.shared[0] as *const _ as usize)) / size + } + + pub(super) fn shared(&self) -> &[Shared

] { + &self.shared + } + + pub(super) fn owned(&self) -> &[UnsafeCell>>] { + &self.owned + } + + pub(super) fn idle(&self) -> &Idle { + &self.idle + } +} + +impl Drop for Set

{ + fn drop(&mut self) { + // Before proceeding, wait for all concurrent wakers to exit + self.inject.wait_for_unlocked(); + } +} + +impl Set> { + pub(crate) fn spawn_typed(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + let (task, handle) = task::joinable(future); + self.schedule(task); + JoinHandle::new(handle) + } +} + +impl

Executor for &Set

+where + P: Unpark, +{ + fn spawn(&mut self, future: BoxFuture) -> Result<(), SpawnError> { + self.spawn_background(future); + Ok(()) + } + + fn status(&self) -> Result<(), SpawnError> { + Ok(()) + } +} diff --git a/tokio-executor/src/thread_pool/shared.rs b/tokio-executor/src/thread_pool/shared.rs new file mode 100644 index 00000000000..b717493c2db --- /dev/null +++ b/tokio-executor/src/thread_pool/shared.rs @@ -0,0 +1,104 @@ +use crate::park::Unpark; +use crate::task::{self, Schedule, Task}; +use crate::thread_pool::worker; + +use std::ptr; + +/// Per-worker data accessible from any thread. +/// +/// Accessed by: +/// +/// - other workers +/// - tasks +/// +pub(crate) struct Shared

+where + P: 'static, +{ + /// Thread unparker + unpark: P, + + /// Tasks pending drop. Any worker pushes tasks, only the "owning" worker + /// pops. + pub(super) pending_drop: task::TransferStack, + + /// Untracked pointer to the pool. + /// + /// The pool itself is tracked by an `Arc`, but this pointer is not included + /// in the ref count. + /// + /// # Safety + /// + /// `Worker` instances are stored in the `Pool` and are never removed. + set: *const worker::Set

, +} + +unsafe impl Send for Shared

{} +unsafe impl Sync for Shared

{} + +impl

Shared

+where + P: Unpark, +{ + pub(super) fn new(unpark: P) -> Shared

{ + Shared { + unpark, + pending_drop: task::TransferStack::new(), + set: ptr::null(), + } + } + + pub(crate) fn schedule(&self, task: Task) { + self.set().schedule(task); + } + + pub(super) fn unpark(&self) { + self.unpark.unpark(); + } + + pub(super) fn set_container_ptr(&mut self, set: *const worker::Set

) { + self.set = set; + } + + fn set(&self) -> &worker::Set

{ + unsafe { &*self.set } + } +} + +impl

Schedule for Shared

+where + P: Unpark, +{ + fn bind(&self, task: &Task) { + // Get access to the Owned component. This function can only be called + // when on the worker. + unsafe { + let index = self.set().index_of(self); + let owned = &mut *self.set().owned()[index].get(); + + owned.bind_task(task); + } + } + + fn release(&self, task: Task) { + // This stores the task with the owning worker. The worker is not + // notified. Instead, the worker will clean up the tasks "eventually". + // + self.pending_drop.push(task); + } + + fn release_local(&self, task: &Task) { + // Get access to the Owned component. This function can only be called + // when on the worker. + unsafe { + let index = self.set().index_of(self); + let owned = &mut *self.set().owned()[index].get(); + + owned.release_task(task); + } + } + + fn schedule(&self, task: Task) { + Self::schedule(self, task); + } +} diff --git a/tokio-executor/src/thread_pool/shutdown.rs b/tokio-executor/src/thread_pool/shutdown.rs new file mode 100644 index 00000000000..c0609bde425 --- /dev/null +++ b/tokio-executor/src/thread_pool/shutdown.rs @@ -0,0 +1,48 @@ +//! A shutdown channel. +//! +//! Each worker holds the `Sender` half. When all the `Sender` halves are +//! dropped, the `Receiver` receives a notification. + +use crate::loom::sync::Arc; + +use tokio_sync::oneshot; + +#[derive(Debug, Clone)] +pub(super) struct Sender { + tx: Arc>, +} + +#[derive(Debug)] +pub(super) struct Receiver { + rx: oneshot::Receiver<()>, +} + +pub(super) fn channel() -> (Sender, Receiver) { + let (tx, rx) = oneshot::channel(); + let tx = Sender { tx: Arc::new(tx) }; + let rx = Receiver { rx }; + + (tx, rx) +} + +impl Receiver { + /// Block the current thread until all `Sender` handles drop. + pub(crate) fn wait(&mut self) { + use crate::enter; + + let mut e = match enter() { + Ok(e) => e, + Err(_) => { + if std::thread::panicking() { + // Already panicking, avoid a double panic + return; + } else { + panic!("cannot block on shutdown from the Tokio runtime"); + } + } + }; + + // The oneshot completes with an Err + let _ = e.block_on(&mut self.rx); + } +} diff --git a/tokio-executor/src/thread_pool/spawner.rs b/tokio-executor/src/thread_pool/spawner.rs new file mode 100644 index 00000000000..d723514f0f9 --- /dev/null +++ b/tokio-executor/src/thread_pool/spawner.rs @@ -0,0 +1,57 @@ +use crate::loom::sync::Arc; +use crate::park::Unpark; +use crate::thread_pool::{worker, JoinHandle}; + +use std::fmt; +use std::future::Future; + +/// Submit futures to the associated thread pool for execution. +/// +/// A `Spawner` instance is a handle to a single thread pool, allowing the owner +/// of the handle to spawn futures onto the thread pool. +/// +/// The `Spawner` handle is *only* used for spawning new futures. It does not +/// impact the lifecycle of the thread pool in any way. The thread pool may +/// shutdown while there are outstanding `Spawner` instances. +/// +/// `Spawner` instances are obtained by calling [`ThreadPool::spawner`]. +/// +/// [`ThreadPool::spawner`]: struct.ThreadPool.html#method.spawner +#[derive(Clone)] +pub struct Spawner { + workers: Arc>>, +} + +impl Spawner { + pub(super) fn new(workers: Arc>>) -> Spawner { + Spawner { workers } + } + + /// Spawn a future onto the thread pool + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + self.workers.spawn_typed(future) + } + + /// Spawn a task in the background + pub(super) fn spawn_background(&self, future: F) + where + F: Future + Send + 'static, + { + self.workers.spawn_background(future); + } + + /// Reference to the worker set. Used by `ThreadPool` to initiate shutdown. + pub(super) fn workers(&self) -> &worker::Set> { + &*self.workers + } +} + +impl fmt::Debug for Spawner { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Spawner").finish() + } +} diff --git a/tokio-executor/src/thread_pool/tests/loom_pool.rs b/tokio-executor/src/thread_pool/tests/loom_pool.rs new file mode 100644 index 00000000000..6256048b24b --- /dev/null +++ b/tokio-executor/src/thread_pool/tests/loom_pool.rs @@ -0,0 +1,134 @@ +use crate::loom::sync::atomic::Ordering::{Acquire, Relaxed, Release}; +use crate::loom::sync::atomic::{AtomicBool, AtomicUsize}; +use crate::loom::sync::{Arc, Mutex}; +use crate::spawn; +use crate::tests::loom_oneshot as oneshot; +use crate::thread_pool::ThreadPool; + +use std::future::Future; + +#[test] +fn pool_multi_spawn() { + loom::model(|| { + let pool = ThreadPool::new(); + + let c1 = Arc::new(AtomicUsize::new(0)); + + let (tx, rx) = oneshot::channel(); + let tx1 = Arc::new(Mutex::new(Some(tx))); + + // Spawn a task + let c2 = c1.clone(); + let tx2 = tx1.clone(); + pool.spawn(async move { + spawn(async move { + if 1 == c1.fetch_add(1, Relaxed) { + tx1.lock().unwrap().take().unwrap().send(()); + } + }); + }); + + // Spawn a second task + pool.spawn(async move { + spawn(async move { + if 1 == c2.fetch_add(1, Relaxed) { + tx2.lock().unwrap().take().unwrap().send(()); + } + }); + }); + + rx.recv(); + }); +} + +#[test] +fn pool_multi_notify() { + loom::model(|| { + let pool = ThreadPool::new(); + + let c1 = Arc::new(AtomicUsize::new(0)); + + let (done_tx, done_rx) = oneshot::channel(); + let done_tx1 = Arc::new(Mutex::new(Some(done_tx))); + + // Spawn a task + let c2 = c1.clone(); + let done_tx2 = done_tx1.clone(); + pool.spawn(async move { + gated().await; + gated().await; + + if 1 == c1.fetch_add(1, Relaxed) { + done_tx1.lock().unwrap().take().unwrap().send(()); + } + }); + + // Spawn a second task + pool.spawn(async move { + gated().await; + gated().await; + + if 1 == c2.fetch_add(1, Relaxed) { + done_tx2.lock().unwrap().take().unwrap().send(()); + } + }); + + done_rx.recv(); + }); +} + +#[test] +fn pool_shutdown() { + loom::model(|| { + let pool = ThreadPool::new(); + + pool.spawn(async move { + gated2(true).await; + }); + + drop(pool); + }); +} + +fn gated() -> impl Future { + gated2(false) +} + +fn gated2(thread: bool) -> impl Future { + use crate::loom::thread; + use futures_util::future::poll_fn; + use std::sync::Arc; + use std::task::Poll; + + let gate = Arc::new(AtomicBool::new(false)); + let mut fired = false; + + poll_fn(move |cx| { + if !fired { + let gate = gate.clone(); + let waker = cx.waker().clone(); + + if thread { + thread::spawn(move || { + gate.store(true, Release); + waker.wake_by_ref(); + }); + } else { + spawn(async move { + gate.store(true, Release); + waker.wake_by_ref(); + }); + } + + fired = true; + + return Poll::Pending; + } + + if gate.load(Acquire) { + Poll::Ready("hello world") + } else { + Poll::Pending + } + }) +} diff --git a/tokio-executor/src/thread_pool/tests/loom_queue.rs b/tokio-executor/src/thread_pool/tests/loom_queue.rs new file mode 100644 index 00000000000..1721727be4a --- /dev/null +++ b/tokio-executor/src/thread_pool/tests/loom_queue.rs @@ -0,0 +1,68 @@ +use crate::task::{self, Task}; +use crate::tests::mock_schedule::{Noop, NOOP_SCHEDULE}; +use crate::thread_pool::queue; + +use loom::thread; + +use std::cell::Cell; +use std::rc::Rc; + +#[test] +fn multi_worker() { + const THREADS: usize = 2; + const PER_THREAD: usize = 7; + + fn work(_i: usize, q: queue::Worker, rem: Rc>) { + let mut rem_local = PER_THREAD; + + while rem.get() != 0 { + for _ in 0..3 { + if rem_local > 0 { + q.push(val(0)); + rem_local -= 1; + } + } + + // Try to work + while let Some(task) = q.pop_local_first() { + assert!(task.run(From::from(&NOOP_SCHEDULE)).is_none()); + let r = rem.get(); + assert!(r > 0); + rem.set(r - 1); + } + + // Try to steal + if let Some(task) = q.steal(0) { + assert!(task.run(From::from(&NOOP_SCHEDULE)).is_none()); + let r = rem.get(); + assert!(r > 0); + rem.set(r - 1); + } + + thread::yield_now(); + } + } + + loom::model(|| { + let rem = Rc::new(Cell::new(THREADS * PER_THREAD)); + + let mut qs = queue::build(THREADS); + let q1 = qs.remove(0); + + for i in 1..THREADS { + let q = qs.remove(0); + let rem = rem.clone(); + thread::spawn(move || { + work(i, q, rem); + }); + } + + work(0, q1, rem); + + // th.join().unwrap(); + }); +} + +fn val(num: u32) -> Task { + task::background(async move { num }) +} diff --git a/tokio-executor/src/thread_pool/tests/mod.rs b/tokio-executor/src/thread_pool/tests/mod.rs new file mode 100644 index 00000000000..811393a3cfa --- /dev/null +++ b/tokio-executor/src/thread_pool/tests/mod.rs @@ -0,0 +1,11 @@ +#[cfg(loom)] +mod loom_pool; + +#[cfg(loom)] +mod loom_queue; + +#[cfg(not(loom))] +mod queue; + +#[cfg(not(loom))] +mod worker; diff --git a/tokio-executor/src/thread_pool/tests/queue.rs b/tokio-executor/src/thread_pool/tests/queue.rs new file mode 100644 index 00000000000..04cfa1536a5 --- /dev/null +++ b/tokio-executor/src/thread_pool/tests/queue.rs @@ -0,0 +1,281 @@ +use crate::task::{self, Task}; +use crate::tests::mock_schedule::{Noop, NOOP_SCHEDULE}; +use crate::thread_pool::{queue, LOCAL_QUEUE_CAPACITY}; + +macro_rules! assert_pop { + ($q:expr, $expect:expr) => { + assert_eq!( + match $q.pop_local_first() { + Some(v) => num(v), + None => panic!("queue empty"), + }, + $expect + ) + }; +} + +macro_rules! assert_pop_global { + ($q:expr, $expect:expr) => { + assert_eq!( + match $q.pop_global_first() { + Some(v) => num(v), + None => panic!("queue empty"), + }, + $expect + ) + }; +} + +macro_rules! assert_steal { + ($q:expr, $n:expr, $expect:expr) => { + assert_eq!( + match $q.steal($n) { + Some(v) => num(v), + None => panic!("queue empty"), + }, + $expect + ) + }; +} + +macro_rules! assert_empty { + ($q:expr) => {{ + let q: &mut queue::Worker = &mut $q; + match q.pop_local_first() { + Some(v) => panic!("expected emtpy queue; got {}", num(v)), + None => {} + } + }}; +} + +#[test] +fn single_worker_push_pop() { + let mut q = queue::build(1).remove(0); + + // Queue is empty + assert_empty!(q); + + // Push a value + q.push(val(0)); + + // Pop the value + assert_pop!(q, 0); + + // Push two values + q.push(val(1)); + q.push(val(2)); + q.push(val(3)); + + // Pop the value + assert_pop!(q, 3); + assert_pop!(q, 1); + assert_pop!(q, 2); + assert_empty!(q); +} + +#[test] +fn multi_worker_push_pop() { + let (mut q1, mut q2) = queues_2(); + + // Queue is empty + assert_empty!(q1); + assert_empty!(q2); + + // Push a value + q1.push(val(0)); + + // Not available on other queue + assert_empty!(q2); + assert_pop!(q1, 0); + + q2.push(val(1)); + assert_pop!(q2, 1); + assert_empty!(q1); +} + +#[test] +fn multi_worker_inject_pop() { + let (mut q1, mut q2) = queues_2(); + let i = q1.injector(); + + // Push a value + i.push(val(0), is_ok); + assert_pop!(q1, 0); + assert_empty!(q2); + + // Push another value + i.push(val(1), is_ok); + assert_pop!(q2, 1); + assert_empty!(q1); + + i.push(val(2), is_ok); + i.push(val(3), is_ok); + i.push(val(4), is_ok); + assert_pop!(q2, 2); + assert_pop!(q1, 3); + assert_pop!(q1, 4); +} + +#[test] +fn overflow_local_queue() { + let (mut q1, mut q2) = queues_2(); + + for i in 0..LOCAL_QUEUE_CAPACITY { + q1.push(val(i as u32)); + } + + assert_empty!(q2); + + // Fill `next` slot + q1.push(val(999)); + + // overflow + q1.push(val(1000)); + + assert_pop!(q2, 0); + assert_pop!(q1, 1000); + + // Half the values were moved to the global queue + for i in 128..LOCAL_QUEUE_CAPACITY { + assert_pop!(q1, i as u32); + } + + for i in 1..128 { + assert_pop!(q2, i); + } + + assert_pop!(q2, 999); + assert_empty!(q2); + + assert_empty!(q1); +} + +#[test] +fn polling_global_first() { + let (q, _) = queues_2(); + let i = q.injector(); + + i.push(val(1000), is_ok); + i.push(val(1001), is_ok); + + for n in 0..5 { + q.push(val(n)); + } + + assert_pop_global!(q, 1000); + assert_pop!(q, 4); + assert_pop_global!(q, 1001); + assert_pop_global!(q, 0); + assert_pop!(q, 1); + assert_pop_global!(q, 2); + assert_pop_global!(q, 3); + + assert!(q.pop_global_first().is_none()); +} + +#[test] +fn steal() { + let mut qs = queue::build(3); + let (mut q1, mut q2, mut q3) = (qs.remove(0), qs.remove(0), qs.remove(0)); + + assert!(q1.steal(0).is_none()); + assert!(q2.steal(0).is_none()); + assert!(q3.steal(0).is_none()); + + // Steal one value, but not the first one + q1.push(val(0)); + q1.push(val(999)); + assert_steal!(q2, 0, 0); + assert!(q2.steal(0).is_none()); + assert_pop!(q1, 999); + + // Steals half the queue + for i in 0..4 { + q1.push(val(i)); + } + + q1.push(val(999)); + + assert_steal!(q2, 0, 1); + assert_pop!(q2, 0); + assert_empty!(q2); + assert_pop!(q1, 999); + assert_pop!(q1, 2); + assert_pop!(q1, 3); + assert_empty!(q1); + + // Searches multiple queues + q3.push(val(0)); + q3.push(val(999)); + assert_steal!(q2, 0, 0); + assert_pop!(q3, 999); + assert_empty!(q3); + + // Steals from one queue at a time + q1.push(val(0)); + q1.push(val(998)); + q2.push(val(1)); + q2.push(val(999)); + + assert_steal!(q3, 0, 0); + assert_pop!(q2, 999); + assert_pop!(q2, 1); + assert_empty!(q2); + + assert_pop!(q1, 998); + assert_empty!(q1); +} + +fn queues_2() -> (queue::Worker, queue::Worker) { + let mut qs = queue::build(2); + (qs.remove(0), qs.remove(0)) +} + +// pretty big hack to track tasks +use std::cell::RefCell; +use std::collections::HashMap; +thread_local! { + static TASKS: RefCell>> = RefCell::new(HashMap::new()) +} + +fn val(num: u32) -> Task { + let (task, join) = task::joinable(async move { num }); + let prev = TASKS.with(|t| t.borrow_mut().insert(num, join)); + assert!(prev.is_none()); + task +} + +fn num(task: Task) -> u32 { + use futures_util::task::noop_waker_ref; + use std::future::Future; + use std::pin::Pin; + use std::task::Context; + use std::task::Poll::*; + + assert!(task.run(From::from(&NOOP_SCHEDULE)).is_none()); + + // Find the task that completed + TASKS.with(|c| { + let mut map = c.borrow_mut(); + let mut num = None; + + for (_, join) in map.iter_mut() { + let mut cx = Context::from_waker(noop_waker_ref()); + match Pin::new(join).poll(&mut cx) { + Ready(n) => { + num = Some(n.unwrap()); + break; + } + _ => {} + } + } + + let num = num.expect("no task completed"); + map.remove(&num); + num + }) +} + +fn is_ok(r: Result) { + assert!(r.is_ok()) +} diff --git a/tokio-executor/src/thread_pool/tests/worker.rs b/tokio-executor/src/thread_pool/tests/worker.rs new file mode 100644 index 00000000000..6ff0c7788c0 --- /dev/null +++ b/tokio-executor/src/thread_pool/tests/worker.rs @@ -0,0 +1,66 @@ +use crate::tests::track_drop::track_drop; +use crate::thread_pool; + +use tokio_test::assert_ok; + +macro_rules! pool { + (2) => {{ + let (pool, mut w, mock_park) = pool!(!2); + (pool, w.remove(0), w.remove(0), mock_park) + }}; + (! $n:expr) => {{ + let mut mock_park = crate::tests::mock_park::MockPark::new(); + let (pool, workers) = thread_pool::create_pool($n, |index| mock_park.mk_park(index)); + (pool, workers, mock_park) + }}; +} + +macro_rules! enter { + ($w:expr, $expr:expr) => {{ + $w.enter(move || $expr); + }}; +} + +#[test] +fn execute_single_task() { + use std::sync::mpsc; + + let (p, mut w0, _w1, ..) = pool!(2); + let (tx, rx) = mpsc::channel(); + + enter!(w0, p.spawn_background(async move { tx.send(1).unwrap() })); + + w0.tick(); + + assert_ok!(rx.try_recv()); +} + +#[test] +fn task_migrates() { + use std::sync::mpsc; + use tokio::sync::oneshot; + + let (p, mut w0, mut w1, ..) = pool!(2); + let (tx1, rx1) = oneshot::channel(); + let (tx2, rx2) = mpsc::channel(); + + let (task, did_drop) = track_drop(async move { + let msg = rx1.await.unwrap(); + tx2.send(msg).unwrap(); + }); + + enter!(w0, p.spawn_background(task)); + + w0.tick(); + w1.enter(|| tx1.send("hello").unwrap()); + + w1.tick(); + assert_ok!(rx2.try_recv()); + + // Future drops immediately even though the underlying task is not freed + assert!(did_drop.did_drop_future()); + assert!(did_drop.did_drop_output()); + + // Tick the spawning worker in order to free memory + w0.tick(); +} diff --git a/tokio-executor/src/thread_pool/worker.rs b/tokio-executor/src/thread_pool/worker.rs new file mode 100644 index 00000000000..bdf430d3751 --- /dev/null +++ b/tokio-executor/src/thread_pool/worker.rs @@ -0,0 +1,394 @@ +use crate::loom::sync::Arc; +use crate::park::{Park, Unpark}; +use crate::task::Task; +use crate::thread_pool::{current, Owned, Shared}; + +use std::time::Duration; + +// TODO: remove this re-export +pub(super) use crate::thread_pool::set::Set; + +pub(crate) struct Worker { + /// Entry in the set of workers. + entry: Entry, + + /// Park the thread + park: P, +} + +struct Entry { + pool: Arc>, + index: usize, +} + +pub(crate) fn create_set( + pool_size: usize, + mk_park: F, +) -> (Arc>, Vec>) +where + P: Park, + F: FnMut(usize) -> P, +{ + // Create the parks... + let parks: Vec<_> = (0..pool_size).map(mk_park).collect(); + + let mut pool = Arc::new(Set::new(pool_size, |i| parks[i].unpark())); + + // Establish the circular link between the individual worker state + // structure and the container. + Arc::get_mut(&mut pool).unwrap().set_container_ptr(); + + // This will contain each worker. + let workers = parks + .into_iter() + .enumerate() + .map(|(index, park)| Worker::new(pool.clone(), index, park)) + .collect(); + + (pool, workers) +} + +/// After how many ticks is the global queue polled. This helps to ensure +/// fairness. +/// +/// The number is fairly arbitrary. I believe this value was copied from golang. +const GLOBAL_POLL_INTERVAL: u16 = 61; + +impl

Worker

+where + P: Park + 'static, +{ + pub(super) fn new(pool: Arc>, index: usize, park: P) -> Self { + Worker { + entry: Entry { pool, index }, + park, + } + } + + pub(super) fn run(&mut self) { + let mut executor = &*self.entry.pool; + let entry = &self.entry; + let park = &mut self.park; + + // Track the current worker + current::set(&entry.pool, entry.index, || { + let _enter = crate::enter().expect("executor already running on thread"); + + crate::with_default(&mut executor, || { + entry.run(park); + }) + }) + } + + #[cfg(test)] + #[allow(warnings)] + pub(crate) fn enter(&self, f: F) -> R + where + F: FnOnce() -> R, + { + current::set(&self.entry.pool, self.entry.index, f) + } + + #[cfg(test)] + #[allow(warnings)] + pub(crate) fn tick(&mut self) { + self.entry.tick(&mut self.park); + } +} + +impl

Entry

+where + P: Unpark, +{ + fn run(&self, park: &mut impl Park) { + while self.is_running() { + if self.tick(park) { + self.park(park); + } + } + + self.shutdown(park); + } + + fn is_running(&self) -> bool { + self.owned().is_running.get() + } + + /// Returns `true` if the worker needs to park + fn tick(&self, park: &mut impl Park) -> bool { + // Process all pending tasks in the local queue. + if !self.process_local_queue(park) { + return false; + } + + // No more **local** work to process, try transitioning to searching + // in order to attempt to steal work from other workers. + // + // On `false`, the worker has entered the parked state + if self.transition_to_searching() { + // If `true` then work was found + if self.search_for_work() { + return false; + } + } + + true + } + + /// Process all pending tasks in the local queue, occasionally checking the + /// global queue, but never other worker local queues. + /// + /// Returns `false` if processing was interrupted due to the pool shutting + /// down. + fn process_local_queue(&self, park: &mut impl Park) -> bool { + debug_assert!(self.is_running()); + + loop { + let tick = self.tick_fetch_inc(); + + let task = if tick % GLOBAL_POLL_INTERVAL == 0 { + // Sleep light... + self.park_light(park); + + // Perform regularly scheduled maintenance work. + self.maintenance(); + + if !self.is_running() { + return false; + } + + // Check the global queue + self.owned().work_queue.pop_global_first() + } else { + self.owned().work_queue.pop_local_first() + }; + + if let Some(task) = task { + self.run_task(task); + } else { + return true; + } + } + } + + fn steal_work(&self) -> Option>> { + let num_workers = self.pool.len(); + let start = self.owned().rand.fastrand_n(num_workers as u32); + + self.owned() + .work_queue + .steal(start as usize) + // Fallback on checking the local queue, which will also check the + // injector. + .or_else(|| self.owned().work_queue.pop_global_first()) + } + + /// Runs maintenance work such as free pending tasks and check the pool's + /// state. + fn maintenance(&self) { + // Free any completed tasks + self.drain_tasks_pending_drop(); + + // Update the pool state cache + self.owned() + .is_running + .set(!self.owned().work_queue.is_closed()); + } + + fn search_for_work(&self) -> bool { + debug_assert!(self.is_searching()); + + if let Some(task) = self.steal_work() { + self.run_task(task); + true + } else { + // Perform some routine work + self.drain_tasks_pending_drop(); + false + } + } + + fn transition_to_searching(&self) -> bool { + if self.is_searching() { + return true; + } + + let ret = self.set().idle().transition_worker_to_searching(); + self.owned().is_searching.set(ret); + ret + } + + fn transition_from_searching(&self) { + debug_assert!(self.is_searching()); + + self.owned().is_searching.set(false); + + if self.set().idle().transition_worker_from_searching() { + // We are the final searching worker. Because work was found, we + // need to notify another worker. + self.set().notify_work(); + } + } + + /// Returns `true` if the worker must check for any work. + fn transition_to_parked(&self) -> bool { + let ret = self + .set() + .idle() + .transition_worker_to_parked(self.index, self.is_searching()); + + // The worker is no longer searching. Setting this is the local cache + // only. + self.owned().is_searching.set(false); + + // When tasks are submitted locally (from the parker), defer any + // notifications in hopes that the curent worker will grab those tasks. + self.owned().defer_notification.set(true); + + ret + } + + /// Returns `true` if the transition happened. + fn transition_from_parked(&self) -> bool { + if self.owned().did_submit_task.get() || !self.is_running() { + // Remove the worker from the sleep set. + self.set().idle().unpark_worker_by_id(self.index); + + self.owned().is_searching.set(true); + self.owned().defer_notification.set(false); + + true + } else { + let ret = !self.set().idle().is_parked(self.index); + + if ret { + self.owned().is_searching.set(true); + self.owned().defer_notification.set(false); + } + + ret + } + } + + fn run_task(&self, task: Task>) { + if self.is_searching() { + self.transition_from_searching(); + } + + if let Some(task) = task.run(self.shared().into()) { + self.owned().submit_local_yield(task); + self.set().notify_work(); + } + } + + fn final_work_sweep(&self) { + if !self.owned().work_queue.is_empty() { + self.set().notify_work(); + } + } + + fn park(&self, park: &mut impl Park) { + if self.transition_to_parked() { + // We are the final searching worker, check if any work arrived + // before parking + self.final_work_sweep(); + } + + // The state has been transitioned to parked, we can now wait by + // calling the parker. This is done in a loop as spurious wakeups are + // permitted. + loop { + park.park().ok().expect("park failed"); + + // We might have been woken to clean up a dropped task + self.maintenance(); + + if self.transition_from_parked() { + return; + } + } + } + + fn park_light(&self, park: &mut impl Park) { + // When tasks are submitted locally (from the parker), defer any + // notifications in hopes that the curent worker will grab those tasks. + self.owned().defer_notification.set(true); + + park.park_timeout(Duration::from_millis(0)) + .ok() + .expect("park failed"); + + self.owned().defer_notification.set(false); + + if self.owned().did_submit_task.get() { + self.set().notify_work(); + self.owned().did_submit_task.set(false) + } + } + + fn drain_tasks_pending_drop(&self) { + for task in self.shared().pending_drop.drain() { + unsafe { + let owned = &mut *self.set().owned()[self.index].get(); + owned.release_task(&task); + } + drop(task); + } + } + + /// Shutdown the worker. + /// + /// Once the shutdown flag has been observed, it is guaranteed that no + /// further tasks may be pushed into the global queue. + fn shutdown(&self, park: &mut impl Park) { + // Transition all tasks owned by the worker to canceled. + self.owned().owned_tasks.shutdown(); + + // First, drain all tasks from both the local & global queue. + while let Some(task) = self.owned().work_queue.pop_local_first() { + task.shutdown(); + } + + // Notify all workers in case they have pending tasks to drop + // + // Not super efficient, but we are also shutting down. + self.pool.notify_all(); + + // The worker can only shutdown once there are no further owned tasks. + while !self.owned().owned_tasks.is_empty() { + // Wait until task that this worker owns are released. + // + // `transition_to_parked` is not called as we are not working + // anymore. When a task is released, the owning worker is unparked + // directly. + park.park().ok().expect("park failed"); + + // Try draining more tasks + self.drain_tasks_pending_drop(); + } + } + + /// Increment the tick, returning the value from before the increment. + fn tick_fetch_inc(&self) -> u16 { + let tick = self.owned().tick.get(); + self.owned().tick.set(tick.wrapping_add(1)); + tick + } + + fn is_searching(&self) -> bool { + self.owned().is_searching.get() + } + + fn set(&self) -> &Set

{ + &self.pool + } + + fn shared(&self) -> &Shared

{ + &self.set().shared()[self.index] + } + + fn owned(&self) -> &Owned

{ + // safety: we own the slot + unsafe { &*self.set().owned()[self.index].get() } + } +} diff --git a/tokio-executor/src/threadpool/blocking.rs b/tokio-executor/src/threadpool/blocking.rs deleted file mode 100644 index ebe20747e14..00000000000 --- a/tokio-executor/src/threadpool/blocking.rs +++ /dev/null @@ -1,174 +0,0 @@ -use super::worker::Worker; - -use futures_core::ready; -use std::error::Error; -use std::fmt; -use std::task::Poll; - -/// Error raised by `blocking`. -pub struct BlockingError { - _p: (), -} - -/// Enter a blocking section of code. -/// -/// The `blocking` function annotates a section of code that performs a blocking -/// operation, either by issuing a blocking syscall or by performing a long -/// running CPU-bound computation. -/// -/// When the `blocking` function enters, it hands off the responsibility of -/// processing the current work queue to another thread. Then, it calls the -/// supplied closure. The closure is permitted to block indefinitely. -/// -/// If the maximum number of concurrent `blocking` calls has been reached, then -/// `NotReady` is returned and the task is notified once existing `blocking` -/// calls complete. The maximum value is specified when creating a thread pool -/// using [`Builder::max_blocking`][build] -/// -/// NB: The entire task that called `blocking` is blocked whenever the supplied -/// closure blocks, even if you have used future combinators such as `select` - -/// the other futures in this task will not make progress until the closure -/// returns. -/// If this is not desired, ensure that `blocking` runs in its own task (e.g. -/// using `futures::sync::oneshot::spawn`). -/// -/// [build]: struct.Builder.html#method.max_blocking -/// -/// # Return -/// -/// When the blocking closure is executed, `Ok(Ready(T))` is returned, where -/// `T` is the closure's return value. -/// -/// If the thread pool has shutdown, `Err` is returned. -/// -/// If the number of concurrent `blocking` calls has reached the maximum, -/// `Ok(NotReady)` is returned and the current task is notified when a call to -/// `blocking` will succeed. -/// -/// If `blocking` is called from outside the context of a Tokio thread pool, -/// `Err` is returned. -/// -/// # Background -/// -/// By default, the Tokio thread pool expects that tasks will only run for short -/// periods at a time before yielding back to the thread pool. This is the basic -/// premise of cooperative multitasking. -/// -/// However, it is common to want to perform a blocking operation while -/// processing an asynchronous computation. Examples of blocking operation -/// include: -/// -/// * Performing synchronous file operations (reading and writing). -/// * Blocking on acquiring a mutex. -/// * Performing a CPU bound computation, like cryptographic encryption or -/// decryption. -/// -/// One option for dealing with blocking operations in an asynchronous context -/// is to use a thread pool dedicated to performing these operations. This not -/// ideal as it requires bidirectional message passing as well as a channel to -/// communicate which adds a level of buffering. -/// -/// Instead, `blocking` hands off the responsibility of processing the work queue -/// to another thread. This hand off is light compared to a channel and does not -/// require buffering. -/// -/// # Examples -/// -/// Block on receiving a message from a `std` channel. This example is a little -/// silly as using the non-blocking channel from the `futures` crate would make -/// more sense. The blocking receive can be replaced with any blocking operation -/// that needs to be performed. -/// -/// ```rust -/// use tokio_executor::threadpool::{ThreadPool, blocking}; -/// -/// use futures_util::future::poll_fn; -/// use std::sync::mpsc; -/// use std::thread; -/// use std::time::Duration; -/// -/// pub fn main() { -/// // This is a *blocking* channel -/// let (tx, rx) = mpsc::channel(); -/// -/// // Spawn a thread to send a message -/// thread::spawn(move || { -/// thread::sleep(Duration::from_millis(500)); -/// tx.send("hello").unwrap(); -/// }); -/// -/// let pool = ThreadPool::new(); -/// -/// pool.spawn(async move { -/// // Because `blocking` returns `Poll`, it is intended to be used -/// // from the context of a `Future` implementation. Since we don't -/// // have a complicated requirement, we can use `poll_fn` in this -/// // case. -/// let _ = poll_fn(move |_| { -/// blocking(|| { -/// let msg = rx.recv().unwrap(); -/// println!("message = {}", msg); -/// }).map_err(|_| panic!("the threadpool shut down")) -/// }).await; -/// }); -/// -/// // Wait for the task we just spawned to complete. -/// pool.shutdown_on_idle().wait(); -/// } -/// ``` -pub fn blocking(f: F) -> Poll> -where - F: FnOnce() -> T, -{ - let res = Worker::with_current(|worker| { - let worker = match worker { - Some(worker) => worker, - None => { - return Poll::Ready(Err(BlockingError { _p: () })); - } - }; - - // Transition the worker state to blocking. This will exit the fn early - // with `NotReady` if the pool does not have enough capacity to enter - // blocking mode. - worker.transition_to_blocking() - }); - - // If the transition cannot happen, exit early - ready!(res)?; - - // Currently in blocking mode, so call the inner closure - // - // "Exit" the current executor in case the blocking function wants - // to call a different executor. - let ret = crate::exit(move || f()); - - // Try to transition out of blocking mode. This is a fast path that takes - // back ownership of the worker if the worker handoff didn't complete yet. - Worker::with_current(|worker| { - // Worker must be set since it was above. - worker.unwrap().transition_from_blocking(); - }); - - // Return the result - Poll::Ready(Ok(ret)) -} - -impl fmt::Display for BlockingError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - fmt, - "`blocking` annotation used from outside the context of a thread pool" - ) - } -} - -impl fmt::Debug for BlockingError { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("BlockingError") - .field("reason", &format!("{}", self)) - .finish() - } -} - -impl Error for BlockingError {} diff --git a/tokio-executor/src/threadpool/builder.rs b/tokio-executor/src/threadpool/builder.rs deleted file mode 100644 index 34d4c682b9a..00000000000 --- a/tokio-executor/src/threadpool/builder.rs +++ /dev/null @@ -1,426 +0,0 @@ -use super::callback::Callback; -use super::config::{Config, MAX_WORKERS}; -use super::park::{BoxPark, BoxedPark, DefaultPark}; -use super::pool::{Pool, MAX_BACKUP}; -use super::shutdown::ShutdownTrigger; -use super::thread_pool::ThreadPool; -use super::worker::{self, Worker, WorkerId}; -use crate::park::Park; - -use crossbeam_deque::Injector; -use num_cpus; -use std::any::Any; -use std::cmp::max; -use std::error::Error; -use std::fmt; -use std::sync::Arc; -use std::time::Duration; - -/// Builds a thread pool with custom configuration values. -/// -/// Methods can be chained in order to set the configuration values. The thread -/// pool is constructed by calling [`build`]. -/// -/// New instances of `Builder` are obtained via [`Builder::new`]. -/// -/// See function level documentation for details on the various configuration -/// settings. -/// -/// [`build`]: #method.build -/// [`Builder::new`]: #method.new -/// -/// # Examples -/// -/// ``` -/// use tokio_executor::threadpool::Builder; -/// -/// use std::time::Duration; -/// -/// let thread_pool = Builder::new() -/// .pool_size(4) -/// .keep_alive(Some(Duration::from_secs(30))) -/// .build(); -/// -/// thread_pool.spawn(async { -/// println!("called from a worker thread"); -/// }); -/// -/// // Gracefully shutdown the threadpool -/// thread_pool.shutdown().wait(); -/// ``` -pub struct Builder { - /// Thread pool specific configuration values - config: Config, - - /// Number of workers to spawn - pool_size: usize, - - /// Maximum number of futures that can be in a blocking section - /// concurrently. - max_blocking: usize, - - /// Generates the `Park` instances - new_park: Box BoxPark>, -} - -impl Builder { - /// Returns a new thread pool builder initialized with default configuration - /// values. - /// - /// Configuration methods can be chained on the return value. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// use std::time::Duration; - /// - /// let thread_pool = Builder::new() - /// .pool_size(4) - /// .keep_alive(Some(Duration::from_secs(30))) - /// .build(); - /// ``` - pub fn new() -> Builder { - let num_cpus = max(1, num_cpus::get()); - - let new_park = - Box::new(|_: &WorkerId| Box::new(BoxedPark::new(DefaultPark::new())) as BoxPark); - - Builder { - pool_size: num_cpus, - max_blocking: 100, - config: Config { - keep_alive: None, - name_prefix: None, - stack_size: None, - around_worker: None, - after_start: None, - before_stop: None, - panic_handler: None, - }, - new_park, - } - } - - /// Set the maximum number of worker threads for the thread pool instance. - /// - /// This must be a number between 1 and 32,768 though it is advised to keep - /// this value on the smaller side. - /// - /// The default value is the number of cores available to the system. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .pool_size(4) - /// .build(); - /// ``` - pub fn pool_size(&mut self, val: usize) -> &mut Self { - assert!(val >= 1, "at least one thread required"); - assert!(val <= MAX_WORKERS, "max value is {}", MAX_WORKERS); - - self.pool_size = val; - self - } - - /// Set the maximum number of concurrent blocking sections. - /// - /// When the maximum concurrent `blocking` calls is reached, any further - /// calls to `blocking` will return `NotReady` and the task is notified once - /// previously in-flight calls to `blocking` return. - /// - /// This must be a number between 1 and 32,768 though it is advised to keep - /// this value on the smaller side. - /// - /// The default value is 100. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .max_blocking(200) - /// .build(); - /// ``` - pub fn max_blocking(&mut self, val: usize) -> &mut Self { - assert!(val <= MAX_BACKUP, "max value is {}", MAX_BACKUP); - self.max_blocking = val; - self - } - - /// Set the thread keep alive duration - /// - /// If set, a thread that has completed a `blocking` call will wait for up - /// to the specified duration to become a worker thread again. Once the - /// duration elapses, the thread will shutdown. - /// - /// When the value is `None`, the thread will wait to become a worker - /// thread forever. - /// - /// The default value is `None`. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// use std::time::Duration; - /// - /// let thread_pool = Builder::new() - /// .keep_alive(Some(Duration::from_secs(30))) - /// .build(); - /// ``` - pub fn keep_alive(&mut self, val: Option) -> &mut Self { - self.config.keep_alive = val; - self - } - - /// Sets a callback to be triggered when a panic during a future bubbles up - /// to Tokio. By default Tokio catches these panics, and they will be - /// ignored. The parameter passed to this callback is the same error value - /// returned from std::panic::catch_unwind(). To abort the process on - /// panics, use std::panic::resume_unwind() in this callback as shown - /// below. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .panic_handler(|err| std::panic::resume_unwind(err)) - /// .build(); - /// ``` - pub fn panic_handler(&mut self, f: F) -> &mut Self - where - F: Fn(Box) + Send + Sync + 'static, - { - self.config.panic_handler = Some(Arc::new(f)); - self - } - - /// Set name prefix of threads spawned by the scheduler - /// - /// Thread name prefix is used for generating thread names. For example, if - /// prefix is `my-pool-`, then threads in the pool will get names like - /// `my-pool-1` etc. - /// - /// If this configuration is not set, then the thread will use the system - /// default naming scheme. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .name_prefix("my-pool-") - /// .build(); - /// ``` - pub fn name_prefix>(&mut self, val: S) -> &mut Self { - self.config.name_prefix = Some(val.into()); - self - } - - /// Set the stack size (in bytes) for worker threads. - /// - /// The actual stack size may be greater than this value if the platform - /// specifies minimal stack size. - /// - /// The default stack size for spawned threads is 2 MiB, though this - /// particular stack size is subject to change in the future. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .stack_size(32 * 1024) - /// .build(); - /// ``` - pub fn stack_size(&mut self, val: usize) -> &mut Self { - self.config.stack_size = Some(val); - self - } - - /// Execute function `f` on each worker thread. - /// - /// This function is provided a handle to the worker and is expected to call - /// [`Worker::run`], otherwise the worker thread will shutdown without doing - /// any work. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .around_worker(|worker| { - /// println!("worker is starting up"); - /// worker.run(); - /// println!("worker is shutting down"); - /// }) - /// .build(); - /// ``` - /// - /// [`Worker::run`]: struct.Worker.html#method.run - pub fn around_worker(&mut self, f: F) -> &mut Self - where - F: Fn(&Worker) + Send + Sync + 'static, - { - self.config.around_worker = Some(Callback::new(f)); - self - } - - /// Execute function `f` after each thread is started but before it starts - /// doing work. - /// - /// This is intended for bookkeeping and monitoring use cases. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .after_start(|| { - /// println!("thread started"); - /// }) - /// .build(); - /// ``` - pub fn after_start(&mut self, f: F) -> &mut Self - where - F: Fn() + Send + Sync + 'static, - { - self.config.after_start = Some(Arc::new(f)); - self - } - - /// Execute function `f` before each thread stops. - /// - /// This is intended for bookkeeping and monitoring use cases. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .before_stop(|| { - /// println!("thread stopping"); - /// }) - /// .build(); - /// ``` - pub fn before_stop(&mut self, f: F) -> &mut Self - where - F: Fn() + Send + Sync + 'static, - { - self.config.before_stop = Some(Arc::new(f)); - self - } - - /// Customize the `park` instance used by each worker thread. - /// - /// The provided closure `f` is called once per worker and returns a `Park` - /// instance that is used by the worker to put itself to sleep. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// use tokio_executor::threadpool::park::DefaultPark; - /// # fn decorate(f: F) -> F { f } - /// - /// let thread_pool = Builder::new() - /// .custom_park(|_| { - /// // This is the default park type that the worker would use if we - /// // did not customize it. - /// let park = DefaultPark::new(); - /// - /// // Decorate the `park` instance, allowing us to customize work - /// // that happens when a worker thread goes to sleep. - /// decorate(park) - /// }) - /// .build(); - /// ``` - pub fn custom_park(&mut self, f: F) -> &mut Self - where - F: Fn(&WorkerId) -> P + 'static, - P: Park + Send + 'static, - P::Error: Error, - { - self.new_park = Box::new(move |id| Box::new(BoxedPark::new(f(id)))); - - self - } - - /// Create the configured `ThreadPool`. - /// - /// The returned `ThreadPool` instance is ready to spawn tasks. - /// - /// # Examples - /// - /// ``` - /// use tokio_executor::threadpool::Builder; - /// - /// let thread_pool = Builder::new() - /// .build(); - /// ``` - pub fn build(&self) -> ThreadPool { - trace!(message = "build;", num_workers = self.pool_size); - - // Create the worker entry list - let workers: Arc<[worker::Entry]> = { - let mut workers = vec![]; - - for i in 0..self.pool_size { - let id = WorkerId::new(i); - let park = (self.new_park)(&id); - let unpark = park.unpark(); - - workers.push(worker::Entry::new(park, unpark)); - } - - workers.into() - }; - - let queue = Arc::new(Injector::new()); - - // Create a trigger that will clean up resources on shutdown. - // - // The `Pool` contains a weak reference to it, while `Worker`s and the `ThreadPool` contain - // strong references. - let trigger = Arc::new(ShutdownTrigger::new(workers.clone(), queue.clone())); - - // Create the pool - let pool = Arc::new(Pool::new( - workers, - Arc::downgrade(&trigger), - self.max_blocking, - self.config.clone(), - queue, - )); - - ThreadPool::new2(pool, trigger) - } -} - -impl fmt::Debug for Builder { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Builder") - .field("config", &self.config) - .field("pool_size", &self.pool_size) - .field("new_park", &"Box BoxPark>") - .finish() - } -} - -impl Default for Builder { - fn default() -> Self { - Self::new() - } -} diff --git a/tokio-executor/src/threadpool/callback.rs b/tokio-executor/src/threadpool/callback.rs deleted file mode 100644 index b3a47ddab4c..00000000000 --- a/tokio-executor/src/threadpool/callback.rs +++ /dev/null @@ -1,28 +0,0 @@ -use super::worker::Worker; - -use std::fmt; -use std::sync::Arc; - -#[derive(Clone)] -pub(crate) struct Callback { - f: Arc, -} - -impl Callback { - pub(crate) fn new(f: F) -> Self - where - F: Fn(&Worker) + Send + Sync + 'static, - { - Callback { f: Arc::new(f) } - } - - pub(crate) fn call(&self, worker: &Worker) { - (self.f)(worker) - } -} - -impl fmt::Debug for Callback { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "Fn") - } -} diff --git a/tokio-executor/src/threadpool/config.rs b/tokio-executor/src/threadpool/config.rs deleted file mode 100644 index cd34dbbb30d..00000000000 --- a/tokio-executor/src/threadpool/config.rs +++ /dev/null @@ -1,37 +0,0 @@ -use super::callback::Callback; - -use std::any::Any; -use std::fmt; -use std::sync::Arc; -use std::time::Duration; - -/// Thread pool specific configuration values -#[derive(Clone)] -pub(crate) struct Config { - pub(crate) keep_alive: Option, - // Used to configure a worker thread - pub(crate) name_prefix: Option, - pub(crate) stack_size: Option, - pub(crate) around_worker: Option, - pub(crate) after_start: Option>, - pub(crate) before_stop: Option>, - pub(crate) panic_handler: Option, -} - -// Define type alias to avoid clippy::type_complexity. -type PanicHandler = Arc) + Send + Sync>; - -/// Max number of workers that can be part of a pool. This is the most that can -/// fit in the scheduler state. Note, that this is the max number of **active** -/// threads. There can be more standby threads. -pub(crate) const MAX_WORKERS: usize = 1 << 15; - -impl fmt::Debug for Config { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Config") - .field("keep_alive", &self.keep_alive) - .field("name_prefix", &self.name_prefix) - .field("stack_size", &self.stack_size) - .finish() - } -} diff --git a/tokio-executor/src/threadpool/mod.rs b/tokio-executor/src/threadpool/mod.rs deleted file mode 100644 index 47b52a93dbb..00000000000 --- a/tokio-executor/src/threadpool/mod.rs +++ /dev/null @@ -1,146 +0,0 @@ -//! A work-stealing based thread pool for executing futures. -//! -//! The Tokio thread pool supports scheduling futures and processing them on -//! multiple CPU cores. It is optimized for the primary Tokio use case of many -//! independent tasks with limited computation and with most tasks waiting on -//! I/O. Usually, users will not create a `ThreadPool` instance directly, but -//! will use one via a [`runtime`]. -//! -//! The `ThreadPool` structure manages two sets of threads: -//! -//! * Worker threads. -//! * Backup threads. -//! -//! Worker threads are used to schedule futures using a work-stealing strategy. -//! Backup threads, on the other hand, are intended only to support the -//! `blocking` API. Threads will transition between the two sets. -//! -//! The advantage of the work-stealing strategy is minimal cross-thread -//! coordination. The thread pool attempts to make as much progress as possible -//! without communicating across threads. -//! -//! ## Worker overview -//! -//! Each worker has two queues: a deque and a mpsc channel. The deque is the -//! primary queue for tasks that are scheduled to run on the worker thread. Tasks -//! can only be pushed onto the deque by the worker, but other workers may -//! "steal" from that deque. The mpsc channel is used to submit futures while -//! external to the pool. -//! -//! As long as the thread pool has not been shutdown, a worker will run in a -//! loop. Each loop, it consumes all tasks on its mpsc channel and pushes it onto -//! the deque. It then pops tasks off of the deque and executes them. -//! -//! If a worker has no work, i.e., both queues are empty. It attempts to steal. -//! To do this, it randomly scans other workers' deques and tries to pop a task. -//! If it finds no work to steal, the thread goes to sleep. -//! -//! When the worker detects that the pool has been shut down, it exits the loop, -//! cleans up its state, and shuts the thread down. -//! -//! ## Thread pool initialization -//! -//! Note, users normally will use the threadpool created by a [`runtime`]. -//! -//! By default, no threads are spawned on creation. Instead, when new futures are -//! spawned, the pool first checks if there are enough active worker threads. If -//! not, a new worker thread is spawned. -//! -//! ## Spawning futures -//! -//! The spawning behavior depends on whether a future was spawned from within a -//! worker or thread or if it was spawned from an external handle. -//! -//! When spawning a future while external to the thread pool, the current -//! strategy is to randomly pick a worker to submit the task to. The task is then -//! pushed onto that worker's mpsc channel. -//! -//! When spawning a future while on a worker thread, the task is pushed onto the -//! back of the current worker's deque. -//! -//! ## Blocking annotation strategy -//! -//! The [`blocking`] function is used to annotate a section of code that -//! performs a blocking operation, either by issuing a blocking syscall or -//! performing any long running CPU-bound computation. -//! -//! The strategy for handling blocking closures is to hand off the worker to a -//! new thread. This implies handing off the `deque` and `mpsc`. Once this is -//! done, the new thread continues to process the work queue and the original -//! thread is able to block. Once it finishes processing the blocking future, the -//! thread has no additional work and is inserted into the backup pool. This -//! makes it available to other workers that encounter a [`blocking`] call. -//! -//! [`blocking`]: fn.blocking.html -//! [`runtime`]: https://docs.rs/tokio/0.1/tokio/runtime/ - -// ## Crate layout -// -// The primary type, `Pool`, holds the majority of a thread pool's state, -// including the state for each worker. Each worker's state is maintained in an -// instance of `worker::Entry`. -// -// `Worker` contains the logic that runs on each worker thread. It holds an -// `Arc` to `Pool` and is able to access its state from `Pool`. -// -// `Task` is a harness around an individual future. It manages polling and -// scheduling that future. -// -// ## Sleeping workers -// -// Sleeping workers are tracked using a [Treiber stack]. This results in the -// thread that most recently went to sleep getting woken up first. When the pool -// is not under load, this helps threads shutdown faster. -// -// Sleeping is done by using `tokio_executor::Park` implementations. This allows -// the user of the thread pool to customize the work that is performed to sleep. -// This is how injecting timers and other functionality into the thread pool is -// done. -// -// ## Notifying workers -// -// When there is work to be done, workers must be notified. However, notifying a -// worker requires cross thread coordination. Ideally, a worker would only be -// notified when it is sleeping, but there is no way to know if a worker is -// sleeping without cross thread communication. -// -// The two cases when a worker might need to be notified are: -// -// 1. A task is externally submitted to a worker via the mpsc channel. -// 2. A worker has a back log of work and needs other workers to steal from it. -// -// In the first case, the worker will always be notified. However, it could be -// possible to avoid the notification if the mpsc channel has two or greater -// number of tasks *after* the task is submitted. In this case, we are able to -// assume that the worker has previously been notified. -// -// The second case is trickier. Currently, whenever a worker spawns a new future -// (pushing it onto its deque) and when it pops a future from its mpsc, it tries -// to notify a sleeping worker to wake up and start stealing. This is a lot of -// notification and it **might** be possible to reduce it. -// -// Also, whenever a worker is woken up via a signal and it does find work, it, -// in turn, will try to wake up a new worker. -// -// [Treiber stack]: https://en.wikipedia.org/wiki/Treiber_Stack - -pub mod park; - -mod blocking; -mod builder; -mod callback; -mod config; -mod pool; -mod sender; -mod shutdown; -mod task; -mod thread_pool; -mod waker; -mod worker; - -pub use self::blocking::{blocking, BlockingError}; -pub use self::builder::Builder; -pub use self::sender::Sender; -pub use self::shutdown::Shutdown; -pub use self::thread_pool::ThreadPool; -pub use self::worker::{Worker, WorkerId}; diff --git a/tokio-executor/src/threadpool/park/boxed.rs b/tokio-executor/src/threadpool/park/boxed.rs deleted file mode 100644 index 14dbc363e61..00000000000 --- a/tokio-executor/src/threadpool/park/boxed.rs +++ /dev/null @@ -1,46 +0,0 @@ -use crate::park::{Park, Unpark}; - -use std::error::Error; -use std::time::Duration; - -pub(crate) type BoxPark = Box + Send>; -pub(crate) type BoxUnpark = Box; - -pub(crate) struct BoxedPark(T); - -impl BoxedPark { - pub(crate) fn new(inner: T) -> Self { - BoxedPark(inner) - } -} - -impl Park for BoxedPark -where - T::Error: Error, -{ - type Unpark = BoxUnpark; - type Error = (); - - fn unpark(&self) -> Self::Unpark { - Box::new(self.0.unpark()) - } - - fn park(&mut self) -> Result<(), Self::Error> { - self.0.park().map_err(|_e| { - // if tracing is disabled, the compiler will flag this as unused. - warn!( - message = "calling `park` on worker thread errored -- shutting down thread", - error = %_e - ); - }) - } - - fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { - self.0.park_timeout(duration).map_err(|_e| { - warn!( - message = "calling `park` on worker thread errored -- shutting down thread", - error = %_e, - ); - }) - } -} diff --git a/tokio-executor/src/threadpool/park/default_park.rs b/tokio-executor/src/threadpool/park/default_park.rs deleted file mode 100644 index 85c95cb7c17..00000000000 --- a/tokio-executor/src/threadpool/park/default_park.rs +++ /dev/null @@ -1,98 +0,0 @@ -use crate::park::{Park, Unpark}; - -use crossbeam_utils::sync::{Parker, Unparker}; -use std::error::Error; -use std::fmt; -use std::time::Duration; - -/// Parks the thread. -#[derive(Debug)] -pub struct DefaultPark { - inner: Parker, -} - -/// Unparks threads that were parked by `DefaultPark`. -#[derive(Debug)] -pub struct DefaultUnpark { - inner: Unparker, -} - -/// Error returned by [`ParkThread`] -/// -/// This currently is never returned, but might at some point in the future. -/// -/// [`ParkThread`]: struct.ParkThread.html -#[derive(Debug)] -pub struct ParkError { - _p: (), -} - -// ===== impl DefaultPark ===== - -impl DefaultPark { - /// Creates a new `DefaultPark` instance. - pub fn new() -> DefaultPark { - DefaultPark { - inner: Parker::new(), - } - } - - /// Unpark the thread without having to clone the unpark handle. - /// - /// Named `notify` to avoid conflicting with the `unpark` fn. - pub(crate) fn notify(&self) { - self.inner.unparker().unpark(); - } - - pub(crate) fn park_sync(&self, duration: Option) { - match duration { - None => self.inner.park(), - Some(duration) => self.inner.park_timeout(duration), - } - } -} - -impl Park for DefaultPark { - type Unpark = DefaultUnpark; - type Error = ParkError; - - fn unpark(&self) -> Self::Unpark { - DefaultUnpark { - inner: self.inner.unparker().clone(), - } - } - - fn park(&mut self) -> Result<(), Self::Error> { - self.inner.park(); - Ok(()) - } - - fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { - self.inner.park_timeout(duration); - Ok(()) - } -} - -impl Default for DefaultPark { - fn default() -> Self { - Self::new() - } -} - -// ===== impl DefaultUnpark ===== - -impl Unpark for DefaultUnpark { - fn unpark(&self) { - self.inner.unpark(); - } -} - -// ===== impl ParkError ===== - -impl fmt::Display for ParkError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "unknown park error") - } -} - -impl Error for ParkError {} diff --git a/tokio-executor/src/threadpool/park/mod.rs b/tokio-executor/src/threadpool/park/mod.rs deleted file mode 100644 index e7c5f40d367..00000000000 --- a/tokio-executor/src/threadpool/park/mod.rs +++ /dev/null @@ -1,8 +0,0 @@ -//! Thread parking utilities. - -mod boxed; -mod default_park; - -pub use self::default_park::{DefaultPark, DefaultUnpark, ParkError}; - -pub(crate) use self::boxed::{BoxPark, BoxUnpark, BoxedPark}; diff --git a/tokio-executor/src/threadpool/pool/backup.rs b/tokio-executor/src/threadpool/pool/backup.rs deleted file mode 100644 index 46ca3dadbb4..00000000000 --- a/tokio-executor/src/threadpool/pool/backup.rs +++ /dev/null @@ -1,308 +0,0 @@ -use super::super::park::DefaultPark; -use super::super::worker::WorkerId; - -use std::cell::UnsafeCell; -use std::fmt; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::{self, AcqRel, Acquire, Relaxed}; -use std::time::{Duration, Instant}; - -/// State associated with a thread in the thread pool. -/// -/// The pool manages a number of threads. Some of those threads are considered -/// "primary" threads and process the work queue. When a task being run on a -/// primary thread enters a blocking context, the responsibility of processing -/// the work queue must be handed off to another thread. This is done by first -/// checking for idle threads on the backup stack. If one is found, the worker -/// token (`WorkerId`) is handed off to that running thread. If none are found, -/// a new thread is spawned. -/// -/// This state manages the exchange. A thread that is idle, not assigned to a -/// work queue, sits around for a specified amount of time. When the worker -/// token is handed off, it is first stored in `handoff`. The backup thread is -/// then signaled. At this point, the backup thread wakes up from sleep and -/// reads `handoff`. At that point, it has been promoted to a primary thread and -/// will begin processing inbound work on the work queue. -/// -/// The name `Backup` isn't really great for what the type does, but I have not -/// come up with a better name... Maybe it should just be named `Thread`. -#[derive(Debug)] -pub(crate) struct Backup { - /// Worker ID that is being handed to this thread. - handoff: UnsafeCell>, - - /// Thread state. - /// - /// This tracks: - /// - /// * Is queued flag - /// * If the pool is shutting down. - /// * If the thread is running - state: AtomicUsize, - - /// Next entry in the Treiber stack. - next_sleeper: UnsafeCell, - - /// Used to put the thread to sleep - park: DefaultPark, -} - -#[derive(Debug, Eq, PartialEq, Copy, Clone)] -pub(crate) struct BackupId(pub(crate) usize); - -#[derive(Debug)] -pub(crate) enum Handoff { - Worker(WorkerId), - Idle, - Terminated, -} - -/// Tracks thread state. -#[derive(Clone, Copy, Eq, PartialEq)] -struct State(usize); - -/// Set when the worker is pushed onto the scheduler's stack of sleeping -/// threads. -/// -/// This flag also serves as a "notification" bit. If another thread is -/// attempting to hand off a worker to the backup thread, then the pushed bit -/// will not be set when the thread tries to shutdown. -pub(crate) const PUSHED: usize = 0b001; - -/// Set when the thread is running -pub(crate) const RUNNING: usize = 0b010; - -/// Set when the thread pool has terminated -pub(crate) const TERMINATED: usize = 0b100; - -// ===== impl Backup ===== - -impl Backup { - pub(crate) fn new() -> Backup { - Backup { - handoff: UnsafeCell::new(None), - state: AtomicUsize::new(State::new().into()), - next_sleeper: UnsafeCell::new(BackupId(0)), - park: DefaultPark::new(), - } - } - - /// Called when the thread is starting - pub(crate) fn start(&self, worker_id: &WorkerId) { - debug_assert!({ - let state: State = self.state.load(Relaxed).into(); - - debug_assert!(!state.is_pushed()); - debug_assert!(state.is_running()); - debug_assert!(!state.is_terminated()); - - true - }); - - // The handoff value is equal to `worker_id` - debug_assert_eq!(unsafe { (*self.handoff.get()).as_ref() }, Some(worker_id)); - - unsafe { - *self.handoff.get() = None; - } - } - - pub(crate) fn is_running(&self) -> bool { - let state: State = self.state.load(Relaxed).into(); - state.is_running() - } - - /// Hands off the worker to a thread. - /// - /// Returns `true` if the thread needs to be spawned. - pub(crate) fn worker_handoff(&self, worker_id: WorkerId) -> bool { - unsafe { - // The backup worker should not already have been handoff a worker. - debug_assert!((*self.handoff.get()).is_none()); - - // Set the handoff - *self.handoff.get() = Some(worker_id); - } - - // This *probably* can just be `Release`... memory orderings, how do - // they work? - let prev = State::worker_handoff(&self.state); - debug_assert!(prev.is_pushed()); - - if prev.is_running() { - // Wakeup the backup thread - self.park.notify(); - false - } else { - true - } - } - - /// Terminate the worker - pub(crate) fn signal_stop(&self) { - let prev: State = self.state.fetch_xor(TERMINATED | PUSHED, AcqRel).into(); - - debug_assert!(!prev.is_terminated()); - debug_assert!(prev.is_pushed()); - - if prev.is_running() { - self.park.notify(); - } - } - - /// Release the worker - pub(crate) fn release(&self) { - let prev: State = self.state.fetch_xor(RUNNING, AcqRel).into(); - - debug_assert!(prev.is_running()); - } - - /// Wait for a worker handoff - pub(crate) fn wait_for_handoff(&self, timeout: Option) -> Handoff { - let sleep_until = timeout.map(|dur| Instant::now() + dur); - let mut state: State = self.state.load(Acquire).into(); - - // Run in a loop since there can be spurious wakeups - loop { - if !state.is_pushed() { - if state.is_terminated() { - return Handoff::Terminated; - } - - let worker_id = unsafe { (*self.handoff.get()).take().expect("no worker handoff") }; - return Handoff::Worker(worker_id); - } - - match sleep_until { - None => { - self.park.park_sync(None); - state = self.state.load(Acquire).into(); - } - Some(when) => { - let now = Instant::now(); - - if now < when { - self.park.park_sync(Some(when - now)); - state = self.state.load(Acquire).into(); - } else { - debug_assert!(state.is_running()); - - // Transition out of running - let mut next = state; - next.unset_running(); - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - debug_assert!(!next.is_running()); - return Handoff::Idle; - } - - state = actual; - } - } - } - } - } - - pub(crate) fn is_pushed(&self) -> bool { - let state: State = self.state.load(Relaxed).into(); - state.is_pushed() - } - - pub(crate) fn set_pushed(&self, ordering: Ordering) { - let prev: State = self.state.fetch_or(PUSHED, ordering).into(); - debug_assert!(!prev.is_pushed()); - } - - #[inline] - pub(crate) fn next_sleeper(&self) -> BackupId { - unsafe { *self.next_sleeper.get() } - } - - #[inline] - pub(crate) fn set_next_sleeper(&self, val: BackupId) { - unsafe { - *self.next_sleeper.get() = val; - } - } -} - -// ===== impl State ===== - -impl State { - /// Returns a new, default, thread `State` - pub(crate) fn new() -> State { - State(0) - } - - /// Returns true if the thread entry is pushed in the sleeper stack - pub(crate) fn is_pushed(self) -> bool { - self.0 & PUSHED == PUSHED - } - - fn unset_pushed(&mut self) { - self.0 &= !PUSHED; - } - - pub(crate) fn is_running(self) -> bool { - self.0 & RUNNING == RUNNING - } - - pub(crate) fn set_running(&mut self) { - self.0 |= RUNNING; - } - - pub(crate) fn unset_running(&mut self) { - self.0 &= !RUNNING; - } - - pub(crate) fn is_terminated(self) -> bool { - self.0 & TERMINATED == TERMINATED - } - - fn worker_handoff(state: &AtomicUsize) -> State { - let mut curr: State = state.load(Acquire).into(); - - loop { - let mut next = curr; - next.set_running(); - next.unset_pushed(); - - let actual = state - .compare_and_swap(curr.into(), next.into(), AcqRel) - .into(); - - if actual == curr { - return curr; - } - - curr = actual; - } - } -} - -impl From for State { - fn from(src: usize) -> State { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> usize { - src.0 - } -} - -impl fmt::Debug for State { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("backup::State") - .field("is_pushed", &self.is_pushed()) - .field("is_running", &self.is_running()) - .field("is_terminated", &self.is_terminated()) - .finish() - } -} diff --git a/tokio-executor/src/threadpool/pool/backup_stack.rs b/tokio-executor/src/threadpool/pool/backup_stack.rs deleted file mode 100644 index b55112bd969..00000000000 --- a/tokio-executor/src/threadpool/pool/backup_stack.rs +++ /dev/null @@ -1,188 +0,0 @@ -use super::{Backup, BackupId}; - -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::{AcqRel, Acquire}; - -#[derive(Debug)] -pub(crate) struct BackupStack { - state: AtomicUsize, -} - -#[derive(Debug, Eq, PartialEq, Clone, Copy)] -struct State(usize); - -pub(crate) const MAX_BACKUP: usize = 1 << 15; - -/// Extracts the head of the backup stack from the state -const STACK_MASK: usize = ((1 << 16) - 1); - -/// Used to mark the stack as empty -pub(crate) const EMPTY: BackupId = BackupId(MAX_BACKUP); - -/// Used to mark the stack as terminated -pub(crate) const TERMINATED: BackupId = BackupId(EMPTY.0 + 1); - -/// How many bits the Treiber ABA guard is offset by -const ABA_GUARD_SHIFT: usize = 16; - -#[cfg(target_pointer_width = "64")] -const ABA_GUARD_MASK: usize = (1 << (64 - ABA_GUARD_SHIFT)) - 1; - -#[cfg(target_pointer_width = "32")] -const ABA_GUARD_MASK: usize = (1 << (32 - ABA_GUARD_SHIFT)) - 1; - -// ===== impl BackupStack ===== - -impl BackupStack { - pub(crate) fn new() -> BackupStack { - let state = AtomicUsize::new(State::new().into()); - BackupStack { state } - } - - /// Push a backup thread onto the stack - /// - /// # Return - /// - /// Returns `Ok` on success. - /// - /// Returns `Err` if the pool has transitioned to the `TERMINATED` state. - /// When terminated, pushing new entries is no longer permitted. - pub(crate) fn push(&self, entries: &[Backup], id: BackupId) -> Result<(), ()> { - let mut state: State = self.state.load(Acquire).into(); - - entries[id.0].set_pushed(AcqRel); - - loop { - let mut next = state; - - let head = state.head(); - - if head == TERMINATED { - // The pool is terminated, cannot push the sleeper. - return Err(()); - } - - entries[id.0].set_next_sleeper(head); - next.set_head(id); - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if state == actual { - return Ok(()); - } - - state = actual; - } - } - - /// Pop a backup thread off the stack. - /// - /// If `terminate` is set and the stack is empty when this function is - /// called, the state of the stack is transitioned to "terminated". At this - /// point, no further entries can be pushed onto the stack. - /// - /// # Return - /// - /// * Returns the index of the popped worker and the worker's observed - /// state. - /// - /// * `Ok(None)` if the stack is empty. - /// * `Err(_)` is returned if the pool has been shutdown. - pub(crate) fn pop(&self, entries: &[Backup], terminate: bool) -> Result, ()> { - // Figure out the empty value - let terminal = if terminate { TERMINATED } else { EMPTY }; - - let mut state: State = self.state.load(Acquire).into(); - - loop { - let head = state.head(); - - if head == EMPTY { - let mut next = state; - next.set_head(terminal); - - if next == state { - debug_assert!(terminal == EMPTY); - return Ok(None); - } - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual != state { - state = actual; - continue; - } - - return Ok(None); - } else if head == TERMINATED { - return Err(()); - } - - debug_assert!(head.0 < MAX_BACKUP); - - let mut next = state; - - let next_head = entries[head.0].next_sleeper(); - - // TERMINATED can never be set as the "next pointer" on a worker. - debug_assert!(next_head != TERMINATED); - - if next_head == EMPTY { - next.set_head(terminal); - } else { - next.set_head(next_head); - } - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - debug_assert!(entries[head.0].is_pushed()); - return Ok(Some(head)); - } - - state = actual; - } - } -} - -// ===== impl State ===== - -impl State { - fn new() -> State { - State(EMPTY.0) - } - - fn head(self) -> BackupId { - BackupId(self.0 & STACK_MASK) - } - - fn set_head(&mut self, val: BackupId) { - let val = val.0; - - // The ABA guard protects against the ABA problem w/ Treiber stacks - let aba_guard = ((self.0 >> ABA_GUARD_SHIFT) + 1) & ABA_GUARD_MASK; - - self.0 = (aba_guard << ABA_GUARD_SHIFT) | val; - } -} - -impl From for State { - fn from(src: usize) -> Self { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> Self { - src.0 - } -} diff --git a/tokio-executor/src/threadpool/pool/mod.rs b/tokio-executor/src/threadpool/pool/mod.rs deleted file mode 100644 index 5d6f05f1a5e..00000000000 --- a/tokio-executor/src/threadpool/pool/mod.rs +++ /dev/null @@ -1,481 +0,0 @@ -mod backup; -mod backup_stack; -mod state; - -pub(crate) use self::backup::{Backup, BackupId}; -pub(crate) use self::backup_stack::MAX_BACKUP; -pub(crate) use self::state::{Lifecycle, State, MAX_FUTURES}; - -use self::backup::Handoff; -use self::backup_stack::BackupStack; -use super::config::Config; -use super::shutdown::ShutdownTrigger; -use super::task::{Blocking, Task}; -use super::worker::{self, Worker, WorkerId}; -use super::BlockingError; - -use crossbeam_deque::Injector; -use crossbeam_utils::CachePadded; -use lazy_static::lazy_static; -use std::cell::Cell; -use std::collections::hash_map::RandomState; -use std::hash::{BuildHasher, Hash, Hasher}; -use std::num::Wrapping; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::{AcqRel, Acquire}; -use std::sync::{Arc, Weak}; -use std::task::Poll; -use std::thread; - -#[derive(Debug)] -pub(crate) struct Pool { - // Tracks the state of the thread pool (running, shutting down, ...). - // - // While workers check this field as a hint to detect shutdown, it is - // **not** used as a primary point of coordination for workers. The sleep - // stack is used as the primary point of coordination for workers. - // - // The value of this atomic is deserialized into a `pool::State` instance. - // See comments for that type. - pub(crate) state: CachePadded, - - // Stack tracking sleeping workers. - sleep_stack: CachePadded, - - // Worker state - // - // A worker is a thread that is processing the work queue and polling - // futures. - // - // The number of workers will *usually* be small. - pub(crate) workers: Arc<[worker::Entry]>, - - // The global MPMC queue of tasks. - // - // Spawned tasks are pushed into this queue. Although worker threads have their own dedicated - // task queues, they periodically steal tasks from this global queue, too. - pub(crate) queue: Arc>>, - - // Completes the shutdown process when the `ThreadPool` and all `Worker`s get dropped. - // - // When spawning a new `Worker`, this weak reference is upgraded and handed out to the new - // thread. - pub(crate) trigger: Weak, - - // Backup thread state - // - // In order to efficiently support `blocking`, a pool of backup threads is - // needed. These backup threads are ready to take over a worker if the - // future being processed requires blocking. - backup: Box<[Backup]>, - - // Stack of sleeping backup threads - pub(crate) backup_stack: BackupStack, - - // State regarding coordinating blocking sections and tracking tasks that - // are pending blocking capacity. - blocking: Blocking, - - // Configuration - pub(crate) config: Config, -} - -impl Pool { - /// Create a new `Pool` - pub(crate) fn new( - workers: Arc<[worker::Entry]>, - trigger: Weak, - max_blocking: usize, - config: Config, - queue: Arc>>, - ) -> Pool { - let pool_size = workers.len(); - let total_size = max_blocking + pool_size; - - // Create the set of backup entries - // - // This is `backup + pool_size` because the core thread pool running the - // workers is spawned from backup as well. - let backup = (0..total_size) - .map(|_| Backup::new()) - .collect::>() - .into_boxed_slice(); - - let backup_stack = BackupStack::new(); - - for i in (0..backup.len()).rev() { - backup_stack.push(&backup, BackupId(i)).unwrap(); - } - - // Initialize the blocking state - let blocking = Blocking::new(max_blocking); - - let ret = Pool { - state: CachePadded::new(AtomicUsize::new(State::new().into())), - sleep_stack: CachePadded::new(worker::Stack::new()), - workers, - queue, - trigger, - backup, - backup_stack, - blocking, - config, - }; - - // Now, we prime the sleeper stack - for i in 0..pool_size { - ret.sleep_stack.push(&ret.workers, i).unwrap(); - } - - ret - } - - /// Start shutting down the pool. This means that no new futures will be - /// accepted. - #[cfg_attr(feature = "tracing", tracing::instrument(level = "trace"))] - pub(crate) fn shutdown(&self, now: bool, purge_queue: bool) { - let mut state: State = self.state.load(Acquire).into(); - trace!(?state); - - // For now, this must be true - debug_assert!(!purge_queue || now); - - // Start by setting the shutdown flag - loop { - let mut next = state; - - let num_futures = next.num_futures(); - - if next.lifecycle() == Lifecycle::ShutdownNow { - // Already transitioned to shutting down state - - if !purge_queue || num_futures == 0 { - // Nothing more to do - return; - } - - // The queue must be purged - debug_assert!(purge_queue); - next.clear_num_futures(); - } else { - next.set_lifecycle(if now || num_futures == 0 { - // If already idle, always transition to shutdown now. - Lifecycle::ShutdownNow - } else { - Lifecycle::ShutdownOnIdle - }); - - if purge_queue { - next.clear_num_futures(); - } - } - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if state == actual { - state = next; - break; - } - - state = actual; - } - - trace!("transitioned to shutdown"); - - // Only transition to terminate if there are no futures currently on the - // pool - if state.num_futures() != 0 { - return; - } - - self.terminate_sleeping_workers(); - } - - /// Called by `Worker` as it tries to enter a sleeping state. Before it - /// sleeps, it must push itself onto the sleep stack. This enables other - /// threads to see it when signaling work. - pub(crate) fn push_sleeper(&self, idx: usize) -> Result<(), ()> { - self.sleep_stack.push(&self.workers, idx) - } - - pub(crate) fn terminate_sleeping_workers(&self) { - use super::worker::Lifecycle::Signaled; - - trace!("shutting down workers"); - // Wakeup all sleeping workers. They will wake up, see the state - // transition, and terminate. - while let Some((idx, worker_state)) = self.sleep_stack.pop(&self.workers, Signaled, true) { - self.workers[idx].signal_stop(worker_state); - } - - // Now terminate any backup threads - // - // The call to `pop` must be successful because shutting down the pool - // is coordinated and at this point, this is the only thread that will - // attempt to transition the backup stack to "terminated". - while let Ok(Some(backup_id)) = self.backup_stack.pop(&self.backup, true) { - self.backup[backup_id.0].signal_stop(); - } - } - - pub(crate) fn poll_blocking_capacity( - &self, - task: &Arc, - ) -> Poll> { - self.blocking.poll_blocking_capacity(task) - } - - /// Submit a task to the scheduler. - /// - /// Called from either inside or outside of the scheduler. If currently on - /// the scheduler, then a fast path is taken. - pub(crate) fn submit(&self, task: Arc, pool: &Arc) { - debug_assert_eq!(*self, **pool); - - Worker::with_current(|worker| { - if let Some(worker) = worker { - // If the worker is in blocking mode, then even though the - // thread-local variable is set, the current thread does not - // have ownership of that worker entry. This is because the - // worker entry has already been handed off to another thread. - // - // The second check handles the case where the current thread is - // part of a different threadpool than the one being submitted - // to. - if !worker.is_blocking() && *self == *worker.pool { - let idx = worker.id.0; - - trace!(message = "submit internal;", idx); - - worker.pool.workers[idx].submit_internal(task); - worker.pool.signal_work(pool); - return; - } - } - - self.submit_external(task, pool); - }); - } - - /// Submit a task to the scheduler from off worker - /// - /// Called from outside of the scheduler, this function is how new tasks - /// enter the system. - pub(crate) fn submit_external(&self, task: Arc, pool: &Arc) { - debug_assert_eq!(*self, **pool); - - trace!("submit external"); - - self.queue.push(task); - self.signal_work(pool); - } - - pub(crate) fn release_backup(&self, backup_id: BackupId) -> Result<(), ()> { - // First update the state, this cannot fail because the caller must have - // exclusive access to the backup token. - self.backup[backup_id.0].release(); - - // Push the backup entry back on the stack - self.backup_stack.push(&self.backup, backup_id) - } - - pub(crate) fn notify_blocking_task(&self, pool: &Arc) { - debug_assert_eq!(*self, **pool); - self.blocking.notify_task(&pool); - } - - /// Provision a thread to run a worker - pub(crate) fn spawn_thread(&self, id: WorkerId, pool: &Arc) { - debug_assert_eq!(*self, **pool); - - let backup_id = match self.backup_stack.pop(&self.backup, false) { - Ok(Some(backup_id)) => backup_id, - Ok(None) => panic!("no thread available"), - Err(_) => { - debug!("failed to spawn worker thread due to the thread pool shutting down"); - return; - } - }; - - let need_spawn = self.backup[backup_id.0].worker_handoff(id.clone()); - - if !need_spawn { - return; - } - - let trigger = match self.trigger.upgrade() { - None => { - // The pool is shutting down. - return; - } - Some(t) => t, - }; - - let mut th = thread::Builder::new(); - - if let Some(ref prefix) = pool.config.name_prefix { - th = th.name(format!("{}{}", prefix, backup_id.0)); - } - - if let Some(stack) = pool.config.stack_size { - th = th.stack_size(stack); - } - - let pool = pool.clone(); - - let res = th.spawn(move || { - if let Some(ref f) = pool.config.after_start { - f(); - } - - let mut worker_id = id; - - pool.backup[backup_id.0].start(&worker_id); - - loop { - // The backup token should be in the running state. - debug_assert!(pool.backup[backup_id.0].is_running()); - - // TODO: Avoid always cloning - let worker = Worker::new(worker_id, backup_id, pool.clone(), trigger.clone()); - - // Run the worker. If the worker transitioned to a "blocking" - // state, then `is_blocking` will be true. - if !worker.do_run() { - // The worker shutdown, so exit the thread. - break; - } - - debug_assert!(!pool.backup[backup_id.0].is_pushed()); - - // Push the thread back onto the backup stack. This makes it - // available for future handoffs. - // - // This **must** happen before notifying the task. - let res = pool.backup_stack.push(&pool.backup, backup_id); - - if res.is_err() { - // The pool is being shutdown. - break; - } - - // The task switched the current thread to blocking mode. - // Now that the blocking task completed, any tasks - pool.notify_blocking_task(&pool); - - debug_assert!(pool.backup[backup_id.0].is_running()); - - // Wait for a handoff - let handoff = pool.backup[backup_id.0].wait_for_handoff(pool.config.keep_alive); - - match handoff { - Handoff::Worker(id) => { - debug_assert!(pool.backup[backup_id.0].is_running()); - worker_id = id; - } - Handoff::Idle | Handoff::Terminated => { - break; - } - } - } - - if let Some(ref f) = pool.config.before_stop { - f(); - } - }); - - if let Err(err) = res { - error!(message = "failed to spawn worker thread;", ?err); - panic!("failed to spawn worker thread: {:?}", err); - } - } - - /// If there are any other workers currently relaxing, signal them that work - /// is available so that they can try to find more work to process. - #[allow(clippy::cognitive_complexity)] // https://github.com/rust-lang/rust-clippy/issues/3900 - pub(crate) fn signal_work(&self, pool: &Arc) { - debug_assert_eq!(*self, **pool); - - use super::worker::Lifecycle::Signaled; - - if let Some((idx, worker_state)) = self.sleep_stack.pop(&self.workers, Signaled, false) { - let span = trace_span!("signal_work", idx); - let _enter = span.enter(); - - let entry = &self.workers[idx]; - - debug_assert!( - worker_state.lifecycle() != Signaled, - "actual={:?}", - worker_state.lifecycle(), - ); - - trace!("notify"); - - if !entry.notify(worker_state) { - trace!("spawn;"); - self.spawn_thread(WorkerId(idx), pool); - } - } - } - - /// Generates a random number - /// - /// Uses a thread-local random number generator based on XorShift. - pub(crate) fn rand_usize(&self) -> usize { - thread_local! { - static RNG: Cell> = Cell::new(Wrapping(prng_seed())); - } - - RNG.with(|rng| { - // This is the 32-bit variant of Xorshift. - // https://en.wikipedia.org/wiki/Xorshift - let mut x = rng.get(); - x ^= x << 13; - x ^= x >> 17; - x ^= x << 5; - rng.set(x); - x.0 as usize - }) - } -} - -impl PartialEq for Pool { - fn eq(&self, other: &Pool) -> bool { - self as *const _ == other as *const _ - } -} - -unsafe impl Send for Pool {} -unsafe impl Sync for Pool {} - -// Return a thread-specific, 32-bit, non-zero seed value suitable for a 32-bit -// PRNG. This uses one libstd RandomState for a default hasher and hashes on -// the current thread ID to obtain an unpredictable, collision resistant seed. -fn prng_seed() -> u32 { - // This obtains a small number of random bytes from the host system (for - // example, on unix via getrandom(2)) in order to seed an unpredictable and - // HashDoS resistant 64-bit hash function (currently: `SipHasher13` with - // 128-bit state). We only need one of these, to make the seeds for all - // process threads different via hashed IDs, collision resistant, and - // unpredictable. - lazy_static! { - static ref RND_STATE: RandomState = RandomState::new(); - } - - // Hash the current thread ID to produce a u32 value - let mut hasher = RND_STATE.build_hasher(); - thread::current().id().hash(&mut hasher); - let hash: u64 = hasher.finish(); - let seed = (hash as u32) ^ ((hash >> 32) as u32); - - // Ensure non-zero seed (Xorshift yields only zero's for that seed) - if seed == 0 { - 0x9b4e_6d25 // misc bits, could be any non-zero - } else { - seed - } -} diff --git a/tokio-executor/src/threadpool/pool/state.rs b/tokio-executor/src/threadpool/pool/state.rs deleted file mode 100644 index d9da1cbbf7d..00000000000 --- a/tokio-executor/src/threadpool/pool/state.rs +++ /dev/null @@ -1,132 +0,0 @@ -use std::{fmt, usize}; - -/// ThreadPool state. -/// -/// The two least significant bits are the shutdown flags. (0 for active, 1 for -/// shutdown on idle, 2 for shutting down). The remaining bits represent the -/// number of futures that still need to complete. -#[derive(Eq, PartialEq, Clone, Copy)] -pub(crate) struct State(usize); - -#[derive(Debug, Eq, PartialEq, Ord, PartialOrd, Clone, Copy)] -#[repr(usize)] -pub(crate) enum Lifecycle { - /// The thread pool is currently running - Running = 0, - - /// The thread pool should shutdown once it reaches an idle state. - ShutdownOnIdle = 1, - - /// The thread pool should start the process of shutting down. - ShutdownNow = 2, -} - -/// Mask used to extract the number of futures from the state -const LIFECYCLE_MASK: usize = 0b11; -const NUM_FUTURES_MASK: usize = !LIFECYCLE_MASK; -const NUM_FUTURES_OFFSET: usize = 2; - -/// Max number of futures the pool can handle. -pub(crate) const MAX_FUTURES: usize = usize::MAX >> NUM_FUTURES_OFFSET; - -// ===== impl State ===== - -impl State { - #[inline] - pub(crate) fn new() -> State { - State(0) - } - - /// Returns the number of futures still pending completion. - pub(crate) fn num_futures(self) -> usize { - self.0 >> NUM_FUTURES_OFFSET - } - - /// Increment the number of futures pending completion. - /// - /// Returns false on failure. - pub(crate) fn inc_num_futures(&mut self) { - debug_assert!(self.num_futures() < MAX_FUTURES); - debug_assert!(self.lifecycle() < Lifecycle::ShutdownNow); - - self.0 += 1 << NUM_FUTURES_OFFSET; - } - - /// Decrement the number of futures pending completion. - pub(crate) fn dec_num_futures(&mut self) { - let num_futures = self.num_futures(); - - if num_futures == 0 { - // Already zero - return; - } - - self.0 -= 1 << NUM_FUTURES_OFFSET; - - if self.lifecycle() == Lifecycle::ShutdownOnIdle && num_futures == 1 { - self.set_lifecycle(Lifecycle::ShutdownNow); - } - } - - /// Set the number of futures pending completion to zero - pub(crate) fn clear_num_futures(&mut self) { - self.0 &= LIFECYCLE_MASK; - } - - pub(crate) fn lifecycle(self) -> Lifecycle { - (self.0 & LIFECYCLE_MASK).into() - } - - pub(crate) fn set_lifecycle(&mut self, val: Lifecycle) { - self.0 = (self.0 & NUM_FUTURES_MASK) | (val as usize); - } - - pub(crate) fn is_terminated(self) -> bool { - self.lifecycle() == Lifecycle::ShutdownNow && self.num_futures() == 0 - } -} - -impl From for State { - fn from(src: usize) -> Self { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> Self { - src.0 - } -} - -impl fmt::Debug for State { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("pool::State") - .field("lifecycle", &self.lifecycle()) - .field("num_futures", &self.num_futures()) - .finish() - } -} - -// ===== impl Lifecycle ===== - -impl From for Lifecycle { - fn from(src: usize) -> Lifecycle { - use self::Lifecycle::*; - - debug_assert!( - src == Running as usize - || src == ShutdownOnIdle as usize - || src == ShutdownNow as usize - ); - - unsafe { ::std::mem::transmute(src) } - } -} - -impl From for usize { - fn from(src: Lifecycle) -> usize { - let v = src as usize; - debug_assert!(v & LIFECYCLE_MASK == v); - v - } -} diff --git a/tokio-executor/src/threadpool/sender.rs b/tokio-executor/src/threadpool/sender.rs deleted file mode 100644 index cd76ea4d60e..00000000000 --- a/tokio-executor/src/threadpool/sender.rs +++ /dev/null @@ -1,193 +0,0 @@ -use super::pool::{self, Lifecycle, Pool, MAX_FUTURES}; -use super::task::Task; - -use crate::{Executor, SpawnError, TypedExecutor}; - -use std::future::Future; -use std::pin::Pin; -use std::sync::atomic::Ordering::{AcqRel, Acquire}; -use std::sync::Arc; - -/// Submit futures to the associated thread pool for execution. -/// -/// A `Sender` instance is a handle to a single thread pool, allowing the owner -/// of the handle to spawn futures onto the thread pool. New futures are spawned -/// using [`Sender::spawn`]. -/// -/// The `Sender` handle is *only* used for spawning new futures. It does not -/// impact the lifecycle of the thread pool in any way. -/// -/// `Sender` instances are obtained by calling [`ThreadPool::sender`]. The -/// `Sender` struct implements the `Executor` trait. -/// -/// [`Sender::spawn`]: #method.spawn -/// [`ThreadPool::sender`]: struct.ThreadPool.html#method.sender -#[derive(Debug)] -pub struct Sender { - pub(crate) pool: Arc, -} - -impl Sender { - /// Spawn a future onto the thread pool - /// - /// This function takes ownership of the future and spawns it onto the - /// thread pool, assigning it to a worker thread. The exact strategy used to - /// assign a future to a worker depends on if the caller is already on a - /// worker thread or external to the thread pool. - /// - /// If the caller is currently on the thread pool, the spawned future will - /// be assigned to the same worker that the caller is on. If the caller is - /// external to the thread pool, the future will be assigned to a random - /// worker. - /// - /// If `spawn` returns `Ok`, this does not mean that the future will be - /// executed. The thread pool can be forcibly shutdown between the time - /// `spawn` is called and the future has a chance to execute. - /// - /// If `spawn` returns `Err`, then the future failed to be spawned. There - /// are two possible causes: - /// - /// * The thread pool is at capacity and is unable to spawn a new future. - /// This is a temporary failure. At some point in the future, the thread - /// pool might be able to spawn new futures. - /// * The thread pool is shutdown. This is a permanent failure indicating - /// that the handle will never be able to spawn new futures. - /// - /// The status of the thread pool can be queried before calling `spawn` - /// using the `status` function (part of the `Executor` trait). - /// - /// # Examples - /// - /// ```rust - /// use tokio_executor::threadpool::ThreadPool; - /// - /// // Create a thread pool with default configuration values - /// let thread_pool = ThreadPool::new(); - /// - /// thread_pool.sender().spawn(async { - /// println!("called from a worker thread"); - /// }).unwrap(); - /// - /// // Gracefully shutdown the threadpool - /// thread_pool.shutdown().wait(); - /// ``` - pub fn spawn(&self, future: F) -> Result<(), SpawnError> - where - F: Future + Send + 'static, - { - let mut s = self; - Executor::spawn(&mut s, Box::pin(future)) - } - - /// Logic to prepare for spawning - fn prepare_for_spawn(&self) -> Result<(), SpawnError> { - let mut state: pool::State = self.pool.state.load(Acquire).into(); - - // Increment the number of futures spawned on the pool as well as - // validate that the pool is still running/ - loop { - let mut next = state; - - if next.num_futures() == MAX_FUTURES { - // No capacity - return Err(SpawnError::at_capacity()); - } - - if next.lifecycle() == Lifecycle::ShutdownNow { - // Cannot execute the future, executor is shutdown. - return Err(SpawnError::shutdown()); - } - - next.inc_num_futures(); - - let actual = self - .pool - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - trace!(message = "execute;", count = next.num_futures()); - break; - } - - state = actual; - } - - Ok(()) - } -} - -impl Executor for Sender { - fn status(&self) -> Result<(), SpawnError> { - let s = self; - Executor::status(&s) - } - - fn spawn( - &mut self, - future: Pin + Send>>, - ) -> Result<(), SpawnError> { - let mut s = &*self; - Executor::spawn(&mut s, future) - } -} - -impl Executor for &Sender { - fn status(&self) -> Result<(), SpawnError> { - let state: pool::State = self.pool.state.load(Acquire).into(); - - if state.num_futures() == MAX_FUTURES { - // No capacity - return Err(SpawnError::at_capacity()); - } - - if state.lifecycle() == Lifecycle::ShutdownNow { - // Cannot execute the future, executor is shutdown. - return Err(SpawnError::shutdown()); - } - - Ok(()) - } - - fn spawn( - &mut self, - future: Pin + Send>>, - ) -> Result<(), SpawnError> { - self.prepare_for_spawn()?; - - // At this point, the pool has accepted the future, so schedule it for - // execution. - - // Create a new task for the future - let task = Arc::new(Task::new(future)); - - // Call `submit_external()` in order to place the task into the global - // queue. This way all workers have equal chance of running this task, - // which means IO handles will be assigned to reactors more evenly. - self.pool.submit_external(task, &self.pool); - - Ok(()) - } -} - -impl TypedExecutor for Sender -where - T: Future + Send + 'static, -{ - fn status(&self) -> Result<(), SpawnError> { - Executor::status(self) - } - - fn spawn(&mut self, future: T) -> Result<(), SpawnError> { - Executor::spawn(self, Box::pin(future)) - } -} - -impl Clone for Sender { - #[inline] - fn clone(&self) -> Sender { - let pool = self.pool.clone(); - Sender { pool } - } -} diff --git a/tokio-executor/src/threadpool/shutdown.rs b/tokio-executor/src/threadpool/shutdown.rs deleted file mode 100644 index 39c08e60127..00000000000 --- a/tokio-executor/src/threadpool/shutdown.rs +++ /dev/null @@ -1,110 +0,0 @@ -use super::task::Task; -use super::worker; - -use tokio_sync::AtomicWaker; - -use crossbeam_deque::Injector; -use std::future::Future; -use std::pin::Pin; -use std::sync::{Arc, Mutex}; -use std::task::{Context, Poll}; - -/// Future that resolves when the thread pool is shutdown. -/// -/// A `ThreadPool` is shutdown once all the worker have drained their queues and -/// shutdown their threads. -/// -/// `Shutdown` is returned by [`shutdown`], [`shutdown_on_idle`], and -/// [`shutdown_now`]. -/// -/// [`shutdown`]: struct.ThreadPool.html#method.shutdown -/// [`shutdown_on_idle`]: struct.ThreadPool.html#method.shutdown_on_idle -/// [`shutdown_now`]: struct.ThreadPool.html#method.shutdown_now -#[derive(Debug)] -pub struct Shutdown { - inner: Arc>, -} - -/// Shared state between `Shutdown` and `ShutdownTrigger`. -/// -/// This is used for notifying the `Shutdown` future when `ShutdownTrigger` gets dropped. -#[derive(Debug)] -struct Inner { - /// The task to notify when the threadpool completes the shutdown process. - task: AtomicWaker, - /// `true` if the threadpool has been shut down. - completed: bool, -} - -impl Shutdown { - pub(crate) fn new(trigger: &ShutdownTrigger) -> Shutdown { - Shutdown { - inner: trigger.inner.clone(), - } - } - - /// Wait for the shutdown to complete - pub fn wait(self) { - let mut enter = crate::enter().unwrap(); - enter.block_on(self); - } -} - -impl Future for Shutdown { - type Output = (); - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { - let inner = self.inner.lock().unwrap(); - - if !inner.completed { - inner.task.register_by_ref(cx.waker()); - Poll::Pending - } else { - Poll::Ready(()) - } - } -} - -/// When dropped, cleans up threadpool's resources and completes the shutdown process. -#[derive(Debug)] -pub(crate) struct ShutdownTrigger { - inner: Arc>, - workers: Arc<[worker::Entry]>, - queue: Arc>>, -} - -unsafe impl Send for ShutdownTrigger {} -unsafe impl Sync for ShutdownTrigger {} - -impl ShutdownTrigger { - pub(crate) fn new( - workers: Arc<[worker::Entry]>, - queue: Arc>>, - ) -> ShutdownTrigger { - ShutdownTrigger { - inner: Arc::new(Mutex::new(Inner { - task: AtomicWaker::new(), - completed: false, - })), - workers, - queue, - } - } -} - -impl Drop for ShutdownTrigger { - fn drop(&mut self) { - // Drain the global task queue. - while !self.queue.steal().is_empty() {} - - // Drop the remaining incomplete tasks and parkers assosicated with workers. - for worker in self.workers.iter() { - worker.shutdown(); - } - - // Notify the task interested in shutdown. - let mut inner = self.inner.lock().unwrap(); - inner.completed = true; - inner.task.wake(); - } -} diff --git a/tokio-executor/src/threadpool/task/blocking.rs b/tokio-executor/src/threadpool/task/blocking.rs deleted file mode 100644 index fe21e0d91f0..00000000000 --- a/tokio-executor/src/threadpool/task/blocking.rs +++ /dev/null @@ -1,499 +0,0 @@ -use super::super::pool::Pool; -use super::super::task::{BlockingState, Task}; -use crate::threadpool::BlockingError; - -use std::cell::UnsafeCell; -use std::fmt; -use std::ptr; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::{AcqRel, Acquire, Relaxed, Release}; -use std::sync::Arc; -use std::task::Poll; -use std::thread; - -/// Manages the state around entering a blocking section and tasks that are -/// queued pending the ability to block. -/// -/// This is a hybrid counter and intrusive mpsc channel (like `Queue`). -#[derive(Debug)] -pub(crate) struct Blocking { - /// Queue head. - /// - /// This is either the current remaining capacity for blocking sections - /// **or** if the max has been reached, the head of a pending blocking - /// capacity channel of tasks. - /// - /// When this points to a task, it represents a strong reference, i.e. - /// `Arc`. - state: AtomicUsize, - - /// Tail pointer. This is `Arc` unless it points to `stub`. - tail: UnsafeCell<*mut Task>, - - /// Stub pointer, used as part of the intrusive mpsc channel algorithm - /// described by 1024cores. - stub: Box, - - /// The channel algorithm is MPSC. This means that, in order to pop tasks, - /// coordination is required. - /// - /// Since it doesn't matter *which* task pops & notifies the queued task, we - /// can avoid a full mutex and make the "lock" lock free. - /// - /// Instead, threads race to set the "entered" bit. When the transition is - /// successfully made, the thread has permission to pop tasks off of the - /// queue. If a thread loses the race, instead of waiting to pop a task, it - /// signals to the winning thread that it should pop an additional task. - lock: AtomicUsize, -} - -#[derive(Debug, Clone, Copy, Eq, PartialEq)] -pub(crate) enum CanBlock { - /// Blocking capacity has been allocated to this task. - /// - /// The capacity allocation is initially checked before a task is polled. If - /// capacity has been allocated, it is consumed and tracked as `Allocated`. - Allocated, - - /// Allocation capacity must be either available to the task when it is - /// polled or not available. This means that a task can only ask for - /// capacity once. This state is used to track a task that has not yet asked - /// for blocking capacity. When a task needs blocking capacity, if it is in - /// this state, it can immediately try to get an allocation. - CanRequest, - - /// The task has requested blocking capacity, but none is available. - NoCapacity, -} - -/// Decorates the `usize` value of `Blocking::state`, providing fns to -/// manipulate the state instead of requiring bit ops. -#[derive(Copy, Clone, Eq, PartialEq)] -struct State(usize); - -/// Flag differentiating between remaining capacity and task pointers. -/// -/// If we assume pointers are properly aligned, then the least significant bit -/// will always be zero. So, we use that bit to track if the value represents a -/// number. -const NUM_FLAG: usize = 1; - -/// When representing "numbers", the state has to be shifted this much (to get -/// rid of the flag bit). -const NUM_SHIFT: usize = 1; - -// ====== impl Blocking ===== -// -impl Blocking { - /// Create a new `Blocking`. - pub(crate) fn new(capacity: usize) -> Blocking { - assert!(capacity > 0, "blocking capacity must be greater than zero"); - - let stub = Box::new(Task::stub()); - let ptr = &*stub as *const _ as *mut _; - - // Allocations are aligned - debug_assert!(ptr as usize & NUM_FLAG == 0); - - // The initial state value. This starts at the max capacity. - let init = State::new(capacity); - - Blocking { - state: AtomicUsize::new(init.into()), - tail: UnsafeCell::new(ptr), - stub, - lock: AtomicUsize::new(0), - } - } - - /// Atomically either acquire blocking capacity or queue the task to be - /// notified once capacity becomes available. - /// - /// The caller must ensure that `task` has not previously been queued to be - /// notified when capacity becomes available. - pub(crate) fn poll_blocking_capacity( - &self, - task: &Arc, - ) -> Poll> { - // This requires atomically claiming blocking capacity and if none is - // available, queuing &task. - - // The task cannot be queued at this point. The caller must ensure this. - debug_assert!(!BlockingState::from(task.blocking.load(Acquire)).is_queued()); - - // Don't bump the ref count unless necessary. - let mut strong: Option<*const Task> = None; - - // Load the state - let mut curr: State = self.state.load(Acquire).into(); - - loop { - let mut next = curr; - - if !next.claim_capacity(&self.stub) { - debug_assert!(curr.ptr().is_some()); - - // Unable to claim capacity, so we must queue `task` onto the - // channel. - // - // This guard also serves to ensure that queuing work that is - // only needed to run once only gets run once. - if strong.is_none() { - // First, transition the task to a "queued" state. This - // prevents double queuing. - // - // This is also the only thread that can set the queued flag - // at this point. And, the goal is for this to only be - // visible when the task node is polled from the channel. - // The memory ordering is established by MPSC queue - // operation. - // - // Note that, if the task doesn't get queued (because the - // CAS fails and capacity is now available) then this flag - // must be unset. Again, there is no race because until the - // task is queued, no other thread can see it. - let prev = BlockingState::toggle_queued(&task.blocking, Relaxed); - debug_assert!(!prev.is_queued()); - - // Bump the ref count - strong = Some(Arc::into_raw(task.clone())); - - // Set the next pointer. This does not require an atomic - // operation as this node is not currently accessible to - // other threads via the queue. - task.next_blocking.store(ptr::null_mut(), Relaxed); - } - - let ptr = strong.unwrap(); - - // Update the head to point to the new node. We need to see the - // previous node in order to update the next pointer as well as - // release `task` to any other threads calling `push`. - next.set_ptr(ptr); - } - - debug_assert_ne!(curr.0, 0); - debug_assert_ne!(next.0, 0); - - let actual = self - .state - .compare_and_swap(curr.into(), next.into(), AcqRel) - .into(); - - if curr == actual { - break; - } - - curr = actual; - } - - match curr.ptr() { - Some(prev) => { - let ptr = strong.unwrap(); - - // Finish pushing - unsafe { - (*prev).next_blocking.store(ptr as *mut _, Release); - } - - // The node was queued to be notified once capacity is made - // available. - Poll::Pending - } - None => { - debug_assert!(curr.remaining_capacity() > 0); - - // If `strong` is set, gotta undo a bunch of work - if let Some(ptr) = strong { - let _ = unsafe { Arc::from_raw(ptr) }; - - // Unset the queued flag. - let prev = BlockingState::toggle_queued(&task.blocking, Relaxed); - debug_assert!(prev.is_queued()); - } - - // Capacity has been obtained - Poll::Ready(Ok(())) - } - } - } - - unsafe fn push_stub(&self) { - let task: *mut Task = &*self.stub as *const _ as *mut _; - - // Set the next pointer. This does not require an atomic operation as - // this node is not accessible. The write will be flushed with the next - // operation - (*task).next_blocking.store(ptr::null_mut(), Relaxed); - - // Update the head to point to the new node. We need to see the previous - // node in order to update the next pointer as well as release `task` - // to any other threads calling `push`. - let prev = self.state.swap(task as usize, AcqRel); - - // The stub is only pushed when there are pending tasks. Because of - // this, the state must *always* be in pointer mode. - debug_assert!(State::from(prev).is_ptr()); - - let prev = prev as *const Task; - - // We don't want the *existing* pointer to be a stub. - debug_assert_ne!(prev, task); - - // Release `task` to the consume end. - (*prev).next_blocking.store(task, Release); - } - - pub(crate) fn notify_task(&self, pool: &Arc) { - let prev = self.lock.fetch_add(1, AcqRel); - - if prev != 0 { - // Another thread has the lock and will be responsible for notifying - // pending tasks. - return; - } - - let mut dec = 1; - - loop { - let mut remaining_pops = dec; - while remaining_pops > 0 { - remaining_pops -= 1; - - let task = match self.pop(remaining_pops) { - Some(t) => t, - None => break, - }; - - Task::notify_blocking(task, pool); - } - - // Decrement the number of handled notifications - let actual = self.lock.fetch_sub(dec, AcqRel); - - if actual == dec { - break; - } - - // This can only be greater than expected as we are the only thread - // that is decrementing. - debug_assert!(actual > dec); - dec = actual - dec; - } - } - - /// Pop a task - /// - /// `rem` represents the remaining number of times the caller will pop. If - /// there are no more tasks to pop, `rem` is used to set the remaining - /// capacity. - fn pop(&self, rem: usize) -> Option> { - 'outer: loop { - unsafe { - let mut tail = *self.tail.get(); - let mut next = (*tail).next_blocking.load(Acquire); - - let stub = &*self.stub as *const _ as *mut _; - - if tail == stub { - if next.is_null() { - // This loop is not part of the standard intrusive mpsc - // channel algorithm. This is where we atomically pop - // the last task and add `rem` to the remaining capacity. - // - // This modification to the pop algorithm works because, - // at this point, we have not done any work (only done - // reading). We have a *pretty* good idea that there is - // no concurrent pusher. - // - // The capacity is then atomically added by doing an - // AcqRel CAS on `state`. The `state` cell is the - // linchpin of the algorithm. - // - // By successfully CASing `head` w/ AcqRel, we ensure - // that, if any thread was racing and entered a push, we - // see that and abort pop, retrying as it is - // "inconsistent". - let mut curr: State = self.state.load(Acquire).into(); - - loop { - if curr.has_task(&self.stub) { - // Inconsistent state, yield the thread and try - // again. - thread::yield_now(); - continue 'outer; - } - - let mut after = curr; - - // +1 here because `rem` represents the number of - // pops that will come after the current one. - after.add_capacity(rem + 1, &self.stub); - - let actual: State = self - .state - .compare_and_swap(curr.into(), after.into(), AcqRel) - .into(); - - if actual == curr { - // Successfully returned the remaining capacity - return None; - } - - curr = actual; - } - } - - *self.tail.get() = next; - tail = next; - next = (*next).next_blocking.load(Acquire); - } - - if !next.is_null() { - *self.tail.get() = next; - - // No ref_count inc is necessary here as this poll is paired - // with a `push` which "forgets" the handle. - return Some(Arc::from_raw(tail)); - } - - let state = self.state.load(Acquire); - - // This must always be a pointer - debug_assert!(State::from(state).is_ptr()); - - if state != tail as usize { - // Try again - thread::yield_now(); - continue 'outer; - } - - self.push_stub(); - - next = (*tail).next_blocking.load(Acquire); - - if !next.is_null() { - *self.tail.get() = next; - - return Some(Arc::from_raw(tail)); - } - - thread::yield_now(); - // Try again - } - } - } -} - -// ====== impl State ===== - -impl State { - /// Return a new `State` representing the remaining capacity at the maximum - /// value. - fn new(capacity: usize) -> State { - State((capacity << NUM_SHIFT) | NUM_FLAG) - } - - fn remaining_capacity(self) -> usize { - if !self.has_remaining_capacity() { - return 0; - } - - self.0 >> 1 - } - - fn has_remaining_capacity(self) -> bool { - self.0 & NUM_FLAG == NUM_FLAG - } - - fn has_task(self, stub: &Task) -> bool { - !(self.has_remaining_capacity() || self.is_stub(stub)) - } - - fn is_stub(self, stub: &Task) -> bool { - self.0 == stub as *const _ as usize - } - - /// Try to claim blocking capacity. - /// - /// # Return - /// - /// Returns `true` if the capacity was claimed, `false` otherwise. If - /// `false` is returned, it can be assumed that `State` represents the head - /// pointer in the mpsc channel. - fn claim_capacity(&mut self, stub: &Task) -> bool { - if !self.has_remaining_capacity() { - return false; - } - - debug_assert!(self.0 != 1); - - self.0 -= 1 << NUM_SHIFT; - - if self.0 == NUM_FLAG { - // Set the state to the stub pointer. - self.0 = stub as *const _ as usize; - } - - true - } - - /// Add blocking capacity. - fn add_capacity(&mut self, capacity: usize, stub: &Task) -> bool { - debug_assert!(capacity > 0); - - if self.is_stub(stub) { - self.0 = (capacity << NUM_SHIFT) | NUM_FLAG; - true - } else if self.has_remaining_capacity() { - self.0 += capacity << NUM_SHIFT; - true - } else { - false - } - } - - fn is_ptr(self) -> bool { - self.0 & NUM_FLAG == 0 - } - - fn ptr(self) -> Option<*const Task> { - if self.is_ptr() { - Some(self.0 as *const Task) - } else { - None - } - } - - fn set_ptr(&mut self, ptr: *const Task) { - let ptr = ptr as usize; - debug_assert!(ptr & NUM_FLAG == 0); - self.0 = ptr - } -} - -impl From for State { - fn from(src: usize) -> State { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> usize { - src.0 - } -} - -impl fmt::Debug for State { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - let mut fmt = fmt.debug_struct("State"); - - if self.is_ptr() { - fmt.field("ptr", &self.0); - } else { - fmt.field("remaining", &self.remaining_capacity()); - } - - fmt.finish() - } -} diff --git a/tokio-executor/src/threadpool/task/blocking_state.rs b/tokio-executor/src/threadpool/task/blocking_state.rs deleted file mode 100644 index 8294f565ce1..00000000000 --- a/tokio-executor/src/threadpool/task/blocking_state.rs +++ /dev/null @@ -1,89 +0,0 @@ -use super::CanBlock; - -use std::fmt; -use std::sync::atomic::{AtomicUsize, Ordering}; - -/// State tracking task level state to support `blocking`. -/// -/// This tracks two separate flags. -/// -/// a) If the task is queued in the pending blocking channel. This prevents -/// double queuing (which would break the linked list). -/// -/// b) If the task has been allocated capacity to block. -#[derive(Eq, PartialEq)] -pub(crate) struct BlockingState(usize); - -const QUEUED: usize = 0b01; -const ALLOCATED: usize = 0b10; - -impl BlockingState { - /// Create a new, default, `BlockingState`. - pub(crate) fn new() -> BlockingState { - BlockingState(0) - } - - /// Returns `true` if the state represents the associated task being queued - /// in the pending blocking capacity channel - pub(crate) fn is_queued(&self) -> bool { - self.0 & QUEUED == QUEUED - } - - /// Toggle the queued flag - /// - /// Returns the state before the flag has been toggled. - pub(crate) fn toggle_queued(state: &AtomicUsize, ordering: Ordering) -> BlockingState { - state.fetch_xor(QUEUED, ordering).into() - } - - /// Returns `true` if the state represents the associated task having been - /// allocated capacity to block. - pub(crate) fn is_allocated(&self) -> bool { - self.0 & ALLOCATED == ALLOCATED - } - - /// Atomically consume the capacity allocation and return if the allocation - /// was present. - /// - /// If this returns `true`, then the task has the ability to block for the - /// duration of the `poll`. - pub(crate) fn consume_allocation(state: &AtomicUsize, ordering: Ordering) -> CanBlock { - let state: Self = state.fetch_and(!ALLOCATED, ordering).into(); - - if state.is_allocated() { - CanBlock::Allocated - } else if state.is_queued() { - CanBlock::NoCapacity - } else { - CanBlock::CanRequest - } - } - - pub(crate) fn notify_blocking(state: &AtomicUsize, ordering: Ordering) { - let prev: Self = state.fetch_xor(ALLOCATED | QUEUED, ordering).into(); - - debug_assert!(prev.is_queued()); - debug_assert!(!prev.is_allocated()); - } -} - -impl From for BlockingState { - fn from(src: usize) -> BlockingState { - BlockingState(src) - } -} - -impl From for usize { - fn from(src: BlockingState) -> usize { - src.0 - } -} - -impl fmt::Debug for BlockingState { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("BlockingState") - .field("is_queued", &self.is_queued()) - .field("is_allocated", &self.is_allocated()) - .finish() - } -} diff --git a/tokio-executor/src/threadpool/task/mod.rs b/tokio-executor/src/threadpool/task/mod.rs deleted file mode 100644 index c2fe9dc88af..00000000000 --- a/tokio-executor/src/threadpool/task/mod.rs +++ /dev/null @@ -1,323 +0,0 @@ -mod blocking; -mod blocking_state; -mod state; - -pub(crate) use self::blocking::{Blocking, CanBlock}; -use self::blocking_state::BlockingState; -use self::state::State; -use super::pool::Pool; -use super::waker::Waker; - -use futures_util::task; -use std::cell::{Cell, UnsafeCell}; -use std::future::Future; -use std::pin::Pin; -use std::sync::atomic::Ordering::{AcqRel, Acquire, Release}; -use std::sync::atomic::{AtomicPtr, AtomicUsize}; -use std::sync::Arc; -use std::task::{Context, Poll}; -use std::{fmt, panic, ptr}; - -/// Harness around a future. -/// -/// This also behaves as a node in the inbound work queue and the blocking -/// queue. -pub(crate) struct Task { - /// Task lifecycle state - state: AtomicUsize, - - /// Task blocking related state - blocking: AtomicUsize, - - /// Next pointer in the queue of tasks pending blocking capacity. - next_blocking: AtomicPtr, - - /// ID of the worker that polled this task first. - /// - /// This field can be a `Cell` because it's only accessed by the worker thread that is - /// executing the task. - /// - /// The worker ID is represented by a `u32` rather than `usize` in order to save some space - /// on 64-bit platforms. - pub(crate) reg_worker: Cell>, - - /// The key associated with this task in the `Slab` it was registered in. - /// - /// This field can be a `Cell` because it's only accessed by the worker thread that has - /// registered the task. - pub(crate) reg_index: Cell, - - /// Store the future at the head of the struct - /// - /// The future is dropped immediately when it transitions to Complete - future: UnsafeCell>, -} - -#[derive(Debug)] -pub(crate) enum Run { - Idle, - Schedule, - Complete, -} - -type BoxFuture = Pin + Send + 'static>>; - -// ===== impl Task ===== - -impl Task { - /// Create a new `Task` as a harness for `future`. - pub(crate) fn new(future: BoxFuture) -> Task { - Task { - state: AtomicUsize::new(State::new().into()), - blocking: AtomicUsize::new(BlockingState::new().into()), - next_blocking: AtomicPtr::new(ptr::null_mut()), - reg_worker: Cell::new(None), - reg_index: Cell::new(0), - future: UnsafeCell::new(Some(future)), - } - } - - /// Create a fake `Task` to be used as part of the intrusive mpsc channel - /// algorithm. - fn stub() -> Task { - let future = Box::pin(Empty) as BoxFuture; - - Task { - state: AtomicUsize::new(State::stub().into()), - blocking: AtomicUsize::new(BlockingState::new().into()), - next_blocking: AtomicPtr::new(ptr::null_mut()), - reg_worker: Cell::new(None), - reg_index: Cell::new(0), - future: UnsafeCell::new(Some(future)), - } - } - - /// Execute the task returning `Run::Schedule` if the task needs to be - /// scheduled again. - /// - // tracing macro expansion adds enough branches to make clippy angry here. - #[allow(clippy::cognitive_complexity)] // https://github.com/rust-lang/rust-clippy/issues/3900 - pub(crate) fn run(me: &Arc, pool: &Arc) -> Run { - use self::State::*; - #[cfg(feature = "tracing")] - use std::sync::atomic::Ordering::Relaxed; - - // Transition task to running state. At this point, the task must be - // scheduled. - let actual: State = me - .state - .compare_and_swap(Scheduled.into(), Running.into(), AcqRel) - .into(); - - match actual { - Scheduled => {} - _ => panic!("unexpected task state; {:?}", actual), - } - let span = trace_span!("Task::run"); - let _enter = span.enter(); - - trace!(state = ?State::from(me.state.load(Relaxed))); - - // The transition to `Running` done above ensures that a lock on the - // future has been obtained. - let fut = unsafe { &mut (*me.future.get()) }; - - // This block deals with the future panicking while being polled. - // - // If the future panics, then the drop handler must be called such that - // `thread::panicking() -> true`. To do this, the future is dropped from - // within the catch_unwind block. - let res = panic::catch_unwind(panic::AssertUnwindSafe(|| { - struct Guard<'a>(&'a mut Option, bool); - - impl Drop for Guard<'_> { - fn drop(&mut self) { - // This drops the future - if self.1 { - let _ = self.0.take(); - } - } - } - - let mut g = Guard(fut, true); - - let waker = task::waker(Arc::new(Waker { - task: me.clone(), - pool: pool.clone(), - })); - - let mut cx = Context::from_waker(&waker); - - let ret = g.0.as_mut().unwrap().as_mut().poll(&mut cx); - - g.1 = false; - - ret - })); - - match res { - Ok(Poll::Ready(_)) | Err(_) => { - trace!("task complete"); - - // The future has completed. Drop it immediately to free - // resources and run drop handlers. - // - // The `Task` harness will stay around longer if it is contained - // by any of the various queues. - me.drop_future(); - - // Transition to the completed state - me.state.store(State::Complete.into(), Release); - - if let Err(panic_err) = res { - if let Some(ref f) = pool.config.panic_handler { - f(panic_err); - } - } - - Run::Complete - } - Ok(Poll::Pending) => { - trace!("not ready"); - - // Attempt to transition from Running -> Idle, if successful, - // then the task does not need to be scheduled again. If the CAS - // fails, then the task has been unparked concurrent to running, - // in which case it transitions immediately back to scheduled - // and we return `true`. - let prev: State = me - .state - .compare_and_swap(Running.into(), Idle.into(), AcqRel) - .into(); - - match prev { - Running => Run::Idle, - Notified => { - me.state.store(Scheduled.into(), Release); - Run::Schedule - } - _ => unreachable!(), - } - } - } - } - - /// Aborts this task. - /// - /// This is called when the threadpool shuts down and the task has already beed polled but not - /// completed. - pub(crate) fn abort(&self) { - use self::State::*; - - let mut state = self.state.load(Acquire).into(); - - loop { - match state { - Idle | Scheduled => {} - Running | Notified | Complete | Aborted => { - // It is assumed that no worker threads are running so the task must be either - // in the idle or scheduled state. - panic!("unexpected state while aborting task: {:?}", state); - } - } - - let actual = self - .state - .compare_and_swap(state.into(), Aborted.into(), AcqRel) - .into(); - - if actual == state { - // The future has been aborted. Drop it immediately to free resources and run drop - // handlers. - self.drop_future(); - break; - } - - state = actual; - } - } - - /// Notify the task it has been allocated blocking capacity - pub(crate) fn notify_blocking(me: Arc, pool: &Arc) { - BlockingState::notify_blocking(&me.blocking, AcqRel); - Task::schedule(&me, pool); - } - - pub(crate) fn schedule(me: &Arc, pool: &Arc) { - if me.schedule2() { - let task = me.clone(); - pool.submit(task, &pool); - } - } - - /// Transition the task state to scheduled. - /// - /// Returns `true` if the caller is permitted to schedule the task. - fn schedule2(&self) -> bool { - use self::State::*; - - loop { - // Scheduling can only be done from the `Idle` state. - let actual = self - .state - .compare_and_swap(Idle.into(), Scheduled.into(), AcqRel) - .into(); - - match actual { - Idle => return true, - Running => { - // The task is already running on another thread. Transition - // the state to `Notified`. If this CAS fails, then restart - // the logic again from `Idle`. - let actual = self - .state - .compare_and_swap(Running.into(), Notified.into(), AcqRel) - .into(); - - match actual { - Idle => continue, - _ => return false, - } - } - Complete | Aborted | Notified | Scheduled => return false, - } - } - } - - /// Consumes any allocated capacity to block. - /// - /// Returns `true` if capacity was allocated, `false` otherwise. - pub(crate) fn consume_blocking_allocation(&self) -> CanBlock { - // This flag is the primary point of coordination. The queued flag - // happens "around" setting the blocking capacity. - BlockingState::consume_allocation(&self.blocking, AcqRel) - } - - /// Drop the future - /// - /// This must only be called by the thread that successfully transitioned - /// the future state to `Running`. - fn drop_future(&self) { - let _ = unsafe { (*self.future.get()).take() }; - } -} - -impl fmt::Debug for Task { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Task") - .field("state", &self.state) - .field("future", &"BoxFuture") - .finish() - } -} - -struct Empty; - -impl Future for Empty { - type Output = (); - - fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<()> { - // Never used - unreachable!(); - } -} diff --git a/tokio-executor/src/threadpool/task/state.rs b/tokio-executor/src/threadpool/task/state.rs deleted file mode 100644 index 89430a9e728..00000000000 --- a/tokio-executor/src/threadpool/task/state.rs +++ /dev/null @@ -1,57 +0,0 @@ -#[repr(usize)] -#[derive(Debug, Clone, Copy, Eq, PartialEq)] -pub(crate) enum State { - /// Task is currently idle - Idle = 0, - - /// Task is currently running - Running = 1, - - /// Task is currently running, but has been notified that it must run again. - Notified = 2, - - /// Task has been scheduled - Scheduled = 3, - - /// Task is complete - Complete = 4, - - /// Task was aborted because the thread pool has been shut down - Aborted = 5, -} - -// ===== impl State ===== - -impl State { - /// Returns the initial task state. - /// - /// Tasks start in the scheduled state as they are immediately scheduled on - /// creation. - pub(crate) fn new() -> State { - State::Scheduled - } - - pub(crate) fn stub() -> State { - State::Idle - } -} - -impl From for State { - fn from(src: usize) -> Self { - use self::State::*; - - debug_assert!( - src >= Idle as usize && src <= Aborted as usize, - "actual={}", - src - ); - - unsafe { ::std::mem::transmute(src) } - } -} - -impl From for usize { - fn from(src: State) -> Self { - src as usize - } -} diff --git a/tokio-executor/src/threadpool/thread_pool.rs b/tokio-executor/src/threadpool/thread_pool.rs deleted file mode 100644 index ff6ada03440..00000000000 --- a/tokio-executor/src/threadpool/thread_pool.rs +++ /dev/null @@ -1,226 +0,0 @@ -use super::builder::Builder; -use super::pool::Pool; -use super::sender::Sender; -use super::shutdown::{Shutdown, ShutdownTrigger}; - -use std::future::Future; -use std::sync::Arc; - -/// Work-stealing based thread pool for executing futures. -/// -/// If a `ThreadPool` instance is dropped without explicitly being shutdown, -/// `shutdown_now` is called implicitly, forcing all tasks that have not yet -/// completed to be dropped. -/// -/// Create `ThreadPool` instances using `Builder`. -#[derive(Debug)] -pub struct ThreadPool { - inner: Option, -} - -#[derive(Debug)] -struct Inner { - sender: Sender, - trigger: Arc, -} - -impl ThreadPool { - /// Create a new `ThreadPool` with default values. - /// - /// Use [`Builder`] for creating a configured thread pool. - /// - /// [`Builder`]: struct.Builder.html - pub fn new() -> ThreadPool { - Builder::new().build() - } - - pub(crate) fn new2(pool: Arc, trigger: Arc) -> ThreadPool { - ThreadPool { - inner: Some(Inner { - sender: Sender { pool }, - trigger, - }), - } - } - - /// Spawn a future onto the thread pool. - /// - /// This function takes ownership of the future and randomly assigns it to a - /// worker thread. The thread will then start executing the future. - /// - /// # Examples - /// - /// ```rust - /// use tokio_executor::threadpool::ThreadPool; - /// - /// // Create a thread pool with default configuration values - /// let thread_pool = ThreadPool::new(); - /// - /// thread_pool.spawn(async { - /// println!("called from a worker thread"); - /// }); - /// - /// // Gracefully shutdown the threadpool - /// thread_pool.shutdown().wait(); - /// ``` - /// - /// # Panics - /// - /// This function panics if the spawn fails. Use [`Sender::spawn`] for a - /// version that returns a `Result` instead of panicking. - pub fn spawn(&self, future: F) - where - F: Future + Send + 'static, - { - self.sender().spawn(future).unwrap(); - } - - /* - * TODO: Bring back - - /// Spawn a future on to the thread pool, return a future representing - /// the produced value. - /// - /// The SpawnHandle returned is a future that is a proxy for future itself. - /// When future completes on this thread pool then the SpawnHandle will itself - /// be resolved. - /// - /// # Examples - /// - /// ```rust - /// # use tokio_executor::threadpool::ThreadPool; - /// use futures::future::{Future, lazy}; - /// - /// // Create a thread pool with default configuration values - /// let thread_pool = ThreadPool::new(); - /// - /// let handle = thread_pool.spawn_handle(lazy(|| Ok::<_, ()>(42))); - /// - /// let value = handle.wait().unwrap(); - /// assert_eq!(value, 42); - /// - /// // Gracefully shutdown the threadpool - /// thread_pool.shutdown().wait().unwrap(); - /// ``` - /// - /// # Panics - /// - /// This function panics if the spawn fails. - pub fn spawn_handle(&self, future: F) -> SpawnHandle - where - F: Future + Send + 'static, - F::Item: Send + 'static, - F::Error: Send + 'static, - { - SpawnHandle(oneshot::spawn(future, self.sender())) - } - - */ - - /// Return a reference to the sender handle - /// - /// The handle is used to spawn futures onto the thread pool. It also - /// implements the `Executor` trait. - pub fn sender(&self) -> &Sender { - &self.inner.as_ref().unwrap().sender - } - - /// Return a mutable reference to the sender handle - pub fn sender_mut(&mut self) -> &mut Sender { - &mut self.inner.as_mut().unwrap().sender - } - - /// Shutdown the pool once it becomes idle. - /// - /// Idle is defined as the completion of all futures that have been spawned - /// onto the thread pool. There may still be outstanding handles when the - /// thread pool reaches an idle state. - /// - /// Once the idle state is reached, calling `spawn` on any outstanding - /// handle will result in an error. All worker threads are signaled and will - /// shutdown. The returned future completes once all worker threads have - /// completed the shutdown process. - pub fn shutdown_on_idle(mut self) -> Shutdown { - let inner = self.inner.take().unwrap(); - inner.sender.pool.shutdown(false, false); - Shutdown::new(&inner.trigger) - } - - /// Shutdown the pool - /// - /// This prevents the thread pool from accepting new tasks but will allow - /// any existing tasks to complete. - /// - /// Calling `spawn` on any outstanding handle will result in an error. All - /// worker threads are signaled and will shutdown. The returned future - /// completes once all worker threads have completed the shutdown process. - pub fn shutdown(mut self) -> Shutdown { - let inner = self.inner.take().unwrap(); - inner.sender.pool.shutdown(true, false); - Shutdown::new(&inner.trigger) - } - - /// Shutdown the pool immediately - /// - /// This will prevent the thread pool from accepting new tasks **and** - /// abort any tasks that are currently running on the thread pool. - /// - /// Calling `spawn` on any outstanding handle will result in an error. All - /// worker threads are signaled and will shutdown. The returned future - /// completes once all worker threads have completed the shutdown process. - pub fn shutdown_now(mut self) -> Shutdown { - let inner = self.inner.take().unwrap(); - inner.sender.pool.shutdown(true, true); - Shutdown::new(&inner.trigger) - } -} - -impl Drop for ThreadPool { - fn drop(&mut self) { - if let Some(inner) = self.inner.take() { - // Begin the shutdown process. - inner.sender.pool.shutdown(true, true); - let shutdown = Shutdown::new(&inner.trigger); - - // Drop `inner` in order to drop its shutdown trigger. - drop(inner); - - // Wait until all worker threads terminate and the threadpool's resources clean up. - let mut enter = match crate::enter() { - Ok(e) => e, - Err(_) => return, - }; - - enter.block_on(shutdown); - } - } -} - -impl Default for ThreadPool { - fn default() -> Self { - Self::new() - } -} - -/* - * TODO: Bring back - -/// Handle returned from ThreadPool::spawn_handle. -/// -/// This handle is a future representing the completion of a different future -/// spawned on to the thread pool. Created through the ThreadPool::spawn_handle -/// function this handle will resolve when the future provided resolves on the -/// thread pool. -#[derive(Debug)] -pub struct SpawnHandle(oneshot::SpawnHandle); - -impl Future for SpawnHandle { - type Item = T; - type Error = E; - - fn poll(&mut self) -> Poll { - self.0.poll() - } -} - -*/ diff --git a/tokio-executor/src/threadpool/waker.rs b/tokio-executor/src/threadpool/waker.rs deleted file mode 100644 index 454cbda4d02..00000000000 --- a/tokio-executor/src/threadpool/waker.rs +++ /dev/null @@ -1,24 +0,0 @@ -use super::pool::Pool; -use super::task::Task; - -use futures_util::task::ArcWake; -use std::sync::Arc; - -/// Implements the future `Waker` API. -/// -/// This is how external events are able to signal the task, informing it to try -/// to poll the future again. -#[derive(Debug)] -pub(crate) struct Waker { - pub(crate) pool: Arc, - pub(crate) task: Arc, -} - -unsafe impl Send for Waker {} -unsafe impl Sync for Waker {} - -impl ArcWake for Waker { - fn wake_by_ref(me: &Arc) { - Task::schedule(&me.task, &me.pool); - } -} diff --git a/tokio-executor/src/threadpool/worker/entry.rs b/tokio-executor/src/threadpool/worker/entry.rs deleted file mode 100644 index 03c3b21d38f..00000000000 --- a/tokio-executor/src/threadpool/worker/entry.rs +++ /dev/null @@ -1,329 +0,0 @@ -use super::super::park::{BoxPark, BoxUnpark}; -use super::super::task::Task; -use super::state::{State, PUSHED_MASK}; - -use crossbeam_deque::{Steal, Stealer, Worker}; -use crossbeam_queue::SegQueue; -use crossbeam_utils::CachePadded; -use slab::Slab; -use std::cell::UnsafeCell; -use std::fmt; -use std::sync::atomic::Ordering::{AcqRel, Acquire, Relaxed, Release}; -use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; -use std::sync::Arc; -use std::time::Duration; - -// TODO: None of the fields should be public -// -// It would also be helpful to split up the state across what fields / -// operations are thread-safe vs. which ones require ownership of the worker. -pub(crate) struct WorkerEntry { - // Worker state. This is mutated when notifying the worker. - // - // The `usize` value is deserialized to a `worker::State` instance. See - // comments on that type. - pub(crate) state: CachePadded, - - // Next entry in the parked Trieber stack - next_sleeper: UnsafeCell, - - // Worker half of deque - pub(crate) worker: Worker>, - - // Stealer half of deque - stealer: Stealer>, - - // Thread parker - park: UnsafeCell>, - - // Thread unparker - unpark: UnsafeCell>, - - // Tasks that have been first polled by this worker, but not completed yet. - running_tasks: UnsafeCell>>, - - // Tasks that have been first polled by this worker, but completed by another worker. - remotely_completed_tasks: SegQueue>, - - // Set to `true` when `remotely_completed_tasks` has tasks that need to be removed from - // `running_tasks`. - needs_drain: AtomicBool, -} - -impl WorkerEntry { - pub(crate) fn new(park: BoxPark, unpark: BoxUnpark) -> Self { - let w = Worker::new_fifo(); - let s = w.stealer(); - - WorkerEntry { - state: CachePadded::new(AtomicUsize::new(State::default().into())), - next_sleeper: UnsafeCell::new(0), - worker: w, - stealer: s, - park: UnsafeCell::new(Some(park)), - unpark: UnsafeCell::new(Some(unpark)), - running_tasks: UnsafeCell::new(Slab::new()), - remotely_completed_tasks: SegQueue::new(), - needs_drain: AtomicBool::new(false), - } - } - - /// Atomically unset the pushed flag. - /// - /// # Return - /// - /// The state *before* the push flag is unset. - /// - /// # Ordering - /// - /// The specified ordering is established on the entry's state variable. - pub(crate) fn fetch_unset_pushed(&self, ordering: Ordering) -> State { - self.state.fetch_and(!PUSHED_MASK, ordering).into() - } - - /// Submit a task to this worker while currently on the same thread that is - /// running the worker. - #[inline] - pub(crate) fn submit_internal(&self, task: Arc) { - self.push_internal(task); - } - - /// Notifies the worker and returns `false` if it needs to be spawned. - /// - /// # Ordering - /// - /// The `state` must have been obtained with an `Acquire` ordering. - #[inline] - pub(crate) fn notify(&self, mut state: State) -> bool { - use super::Lifecycle::*; - - loop { - let mut next = state; - next.notify(); - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if state == actual { - break; - } - - state = actual; - } - - match state.lifecycle() { - Sleeping => { - // The worker is currently sleeping, the condition variable must - // be signaled - self.unpark(); - true - } - Shutdown => false, - Running | Notified | Signaled => { - // In these states, the worker is active and will eventually see - // the task that was just submitted. - true - } - } - } - - /// Signals to the worker that it should stop - /// - /// `state` is the last observed state for the worker. This allows skipping - /// the initial load from the state atomic. - /// - /// # Return - /// - /// Returns `Ok` when the worker was successfully signaled. - /// - /// Returns `Err` if the worker has already terminated. - pub(crate) fn signal_stop(&self, mut state: State) { - use super::Lifecycle::*; - - // Transition the worker state to signaled - loop { - let mut next = state; - - match state.lifecycle() { - Shutdown => { - return; - } - Running | Sleeping => {} - Notified | Signaled => { - // These two states imply that the worker is active, thus it - // will eventually see the shutdown signal, so we don't need - // to do anything. - // - // The worker is forced to see the shutdown signal - // eventually as: - // - // a) No more work will arrive - // b) The shutdown signal is stored as the head of the - // sleep, stack which will prevent the worker from going to - // sleep again. - return; - } - } - - next.set_lifecycle(Signaled); - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - break; - } - - state = actual; - } - - // Wakeup the worker - self.unpark(); - } - - /// Pop a task - /// - /// This **must** only be called by the thread that owns the worker entry. - /// This function is not `Sync`. - #[inline] - pub(crate) fn pop_task(&self) -> Option> { - self.worker.pop() - } - - /// Steal tasks - /// - /// This is called by *other* workers to steal a task for processing. This - /// function is `Sync`. - /// - /// At the same time, this method steals some additional tasks and moves - /// them into `dest` in order to balance the work distribution among - /// workers. - pub(crate) fn steal_tasks(&self, dest: &Self) -> Steal> { - self.stealer.steal_batch_and_pop(&dest.worker) - } - - /// Drain (and drop) all tasks that are queued for work. - /// - /// This is called when the pool is shutting down. - pub(crate) fn drain_tasks(&self) { - while self.worker.pop().is_some() {} - } - - /// Parks the worker thread. - pub(crate) fn park(&self) { - if let Some(park) = unsafe { (*self.park.get()).as_mut() } { - park.park().unwrap(); - } - } - - /// Parks the worker thread for at most `duration`. - pub(crate) fn park_timeout(&self, duration: Duration) { - if let Some(park) = unsafe { (*self.park.get()).as_mut() } { - park.park_timeout(duration).unwrap(); - } - } - - /// Unparks the worker thread. - #[inline] - pub(crate) fn unpark(&self) { - if let Some(park) = unsafe { (*self.unpark.get()).as_ref() } { - park.unpark(); - } - } - - /// Registers a task in this worker. - /// - /// Called when the task is being polled for the first time. - #[inline] - pub(crate) fn register_task(&self, task: &Arc) { - let running_tasks = unsafe { &mut *self.running_tasks.get() }; - - let key = running_tasks.insert(task.clone()); - task.reg_index.set(key); - } - - /// Unregisters a task from this worker. - /// - /// Called when the task is completed and was previously registered in this worker. - #[inline] - pub(crate) fn unregister_task(&self, task: Arc) { - let running_tasks = unsafe { &mut *self.running_tasks.get() }; - running_tasks.remove(task.reg_index.get()); - self.drain_remotely_completed_tasks(); - } - - /// Unregisters a task from this worker. - /// - /// Called when the task is completed by another worker and was previously registered in this - /// worker. - #[inline] - pub(crate) fn remotely_complete_task(&self, task: Arc) { - self.remotely_completed_tasks.push(task); - self.needs_drain.store(true, Release); - } - - /// Drops the remaining incomplete tasks and the parker associated with this worker. - /// - /// This function is called by the shutdown trigger. - pub(crate) fn shutdown(&self) { - self.drain_remotely_completed_tasks(); - - // Abort all incomplete tasks. - let running_tasks = unsafe { &mut *self.running_tasks.get() }; - for (_, task) in running_tasks.iter() { - task.abort(); - } - running_tasks.clear(); - - unsafe { - *self.park.get() = None; - *self.unpark.get() = None; - } - } - - /// Drains the `remotely_completed_tasks` queue and removes tasks from `running_tasks`. - #[inline] - fn drain_remotely_completed_tasks(&self) { - if self.needs_drain.compare_and_swap(true, false, Acquire) { - let running_tasks = unsafe { &mut *self.running_tasks.get() }; - - while let Ok(task) = self.remotely_completed_tasks.pop() { - running_tasks.remove(task.reg_index.get()); - } - } - } - - #[inline] - pub(crate) fn push_internal(&self, task: Arc) { - self.worker.push(task); - } - - #[inline] - pub(crate) fn next_sleeper(&self) -> usize { - unsafe { *self.next_sleeper.get() } - } - - #[inline] - pub(crate) fn set_next_sleeper(&self, val: usize) { - unsafe { - *self.next_sleeper.get() = val; - } - } -} - -impl fmt::Debug for WorkerEntry { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("WorkerEntry") - .field("state", &self.state.load(Relaxed)) - .field("next_sleeper", &"UnsafeCell") - .field("worker", &self.worker) - .field("stealer", &self.stealer) - .field("park", &"UnsafeCell") - .field("unpark", &"BoxUnpark") - .finish() - } -} diff --git a/tokio-executor/src/threadpool/worker/mod.rs b/tokio-executor/src/threadpool/worker/mod.rs deleted file mode 100644 index 91c58e48adb..00000000000 --- a/tokio-executor/src/threadpool/worker/mod.rs +++ /dev/null @@ -1,792 +0,0 @@ -mod entry; -mod stack; -mod state; - -pub(crate) use self::entry::WorkerEntry as Entry; -pub(crate) use self::stack::Stack; -pub(crate) use self::state::{Lifecycle, State}; - -use super::pool::{self, BackupId, Pool}; -use super::sender::Sender; -use super::shutdown::ShutdownTrigger; -use super::task::{self, CanBlock, Task}; -use super::BlockingError; - -use std::cell::Cell; -use std::marker::PhantomData; -use std::ptr; -use std::rc::Rc; -use std::sync::atomic::Ordering::{AcqRel, Acquire}; -use std::sync::Arc; -use std::task::Poll; -use std::thread; -use std::time::Duration; - -/// Thread worker -/// -/// This is passed to the [`around_worker`] callback set on [`Builder`]. This -/// callback is only expected to call [`run`] on it. -/// -/// [`Builder`]: struct.Builder.html -/// [`around_worker`]: struct.Builder.html#method.around_worker -/// [`run`]: struct.Worker.html#method.run -#[derive(Debug)] -pub struct Worker { - // Shared scheduler data - pub(crate) pool: Arc, - - // WorkerEntry index - pub(crate) id: WorkerId, - - // Backup thread ID assigned to processing this worker. - backup_id: BackupId, - - // Set to the task that is currently being polled by the worker. This is - // needed so that `blocking` blocks are able to interact with this task. - // - // This has to be a raw pointer to make it compile, but great care is taken - // when this is set. - current_task: CurrentTask, - - // Set when the thread is in blocking mode. - is_blocking: Cell, - - // Set when the worker should finalize on drop - should_finalize: Cell, - - // Completes the shutdown process when the `ThreadPool` and all `Worker`s get dropped. - trigger: Arc, - - // Keep the value on the current thread. - _p: PhantomData>, -} - -/// Tracks the state related to the currently running task. -#[derive(Debug)] -struct CurrentTask { - /// This has to be a raw pointer to make it compile, but great care is taken - /// when this is set. - task: Cell>>, - - /// Tracks the blocking capacity allocation state. - can_block: Cell, -} - -/// Identifies a thread pool worker. -/// -/// This identifier is unique scoped by the thread pool. It is possible that -/// different thread pool instances share worker identifier values. -#[derive(Debug, Clone, Hash, Eq, PartialEq)] -pub struct WorkerId(pub(crate) usize); - -// Pointer to the current worker info -thread_local!(static CURRENT_WORKER: Cell<*const Worker> = Cell::new(ptr::null())); - -impl Worker { - pub(crate) fn new( - id: WorkerId, - backup_id: BackupId, - pool: Arc, - trigger: Arc, - ) -> Worker { - Worker { - pool, - id, - backup_id, - current_task: CurrentTask::new(), - is_blocking: Cell::new(false), - should_finalize: Cell::new(false), - trigger, - _p: PhantomData, - } - } - - pub(crate) fn is_blocking(&self) -> bool { - self.is_blocking.get() - } - - /// Run the worker - /// - /// Returns `true` if the thread should keep running as a `backup` thread. - pub(crate) fn do_run(&self) -> bool { - // Create another worker... It's ok, this is just a new type around - // `Pool` that is expected to stay on the current thread. - CURRENT_WORKER.with(|c| { - c.set(self as *const _); - - let pool = self.pool.clone(); - let mut sender = Sender { pool }; - - // Enter an execution context - let _enter = crate::enter().unwrap(); - - crate::with_default(&mut sender, || { - if let Some(ref callback) = self.pool.config.around_worker { - callback.call(self); - } else { - self.run(); - } - }); - }); - - // Can't be in blocking mode and finalization mode - debug_assert!(!self.is_blocking.get() || !self.should_finalize.get()); - - self.is_blocking.get() - } - - pub(crate) fn with_current) -> R, R>(f: F) -> R { - CURRENT_WORKER.with(move |c| { - let ptr = c.get(); - - if ptr.is_null() { - f(None) - } else { - f(Some(unsafe { &*ptr })) - } - }) - } - - /// Transition the current worker to a blocking worker - pub(crate) fn transition_to_blocking(&self) -> Poll> { - use self::CanBlock::*; - - // If we get this far, then `current_task` has been set. - let task_ref = self.current_task.get_ref(); - - // First step is to acquire blocking capacity for the task. - match self.current_task.can_block() { - // Capacity to block has already been allocated to this task. - Allocated => {} - - // The task has already requested capacity to block, but there is - // none yet available. - NoCapacity => return Poll::Pending, - - // The task has yet to ask for capacity - CanRequest => { - // Atomically attempt to acquire blocking capacity, and if none - // is available, register the task to be notified once capacity - // becomes available. - match self.pool.poll_blocking_capacity(task_ref)? { - Poll::Ready(()) => { - self.current_task.set_can_block(Allocated); - } - Poll::Pending => { - self.current_task.set_can_block(NoCapacity); - return Poll::Pending; - } - } - } - } - - // The task has been allocated blocking capacity. At this point, this is - // when the current thread transitions from a worker to a backup thread. - // To do so requires handing over the worker to another backup thread. - - if self.is_blocking.get() { - // The thread is already in blocking mode, so there is nothing else - // to do. Return `Ready` and allow the caller to block the thread. - return Poll::Ready(Ok(())); - } - - trace!("transition to blocking state"); - - // Transitioning to blocking requires handing over the worker state to - // another thread so that the work queue can continue to be processed. - - self.pool.spawn_thread(self.id.clone(), &self.pool); - - // Track that the thread has now fully entered the blocking state. - self.is_blocking.set(true); - - Poll::Ready(Ok(())) - } - - /// Transition from blocking - pub(crate) fn transition_from_blocking(&self) { - // TODO: Attempt to take ownership of the worker again. - } - - /// Returns a reference to the worker's identifier. - /// - /// This identifier is unique scoped by the thread pool. It is possible that - /// different thread pool instances share worker identifier values. - pub fn id(&self) -> &WorkerId { - &self.id - } - - /// Run the worker - /// - /// This function blocks until the worker is shutting down. - pub fn run(&self) { - const MAX_SPINS: usize = 3; - const LIGHT_SLEEP_INTERVAL: usize = 32; - - let mut first = true; - let mut spin_cnt = 0; - let mut tick = 0; - - while self.check_run_state(first) { - first = false; - - // Run the next available task - if self.try_run_task(&self.pool) { - if self.is_blocking.get() { - // Exit out of the run state - return; - } - - // Poll the reactor and the global queue every now and then to - // ensure no task gets left behind. - if tick % LIGHT_SLEEP_INTERVAL == 0 { - self.sleep_light(); - } - - tick = tick.wrapping_add(1); - spin_cnt = 0; - - // As long as there is work, keep looping. - continue; - } - - spin_cnt += 1; - - // Yield the thread several times before it actually goes to sleep. - if spin_cnt <= MAX_SPINS { - thread::yield_now(); - continue; - } - - tick = 0; - spin_cnt = 0; - - // Starting to get sleeeeepy - if !self.sleep() { - return; - } - - // If there still isn't any work to do, shutdown the worker? - } - - // The pool is terminating. However, transitioning the pool state to - // terminated is the very first step of the finalization process. Other - // threads may not see this state and try to spawn a new thread. To - // ensure consistency, before the current thread shuts down, it must - // return the backup token to the stack. - // - // The returned result is ignored because `Err` represents the pool - // shutting down. We are currently aware of this fact. - let _ = self.pool.release_backup(self.backup_id); - - self.should_finalize.set(true); - } - - /// Try to run a task - /// - /// Returns `true` if work was found. - #[inline] - fn try_run_task(&self, pool: &Arc) -> bool { - if self.try_run_owned_task(pool) { - return true; - } - - self.try_steal_task(pool) - } - - /// Checks the worker's current state, updating it as needed. - /// - /// Returns `true` if the worker should run. - #[inline] - fn check_run_state(&self, first: bool) -> bool { - use self::Lifecycle::*; - - debug_assert!(!self.is_blocking.get()); - - let mut state: State = self.entry().state.load(Acquire).into(); - - loop { - let pool_state: pool::State = self.pool.state.load(Acquire).into(); - - if pool_state.is_terminated() { - return false; - } - - let mut next = state; - - match state.lifecycle() { - Running => break, - Notified | Signaled => { - // transition back to running - next.set_lifecycle(Running); - } - Shutdown | Sleeping => { - // The worker should never be in these states when calling - // this function. - panic!("unexpected worker state; lifecycle={:?}", state.lifecycle()); - } - } - - let actual = self - .entry() - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - break; - } - - state = actual; - } - - // `first` is set to true the first time this function is called after - // the thread has started. - // - // This check is to handle the scenario where a worker gets signaled - // while it is already happily running. The `is_signaled` state is - // intended to wake up a worker that has been previously sleeping in - // effect increasing the number of active workers. If this is the first - // time `check_run_state` is called, then being in a signalled state is - // normal and the thread was started to handle it. However, if this is - // **not** the first time the fn was called, then the number of active - // workers has not been increased by the signal, so `signal_work` has to - // be called again to try to wake up another worker. - // - // For example, if the thread pool is configured to allow 4 workers. - // Worker 1 is processing tasks from its `deque`. Worker 2 receives its - // first task. Worker 2 will pick a random worker to signal. It does - // this by popping off the sleep stack, but there is no guarantee that - // workers on the sleep stack are actually sleeping. It is possible that - // Worker 1 gets signaled. - // - // Without this check, in the above case, no additional workers will get - // started, which results in the thread pool permanently being at 2 - // workers even though it should reach 4. - if !first && state.is_signaled() { - trace!("Worker::check_run_state; delegate signal"); - // This worker is not ready to be signaled, so delegate the signal - // to another worker. - self.pool.signal_work(&self.pool); - } - - true - } - - /// Runs the next task on this worker's queue. - /// - /// Returns `true` if work was found. - fn try_run_owned_task(&self, pool: &Arc) -> bool { - // Poll the internal queue for a task to run - match self.entry().pop_task() { - Some(task) => { - self.run_task(task, pool); - true - } - None => false, - } - } - - /// Tries to steal a task from another worker. - /// - /// Returns `true` if work was found - fn try_steal_task(&self, pool: &Arc) -> bool { - use crossbeam_deque::Steal; - - debug_assert!(!self.is_blocking.get()); - - let len = self.pool.workers.len(); - let mut idx = self.pool.rand_usize() % len; - let mut found_work = false; - let start = idx; - - loop { - if idx < len { - match self.pool.workers[idx].steal_tasks(self.entry()) { - Steal::Success(task) => { - trace!("stole task from another worker"); - - self.run_task(task, pool); - - trace!( - message = "try_steal_task -- signal_work;", - self = self.id.0, - from = idx, - ); - - // Signal other workers that work is available - // - // TODO: Should this be called here or before - // `run_task`? - self.pool.signal_work(&self.pool); - - return true; - } - Steal::Empty => {} - Steal::Retry => found_work = true, - } - - idx += 1; - } else { - idx = 0; - } - - if idx == start { - break; - } - } - - found_work - } - - fn run_task(&self, task: Arc, pool: &Arc) { - use super::task::Run::*; - - // If this is the first time this task is being polled, register it so that we can keep - // track of tasks that are in progress. - if task.reg_worker.get().is_none() { - task.reg_worker.set(Some(self.id.0 as u32)); - self.entry().register_task(&task); - } - - let run = self.run_task2(&task, pool); - - // TODO: Try to claim back the worker state in case the backup thread - // did not start up fast enough. This is a performance optimization. - - match run { - Idle => {} - Schedule => { - if self.is_blocking.get() { - // The future has been notified while it was running. - // However, the future also entered a blocking section, - // which released the worker state from this thread. - // - // This means that scheduling the future must be done from - // a point of view external to the worker set. - // - // We have to call `submit_external` instead of `submit` - // here because `self` is still set as the current worker. - self.pool.submit_external(task, &self.pool); - } else { - self.entry().push_internal(task); - } - } - Complete => { - let mut state: pool::State = self.pool.state.load(Acquire).into(); - - loop { - let mut next = state; - next.dec_num_futures(); - - let actual = self - .pool - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - trace!(message = "task complete;", state = ?next); - - if state.num_futures() == 1 { - // If the thread pool has been flagged as shutdown, - // start terminating workers. This involves waking - // up any sleeping worker so that they can notice - // the shutdown state. - if next.is_terminated() { - self.pool.terminate_sleeping_workers(); - } - } - - // Find which worker polled this task first. - let worker = task.reg_worker.get().unwrap() as usize; - - // Unregister the task from the worker it was registered in. - if !self.is_blocking.get() && worker == self.id.0 { - self.entry().unregister_task(task); - } else { - self.pool.workers[worker].remotely_complete_task(task); - } - - // The worker's run loop will detect the shutdown state - // next iteration. - return; - } - - state = actual; - } - } - } - } - - /// Actually run the task. This is where `Worker::current_task` is set. - /// - /// Great care is needed to ensure that `current_task` is unset in this - /// function. - fn run_task2(&self, task: &Arc, pool: &Arc) -> task::Run { - struct Guard<'a> { - worker: &'a Worker, - } - - impl Drop for Guard<'_> { - fn drop(&mut self) { - // A task is allocated at run when it was explicitly notified - // that the task has capacity to block. When this happens, that - // capacity is automatically allocated to the notified task. - // This capacity is "use it or lose it", so if the thread is not - // transitioned to blocking in this call, then another task has - // to be notified. - // - // If the task has consumed its blocking allocation but hasn't - // used it, it must be given to some other task instead. - if !self.worker.is_blocking.get() { - let can_block = self.worker.current_task.can_block(); - if can_block == CanBlock::Allocated { - self.worker.pool.notify_blocking_task(&self.worker.pool); - } - } - - self.worker.current_task.clear(); - } - } - - // Set `current_task` - self.current_task.set(task, CanBlock::CanRequest); - - // Create the guard, this ensures that `current_task` is unset when the - // function returns, even if the return is caused by a panic. - let _g = Guard { worker: self }; - - Task::run(task, pool) - } - - /// Put the worker to sleep - /// - /// Returns `true` if woken up due to new work arriving. - // tracing macro expansion adds enough branches to make clippy angry here. - #[allow(clippy::cognitive_complexity)] // https://github.com/rust-lang/rust-clippy/issues/3900 - fn sleep(&self) -> bool { - use self::Lifecycle::*; - - // Putting a worker to sleep is a multipart operation. This is, in part, - // due to the fact that a worker can be notified without it being popped - // from the sleep stack. Extra care is needed to deal with this. - let span = trace_span!("Worker::sleep", idx = self.id.0, id = ?self.id); - let _e = span.enter(); - - let mut state: State = self.entry().state.load(Acquire).into(); - - // The first part of the sleep process is to transition the worker state - // to "pushed". Now, it may be that the worker is already pushed on the - // sleeper stack, in which case, we don't push again. - - loop { - let mut next = state; - - match state.lifecycle() { - Running => { - // Try setting the pushed state - next.set_pushed(); - - // Transition the worker state to sleeping - next.set_lifecycle(Sleeping); - } - Notified | Signaled => { - // No need to sleep, transition back to running and move on. - next.set_lifecycle(Running); - } - Shutdown | Sleeping => { - // The worker cannot transition to sleep when already in a - // sleeping state. - panic!("unexpected worker state; actual={:?}", state.lifecycle()); - } - } - - let actual = self - .entry() - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - if state.is_notified() { - // The previous state was notified, so we don't need to - // sleep. - return true; - } - - if !state.is_pushed() { - debug_assert!(next.is_pushed()); - - trace!("push to stack"); - - // We obtained permission to push the worker into the - // sleeper queue. - if self.pool.push_sleeper(self.id.0).is_err() { - trace!("push to stack failed"); - // The push failed due to the pool being terminated. - // - // This is true because the "work" being woken up for is - // shutting down. - return true; - } - } - - break; - } - - state = actual; - } - - trace!("starting to sleep"); - - // Do a quick check to see if there are any notifications in the - // reactor or new tasks in the global queue. Since this call will - // clear the wakeup token, we need to check the state again and - // only after that go to sleep. - self.sleep_light(); - - // The state has been transitioned to sleeping, we can now wait by - // calling the parker. This is done in a loop as condvars can wakeup - // spuriously. - loop { - // Reload the state - state = self.entry().state.load(Acquire).into(); - - // If the worker has been notified, transition back to running. - match state.lifecycle() { - Sleeping => { - // Still sleeping. Park again. - } - Notified | Signaled => { - // Transition back to running - loop { - let mut next = state; - next.set_lifecycle(Running); - - let actual = self - .entry() - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - return true; - } - - state = actual; - } - } - Shutdown | Running => { - // To get here, the block above transitioned the state to - // `Sleeping`. No other thread can concurrently - // transition to `Shutdown` or `Running`. - unreachable!(); - } - } - - self.entry().park(); - - trace!("wakeup"); - } - } - - /// This doesn't actually put the thread to sleep. It calls - /// `park.park_timeout` with a duration of 0. This allows the park - /// implementation to perform any work that might be done on an interval. - /// - /// Returns `true` if this worker has tasks in its queue. - fn sleep_light(&self) { - self.entry().park_timeout(Duration::from_millis(0)); - - use crossbeam_deque::Steal; - loop { - match self.pool.queue.steal_batch(&self.entry().worker) { - Steal::Success(()) => { - self.pool.signal_work(&self.pool); - break; - } - Steal::Empty => break, - Steal::Retry => {} - } - } - } - - fn entry(&self) -> &Entry { - debug_assert!(!self.is_blocking.get()); - &self.pool.workers[self.id.0] - } -} - -impl Drop for Worker { - fn drop(&mut self) { - trace!(message = "shutting down thread", idx = self.id.0); - - if self.should_finalize.get() { - // Drain the work queue - self.entry().drain_tasks(); - } - } -} - -// ===== impl CurrentTask ===== - -impl CurrentTask { - /// Returns a default `CurrentTask` representing no task. - fn new() -> CurrentTask { - CurrentTask { - task: Cell::new(None), - can_block: Cell::new(CanBlock::CanRequest), - } - } - - /// Returns a reference to the task. - fn get_ref(&self) -> &Arc { - unsafe { &*self.task.get().unwrap() } - } - - fn can_block(&self) -> CanBlock { - use self::CanBlock::*; - - match self.can_block.get() { - Allocated => Allocated, - CanRequest | NoCapacity => { - let can_block = self.get_ref().consume_blocking_allocation(); - self.can_block.set(can_block); - can_block - } - } - } - - fn set_can_block(&self, can_block: CanBlock) { - self.can_block.set(can_block); - } - - fn set(&self, task: &Arc, can_block: CanBlock) { - self.task.set(Some(task as *const _)); - self.can_block.set(can_block); - } - - /// Reset the `CurrentTask` to null state. - fn clear(&self) { - self.task.set(None); - self.can_block.set(CanBlock::CanRequest); - } -} - -// ===== impl WorkerId ===== - -impl WorkerId { - /// Returns a `WorkerId` representing the worker entry at index `idx`. - pub(crate) fn new(idx: usize) -> WorkerId { - WorkerId(idx) - } - - /// Returns this identifier represented as an integer. - /// - /// Worker identifiers in a single thread pool are guaranteed to correspond to integers in the - /// range `0..pool_size`. - pub fn to_usize(&self) -> usize { - self.0 - } -} diff --git a/tokio-executor/src/threadpool/worker/stack.rs b/tokio-executor/src/threadpool/worker/stack.rs deleted file mode 100644 index 2559fb68e6a..00000000000 --- a/tokio-executor/src/threadpool/worker/stack.rs +++ /dev/null @@ -1,257 +0,0 @@ -use super::super::config::MAX_WORKERS; -use super::super::worker; - -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::{AcqRel, Acquire, Relaxed}; -use std::{fmt, usize}; - -/// Lock-free stack of sleeping workers. -/// -/// This is implemented as a Treiber stack and references to nodes are -/// `usize` values, indexing the entry in the `[worker::Entry]` array stored by -/// `Pool`. Each `Entry` instance maintains a `pushed` bit in its state. This -/// bit tracks if the entry is already pushed onto the stack or not. A single -/// entry can only be stored on the stack a single time. -/// -/// By using indexes instead of pointers, that allows a much greater amount of -/// data to be used for the ABA guard (see correctness section of wikipedia -/// page). -/// -/// Treiber stack: https://en.wikipedia.org/wiki/Treiber_Stack -#[derive(Debug)] -pub(crate) struct Stack { - state: AtomicUsize, -} - -/// State related to the stack of sleeping workers. -/// -/// - Parked head 16 bits -/// - Sequence remaining -/// -/// The parked head value has a couple of special values: -/// -/// - EMPTY: No sleepers -/// - TERMINATED: Don't spawn more threads -#[derive(Eq, PartialEq, Clone, Copy)] -pub struct State(usize); - -/// Extracts the head of the worker stack from the scheduler state -/// -/// The 16 relates to the value of MAX_WORKERS -const STACK_MASK: usize = ((1 << 16) - 1); - -/// Used to mark the stack as empty -pub(crate) const EMPTY: usize = MAX_WORKERS; - -/// Used to mark the stack as terminated -pub(crate) const TERMINATED: usize = EMPTY + 1; - -/// How many bits the Treiber ABA guard is offset by -const ABA_GUARD_SHIFT: usize = 16; - -#[cfg(target_pointer_width = "64")] -const ABA_GUARD_MASK: usize = (1 << (64 - ABA_GUARD_SHIFT)) - 1; - -#[cfg(target_pointer_width = "32")] -const ABA_GUARD_MASK: usize = (1 << (32 - ABA_GUARD_SHIFT)) - 1; - -// ===== impl Stack ===== - -impl Stack { - /// Create a new `Stack` representing the empty state. - pub(crate) fn new() -> Stack { - let state = AtomicUsize::new(State::new().into()); - Stack { state } - } - - /// Push a worker onto the stack - /// - /// # Return - /// - /// Returns `Ok` on success. - /// - /// Returns `Err` if the pool has transitioned to the `TERMINATED` state. - /// When terminated, pushing new entries is no longer permitted. - pub(crate) fn push(&self, entries: &[worker::Entry], idx: usize) -> Result<(), ()> { - let mut state: State = self.state.load(Acquire).into(); - - debug_assert!(worker::State::from(entries[idx].state.load(Relaxed)).is_pushed()); - - loop { - let mut next = state; - - let head = state.head(); - - if head == TERMINATED { - // The pool is terminated, cannot push the sleeper. - return Err(()); - } - - entries[idx].set_next_sleeper(head); - next.set_head(idx); - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if state == actual { - return Ok(()); - } - - state = actual; - } - } - - /// Pop a worker off the stack. - /// - /// If `terminate` is set and the stack is empty when this function is - /// called, the state of the stack is transitioned to "terminated". At this - /// point, no further workers can be pushed onto the stack. - /// - /// # Return - /// - /// Returns the index of the popped worker and the worker's observed state. - /// - /// `None` if the stack is empty. - pub(crate) fn pop( - &self, - entries: &[worker::Entry], - max_lifecycle: worker::Lifecycle, - terminate: bool, - ) -> Option<(usize, worker::State)> { - // Figure out the empty value - let terminal = if terminate { TERMINATED } else { EMPTY }; - - // If terminating, the max lifecycle *must* be `Signaled`, which is the - // highest lifecycle. By passing the greatest possible lifecycle value, - // no entries are skipped by this function. - // - // TODO: It would be better to terminate in a separate function that - // atomically takes all values and transitions to a terminated state. - debug_assert!(!terminate || max_lifecycle == worker::Lifecycle::Signaled); - - let mut state: State = self.state.load(Acquire).into(); - - loop { - let head = state.head(); - - if head == EMPTY { - let mut next = state; - next.set_head(terminal); - - if next == state { - debug_assert!(terminal == EMPTY); - return None; - } - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual != state { - state = actual; - continue; - } - - return None; - } else if head == TERMINATED { - return None; - } - - debug_assert!(head < MAX_WORKERS); - - let mut next = state; - - let next_head = entries[head].next_sleeper(); - - // TERMINATED can never be set as the "next pointer" on a worker. - debug_assert!(next_head != TERMINATED); - - if next_head == EMPTY { - next.set_head(terminal); - } else { - next.set_head(next_head); - } - - let actual = self - .state - .compare_and_swap(state.into(), next.into(), AcqRel) - .into(); - - if actual == state { - // Release ordering is needed to ensure that unsetting the - // `pushed` flag happens after popping the sleeper from the - // stack. - // - // Acquire ordering is required to acquire any memory associated - // with transitioning the worker's lifecycle. - let state = entries[head].fetch_unset_pushed(AcqRel); - - if state.lifecycle() >= max_lifecycle { - // If the worker has already been notified, then it is - // warming up to do more work. In this case, try to pop - // another thread that might be in a relaxed state. - continue; - } - - return Some((head, state)); - } - - state = actual; - } - } -} - -// ===== impl State ===== - -impl State { - #[inline] - fn new() -> State { - State(EMPTY) - } - - #[inline] - fn head(self) -> usize { - self.0 & STACK_MASK - } - - #[inline] - fn set_head(&mut self, val: usize) { - // The ABA guard protects against the ABA problem w/ Treiber stacks - let aba_guard = ((self.0 >> ABA_GUARD_SHIFT) + 1) & ABA_GUARD_MASK; - - self.0 = (aba_guard << ABA_GUARD_SHIFT) | val; - } -} - -impl From for State { - fn from(src: usize) -> Self { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> Self { - src.0 - } -} - -impl fmt::Debug for State { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - let head = self.head(); - - let mut fmt = fmt.debug_struct("stack::State"); - - if head < MAX_WORKERS { - fmt.field("head", &head); - } else if head == EMPTY { - fmt.field("head", &"EMPTY"); - } else if head == TERMINATED { - fmt.field("head", &"TERMINATED"); - } - - fmt.finish() - } -} diff --git a/tokio-executor/src/threadpool/worker/state.rs b/tokio-executor/src/threadpool/worker/state.rs deleted file mode 100644 index 68b5bc47751..00000000000 --- a/tokio-executor/src/threadpool/worker/state.rs +++ /dev/null @@ -1,153 +0,0 @@ -use std::fmt; - -/// Tracks worker state -#[derive(Clone, Copy, Eq, PartialEq)] -pub(crate) struct State(usize); - -/// Set when the worker is pushed onto the scheduler's stack of sleeping -/// threads. -pub(crate) const PUSHED_MASK: usize = 0b001; - -/// Manages the worker lifecycle part of the state -const LIFECYCLE_MASK: usize = 0b1110; -const LIFECYCLE_SHIFT: usize = 1; - -#[derive(Debug, Eq, PartialEq, Ord, PartialOrd, Clone, Copy)] -#[repr(usize)] -pub(crate) enum Lifecycle { - /// The worker does not currently have an associated thread. - Shutdown = 0 << LIFECYCLE_SHIFT, - - /// The worker is doing work - Running = 1 << LIFECYCLE_SHIFT, - - /// The worker is currently asleep in the condvar - Sleeping = 2 << LIFECYCLE_SHIFT, - - /// The worker has been notified it should process more work. - Notified = 3 << LIFECYCLE_SHIFT, - - /// A stronger form of notification. In this case, the worker is expected to - /// wakeup and try to acquire more work... if it enters this state while - /// already busy with other work, it is expected to signal another worker. - Signaled = 4 << LIFECYCLE_SHIFT, -} - -impl State { - /// Returns true if the worker entry is pushed in the sleeper stack - pub(crate) fn is_pushed(self) -> bool { - self.0 & PUSHED_MASK == PUSHED_MASK - } - - pub(crate) fn set_pushed(&mut self) { - self.0 |= PUSHED_MASK - } - - pub(crate) fn is_notified(self) -> bool { - use self::Lifecycle::*; - - match self.lifecycle() { - Notified | Signaled => true, - _ => false, - } - } - - pub(crate) fn lifecycle(self) -> Lifecycle { - Lifecycle::from(self.0 & LIFECYCLE_MASK) - } - - pub(crate) fn set_lifecycle(&mut self, val: Lifecycle) { - self.0 = (self.0 & !LIFECYCLE_MASK) | (val as usize) - } - - pub(crate) fn is_signaled(self) -> bool { - self.lifecycle() == Lifecycle::Signaled - } - - pub(crate) fn notify(&mut self) { - use self::Lifecycle::Signaled; - - if self.lifecycle() != Signaled { - self.set_lifecycle(Signaled) - } - } -} - -impl Default for State { - fn default() -> State { - // All workers will start pushed in the sleeping stack - State(PUSHED_MASK) - } -} - -impl From for State { - fn from(src: usize) -> Self { - State(src) - } -} - -impl From for usize { - fn from(src: State) -> Self { - src.0 - } -} - -impl fmt::Debug for State { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("worker::State") - .field("lifecycle", &self.lifecycle()) - .field("is_pushed", &self.is_pushed()) - .finish() - } -} - -// ===== impl Lifecycle ===== - -impl From for Lifecycle { - fn from(src: usize) -> Lifecycle { - use self::Lifecycle::*; - - debug_assert!( - src == Shutdown as usize - || src == Running as usize - || src == Sleeping as usize - || src == Notified as usize - || src == Signaled as usize - ); - - unsafe { ::std::mem::transmute(src) } - } -} - -impl From for usize { - fn from(src: Lifecycle) -> usize { - let v = src as usize; - debug_assert!(v & LIFECYCLE_MASK == v); - v - } -} - -#[cfg(test)] -mod test { - use super::Lifecycle::*; - use super::*; - - #[test] - fn lifecycle_encode() { - let lifecycles = &[Shutdown, Running, Sleeping, Notified, Signaled]; - - for &lifecycle in lifecycles { - let mut v: usize = lifecycle.into(); - v &= LIFECYCLE_MASK; - - assert_eq!(lifecycle, Lifecycle::from(v)); - } - } - - #[test] - fn lifecycle_ord() { - assert!(Running >= Shutdown); - assert!(Signaled >= Notified); - assert!(Signaled >= Sleeping); - } -} diff --git a/tokio-executor/src/tracing.rs b/tokio-executor/src/tracing.rs index 4d36049be97..0cf5832d6f1 100644 --- a/tokio-executor/src/tracing.rs +++ b/tokio-executor/src/tracing.rs @@ -1,3 +1,5 @@ +#![allow(unused_macros)] + //! This module provides a small facade that wraps the `tracing` APIs we use, so //! that when the `tracing` dependency is disabled, `tracing`'s macros expand to //! no-ops. diff --git a/tokio-executor/src/util/mod.rs b/tokio-executor/src/util/mod.rs new file mode 100644 index 00000000000..f16514439ce --- /dev/null +++ b/tokio-executor/src/util/mod.rs @@ -0,0 +1,5 @@ +mod pad; +mod rand; + +pub(crate) use self::pad::CachePadded; +pub(crate) use self::rand::FastRand; diff --git a/tokio-executor/src/util/pad.rs b/tokio-executor/src/util/pad.rs new file mode 100644 index 00000000000..bf0913ca853 --- /dev/null +++ b/tokio-executor/src/util/pad.rs @@ -0,0 +1,52 @@ +use core::fmt; +use core::ops::{Deref, DerefMut}; + +#[derive(Clone, Copy, Default, Hash, PartialEq, Eq)] +// Starting from Intel's Sandy Bridge, spatial prefetcher is now pulling pairs of 64-byte cache +// lines at a time, so we have to align to 128 bytes rather than 64. +// +// Sources: +// - https://www.intel.com/content/dam/www/public/us/en/documents/manuals/64-ia-32-architectures-optimization-manual.pdf +// - https://github.com/facebook/folly/blob/1b5288e6eea6df074758f877c849b6e73bbb9fbb/folly/lang/Align.h#L107 +#[cfg_attr(target_arch = "x86_64", repr(align(128)))] +#[cfg_attr(not(target_arch = "x86_64"), repr(align(64)))] +pub(crate) struct CachePadded { + value: T, +} + +unsafe impl Send for CachePadded {} +unsafe impl Sync for CachePadded {} + +impl CachePadded { + pub(crate) fn new(t: T) -> CachePadded { + CachePadded:: { value: t } + } +} + +impl Deref for CachePadded { + type Target = T; + + fn deref(&self) -> &T { + &self.value + } +} + +impl DerefMut for CachePadded { + fn deref_mut(&mut self) -> &mut T { + &mut self.value + } +} + +impl fmt::Debug for CachePadded { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("CachePadded") + .field("value", &self.value) + .finish() + } +} + +impl From for CachePadded { + fn from(t: T) -> Self { + CachePadded::new(t) + } +} diff --git a/tokio-executor/src/util/rand.rs b/tokio-executor/src/util/rand.rs new file mode 100644 index 00000000000..11617e26356 --- /dev/null +++ b/tokio-executor/src/util/rand.rs @@ -0,0 +1,52 @@ +use std::cell::Cell; + +/// Fast random number generate +/// +/// Implement xorshift64+: 2 32-bit xorshift sequences added together. +/// Shift triplet [17,7,16] was calculated as indicated in Marsaglia's +/// Xorshift paper: https://www.jstatsoft.org/article/view/v008i14/xorshift.pdf +/// This generator passes the SmallCrush suite, part of TestU01 framework: +/// http://simul.iro.umontreal.ca/testu01/tu01.html +#[derive(Debug)] +pub(crate) struct FastRand { + one: Cell, + two: Cell, +} + +impl FastRand { + /// Initialize a new, thread-local, fast random number generator. + pub(crate) fn new(seed: u64) -> FastRand { + let one = (seed >> 32) as u32; + let mut two = seed as u32; + + if two == 0 { + // This value cannot be zero + two = 1; + } + + FastRand { + one: Cell::new(one), + two: Cell::new(two), + } + } + + pub(crate) fn fastrand_n(&self, n: u32) -> u32 { + // This is similar to fastrand() % n, but faster. + // See https://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/ + let mul = (self.fastrand() as u64).wrapping_mul(n as u64); + (mul >> 32) as u32 + } + + fn fastrand(&self) -> u32 { + let mut s1 = self.one.get(); + let s0 = self.two.get(); + + s1 ^= s1 << 17; + s1 = s1 ^ s0 ^ s1 >> 7 ^ s0 >> 16; + + self.one.set(s0); + self.two.set(s1); + + s0.wrapping_add(s1) + } +} diff --git a/tokio-executor/tests/current_thread.rs b/tokio-executor/tests/current_thread.rs index a33edc82bb6..0f69730f3c3 100644 --- a/tokio-executor/tests/current_thread.rs +++ b/tokio-executor/tests/current_thread.rs @@ -1,4 +1,5 @@ #![warn(rust_2018_idioms)] +#![cfg(not(miri))] use tokio::sync::oneshot; use tokio_executor::current_thread::{self, block_on_all, CurrentThread, TaskExecutor}; @@ -141,6 +142,7 @@ mod from_block_on_future { mod outstanding_tasks_are_dropped_when_executor_is_dropped { use super::*; + #[allow(unreachable_code)] // TODO: remove this when https://github.com/rust-lang/rust/issues/64636 fixed. async fn never(_rc: Rc<()>) { loop { yield_once().await; @@ -241,6 +243,7 @@ mod run_in_future { fn tick_on_infini_future() { let num = Rc::new(Cell::new(0)); + #[allow(unreachable_code)] // TODO: remove this when https://github.com/rust-lang/rust/issues/64636 fixed. async fn infini(num: Rc>) { loop { num.set(1 + num.get()); @@ -259,6 +262,7 @@ fn tick_on_infini_future() { mod tasks_are_scheduled_fairly { use super::*; + #[allow(unreachable_code)] // TODO: remove this when https://github.com/rust-lang/rust/issues/64636 fixed. async fn spin(state: Rc>, idx: usize) { loop { // borrow_mut scope diff --git a/tokio-executor/tests/threadpool.rs b/tokio-executor/tests/thread_pool.rs similarity index 55% rename from tokio-executor/tests/threadpool.rs rename to tokio-executor/tests/thread_pool.rs index 171260b7086..adb7b2c4aae 100644 --- a/tokio-executor/tests/threadpool.rs +++ b/tokio-executor/tests/thread_pool.rs @@ -1,11 +1,9 @@ #![warn(rust_2018_idioms)] use tokio_executor::park::{Park, Unpark}; -use tokio_executor::threadpool; -use tokio_executor::threadpool::park::{DefaultPark, DefaultUnpark}; -use tokio_executor::threadpool::*; -use tokio_test::assert_pending; +use tokio_executor::thread_pool::*; +use futures_util::future::poll_fn; use std::cell::Cell; use std::future::Future; use std::pin::Pin; @@ -18,74 +16,7 @@ use std::time::Duration; thread_local!(static FOO: Cell = Cell::new(0)); #[test] -fn natural_shutdown_simple_futures() { - for _ in 0..1_000 { - let num_inc = Arc::new(AtomicUsize::new(0)); - let num_dec = Arc::new(AtomicUsize::new(0)); - - FOO.with(|f| { - f.set(1); - - let pool = { - let num_inc = num_inc.clone(); - let num_dec = num_dec.clone(); - - Builder::new() - .around_worker(move |w| { - num_inc.fetch_add(1, Relaxed); - w.run(); - num_dec.fetch_add(1, Relaxed); - }) - .build() - }; - - let tx = pool.sender().clone(); - - let a = { - let (t, rx) = mpsc::channel(); - tx.spawn(async move { - // Makes sure this runs on a worker thread - FOO.with(|f| assert_eq!(f.get(), 0)); - - t.send("one").unwrap(); - }) - .unwrap(); - rx - }; - - let b = { - let (t, rx) = mpsc::channel(); - tx.spawn(async move { - // Makes sure this runs on a worker thread - FOO.with(|f| assert_eq!(f.get(), 0)); - - t.send("two").unwrap(); - }) - .unwrap(); - rx - }; - - drop(tx); - - assert_eq!("one", a.recv().unwrap()); - assert_eq!("two", b.recv().unwrap()); - - // Wait for the pool to shutdown - pool.shutdown().wait(); - - // Assert that at least one thread started - let num_inc = num_inc.load(Relaxed); - assert!(num_inc > 0); - - // Assert that all threads shutdown - let num_dec = num_dec.load(Relaxed); - assert_eq!(num_inc, num_dec); - }); - } -} - -#[test] -fn force_shutdown_drops_futures() { +fn shutdown_drops_futures() { for _ in 0..1_000 { let num_inc = Arc::new(AtomicUsize::new(0)); let num_dec = Arc::new(AtomicUsize::new(0)); @@ -110,19 +41,20 @@ fn force_shutdown_drops_futures() { let a = num_inc.clone(); let b = num_dec.clone(); - let pool = Builder::new() - .around_worker(move |w| { + let mut pool = Builder::new() + .around_worker(move |_, work| { a.fetch_add(1, Relaxed); - w.run(); + work(); b.fetch_add(1, Relaxed); }) .build(); - let tx = pool.sender().clone(); - tx.spawn(Never(num_drop.clone())).unwrap(); + // let tx = pool.sender().clone(); + + pool.spawn(Never(num_drop.clone())); // Wait for the pool to shutdown - pool.shutdown_now().wait(); + pool.shutdown_now(); // Assert that only a single thread was spawned. let a = num_inc.load(Relaxed); @@ -140,6 +72,8 @@ fn force_shutdown_drops_futures() { #[test] fn drop_threadpool_drops_futures() { + const NUM_THREADS: usize = 10; + for _ in 0..1_000 { let num_inc = Arc::new(AtomicUsize::new(0)); let num_dec = Arc::new(AtomicUsize::new(0)); @@ -165,24 +99,22 @@ fn drop_threadpool_drops_futures() { let b = num_dec.clone(); let pool = Builder::new() - .max_blocking(2) - .pool_size(20) - .around_worker(move |w| { + .num_threads(NUM_THREADS) + .around_worker(move |_, work| { a.fetch_add(1, Relaxed); - w.run(); + work(); b.fetch_add(1, Relaxed); }) .build(); - let tx = pool.sender().clone(); - tx.spawn(Never(num_drop.clone())).unwrap(); + pool.spawn(Never(num_drop.clone())); // Wait for the pool to shutdown drop(pool); - // Assert that only a single thread was spawned. + // Assert that all the threads spawned let a = num_inc.load(Relaxed); - assert!(a >= 1); + assert_eq!(a, NUM_THREADS); // Assert that all threads shutdown let b = num_dec.load(Relaxed); @@ -196,26 +128,32 @@ fn drop_threadpool_drops_futures() { #[test] fn many_oneshot_futures() { + // used for notifying the main thread const NUM: usize = 10_000; for _ in 0..50 { - let pool = ThreadPool::new(); - let tx = pool.sender().clone(); + let (tx, rx) = mpsc::channel(); + + let mut pool = new_pool(); let cnt = Arc::new(AtomicUsize::new(0)); for _ in 0..NUM { let cnt = cnt.clone(); - tx.spawn(async move { - cnt.fetch_add(1, Relaxed); - }) - .unwrap(); + let tx = tx.clone(); + + pool.spawn(async move { + let num = cnt.fetch_add(1, Relaxed) + 1; + + if num == NUM { + tx.send(()).unwrap(); + } + }); } - // Wait for the pool to shutdown - pool.shutdown().wait(); + rx.recv().unwrap(); - let num = cnt.load(Relaxed); - assert_eq!(num, NUM); + // Wait for the pool to shutdown + pool.shutdown_now(); } } @@ -228,9 +166,7 @@ fn many_multishot_futures() { const TRACKS: usize = 50; for _ in 0..50 { - let pool = ThreadPool::new(); - let pool_tx = pool.sender().clone(); - + let pool = new_pool(); let mut start_txs = Vec::with_capacity(TRACKS); let mut final_rxs = Vec::with_capacity(TRACKS); @@ -241,13 +177,11 @@ fn many_multishot_futures() { let (mut next_tx, next_rx) = mpsc::channel(10); // Forward all the messages - pool_tx - .spawn(async move { - while let Some(v) = chain_rx.recv().await { - next_tx.send(v).await.unwrap(); - } - }) - .unwrap(); + pool.spawn(async move { + while let Some(v) = chain_rx.recv().await { + next_tx.send(v).await.unwrap(); + } + }); chain_rx = next_rx; } @@ -257,21 +191,19 @@ fn many_multishot_futures() { let mut cycle_tx = start_tx.clone(); let mut rem = CYCLES; - pool_tx - .spawn(async move { - for _ in 0..CYCLES { - let msg = chain_rx.recv().await.unwrap(); + pool.spawn(async move { + for _ in 0..CYCLES { + let msg = chain_rx.recv().await.unwrap(); - rem -= 1; + rem -= 1; - if rem == 0 { - final_tx.send(msg).await.unwrap(); - } else { - cycle_tx.send(msg).await.unwrap(); - } + if rem == 0 { + final_tx.send(msg).await.unwrap(); + } else { + cycle_tx.send(msg).await.unwrap(); } - }) - .unwrap(); + } + }); start_txs.push(start_tx); final_rxs.push(final_rx); @@ -290,80 +222,36 @@ fn many_multishot_futures() { } }); } - - // Shutdown the pool - pool.shutdown().wait(); } } #[test] fn global_executor_is_configured() { - let pool = ThreadPool::new(); - let tx = pool.sender().clone(); + let pool = new_pool(); let (signal_tx, signal_rx) = mpsc::channel(); - tx.spawn(async move { + pool.spawn(async move { tokio_executor::spawn(async move { signal_tx.send(()).unwrap(); }); - }) - .unwrap(); + }); signal_rx.recv().unwrap(); - - pool.shutdown().wait(); } #[test] fn new_threadpool_is_idle() { - let pool = ThreadPool::new(); - pool.shutdown_on_idle().wait(); -} - -#[test] -fn busy_threadpool_is_not_idle() { - use tokio_sync::oneshot; - - // let pool = ThreadPool::new(); - let pool = Builder::new().pool_size(4).max_blocking(2).build(); - let tx = pool.sender().clone(); - - let (term_tx, term_rx) = oneshot::channel(); - - tx.spawn(async move { - term_rx.await.unwrap(); - }) - .unwrap(); - - let mut idle = pool.shutdown_on_idle(); - - struct IdleFut<'a>(&'a mut Shutdown); - - impl Future for IdleFut<'_> { - type Output = (); - - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> { - assert_pending!(Pin::new(&mut self.as_mut().0).poll(cx)); - Poll::Ready(()) - } - } - - let idle_fut = IdleFut(&mut idle); - tokio_executor::enter().unwrap().block_on(idle_fut); - - term_tx.send(()).unwrap(); - - let idle_fut = IdleFut(&mut idle); - tokio_executor::enter().unwrap().block_on(idle_fut); + let mut pool = new_pool(); + pool.shutdown_now(); } #[test] fn panic_in_task() { - let pool = ThreadPool::new(); - let tx = pool.sender().clone(); + let pool = new_pool(); + let (tx, rx) = mpsc::channel(); - struct Boom; + struct Boom(mpsc::Sender<()>); impl Future for Boom { type Output = (); @@ -376,37 +264,20 @@ fn panic_in_task() { impl Drop for Boom { fn drop(&mut self) { assert!(::std::thread::panicking()); + self.0.send(()).unwrap(); } } - tx.spawn(Boom).unwrap(); - - pool.shutdown_on_idle().wait(); -} - -#[test] -fn count_panics() { - let counter = Arc::new(AtomicUsize::new(0)); - let counter_ = counter.clone(); - let pool = threadpool::Builder::new() - .panic_handler(move |_err| { - // We caught a panic. - counter_.fetch_add(1, Relaxed); - }) - .build(); - // Spawn a future that will panic. - pool.spawn(async { panic!() }); - pool.shutdown_on_idle().wait(); - let counter = counter.load(Relaxed); - assert_eq!(counter, 1); + pool.spawn(Boom(tx)); + rx.recv().unwrap(); } #[test] fn multi_threadpool() { use tokio_sync::oneshot; - let pool1 = ThreadPool::new(); - let pool2 = ThreadPool::new(); + let pool1 = new_pool(); + let pool2 = new_pool(); let (tx, rx) = oneshot::channel(); let (done_tx, done_rx) = mpsc::channel(); @@ -425,10 +296,11 @@ fn multi_threadpool() { #[test] fn eagerly_drops_futures() { - use std::sync::mpsc; + use std::sync::{mpsc, Mutex}; struct MyPark { - inner: DefaultPark, + rx: mpsc::Receiver<()>, + tx: Mutex>, #[allow(dead_code)] park_tx: mpsc::SyncSender<()>, unpark_tx: mpsc::SyncSender<()>, @@ -436,33 +308,35 @@ fn eagerly_drops_futures() { impl Park for MyPark { type Unpark = MyUnpark; - type Error = ::Error; + type Error = (); fn unpark(&self) -> Self::Unpark { MyUnpark { - inner: self.inner.unpark(), + tx: Mutex::new(self.tx.lock().unwrap().clone()), unpark_tx: self.unpark_tx.clone(), } } fn park(&mut self) -> Result<(), Self::Error> { - self.inner.park() + let _ = self.rx.recv(); + Ok(()) } fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { - self.inner.park_timeout(duration) + let _ = self.rx.recv_timeout(duration); + Ok(()) } } struct MyUnpark { - inner: DefaultUnpark, + tx: Mutex>, #[allow(dead_code)] unpark_tx: mpsc::SyncSender<()>, } impl Unpark for MyUnpark { fn unpark(&self) { - self.inner.unpark() + let _ = self.tx.lock().unwrap().send(()); } } @@ -471,13 +345,15 @@ fn eagerly_drops_futures() { let (park_tx, park_rx) = mpsc::sync_channel(0); let (unpark_tx, unpark_rx) = mpsc::sync_channel(0); - let pool = threadpool::Builder::new() - .custom_park(move |_| MyPark { - inner: DefaultPark::new(), + let pool = Builder::new().num_threads(4).build_with_park(move |_| { + let (tx, rx) = mpsc::channel(); + MyPark { + tx: Mutex::new(tx), + rx, park_tx: park_tx.clone(), unpark_tx: unpark_tx.clone(), - }) - .build(); + } + }); struct MyTask { task_tx: Option>, @@ -523,3 +399,80 @@ fn eagerly_drops_futures() { // Ensure `task` lives until after the test completes. drop(task); } + +#[test] +fn park_called_at_interval() { + struct MyPark { + park_light: Arc, + } + + struct MyUnpark {} + + impl Park for MyPark { + type Unpark = MyUnpark; + type Error = (); + + fn unpark(&self) -> Self::Unpark { + MyUnpark {} + } + + fn park(&mut self) -> Result<(), Self::Error> { + use std::thread; + use std::time::Duration; + + thread::sleep(Duration::from_millis(1)); + Ok(()) + } + + fn park_timeout(&mut self, duration: Duration) -> Result<(), Self::Error> { + if duration == Duration::from_millis(0) { + self.park_light.store(true, Relaxed); + Ok(()) + } else { + self.park() + } + } + } + + impl Unpark for MyUnpark { + fn unpark(&self) {} + } + + let park_light_1 = Arc::new(AtomicBool::new(false)); + let park_light_2 = park_light_1.clone(); + + let (done_tx, done_rx) = mpsc::channel(); + + // Use 1 thread to ensure the worker stays busy. + let pool = Builder::new().num_threads(1).build_with_park(move |idx| { + assert_eq!(idx, 0); + MyPark { + park_light: park_light_2.clone(), + } + }); + + let mut cnt = 0; + + pool.spawn(poll_fn(move |cx| { + let did_park_light = park_light_1.load(Relaxed); + + if did_park_light { + // There is a bit of a race where the worker can tick a few times + // before seeing the task + assert!(cnt > 50); + done_tx.send(()).unwrap(); + return Poll::Ready(()); + } + + cnt += 1; + + cx.waker().wake_by_ref(); + Poll::Pending + })); + + done_rx.recv().unwrap(); +} + +fn new_pool() -> ThreadPool { + Builder::new().num_threads(4).build() +} diff --git a/tokio-executor/tests/threadpool_blocking.rs b/tokio-executor/tests/threadpool_blocking.rs deleted file mode 100644 index 58c1ca3622e..00000000000 --- a/tokio-executor/tests/threadpool_blocking.rs +++ /dev/null @@ -1,412 +0,0 @@ -#![warn(rust_2018_idioms)] - -use tokio_executor::threadpool::*; -use tokio_test::*; - -use futures_core::ready; -use futures_util::future::poll_fn; -use rand::*; -use std::sync::atomic::Ordering::*; -use std::sync::atomic::*; -use std::sync::*; -use std::task::{Poll, Waker}; -use std::thread; -use std::time::Duration; - -#[test] -fn basic() { - let pool = Builder::new().pool_size(1).max_blocking(1).build(); - - let (tx1, rx1) = mpsc::channel(); - let (tx2, rx2) = mpsc::channel(); - - pool.spawn(async move { - let res = blocking(|| { - let v = rx1.recv().unwrap(); - tx2.send(v).unwrap(); - }); - - assert_ready!(res).unwrap(); - }); - - pool.spawn(async move { - tx1.send(()).unwrap(); - }); - - rx2.recv().unwrap(); -} - -#[test] -fn other_executors_can_run_inside_blocking() { - let pool = Builder::new().pool_size(1).max_blocking(1).build(); - - let (tx, rx) = mpsc::channel(); - - pool.spawn(async move { - let res = blocking(|| { - let _e = tokio_executor::enter().expect("nested blocking enter"); - tx.send(()).unwrap(); - }); - - assert_ready!(res).unwrap(); - }); - - rx.recv().unwrap(); -} - -#[test] -fn notify_task_on_capacity() { - const BLOCKING: usize = 10; - - let pool = Builder::new().pool_size(1).max_blocking(1).build(); - - let rem = Arc::new(AtomicUsize::new(BLOCKING)); - let (tx, rx) = mpsc::channel(); - - for _ in 0..BLOCKING { - let rem = rem.clone(); - let tx = tx.clone(); - - pool.spawn(async move { - poll_fn(move |_| { - blocking(|| { - thread::sleep(Duration::from_millis(100)); - let prev = rem.fetch_sub(1, SeqCst); - - if prev == 1 { - tx.send(()).unwrap(); - } - }) - .map_err(|e| panic!("blocking err {:?}", e)) - }) - .await - .unwrap() - }); - } - - rx.recv().unwrap(); - - assert_eq!(0, rem.load(SeqCst)); -} - -#[test] -fn capacity_is_use_it_or_lose_it() { - use tokio_sync::oneshot; - - // TODO: Run w/ bigger pool size - - let pool = Builder::new().pool_size(1).max_blocking(1).build(); - - let (tx1, rx1) = mpsc::channel(); - let (tx2, rx2) = oneshot::channel(); - let (tx3, rx3) = mpsc::channel(); - let (tx4, rx4) = mpsc::channel(); - - // First, fill the blocking capacity - pool.spawn(async move { - poll_fn(move |_| { - blocking(|| { - rx1.recv().unwrap(); - }) - }) - .await - .unwrap() - }); - - pool.spawn(async move { - let task: Waker = rx2.await.unwrap(); - - poll_fn(move |_| { - blocking(|| { - // Notify the other task - task.wake_by_ref(); - - // Block until woken - rx3.recv().unwrap(); - }) - }) - .await - .unwrap(); - }); - - // Spawn a future that will try to block, get notified, then not actually - // use the blocking - let mut i = 0; - let mut tx2 = Some(tx2); - - pool.spawn(async move { - poll_fn(move |cx| { - match i { - 0 => { - i = 1; - - let res = blocking(|| unreachable!()).map_err(|_| panic!()); - - assert_pending!(res); - - // Unblock the first blocker - tx1.send(()).unwrap(); - - return Poll::Pending; - } - 1 => { - i = 2; - - // Skip blocking, and notify the second task that it should - // start blocking - let me = cx.waker().clone(); - tx2.take().unwrap().send(me).unwrap(); - - return Poll::Pending; - } - 2 => { - let res = blocking(|| unreachable!()).map_err(|_| panic!()); - - assert_pending!(res); - - // Unblock the first blocker - tx3.send(()).unwrap(); - tx4.send(()).unwrap(); - Poll::Ready(()) - } - _ => unreachable!(), - } - }) - .await - }); - - rx4.recv().unwrap(); -} - -#[test] -fn blocking_thread_does_not_take_over_shutdown_worker_thread() { - let pool = Builder::new().pool_size(2).max_blocking(1).build(); - - let (enter_tx, enter_rx) = mpsc::channel(); - let (exit_tx, exit_rx) = mpsc::channel(); - let (try_tx, try_rx) = mpsc::channel(); - - let exited = Arc::new(AtomicBool::new(false)); - - { - let exited = exited.clone(); - - pool.spawn(async move { - poll_fn(move |_| { - blocking(|| { - enter_tx.send(()).unwrap(); - exit_rx.recv().unwrap(); - exited.store(true, SeqCst); - }) - }) - .await - .unwrap() - }); - } - - // Wait for the task to block - let _ = enter_rx.recv().unwrap(); - - // Spawn another task that attempts to block - pool.spawn(async move { - poll_fn(move |_| { - let res = blocking(|| {}); - - assert_eq!(res.is_ready(), exited.load(SeqCst)); - - try_tx.send(res.is_ready()).unwrap(); - - res.map(|_| ()) - }) - .await - }); - - // Wait for the second task to try to block (and not be ready). - let res = try_rx.recv().unwrap(); - assert!(!res); - - // Unblock the first task - exit_tx.send(()).unwrap(); - - // Wait for the second task to successfully block. - let res = try_rx.recv().unwrap(); - assert!(res); - - drop(pool); -} - -#[test] -fn blocking_one_time_gets_capacity_for_multiple_blocks() { - const ITER: usize = 1; - const BLOCKING: usize = 2; - - for _ in 0..ITER { - let pool = Builder::new().pool_size(4).max_blocking(1).build(); - - let rem = Arc::new(AtomicUsize::new(BLOCKING)); - let (tx, rx) = mpsc::channel(); - - for _ in 0..BLOCKING { - let rem = rem.clone(); - let tx = tx.clone(); - - pool.spawn(async move { - poll_fn(move |_| { - // First block - let res = blocking(|| { - thread::sleep(Duration::from_millis(100)); - }); - - ready!(res).unwrap(); - - let res = blocking(|| { - thread::sleep(Duration::from_millis(100)); - let prev = rem.fetch_sub(1, SeqCst); - - if prev == 1 { - tx.send(()).unwrap(); - } - }); - - assert!(res.is_ready()); - - Poll::Ready(()) - }) - .await - }); - } - - rx.recv().unwrap(); - - assert_eq!(0, rem.load(SeqCst)); - } -} - -#[test] -fn shutdown() { - const ITER: usize = 1_000; - const BLOCKING: usize = 10; - - for _ in 0..ITER { - let num_inc = Arc::new(AtomicUsize::new(0)); - let num_dec = Arc::new(AtomicUsize::new(0)); - let (tx, rx) = mpsc::channel(); - - let pool = { - let num_inc = num_inc.clone(); - let num_dec = num_dec.clone(); - - Builder::new() - .pool_size(1) - .max_blocking(BLOCKING) - .after_start(move || { - num_inc.fetch_add(1, SeqCst); - }) - .before_stop(move || { - num_dec.fetch_add(1, SeqCst); - }) - .build() - }; - - let barrier = Arc::new(Barrier::new(BLOCKING)); - - for _ in 0..BLOCKING { - let barrier = barrier.clone(); - let tx = tx.clone(); - - pool.spawn(async move { - let res = blocking(|| { - barrier.wait(); - Ok::<_, ()>(()) - }); - - tx.send(()).unwrap(); - - assert!(res.is_ready()); - }); - } - - for _ in 0..BLOCKING { - rx.recv().unwrap(); - } - - // Shutdown - drop(pool); - - assert_eq!(11, num_inc.load(SeqCst)); - assert_eq!(11, num_dec.load(SeqCst)); - } -} - -#[derive(Debug, Copy, Clone)] -enum Sleep { - Skip, - Yield, - Rand, - Fixed(Duration), -} - -#[test] -fn hammer() { - use self::Sleep::*; - - const ITER: usize = 5; - - let combos = [ - (2, 4, 1_000, Skip), - (2, 4, 1_000, Yield), - (2, 4, 100, Rand), - (2, 4, 100, Fixed(Duration::from_millis(3))), - (2, 4, 100, Fixed(Duration::from_millis(12))), - ]; - - for &(size, max_blocking, n, sleep) in &combos { - for _ in 0..ITER { - let pool = Builder::new() - .pool_size(size) - .max_blocking(max_blocking) - .build(); - - let cnt_task = Arc::new(AtomicUsize::new(0)); - let cnt_block = Arc::new(AtomicUsize::new(0)); - - for _ in 0..n { - let cnt_task = cnt_task.clone(); - let cnt_block = cnt_block.clone(); - - pool.spawn(async move { - cnt_task.fetch_add(1, SeqCst); - - poll_fn(move |_| { - blocking(|| { - match sleep { - Skip => {} - Yield => { - thread::yield_now(); - } - Rand => { - let ms = thread_rng().gen_range(3, 12); - thread::sleep(Duration::from_millis(ms)); - } - Fixed(dur) => { - thread::sleep(dur); - } - } - - cnt_block.fetch_add(1, SeqCst); - }) - .map_err(|_| panic!()) - }) - .await - .unwrap() - }); - } - - // Wait for the work to complete - pool.shutdown_on_idle().wait(); - - assert_eq!(n, cnt_task.load(SeqCst)); - assert_eq!(n, cnt_block.load(SeqCst)); - } - } -} diff --git a/tokio-executor/tests/threadpool_hammer.rs b/tokio-executor/tests/threadpool_hammer.rs deleted file mode 100644 index cec0466bc86..00000000000 --- a/tokio-executor/tests/threadpool_hammer.rs +++ /dev/null @@ -1,126 +0,0 @@ -#![warn(rust_2018_idioms)] - -use tokio_executor::threadpool::*; -use tokio_sync::{mpsc, oneshot}; - -use std::future::Future; -use std::pin::Pin; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::*; -use std::sync::Arc; -use std::task::{Context, Poll}; - -#[test] -fn hammer() { - const N: usize = 1000; - const ITER: usize = 20; - - struct Counted { - cnt: Arc, - inner: T, - } - - impl Future for Counted { - type Output = T::Output; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - unsafe { - let inner = &mut self.get_unchecked_mut().inner; - Pin::new_unchecked(inner).poll(cx) - } - } - } - - impl Drop for Counted { - fn drop(&mut self) { - self.cnt.fetch_add(1, Relaxed); - } - } - - for _ in 0..ITER { - let pool = Builder::new() - // .pool_size(30) - .build(); - - let cnt = Arc::new(AtomicUsize::new(0)); - - let (mut listen_tx, mut listen_rx) = - mpsc::unbounded_channel::>>(); - - pool.spawn({ - let c1 = cnt.clone(); - let c2 = cnt.clone(); - let pool = pool.sender().clone(); - let task = async move { - while let Some(tx) = listen_rx.recv().await { - let task = async { - let (tx2, rx2) = oneshot::channel(); - tx.send(tx2).unwrap(); - rx2.await.unwrap() - }; - - pool.spawn(Counted { - inner: task, - cnt: c1.clone(), - }) - .unwrap(); - } - }; - - /* - let task = listen_rx - .map_err(|e| panic!("accept error = {:?}", e)) - .for_each(move |tx| { - let task = future::lazy(|| { - let (tx2, rx2) = oneshot::channel(); - - tx.send(tx2).unwrap(); - rx2 - }) - .map_err(|e| panic!("e={:?}", e)) - .and_then(|_| Ok(())); - - pool.spawn(Counted { - inner: task, - cnt: c1.clone(), - }) - .unwrap(); - - Ok(()) - }); - */ - - Counted { - inner: task, - cnt: c2, - } - }); - - for _ in 0..N { - let cnt = cnt.clone(); - let (tx, rx) = oneshot::channel(); - listen_tx.try_send(tx).unwrap(); - - pool.spawn(async { - let task = async { - let tx = rx.await.unwrap(); - tx.send(()).unwrap(); - }; - - /* - let task = rx.map_err(|e| panic!("rx err={:?}", e)).and_then(|tx| { - tx.send(()).unwrap(); - Ok(()) - }); - */ - - Counted { inner: task, cnt }.await - }); - } - - drop(listen_tx); - - pool.shutdown_on_idle().wait(); - assert_eq!(N * 2 + 1, cnt.load(Relaxed)); - } -} diff --git a/tokio-fs/examples/std-echo.rs b/tokio-fs/examples/std-echo.rs deleted file mode 100644 index 311f1248a48..00000000000 --- a/tokio-fs/examples/std-echo.rs +++ /dev/null @@ -1,36 +0,0 @@ -//! Echo everything received on STDIN to STDOUT and STDERR. - -use futures_util::{FutureExt, SinkExt, StreamExt, TryFutureExt}; - -use tokio::codec::{FramedRead, FramedWrite, LinesCodec, LinesCodecError}; -use tokio::future::ready; -use tokio_executor::threadpool::Builder; -use tokio_fs::{stderr, stdin, stdout}; - -#[tokio::main] -async fn main() -> Result<(), Box> { - let pool = Builder::new().pool_size(1).build(); - - pool.spawn( - async { - let mut input = FramedRead::new(stdin(), LinesCodec::new()); - let mut output = FramedWrite::new(stdout(), LinesCodec::new()); - let mut error = FramedWrite::new(stderr(), LinesCodec::new()); - - while let Some(line) = input.next().await { - let line = line?; - // https://github.com/rust-lang/rust/pull/64856 - let s = format!("OUT: {}", line); - output.send(s).await?; - let s = format!("ERR: {}", line); - error.send(s).await?; - } - Ok(()) - } - .map_err(|e: LinesCodecError| panic!(e)) - .then(|_| ready(())), - ); - - pool.shutdown_on_idle().await; - Ok(()) -} diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index ce6b3b63347..6ed8002c714 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -53,7 +53,7 @@ rt-full = [ "sync", "timer", "tokio-executor/current-thread", - "tokio-executor/threadpool", + "tokio-executor/thread-pool", "tracing-core", ] signal = ["tokio-net/signal"] diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index aa183cbd39c..6e42a486786 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -70,9 +70,6 @@ //! } //! ``` //! -//! In this function, the `run` function blocks until the runtime becomes idle. -//! See [`shutdown_on_idle`][idle] for more shutdown details. -//! //! From within the context of the runtime, additional tasks are spawned using //! the [`tokio::spawn`] function. Futures spawned using this function will be //! executed on the same thread pool used by the [`Runtime`]. @@ -129,7 +126,6 @@ //! [`Reactor`]: ../reactor/struct.Reactor.html //! [`ThreadPool`]: https://docs.rs/tokio-executor/0.2.0-alpha.2/tokio_executor/threadpool/struct.ThreadPool.html //! [`run`]: fn.run.html -//! [idle]: struct.Runtime.html#method.shutdown_on_idle //! [`tokio::spawn`]: ../executor/fn.spawn.html //! [`Timer`]: https://docs.rs/tokio-timer/0.2/tokio_timer/timer/struct.Timer.html //! [`tokio::main`]: ../../tokio_macros/attr.main.html @@ -141,8 +137,9 @@ mod threadpool; #[cfg(feature = "rt-full")] pub use self::threadpool::{ Builder, + JoinHandle, Runtime, - TaskExecutor, + Spawner, }; // Internal export, don't use. diff --git a/tokio/src/runtime/threadpool/background.rs b/tokio/src/runtime/threadpool/background.rs deleted file mode 100644 index 3d884118dfa..00000000000 --- a/tokio/src/runtime/threadpool/background.rs +++ /dev/null @@ -1,61 +0,0 @@ -//! Temporary reactor + timer that runs on a background thread. This it to make -//! `block_on` work. - -use tokio_executor::current_thread::CurrentThread; -use tokio_net::driver::{self, Reactor}; -use tokio_sync::oneshot; -use tokio_timer::clock::Clock; -use tokio_timer::timer::{self, Timer}; - -use std::{io, thread}; - -#[derive(Debug)] -pub(crate) struct Background { - reactor_handle: driver::Handle, - timer_handle: timer::Handle, - shutdown_tx: Option>, - thread: Option>, -} - -pub(crate) fn spawn(clock: &Clock) -> io::Result { - let clock = clock.clone(); - - let reactor = Reactor::new()?; - let reactor_handle = reactor.handle(); - - let timer = Timer::new_with_now(reactor, clock); - let timer_handle = timer.handle(); - - let (shutdown_tx, shutdown_rx) = oneshot::channel(); - let shutdown_tx = Some(shutdown_tx); - - let thread = thread::spawn(move || { - let mut rt = CurrentThread::new_with_park(timer); - let _ = rt.block_on(shutdown_rx); - }); - let thread = Some(thread); - - Ok(Background { - reactor_handle, - timer_handle, - shutdown_tx, - thread, - }) -} - -impl Background { - pub(super) fn reactor(&self) -> &driver::Handle { - &self.reactor_handle - } - - pub(super) fn timer(&self) -> &timer::Handle { - &self.timer_handle - } -} - -impl Drop for Background { - fn drop(&mut self) { - let _ = self.shutdown_tx.take().unwrap().send(()); - let _ = self.thread.take().unwrap().join(); - } -} diff --git a/tokio/src/runtime/threadpool/builder.rs b/tokio/src/runtime/threadpool/builder.rs index 13173095612..a45abdbcecf 100644 --- a/tokio/src/runtime/threadpool/builder.rs +++ b/tokio/src/runtime/threadpool/builder.rs @@ -1,16 +1,13 @@ -use super::{background, Inner, Runtime}; +use super::{Inner, Runtime}; -use tokio_executor::threadpool; +use tokio_executor::thread_pool; use tokio_net::driver::{self, Reactor}; use tokio_timer::clock::{self, Clock}; use tokio_timer::timer::{self, Timer}; -use num_cpus; use tracing_core as trace; -use std::io; -use std::sync::Mutex; -use std::time::Duration; -use std::any::Any; +use std::{fmt, io}; +use std::sync::{Arc, Mutex}; /// Builds Tokio Runtime with custom configuration values. /// @@ -28,18 +25,14 @@ use std::any::Any; /// # Examples /// /// ``` -/// use std::time::Duration; -/// /// use tokio::runtime::Builder; /// use tokio_timer::clock::Clock; /// /// fn main() { /// // build Runtime /// let runtime = Builder::new() -/// .blocking_threads(4) /// .clock(Clock::system()) -/// .core_threads(4) -/// .keep_alive(Some(Duration::from_secs(60))) +/// .num_threads(4) /// .name_prefix("my-custom-name-") /// .stack_size(3 * 1024 * 1024) /// .build() @@ -48,13 +41,18 @@ use std::any::Any; /// // use runtime ... /// } /// ``` -#[derive(Debug)] pub struct Builder { /// Thread pool specific builder - threadpool_builder: threadpool::Builder, + thread_pool_builder: thread_pool::Builder, /// The number of worker threads - core_threads: usize, + num_threads: usize, + + /// To run after each worker thread starts + after_start: Option>, + + /// To run before each worker thread stops + before_stop: Option>, /// The clock to use clock: Clock, @@ -66,15 +64,18 @@ impl Builder { /// /// Configuration methods can be chained on the return value. pub fn new() -> Builder { - let core_threads = num_cpus::get().max(1); + let num_threads = num_cpus::get().max(1); - let mut threadpool_builder = threadpool::Builder::new(); - threadpool_builder.name_prefix("tokio-runtime-worker-"); - threadpool_builder.pool_size(core_threads); + let mut thread_pool_builder = thread_pool::Builder::new(); + thread_pool_builder + .name_prefix("tokio-runtime-worker-") + .num_threads(num_threads); Builder { - threadpool_builder, - core_threads, + thread_pool_builder, + num_threads, + after_start: None, + before_stop: None, clock: Clock::new(), } } @@ -85,35 +86,6 @@ impl Builder { self } - /// Sets a callback to handle panics in futures. - /// - /// The callback is triggered when a panic during a future bubbles up to - /// Tokio. By default Tokio catches these panics, and they will be ignored. - /// The parameter passed to this callback is the same error value returned - /// from `std::panic::catch_unwind()`. To abort the process on panics, use - /// `std::panic::resume_unwind()` in this callback as shown below. - /// - /// # Examples - /// - /// ``` - /// # use tokio::runtime; - /// - /// # pub fn main() { - /// let rt = runtime::Builder::new() - /// .panic_handler(|err| std::panic::resume_unwind(err)) - /// .build() - /// .unwrap(); - /// # } - /// ``` - pub fn panic_handler(&mut self, f: F) -> &mut Self - where - F: Fn(Box) + Send + Sync + 'static, - { - self.threadpool_builder.panic_handler(f); - self - } - - /// Set the maximum number of worker threads for the `Runtime`'s thread pool. /// /// This must be a number between 1 and 32,768 though it is advised to keep @@ -128,71 +100,14 @@ impl Builder { /// /// # pub fn main() { /// let rt = runtime::Builder::new() - /// .core_threads(4) + /// .num_threads(4) /// .build() /// .unwrap(); /// # } /// ``` - pub fn core_threads(&mut self, val: usize) -> &mut Self { - self.core_threads = val; - self.threadpool_builder.pool_size(val); - self - } - - /// Set the maximum number of concurrent blocking sections in the `Runtime`'s - /// thread pool. - /// - /// When the maximum concurrent `blocking` calls is reached, any further - /// calls to `blocking` will return `NotReady` and the task is notified once - /// previously in-flight calls to `blocking` return. - /// - /// This must be a number between 1 and 32,768 though it is advised to keep - /// this value on the smaller side. - /// - /// The default value is 100. - /// - /// # Examples - /// - /// ``` - /// # use tokio::runtime; - /// - /// # pub fn main() { - /// let rt = runtime::Builder::new() - /// .blocking_threads(200) - /// .build(); - /// # } - /// ``` - pub fn blocking_threads(&mut self, val: usize) -> &mut Self { - self.threadpool_builder.max_blocking(val); - self - } - - /// Set the worker thread keep alive duration for threads in the `Runtime`'s - /// thread pool. - /// - /// If set, a worker thread will wait for up to the specified duration for - /// work, at which point the thread will shutdown. When work becomes - /// available, a new thread will eventually be spawned to replace the one - /// that shut down. - /// - /// When the value is `None`, the thread will wait for work forever. - /// - /// The default value is `None`. - /// - /// # Examples - /// - /// ``` - /// # use tokio::runtime; - /// use std::time::Duration; - /// - /// # pub fn main() { - /// let rt = runtime::Builder::new() - /// .keep_alive(Some(Duration::from_secs(30))) - /// .build(); - /// # } - /// ``` - pub fn keep_alive(&mut self, val: Option) -> &mut Self { - self.threadpool_builder.keep_alive(val); + pub fn num_threads(&mut self, val: usize) -> &mut Self { + self.num_threads = val; + self.thread_pool_builder.num_threads(val); self } @@ -216,7 +131,7 @@ impl Builder { /// # } /// ``` pub fn name_prefix>(&mut self, val: S) -> &mut Self { - self.threadpool_builder.name_prefix(val); + self.thread_pool_builder.name_prefix(val); self } @@ -240,7 +155,7 @@ impl Builder { /// # } /// ``` pub fn stack_size(&mut self, val: usize) -> &mut Self { - self.threadpool_builder.stack_size(val); + self.thread_pool_builder.stack_size(val); self } @@ -265,7 +180,7 @@ impl Builder { pub fn after_start(&mut self, f: F) -> &mut Self where F: Fn() + Send + Sync + 'static { - self.threadpool_builder.after_start(f); + self.after_start = Some(Arc::new(f)); self } @@ -289,7 +204,7 @@ impl Builder { pub fn before_stop(&mut self, f: F) -> &mut Self where F: Fn() + Send + Sync + 'static { - self.threadpool_builder.before_stop(f); + self.before_stop = Some(Arc::new(f)); self } @@ -308,14 +223,11 @@ impl Builder { /// # } /// ``` pub fn build(&mut self) -> io::Result { - // TODO(stjepang): Once we remove the `threadpool_builder` method, remove this line too. - self.threadpool_builder.pool_size(self.core_threads); - let mut reactor_handles = Vec::new(); let mut timer_handles = Vec::new(); let mut timers = Vec::new(); - for _ in 0..self.core_threads { + for _ in 0..self.num_threads { // Create a new reactor. let reactor = Reactor::new()?; reactor_handles.push(reactor.handle()); @@ -336,36 +248,44 @@ impl Builder { let dispatch = trace::dispatcher::get_default(trace::Dispatch::clone); let trace = dispatch.clone(); - let background = background::spawn(&clock)?; + let around_reactor_handles = reactor_handles.clone(); + let around_timer_handles = timer_handles.clone(); - let pool = self - .threadpool_builder - .around_worker(move |w| { - let index = w.id().to_usize(); + let after_start = self.after_start.clone(); + let before_stop = self.before_stop.clone(); - let _reactor = driver::set_default(&reactor_handles[index]); + let pool = self + .thread_pool_builder + .around_worker(move |index, next| { + let _reactor = driver::set_default(&around_reactor_handles[index]); clock::with_default(&clock, || { - let _timer = timer::set_default(&timer_handles[index]); + let _timer = timer::set_default(&around_timer_handles[index]); trace::dispatcher::with_default(&dispatch, || { - w.run(); + if let Some(after_start) = after_start.as_ref() { + after_start(); + } + + next(); + + if let Some(before_stop) = before_stop.as_ref() { + before_stop(); + } }) }) }) - .custom_park(move |worker_id| { - let index = worker_id.to_usize(); - + .build_with_park(move |index| { timers[index] .lock() .unwrap() .take() .unwrap() - }) - .build(); + }); Ok(Runtime { inner: Some(Inner { pool, - background, + reactor_handles, + timer_handles, trace, }), }) @@ -377,3 +297,12 @@ impl Default for Builder { Self::new() } } + +impl fmt::Debug for Builder { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Builder") + .field("thread_pool_builder", &self.thread_pool_builder) + .field("after_start", &self.after_start.as_ref().map(|_| "...")) + .finish() + } +} diff --git a/tokio/src/runtime/threadpool/mod.rs b/tokio/src/runtime/threadpool/mod.rs index 47009cc4c9f..0717589c8fe 100644 --- a/tokio/src/runtime/threadpool/mod.rs +++ b/tokio/src/runtime/threadpool/mod.rs @@ -1,15 +1,15 @@ -mod background; mod builder; -mod task_executor; - #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::builder::Builder; + +mod spawner; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::spawner::Spawner; + #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::task_executor::TaskExecutor; -use background::Background; +pub use tokio_executor::thread_pool::JoinHandle; -use tokio_executor::enter; -use tokio_executor::threadpool::ThreadPool; +use tokio_executor::thread_pool::ThreadPool; use tokio_net::driver; use tokio_timer::timer; @@ -41,18 +41,14 @@ struct Inner { /// Task execution pool. pool: ThreadPool, + /// Reactor handles + reactor_handles: Vec, + + /// Timer handles + timer_handles: Vec, + /// Tracing dispatcher trace: trace::Dispatch, - - /// Maintains a reactor and timer that are always running on a background - /// thread. This is to support `runtime.block_on` w/o requiring the future - /// to be `Send`. - /// - /// A dedicated background thread is required as the threadpool threads - /// might not be running. However, this is a temporary work around. - /// - /// TODO: Delete this - background: Background, } // ===== impl Runtime ===== @@ -80,9 +76,6 @@ impl Runtime { /// .unwrap(); /// /// // Use the runtime... - /// - /// // Shutdown the runtime - /// rt.shutdown_now(); /// ``` /// /// [mod]: index.html @@ -90,27 +83,6 @@ impl Runtime { Builder::new().build() } - /// Return a handle to the runtime's executor. - /// - /// The returned handle can be used to spawn tasks that run on this runtime. - /// - /// # Examples - /// - /// ``` - /// use tokio::runtime::Runtime; - /// - /// let rt = Runtime::new() - /// .unwrap(); - /// - /// let executor_handle = rt.executor(); - /// - /// // use `executor_handle` - /// ``` - pub fn executor(&self) -> TaskExecutor { - let inner = self.inner().pool.sender().clone(); - TaskExecutor { inner } - } - /// Spawn a future onto the Tokio runtime. /// /// This spawns the given future onto the runtime's executor, usually a @@ -134,8 +106,6 @@ impl Runtime { /// rt.spawn(async { /// println!("now running on a worker thread"); /// }); - /// - /// rt.shutdown_on_idle(); /// } /// ``` /// @@ -166,32 +136,19 @@ impl Runtime { where F: Future, { - let mut entered = enter().expect("nested block_on"); - - let bg = &self.inner().background; let trace = &self.inner().trace; - tokio_executor::with_default(&mut self.inner().pool.sender(), || { - let _reactor = driver::set_default(bg.reactor()); - let _timer = timer::set_default(bg.timer()); - trace::dispatcher::with_default(trace, || { - entered.block_on(future) - }) + let _reactor = driver::set_default(&self.inner().reactor_handles[0]); + let _timer = timer::set_default(&self.inner().timer_handles[0]); + + trace::dispatcher::with_default(trace, || { + self.inner().pool.block_on(future) }) } - /// Signals the runtime to shutdown once it becomes idle. - /// - /// Blocks the current thread until the shutdown operation has completed. - /// This function can be used to perform a graceful shutdown of the runtime. + /// Return a handle to the runtime's spawner. /// - /// The runtime enters an idle state once **all** of the following occur. - /// - /// * The thread pool has no tasks to execute, i.e., all tasks that were - /// spawned have completed. - /// * The reactor is not managing any I/O resources. - /// - /// See [module level][mod] documentation for more details. + /// The returned handle can be used to spawn tasks that run on this runtime. /// /// # Examples /// @@ -201,18 +158,13 @@ impl Runtime { /// let rt = Runtime::new() /// .unwrap(); /// - /// // Use the runtime... + /// let spawner = rt.spawner(); /// - /// // Shutdown the runtime - /// rt.shutdown_on_idle(); + /// spawner.spawn(async { println!("hello"); }); /// ``` - /// - /// [mod]: index.html - pub fn shutdown_on_idle(mut self) { - let mut e = tokio_executor::enter().unwrap(); - - let inner = self.inner.take().unwrap(); - e.block_on(inner.pool.shutdown_on_idle()); + pub fn spawner(&self) -> Spawner { + let inner = self.inner().pool.spawner().clone(); + Spawner::new(inner) } /// Signals the runtime to shutdown immediately. @@ -248,11 +200,9 @@ impl Runtime { /// ``` /// /// [mod]: index.html + #[allow(warnings)] pub fn shutdown_now(mut self) { - let mut e = tokio_executor::enter().unwrap(); - let inner = self.inner.take().unwrap(); - - e.block_on(inner.pool.shutdown_now()); + self.inner.unwrap().pool.shutdown_now(); } fn inner(&self) -> &Inner { diff --git a/tokio/src/runtime/threadpool/task_executor.rs b/tokio/src/runtime/threadpool/spawner.rs similarity index 58% rename from tokio/src/runtime/threadpool/task_executor.rs rename to tokio/src/runtime/threadpool/spawner.rs index 3107ac11701..204bcc0e4ef 100644 --- a/tokio/src/runtime/threadpool/task_executor.rs +++ b/tokio/src/runtime/threadpool/spawner.rs @@ -1,21 +1,25 @@ -use tokio_executor::SpawnError; -use tokio_executor::threadpool::Sender; +use crate::runtime::JoinHandle; + +use tokio_executor::thread_pool; use std::future::Future; -use std::pin::Pin; -/// Executes futures on the runtime +/// Spawns futures on the runtime /// /// All futures spawned using this executor will be submitted to the associated /// Runtime's executor. This executor is usually a thread pool. /// /// For more details, see the [module level](index.html) documentation. #[derive(Debug, Clone)] -pub struct TaskExecutor { - pub(super) inner: Sender, +pub struct Spawner { + inner: thread_pool::Spawner, } -impl TaskExecutor { +impl Spawner { + pub(super) fn new(inner: thread_pool::Spawner) -> Spawner { + Spawner { inner } + } + /// Spawn a future onto the Tokio runtime. /// /// This spawns the given future onto the runtime's executor, usually a @@ -34,10 +38,10 @@ impl TaskExecutor { /// # fn dox() { /// // Create the runtime /// let rt = Runtime::new().unwrap(); - /// let executor = rt.executor(); + /// let spawner = rt.spawner(); /// /// // Spawn a future onto the runtime - /// executor.spawn(async { + /// spawner.spawn(async { /// println!("now running on a worker thread"); /// }); /// # } @@ -47,27 +51,10 @@ impl TaskExecutor { /// /// This function panics if the spawn fails. Failure occurs if the executor /// is currently at capacity and is unable to spawn a new future. - pub fn spawn(&self, future: F) - where F: Future + Send + 'static, + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, { - self.inner.spawn(future).unwrap(); - } -} - -impl tokio_executor::Executor for TaskExecutor { - fn spawn( - &mut self, - future: Pin + Send>>, - ) -> Result<(), SpawnError> { self.inner.spawn(future) } } - -impl tokio_executor::TypedExecutor for TaskExecutor -where - T: Future + Send + 'static, -{ - fn spawn(&mut self, future: T) -> Result<(), crate::executor::SpawnError> { - crate::executor::Executor::spawn(self, Box::pin(future)) - } -} diff --git a/tokio/tests/runtime_threaded.rs b/tokio/tests/runtime_threaded.rs index 36f652dda75..96f34b98813 100644 --- a/tokio/tests/runtime_threaded.rs +++ b/tokio/tests/runtime_threaded.rs @@ -47,14 +47,13 @@ fn spawn_shutdown() { rt.spawn(client_server(tx.clone())); - // Use executor trait - let f = Box::pin(client_server(tx)); - tokio_executor::Executor::spawn(&mut rt.executor(), f).unwrap(); + // Use spawner + rt.spawner().spawn(client_server(tx)); - rt.shutdown_on_idle(); + assert_ok!(rx.recv()); + assert_ok!(rx.recv()); - assert_ok!(rx.try_recv()); - assert_ok!(rx.try_recv()); + rt.shutdown_now(); assert_err!(rx.try_recv()); } @@ -68,8 +67,6 @@ fn block_on_timer() { }); assert_eq!(v, 42); - - rt.shutdown_on_idle(); } #[test] @@ -111,30 +108,35 @@ fn block_waits() { }); assert_ok!(b_rx.try_recv()); - - rt.shutdown_on_idle(); } #[test] fn spawn_many() { const ITER: usize = 200; - let cnt = Arc::new(Mutex::new(0)); let rt = Runtime::new().unwrap(); + let cnt = Arc::new(Mutex::new(0)); + let (tx, rx) = mpsc::channel(); + let tx = Arc::new(Mutex::new(tx)); + let c = cnt.clone(); rt.block_on(async move { for _ in 0..ITER { let c = c.clone(); + let tx = tx.clone(); tokio::spawn(async move { let mut x = c.lock().unwrap(); *x = 1 + *x; + + if *x == ITER { + tx.lock().unwrap().send(()).unwrap(); + } }); } }); - rt.shutdown_on_idle(); - + rx.recv().unwrap(); assert_eq!(ITER, *cnt.lock().unwrap()); } @@ -146,30 +148,12 @@ fn nested_enter() { rt.block_on(async { assert_err!(tokio_executor::enter()); - // Since this is testing panics in other threads, printing about panics - // is noisy and can give the impression that the test is ignoring panics. - // - // It *is* ignoring them, but on purpose. - let prev_hook = panic::take_hook(); - panic::set_hook(Box::new(|info| { - let s = info.to_string(); - if s.starts_with("panicked at 'nested ") - || s.starts_with("panicked at 'Multiple executors at once") - { - // expected, noop - } else { - println!("{}", s); - } - })); - let res = panic::catch_unwind(move || { let rt = Runtime::new().unwrap(); rt.block_on(async {}); }); assert_err!(res); - - panic::set_hook(prev_hook); }); } @@ -198,7 +182,7 @@ fn after_start_and_before_stop_is_called() { rt.block_on(client_server(tx)); - rt.shutdown_on_idle(); + drop(rt); assert_ok!(rx.try_recv()); diff --git a/tokio/tests/timer.rs b/tokio/tests/timer.rs index 138e8a2f55c..ca716ccbd83 100644 --- a/tokio/tests/timer.rs +++ b/tokio/tests/timer.rs @@ -23,8 +23,6 @@ fn timer_with_threaded_runtime() { tx.send(()).unwrap(); }); - rt.shutdown_on_idle(); - rx.recv().unwrap(); }