This commit is contained in:
Stjepan Glavina 2020-06-14 14:49:27 +02:00
parent 0408c19531
commit 3e78e59f52
5 changed files with 278 additions and 230 deletions

View File

@ -5,6 +5,9 @@ use std::sync::atomic::Ordering::SeqCst;
use std::sync::{Arc, Condvar, Mutex};
use std::time::{Duration, Instant};
use once_cell::sync::Lazy;
use crate::io_event::IoEvent;
use crate::reactor::{Reactor, ReactorLock};
pub(crate) struct IoParker {
@ -51,6 +54,12 @@ impl IoParker {
}
}
impl Drop for IoParker {
fn drop(&mut self) {
// TODO: wake up another active IoParker
}
}
impl fmt::Debug for IoParker {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.pad("IoParker { .. }")
@ -86,7 +95,10 @@ impl Clone for IoUnparker {
const EMPTY: usize = 0;
const PARKED: usize = 1;
const NOTIFIED: usize = 2;
const POLLING: usize = 2;
const NOTIFIED: usize = 3;
static EVENT: Lazy<IoEvent> = Lazy::new(|| IoEvent::new().unwrap());
struct Inner {
state: AtomicUsize,
@ -96,18 +108,25 @@ struct Inner {
impl Inner {
fn park(&self, timeout: Option<Duration>) -> bool {
let mut reactor_lock = Reactor::get().try_lock();
// If we were previously notified then we consume this notification and return quickly.
if self
.state
.compare_exchange(NOTIFIED, EMPTY, SeqCst, SeqCst)
.is_ok()
{
// Process available I/O events.
if let Some(mut reactor_lock) = Reactor::get().try_lock() {
reactor_lock.poll().expect("failure while polling I/O");
}
return true;
}
// If the timeout is zero, then there is no need to actually block.
if let Some(dur) = timeout {
if dur == Duration::from_millis(0) {
// Process available I/O events.
if let Some(mut reactor_lock) = Reactor::get().try_lock() {
reactor_lock.poll().expect("failure while polling I/O");
}
@ -118,7 +137,12 @@ impl Inner {
// Otherwise we need to coordinate going to sleep.
let mut m = self.lock.lock().unwrap();
match self.state.compare_exchange(EMPTY, PARKED, SeqCst, SeqCst) {
let state = match reactor_lock {
None => PARKED,
Some(_) => POLLING,
};
match self.state.compare_exchange(EMPTY, state, SeqCst, SeqCst) {
Ok(_) => {}
// Consume this notification to avoid spurious wakeups in the next park.
Err(NOTIFIED) => {
@ -138,7 +162,18 @@ impl Inner {
None => {
loop {
// Block the current thread on the conditional variable.
m = self.cvar.wait(m).unwrap();
match &mut reactor_lock {
None => m = self.cvar.wait(m).unwrap(),
Some(reactor_lock) => {
drop(m);
if EVENT.clear() {
reactor_lock.poll().expect("TODO");
} else {
reactor_lock.wait().expect("TODO");
}
m = self.lock.lock().unwrap();
}
}
match self.state.compare_exchange(NOTIFIED, EMPTY, SeqCst, SeqCst) {
Ok(_) => return true, // got a notification
@ -150,11 +185,22 @@ impl Inner {
// 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.
let (_m, _result) = self.cvar.wait_timeout(m, timeout).unwrap();
let _m = match &mut reactor_lock {
None => self.cvar.wait_timeout(m, timeout).unwrap().0,
Some(reactor_lock) => {
drop(m);
if EVENT.clear() {
reactor_lock.poll().expect("TODO");
} else {
reactor_lock.wait().expect("TODO"); // TODO: use actual timeout
}
self.lock.lock().unwrap()
}
};
match self.state.swap(EMPTY, SeqCst) {
NOTIFIED => true, // got a notification
PARKED => false, // no notification
NOTIFIED => true, // got a notification
PARKED | POLLING => false, // no notification
n => panic!("inconsistent park_timeout state: {}", n),
}
}
@ -166,12 +212,11 @@ impl Inner {
// 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) {
let state = match self.state.swap(NOTIFIED, SeqCst) {
EMPTY => return, // no one was waiting
NOTIFIED => return, // already unparked
PARKED => {} // gotta go wake someone up
_ => panic!("inconsistent state in unpark"),
}
state => state, // gotta go wake someone up
};
// 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`.
@ -182,6 +227,11 @@ impl Inner {
// 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().unwrap());
self.cvar.notify_one();
if state == PARKED {
self.cvar.notify_one();
} else {
EVENT.notify();
}
}
}

View File

@ -5,16 +5,18 @@
use std::future::Future;
use std::task::{Context, Poll};
use std::thread;
use std::time::Duration;
use futures_util::future::{self, Either};
use crate::block_on;
use crate::context;
use crate::io_event::IoEvent;
use crate::io_parking::{IoParker, IoUnparker};
use crate::reactor::{Reactor, ReactorLock};
use crate::thread_local::ThreadLocalExecutor;
use crate::thread_local::LocalExecutor;
use crate::throttle;
use crate::work_stealing::{WorkStealingExecutor, Worker};
use crate::work_stealing::{Executor, Worker};
/// Runs executors and polls the reactor.
///
@ -95,101 +97,53 @@ use crate::work_stealing::{WorkStealingExecutor, Worker};
/// }
/// ```
pub fn run<T>(future: impl Future<Output = T>) -> T {
let event = IoEvent::new().expect("cannot create an `IoEvent`");
let ws_executor = WorkStealingExecutor::get();
let worker = ws_executor.worker();
let reactor = Reactor::get();
let parker = IoParker::new();
let unparker = parker.unparker();
let worker = Executor::get().worker(move || unparker.unpark());
let unparker = parker.unparker();
let local = LocalExecutor::new(move || unparker.unpark());
// Create a waker that triggers an I/O event in the thread-local scheduler.
let ev = event.clone();
let waker = async_task::waker_fn(move || ev.notify());
let unparker = parker.unparker();
let waker = async_task::waker_fn(move || unparker.unpark());
let cx = &mut Context::from_waker(&waker);
futures_util::pin_mut!(future);
// Set up tokio (if enabled) and the thread-locals before execution begins.
let enter = context::enter;
let enter = |f| worker.enter(|| enter(f));
// Set up tokio if enabled.
// let mut enter = context::enter;
enter(|| {
// We run four components at the same time, treating them all fairly and making sure none
// of them get starved:
//
// 1. `future` - the main future.
// 2. `local - the thread-local executor.
// 3. `worker` - the work-stealing executor.
// 4. `reactor` - the reactor.
//
// When all four components are out of work, we block the current thread on
// epoll/kevent/wepoll. If new work comes in that isn't naturally triggered by an I/O event
// registered with `Async` handles, we use `IoEvent`s to simulate an I/O event that will
// unblock the thread:
//
// - When the main future is woken, `local.event()` is triggered.
// - When thread-local executor gets new work, `local.event()` is triggered.
// - When work-stealing executor gets new work, `ws_executor.event()` is triggered.
// - When a new earliest timer is registered, `reactor.event()` is triggered.
//
// This way we make sure that if any changes happen that might give us new work will
// unblock epoll/kevent/wepoll and let us continue the loop.
loop {
if let Poll::Ready(val) = throttle::setup(|| future.as_mut().poll(cx)) {
return val;
}
let more = worker.execute();
if more {
if let Some(mut reactor_lock) = reactor.try_lock() {
reactor_lock.poll().expect("failure while polling I/O");
}
continue;
}
// 4. Poll the reactor.
if let Some(reactor_lock) = reactor.try_lock() {
react(&worker, reactor_lock, &event);
continue;
}
// Prepare for blocking until the reactor is locked or `local.event()` is triggered.
//
// Note that there is no need to wait for `ws_executor.event()`. If we lock the reactor
// immediately, we'll check for the I/O event right after that anyway.
//
// If some other worker is holding the reactor locked, it will unlock it as soon as the
// I/O event is triggered. Then, another worker will be allowed to lock the reactor,
// and will unlock it if there is more work to do because every worker triggers the I/O
// event whenever it finds a runnable task.
let lock = reactor.lock();
let notified = event.notified();
futures_util::pin_mut!(lock);
futures_util::pin_mut!(notified);
// Block until either the reactor is locked or `local.event()` is triggered.
if let Either::Left((reactor_lock, _)) = block_on(future::select(lock, notified)) {
react(&worker, reactor_lock, &event);
} else {
// Clear `local.event()` because it was triggered.
event.clear();
}
loop {
// Poll the main future.
if let Poll::Ready(val) = throttle::setup(|| future.as_mut().poll(cx)) {
return val;
}
})
}
/// Polls or waits on the locked reactor.
///
/// If any of the I/O events are ready or there are more tasks to run, the reactor is polled.
/// Otherwise, the current thread waits on it until a timer fires or an I/O event occurs.
///
/// I/O events are cleared at the end of this function.
fn react(worker: &Worker<'_>, mut reactor_lock: ReactorLock<'_>, event: &IoEvent) {
if event.clear() {
// If there might be more tasks to run, just poll without blocking.
reactor_lock.poll().expect("failure while polling I/O");
} else {
// Otherwise, block until the first I/O event or a timer.
reactor_lock.wait().expect("failure while waiting on I/O");
let mut more_worker = true;
let mut more_local = true;
// enter(|| {
for _ in 0..200 {
if !worker.tick() {
more_worker = false;
break;
}
}
event.clear();
for _ in 0..200 {
if !local.tick() {
more_local = false;
break;
}
}
// });
if more_local || more_worker {
// Process ready I/O events without blocking.
parker.park_timeout(Duration::from_secs(0));
} else {
// Wait until unparked.
parker.park();
}
}
}

View File

@ -8,8 +8,8 @@ use std::pin::Pin;
use std::task::{Context, Poll};
use crate::blocking::BlockingExecutor;
use crate::thread_local::ThreadLocalExecutor;
use crate::work_stealing::WorkStealingExecutor;
use crate::thread_local::LocalExecutor;
use crate::work_stealing::Executor;
/// A runnable future, ready for execution.
///
@ -77,7 +77,7 @@ impl<T: 'static> Task<T> {
///
/// [`run()`]: `crate::run()`
pub fn local(future: impl Future<Output = T> + 'static) -> Task<T> {
ThreadLocalExecutor::spawn(future)
LocalExecutor::spawn(future)
}
}
@ -99,7 +99,7 @@ impl<T: Send + 'static> Task<T> {
///
/// [`run()`]: `crate::run()`
pub fn spawn(future: impl Future<Output = T> + Send + 'static) -> Task<T> {
WorkStealingExecutor::get().spawn(future)
Executor::get().spawn(future)
}
/// Spawns a future onto the blocking executor.

View File

@ -4,7 +4,7 @@
//! [`run()`][`crate::run()`] creates a thread-local executor. Tasks cannot be spawned onto a
//! thread-local executor if it is not running.
use std::cell::RefCell;
use std::cell::{Cell, RefCell};
use std::collections::VecDeque;
use std::future::Future;
use std::sync::Arc;
@ -13,55 +13,54 @@ use std::thread::{self, ThreadId};
use concurrent_queue::ConcurrentQueue;
use scoped_tls_hkt::scoped_thread_local;
use crate::io_event::IoEvent;
use crate::task::{Runnable, Task};
use crate::throttle;
scoped_thread_local! {
/// The thread-local executor.
///
/// This thread-local is only set while inside [`ThreadLocalExecutor::enter()`].
static EXECUTOR: ThreadLocalExecutor
/// This thread-local is only set while inside [`LocalExecutor::enter()`].
static EXECUTOR: LocalExecutor
}
/// An executor for thread-local tasks.
///
/// Thread-local tasks are spawned by calling [`Task::local()`] and their futures do not have to
/// implement [`Send`]. They can only be run by the same thread that created them.
pub(crate) struct ThreadLocalExecutor {
pub(crate) struct LocalExecutor {
/// The main task queue.
queue: RefCell<VecDeque<Runnable>>,
/// When another thread wakes a task belonging to this executor, it goes into this queue.
injector: Arc<ConcurrentQueue<Runnable>>,
/// An I/O event that is triggered when another thread wakes a task belonging to this executor.
event: IoEvent,
callback: Arc<dyn Fn() + Send + Sync>,
sleeping: Cell<bool>,
ticks: Cell<usize>,
}
impl ThreadLocalExecutor {
impl LocalExecutor {
/// Creates a new thread-local executor.
pub fn new() -> ThreadLocalExecutor {
ThreadLocalExecutor {
pub fn new(notify: impl Fn() + Send + Sync + 'static) -> LocalExecutor {
LocalExecutor {
queue: RefCell::new(VecDeque::new()),
injector: Arc::new(ConcurrentQueue::unbounded()),
event: IoEvent::new().expect("cannot create an `IoEvent`"),
callback: Arc::new(notify),
sleeping: Cell::new(false),
ticks: Cell::new(0),
}
}
/// Enters the context of this executor.
pub fn enter<T>(&self, f: impl FnOnce() -> T) -> T {
// TODO(stjepang): Allow recursive executors.
if EXECUTOR.is_set() {
panic!("cannot run an executor inside another executor");
}
EXECUTOR.set(self, f)
}
/// Returns the event indicating there is a scheduled task.
pub fn event(&self) -> &IoEvent {
&self.event
}
/// Spawns a future onto this executor.
///
/// Returns a [`Task`] handle for the spawned task.
@ -74,7 +73,7 @@ impl ThreadLocalExecutor {
// Why weak reference here? Injector may hold the task while the task's waker holds a
// reference to the injector. So this reference must be weak to break the cycle.
let injector = Arc::downgrade(&ex.injector);
let event = ex.event.clone();
let callback = ex.callback.clone();
let id = thread_id();
// The function that schedules a runnable task when it gets woken up.
@ -90,7 +89,7 @@ impl ThreadLocalExecutor {
// Trigger an I/O event to let the original thread know that a task has been
// scheduled. If that thread is inside epoll/kqueue/wepoll, an I/O event will wake
// it up.
event.notify();
callback();
};
// Create a task, push it into the queue by scheduling it, and return its `Task` handle.
@ -100,30 +99,23 @@ impl ThreadLocalExecutor {
})
}
/// Executes a batch of tasks and returns `true` if there may be more tasks to run.
pub fn execute(&self) -> bool {
// Execute 4 series of 50 tasks.
for _ in 0..4 {
for _ in 0..50 {
// Find the next task to run.
match self.search() {
None => {
// There are no more tasks to run.
return false;
}
Some(r) => {
// Run the task.
throttle::setup(|| r.run());
}
}
pub fn tick(&self) -> bool {
match self.search() {
None => {
self.ticks.set(0);
false
}
Some(r) => {
self.ticks.set(self.ticks.get() + 1);
if self.ticks.get() == 50 {
self.ticks.set(0);
self.fetch();
}
// Drain the injector queue occasionally for fair scheduling.
self.fetch();
self.enter(|| r.run());
true
}
}
// There are likely more tasks to run.
true
}
/// Finds the next task to run.

View File

@ -1,11 +1,11 @@
//! The work-stealing executor.
//!
//! Tasks created by [`Task::spawn()`] go into this executor. Every thread calling [`run()`]
//! initializes a [`Worker`] that participates in work stealing, which is allowed to run any task in
//! this executor or in other workers. Since tasks can be stolen by any worker and thus move from
//! thread to thread, their futures must implement [`Send`].
//! initializes a [`Worker`] that participates in work stealing, which is allowed to run any task
//! in this executor or in other workers. Since tasks can be stolen by any worker and thus move
//! from thread to thread, their futures must implement [`Send`].
//!
//! There is only one global instance of this type, accessible by [`WorkStealingExecutor::get()`].
//! There is only one global instance of this type, accessible by [`Executor::get()`].
//!
//! [Work stealing] is a strategy that reduces contention in multi-threaded environments. If all
//! invocations of [`run()`] used the same global task queue all the time, they would contend on
@ -21,7 +21,7 @@
use std::cell::Cell;
use std::future::Future;
use std::panic;
use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, Mutex, RwLock};
use concurrent_queue::ConcurrentQueue;
@ -29,9 +29,7 @@ use once_cell::sync::Lazy;
use scoped_tls_hkt::scoped_thread_local;
use slab::Slab;
use crate::io_event::IoEvent;
use crate::task::{Runnable, Task};
use crate::throttle;
scoped_thread_local! {
/// The current thread's worker.
@ -44,7 +42,7 @@ scoped_thread_local! {
}
/// The global work-stealing executor.
pub(crate) struct WorkStealingExecutor {
pub(crate) struct Executor {
/// When a thread that is not inside [`run()`][`crate::run()`] spawns or wakes a task, it goes
/// into this queue.
injector: ConcurrentQueue<Runnable>,
@ -52,35 +50,44 @@ pub(crate) struct WorkStealingExecutor {
/// Registered handles for stealing tasks from workers.
stealers: RwLock<Slab<Arc<ConcurrentQueue<Runnable>>>>,
/// An I/O event that is triggered whenever there might be available tasks to run.
event: IoEvent,
notified: AtomicBool,
sleeping: Mutex<Vec<Arc<dyn Fn() + Send + Sync>>>,
sleepers: Mutex<Sleepers>,
}
impl WorkStealingExecutor {
struct Sleepers {
count: usize,
items: Vec<Arc<dyn Fn() + Send + Sync>>,
}
impl Executor {
/// Returns a reference to the global work-stealing executor.
pub fn get() -> &'static WorkStealingExecutor {
static EXECUTOR: Lazy<WorkStealingExecutor> = Lazy::new(|| WorkStealingExecutor {
pub fn get() -> &'static Executor {
static EXECUTOR: Lazy<Executor> = Lazy::new(|| Executor {
injector: ConcurrentQueue::unbounded(),
stealers: RwLock::new(Slab::new()),
event: IoEvent::new().expect("cannot create an `IoEvent`"),
notified: AtomicBool::new(false),
sleeping: Mutex::new(Vec::new()),
notified: AtomicBool::new(true),
sleepers: Mutex::new(Sleepers {
count: 0,
items: Vec::new(),
}),
});
&EXECUTOR
}
fn notify(&self) {
// if !self.notified.compare_and_swap(false, true, Ordering::SeqCst) {
// let mut sleeping = self.sleeping.lock().unwrap();
// if let Some(callback) = sleeping.pop() {
// callback();
// }
// }
self.event.notify();
if !self
.notified
.compare_and_swap(false, true, Ordering::SeqCst)
{
let mut sleepers = self.sleepers.lock().unwrap();
if sleepers.items.len() == sleepers.count {
if let Some(callback) = sleepers.items.pop() {
let callback = callback.clone();
drop(sleepers);
callback();
}
}
}
}
/// Spawns a future onto this executor.
@ -99,7 +106,6 @@ impl WorkStealingExecutor {
// If scheduling from a non-worker thread, push into the injector queue.
self.injector.push(runnable).unwrap();
// Notify workers that there is a task in the injector queue.
self.notify();
}
};
@ -113,7 +119,7 @@ impl WorkStealingExecutor {
/// Registers a new worker.
///
/// The worker will automatically deregister itself when dropped.
pub fn worker(&self) -> Worker<'_> {
pub fn worker(&self, notify: impl Fn() + Send + Sync + 'static) -> Worker<'_> {
let mut stealers = self.stealers.write().unwrap();
let vacant = stealers.vacant_entry();
@ -123,8 +129,12 @@ impl WorkStealingExecutor {
slot: Cell::new(None),
queue: Arc::new(ConcurrentQueue::bounded(512)),
executor: self,
callback: Arc::new(notify),
sleeping: Cell::new(false),
ticks: Cell::new(0),
};
vacant.insert(worker.queue.clone());
drop(stealers);
worker
}
@ -148,81 +158,121 @@ pub(crate) struct Worker<'a> {
queue: Arc<ConcurrentQueue<Runnable>>,
/// The parent work-stealing executor.
executor: &'a WorkStealingExecutor,
executor: &'a Executor,
callback: Arc<dyn Fn() + Send + Sync>,
sleeping: Cell<bool>,
ticks: Cell<usize>,
}
impl Worker<'_> {
/// Enters the context of this executor.
pub fn enter<T>(&self, f: impl FnOnce() -> T) -> T {
fn enter<T>(&self, f: impl FnOnce() -> T) -> T {
// TODO(stjepang): Allow recursive executors.
if WORKER.is_set() {
panic!("cannot run an executor inside another executor");
}
WORKER.set(self, f)
}
/// Executes a batch of tasks and returns `true` if there may be more tasks to run.
pub fn execute(&self) -> bool {
// Execute 4 series of 50 tasks.
for _ in 0..4 {
for _ in 0..50 {
// Find the next task to run.
match self.search() {
None => {
// There are no more tasks to run.
return false;
}
Some(r) => {
// Notify other workers that there may be stealable tasks.
//
// Instead of notifying when we find a task, we could notify when we push a
// task into the local queue - either strategy works.
//
// Notifying when we find a task is somewhat simpler because then we don't
// need to worry about `search()` re-shuffling tasks between queues, which
// races with other workers searching for tasks. Other workers might not
// find a task while there is one! Notifying here avoids this problem.
self.executor.notify();
fn sleep(&self) -> bool {
let sleeping = self.sleeping.get();
self.sleeping.set(true);
// Run the task.
if throttle::setup(|| r.run()) {
// The task was woken while it was running, which means it got
// scheduled the moment running completed. Therefore, it is now inside
// the slot and would be the next task to run.
//
// Instead of re-running the task in the next iteration, let's flush
// the slot in order to give other tasks a chance to run.
//
// This is a necessary step to ensure task yielding works as expected.
// If a task wakes itself and returns `Poll::Pending`, we don't want it
// to run immediately after that because that'd defeat the whole
// purpose of yielding.
self.flush_slot();
}
}
}
}
// Flush the slot occasionally for fair scheduling.
//
// It is possible for two tasks to be exchanging messages between each other forever so
// that every time one of them runs, it wakes the other one and puts it into the slot.
// Flushing the slot prevena them from hogging the executor.
self.flush_slot();
// Steal some tasks from the injector queue.
//
// If the executor always has tasks in the local queue, it might never get to run tasks
// in the injector queue. To prevent them from starvation, we must move them into the
// local queue every now and then.
if let Some(r) = self.steal_global() {
self.push(r);
let mut sleepers = self.executor.sleepers.lock().unwrap();
if sleeping {
if sleepers
.items
.iter()
.all(|i| !Arc::ptr_eq(i, &self.callback))
{
sleepers.items.push(self.callback.clone());
}
} else {
sleepers.count += 1;
sleepers.items.push(self.callback.clone());
}
if sleepers.count == 0 || sleepers.count > sleepers.items.len() {
self.executor.notified.swap(true, Ordering::SeqCst);
} else {
self.executor.notified.swap(false, Ordering::SeqCst);
}
!sleeping
}
fn wake(&self) -> bool {
if !self.sleeping.get() {
return false;
}
self.sleeping.set(false);
let mut sleepers = self.executor.sleepers.lock().unwrap();
sleepers.count -= 1;
sleepers.items.retain(|i| !Arc::ptr_eq(i, &self.callback)); // TODO: optimize
if sleepers.count == 0 || sleepers.count > sleepers.items.len() {
self.executor.notified.swap(true, Ordering::SeqCst);
} else {
self.executor.notified.swap(false, Ordering::SeqCst);
}
// There are likely more tasks to run.
true
}
/// Executes a batch of tasks and returns `true` if there may be more tasks to run.
pub fn tick(&self) -> bool {
loop {
match self.search() {
None => {
if !self.sleep() {
// self.ticks.set(self.ticks.get().wrapping_add(1));
return false;
}
}
Some(r) => {
if !self.wake() {
self.executor.notify();
}
// Run the task.
if self.enter(|| r.run()) {
// The task was woken while it was running, which means it got
// scheduled the moment running completed. Therefore, it is now inside
// the slot and would be the next task to run.
//
// Instead of re-running the task in the next iteration, let's flush
// the slot in order to give other tasks a chance to run.
//
// This is a necessary step to ensure task yielding works as expected.
// If a task wakes itself and returns `Poll::Pending`, we don't want it
// to run immediately after that because that'd defeat the whole
// purpose of yielding.
self.flush_slot();
}
let ticks = self.ticks.get();
self.ticks.set(ticks.wrapping_add(1));
if ticks % 16 == 0 {
self.flush_slot();
}
if ticks % 64 == 0 {
if let Some(r) = self.steal_global() {
self.push(r);
}
}
return true;
}
}
}
}
/// Pushes a task into this worker.
fn push(&self, runnable: Runnable) {
// Put the task into the slot.
@ -335,6 +385,8 @@ impl Drop for Worker<'_> {
r.schedule();
}
self.wake();
// This task will not search for tasks anymore and therefore won't notify other workers if
// new tasks are found. Notify another worker to start searching right away.
self.executor.notify();