From 36d24cd0e1f5049be4caa2461a43c704eb61975e Mon Sep 17 00:00:00 2001 From: Stjepan Glavina Date: Mon, 16 Dec 2019 13:57:27 +0100 Subject: [PATCH 01/60] New scheduler resilient to blocking --- Cargo.toml | 2 + src/lib.rs | 2 + src/net/mod.rs | 1 - src/net/tcp/listener.rs | 2 +- src/net/tcp/stream.rs | 2 +- src/net/udp/mod.rs | 4 +- src/os/unix/net/datagram.rs | 2 +- src/os/unix/net/listener.rs | 2 +- src/os/unix/net/stream.rs | 2 +- src/rt/mod.rs | 23 ++ src/{net/driver/mod.rs => rt/reactor.rs} | 109 ++---- src/rt/runtime.rs | 449 +++++++++++++++++++++++ src/task/block_on.rs | 32 +- src/task/builder.rs | 30 +- src/task/executor/mod.rs | 13 - src/task/executor/pool.rs | 179 --------- src/task/executor/sleepers.rs | 52 --- src/task/join_handle.rs | 3 - src/task/mod.rs | 5 +- src/task/spawn_blocking.rs | 90 +---- src/task/yield_now.rs | 4 +- src/utils.rs | 71 ++++ 22 files changed, 623 insertions(+), 456 deletions(-) create mode 100644 src/rt/mod.rs rename src/{net/driver/mod.rs => rt/reactor.rs} (77%) create mode 100644 src/rt/runtime.rs delete mode 100644 src/task/executor/mod.rs delete mode 100644 src/task/executor/pool.rs delete mode 100644 src/task/executor/sleepers.rs diff --git a/Cargo.toml b/Cargo.toml index 7c4613b..e5ae02d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,6 +26,7 @@ default = [ "async-task", "crossbeam-channel", "crossbeam-deque", + "crossbeam-queue", "futures-timer", "kv-log-macro", "log", @@ -56,6 +57,7 @@ async-task = { version = "1.0.0", optional = true } broadcaster = { version = "0.2.6", optional = true, default-features = false, features = ["default-channels"] } crossbeam-channel = { version = "0.4.0", optional = true } crossbeam-deque = { version = "0.7.2", optional = true } +crossbeam-queue = { version = "0.2.0", optional = true } crossbeam-utils = { version = "0.7.0", optional = true } futures-core = { version = "0.3.1", optional = true } futures-io = { version = "0.3.1", optional = true } diff --git a/src/lib.rs b/src/lib.rs index d0c87ff..070df88 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -246,6 +246,8 @@ cfg_std! { pub mod stream; pub mod sync; pub mod task; + + pub(crate) mod rt; } cfg_default! { diff --git a/src/net/mod.rs b/src/net/mod.rs index 29e4309..fe83d3b 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -66,6 +66,5 @@ pub use tcp::{Incoming, TcpListener, TcpStream}; pub use udp::UdpSocket; mod addr; -pub(crate) mod driver; mod tcp; mod udp; diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index fe06a96..b389518 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -4,7 +4,7 @@ use std::pin::Pin; use crate::future; use crate::io; -use crate::net::driver::Watcher; +use crate::rt::Watcher; use crate::net::{TcpStream, ToSocketAddrs}; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 4131783..71245a3 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -4,7 +4,7 @@ use std::pin::Pin; use crate::future; use crate::io::{self, Read, Write}; -use crate::net::driver::Watcher; +use crate::rt::Watcher; use crate::net::ToSocketAddrs; use crate::task::{spawn_blocking, Context, Poll}; use crate::utils::Context as _; diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 418b4b6..961288a 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -3,8 +3,8 @@ use std::net::SocketAddr; use std::net::{Ipv4Addr, Ipv6Addr}; use crate::future; -use crate::net::driver::Watcher; use crate::net::ToSocketAddrs; +use crate::rt::Watcher; use crate::utils::Context as _; /// A UDP socket. @@ -102,7 +102,7 @@ impl UdpSocket { /// ```no_run /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { /// # - /// use async_std::net::UdpSocket; + /// use async_std::net::UdpSocket; /// /// let socket = UdpSocket::bind("127.0.0.1:0").await?; /// let addr = socket.local_addr()?; diff --git a/src/os/unix/net/datagram.rs b/src/os/unix/net/datagram.rs index fc426b7..5a2d6ec 100644 --- a/src/os/unix/net/datagram.rs +++ b/src/os/unix/net/datagram.rs @@ -8,7 +8,7 @@ use mio_uds; use super::SocketAddr; use crate::future; use crate::io; -use crate::net::driver::Watcher; +use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; use crate::task::spawn_blocking; diff --git a/src/os/unix/net/listener.rs b/src/os/unix/net/listener.rs index 675ef48..9f6bdcb 100644 --- a/src/os/unix/net/listener.rs +++ b/src/os/unix/net/listener.rs @@ -10,7 +10,7 @@ use super::SocketAddr; use super::UnixStream; use crate::future; use crate::io; -use crate::net::driver::Watcher; +use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; use crate::stream::Stream; diff --git a/src/os/unix/net/stream.rs b/src/os/unix/net/stream.rs index 647edc9..a1c83f1 100644 --- a/src/os/unix/net/stream.rs +++ b/src/os/unix/net/stream.rs @@ -9,7 +9,7 @@ use mio_uds; use super::SocketAddr; use crate::io::{self, Read, Write}; -use crate::net::driver::Watcher; +use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; use crate::task::{spawn_blocking, Context, Poll}; diff --git a/src/rt/mod.rs b/src/rt/mod.rs new file mode 100644 index 0000000..2149d24 --- /dev/null +++ b/src/rt/mod.rs @@ -0,0 +1,23 @@ +//! The runtime. + +use std::thread; + +use once_cell::sync::Lazy; + +use crate::utils::abort_on_panic; + +pub use reactor::{Reactor, Watcher}; +pub use runtime::Runtime; + +mod reactor; +mod runtime; + +/// The global runtime. +pub static RUNTIME: Lazy = Lazy::new(|| { + thread::Builder::new() + .name("async-std/runtime".to_string()) + .spawn(|| abort_on_panic(|| RUNTIME.run())) + .expect("cannot start a runtime thread"); + + Runtime::new() +}); diff --git a/src/net/driver/mod.rs b/src/rt/reactor.rs similarity index 77% rename from src/net/driver/mod.rs rename to src/rt/reactor.rs index 7f33e85..c0046c9 100644 --- a/src/net/driver/mod.rs +++ b/src/rt/reactor.rs @@ -1,13 +1,13 @@ use std::fmt; use std::sync::{Arc, Mutex}; +use std::time::Duration; use mio::{self, Evented}; -use once_cell::sync::Lazy; use slab::Slab; use crate::io; +use crate::rt::RUNTIME; use crate::task::{Context, Poll, Waker}; -use crate::utils::abort_on_panic; /// Data associated with a registered I/O handle. #[derive(Debug)] @@ -18,15 +18,18 @@ struct Entry { /// Tasks that are blocked on reading from this I/O handle. readers: Mutex>, - /// Thasks that are blocked on writing to this I/O handle. + /// Tasks that are blocked on writing to this I/O handle. writers: Mutex>, } /// The state of a networking driver. -struct Reactor { +pub struct Reactor { /// A mio instance that polls for new events. poller: mio::Poll, + /// A list into which mio stores events. + events: Mutex, + /// A collection of registered I/O handles. entries: Mutex>>, @@ -39,12 +42,13 @@ struct Reactor { impl Reactor { /// Creates a new reactor for polling I/O events. - fn new() -> io::Result { + pub fn new() -> io::Result { let poller = mio::Poll::new()?; let notify_reg = mio::Registration::new2(); let mut reactor = Reactor { poller, + events: Mutex::new(mio::Events::with_capacity(1000)), entries: Mutex::new(Slab::new()), notify_reg, notify_token: mio::Token(0), @@ -92,50 +96,32 @@ impl Reactor { Ok(()) } - // fn notify(&self) { - // self.notify_reg - // .1 - // .set_readiness(mio::Ready::readable()) - // .unwrap(); - // } -} + /// Notifies the reactor so that polling stops blocking. + pub fn notify(&self) -> io::Result<()> { + self.notify_reg.1.set_readiness(mio::Ready::readable()) + } + + /// Waits on the poller for new events and wakes up tasks blocked on I/O handles. + /// + /// Returns `Ok(true)` if at least one new task was woken. + pub fn poll(&self, timeout: Option) -> io::Result { + let mut events = self.events.lock().unwrap(); -/// The state of the global networking driver. -static REACTOR: Lazy = Lazy::new(|| { - // Spawn a thread that waits on the poller for new events and wakes up tasks blocked on I/O - // handles. - std::thread::Builder::new() - .name("async-std/net".to_string()) - .spawn(move || { - // If the driver thread panics, there's not much we can do. It is not a - // recoverable error and there is no place to propagate it into so we just abort. - abort_on_panic(|| { - main_loop().expect("async networking thread has panicked"); - }) - }) - .expect("cannot start a thread driving blocking tasks"); - - Reactor::new().expect("cannot initialize reactor") -}); - -/// Waits on the poller for new events and wakes up tasks blocked on I/O handles. -fn main_loop() -> io::Result<()> { - let reactor = &REACTOR; - let mut events = mio::Events::with_capacity(1000); - - loop { // Block on the poller until at least one new event comes in. - reactor.poller.poll(&mut events, None)?; + self.poller.poll(&mut events, timeout)?; // Lock the entire entry table while we're processing new events. - let entries = reactor.entries.lock().unwrap(); + let entries = self.entries.lock().unwrap(); + + // The number of woken tasks. + let mut progress = false; for event in events.iter() { let token = event.token(); - if token == reactor.notify_token { + if token == self.notify_token { // If this is the notification token, we just need the notification state. - reactor.notify_reg.1.set_readiness(mio::Ready::empty())?; + self.notify_reg.1.set_readiness(mio::Ready::empty())?; } else { // Otherwise, look for the entry associated with this token. if let Some(entry) = entries.get(token.0) { @@ -143,21 +129,27 @@ fn main_loop() -> io::Result<()> { let readiness = event.readiness(); // Wake up reader tasks blocked on this I/O handle. - if !(readiness & reader_interests()).is_empty() { + let reader_interests = mio::Ready::all() - mio::Ready::writable(); + if !(readiness & reader_interests).is_empty() { for w in entry.readers.lock().unwrap().drain(..) { w.wake(); + progress = true; } } // Wake up writer tasks blocked on this I/O handle. - if !(readiness & writer_interests()).is_empty() { + let writer_interests = mio::Ready::all() - mio::Ready::readable(); + if !(readiness & writer_interests).is_empty() { for w in entry.writers.lock().unwrap().drain(..) { w.wake(); + progress = true; } } } } } + + Ok(progress) } } @@ -180,7 +172,8 @@ impl Watcher { /// lifetime of the returned I/O handle. pub fn new(source: T) -> Watcher { Watcher { - entry: REACTOR + entry: RUNTIME + .reactor() .register(&source) .expect("cannot register an I/O event source"), source: Some(source), @@ -264,7 +257,8 @@ impl Watcher { #[allow(dead_code)] pub fn into_inner(mut self) -> T { let source = self.source.take().unwrap(); - REACTOR + RUNTIME + .reactor() .deregister(&source, &self.entry) .expect("cannot deregister I/O event source"); source @@ -274,7 +268,8 @@ impl Watcher { impl Drop for Watcher { fn drop(&mut self) { if let Some(ref source) = self.source { - REACTOR + RUNTIME + .reactor() .deregister(source, &self.entry) .expect("cannot deregister I/O event source"); } @@ -289,27 +284,3 @@ impl fmt::Debug for Watcher { .finish() } } - -/// Returns a mask containing flags that interest tasks reading from I/O handles. -#[inline] -fn reader_interests() -> mio::Ready { - mio::Ready::all() - mio::Ready::writable() -} - -/// Returns a mask containing flags that interest tasks writing into I/O handles. -#[inline] -fn writer_interests() -> mio::Ready { - mio::Ready::writable() | hup() -} - -/// Returns a flag containing the hangup status. -#[inline] -fn hup() -> mio::Ready { - #[cfg(unix)] - let ready = mio::unix::UnixReady::hup().into(); - - #[cfg(not(unix))] - let ready = mio::Ready::empty(); - - ready -} diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs new file mode 100644 index 0000000..35ebe50 --- /dev/null +++ b/src/rt/runtime.rs @@ -0,0 +1,449 @@ +use std::cell::Cell; +use std::io; +use std::iter; +use std::ptr; +use std::sync::atomic::{self, AtomicBool, Ordering}; +use std::sync::{Arc, Mutex}; +use std::thread; +use std::time::Duration; + +use crossbeam_deque::{Injector, Steal, Stealer, Worker}; +use crossbeam_utils::thread::scope; +use once_cell::unsync::OnceCell; + +use crate::rt::Reactor; +use crate::task::Runnable; +use crate::utils::{abort_on_panic, random, Spinlock}; + +thread_local! { + /// A reference to the current machine, if the current thread runs tasks. + static MACHINE: OnceCell> = OnceCell::new(); + + /// This flag is set to true whenever `task::yield_now()` is invoked. + static YIELD_NOW: Cell = Cell::new(false); +} + +/// Scheduler state. +struct Scheduler { + /// Set to `true` every time before a machine blocks polling the reactor. + progress: bool, + + /// Set to `true` while a machine is polling the reactor. + polling: bool, + + /// Idle processors. + processors: Vec, + + /// Running machines. + machines: Vec>, +} + +/// An async runtime. +pub struct Runtime { + /// The reactor. + reactor: Reactor, + + /// The global queue of tasks. + injector: Injector, + + /// Handles to local queues for stealing work. + stealers: Vec>, + + /// The scheduler state. + sched: Mutex, +} + +impl Runtime { + /// Creates a new runtime. + pub fn new() -> Runtime { + let cpus = num_cpus::get().max(1); + let processors: Vec<_> = (0..cpus).map(|_| Processor::new()).collect(); + let stealers = processors.iter().map(|p| p.worker.stealer()).collect(); + + Runtime { + reactor: Reactor::new().unwrap(), + injector: Injector::new(), + stealers, + sched: Mutex::new(Scheduler { + processors, + machines: Vec::new(), + progress: false, + polling: false, + }), + } + } + + /// Returns a reference to the reactor. + pub fn reactor(&self) -> &Reactor { + &self.reactor + } + + /// Flushes the task slot so that tasks get run more fairly. + pub fn yield_now(&self) { + YIELD_NOW.with(|flag| flag.set(true)); + } + + /// Schedules a task. + pub fn schedule(&self, task: Runnable) { + MACHINE.with(|machine| { + // If the current thread is a worker thread, schedule it onto the current machine. + // Otherwise, push it into the global task queue. + match machine.get() { + None => { + self.injector.push(task); + self.notify(); + } + Some(m) => m.schedule(&self, task), + } + }); + } + + /// Runs the runtime on the current thread. + pub fn run(&self) { + scope(|s| { + let mut idle = 0; + let mut delay = 0; + + loop { + // Get a list of new machines to start, if any need to be started. + for m in self.make_machines() { + idle = 0; + + s.builder() + .name("async-std/machine".to_string()) + .spawn(move |_| { + abort_on_panic(|| { + let _ = MACHINE.with(|machine| machine.set(m.clone())); + m.run(self); + }) + }) + .expect("cannot start a machine thread"); + } + + // Sleep for a bit longer if the scheduler state hasn't changed in a while. + if idle > 10 { + delay = (delay * 2).min(10_000); + } else { + idle += 1; + delay = 1000; + } + + thread::sleep(Duration::from_micros(delay)); + } + }) + .unwrap(); + } + + /// Returns a list of machines that need to be started. + fn make_machines(&self) -> Vec> { + let mut sched = self.sched.lock().unwrap(); + let mut to_start = Vec::new(); + + // If there is a machine that is stuck on a task and not making any progress, steal its + // processor and set up a new machine to take over. + for m in &mut sched.machines { + if !m.progress.swap(false, Ordering::SeqCst) { + let opt_p = m.processor.try_lock().and_then(|mut p| p.take()); + + if let Some(p) = opt_p { + *m = Arc::new(Machine::new(p)); + to_start.push(m.clone()); + } + } + } + + // If no machine has been polling the reactor in a while, that means the runtime is + // overloaded with work and we need to start another machine. + if !sched.polling { + if !sched.progress { + if let Some(p) = sched.processors.pop() { + let m = Arc::new(Machine::new(p)); + to_start.push(m.clone()); + sched.machines.push(m); + } + } + + sched.progress = false; + } + + to_start + } + + /// Unparks a thread polling the reactor. + fn notify(&self) { + atomic::fence(Ordering::SeqCst); + self.reactor.notify().unwrap(); + } + + /// Attempts to poll the reactor without blocking on it. + /// + /// Returns `Ok(true)` if at least one new task was woken. + /// + /// This function might not poll the reactor at all so do not rely on it doing anything. Only + /// use for optimization. + fn quick_poll(&self) -> io::Result { + if let Ok(sched) = self.sched.try_lock() { + if !sched.polling { + return self.reactor.poll(Some(Duration::from_secs(0))); + } + } + Ok(false) + } +} + +/// A thread running a processor. +struct Machine { + /// Holds the processor until it gets stolen. + processor: Spinlock>, + + /// Gets set to `true` before running every task to indicate the machine is not stuck. + progress: AtomicBool, +} + +impl Machine { + /// Creates a new machine running a processor. + fn new(p: Processor) -> Machine { + Machine { + processor: Spinlock::new(Some(p)), + progress: AtomicBool::new(true), + } + } + + /// Schedules a task onto the machine. + fn schedule(&self, rt: &Runtime, task: Runnable) { + match self.processor.lock().as_mut() { + None => { + rt.injector.push(task); + rt.notify(); + } + Some(p) => p.schedule(rt, task), + } + } + + /// Finds the next runnable task. + fn find_task(&self, rt: &Runtime) -> Steal { + let mut retry = false; + + // First try finding a task in the local queue or in the global queue. + if let Some(p) = self.processor.lock().as_mut() { + if let Some(task) = p.pop_task() { + return Steal::Success(task); + } + + match p.steal_from_global(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), + } + } + + // Try polling the reactor, but don't block on it. + let progress = rt.quick_poll().unwrap(); + + // Try finding a task in the local queue, which might hold tasks woken by the reactor. If + // the local queue is still empty, try stealing from other processors. + if let Some(p) = self.processor.lock().as_mut() { + if progress { + if let Some(task) = p.pop_task() { + return Steal::Success(task); + } + } + + match p.steal_from_others(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), + } + } + + if retry { Steal::Retry } else { Steal::Empty } + } + + /// Runs the machine on the current thread. + fn run(&self, rt: &Runtime) { + /// Number of yields when no runnable task is found. + const YIELDS: u32 = 3; + /// Number of short sleeps when no runnable task in found. + const SLEEPS: u32 = 10; + /// Number of runs in a row before the global queue is inspected. + const RUNS: u32 = 64; + + // The number of times the thread found work in a row. + let mut runs = 0; + // The number of times the thread didn't find work in a row. + let mut fails = 0; + + loop { + // let the scheduler know this machine is making progress. + self.progress.store(true, Ordering::SeqCst); + + // Check if `task::yield_now()` was invoked and flush the slot if so. + YIELD_NOW.with(|flag| { + if flag.replace(false) { + if let Some(p) = self.processor.lock().as_mut() { + p.flush_slot(rt); + } + } + }); + + // After a number of runs in a row, do some work to ensure no task is left behind + // indefinitely. Poll the reactor, steal tasks from the global queue, and flush the + // task slot. + if runs >= RUNS { + runs = 0; + rt.quick_poll().unwrap(); + + if let Some(p) = self.processor.lock().as_mut() { + if let Steal::Success(task) = p.steal_from_global(rt) { + p.schedule(rt, task); + } + + p.flush_slot(rt); + } + } + + // Try to find a runnable task. + if let Steal::Success(task) = self.find_task(rt) { + task.run(); + runs += 1; + fails = 0; + continue; + } + + fails += 1; + + // Check if the processor was stolen. + if self.processor.lock().is_none() { + break; + } + + // Yield the current thread a few times. + if fails <= YIELDS { + thread::yield_now(); + continue; + } + + // Put the current thread to sleep a few times. + if fails <= YIELDS + SLEEPS { + let opt_p = self.processor.lock().take(); + thread::sleep(Duration::from_micros(10)); + *self.processor.lock() = opt_p; + continue; + } + + let mut sched = rt.sched.lock().unwrap(); + + // One final check for available tasks while the scheduler is locked. + if let Some(task) = iter::repeat_with(|| self.find_task(rt)) + .find(|s| !s.is_retry()) + .and_then(|s| s.success()) + { + self.schedule(rt, task); + continue; + } + + // If another thread is already blocked on the reactor, there is no point in keeping + // the current thread around since there is too little work to do. + if sched.polling { + break; + } + + // Take out the machine associated with the current thread. + let m = match sched + .machines + .iter() + .position(|elem| ptr::eq(&**elem, self)) + { + None => break, // The processor was stolen. + Some(pos) => sched.machines.swap_remove(pos), + }; + + // Unlock the schedule poll the reactor until new I/O events arrive. + sched.polling = true; + drop(sched); + rt.reactor.poll(None).unwrap(); + + // Lock the scheduler again and re-register the machine. + sched = rt.sched.lock().unwrap(); + sched.polling = false; + sched.machines.push(m); + sched.progress = true; + + runs = 0; + fails = 0; + } + + // When shutting down the thread, take the processor out if still available. + let opt_p = self.processor.lock().take(); + + // Return the processor to the scheduler and remove the machine. + if let Some(p) = opt_p { + let mut sched = rt.sched.lock().unwrap(); + sched.processors.push(p); + sched.machines.retain(|elem| !ptr::eq(&**elem, self)); + } + } +} + +struct Processor { + /// The local task queue. + worker: Worker, + + /// Contains the next task to run as an optimization that skips the queue. + slot: Option, +} + +impl Processor { + /// Creates a new processor. + fn new() -> Processor { + Processor { + worker: Worker::new_fifo(), + slot: None, + } + } + + /// Schedules a task to run on this processor. + fn schedule(&mut self, rt: &Runtime, task: Runnable) { + match self.slot.replace(task) { + None => {} + Some(task) => { + self.worker.push(task); + rt.notify(); + } + } + } + + /// Flushes a task from the slot into the local queue. + fn flush_slot(&mut self, rt: &Runtime) { + if let Some(task) = self.slot.take() { + self.worker.push(task); + rt.notify(); + } + } + + /// Pops a task from this processor. + fn pop_task(&mut self) -> Option { + self.slot.take().or_else(|| self.worker.pop()) + } + + /// Steals a task from the global queue. + fn steal_from_global(&mut self, rt: &Runtime) -> Steal { + rt.injector.steal_batch_and_pop(&self.worker) + } + + /// Steals a task from other processors. + fn steal_from_others(&mut self, rt: &Runtime) -> Steal { + // Pick a random starting point in the list of queues. + let len = rt.stealers.len(); + let start = random(len as u32) as usize; + + // Create an iterator over stealers that starts from the chosen point. + let (l, r) = rt.stealers.split_at(start); + let stealers = r.iter().chain(l.iter()); + + // Try stealing a batch of tasks from each queue. + stealers + .map(|s| s.steal_batch_and_pop(&self.worker)) + .collect() + } +} diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 80259c5..4bade5b 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -3,11 +3,9 @@ use std::future::Future; use std::mem::{self, ManuallyDrop}; use std::sync::Arc; use std::task::{RawWaker, RawWakerVTable}; -use std::thread; use crossbeam_utils::sync::Parker; use kv_log_macro::trace; -use log::log_enabled; use crate::task::{Context, Poll, Task, Waker}; @@ -42,12 +40,10 @@ where let task = Task::new(None); // Log this `block_on` operation. - if log_enabled!(log::Level::Trace) { - trace!("block_on", { - task_id: task.id().0, - parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), - }); - } + trace!("block_on", { + task_id: task.id().0, + parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), + }); let future = async move { // Drop task-locals on exit. @@ -57,13 +53,9 @@ where // Log completion on exit. defer! { - if log_enabled!(log::Level::Trace) { - Task::get_current(|t| { - trace!("completed", { - task_id: t.id().0, - }); - }); - } + trace!("completed", { + task_id: Task::get_current(|t| t.id().0), + }); } future.await @@ -125,7 +117,6 @@ where let waker = unsafe { ManuallyDrop::new(Waker::from_raw(RawWaker::new(ptr, &VTABLE))) }; let cx = &mut Context::from_waker(&waker); - let mut step = 0; loop { if let Poll::Ready(t) = future.as_mut().poll(cx) { // Save the parker for the next invocation of `block`. @@ -133,14 +124,7 @@ where return t; } - // Yield a few times or park the current thread. - if step < 3 { - thread::yield_now(); - step += 1; - } else { - arc_parker.park(); - step = 0; - } + arc_parker.park(); } }) } diff --git a/src/task/builder.rs b/src/task/builder.rs index afd4c2c..f1fef59 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -1,9 +1,9 @@ -use kv_log_macro::trace; -use log::log_enabled; use std::future::Future; +use kv_log_macro::trace; + use crate::io; -use crate::task::executor; +use crate::rt::RUNTIME; use crate::task::{JoinHandle, Task}; use crate::utils::abort_on_panic; @@ -37,12 +37,10 @@ impl Builder { let task = Task::new(self.name); // Log this `spawn` operation. - if log_enabled!(log::Level::Trace) { - trace!("spawn", { - task_id: task.id().0, - parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), - }); - } + trace!("spawn", { + task_id: task.id().0, + parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), + }); let future = async move { // Drop task-locals on exit. @@ -52,19 +50,15 @@ impl Builder { // Log completion on exit. defer! { - if log_enabled!(log::Level::Trace) { - Task::get_current(|t| { - trace!("completed", { - task_id: t.id().0, - }); - }); - } + trace!("completed", { + task_id: Task::get_current(|t| t.id().0), + }); } future.await }; - let schedule = move |t| executor::schedule(Runnable(t)); + let schedule = move |t| RUNTIME.schedule(Runnable(t)); let (task, handle) = async_task::spawn(future, schedule, task); task.schedule(); Ok(JoinHandle::new(handle)) @@ -72,7 +66,7 @@ impl Builder { } /// A runnable task. -pub(crate) struct Runnable(async_task::Task); +pub struct Runnable(async_task::Task); impl Runnable { /// Runs the task by polling its future once. diff --git a/src/task/executor/mod.rs b/src/task/executor/mod.rs deleted file mode 100644 index 2a6a696..0000000 --- a/src/task/executor/mod.rs +++ /dev/null @@ -1,13 +0,0 @@ -//! Task executor. -//! -//! API bindings between `crate::task` and this module are very simple: -//! -//! * The only export is the `schedule` function. -//! * The only import is the `crate::task::Runnable` type. - -pub(crate) use pool::schedule; - -use sleepers::Sleepers; - -mod pool; -mod sleepers; diff --git a/src/task/executor/pool.rs b/src/task/executor/pool.rs deleted file mode 100644 index 5249b3d..0000000 --- a/src/task/executor/pool.rs +++ /dev/null @@ -1,179 +0,0 @@ -use std::cell::Cell; -use std::iter; -use std::thread; -use std::time::Duration; - -use crossbeam_deque::{Injector, Stealer, Worker}; -use once_cell::sync::Lazy; -use once_cell::unsync::OnceCell; - -use crate::task::executor::Sleepers; -use crate::task::Runnable; -use crate::utils::{abort_on_panic, random}; - -/// The state of an executor. -struct Pool { - /// The global queue of tasks. - injector: Injector, - - /// Handles to local queues for stealing work from worker threads. - stealers: Vec>, - - /// Used for putting idle workers to sleep and notifying them when new tasks come in. - sleepers: Sleepers, -} - -/// Global executor that runs spawned tasks. -static POOL: Lazy = Lazy::new(|| { - let num_threads = num_cpus::get().max(1); - let mut stealers = Vec::new(); - - // Spawn worker threads. - for _ in 0..num_threads { - let worker = Worker::new_fifo(); - stealers.push(worker.stealer()); - - let proc = Processor { - worker, - slot: Cell::new(None), - slot_runs: Cell::new(0), - }; - - thread::Builder::new() - .name("async-std/executor".to_string()) - .spawn(|| { - let _ = PROCESSOR.with(|p| p.set(proc)); - abort_on_panic(main_loop); - }) - .expect("cannot start a thread driving tasks"); - } - - Pool { - injector: Injector::new(), - stealers, - sleepers: Sleepers::new(), - } -}); - -/// The state of a worker thread. -struct Processor { - /// The local task queue. - worker: Worker, - - /// Contains the next task to run as an optimization that skips queues. - slot: Cell>, - - /// How many times in a row tasks have been taked from the slot rather than the queue. - slot_runs: Cell, -} - -thread_local! { - /// Worker thread state. - static PROCESSOR: OnceCell = OnceCell::new(); -} - -/// Schedules a new runnable task for execution. -pub(crate) fn schedule(task: Runnable) { - PROCESSOR.with(|proc| { - // If the current thread is a worker thread, store it into its task slot or push it into - // its local task queue. Otherwise, push it into the global task queue. - match proc.get() { - Some(proc) => { - // Replace the task in the slot. - if let Some(task) = proc.slot.replace(Some(task)) { - // If the slot already contained a task, push it into the local task queue. - proc.worker.push(task); - POOL.sleepers.notify_one(); - } - } - None => { - POOL.injector.push(task); - POOL.sleepers.notify_one(); - } - } - }) -} - -/// Main loop running a worker thread. -fn main_loop() { - /// Number of yields when no runnable task is found. - const YIELDS: u32 = 3; - /// Number of short sleeps when no runnable task in found. - const SLEEPS: u32 = 1; - - // The number of times the thread didn't find work in a row. - let mut fails = 0; - - loop { - // Try to find a runnable task. - match find_runnable() { - Some(task) => { - fails = 0; - - // Run the found task. - task.run(); - } - None => { - fails += 1; - - // Yield the current thread or put it to sleep. - if fails <= YIELDS { - thread::yield_now(); - } else if fails <= YIELDS + SLEEPS { - thread::sleep(Duration::from_micros(10)); - } else { - POOL.sleepers.wait(); - fails = 0; - } - } - } - } -} - -/// Find the next runnable task. -fn find_runnable() -> Option { - /// Maximum number of times the slot can be used in a row. - const SLOT_LIMIT: u32 = 16; - - PROCESSOR.with(|proc| { - let proc = proc.get().unwrap(); - - // Try taking a task from the slot. - let runs = proc.slot_runs.get(); - if runs < SLOT_LIMIT { - if let Some(task) = proc.slot.take() { - proc.slot_runs.set(runs + 1); - return Some(task); - } - } - proc.slot_runs.set(0); - - // Pop a task from the local queue, if not empty. - proc.worker.pop().or_else(|| { - // Otherwise, we need to look for a task elsewhere. - iter::repeat_with(|| { - // Try stealing a batch of tasks from the global queue. - POOL.injector - .steal_batch_and_pop(&proc.worker) - // Or try stealing a batch of tasks from one of the other threads. - .or_else(|| { - // First, pick a random starting point in the list of local queues. - let len = POOL.stealers.len(); - let start = random(len as u32) as usize; - - // Try stealing a batch of tasks from each local queue starting from the - // chosen point. - let (l, r) = POOL.stealers.split_at(start); - let stealers = r.iter().chain(l.iter()); - stealers - .map(|s| s.steal_batch_and_pop(&proc.worker)) - .collect() - }) - }) - // Loop while no task was stolen and any steal operation needs to be retried. - .find(|s| !s.is_retry()) - // Extract the stolen task, if there is one. - .and_then(|s| s.success()) - }) - }) -} diff --git a/src/task/executor/sleepers.rs b/src/task/executor/sleepers.rs deleted file mode 100644 index 4e70129..0000000 --- a/src/task/executor/sleepers.rs +++ /dev/null @@ -1,52 +0,0 @@ -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::{Condvar, Mutex}; - -/// The place where worker threads go to sleep. -/// -/// Similar to how thread parking works, if a notification comes up while no threads are sleeping, -/// the next thread that attempts to go to sleep will pick up the notification immediately. -pub struct Sleepers { - /// How many threads are currently a sleep. - sleep: Mutex, - - /// A condvar for notifying sleeping threads. - wake: Condvar, - - /// Set to `true` if a notification came up while nobody was sleeping. - notified: AtomicBool, -} - -impl Sleepers { - /// Creates a new `Sleepers`. - pub fn new() -> Sleepers { - Sleepers { - sleep: Mutex::new(0), - wake: Condvar::new(), - notified: AtomicBool::new(false), - } - } - - /// Puts the current thread to sleep. - pub fn wait(&self) { - let mut sleep = self.sleep.lock().unwrap(); - - if !self.notified.swap(false, Ordering::SeqCst) { - *sleep += 1; - let _ = self.wake.wait(sleep).unwrap(); - } - } - - /// Notifies one thread. - pub fn notify_one(&self) { - if !self.notified.load(Ordering::SeqCst) { - let mut sleep = self.sleep.lock().unwrap(); - - if *sleep > 0 { - *sleep -= 1; - self.wake.notify_one(); - } else { - self.notified.store(true, Ordering::SeqCst); - } - } - } -} diff --git a/src/task/join_handle.rs b/src/task/join_handle.rs index 9fefff2..d929d11 100644 --- a/src/task/join_handle.rs +++ b/src/task/join_handle.rs @@ -14,9 +14,6 @@ use crate::task::{Context, Poll, Task}; #[derive(Debug)] pub struct JoinHandle(async_task::JoinHandle); -unsafe impl Send for JoinHandle {} -unsafe impl Sync for JoinHandle {} - impl JoinHandle { /// Creates a new `JoinHandle`. pub(crate) fn new(inner: async_task::JoinHandle) -> JoinHandle { diff --git a/src/task/mod.rs b/src/task/mod.rs index 8e181d1..075e71d 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -140,13 +140,12 @@ cfg_default! { pub use spawn::spawn; pub use task_local::{AccessError, LocalKey}; - use builder::Runnable; - use task_local::LocalsMap; + pub(crate) use builder::Runnable; + pub(crate) use task_local::LocalsMap; mod block_on; mod builder; mod current; - mod executor; mod join_handle; mod sleep; mod spawn; diff --git a/src/task/spawn_blocking.rs b/src/task/spawn_blocking.rs index 578afa4..e22c5cb 100644 --- a/src/task/spawn_blocking.rs +++ b/src/task/spawn_blocking.rs @@ -1,12 +1,4 @@ -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::thread; -use std::time::Duration; - -use crossbeam_channel::{unbounded, Receiver, Sender}; -use once_cell::sync::Lazy; - -use crate::task::{JoinHandle, Task}; -use crate::utils::abort_on_panic; +use crate::task::{spawn, JoinHandle}; /// Spawns a blocking task. /// @@ -31,7 +23,8 @@ use crate::utils::abort_on_panic; /// /// task::spawn_blocking(|| { /// println!("long-running task here"); -/// }).await; +/// }) +/// .await; /// # /// # }) /// ``` @@ -42,80 +35,5 @@ where F: FnOnce() -> T + Send + 'static, T: Send + 'static, { - let schedule = |task| POOL.sender.send(task).unwrap(); - let (task, handle) = async_task::spawn(async { f() }, schedule, Task::new(None)); - task.schedule(); - JoinHandle::new(handle) -} - -type Runnable = async_task::Task; - -/// The number of sleeping worker threads. -static SLEEPING: AtomicUsize = AtomicUsize::new(0); - -struct Pool { - sender: Sender, - receiver: Receiver, -} - -static POOL: Lazy = Lazy::new(|| { - // Start a single worker thread waiting for the first task. - start_thread(); - - let (sender, receiver) = unbounded(); - Pool { sender, receiver } -}); - -fn start_thread() { - SLEEPING.fetch_add(1, Ordering::SeqCst); - let timeout = Duration::from_secs(1); - - thread::Builder::new() - .name("async-std/blocking".to_string()) - .spawn(move || { - loop { - let mut task = match POOL.receiver.recv_timeout(timeout) { - Ok(task) => task, - Err(_) => { - // Check whether this is the last sleeping thread. - if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { - // If so, then restart the thread to make sure there is always at least - // one sleeping thread. - if SLEEPING.compare_and_swap(0, 1, Ordering::SeqCst) == 0 { - continue; - } - } - - // Stop the thread. - return; - } - }; - - // If there are no sleeping threads, then start one to make sure there is always at - // least one sleeping thread. - if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { - start_thread(); - } - - loop { - // Run the task. - abort_on_panic(|| task.run()); - - // Try taking another task if there are any available. - task = match POOL.receiver.try_recv() { - Ok(task) => task, - Err(_) => break, - }; - } - - // If there is at least one sleeping thread, stop this thread instead of putting it - // to sleep. - if SLEEPING.load(Ordering::SeqCst) > 0 { - return; - } - - SLEEPING.fetch_add(1, Ordering::SeqCst); - } - }) - .expect("cannot start a blocking thread"); + spawn(async { f() }) } diff --git a/src/task/yield_now.rs b/src/task/yield_now.rs index 4030696..c7ddb17 100644 --- a/src/task/yield_now.rs +++ b/src/task/yield_now.rs @@ -1,6 +1,7 @@ -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; +use crate::rt::RUNTIME; use crate::task::{Context, Poll}; /// Cooperatively gives up a timeslice to the task scheduler. @@ -43,6 +44,7 @@ impl Future for YieldNow { if !self.0 { self.0 = true; cx.waker().wake_by_ref(); + RUNTIME.yield_now(); Poll::Pending } else { Poll::Ready(()) diff --git a/src/utils.rs b/src/utils.rs index 7d253b4..79c2fdf 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -1,3 +1,9 @@ +use std::cell::UnsafeCell; +use std::ops::{Deref, DerefMut}; +use std::sync::atomic::{AtomicBool, Ordering}; + +use crossbeam_utils::Backoff; + /// Calls a function and aborts if it panics. /// /// This is useful in unsafe code where we can't recover from panics. @@ -52,6 +58,71 @@ pub fn random(n: u32) -> u32 { }) } +/// A simple spinlock. +pub struct Spinlock { + flag: AtomicBool, + value: UnsafeCell, +} + +unsafe impl Send for Spinlock {} +unsafe impl Sync for Spinlock {} + +impl Spinlock { + /// Returns a new spinlock initialized with `value`. + pub const fn new(value: T) -> Spinlock { + Spinlock { + flag: AtomicBool::new(false), + value: UnsafeCell::new(value), + } + } + + /// Locks the spinlock. + pub fn lock(&self) -> SpinlockGuard<'_, T> { + let backoff = Backoff::new(); + while self.flag.swap(true, Ordering::Acquire) { + backoff.snooze(); + } + SpinlockGuard { parent: self } + } + + /// Attempts to lock the spinlock. + pub fn try_lock(&self) -> Option> { + if self.flag.swap(true, Ordering::Acquire) { + None + } else { + Some(SpinlockGuard { parent: self }) + } + } +} + +/// A guard holding a spinlock locked. +pub struct SpinlockGuard<'a, T> { + parent: &'a Spinlock, +} + +unsafe impl Send for SpinlockGuard<'_, T> {} +unsafe impl Sync for SpinlockGuard<'_, T> {} + +impl<'a, T> Drop for SpinlockGuard<'a, T> { + fn drop(&mut self) { + self.parent.flag.store(false, Ordering::Release); + } +} + +impl<'a, T> Deref for SpinlockGuard<'a, T> { + type Target = T; + + fn deref(&self) -> &T { + unsafe { &*self.parent.value.get() } + } +} + +impl<'a, T> DerefMut for SpinlockGuard<'a, T> { + fn deref_mut(&mut self) -> &mut T { + unsafe { &mut *self.parent.value.get() } + } +} + /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; From ceba324bef9641d61239ed68a2f6671f59fa2831 Mon Sep 17 00:00:00 2001 From: Stjepan Glavina Date: Mon, 16 Dec 2019 15:53:31 +0100 Subject: [PATCH 02/60] Fix feature flags --- src/lib.rs | 3 +- src/task/yield_now.rs | 6 +- src/utils.rs | 144 +++++++++++++++++++++--------------------- 3 files changed, 78 insertions(+), 75 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index 070df88..40ba15a 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -246,14 +246,13 @@ cfg_std! { pub mod stream; pub mod sync; pub mod task; - - pub(crate) mod rt; } cfg_default! { pub mod fs; pub mod path; pub mod net; + pub(crate) mod rt; } cfg_unstable! { diff --git a/src/task/yield_now.rs b/src/task/yield_now.rs index c7ddb17..bdb08eb 100644 --- a/src/task/yield_now.rs +++ b/src/task/yield_now.rs @@ -1,7 +1,6 @@ use std::future::Future; use std::pin::Pin; -use crate::rt::RUNTIME; use crate::task::{Context, Poll}; /// Cooperatively gives up a timeslice to the task scheduler. @@ -44,7 +43,10 @@ impl Future for YieldNow { if !self.0 { self.0 = true; cx.waker().wake_by_ref(); - RUNTIME.yield_now(); + + #[cfg(feature = "default")] + crate::rt::RUNTIME.yield_now(); + Poll::Pending } else { Poll::Ready(()) diff --git a/src/utils.rs b/src/utils.rs index 79c2fdf..c8984ba 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -1,9 +1,3 @@ -use std::cell::UnsafeCell; -use std::ops::{Deref, DerefMut}; -use std::sync::atomic::{AtomicBool, Ordering}; - -use crossbeam_utils::Backoff; - /// Calls a function and aborts if it panics. /// /// This is useful in unsafe code where we can't recover from panics. @@ -58,71 +52,6 @@ pub fn random(n: u32) -> u32 { }) } -/// A simple spinlock. -pub struct Spinlock { - flag: AtomicBool, - value: UnsafeCell, -} - -unsafe impl Send for Spinlock {} -unsafe impl Sync for Spinlock {} - -impl Spinlock { - /// Returns a new spinlock initialized with `value`. - pub const fn new(value: T) -> Spinlock { - Spinlock { - flag: AtomicBool::new(false), - value: UnsafeCell::new(value), - } - } - - /// Locks the spinlock. - pub fn lock(&self) -> SpinlockGuard<'_, T> { - let backoff = Backoff::new(); - while self.flag.swap(true, Ordering::Acquire) { - backoff.snooze(); - } - SpinlockGuard { parent: self } - } - - /// Attempts to lock the spinlock. - pub fn try_lock(&self) -> Option> { - if self.flag.swap(true, Ordering::Acquire) { - None - } else { - Some(SpinlockGuard { parent: self }) - } - } -} - -/// A guard holding a spinlock locked. -pub struct SpinlockGuard<'a, T> { - parent: &'a Spinlock, -} - -unsafe impl Send for SpinlockGuard<'_, T> {} -unsafe impl Sync for SpinlockGuard<'_, T> {} - -impl<'a, T> Drop for SpinlockGuard<'a, T> { - fn drop(&mut self) { - self.parent.flag.store(false, Ordering::Release); - } -} - -impl<'a, T> Deref for SpinlockGuard<'a, T> { - type Target = T; - - fn deref(&self) -> &T { - unsafe { &*self.parent.value.get() } - } -} - -impl<'a, T> DerefMut for SpinlockGuard<'a, T> { - fn deref_mut(&mut self) -> &mut T { - unsafe { &mut *self.parent.value.get() } - } -} - /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; @@ -338,3 +267,76 @@ macro_rules! extension_trait { extension_trait!($($tail)*); }; } + +cfg_default! { + use std::cell::UnsafeCell; + use std::ops::{Deref, DerefMut}; + use std::sync::atomic::{AtomicBool, Ordering}; + + use crossbeam_utils::Backoff; + + /// A simple spinlock. + pub struct Spinlock { + flag: AtomicBool, + value: UnsafeCell, + } + + unsafe impl Send for Spinlock {} + unsafe impl Sync for Spinlock {} + + impl Spinlock { + /// Returns a new spinlock initialized with `value`. + pub const fn new(value: T) -> Spinlock { + Spinlock { + flag: AtomicBool::new(false), + value: UnsafeCell::new(value), + } + } + + /// Locks the spinlock. + pub fn lock(&self) -> SpinlockGuard<'_, T> { + let backoff = Backoff::new(); + while self.flag.swap(true, Ordering::Acquire) { + backoff.snooze(); + } + SpinlockGuard { parent: self } + } + + /// Attempts to lock the spinlock. + pub fn try_lock(&self) -> Option> { + if self.flag.swap(true, Ordering::Acquire) { + None + } else { + Some(SpinlockGuard { parent: self }) + } + } + } + + /// A guard holding a spinlock locked. + pub struct SpinlockGuard<'a, T> { + parent: &'a Spinlock, + } + + unsafe impl Send for SpinlockGuard<'_, T> {} + unsafe impl Sync for SpinlockGuard<'_, T> {} + + impl<'a, T> Drop for SpinlockGuard<'a, T> { + fn drop(&mut self) { + self.parent.flag.store(false, Ordering::Release); + } + } + + impl<'a, T> Deref for SpinlockGuard<'a, T> { + type Target = T; + + fn deref(&self) -> &T { + unsafe { &*self.parent.value.get() } + } + } + + impl<'a, T> DerefMut for SpinlockGuard<'a, T> { + fn deref_mut(&mut self) -> &mut T { + unsafe { &mut *self.parent.value.get() } + } + } +} From 0b0531057d023c71dcf20b475d264c244aeda581 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sun, 8 Mar 2020 20:46:26 +0900 Subject: [PATCH 03/60] feat: update dependence crates --- Cargo.toml | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index c1479c2..c67ccad 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -54,31 +54,31 @@ alloc = [ [dependencies] async-attributes = { version = "1.1.1", optional = true } -async-task = { version = "1.2.1", optional = true } +async-task = { version = "1.3.1", optional = true } broadcaster = { version = "1.0.0", optional = true } -crossbeam-channel = { version = "0.4.0", optional = true } -crossbeam-deque = { version = "0.7.2", optional = true } -crossbeam-utils = { version = "0.7.0", optional = true } -futures-core = { version = "0.3.1", optional = true, default-features = false } -futures-io = { version = "0.3.1", optional = true } +crossbeam-channel = { version = "0.4.2", optional = true } +crossbeam-deque = { version = "0.7.3", optional = true } +crossbeam-utils = { version = "0.7.2", optional = true } +futures-core = { version = "0.3.4", optional = true, default-features = false } +futures-io = { version = "0.3.4", optional = true } futures-timer = { version = "2.0.2", optional = true } kv-log-macro = { version = "1.0.4", optional = true } log = { version = "0.4.8", features = ["kv_unstable"], optional = true } -memchr = { version = "2.3.0", optional = true } +memchr = { version = "2.3.3", optional = true } mio = { version = "0.6.19", optional = true } mio-uds = { version = "0.6.7", optional = true } -num_cpus = { version = "1.11.1", optional = true } -once_cell = { version = "1.2.0", optional = true } -pin-project-lite = { version = "0.1.2", optional = true } +num_cpus = { version = "1.12.0", optional = true } +once_cell = { version = "1.3.1", optional = true } +pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } [dev-dependencies] femme = "1.3.0" rand = "0.7.3" -surf = "1.0.3" +surf = "2.0.0-alpha.0" tempdir = "0.3.7" -futures = "0.3.1" +futures = "0.3.4" [[test]] name = "stream" From f69887a50de8a3e2c4469bd2440224ca88bc8365 Mon Sep 17 00:00:00 2001 From: nasa Date: Mon, 9 Mar 2020 09:09:17 +0900 Subject: [PATCH 04/60] Update Cargo.toml --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index c67ccad..0bba8ec 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -76,7 +76,7 @@ slab = { version = "0.4.2", optional = true } [dev-dependencies] femme = "1.3.0" rand = "0.7.3" -surf = "2.0.0-alpha.0" +surf = "1.0.3" tempdir = "0.3.7" futures = "0.3.4" From 98cbf7f8ebbab626d6cb8e46d2048b54687c554d Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 17 Mar 2020 20:39:30 +0900 Subject: [PATCH 05/60] Restore task::spawn_blocking --- src/task/spawn_blocking.rs | 87 +++++++++++++++++++++++++++++++++++++- 1 file changed, 85 insertions(+), 2 deletions(-) diff --git a/src/task/spawn_blocking.rs b/src/task/spawn_blocking.rs index e22c5cb..27143f7 100644 --- a/src/task/spawn_blocking.rs +++ b/src/task/spawn_blocking.rs @@ -1,4 +1,12 @@ -use crate::task::{spawn, JoinHandle}; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::thread; +use std::time::Duration; + +use crossbeam_channel::{unbounded, Receiver, Sender}; +use once_cell::sync::Lazy; + +use crate::task::{JoinHandle, Task}; +use crate::utils::abort_on_panic; /// Spawns a blocking task. /// @@ -35,5 +43,80 @@ where F: FnOnce() -> T + Send + 'static, T: Send + 'static, { - spawn(async { f() }) + let schedule = |task| POOL.sender.send(task).unwrap(); + let (task, handle) = async_task::spawn(async { f() }, schedule, Task::new(None)); + task.schedule(); + JoinHandle::new(handle) +} + +type Runnable = async_task::Task; + +struct Pool { + sender: Sender, + receiver: Receiver, +} + +/// The number of sleeping worker threads. +static SLEEPING: AtomicUsize = AtomicUsize::new(0); + +static POOL: Lazy = Lazy::new(|| { + // Start a single worker thread waiting for the first task. + start_thread(); + + let (sender, receiver) = unbounded(); + Pool { sender, receiver } +}); + +fn start_thread() { + SLEEPING.fetch_add(1, Ordering::SeqCst); + let timeout = Duration::from_secs(1); + + thread::Builder::new() + .name("async-std/blocking".to_string()) + .spawn(move || { + loop { + let mut task = match POOL.receiver.recv_timeout(timeout) { + Ok(task) => task, + Err(_) => { + // Check whether this is the last sleeping thread. + if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { + // If so, then restart the thread to make sure there is always at least + // one sleeping thread. + if SLEEPING.compare_and_swap(0, 1, Ordering::SeqCst) == 0 { + continue; + } + } + + // Stop the thread. + return; + } + }; + + // If there are no sleeping threads, then start one to make sure there is always at + // least one sleeping thread. + if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { + start_thread(); + } + + loop { + // Run the task. + abort_on_panic(|| task.run()); + + // Try taking another task if there are any available. + task = match POOL.receiver.try_recv() { + Ok(task) => task, + Err(_) => break, + }; + } + + // If there is at least one sleeping thread, stop this thread instead of putting it + // to sleep. + if SLEEPING.load(Ordering::SeqCst) > 0 { + return; + } + + SLEEPING.fetch_add(1, Ordering::SeqCst); + } + }) + .expect("cannot start a blocking thread"); } From b1ec1ea9300de215f2b46d2adcd232602abd214e Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 18 Mar 2020 23:59:12 +0900 Subject: [PATCH 06/60] Move Spinlock to sync module --- src/rt/runtime.rs | 3 +- src/sync/mod.rs | 2 + src/sync/spin_lock.rs | 96 +++++++++++++++++++++++++++++++++++++++++++ src/utils.rs | 73 -------------------------------- 4 files changed, 100 insertions(+), 74 deletions(-) create mode 100644 src/sync/spin_lock.rs diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 35ebe50..a7b7552 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -12,8 +12,9 @@ use crossbeam_utils::thread::scope; use once_cell::unsync::OnceCell; use crate::rt::Reactor; +use crate::sync::Spinlock; use crate::task::Runnable; -use crate::utils::{abort_on_panic, random, Spinlock}; +use crate::utils::{abort_on_panic, random}; thread_local! { /// A reference to the current machine, if the current thread runs tasks. diff --git a/src/sync/mod.rs b/src/sync/mod.rs index c221165..caaf7f7 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -178,9 +178,11 @@ pub use std::sync::{Arc, Weak}; pub use mutex::{Mutex, MutexGuard}; pub use rwlock::{RwLock, RwLockReadGuard, RwLockWriteGuard}; +pub(crate) use spin_lock::Spinlock; mod mutex; mod rwlock; +mod spin_lock; cfg_unstable! { pub use barrier::{Barrier, BarrierWaitResult}; diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs new file mode 100644 index 0000000..39dab78 --- /dev/null +++ b/src/sync/spin_lock.rs @@ -0,0 +1,96 @@ +use std::cell::UnsafeCell; +use std::ops::{Deref, DerefMut}; +use std::sync::atomic::{AtomicBool, Ordering}; + +use crossbeam_utils::Backoff; + +/// A simple spinlock. +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::sync::{Arc, Spinlock}; +/// use async_std::task; +/// +/// let m = Arc::new(Spinlock::new(0)); +/// let mut tasks = vec![]; +/// +/// for _ in 0..10 { +/// let m = m.clone(); +/// tasks.push(task::spawn(async move { +/// *m.lock() += 1; +/// })); +/// } +/// +/// for t in tasks { +/// t.await; +/// } +/// assert_eq!(*m.lock(), 10); +/// # +/// # }) +/// ``` +#[derive(Debug)] +pub struct Spinlock { + flag: AtomicBool, + value: UnsafeCell, +} + +unsafe impl Send for Spinlock {} +unsafe impl Sync for Spinlock {} + +impl Spinlock { + /// Returns a new spinlock initialized with `value`. + pub const fn new(value: T) -> Spinlock { + Spinlock { + flag: AtomicBool::new(false), + value: UnsafeCell::new(value), + } + } + + /// Locks the spinlock. + pub fn lock(&self) -> SpinlockGuard<'_, T> { + let backoff = Backoff::new(); + while self.flag.swap(true, Ordering::Acquire) { + backoff.snooze(); + } + SpinlockGuard { parent: self } + } + + /// Attempts to lock the spinlock. + pub fn try_lock(&self) -> Option> { + if self.flag.swap(true, Ordering::Acquire) { + None + } else { + Some(SpinlockGuard { parent: self }) + } + } +} + +/// A guard holding a spinlock locked. +#[derive(Debug)] +pub struct SpinlockGuard<'a, T> { + parent: &'a Spinlock, +} + +unsafe impl Send for SpinlockGuard<'_, T> {} +unsafe impl Sync for SpinlockGuard<'_, T> {} + +impl<'a, T> Drop for SpinlockGuard<'a, T> { + fn drop(&mut self) { + self.parent.flag.store(false, Ordering::Release); + } +} + +impl<'a, T> Deref for SpinlockGuard<'a, T> { + type Target = T; + + fn deref(&self) -> &T { + unsafe { &*self.parent.value.get() } + } +} + +impl<'a, T> DerefMut for SpinlockGuard<'a, T> { + fn deref_mut(&mut self) -> &mut T { + unsafe { &mut *self.parent.value.get() } + } +} diff --git a/src/utils.rs b/src/utils.rs index 271a857..4bdbd92 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -293,76 +293,3 @@ macro_rules! extension_trait { extension_trait!($($tail)*); }; } - -cfg_default! { - use std::cell::UnsafeCell; - use std::ops::{Deref, DerefMut}; - use std::sync::atomic::{AtomicBool, Ordering}; - - use crossbeam_utils::Backoff; - - /// A simple spinlock. - pub struct Spinlock { - flag: AtomicBool, - value: UnsafeCell, - } - - unsafe impl Send for Spinlock {} - unsafe impl Sync for Spinlock {} - - impl Spinlock { - /// Returns a new spinlock initialized with `value`. - pub const fn new(value: T) -> Spinlock { - Spinlock { - flag: AtomicBool::new(false), - value: UnsafeCell::new(value), - } - } - - /// Locks the spinlock. - pub fn lock(&self) -> SpinlockGuard<'_, T> { - let backoff = Backoff::new(); - while self.flag.swap(true, Ordering::Acquire) { - backoff.snooze(); - } - SpinlockGuard { parent: self } - } - - /// Attempts to lock the spinlock. - pub fn try_lock(&self) -> Option> { - if self.flag.swap(true, Ordering::Acquire) { - None - } else { - Some(SpinlockGuard { parent: self }) - } - } - } - - /// A guard holding a spinlock locked. - pub struct SpinlockGuard<'a, T> { - parent: &'a Spinlock, - } - - unsafe impl Send for SpinlockGuard<'_, T> {} - unsafe impl Sync for SpinlockGuard<'_, T> {} - - impl<'a, T> Drop for SpinlockGuard<'a, T> { - fn drop(&mut self) { - self.parent.flag.store(false, Ordering::Release); - } - } - - impl<'a, T> Deref for SpinlockGuard<'a, T> { - type Target = T; - - fn deref(&self) -> &T { - unsafe { &*self.parent.value.get() } - } - } - - impl<'a, T> DerefMut for SpinlockGuard<'a, T> { - fn deref_mut(&mut self) -> &mut T { - unsafe { &mut *self.parent.value.get() } - } - } -} From 2b44c1be2eb82c30f5b44ba253c886caaef02979 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 19 Mar 2020 18:41:00 +0900 Subject: [PATCH 07/60] refactor: swap to swap_and_compare --- src/sync/spin_lock.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs index 39dab78..55343f1 100644 --- a/src/sync/spin_lock.rs +++ b/src/sync/spin_lock.rs @@ -31,7 +31,7 @@ use crossbeam_utils::Backoff; /// ``` #[derive(Debug)] pub struct Spinlock { - flag: AtomicBool, + locked: AtomicBool, value: UnsafeCell, } @@ -42,7 +42,7 @@ impl Spinlock { /// Returns a new spinlock initialized with `value`. pub const fn new(value: T) -> Spinlock { Spinlock { - flag: AtomicBool::new(false), + locked: AtomicBool::new(false), value: UnsafeCell::new(value), } } @@ -50,7 +50,7 @@ impl Spinlock { /// Locks the spinlock. pub fn lock(&self) -> SpinlockGuard<'_, T> { let backoff = Backoff::new(); - while self.flag.swap(true, Ordering::Acquire) { + while self.locked.compare_and_swap(false, true, Ordering::Acquire) { backoff.snooze(); } SpinlockGuard { parent: self } @@ -58,7 +58,7 @@ impl Spinlock { /// Attempts to lock the spinlock. pub fn try_lock(&self) -> Option> { - if self.flag.swap(true, Ordering::Acquire) { + if self.locked.swap(true, Ordering::Acquire) { None } else { Some(SpinlockGuard { parent: self }) @@ -77,7 +77,7 @@ unsafe impl Sync for SpinlockGuard<'_, T> {} impl<'a, T> Drop for SpinlockGuard<'a, T> { fn drop(&mut self) { - self.parent.flag.store(false, Ordering::Release); + self.parent.locked.store(false, Ordering::Release); } } From d7ee29a03f4ffb89f722e859d10343f6db5d7d7e Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 19 Mar 2020 19:16:12 +0900 Subject: [PATCH 08/60] fix test code --- src/sync/spin_lock.rs | 48 +++++++++++++++++++++---------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs index 55343f1..48ad15d 100644 --- a/src/sync/spin_lock.rs +++ b/src/sync/spin_lock.rs @@ -5,30 +5,6 @@ use std::sync::atomic::{AtomicBool, Ordering}; use crossbeam_utils::Backoff; /// A simple spinlock. -/// -/// ``` -/// # async_std::task::block_on(async { -/// # -/// use async_std::sync::{Arc, Spinlock}; -/// use async_std::task; -/// -/// let m = Arc::new(Spinlock::new(0)); -/// let mut tasks = vec![]; -/// -/// for _ in 0..10 { -/// let m = m.clone(); -/// tasks.push(task::spawn(async move { -/// *m.lock() += 1; -/// })); -/// } -/// -/// for t in tasks { -/// t.await; -/// } -/// assert_eq!(*m.lock(), 10); -/// # -/// # }) -/// ``` #[derive(Debug)] pub struct Spinlock { locked: AtomicBool, @@ -94,3 +70,27 @@ impl<'a, T> DerefMut for SpinlockGuard<'a, T> { unsafe { &mut *self.parent.value.get() } } } + +#[test] +fn spinlock() { + crate::task::block_on(async { + use crate::sync::{Arc, Spinlock}; + use crate::task; + + let m = Arc::new(Spinlock::new(0)); + let mut tasks = vec![]; + + for _ in 0..10 { + let m = m.clone(); + tasks.push(task::spawn(async move { + *m.lock() += 1; + })); + } + + for t in tasks { + t.await; + } + assert_eq!(*m.lock(), 10); + + }) +} From 24c5dbf949e60252f4010714a81174d958d03d48 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Fri, 20 Mar 2020 23:10:45 +0900 Subject: [PATCH 09/60] Remove scheduler state --- src/rt/runtime.rs | 252 ++++++++++-------------------------------- src/sync/mod.rs | 2 - src/sync/spin_lock.rs | 96 ---------------- 3 files changed, 58 insertions(+), 292 deletions(-) delete mode 100644 src/sync/spin_lock.rs diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index a7b7552..3a08bb6 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -1,9 +1,8 @@ use std::cell::Cell; use std::io; use std::iter; -use std::ptr; -use std::sync::atomic::{self, AtomicBool, Ordering}; -use std::sync::{Arc, Mutex}; +use std::sync::atomic::{self, Ordering}; +use std::sync::Arc; use std::thread; use std::time::Duration; @@ -12,7 +11,6 @@ use crossbeam_utils::thread::scope; use once_cell::unsync::OnceCell; use crate::rt::Reactor; -use crate::sync::Spinlock; use crate::task::Runnable; use crate::utils::{abort_on_panic, random}; @@ -24,21 +22,6 @@ thread_local! { static YIELD_NOW: Cell = Cell::new(false); } -/// Scheduler state. -struct Scheduler { - /// Set to `true` every time before a machine blocks polling the reactor. - progress: bool, - - /// Set to `true` while a machine is polling the reactor. - polling: bool, - - /// Idle processors. - processors: Vec, - - /// Running machines. - machines: Vec>, -} - /// An async runtime. pub struct Runtime { /// The reactor. @@ -50,8 +33,7 @@ pub struct Runtime { /// Handles to local queues for stealing work. stealers: Vec>, - /// The scheduler state. - sched: Mutex, + machines: Vec>, } impl Runtime { @@ -59,18 +41,22 @@ impl Runtime { pub fn new() -> Runtime { let cpus = num_cpus::get().max(1); let processors: Vec<_> = (0..cpus).map(|_| Processor::new()).collect(); - let stealers = processors.iter().map(|p| p.worker.stealer()).collect(); + + let machines: Vec<_> = processors + .into_iter() + .map(|p| Arc::new(Machine::new(p))) + .collect(); + + let stealers = machines + .iter() + .map(|m| m.processor.worker.stealer()) + .collect(); Runtime { reactor: Reactor::new().unwrap(), injector: Injector::new(), stealers, - sched: Mutex::new(Scheduler { - processors, - machines: Vec::new(), - progress: false, - polling: false, - }), + machines, } } @@ -102,74 +88,21 @@ impl Runtime { /// Runs the runtime on the current thread. pub fn run(&self) { scope(|s| { - let mut idle = 0; - let mut delay = 0; - - loop { - // Get a list of new machines to start, if any need to be started. - for m in self.make_machines() { - idle = 0; - - s.builder() - .name("async-std/machine".to_string()) - .spawn(move |_| { - abort_on_panic(|| { - let _ = MACHINE.with(|machine| machine.set(m.clone())); - m.run(self); - }) + for m in &self.machines { + s.builder() + .name("async-std/machine".to_string()) + .spawn(move |_| { + abort_on_panic(|| { + let _ = MACHINE.with(|machine| machine.set(m.clone())); + m.run(self); }) - .expect("cannot start a machine thread"); - } - - // Sleep for a bit longer if the scheduler state hasn't changed in a while. - if idle > 10 { - delay = (delay * 2).min(10_000); - } else { - idle += 1; - delay = 1000; - } - - thread::sleep(Duration::from_micros(delay)); + }) + .expect("cannot start a machine thread"); } }) .unwrap(); } - /// Returns a list of machines that need to be started. - fn make_machines(&self) -> Vec> { - let mut sched = self.sched.lock().unwrap(); - let mut to_start = Vec::new(); - - // If there is a machine that is stuck on a task and not making any progress, steal its - // processor and set up a new machine to take over. - for m in &mut sched.machines { - if !m.progress.swap(false, Ordering::SeqCst) { - let opt_p = m.processor.try_lock().and_then(|mut p| p.take()); - - if let Some(p) = opt_p { - *m = Arc::new(Machine::new(p)); - to_start.push(m.clone()); - } - } - } - - // If no machine has been polling the reactor in a while, that means the runtime is - // overloaded with work and we need to start another machine. - if !sched.polling { - if !sched.progress { - if let Some(p) = sched.processors.pop() { - let m = Arc::new(Machine::new(p)); - to_start.push(m.clone()); - sched.machines.push(m); - } - } - - sched.progress = false; - } - - to_start - } - /// Unparks a thread polling the reactor. fn notify(&self) { atomic::fence(Ordering::SeqCst); @@ -183,42 +116,28 @@ impl Runtime { /// This function might not poll the reactor at all so do not rely on it doing anything. Only /// use for optimization. fn quick_poll(&self) -> io::Result { - if let Ok(sched) = self.sched.try_lock() { - if !sched.polling { - return self.reactor.poll(Some(Duration::from_secs(0))); - } - } - Ok(false) + return self.reactor.poll(Some(Duration::from_secs(0))); } } /// A thread running a processor. struct Machine { /// Holds the processor until it gets stolen. - processor: Spinlock>, - - /// Gets set to `true` before running every task to indicate the machine is not stuck. - progress: AtomicBool, + processor: Processor, } +unsafe impl Send for Machine {} +unsafe impl Sync for Machine {} + impl Machine { /// Creates a new machine running a processor. fn new(p: Processor) -> Machine { - Machine { - processor: Spinlock::new(Some(p)), - progress: AtomicBool::new(true), - } + Machine { processor: p } } /// Schedules a task onto the machine. fn schedule(&self, rt: &Runtime, task: Runnable) { - match self.processor.lock().as_mut() { - None => { - rt.injector.push(task); - rt.notify(); - } - Some(p) => p.schedule(rt, task), - } + self.processor.schedule(rt, task); } /// Finds the next runnable task. @@ -226,16 +145,14 @@ impl Machine { let mut retry = false; // First try finding a task in the local queue or in the global queue. - if let Some(p) = self.processor.lock().as_mut() { - if let Some(task) = p.pop_task() { - return Steal::Success(task); - } + if let Some(task) = self.processor.pop_task() { + return Steal::Success(task); + } - match p.steal_from_global(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), - } + match self.processor.steal_from_global(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), } // Try polling the reactor, but don't block on it. @@ -243,18 +160,16 @@ impl Machine { // Try finding a task in the local queue, which might hold tasks woken by the reactor. If // the local queue is still empty, try stealing from other processors. - if let Some(p) = self.processor.lock().as_mut() { - if progress { - if let Some(task) = p.pop_task() { - return Steal::Success(task); - } + if progress { + if let Some(task) = self.processor.pop_task() { + return Steal::Success(task); } + } - match p.steal_from_others(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), - } + match self.processor.steal_from_others(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), } if retry { Steal::Retry } else { Steal::Empty } @@ -275,15 +190,10 @@ impl Machine { let mut fails = 0; loop { - // let the scheduler know this machine is making progress. - self.progress.store(true, Ordering::SeqCst); - // Check if `task::yield_now()` was invoked and flush the slot if so. YIELD_NOW.with(|flag| { if flag.replace(false) { - if let Some(p) = self.processor.lock().as_mut() { - p.flush_slot(rt); - } + self.processor.flush_slot(rt); } }); @@ -294,13 +204,11 @@ impl Machine { runs = 0; rt.quick_poll().unwrap(); - if let Some(p) = self.processor.lock().as_mut() { - if let Steal::Success(task) = p.steal_from_global(rt) { - p.schedule(rt, task); - } - - p.flush_slot(rt); + if let Steal::Success(task) = self.processor.steal_from_global(rt) { + self.processor.schedule(rt, task); } + + self.processor.flush_slot(rt); } // Try to find a runnable task. @@ -313,11 +221,6 @@ impl Machine { fails += 1; - // Check if the processor was stolen. - if self.processor.lock().is_none() { - break; - } - // Yield the current thread a few times. if fails <= YIELDS { thread::yield_now(); @@ -326,14 +229,10 @@ impl Machine { // Put the current thread to sleep a few times. if fails <= YIELDS + SLEEPS { - let opt_p = self.processor.lock().take(); thread::sleep(Duration::from_micros(10)); - *self.processor.lock() = opt_p; continue; } - let mut sched = rt.sched.lock().unwrap(); - // One final check for available tasks while the scheduler is locked. if let Some(task) = iter::repeat_with(|| self.find_task(rt)) .find(|s| !s.is_retry()) @@ -343,46 +242,11 @@ impl Machine { continue; } - // If another thread is already blocked on the reactor, there is no point in keeping - // the current thread around since there is too little work to do. - if sched.polling { - break; - } - - // Take out the machine associated with the current thread. - let m = match sched - .machines - .iter() - .position(|elem| ptr::eq(&**elem, self)) - { - None => break, // The processor was stolen. - Some(pos) => sched.machines.swap_remove(pos), - }; - - // Unlock the schedule poll the reactor until new I/O events arrive. - sched.polling = true; - drop(sched); rt.reactor.poll(None).unwrap(); - // Lock the scheduler again and re-register the machine. - sched = rt.sched.lock().unwrap(); - sched.polling = false; - sched.machines.push(m); - sched.progress = true; - runs = 0; fails = 0; } - - // When shutting down the thread, take the processor out if still available. - let opt_p = self.processor.lock().take(); - - // Return the processor to the scheduler and remove the machine. - if let Some(p) = opt_p { - let mut sched = rt.sched.lock().unwrap(); - sched.processors.push(p); - sched.machines.retain(|elem| !ptr::eq(&**elem, self)); - } } } @@ -391,7 +255,7 @@ struct Processor { worker: Worker, /// Contains the next task to run as an optimization that skips the queue. - slot: Option, + slot: Cell>, } impl Processor { @@ -399,13 +263,13 @@ impl Processor { fn new() -> Processor { Processor { worker: Worker::new_fifo(), - slot: None, + slot: Cell::new(None), } } /// Schedules a task to run on this processor. - fn schedule(&mut self, rt: &Runtime, task: Runnable) { - match self.slot.replace(task) { + fn schedule(&self, rt: &Runtime, task: Runnable) { + match self.slot.replace(Some(task)) { None => {} Some(task) => { self.worker.push(task); @@ -415,7 +279,7 @@ impl Processor { } /// Flushes a task from the slot into the local queue. - fn flush_slot(&mut self, rt: &Runtime) { + fn flush_slot(&self, rt: &Runtime) { if let Some(task) = self.slot.take() { self.worker.push(task); rt.notify(); @@ -423,17 +287,17 @@ impl Processor { } /// Pops a task from this processor. - fn pop_task(&mut self) -> Option { + fn pop_task(&self) -> Option { self.slot.take().or_else(|| self.worker.pop()) } /// Steals a task from the global queue. - fn steal_from_global(&mut self, rt: &Runtime) -> Steal { + fn steal_from_global(&self, rt: &Runtime) -> Steal { rt.injector.steal_batch_and_pop(&self.worker) } /// Steals a task from other processors. - fn steal_from_others(&mut self, rt: &Runtime) -> Steal { + fn steal_from_others(&self, rt: &Runtime) -> Steal { // Pick a random starting point in the list of queues. let len = rt.stealers.len(); let start = random(len as u32) as usize; diff --git a/src/sync/mod.rs b/src/sync/mod.rs index caaf7f7..c221165 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -178,11 +178,9 @@ pub use std::sync::{Arc, Weak}; pub use mutex::{Mutex, MutexGuard}; pub use rwlock::{RwLock, RwLockReadGuard, RwLockWriteGuard}; -pub(crate) use spin_lock::Spinlock; mod mutex; mod rwlock; -mod spin_lock; cfg_unstable! { pub use barrier::{Barrier, BarrierWaitResult}; diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs deleted file mode 100644 index 48ad15d..0000000 --- a/src/sync/spin_lock.rs +++ /dev/null @@ -1,96 +0,0 @@ -use std::cell::UnsafeCell; -use std::ops::{Deref, DerefMut}; -use std::sync::atomic::{AtomicBool, Ordering}; - -use crossbeam_utils::Backoff; - -/// A simple spinlock. -#[derive(Debug)] -pub struct Spinlock { - locked: AtomicBool, - value: UnsafeCell, -} - -unsafe impl Send for Spinlock {} -unsafe impl Sync for Spinlock {} - -impl Spinlock { - /// Returns a new spinlock initialized with `value`. - pub const fn new(value: T) -> Spinlock { - Spinlock { - locked: AtomicBool::new(false), - value: UnsafeCell::new(value), - } - } - - /// Locks the spinlock. - pub fn lock(&self) -> SpinlockGuard<'_, T> { - let backoff = Backoff::new(); - while self.locked.compare_and_swap(false, true, Ordering::Acquire) { - backoff.snooze(); - } - SpinlockGuard { parent: self } - } - - /// Attempts to lock the spinlock. - pub fn try_lock(&self) -> Option> { - if self.locked.swap(true, Ordering::Acquire) { - None - } else { - Some(SpinlockGuard { parent: self }) - } - } -} - -/// A guard holding a spinlock locked. -#[derive(Debug)] -pub struct SpinlockGuard<'a, T> { - parent: &'a Spinlock, -} - -unsafe impl Send for SpinlockGuard<'_, T> {} -unsafe impl Sync for SpinlockGuard<'_, T> {} - -impl<'a, T> Drop for SpinlockGuard<'a, T> { - fn drop(&mut self) { - self.parent.locked.store(false, Ordering::Release); - } -} - -impl<'a, T> Deref for SpinlockGuard<'a, T> { - type Target = T; - - fn deref(&self) -> &T { - unsafe { &*self.parent.value.get() } - } -} - -impl<'a, T> DerefMut for SpinlockGuard<'a, T> { - fn deref_mut(&mut self) -> &mut T { - unsafe { &mut *self.parent.value.get() } - } -} - -#[test] -fn spinlock() { - crate::task::block_on(async { - use crate::sync::{Arc, Spinlock}; - use crate::task; - - let m = Arc::new(Spinlock::new(0)); - let mut tasks = vec![]; - - for _ in 0..10 { - let m = m.clone(); - tasks.push(task::spawn(async move { - *m.lock() += 1; - })); - } - - for t in tasks { - t.await; - } - assert_eq!(*m.lock(), 10); - - }) -} From f9607768461242b129454256ec4528c680c9a946 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 21 Mar 2020 13:37:37 +0900 Subject: [PATCH 10/60] fix --- src/rt/runtime.rs | 34 +++++++++++++++++++++++++++++++--- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 3a08bb6..8f84f54 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -2,7 +2,7 @@ use std::cell::Cell; use std::io; use std::iter; use std::sync::atomic::{self, Ordering}; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use std::thread; use std::time::Duration; @@ -22,6 +22,11 @@ thread_local! { static YIELD_NOW: Cell = Cell::new(false); } +struct Scheduler { + /// Set to `true` while a machine is polling the reactor. + polling: bool, +} + /// An async runtime. pub struct Runtime { /// The reactor. @@ -33,7 +38,11 @@ pub struct Runtime { /// Handles to local queues for stealing work. stealers: Vec>, + /// Machines to start machines: Vec>, + + /// The scheduler state. + sched: Mutex, } impl Runtime { @@ -57,6 +66,7 @@ impl Runtime { injector: Injector::new(), stealers, machines, + sched: Mutex::new(Scheduler { polling: false }), } } @@ -116,7 +126,25 @@ impl Runtime { /// This function might not poll the reactor at all so do not rely on it doing anything. Only /// use for optimization. fn quick_poll(&self) -> io::Result { - return self.reactor.poll(Some(Duration::from_secs(0))); + if let Ok(sched) = self.sched.try_lock() { + if !sched.polling { + return self.reactor.poll(Some(Duration::from_secs(0))); + } + } + Ok(false) + } + + fn poll(&self) -> io::Result { + let mut sched = self.sched.lock().unwrap(); + sched.polling = true; + drop(sched); + + let result = self.reactor.poll(None); + + let mut sched = self.sched.lock().unwrap(); + sched.polling = false; + + result } } @@ -242,7 +270,7 @@ impl Machine { continue; } - rt.reactor.poll(None).unwrap(); + rt.poll().unwrap(); runs = 0; fails = 0; From 6d3ca5a06f6b9a15cb03b9059540867f18c5266f Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 21 Mar 2020 14:19:38 +0900 Subject: [PATCH 11/60] remove poll function --- src/rt/runtime.rs | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 8f84f54..3e810d0 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -133,19 +133,6 @@ impl Runtime { } Ok(false) } - - fn poll(&self) -> io::Result { - let mut sched = self.sched.lock().unwrap(); - sched.polling = true; - drop(sched); - - let result = self.reactor.poll(None); - - let mut sched = self.sched.lock().unwrap(); - sched.polling = false; - - result - } } /// A thread running a processor. @@ -270,7 +257,20 @@ impl Machine { continue; } - rt.poll().unwrap(); + let mut sched = rt.sched.lock().unwrap(); + + if sched.polling { + thread::sleep(Duration::from_micros(10)); + continue; + } + + sched.polling = true; + drop(sched); + + rt.reactor.poll(None).unwrap(); + + let mut sched = rt.sched.lock().unwrap(); + sched.polling = false; runs = 0; fails = 0; From 57c648cf01b6a0b577a32a29eb87ba1a01d4e838 Mon Sep 17 00:00:00 2001 From: sunli Date: Sat, 21 Mar 2020 15:49:15 +0800 Subject: [PATCH 12/60] Add async-graphql to the ecosystems inside the readme --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 4ebf592..69ed7ea 100644 --- a/README.md +++ b/README.md @@ -136,6 +136,8 @@ documentation] on how to enable them. * [Xactor](https://crates.io/crates/xactor) — Xactor is a rust actors framework based on async-std. + * [async-graphql](https://crates.io/crates/async-graphql) — The GraphQL server library implemented by rust, fully support async/await. + ## License From cfaec2aa9506dc555870b0bcc501f65c51b88ac2 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sun, 22 Mar 2020 19:19:17 +0900 Subject: [PATCH 13/60] re add spin_lock --- src/sync/mod.rs | 2 + src/sync/spin_lock.rs | 89 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 91 insertions(+) create mode 100644 src/sync/spin_lock.rs diff --git a/src/sync/mod.rs b/src/sync/mod.rs index c221165..b9162f1 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -192,3 +192,5 @@ cfg_unstable! { pub(crate) mod waker_set; pub(crate) use waker_set::WakerSet; +pub(crate) mod spin_lock; +pub(crate) use spin_lock::Spinlock; diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs new file mode 100644 index 0000000..854b7e0 --- /dev/null +++ b/src/sync/spin_lock.rs @@ -0,0 +1,89 @@ +use std::cell::UnsafeCell; +use std::ops::{Deref, DerefMut}; +use std::sync::atomic::{AtomicBool, Ordering}; + +use crossbeam_utils::Backoff; + +/// A simple spinlock. +#[derive(Debug)] +pub struct Spinlock { + locked: AtomicBool, + value: UnsafeCell, +} + +unsafe impl Send for Spinlock {} +unsafe impl Sync for Spinlock {} + +impl Spinlock { + /// Returns a new spinlock initialized with `value`. + pub const fn new(value: T) -> Spinlock { + Spinlock { + locked: AtomicBool::new(false), + value: UnsafeCell::new(value), + } + } + + /// Locks the spinlock. + pub fn lock(&self) -> SpinlockGuard<'_, T> { + let backoff = Backoff::new(); + while self.locked.compare_and_swap(false, true, Ordering::Acquire) { + backoff.snooze(); + } + SpinlockGuard { parent: self } + } +} + +/// A guard holding a spinlock locked. +#[derive(Debug)] +pub struct SpinlockGuard<'a, T> { + parent: &'a Spinlock, +} + +unsafe impl Send for SpinlockGuard<'_, T> {} +unsafe impl Sync for SpinlockGuard<'_, T> {} + +impl<'a, T> Drop for SpinlockGuard<'a, T> { + fn drop(&mut self) { + self.parent.locked.store(false, Ordering::Release); + } +} + +impl<'a, T> Deref for SpinlockGuard<'a, T> { + type Target = T; + + fn deref(&self) -> &T { + unsafe { &*self.parent.value.get() } + } +} + +impl<'a, T> DerefMut for SpinlockGuard<'a, T> { + fn deref_mut(&mut self) -> &mut T { + unsafe { &mut *self.parent.value.get() } + } +} + +#[test] +fn spinlock() { + use std::sync::Arc; + + use crate::sync::{Spinlock}; + use crate::task; + + task::block_on(async { + + let m = Arc::new(Spinlock::new(0)); + let mut tasks = vec![]; + + for _ in 0..10 { + let m = m.clone(); + tasks.push(task::spawn(async move { + *m.lock() += 1; + })); + } + + for t in tasks { + t.await; + } + assert_eq!(*m.lock(), 10); + }) +} From 322911142cdd3ed9315f73da90b41ea8bb26d0b5 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sun, 22 Mar 2020 19:20:01 +0900 Subject: [PATCH 14/60] lock processor and remove unsafe Send, Sync --- src/rt/runtime.rs | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 3e810d0..a832b9f 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -11,6 +11,7 @@ use crossbeam_utils::thread::scope; use once_cell::unsync::OnceCell; use crate::rt::Reactor; +use crate::sync::Spinlock; use crate::task::Runnable; use crate::utils::{abort_on_panic, random}; @@ -58,7 +59,7 @@ impl Runtime { let stealers = machines .iter() - .map(|m| m.processor.worker.stealer()) + .map(|m| m.processor.lock().worker.stealer()) .collect(); Runtime { @@ -138,21 +139,20 @@ impl Runtime { /// A thread running a processor. struct Machine { /// Holds the processor until it gets stolen. - processor: Processor, + processor: Spinlock, } -unsafe impl Send for Machine {} -unsafe impl Sync for Machine {} - impl Machine { /// Creates a new machine running a processor. fn new(p: Processor) -> Machine { - Machine { processor: p } + Machine { + processor: Spinlock::new(p), + } } /// Schedules a task onto the machine. fn schedule(&self, rt: &Runtime, task: Runnable) { - self.processor.schedule(rt, task); + self.processor.lock().schedule(rt, task); } /// Finds the next runnable task. @@ -160,11 +160,11 @@ impl Machine { let mut retry = false; // First try finding a task in the local queue or in the global queue. - if let Some(task) = self.processor.pop_task() { + if let Some(task) = self.processor.lock().pop_task() { return Steal::Success(task); } - match self.processor.steal_from_global(rt) { + match self.processor.lock().steal_from_global(rt) { Steal::Empty => {} Steal::Retry => retry = true, Steal::Success(task) => return Steal::Success(task), @@ -176,12 +176,12 @@ impl Machine { // Try finding a task in the local queue, which might hold tasks woken by the reactor. If // the local queue is still empty, try stealing from other processors. if progress { - if let Some(task) = self.processor.pop_task() { + if let Some(task) = self.processor.lock().pop_task() { return Steal::Success(task); } } - match self.processor.steal_from_others(rt) { + match self.processor.lock().steal_from_others(rt) { Steal::Empty => {} Steal::Retry => retry = true, Steal::Success(task) => return Steal::Success(task), @@ -208,7 +208,7 @@ impl Machine { // Check if `task::yield_now()` was invoked and flush the slot if so. YIELD_NOW.with(|flag| { if flag.replace(false) { - self.processor.flush_slot(rt); + self.processor.lock().flush_slot(rt); } }); @@ -219,11 +219,12 @@ impl Machine { runs = 0; rt.quick_poll().unwrap(); - if let Steal::Success(task) = self.processor.steal_from_global(rt) { - self.processor.schedule(rt, task); + let p = self.processor.lock(); + if let Steal::Success(task) = p.steal_from_global(rt) { + p.schedule(rt, task); } - self.processor.flush_slot(rt); + p.flush_slot(rt); } // Try to find a runnable task. From 11ee2a89856dd84e08ced0371d64e8c8cb7af97b Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sun, 22 Mar 2020 19:25:40 +0900 Subject: [PATCH 15/60] fix --- src/sync/mod.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/sync/mod.rs b/src/sync/mod.rs index b9162f1..82759fb 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -192,5 +192,8 @@ cfg_unstable! { pub(crate) mod waker_set; pub(crate) use waker_set::WakerSet; -pub(crate) mod spin_lock; -pub(crate) use spin_lock::Spinlock; + +cfg_default! { + pub(crate) mod spin_lock; + pub(crate) use spin_lock::Spinlock; +} From 68fa054517bcbe2d5f2e16753ab5d0887d38f6cd Mon Sep 17 00:00:00 2001 From: Devashish Dixit Date: Mon, 30 Mar 2020 17:44:56 +0800 Subject: [PATCH 16/60] Update futures-timer to 3.0.2 --- Cargo.toml | 2 +- src/stream/interval.rs | 123 +--------------------------------- src/stream/stream/throttle.rs | 4 +- 3 files changed, 6 insertions(+), 123 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 37374f7..d4a5ad7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -61,7 +61,7 @@ crossbeam-deque = { version = "0.7.3", optional = true } crossbeam-utils = { version = "0.7.2", optional = true } futures-core = { version = "0.3.4", optional = true, default-features = false } futures-io = { version = "0.3.4", optional = true } -futures-timer = { version = "2.0.2", optional = true } +futures-timer = { version = "3.0.2", optional = true } kv-log-macro = { version = "1.0.4", optional = true } log = { version = "0.4.8", features = ["kv_unstable"], optional = true } memchr = { version = "2.3.3", optional = true } diff --git a/src/stream/interval.rs b/src/stream/interval.rs index 7a0c174..be94b06 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -1,6 +1,6 @@ use std::pin::Pin; use std::task::{Context, Poll}; -use std::time::{Duration, Instant}; +use std::time::Duration; use crate::future::Future; use crate::stream::Stream; @@ -71,125 +71,8 @@ impl Stream for Interval { if Pin::new(&mut self.delay).poll(cx).is_pending() { return Poll::Pending; } - let when = Instant::now(); - let next = next_interval(when, Instant::now(), self.interval); - self.delay.reset(next); + let interval = self.interval; + self.delay.reset(interval); Poll::Ready(Some(())) } } - -/// Converts Duration object to raw nanoseconds if possible -/// -/// This is useful to divide intervals. -/// -/// While technically for large duration it's impossible to represent any -/// duration as nanoseconds, the largest duration we can represent is about -/// 427_000 years. Large enough for any interval we would use or calculate in -/// async-std. -fn duration_to_nanos(dur: Duration) -> Option { - dur.as_secs() - .checked_mul(1_000_000_000) - .and_then(|v| v.checked_add(u64::from(dur.subsec_nanos()))) -} - -fn next_interval(prev: Instant, now: Instant, interval: Duration) -> Instant { - let new = prev + interval; - if new > now { - return new; - } - - let spent_ns = duration_to_nanos(now.duration_since(prev)).expect("interval should be expired"); - let interval_ns = - duration_to_nanos(interval).expect("interval is less that 427 thousand years"); - let mult = spent_ns / interval_ns + 1; - assert!( - mult < (1 << 32), - "can't skip more than 4 billion intervals of {:?} \ - (trying to skip {})", - interval, - mult - ); - prev + interval * (mult as u32) -} - -#[cfg(test)] -mod test { - use super::next_interval; - use std::cmp::Ordering; - use std::time::{Duration, Instant}; - - struct Timeline(Instant); - - impl Timeline { - fn new() -> Timeline { - Timeline(Instant::now()) - } - fn at(&self, millis: u64) -> Instant { - self.0 + Duration::from_millis(millis) - } - fn at_ns(&self, sec: u64, nanos: u32) -> Instant { - self.0 + Duration::new(sec, nanos) - } - } - - fn dur(millis: u64) -> Duration { - Duration::from_millis(millis) - } - - // The math around Instant/Duration isn't 100% precise due to rounding - // errors, see #249 for more info - fn almost_eq(a: Instant, b: Instant) -> bool { - match a.cmp(&b) { - Ordering::Equal => true, - Ordering::Greater => a - b < Duration::from_millis(1), - Ordering::Less => b - a < Duration::from_millis(1), - } - } - - #[test] - fn norm_next() { - let tm = Timeline::new(); - assert!(almost_eq( - next_interval(tm.at(1), tm.at(2), dur(10)), - tm.at(11) - )); - assert!(almost_eq( - next_interval(tm.at(7777), tm.at(7788), dur(100)), - tm.at(7877) - )); - assert!(almost_eq( - next_interval(tm.at(1), tm.at(1000), dur(2100)), - tm.at(2101) - )); - } - - #[test] - fn fast_forward() { - let tm = Timeline::new(); - assert!(almost_eq( - next_interval(tm.at(1), tm.at(1000), dur(10)), - tm.at(1001) - )); - assert!(almost_eq( - next_interval(tm.at(7777), tm.at(8888), dur(100)), - tm.at(8977) - )); - assert!(almost_eq( - next_interval(tm.at(1), tm.at(10000), dur(2100)), - tm.at(10501) - )); - } - - /// TODO: this test actually should be successful, but since we can't - /// multiply Duration on anything larger than u32 easily we decided - /// to allow it to fail for now - #[test] - #[should_panic(expected = "can't skip more than 4 billion intervals")] - fn large_skip() { - let tm = Timeline::new(); - assert_eq!( - next_interval(tm.at_ns(0, 1), tm.at_ns(25, 0), Duration::new(0, 2)), - tm.at_ns(25, 1) - ); - } -} diff --git a/src/stream/stream/throttle.rs b/src/stream/stream/throttle.rs index ce8c13b..554ca30 100644 --- a/src/stream/stream/throttle.rs +++ b/src/stream/stream/throttle.rs @@ -1,6 +1,6 @@ use std::future::Future; use std::pin::Pin; -use std::time::{Duration, Instant}; +use std::time::Duration; use futures_timer::Delay; use pin_project_lite::pin_project; @@ -59,7 +59,7 @@ impl Stream for Throttle { Poll::Ready(None) => Poll::Ready(None), Poll::Ready(Some(v)) => { *this.blocked = true; - this.delay.reset(Instant::now() + *this.duration); + this.delay.reset(*this.duration); Poll::Ready(Some(v)) } } From 088aa5662cb3d69bbfefcc134bd7251ef94086ab Mon Sep 17 00:00:00 2001 From: k-nasa Date: Fri, 3 Apr 2020 13:38:07 +0900 Subject: [PATCH 17/60] refactor: Remove wrapping cell --- src/rt/runtime.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index a832b9f..62b85f8 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -219,7 +219,7 @@ impl Machine { runs = 0; rt.quick_poll().unwrap(); - let p = self.processor.lock(); + let mut p = self.processor.lock(); if let Steal::Success(task) = p.steal_from_global(rt) { p.schedule(rt, task); } @@ -284,7 +284,7 @@ struct Processor { worker: Worker, /// Contains the next task to run as an optimization that skips the queue. - slot: Cell>, + slot: Option, } impl Processor { @@ -292,13 +292,13 @@ impl Processor { fn new() -> Processor { Processor { worker: Worker::new_fifo(), - slot: Cell::new(None), + slot: None, } } /// Schedules a task to run on this processor. - fn schedule(&self, rt: &Runtime, task: Runnable) { - match self.slot.replace(Some(task)) { + fn schedule(&mut self, rt: &Runtime, task: Runnable) { + match self.slot.replace(task) { None => {} Some(task) => { self.worker.push(task); @@ -308,7 +308,7 @@ impl Processor { } /// Flushes a task from the slot into the local queue. - fn flush_slot(&self, rt: &Runtime) { + fn flush_slot(&mut self, rt: &Runtime) { if let Some(task) = self.slot.take() { self.worker.push(task); rt.notify(); @@ -316,7 +316,7 @@ impl Processor { } /// Pops a task from this processor. - fn pop_task(&self) -> Option { + fn pop_task(&mut self) -> Option { self.slot.take().or_else(|| self.worker.pop()) } From 0c9a66c1f64b1706f5e31e46fffb4644bfaf1eee Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 9 Apr 2020 17:02:27 +0200 Subject: [PATCH 18/60] fix scheduler loop This now matches more closely the logic as implemented in #631, and fixes the performance regression as far as I have observed. Closes #746 --- src/rt/runtime.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 62b85f8..5a9d302 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -249,6 +249,8 @@ impl Machine { continue; } + let mut sched = rt.sched.lock().unwrap(); + // One final check for available tasks while the scheduler is locked. if let Some(task) = iter::repeat_with(|| self.find_task(rt)) .find(|s| !s.is_retry()) @@ -258,19 +260,19 @@ impl Machine { continue; } - let mut sched = rt.sched.lock().unwrap(); - + // If another thread is already blocked on the reactor, there is no point in keeping + // the current thread around since there is too little work to do. if sched.polling { - thread::sleep(Duration::from_micros(10)); - continue; + break; } + // Unlock the schedule poll the reactor until new I/O events arrive. sched.polling = true; drop(sched); - rt.reactor.poll(None).unwrap(); - let mut sched = rt.sched.lock().unwrap(); + // Lock the scheduler again and re-register the machine. + sched = rt.sched.lock().unwrap(); sched.polling = false; runs = 0; From a4e07e345c496e112169b21bc92ca5c903c818bd Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 10 Apr 2020 02:22:03 +0200 Subject: [PATCH 19/60] fix(rt): bring back dynamic machines Even if we do not make use of the progress blocking, we do need to make use of the dynamic restarting of machines as far as I understand. Keeps the perf, while removing the regression from #747 --- src/rt/runtime.rs | 166 ++++++++++++++++++++++++++++++++-------------- 1 file changed, 118 insertions(+), 48 deletions(-) diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs index 5a9d302..a0d88b9 100644 --- a/src/rt/runtime.rs +++ b/src/rt/runtime.rs @@ -1,6 +1,7 @@ use std::cell::Cell; use std::io; use std::iter; +use std::ptr; use std::sync::atomic::{self, Ordering}; use std::sync::{Arc, Mutex}; use std::thread; @@ -26,6 +27,12 @@ thread_local! { struct Scheduler { /// Set to `true` while a machine is polling the reactor. polling: bool, + + /// Idle processors. + processors: Vec, + + /// Running machines. + machines: Vec>, } /// An async runtime. @@ -39,9 +46,6 @@ pub struct Runtime { /// Handles to local queues for stealing work. stealers: Vec>, - /// Machines to start - machines: Vec>, - /// The scheduler state. sched: Mutex, } @@ -51,23 +55,17 @@ impl Runtime { pub fn new() -> Runtime { let cpus = num_cpus::get().max(1); let processors: Vec<_> = (0..cpus).map(|_| Processor::new()).collect(); - - let machines: Vec<_> = processors - .into_iter() - .map(|p| Arc::new(Machine::new(p))) - .collect(); - - let stealers = machines - .iter() - .map(|m| m.processor.lock().worker.stealer()) - .collect(); + let stealers = processors.iter().map(|p| p.worker.stealer()).collect(); Runtime { reactor: Reactor::new().unwrap(), injector: Injector::new(), stealers, - machines, - sched: Mutex::new(Scheduler { polling: false }), + sched: Mutex::new(Scheduler { + processors, + machines: Vec::new(), + polling: false, + }), } } @@ -99,21 +97,57 @@ impl Runtime { /// Runs the runtime on the current thread. pub fn run(&self) { scope(|s| { - for m in &self.machines { - s.builder() - .name("async-std/machine".to_string()) - .spawn(move |_| { - abort_on_panic(|| { - let _ = MACHINE.with(|machine| machine.set(m.clone())); - m.run(self); + let mut idle = 0; + let mut delay = 0; + + loop { + // Get a list of new machines to start, if any need to be started. + for m in self.make_machines() { + idle = 0; + + s.builder() + .name("async-std/machine".to_string()) + .spawn(move |_| { + abort_on_panic(|| { + let _ = MACHINE.with(|machine| machine.set(m.clone())); + m.run(self); + }) }) - }) - .expect("cannot start a machine thread"); + .expect("cannot start a machine thread"); + } + + // Sleep for a bit longer if the scheduler state hasn't changed in a while. + if idle > 10 { + delay = (delay * 2).min(10_000); + } else { + idle += 1; + delay = 1000; + } + + thread::sleep(Duration::from_micros(delay)); } }) .unwrap(); } + /// Returns a list of machines that need to be started. + fn make_machines(&self) -> Vec> { + let mut sched = self.sched.lock().unwrap(); + let mut to_start = Vec::new(); + + // If no machine has been polling the reactor in a while, that means the runtime is + // overloaded with work and we need to start another machine. + if !sched.polling { + if let Some(p) = sched.processors.pop() { + let m = Arc::new(Machine::new(p)); + to_start.push(m.clone()); + sched.machines.push(m); + } + } + + to_start + } + /// Unparks a thread polling the reactor. fn notify(&self) { atomic::fence(Ordering::SeqCst); @@ -139,20 +173,26 @@ impl Runtime { /// A thread running a processor. struct Machine { /// Holds the processor until it gets stolen. - processor: Spinlock, + processor: Spinlock>, } impl Machine { /// Creates a new machine running a processor. fn new(p: Processor) -> Machine { Machine { - processor: Spinlock::new(p), + processor: Spinlock::new(Some(p)), } } /// Schedules a task onto the machine. fn schedule(&self, rt: &Runtime, task: Runnable) { - self.processor.lock().schedule(rt, task); + match self.processor.lock().as_mut() { + None => { + rt.injector.push(task); + rt.notify(); + } + Some(p) => p.schedule(rt, task), + } } /// Finds the next runnable task. @@ -160,14 +200,16 @@ impl Machine { let mut retry = false; // First try finding a task in the local queue or in the global queue. - if let Some(task) = self.processor.lock().pop_task() { - return Steal::Success(task); - } + if let Some(p) = self.processor.lock().as_mut() { + if let Some(task) = p.pop_task() { + return Steal::Success(task); + } - match self.processor.lock().steal_from_global(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), + match p.steal_from_global(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), + } } // Try polling the reactor, but don't block on it. @@ -175,16 +217,18 @@ impl Machine { // Try finding a task in the local queue, which might hold tasks woken by the reactor. If // the local queue is still empty, try stealing from other processors. - if progress { - if let Some(task) = self.processor.lock().pop_task() { - return Steal::Success(task); + if let Some(p) = self.processor.lock().as_mut() { + if progress { + if let Some(task) = p.pop_task() { + return Steal::Success(task); + } } - } - match self.processor.lock().steal_from_others(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), + match p.steal_from_others(rt) { + Steal::Empty => {} + Steal::Retry => retry = true, + Steal::Success(task) => return Steal::Success(task), + } } if retry { Steal::Retry } else { Steal::Empty } @@ -208,7 +252,9 @@ impl Machine { // Check if `task::yield_now()` was invoked and flush the slot if so. YIELD_NOW.with(|flag| { if flag.replace(false) { - self.processor.lock().flush_slot(rt); + if let Some(p) = self.processor.lock().as_mut() { + p.flush_slot(rt); + } } }); @@ -219,12 +265,13 @@ impl Machine { runs = 0; rt.quick_poll().unwrap(); - let mut p = self.processor.lock(); - if let Steal::Success(task) = p.steal_from_global(rt) { - p.schedule(rt, task); - } + if let Some(p) = self.processor.lock().as_mut() { + if let Steal::Success(task) = p.steal_from_global(rt) { + p.schedule(rt, task); + } - p.flush_slot(rt); + p.flush_slot(rt); + } } // Try to find a runnable task. @@ -245,7 +292,9 @@ impl Machine { // Put the current thread to sleep a few times. if fails <= YIELDS + SLEEPS { + let opt_p = self.processor.lock().take(); thread::sleep(Duration::from_micros(10)); + *self.processor.lock() = opt_p; continue; } @@ -266,6 +315,16 @@ impl Machine { break; } + // Take out the machine associated with the current thread. + let m = match sched + .machines + .iter() + .position(|elem| ptr::eq(&**elem, self)) + { + None => break, // The processor was stolen. + Some(pos) => sched.machines.swap_remove(pos), + }; + // Unlock the schedule poll the reactor until new I/O events arrive. sched.polling = true; drop(sched); @@ -274,10 +333,21 @@ impl Machine { // Lock the scheduler again and re-register the machine. sched = rt.sched.lock().unwrap(); sched.polling = false; + sched.machines.push(m); runs = 0; fails = 0; } + + // When shutting down the thread, take the processor out if still available. + let opt_p = self.processor.lock().take(); + + // Return the processor to the scheduler and remove the machine. + if let Some(p) = opt_p { + let mut sched = rt.sched.lock().unwrap(); + sched.processors.push(p); + sched.machines.retain(|elem| !ptr::eq(&**elem, self)); + } } } From db438abb8fc4c168035ea51ef287d4d72971a41b Mon Sep 17 00:00:00 2001 From: Thayne McCombs Date: Sun, 12 Apr 2020 05:35:18 -0600 Subject: [PATCH 20/60] Implement async_std::sync::Condvar (#369) * Implement async_std::sync::Condvar Part of #217 * More rigourous detection of notification for condvar * Use state of Waker instead of AtomicUsize to keep track of if task was notified. * Add test for notify_all * Implement wait_timeout_until And add warnings about spurious wakeups to wait and wait_timeout * Use WakerSet for Condvar This should also address concerns about spurious wakeups. * Add test for wait_timeout with no lock held * Add comments describing AwaitNotify struct And remove an unnneded comment in a Debug implementation --- src/sync/condvar.rs | 417 ++++++++++++++++++++++++++++++++++++++++++ src/sync/mod.rs | 2 + src/sync/mutex.rs | 5 + src/sync/waker_set.rs | 22 +++ tests/condvar.rs | 91 +++++++++ 5 files changed, 537 insertions(+) create mode 100644 src/sync/condvar.rs create mode 100644 tests/condvar.rs diff --git a/src/sync/condvar.rs b/src/sync/condvar.rs new file mode 100644 index 0000000..67507f3 --- /dev/null +++ b/src/sync/condvar.rs @@ -0,0 +1,417 @@ +use std::fmt; +use std::pin::Pin; +use std::time::Duration; + +use super::mutex::{guard_lock, MutexGuard}; +use crate::future::{timeout, Future}; +use crate::sync::WakerSet; +use crate::task::{Context, Poll}; + +#[derive(Debug, PartialEq, Eq, Copy, Clone)] +pub struct WaitTimeoutResult(bool); + +/// A type indicating whether a timed wait on a condition variable returned due to a time out or +/// not +impl WaitTimeoutResult { + /// Returns `true` if the wait was known to have timed out. + pub fn timed_out(self) -> bool { + self.0 + } +} + +/// A Condition Variable +/// +/// This type is an async version of [`std::sync::Mutex`]. +/// +/// [`std::sync::Condvar`]: https://doc.rust-lang.org/std/sync/struct.Condvar.html +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use std::sync::Arc; +/// +/// use async_std::sync::{Mutex, Condvar}; +/// use async_std::task; +/// +/// let pair = Arc::new((Mutex::new(false), Condvar::new())); +/// let pair2 = pair.clone(); +/// +/// // Inside of our lock, spawn a new thread, and then wait for it to start. +/// task::spawn(async move { +/// let (lock, cvar) = &*pair2; +/// let mut started = lock.lock().await; +/// *started = true; +/// // We notify the condvar that the value has changed. +/// cvar.notify_one(); +/// }); +/// +/// // Wait for the thread to start up. +/// let (lock, cvar) = &*pair; +/// let mut started = lock.lock().await; +/// while !*started { +/// started = cvar.wait(started).await; +/// } +/// +/// # }) +/// ``` +pub struct Condvar { + wakers: WakerSet, +} + +unsafe impl Send for Condvar {} +unsafe impl Sync for Condvar {} + +impl Default for Condvar { + fn default() -> Self { + Condvar::new() + } +} + +impl Condvar { + /// Creates a new condition variable + /// + /// # Examples + /// + /// ``` + /// use async_std::sync::Condvar; + /// + /// let cvar = Condvar::new(); + /// ``` + pub fn new() -> Self { + Condvar { + wakers: WakerSet::new(), + } + } + + /// Blocks the current task until this condition variable receives a notification. + /// + /// Unlike the std equivalent, this does not check that a single mutex is used at runtime. + /// However, as a best practice avoid using with multiple mutexes. + /// + /// # Examples + /// + /// ``` + /// # async_std::task::block_on(async { + /// use std::sync::Arc; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_one(); + /// }); + /// + /// // Wait for the thread to start up. + /// let (lock, cvar) = &*pair; + /// let mut started = lock.lock().await; + /// while !*started { + /// started = cvar.wait(started).await; + /// } + /// # }) + /// ``` + #[allow(clippy::needless_lifetimes)] + pub async fn wait<'a, T>(&self, guard: MutexGuard<'a, T>) -> MutexGuard<'a, T> { + let mutex = guard_lock(&guard); + + self.await_notify(guard).await; + + mutex.lock().await + } + + fn await_notify<'a, T>(&self, guard: MutexGuard<'a, T>) -> AwaitNotify<'_, 'a, T> { + AwaitNotify { + cond: self, + guard: Some(guard), + key: None, + } + } + + /// Blocks the current taks until this condition variable receives a notification and the + /// required condition is met. Spurious wakeups are ignored and this function will only + /// return once the condition has been met. + /// + /// # Examples + /// + /// ``` + /// # async_std::task::block_on(async { + /// # + /// use std::sync::Arc; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_one(); + /// }); + /// + /// // Wait for the thread to start up. + /// let (lock, cvar) = &*pair; + /// // As long as the value inside the `Mutex` is `false`, we wait. + /// let _guard = cvar.wait_until(lock.lock().await, |started| { *started }).await; + /// # + /// # }) + /// ``` + #[allow(clippy::needless_lifetimes)] + pub async fn wait_until<'a, T, F>( + &self, + mut guard: MutexGuard<'a, T>, + mut condition: F, + ) -> MutexGuard<'a, T> + where + F: FnMut(&mut T) -> bool, + { + while !condition(&mut *guard) { + guard = self.wait(guard).await; + } + guard + } + + /// Waits on this condition variable for a notification, timing out after a specified duration. + /// + /// For these reasons `Condvar::wait_timeout_until` is recommended in most cases. + /// + /// # Examples + /// + /// ``` + /// # async_std::task::block_on(async { + /// # + /// use std::sync::Arc; + /// use std::time::Duration; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_one(); + /// }); + /// + /// // wait for the thread to start up + /// let (lock, cvar) = &*pair; + /// let mut started = lock.lock().await; + /// loop { + /// let result = cvar.wait_timeout(started, Duration::from_millis(10)).await; + /// started = result.0; + /// if *started == true { + /// // We received the notification and the value has been updated, we can leave. + /// break + /// } + /// } + /// # + /// # }) + /// ``` + #[allow(clippy::needless_lifetimes)] + pub async fn wait_timeout<'a, T>( + &self, + guard: MutexGuard<'a, T>, + dur: Duration, + ) -> (MutexGuard<'a, T>, WaitTimeoutResult) { + let mutex = guard_lock(&guard); + match timeout(dur, self.wait(guard)).await { + Ok(guard) => (guard, WaitTimeoutResult(false)), + Err(_) => (mutex.lock().await, WaitTimeoutResult(true)), + } + } + + /// Waits on this condition variable for a notification, timing out after a specified duration. + /// Spurious wakes will not cause this function to return. + /// + /// # Examples + /// ``` + /// # async_std::task::block_on(async { + /// use std::sync::Arc; + /// use std::time::Duration; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_one(); + /// }); + /// + /// // wait for the thread to start up + /// let (lock, cvar) = &*pair; + /// let result = cvar.wait_timeout_until( + /// lock.lock().await, + /// Duration::from_millis(100), + /// |&mut started| started, + /// ).await; + /// if result.1.timed_out() { + /// // timed-out without the condition ever evaluating to true. + /// } + /// // access the locked mutex via result.0 + /// # }); + /// ``` + #[allow(clippy::needless_lifetimes)] + pub async fn wait_timeout_until<'a, T, F>( + &self, + guard: MutexGuard<'a, T>, + dur: Duration, + condition: F, + ) -> (MutexGuard<'a, T>, WaitTimeoutResult) + where + F: FnMut(&mut T) -> bool, + { + let mutex = guard_lock(&guard); + match timeout(dur, self.wait_until(guard, condition)).await { + Ok(guard) => (guard, WaitTimeoutResult(false)), + Err(_) => (mutex.lock().await, WaitTimeoutResult(true)), + } + } + + /// Wakes up one blocked task on this condvar. + /// + /// # Examples + /// + /// ``` + /// # fn main() { async_std::task::block_on(async { + /// use std::sync::Arc; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_one(); + /// }); + /// + /// // Wait for the thread to start up. + /// let (lock, cvar) = &*pair; + /// let mut started = lock.lock().await; + /// while !*started { + /// started = cvar.wait(started).await; + /// } + /// # }) } + /// ``` + pub fn notify_one(&self) { + self.wakers.notify_one(); + } + + /// Wakes up all blocked tasks on this condvar. + /// + /// # Examples + /// ``` + /// # fn main() { async_std::task::block_on(async { + /// # + /// use std::sync::Arc; + /// + /// use async_std::sync::{Mutex, Condvar}; + /// use async_std::task; + /// + /// let pair = Arc::new((Mutex::new(false), Condvar::new())); + /// let pair2 = pair.clone(); + /// + /// task::spawn(async move { + /// let (lock, cvar) = &*pair2; + /// let mut started = lock.lock().await; + /// *started = true; + /// // We notify the condvar that the value has changed. + /// cvar.notify_all(); + /// }); + /// + /// // Wait for the thread to start up. + /// let (lock, cvar) = &*pair; + /// let mut started = lock.lock().await; + /// // As long as the value inside the `Mutex` is `false`, we wait. + /// while !*started { + /// started = cvar.wait(started).await; + /// } + /// # + /// # }) } + /// ``` + pub fn notify_all(&self) { + self.wakers.notify_all(); + } +} + +impl fmt::Debug for Condvar { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.pad("Condvar { .. }") + } +} + +/// A future that waits for another task to notify the condition variable. +/// +/// This is an internal future that `wait` and `wait_until` await on. +struct AwaitNotify<'a, 'b, T> { + /// The condition variable that we are waiting on + cond: &'a Condvar, + /// The lock used with `cond`. + /// This will be released the first time the future is polled, + /// after registering the context to be notified. + guard: Option>, + /// A key into the conditions variable's `WakerSet`. + /// This is set to the index of the `Waker` for the context each time + /// the future is polled and not completed. + key: Option, +} + +impl<'a, 'b, T> Future for AwaitNotify<'a, 'b, T> { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.guard.take() { + Some(_) => { + self.key = Some(self.cond.wakers.insert(cx)); + // the guard is dropped when we return, which frees the lock + Poll::Pending + } + None => { + if let Some(key) = self.key { + if self.cond.wakers.remove_if_notified(key, cx) { + self.key = None; + Poll::Ready(()) + } else { + Poll::Pending + } + } else { + // This should only happen if it is polled twice after receiving a notification + Poll::Ready(()) + } + } + } + } +} + +impl<'a, 'b, T> Drop for AwaitNotify<'a, 'b, T> { + fn drop(&mut self) { + if let Some(key) = self.key { + self.cond.wakers.cancel(key); + } + } +} diff --git a/src/sync/mod.rs b/src/sync/mod.rs index 82759fb..1531f8c 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -185,8 +185,10 @@ mod rwlock; cfg_unstable! { pub use barrier::{Barrier, BarrierWaitResult}; pub use channel::{channel, Sender, Receiver, RecvError, TryRecvError, TrySendError}; + pub use condvar::Condvar; mod barrier; + mod condvar; mod channel; } diff --git a/src/sync/mutex.rs b/src/sync/mutex.rs index c62b561..ae953fd 100644 --- a/src/sync/mutex.rs +++ b/src/sync/mutex.rs @@ -287,3 +287,8 @@ impl DerefMut for MutexGuard<'_, T> { unsafe { &mut *self.0.value.get() } } } + +#[cfg(feature = "unstable")] +pub fn guard_lock<'a, T>(guard: &MutexGuard<'a, T>) -> &'a Mutex { + guard.0 +} diff --git a/src/sync/waker_set.rs b/src/sync/waker_set.rs index 7e897af..881304b 100644 --- a/src/sync/waker_set.rs +++ b/src/sync/waker_set.rs @@ -80,6 +80,28 @@ impl WakerSet { } } + /// If the waker for this key is still waiting for a notification, then update + /// the waker for the entry, and return false. If the waker has been notified, + /// treat the entry as completed and return true. + #[cfg(feature = "unstable")] + pub fn remove_if_notified(&self, key: usize, cx: &Context<'_>) -> bool { + let mut inner = self.lock(); + + match &mut inner.entries[key] { + None => { + inner.entries.remove(key); + true + } + Some(w) => { + // We were never woken, so update instead + if !w.will_wake(cx.waker()) { + *w = cx.waker().clone(); + } + false + } + } + } + /// Removes the waker of a cancelled operation. /// /// Returns `true` if another blocked operation from the set was notified. diff --git a/tests/condvar.rs b/tests/condvar.rs new file mode 100644 index 0000000..c4d680f --- /dev/null +++ b/tests/condvar.rs @@ -0,0 +1,91 @@ +#![cfg(feature = "unstable")] +use std::sync::Arc; +use std::time::Duration; + +use async_std::sync::{Condvar, Mutex}; +use async_std::task::{self, JoinHandle}; + +#[test] +fn wait_timeout_with_lock() { + task::block_on(async { + let pair = Arc::new((Mutex::new(false), Condvar::new())); + let pair2 = pair.clone(); + + task::spawn(async move { + let (m, c) = &*pair2; + let _g = m.lock().await; + task::sleep(Duration::from_millis(20)).await; + c.notify_one(); + }); + + let (m, c) = &*pair; + let (_, wait_result) = c + .wait_timeout(m.lock().await, Duration::from_millis(10)) + .await; + assert!(wait_result.timed_out()); + }) +} + +#[test] +fn wait_timeout_without_lock() { + task::block_on(async { + let m = Mutex::new(false); + let c = Condvar::new(); + + let (_, wait_result) = c + .wait_timeout(m.lock().await, Duration::from_millis(10)) + .await; + assert!(wait_result.timed_out()); + }) +} + +#[test] +fn wait_timeout_until_timed_out() { + task::block_on(async { + let m = Mutex::new(false); + let c = Condvar::new(); + + let (_, wait_result) = c + .wait_timeout_until(m.lock().await, Duration::from_millis(10), |&mut started| { + started + }) + .await; + assert!(wait_result.timed_out()); + }) +} + +#[test] +fn notify_all() { + task::block_on(async { + let mut tasks: Vec> = Vec::new(); + let pair = Arc::new((Mutex::new(0u32), Condvar::new())); + + for _ in 0..10 { + let pair = pair.clone(); + tasks.push(task::spawn(async move { + let (m, c) = &*pair; + let mut count = m.lock().await; + while *count == 0 { + count = c.wait(count).await; + } + *count += 1; + })); + } + + // Give some time for tasks to start up + task::sleep(Duration::from_millis(5)).await; + + let (m, c) = &*pair; + { + let mut count = m.lock().await; + *count += 1; + c.notify_all(); + } + + for t in tasks { + t.await; + } + let count = m.lock().await; + assert_eq!(11, *count); + }) +} From e707ea96e077ae66861ab8bd00e95eb3f78b829c Mon Sep 17 00:00:00 2001 From: Fangdun Cai Date: Mon, 27 Apr 2020 00:18:21 +0800 Subject: [PATCH 21/60] docs(readme): add ci status badge --- README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/README.md b/README.md index 4ebf592..7550cd6 100644 --- a/README.md +++ b/README.md @@ -8,6 +8,11 @@
+ + + CI Status + Date: Mon, 27 Apr 2020 01:23:09 +0900 Subject: [PATCH 22/60] ci: speed up github actions --- .github/workflows/ci.yml | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index bcda990..e9fcdcc 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -29,6 +29,24 @@ jobs: toolchain: ${{ matrix.rust }} override: true + - name: Cache cargo registry + uses: actions/cache@v1 + with: + path: ~/.cargo/registry + key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-registry-${{ hashFiles('**/Cargo.toml') }} + + - name: Cache cargo index + uses: actions/cache@v1 + with: + path: ~/.cargo/git + key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-index-${{ hashFiles('**/Cargo.toml') }} + + - name: Cache cargo build + uses: actions/cache@v1 + with: + path: target + key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-build-target-${{ hashFiles('**/Cargo.toml') }} + - name: check uses: actions-rs/cargo@v1 with: @@ -66,12 +84,6 @@ jobs: command: test args: --all --features "unstable attributes" - - name: documentation test - uses: actions-rs/cargo@v1 - with: - command: test - args: --doc --features "unstable attributes" - build__with_no_std: name: Build with no-std runs-on: ubuntu-latest @@ -117,15 +129,3 @@ jobs: - name: Docs run: cargo doc --features docs - - # clippy_check: - # name: Clippy check - # runs-on: ubuntu-latest - # steps: - # - uses: actions/checkout@v1 - # - name: Install rust - # run: rustup update beta && rustup default beta - # - name: Install clippy - # run: rustup component add clippy - # - name: clippy - # run: cargo clippy --all --features unstable From 100c3423c186e7053fca11df078bc0aa9903c293 Mon Sep 17 00:00:00 2001 From: Sunli Date: Mon, 27 Apr 2020 13:49:53 +0800 Subject: [PATCH 23/60] Apply suggestions from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Thank you.😁 Co-Authored-By: Friedel Ziegelmayer --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 69ed7ea..cdbbaa6 100644 --- a/README.md +++ b/README.md @@ -136,7 +136,7 @@ documentation] on how to enable them. * [Xactor](https://crates.io/crates/xactor) — Xactor is a rust actors framework based on async-std. - * [async-graphql](https://crates.io/crates/async-graphql) — The GraphQL server library implemented by rust, fully support async/await. + * [async-graphql](https://crates.io/crates/async-graphql) — A GraphQL server library implemented in rust, with full support for async/await. ## License From 690ab165875cd8ed2dedb67473099d6e724034e2 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 24 Apr 2020 16:16:41 +0200 Subject: [PATCH 24/60] add dependency --- Cargo.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index d49eb95..45662bd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,6 +34,7 @@ default = [ "mio-uds", "num_cpus", "pin-project-lite", + "smol", ] docs = ["attributes", "unstable", "default"] unstable = ["std", "broadcaster", "futures-timer"] @@ -74,6 +75,7 @@ once_cell = { version = "1.3.1", optional = true } pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } +smol = { path = "../smol", optional = true } [dev-dependencies] femme = "1.3.0" From 1308fbdf55cfc585e6403fcecf8657a17751519f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 24 Apr 2020 18:43:31 +0200 Subject: [PATCH 25/60] switch to smol instead of an internal runtime --- Cargo.toml | 2 + src/net/tcp/listener.rs | 67 +++--- src/net/tcp/stream.rs | 89 ++++---- src/net/udp/mod.rs | 116 +++------- src/os/unix/net/datagram.rs | 47 ++-- src/os/unix/net/listener.rs | 54 ++--- src/os/unix/net/stream.rs | 54 ++--- src/rt/mod.rs | 24 +-- src/rt/reactor.rs | 354 ------------------------------ src/rt/runtime.rs | 415 ------------------------------------ src/sync/mod.rs | 10 +- src/sync/spin_lock.rs | 89 -------- src/task/block_on.rs | 83 +------- src/task/builder.rs | 29 +-- src/task/join_handle.rs | 13 +- src/task/mod.rs | 1 - src/task/spawn_blocking.rs | 88 +------- src/task/yield_now.rs | 4 - src/utils.rs | 34 --- tests/mutex.rs | 17 +- 20 files changed, 210 insertions(+), 1380 deletions(-) delete mode 100644 src/rt/reactor.rs delete mode 100644 src/rt/runtime.rs delete mode 100644 src/sync/spin_lock.rs diff --git a/Cargo.toml b/Cargo.toml index 45662bd..3bc9084 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -48,6 +48,7 @@ std = [ "once_cell", "pin-utils", "slab", + "piper", ] alloc = [ "futures-core/alloc", @@ -76,6 +77,7 @@ pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } smol = { path = "../smol", optional = true } +piper = { git = "https://github.com/stjepang/piper.git", branch = "master", optional = true } [dev-dependencies] femme = "1.3.0" diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index 9e15d40..290da0d 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -1,13 +1,13 @@ use std::future::Future; use std::net::SocketAddr; use std::pin::Pin; -use std::sync::Arc; -use crate::future; +use smol::Async; + use crate::io; -use crate::rt::Watcher; use crate::net::{TcpStream, ToSocketAddrs}; use crate::stream::Stream; +use crate::sync::Arc; use crate::task::{Context, Poll}; /// A TCP socket server, listening for connections. @@ -49,7 +49,7 @@ use crate::task::{Context, Poll}; /// ``` #[derive(Debug)] pub struct TcpListener { - watcher: Watcher, + watcher: Async, } impl TcpListener { @@ -79,11 +79,9 @@ impl TcpListener { let addrs = addrs.to_socket_addrs().await?; for addr in addrs { - match mio::net::TcpListener::bind(&addr) { - Ok(mio_listener) => { - return Ok(TcpListener { - watcher: Watcher::new(mio_listener), - }); + match Async::::bind(&addr) { + Ok(listener) => { + return Ok(TcpListener { watcher: listener }); } Err(err) => last_err = Some(err), } @@ -114,13 +112,9 @@ impl TcpListener { /// # Ok(()) }) } /// ``` pub async fn accept(&self) -> io::Result<(TcpStream, SocketAddr)> { - let (io, addr) = - future::poll_fn(|cx| self.watcher.poll_read_with(cx, |inner| inner.accept_std())) - .await?; - - let mio_stream = mio::net::TcpStream::from_stream(io)?; + let (stream, addr) = self.watcher.accept().await?; let stream = TcpStream { - watcher: Arc::new(Watcher::new(mio_stream)), + watcher: Arc::new(stream), }; Ok((stream, addr)) } @@ -206,9 +200,8 @@ impl<'a> Stream for Incoming<'a> { impl From for TcpListener { /// Converts a `std::net::TcpListener` into its asynchronous equivalent. fn from(listener: std::net::TcpListener) -> TcpListener { - let mio_listener = mio::net::TcpListener::from_std(listener).unwrap(); TcpListener { - watcher: Watcher::new(mio_listener), + watcher: Async::new(listener).expect("TcpListener is known to be good"), } } } @@ -230,29 +223,29 @@ cfg_unix! { impl IntoRawFd for TcpListener { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + self.watcher.into_raw_fd() } } } cfg_windows! { - // use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; - // - // impl AsRawSocket for TcpListener { - // fn as_raw_socket(&self) -> RawSocket { - // self.raw_socket - // } - // } - // - // impl FromRawSocket for TcpListener { - // unsafe fn from_raw_socket(handle: RawSocket) -> TcpListener { - // net::TcpListener::from_raw_socket(handle).try_into().unwrap() - // } - // } - // - // impl IntoRawSocket for TcpListener { - // fn into_raw_socket(self) -> RawSocket { - // self.raw_socket - // } - // } + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + + impl AsRawSocket for TcpListener { + fn as_raw_socket(&self) -> RawSocket { + self.watcher.as_raw_socket() + } + } + + impl FromRawSocket for TcpListener { + unsafe fn from_raw_socket(handle: RawSocket) -> TcpListener { + net::TcpListener::from_raw_socket(handle).try_into().unwrap() + } + } + + impl IntoRawSocket for TcpListener { + fn into_raw_socket(self) -> RawSocket { + self.watcher.into_raw_socket() + } + } } diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 1f50e8f..1d2d0ce 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -1,12 +1,12 @@ -use std::io::{IoSlice, IoSliceMut, Read as _, Write as _}; +use std::io::{IoSlice, IoSliceMut}; use std::net::SocketAddr; use std::pin::Pin; -use std::sync::Arc; -use crate::future; +use smol::Async; + use crate::io::{self, Read, Write}; -use crate::rt::Watcher; use crate::net::ToSocketAddrs; +use crate::sync::Arc; use crate::task::{Context, Poll}; /// A TCP stream between a local and a remote socket. @@ -47,7 +47,7 @@ use crate::task::{Context, Poll}; /// ``` #[derive(Debug, Clone)] pub struct TcpStream { - pub(super) watcher: Arc>, + pub(super) watcher: Arc>, } impl TcpStream { @@ -75,28 +75,16 @@ impl TcpStream { let addrs = addrs.to_socket_addrs().await?; for addr in addrs { - // mio's TcpStream::connect is non-blocking and may just be in progress - // when it returns with `Ok`. We therefore wait for write readiness to - // be sure the connection has either been established or there was an - // error which we check for afterwards. - let watcher = match mio::net::TcpStream::connect(&addr) { - Ok(s) => Watcher::new(s), + match Async::::connect(&addr).await { + Ok(stream) => { + return Ok(TcpStream { + watcher: Arc::new(stream), + }); + } Err(e) => { last_err = Some(e); continue; } - }; - - future::poll_fn(|cx| watcher.poll_write_ready(cx)).await; - - match watcher.get_ref().take_error() { - Ok(None) => { - return Ok(TcpStream { - watcher: Arc::new(watcher), - }); - } - Ok(Some(e)) => last_err = Some(e), - Err(e) => last_err = Some(e), } } @@ -214,7 +202,7 @@ impl TcpStream { /// # Ok(()) }) } /// ``` pub async fn peek(&self, buf: &mut [u8]) -> io::Result { - future::poll_fn(|cx| self.watcher.poll_read_with(cx, |inner| inner.peek(buf))).await + self.watcher.peek(buf).await } /// Gets the value of the `TCP_NODELAY` option on this socket. @@ -317,7 +305,7 @@ impl Read for &TcpStream { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - self.watcher.poll_read_with(cx, |mut inner| inner.read(buf)) + Pin::new(&mut &*self.watcher).poll_read(cx, buf) } } @@ -353,26 +341,23 @@ impl Write for &TcpStream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - self.watcher - .poll_write_with(cx, |mut inner| inner.write(buf)) + Pin::new(&mut &*self.watcher).poll_write(cx, buf) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.watcher.poll_write_with(cx, |mut inner| inner.flush()) + Pin::new(&mut &*self.watcher).poll_flush(cx) } - fn poll_close(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { - self.shutdown(std::net::Shutdown::Write)?; - Poll::Ready(Ok(())) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut &*self.watcher).poll_close(cx) } } impl From for TcpStream { /// Converts a `std::net::TcpStream` into its asynchronous equivalent. fn from(stream: std::net::TcpStream) -> TcpStream { - let mio_stream = mio::net::TcpStream::from_stream(stream).unwrap(); TcpStream { - watcher: Arc::new(Watcher::new(mio_stream)), + watcher: Arc::new(Async::new(stream).expect("TcpStream is known to be good")), } } } @@ -403,23 +388,23 @@ cfg_unix! { } cfg_windows! { - // use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; - // - // impl AsRawSocket for TcpStream { - // fn as_raw_socket(&self) -> RawSocket { - // self.raw_socket - // } - // } - // - // impl FromRawSocket for TcpStream { - // unsafe fn from_raw_socket(handle: RawSocket) -> TcpStream { - // net::TcpStream::from_raw_socket(handle).try_into().unwrap() - // } - // } - // - // impl IntoRawSocket for TcpListener { - // fn into_raw_socket(self) -> RawSocket { - // self.raw_socket - // } - // } + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + + impl AsRawSocket for TcpStream { + fn as_raw_socket(&self) -> RawSocket { + self.raw_socket + } + } + + impl FromRawSocket for TcpStream { + unsafe fn from_raw_socket(handle: RawSocket) -> TcpStream { + net::TcpStream::from_raw_socket(handle).try_into().unwrap() + } + } + + impl IntoRawSocket for TcpListener { + fn into_raw_socket(self) -> RawSocket { + self.raw_socket + } + } } diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 774478d..3bc9ad7 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -2,9 +2,9 @@ use std::io; use std::net::SocketAddr; use std::net::{Ipv4Addr, Ipv6Addr}; -use crate::future; +use smol::Async; + use crate::net::ToSocketAddrs; -use crate::rt::Watcher; use crate::utils::Context as _; /// A UDP socket. @@ -45,7 +45,7 @@ use crate::utils::Context as _; /// ``` #[derive(Debug)] pub struct UdpSocket { - watcher: Watcher, + watcher: Async, } impl UdpSocket { @@ -69,16 +69,12 @@ impl UdpSocket { /// ``` pub async fn bind(addrs: A) -> io::Result { let mut last_err = None; - let addrs = addrs - .to_socket_addrs() - .await?; + let addrs = addrs.to_socket_addrs().await?; for addr in addrs { - match mio::net::UdpSocket::bind(&addr) { - Ok(mio_socket) => { - return Ok(UdpSocket { - watcher: Watcher::new(mio_socket), - }); + match Async::::bind(&addr) { + Ok(socket) => { + return Ok(UdpSocket { watcher: socket }); } Err(err) => last_err = Some(err), } @@ -153,12 +149,10 @@ impl UdpSocket { } }; - future::poll_fn(|cx| { - self.watcher - .poll_write_with(cx, |inner| inner.send_to(buf, &addr)) - }) - .await - .context(|| format!("could not send packet to {}", addr)) + self.watcher + .send_to(buf, addr) + .await + .context(|| format!("could not send packet to {}", addr)) } /// Receives data from the socket. @@ -181,22 +175,7 @@ impl UdpSocket { /// # Ok(()) }) } /// ``` pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - future::poll_fn(|cx| { - self.watcher - .poll_read_with(cx, |inner| inner.recv_from(buf)) - }) - .await - .context(|| { - use std::fmt::Write; - - let mut error = String::from("could not receive data on "); - if let Ok(addr) = self.local_addr() { - let _ = write!(&mut error, "{}", addr); - } else { - error.push_str("socket"); - } - error - }) + self.watcher.recv_from(buf).await } /// Connects the UDP socket to a remote address. @@ -267,19 +246,7 @@ impl UdpSocket { /// # Ok(()) }) } /// ``` pub async fn send(&self, buf: &[u8]) -> io::Result { - future::poll_fn(|cx| self.watcher.poll_write_with(cx, |inner| inner.send(buf))) - .await - .context(|| { - use std::fmt::Write; - - let mut error = String::from("could not send data on "); - if let Ok(addr) = self.local_addr() { - let _ = write!(&mut error, "{}", addr); - } else { - error.push_str("socket"); - } - error - }) + self.watcher.send(buf).await } /// Receives data from the socket. @@ -303,19 +270,7 @@ impl UdpSocket { /// # Ok(()) }) } /// ``` pub async fn recv(&self, buf: &mut [u8]) -> io::Result { - future::poll_fn(|cx| self.watcher.poll_read_with(cx, |inner| inner.recv(buf))) - .await - .context(|| { - use std::fmt::Write; - - let mut error = String::from("could not receive data on "); - if let Ok(addr) = self.local_addr() { - let _ = write!(&mut error, "{}", addr); - } else { - error.push_str("socket"); - } - error - }) + self.watcher.recv(buf).await } /// Gets the value of the `SO_BROADCAST` option for this socket. @@ -498,9 +453,8 @@ impl UdpSocket { impl From for UdpSocket { /// Converts a `std::net::UdpSocket` into its asynchronous equivalent. fn from(socket: std::net::UdpSocket) -> UdpSocket { - let mio_socket = mio::net::UdpSocket::from_socket(socket).unwrap(); UdpSocket { - watcher: Watcher::new(mio_socket), + watcher: Async::new(socket).expect("UdpSocket is known to be good"), } } } @@ -522,29 +476,29 @@ cfg_unix! { impl IntoRawFd for UdpSocket { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + self.watcher.into_raw_fd() } } } cfg_windows! { - // use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; - // - // impl AsRawSocket for UdpSocket { - // fn as_raw_socket(&self) -> RawSocket { - // self.raw_socket - // } - // } - // - // impl FromRawSocket for UdpSocket { - // unsafe fn from_raw_socket(handle: RawSocket) -> UdpSocket { - // net::UdpSocket::from_raw_socket(handle).into() - // } - // } - // - // impl IntoRawSocket for UdpSocket { - // fn into_raw_socket(self) -> RawSocket { - // self.raw_socket - // } - // } + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + + impl AsRawSocket for UdpSocket { + fn as_raw_socket(&self) -> RawSocket { + self.watcher.as_raw_socket() + } + } + + impl FromRawSocket for UdpSocket { + unsafe fn from_raw_socket(handle: RawSocket) -> UdpSocket { + net::UdpSocket::from_raw_socket(handle).into() + } + } + + impl IntoRawSocket for UdpSocket { + fn into_raw_socket(self) -> RawSocket { + self.watcher.into_raw_socket() + } + } } diff --git a/src/os/unix/net/datagram.rs b/src/os/unix/net/datagram.rs index 5a2d6ec..6a98736 100644 --- a/src/os/unix/net/datagram.rs +++ b/src/os/unix/net/datagram.rs @@ -2,16 +2,14 @@ use std::fmt; use std::net::Shutdown; +use std::os::unix::net::UnixDatagram as StdUnixDatagram; -use mio_uds; +use smol::Async; use super::SocketAddr; -use crate::future; use crate::io; -use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; -use crate::task::spawn_blocking; /// A Unix datagram socket. /// @@ -42,13 +40,13 @@ use crate::task::spawn_blocking; /// # Ok(()) }) } /// ``` pub struct UnixDatagram { - watcher: Watcher, + watcher: Async, } impl UnixDatagram { - fn new(socket: mio_uds::UnixDatagram) -> UnixDatagram { + fn new(socket: StdUnixDatagram) -> UnixDatagram { UnixDatagram { - watcher: Watcher::new(socket), + watcher: Async::new(socket).expect("UnixDatagram is known to be good"), } } @@ -67,8 +65,8 @@ impl UnixDatagram { /// ``` pub async fn bind>(path: P) -> io::Result { let path = path.as_ref().to_owned(); - let socket = spawn_blocking(move || mio_uds::UnixDatagram::bind(path)).await?; - Ok(UnixDatagram::new(socket)) + let socket = Async::::bind(path)?; + Ok(UnixDatagram { watcher: socket }) } /// Creates a Unix datagram which is not bound to any address. @@ -85,7 +83,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub fn unbound() -> io::Result { - let socket = mio_uds::UnixDatagram::unbound()?; + let socket = StdUnixDatagram::unbound()?; Ok(UnixDatagram::new(socket)) } @@ -105,7 +103,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub fn pair() -> io::Result<(UnixDatagram, UnixDatagram)> { - let (a, b) = mio_uds::UnixDatagram::pair()?; + let (a, b) = StdUnixDatagram::pair()?; let a = UnixDatagram::new(a); let b = UnixDatagram::new(b); Ok((a, b)) @@ -197,11 +195,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - future::poll_fn(|cx| { - self.watcher - .poll_read_with(cx, |inner| inner.recv_from(buf)) - }) - .await + self.watcher.recv_from(buf).await } /// Receives data from the socket. @@ -222,7 +216,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub async fn recv(&self, buf: &mut [u8]) -> io::Result { - future::poll_fn(|cx| self.watcher.poll_read_with(cx, |inner| inner.recv(buf))).await + self.watcher.recv(buf).await } /// Sends data on the socket to the specified address. @@ -242,11 +236,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub async fn send_to>(&self, buf: &[u8], path: P) -> io::Result { - future::poll_fn(|cx| { - self.watcher - .poll_write_with(cx, |inner| inner.send_to(buf, path.as_ref())) - }) - .await + self.watcher.send_to(buf, path.as_ref()).await } /// Sends data on the socket to the socket's peer. @@ -267,7 +257,7 @@ impl UnixDatagram { /// # Ok(()) }) } /// ``` pub async fn send(&self, buf: &[u8]) -> io::Result { - future::poll_fn(|cx| self.watcher.poll_write_with(cx, |inner| inner.send(buf))).await + self.watcher.send(buf).await } /// Shut down the read, write, or both halves of this connection. @@ -312,19 +302,18 @@ impl fmt::Debug for UnixDatagram { } } -impl From for UnixDatagram { +impl From for UnixDatagram { /// Converts a `std::os::unix::net::UnixDatagram` into its asynchronous equivalent. - fn from(datagram: std::os::unix::net::UnixDatagram) -> UnixDatagram { - let mio_datagram = mio_uds::UnixDatagram::from_datagram(datagram).unwrap(); + fn from(datagram: StdUnixDatagram) -> UnixDatagram { UnixDatagram { - watcher: Watcher::new(mio_datagram), + watcher: Async::new(datagram).expect("UnixDatagram is known to be good"), } } } impl AsRawFd for UnixDatagram { fn as_raw_fd(&self) -> RawFd { - self.watcher.get_ref().as_raw_fd() + self.watcher.as_raw_fd() } } @@ -337,6 +326,6 @@ impl FromRawFd for UnixDatagram { impl IntoRawFd for UnixDatagram { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + self.watcher.into_raw_fd() } } diff --git a/src/os/unix/net/listener.rs b/src/os/unix/net/listener.rs index 9f6bdcb..4099bd6 100644 --- a/src/os/unix/net/listener.rs +++ b/src/os/unix/net/listener.rs @@ -1,20 +1,19 @@ //! Unix-specific networking extensions. use std::fmt; -use std::pin::Pin; use std::future::Future; +use std::os::unix::net::UnixListener as StdUnixListener; +use std::pin::Pin; -use mio_uds; +use smol::Async; use super::SocketAddr; use super::UnixStream; -use crate::future; use crate::io; -use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; use crate::stream::Stream; -use crate::task::{spawn_blocking, Context, Poll}; +use crate::task::{Context, Poll}; /// A Unix domain socket server, listening for connections. /// @@ -50,7 +49,7 @@ use crate::task::{spawn_blocking, Context, Poll}; /// # Ok(()) }) } /// ``` pub struct UnixListener { - watcher: Watcher, + watcher: Async, } impl UnixListener { @@ -69,11 +68,9 @@ impl UnixListener { /// ``` pub async fn bind>(path: P) -> io::Result { let path = path.as_ref().to_owned(); - let listener = spawn_blocking(move || mio_uds::UnixListener::bind(path)).await?; + let listener = Async::::bind(path)?; - Ok(UnixListener { - watcher: Watcher::new(listener), - }) + Ok(UnixListener { watcher: listener }) } /// Accepts a new incoming connection to this listener. @@ -93,29 +90,9 @@ impl UnixListener { /// # Ok(()) }) } /// ``` pub async fn accept(&self) -> io::Result<(UnixStream, SocketAddr)> { - future::poll_fn(|cx| { - let res = futures_core::ready!(self.watcher.poll_read_with(cx, |inner| { - match inner.accept_std() { - // Converting to `WouldBlock` so that the watcher will - // add the waker of this task to a list of readers. - Ok(None) => Err(io::ErrorKind::WouldBlock.into()), - res => res, - } - })); - - match res? { - Some((io, addr)) => { - let mio_stream = mio_uds::UnixStream::from_stream(io)?; - let stream = UnixStream { - watcher: Watcher::new(mio_stream), - }; - Poll::Ready(Ok((stream, addr))) - } - // This should never happen since `None` is converted to `WouldBlock` - None => unreachable!(), - } - }) - .await + let (stream, addr) = self.watcher.accept().await?; + + Ok((UnixStream { watcher: stream }, addr)) } /// Returns a stream of incoming connections. @@ -206,19 +183,18 @@ impl Stream for Incoming<'_> { } } -impl From for UnixListener { +impl From for UnixListener { /// Converts a `std::os::unix::net::UnixListener` into its asynchronous equivalent. - fn from(listener: std::os::unix::net::UnixListener) -> UnixListener { - let mio_listener = mio_uds::UnixListener::from_listener(listener).unwrap(); + fn from(listener: StdUnixListener) -> UnixListener { UnixListener { - watcher: Watcher::new(mio_listener), + watcher: Async::new(listener).expect("UnixListener is known to be good"), } } } impl AsRawFd for UnixListener { fn as_raw_fd(&self) -> RawFd { - self.watcher.get_ref().as_raw_fd() + self.watcher.as_raw_fd() } } @@ -231,6 +207,6 @@ impl FromRawFd for UnixListener { impl IntoRawFd for UnixListener { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + self.watcher.into_raw_fd() } } diff --git a/src/os/unix/net/stream.rs b/src/os/unix/net/stream.rs index a1c83f1..7320c85 100644 --- a/src/os/unix/net/stream.rs +++ b/src/os/unix/net/stream.rs @@ -1,18 +1,17 @@ //! Unix-specific networking extensions. use std::fmt; -use std::io::{Read as _, Write as _}; use std::net::Shutdown; +use std::os::unix::net::UnixStream as StdUnixStream; use std::pin::Pin; -use mio_uds; +use smol::Async; use super::SocketAddr; use crate::io::{self, Read, Write}; -use crate::rt::Watcher; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; -use crate::task::{spawn_blocking, Context, Poll}; +use crate::task::{Context, Poll}; /// A Unix stream socket. /// @@ -38,7 +37,7 @@ use crate::task::{spawn_blocking, Context, Poll}; /// # Ok(()) }) } /// ``` pub struct UnixStream { - pub(super) watcher: Watcher, + pub(super) watcher: Async, } impl UnixStream { @@ -57,15 +56,9 @@ impl UnixStream { /// ``` pub async fn connect>(path: P) -> io::Result { let path = path.as_ref().to_owned(); + let stream = Async::::connect(path).await?; - spawn_blocking(move || { - let std_stream = std::os::unix::net::UnixStream::connect(path)?; - let mio_stream = mio_uds::UnixStream::from_stream(std_stream)?; - Ok(UnixStream { - watcher: Watcher::new(mio_stream), - }) - }) - .await + Ok(UnixStream { watcher: stream }) } /// Creates an unnamed pair of connected sockets. @@ -84,13 +77,9 @@ impl UnixStream { /// # Ok(()) }) } /// ``` pub fn pair() -> io::Result<(UnixStream, UnixStream)> { - let (a, b) = mio_uds::UnixStream::pair()?; - let a = UnixStream { - watcher: Watcher::new(a), - }; - let b = UnixStream { - watcher: Watcher::new(b), - }; + let (a, b) = Async::::pair()?; + let a = UnixStream { watcher: a }; + let b = UnixStream { watcher: b }; Ok((a, b)) } @@ -169,7 +158,7 @@ impl Read for &UnixStream { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - self.watcher.poll_read_with(cx, |mut inner| inner.read(buf)) + Pin::new(&mut &self.watcher).poll_read(cx, buf) } } @@ -197,16 +186,15 @@ impl Write for &UnixStream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - self.watcher - .poll_write_with(cx, |mut inner| inner.write(buf)) + Pin::new(&mut &self.watcher).poll_write(cx, buf) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.watcher.poll_write_with(cx, |mut inner| inner.flush()) + Pin::new(&mut &self.watcher).poll_flush(cx) } - fn poll_close(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut &self.watcher).poll_close(cx) } } @@ -227,19 +215,17 @@ impl fmt::Debug for UnixStream { } } -impl From for UnixStream { +impl From for UnixStream { /// Converts a `std::os::unix::net::UnixStream` into its asynchronous equivalent. - fn from(stream: std::os::unix::net::UnixStream) -> UnixStream { - let mio_stream = mio_uds::UnixStream::from_stream(stream).unwrap(); - UnixStream { - watcher: Watcher::new(mio_stream), - } + fn from(stream: StdUnixStream) -> UnixStream { + let stream = Async::new(stream).expect("UnixStream is known to be good"); + UnixStream { watcher: stream } } } impl AsRawFd for UnixStream { fn as_raw_fd(&self) -> RawFd { - self.watcher.get_ref().as_raw_fd() + self.watcher.as_raw_fd() } } @@ -252,6 +238,6 @@ impl FromRawFd for UnixStream { impl IntoRawFd for UnixStream { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + self.watcher.into_raw_fd() } } diff --git a/src/rt/mod.rs b/src/rt/mod.rs index 2149d24..d5d0d61 100644 --- a/src/rt/mod.rs +++ b/src/rt/mod.rs @@ -4,20 +4,20 @@ use std::thread; use once_cell::sync::Lazy; -use crate::utils::abort_on_panic; +use crate::future; -pub use reactor::{Reactor, Watcher}; -pub use runtime::Runtime; - -mod reactor; -mod runtime; +/// Dummy runtime struct. +pub struct Runtime {} /// The global runtime. pub static RUNTIME: Lazy = Lazy::new(|| { - thread::Builder::new() - .name("async-std/runtime".to_string()) - .spawn(|| abort_on_panic(|| RUNTIME.run())) - .expect("cannot start a runtime thread"); - - Runtime::new() + // Create an executor thread pool. + let num_threads = num_cpus::get().max(1); + for _ in 0..num_threads { + thread::Builder::new() + .name("async-std/runtime".to_string()) + .spawn(|| smol::run(future::pending::<()>())) + .expect("cannot start a runtime thread"); + } + Runtime {} }); diff --git a/src/rt/reactor.rs b/src/rt/reactor.rs deleted file mode 100644 index 2a35b72..0000000 --- a/src/rt/reactor.rs +++ /dev/null @@ -1,354 +0,0 @@ -use std::fmt; -use std::sync::{Arc, Mutex}; -use std::time::Duration; - -use mio::{self, Evented}; -use slab::Slab; - -use crate::io; -use crate::rt::RUNTIME; -use crate::task::{Context, Poll, Waker}; - -/// Data associated with a registered I/O handle. -#[derive(Debug)] -struct Entry { - /// A unique identifier. - token: mio::Token, - - /// Tasks that are blocked on reading from this I/O handle. - readers: Mutex, - - /// Tasks that are blocked on writing to this I/O handle. - writers: Mutex, -} - -/// The state of a networking driver. -pub struct Reactor { - /// A mio instance that polls for new events. - poller: mio::Poll, - - /// A list into which mio stores events. - events: Mutex, - - /// A collection of registered I/O handles. - entries: Mutex>>, - - /// Dummy I/O handle that is only used to wake up the polling thread. - notify_reg: (mio::Registration, mio::SetReadiness), - - /// An identifier for the notification handle. - notify_token: mio::Token, -} - -/// The set of `Waker`s interested in read readiness. -#[derive(Debug)] -struct Readers { - /// Flag indicating read readiness. - /// (cf. `Watcher::poll_read_ready`) - ready: bool, - /// The `Waker`s blocked on reading. - wakers: Vec, -} - -/// The set of `Waker`s interested in write readiness. -#[derive(Debug)] -struct Writers { - /// Flag indicating write readiness. - /// (cf. `Watcher::poll_write_ready`) - ready: bool, - /// The `Waker`s blocked on writing. - wakers: Vec, -} - -impl Reactor { - /// Creates a new reactor for polling I/O events. - pub fn new() -> io::Result { - let poller = mio::Poll::new()?; - let notify_reg = mio::Registration::new2(); - - let mut reactor = Reactor { - poller, - events: Mutex::new(mio::Events::with_capacity(1000)), - entries: Mutex::new(Slab::new()), - notify_reg, - notify_token: mio::Token(0), - }; - - // Register a dummy I/O handle for waking up the polling thread. - let entry = reactor.register(&reactor.notify_reg.0)?; - reactor.notify_token = entry.token; - - Ok(reactor) - } - - /// Registers an I/O event source and returns its associated entry. - fn register(&self, source: &dyn Evented) -> io::Result> { - let mut entries = self.entries.lock().unwrap(); - - // Reserve a vacant spot in the slab and use its key as the token value. - let vacant = entries.vacant_entry(); - let token = mio::Token(vacant.key()); - - // Allocate an entry and insert it into the slab. - let entry = Arc::new(Entry { - token, - readers: Mutex::new(Readers { - ready: false, - wakers: Vec::new(), - }), - writers: Mutex::new(Writers { - ready: false, - wakers: Vec::new(), - }), - }); - vacant.insert(entry.clone()); - - // Register the I/O event source in the poller. - let interest = mio::Ready::all(); - let opts = mio::PollOpt::edge(); - self.poller.register(source, token, interest, opts)?; - - Ok(entry) - } - - /// Deregisters an I/O event source associated with an entry. - fn deregister(&self, source: &dyn Evented, entry: &Entry) -> io::Result<()> { - // Deregister the I/O object from the mio instance. - self.poller.deregister(source)?; - - // Remove the entry associated with the I/O object. - self.entries.lock().unwrap().remove(entry.token.0); - - Ok(()) - } - - /// Notifies the reactor so that polling stops blocking. - pub fn notify(&self) -> io::Result<()> { - self.notify_reg.1.set_readiness(mio::Ready::readable()) - } - - /// Waits on the poller for new events and wakes up tasks blocked on I/O handles. - /// - /// Returns `Ok(true)` if at least one new task was woken. - pub fn poll(&self, timeout: Option) -> io::Result { - let mut events = self.events.lock().unwrap(); - - // Block on the poller until at least one new event comes in. - self.poller.poll(&mut events, timeout)?; - - // Lock the entire entry table while we're processing new events. - let entries = self.entries.lock().unwrap(); - - // The number of woken tasks. - let mut progress = false; - - for event in events.iter() { - let token = event.token(); - - if token == self.notify_token { - // If this is the notification token, we just need the notification state. - self.notify_reg.1.set_readiness(mio::Ready::empty())?; - } else { - // Otherwise, look for the entry associated with this token. - if let Some(entry) = entries.get(token.0) { - // Set the readiness flags from this I/O event. - let readiness = event.readiness(); - - // Wake up reader tasks blocked on this I/O handle. - let reader_interests = mio::Ready::all() - mio::Ready::writable(); - if !(readiness & reader_interests).is_empty() { - let mut readers = entry.readers.lock().unwrap(); - readers.ready = true; - for w in readers.wakers.drain(..) { - w.wake(); - progress = true; - } - } - - // Wake up writer tasks blocked on this I/O handle. - let writer_interests = mio::Ready::all() - mio::Ready::readable(); - if !(readiness & writer_interests).is_empty() { - let mut writers = entry.writers.lock().unwrap(); - writers.ready = true; - for w in writers.wakers.drain(..) { - w.wake(); - progress = true; - } - } - } - } - } - - Ok(progress) - } -} - -/// An I/O handle powered by the networking driver. -/// -/// This handle wraps an I/O event source and exposes a "futurized" interface on top of it, -/// implementing traits `AsyncRead` and `AsyncWrite`. -pub struct Watcher { - /// Data associated with the I/O handle. - entry: Arc, - - /// The I/O event source. - source: Option, -} - -impl Watcher { - /// Creates a new I/O handle. - /// - /// The provided I/O event source will be kept registered inside the reactor's poller for the - /// lifetime of the returned I/O handle. - pub fn new(source: T) -> Watcher { - Watcher { - entry: RUNTIME - .reactor() - .register(&source) - .expect("cannot register an I/O event source"), - source: Some(source), - } - } - - /// Returns a reference to the inner I/O event source. - pub fn get_ref(&self) -> &T { - self.source.as_ref().unwrap() - } - - /// Polls the inner I/O source for a non-blocking read operation. - /// - /// If the operation returns an error of the `io::ErrorKind::WouldBlock` kind, the current task - /// will be registered for wakeup when the I/O source becomes readable. - pub fn poll_read_with<'a, F, R>(&'a self, cx: &mut Context<'_>, mut f: F) -> Poll> - where - F: FnMut(&'a T) -> io::Result, - { - // If the operation isn't blocked, return its result. - match f(self.source.as_ref().unwrap()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - - // Lock the waker list. - let mut readers = self.entry.readers.lock().unwrap(); - - // Try running the operation again. - match f(self.source.as_ref().unwrap()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - - // Register the task if it isn't registered already. - - if readers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { - readers.wakers.push(cx.waker().clone()); - } - - Poll::Pending - } - - /// Polls the inner I/O source for a non-blocking write operation. - /// - /// If the operation returns an error of the `io::ErrorKind::WouldBlock` kind, the current task - /// will be registered for wakeup when the I/O source becomes writable. - pub fn poll_write_with<'a, F, R>( - &'a self, - cx: &mut Context<'_>, - mut f: F, - ) -> Poll> - where - F: FnMut(&'a T) -> io::Result, - { - // If the operation isn't blocked, return its result. - match f(self.source.as_ref().unwrap()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - - // Lock the waker list. - let mut writers = self.entry.writers.lock().unwrap(); - - // Try running the operation again. - match f(self.source.as_ref().unwrap()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - - // Register the task if it isn't registered already. - if writers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { - writers.wakers.push(cx.waker().clone()); - } - - Poll::Pending - } - - /// Polls the inner I/O source until a non-blocking read can be performed. - /// - /// If non-blocking reads are currently not possible, the `Waker` - /// will be saved and notified when it can read non-blocking - /// again. - #[allow(dead_code)] - pub fn poll_read_ready(&self, cx: &mut Context<'_>) -> Poll<()> { - // Lock the waker list. - let mut readers = self.entry.readers.lock().unwrap(); - if readers.ready { - return Poll::Ready(()); - } - // Register the task if it isn't registered already. - if readers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { - readers.wakers.push(cx.waker().clone()); - } - Poll::Pending - } - - /// Polls the inner I/O source until a non-blocking write can be performed. - /// - /// If non-blocking writes are currently not possible, the `Waker` - /// will be saved and notified when it can write non-blocking - /// again. - pub fn poll_write_ready(&self, cx: &mut Context<'_>) -> Poll<()> { - // Lock the waker list. - let mut writers = self.entry.writers.lock().unwrap(); - if writers.ready { - return Poll::Ready(()); - } - // Register the task if it isn't registered already. - if writers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { - writers.wakers.push(cx.waker().clone()); - } - Poll::Pending - } - - /// Deregisters and returns the inner I/O source. - /// - /// This method is typically used to convert `Watcher`s to raw file descriptors/handles. - #[allow(dead_code)] - pub fn into_inner(mut self) -> T { - let source = self.source.take().unwrap(); - RUNTIME - .reactor() - .deregister(&source, &self.entry) - .expect("cannot deregister I/O event source"); - source - } -} - -impl Drop for Watcher { - fn drop(&mut self) { - if let Some(ref source) = self.source { - RUNTIME - .reactor() - .deregister(source, &self.entry) - .expect("cannot deregister I/O event source"); - } - } -} - -impl fmt::Debug for Watcher { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Watcher") - .field("entry", &self.entry) - .field("source", &self.source) - .finish() - } -} diff --git a/src/rt/runtime.rs b/src/rt/runtime.rs deleted file mode 100644 index a0d88b9..0000000 --- a/src/rt/runtime.rs +++ /dev/null @@ -1,415 +0,0 @@ -use std::cell::Cell; -use std::io; -use std::iter; -use std::ptr; -use std::sync::atomic::{self, Ordering}; -use std::sync::{Arc, Mutex}; -use std::thread; -use std::time::Duration; - -use crossbeam_deque::{Injector, Steal, Stealer, Worker}; -use crossbeam_utils::thread::scope; -use once_cell::unsync::OnceCell; - -use crate::rt::Reactor; -use crate::sync::Spinlock; -use crate::task::Runnable; -use crate::utils::{abort_on_panic, random}; - -thread_local! { - /// A reference to the current machine, if the current thread runs tasks. - static MACHINE: OnceCell> = OnceCell::new(); - - /// This flag is set to true whenever `task::yield_now()` is invoked. - static YIELD_NOW: Cell = Cell::new(false); -} - -struct Scheduler { - /// Set to `true` while a machine is polling the reactor. - polling: bool, - - /// Idle processors. - processors: Vec, - - /// Running machines. - machines: Vec>, -} - -/// An async runtime. -pub struct Runtime { - /// The reactor. - reactor: Reactor, - - /// The global queue of tasks. - injector: Injector, - - /// Handles to local queues for stealing work. - stealers: Vec>, - - /// The scheduler state. - sched: Mutex, -} - -impl Runtime { - /// Creates a new runtime. - pub fn new() -> Runtime { - let cpus = num_cpus::get().max(1); - let processors: Vec<_> = (0..cpus).map(|_| Processor::new()).collect(); - let stealers = processors.iter().map(|p| p.worker.stealer()).collect(); - - Runtime { - reactor: Reactor::new().unwrap(), - injector: Injector::new(), - stealers, - sched: Mutex::new(Scheduler { - processors, - machines: Vec::new(), - polling: false, - }), - } - } - - /// Returns a reference to the reactor. - pub fn reactor(&self) -> &Reactor { - &self.reactor - } - - /// Flushes the task slot so that tasks get run more fairly. - pub fn yield_now(&self) { - YIELD_NOW.with(|flag| flag.set(true)); - } - - /// Schedules a task. - pub fn schedule(&self, task: Runnable) { - MACHINE.with(|machine| { - // If the current thread is a worker thread, schedule it onto the current machine. - // Otherwise, push it into the global task queue. - match machine.get() { - None => { - self.injector.push(task); - self.notify(); - } - Some(m) => m.schedule(&self, task), - } - }); - } - - /// Runs the runtime on the current thread. - pub fn run(&self) { - scope(|s| { - let mut idle = 0; - let mut delay = 0; - - loop { - // Get a list of new machines to start, if any need to be started. - for m in self.make_machines() { - idle = 0; - - s.builder() - .name("async-std/machine".to_string()) - .spawn(move |_| { - abort_on_panic(|| { - let _ = MACHINE.with(|machine| machine.set(m.clone())); - m.run(self); - }) - }) - .expect("cannot start a machine thread"); - } - - // Sleep for a bit longer if the scheduler state hasn't changed in a while. - if idle > 10 { - delay = (delay * 2).min(10_000); - } else { - idle += 1; - delay = 1000; - } - - thread::sleep(Duration::from_micros(delay)); - } - }) - .unwrap(); - } - - /// Returns a list of machines that need to be started. - fn make_machines(&self) -> Vec> { - let mut sched = self.sched.lock().unwrap(); - let mut to_start = Vec::new(); - - // If no machine has been polling the reactor in a while, that means the runtime is - // overloaded with work and we need to start another machine. - if !sched.polling { - if let Some(p) = sched.processors.pop() { - let m = Arc::new(Machine::new(p)); - to_start.push(m.clone()); - sched.machines.push(m); - } - } - - to_start - } - - /// Unparks a thread polling the reactor. - fn notify(&self) { - atomic::fence(Ordering::SeqCst); - self.reactor.notify().unwrap(); - } - - /// Attempts to poll the reactor without blocking on it. - /// - /// Returns `Ok(true)` if at least one new task was woken. - /// - /// This function might not poll the reactor at all so do not rely on it doing anything. Only - /// use for optimization. - fn quick_poll(&self) -> io::Result { - if let Ok(sched) = self.sched.try_lock() { - if !sched.polling { - return self.reactor.poll(Some(Duration::from_secs(0))); - } - } - Ok(false) - } -} - -/// A thread running a processor. -struct Machine { - /// Holds the processor until it gets stolen. - processor: Spinlock>, -} - -impl Machine { - /// Creates a new machine running a processor. - fn new(p: Processor) -> Machine { - Machine { - processor: Spinlock::new(Some(p)), - } - } - - /// Schedules a task onto the machine. - fn schedule(&self, rt: &Runtime, task: Runnable) { - match self.processor.lock().as_mut() { - None => { - rt.injector.push(task); - rt.notify(); - } - Some(p) => p.schedule(rt, task), - } - } - - /// Finds the next runnable task. - fn find_task(&self, rt: &Runtime) -> Steal { - let mut retry = false; - - // First try finding a task in the local queue or in the global queue. - if let Some(p) = self.processor.lock().as_mut() { - if let Some(task) = p.pop_task() { - return Steal::Success(task); - } - - match p.steal_from_global(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), - } - } - - // Try polling the reactor, but don't block on it. - let progress = rt.quick_poll().unwrap(); - - // Try finding a task in the local queue, which might hold tasks woken by the reactor. If - // the local queue is still empty, try stealing from other processors. - if let Some(p) = self.processor.lock().as_mut() { - if progress { - if let Some(task) = p.pop_task() { - return Steal::Success(task); - } - } - - match p.steal_from_others(rt) { - Steal::Empty => {} - Steal::Retry => retry = true, - Steal::Success(task) => return Steal::Success(task), - } - } - - if retry { Steal::Retry } else { Steal::Empty } - } - - /// Runs the machine on the current thread. - fn run(&self, rt: &Runtime) { - /// Number of yields when no runnable task is found. - const YIELDS: u32 = 3; - /// Number of short sleeps when no runnable task in found. - const SLEEPS: u32 = 10; - /// Number of runs in a row before the global queue is inspected. - const RUNS: u32 = 64; - - // The number of times the thread found work in a row. - let mut runs = 0; - // The number of times the thread didn't find work in a row. - let mut fails = 0; - - loop { - // Check if `task::yield_now()` was invoked and flush the slot if so. - YIELD_NOW.with(|flag| { - if flag.replace(false) { - if let Some(p) = self.processor.lock().as_mut() { - p.flush_slot(rt); - } - } - }); - - // After a number of runs in a row, do some work to ensure no task is left behind - // indefinitely. Poll the reactor, steal tasks from the global queue, and flush the - // task slot. - if runs >= RUNS { - runs = 0; - rt.quick_poll().unwrap(); - - if let Some(p) = self.processor.lock().as_mut() { - if let Steal::Success(task) = p.steal_from_global(rt) { - p.schedule(rt, task); - } - - p.flush_slot(rt); - } - } - - // Try to find a runnable task. - if let Steal::Success(task) = self.find_task(rt) { - task.run(); - runs += 1; - fails = 0; - continue; - } - - fails += 1; - - // Yield the current thread a few times. - if fails <= YIELDS { - thread::yield_now(); - continue; - } - - // Put the current thread to sleep a few times. - if fails <= YIELDS + SLEEPS { - let opt_p = self.processor.lock().take(); - thread::sleep(Duration::from_micros(10)); - *self.processor.lock() = opt_p; - continue; - } - - let mut sched = rt.sched.lock().unwrap(); - - // One final check for available tasks while the scheduler is locked. - if let Some(task) = iter::repeat_with(|| self.find_task(rt)) - .find(|s| !s.is_retry()) - .and_then(|s| s.success()) - { - self.schedule(rt, task); - continue; - } - - // If another thread is already blocked on the reactor, there is no point in keeping - // the current thread around since there is too little work to do. - if sched.polling { - break; - } - - // Take out the machine associated with the current thread. - let m = match sched - .machines - .iter() - .position(|elem| ptr::eq(&**elem, self)) - { - None => break, // The processor was stolen. - Some(pos) => sched.machines.swap_remove(pos), - }; - - // Unlock the schedule poll the reactor until new I/O events arrive. - sched.polling = true; - drop(sched); - rt.reactor.poll(None).unwrap(); - - // Lock the scheduler again and re-register the machine. - sched = rt.sched.lock().unwrap(); - sched.polling = false; - sched.machines.push(m); - - runs = 0; - fails = 0; - } - - // When shutting down the thread, take the processor out if still available. - let opt_p = self.processor.lock().take(); - - // Return the processor to the scheduler and remove the machine. - if let Some(p) = opt_p { - let mut sched = rt.sched.lock().unwrap(); - sched.processors.push(p); - sched.machines.retain(|elem| !ptr::eq(&**elem, self)); - } - } -} - -struct Processor { - /// The local task queue. - worker: Worker, - - /// Contains the next task to run as an optimization that skips the queue. - slot: Option, -} - -impl Processor { - /// Creates a new processor. - fn new() -> Processor { - Processor { - worker: Worker::new_fifo(), - slot: None, - } - } - - /// Schedules a task to run on this processor. - fn schedule(&mut self, rt: &Runtime, task: Runnable) { - match self.slot.replace(task) { - None => {} - Some(task) => { - self.worker.push(task); - rt.notify(); - } - } - } - - /// Flushes a task from the slot into the local queue. - fn flush_slot(&mut self, rt: &Runtime) { - if let Some(task) = self.slot.take() { - self.worker.push(task); - rt.notify(); - } - } - - /// Pops a task from this processor. - fn pop_task(&mut self) -> Option { - self.slot.take().or_else(|| self.worker.pop()) - } - - /// Steals a task from the global queue. - fn steal_from_global(&self, rt: &Runtime) -> Steal { - rt.injector.steal_batch_and_pop(&self.worker) - } - - /// Steals a task from other processors. - fn steal_from_others(&self, rt: &Runtime) -> Steal { - // Pick a random starting point in the list of queues. - let len = rt.stealers.len(); - let start = random(len as u32) as usize; - - // Create an iterator over stealers that starts from the chosen point. - let (l, r) = rt.stealers.split_at(start); - let stealers = r.iter().chain(l.iter()); - - // Try stealing a batch of tasks from each queue. - stealers - .map(|s| s.steal_batch_and_pop(&self.worker)) - .collect() - } -} diff --git a/src/sync/mod.rs b/src/sync/mod.rs index 1531f8c..217709b 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -174,7 +174,10 @@ #![allow(clippy::needless_doctest_main)] #[doc(inline)] -pub use std::sync::{Arc, Weak}; +pub use std::sync::Weak; + +#[doc(inline)] +pub use piper::Arc; pub use mutex::{Mutex, MutexGuard}; pub use rwlock::{RwLock, RwLockReadGuard, RwLockWriteGuard}; @@ -194,8 +197,3 @@ cfg_unstable! { pub(crate) mod waker_set; pub(crate) use waker_set::WakerSet; - -cfg_default! { - pub(crate) mod spin_lock; - pub(crate) use spin_lock::Spinlock; -} diff --git a/src/sync/spin_lock.rs b/src/sync/spin_lock.rs deleted file mode 100644 index 854b7e0..0000000 --- a/src/sync/spin_lock.rs +++ /dev/null @@ -1,89 +0,0 @@ -use std::cell::UnsafeCell; -use std::ops::{Deref, DerefMut}; -use std::sync::atomic::{AtomicBool, Ordering}; - -use crossbeam_utils::Backoff; - -/// A simple spinlock. -#[derive(Debug)] -pub struct Spinlock { - locked: AtomicBool, - value: UnsafeCell, -} - -unsafe impl Send for Spinlock {} -unsafe impl Sync for Spinlock {} - -impl Spinlock { - /// Returns a new spinlock initialized with `value`. - pub const fn new(value: T) -> Spinlock { - Spinlock { - locked: AtomicBool::new(false), - value: UnsafeCell::new(value), - } - } - - /// Locks the spinlock. - pub fn lock(&self) -> SpinlockGuard<'_, T> { - let backoff = Backoff::new(); - while self.locked.compare_and_swap(false, true, Ordering::Acquire) { - backoff.snooze(); - } - SpinlockGuard { parent: self } - } -} - -/// A guard holding a spinlock locked. -#[derive(Debug)] -pub struct SpinlockGuard<'a, T> { - parent: &'a Spinlock, -} - -unsafe impl Send for SpinlockGuard<'_, T> {} -unsafe impl Sync for SpinlockGuard<'_, T> {} - -impl<'a, T> Drop for SpinlockGuard<'a, T> { - fn drop(&mut self) { - self.parent.locked.store(false, Ordering::Release); - } -} - -impl<'a, T> Deref for SpinlockGuard<'a, T> { - type Target = T; - - fn deref(&self) -> &T { - unsafe { &*self.parent.value.get() } - } -} - -impl<'a, T> DerefMut for SpinlockGuard<'a, T> { - fn deref_mut(&mut self) -> &mut T { - unsafe { &mut *self.parent.value.get() } - } -} - -#[test] -fn spinlock() { - use std::sync::Arc; - - use crate::sync::{Spinlock}; - use crate::task; - - task::block_on(async { - - let m = Arc::new(Spinlock::new(0)); - let mut tasks = vec![]; - - for _ in 0..10 { - let m = m.clone(); - tasks.push(task::spawn(async move { - *m.lock() += 1; - })); - } - - for t in tasks { - t.await; - } - assert_eq!(*m.lock(), 10); - }) -} diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 4bade5b..41e0ca0 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -1,13 +1,8 @@ -use std::cell::Cell; use std::future::Future; -use std::mem::{self, ManuallyDrop}; -use std::sync::Arc; -use std::task::{RawWaker, RawWakerVTable}; -use crossbeam_utils::sync::Parker; use kv_log_macro::trace; -use crate::task::{Context, Poll, Task, Waker}; +use crate::task::Task; /// Spawns a task and blocks the current thread on its result. /// @@ -45,11 +40,11 @@ where parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), }); - let future = async move { + let wrapped_future = async move { // Drop task-locals on exit. - defer! { - Task::get_current(|t| unsafe { t.drop_locals() }); - } + // defer! { + // Task::get_current(|t| unsafe { t.drop_locals() }); + // } // Log completion on exit. defer! { @@ -61,70 +56,8 @@ where future.await }; - // Run the future as a task. - unsafe { Task::set_current(&task, || run(future)) } -} - -/// Blocks the current thread on a future's result. -fn run(future: F) -> T -where - F: Future, -{ - thread_local! { - // May hold a pre-allocated parker that can be reused for efficiency. - // - // Note that each invocation of `block` needs its own parker. In particular, if `block` - // recursively calls itself, we must make sure that each recursive call uses a distinct - // parker instance. - static CACHE: Cell>> = Cell::new(None); - } - - // Virtual table for wakers based on `Arc`. - static VTABLE: RawWakerVTable = { - unsafe fn clone_raw(ptr: *const ()) -> RawWaker { - let arc = ManuallyDrop::new(Arc::from_raw(ptr as *const Parker)); - #[allow(clippy::redundant_clone)] - mem::forget(arc.clone()); - RawWaker::new(ptr, &VTABLE) - } - - unsafe fn wake_raw(ptr: *const ()) { - let arc = Arc::from_raw(ptr as *const Parker); - arc.unparker().unpark(); - } - - unsafe fn wake_by_ref_raw(ptr: *const ()) { - let arc = ManuallyDrop::new(Arc::from_raw(ptr as *const Parker)); - arc.unparker().unpark(); - } - - unsafe fn drop_raw(ptr: *const ()) { - drop(Arc::from_raw(ptr as *const Parker)) - } - - RawWakerVTable::new(clone_raw, wake_raw, wake_by_ref_raw, drop_raw) - }; - - // Pin the future on the stack. - pin_utils::pin_mut!(future); - - CACHE.with(|cache| { - // Reuse a cached parker or create a new one for this invocation of `block`. - let arc_parker: Arc = cache.take().unwrap_or_else(|| Arc::new(Parker::new())); - let ptr = (&*arc_parker as *const Parker) as *const (); - - // Create a waker and task context. - let waker = unsafe { ManuallyDrop::new(Waker::from_raw(RawWaker::new(ptr, &VTABLE))) }; - let cx = &mut Context::from_waker(&waker); + once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - loop { - if let Poll::Ready(t) = future.as_mut().poll(cx) { - // Save the parker for the next invocation of `block`. - cache.set(Some(arc_parker)); - return t; - } - - arc_parker.park(); - } - }) + // Run the future as a task. + unsafe { Task::set_current(&task, || smol::block_on(wrapped_future)) } } diff --git a/src/task/builder.rs b/src/task/builder.rs index f1fef59..898308c 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -3,9 +3,7 @@ use std::future::Future; use kv_log_macro::trace; use crate::io; -use crate::rt::RUNTIME; use crate::task::{JoinHandle, Task}; -use crate::utils::abort_on_panic; /// Task builder that configures the settings of a new task. #[derive(Debug, Default)] @@ -42,11 +40,11 @@ impl Builder { parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), }); - let future = async move { + let wrapped_future = async move { // Drop task-locals on exit. - defer! { - Task::get_current(|t| unsafe { t.drop_locals() }); - } + // defer! { + // Task::get_current(|t| unsafe { t.drop_locals() }); + // } // Log completion on exit. defer! { @@ -54,25 +52,12 @@ impl Builder { task_id: Task::get_current(|t| t.id().0), }); } - future.await }; - let schedule = move |t| RUNTIME.schedule(Runnable(t)); - let (task, handle) = async_task::spawn(future, schedule, task); - task.schedule(); - Ok(JoinHandle::new(handle)) - } -} - -/// A runnable task. -pub struct Runnable(async_task::Task); + once_cell::sync::Lazy::force(&crate::rt::RUNTIME); -impl Runnable { - /// Runs the task by polling its future once. - pub fn run(self) { - unsafe { - Task::set_current(self.0.tag(), || abort_on_panic(|| self.0.run())); - } + let task = smol::Task::spawn(wrapped_future); + Ok(JoinHandle::new(task)) } } diff --git a/src/task/join_handle.rs b/src/task/join_handle.rs index d929d11..72fb2e0 100644 --- a/src/task/join_handle.rs +++ b/src/task/join_handle.rs @@ -12,11 +12,11 @@ use crate::task::{Context, Poll, Task}; /// /// [spawned]: fn.spawn.html #[derive(Debug)] -pub struct JoinHandle(async_task::JoinHandle); +pub struct JoinHandle(smol::Task); impl JoinHandle { /// Creates a new `JoinHandle`. - pub(crate) fn new(inner: async_task::JoinHandle) -> JoinHandle { + pub(crate) fn new(inner: smol::Task) -> JoinHandle { JoinHandle(inner) } @@ -36,7 +36,7 @@ impl JoinHandle { /// # /// # }) pub fn task(&self) -> &Task { - self.0.tag() + todo!() } } @@ -44,10 +44,7 @@ impl Future for JoinHandle { type Output = T; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - match Pin::new(&mut self.0).poll(cx) { - Poll::Pending => Poll::Pending, - Poll::Ready(None) => panic!("cannot await the result of a panicked task"), - Poll::Ready(Some(val)) => Poll::Ready(val), - } + dbg!("poll joinhandle"); + Pin::new(&mut self.0).poll(cx) } } diff --git a/src/task/mod.rs b/src/task/mod.rs index 56224a3..61917cd 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -141,7 +141,6 @@ cfg_default! { pub use spawn::spawn; pub use task_local::{AccessError, LocalKey}; - pub(crate) use builder::Runnable; pub(crate) use task_local::LocalsMap; mod block_on; diff --git a/src/task/spawn_blocking.rs b/src/task/spawn_blocking.rs index 27143f7..d7b4fd0 100644 --- a/src/task/spawn_blocking.rs +++ b/src/task/spawn_blocking.rs @@ -1,12 +1,4 @@ -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::thread; -use std::time::Duration; - -use crossbeam_channel::{unbounded, Receiver, Sender}; -use once_cell::sync::Lazy; - -use crate::task::{JoinHandle, Task}; -use crate::utils::abort_on_panic; +use crate::task::JoinHandle; /// Spawns a blocking task. /// @@ -43,80 +35,8 @@ where F: FnOnce() -> T + Send + 'static, T: Send + 'static, { - let schedule = |task| POOL.sender.send(task).unwrap(); - let (task, handle) = async_task::spawn(async { f() }, schedule, Task::new(None)); - task.schedule(); - JoinHandle::new(handle) -} - -type Runnable = async_task::Task; - -struct Pool { - sender: Sender, - receiver: Receiver, -} - -/// The number of sleeping worker threads. -static SLEEPING: AtomicUsize = AtomicUsize::new(0); - -static POOL: Lazy = Lazy::new(|| { - // Start a single worker thread waiting for the first task. - start_thread(); + once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - let (sender, receiver) = unbounded(); - Pool { sender, receiver } -}); - -fn start_thread() { - SLEEPING.fetch_add(1, Ordering::SeqCst); - let timeout = Duration::from_secs(1); - - thread::Builder::new() - .name("async-std/blocking".to_string()) - .spawn(move || { - loop { - let mut task = match POOL.receiver.recv_timeout(timeout) { - Ok(task) => task, - Err(_) => { - // Check whether this is the last sleeping thread. - if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { - // If so, then restart the thread to make sure there is always at least - // one sleeping thread. - if SLEEPING.compare_and_swap(0, 1, Ordering::SeqCst) == 0 { - continue; - } - } - - // Stop the thread. - return; - } - }; - - // If there are no sleeping threads, then start one to make sure there is always at - // least one sleeping thread. - if SLEEPING.fetch_sub(1, Ordering::SeqCst) == 1 { - start_thread(); - } - - loop { - // Run the task. - abort_on_panic(|| task.run()); - - // Try taking another task if there are any available. - task = match POOL.receiver.try_recv() { - Ok(task) => task, - Err(_) => break, - }; - } - - // If there is at least one sleeping thread, stop this thread instead of putting it - // to sleep. - if SLEEPING.load(Ordering::SeqCst) > 0 { - return; - } - - SLEEPING.fetch_add(1, Ordering::SeqCst); - } - }) - .expect("cannot start a blocking thread"); + let handle = smol::Task::blocking(async move { f() }); + JoinHandle::new(handle) } diff --git a/src/task/yield_now.rs b/src/task/yield_now.rs index bdb08eb..2b1fd0b 100644 --- a/src/task/yield_now.rs +++ b/src/task/yield_now.rs @@ -43,10 +43,6 @@ impl Future for YieldNow { if !self.0 { self.0 = true; cx.waker().wake_by_ref(); - - #[cfg(feature = "default")] - crate::rt::RUNTIME.yield_now(); - Poll::Pending } else { Poll::Ready(()) diff --git a/src/utils.rs b/src/utils.rs index 4bdbd92..13ee16d 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -20,40 +20,6 @@ pub fn abort_on_panic(f: impl FnOnce() -> T) -> T { t } -/// Generates a random number in `0..n`. -#[cfg(any(feature = "unstable", feature = "default"))] -pub fn random(n: u32) -> u32 { - use std::cell::Cell; - use std::num::Wrapping; - - thread_local! { - static RNG: Cell> = { - // Take the address of a local value as seed. - let mut x = 0i32; - let r = &mut x; - let addr = r as *mut i32 as usize; - Cell::new(Wrapping(addr as u32)) - } - } - - RNG.with(|rng| { - // This is the 32-bit variant of Xorshift. - // - // Source: https://en.wikipedia.org/wiki/Xorshift - let mut x = rng.get(); - x ^= x << 13; - x ^= x >> 17; - x ^= x << 5; - rng.set(x); - - // This is a fast alternative to `x % n`. - // - // Author: Daniel Lemire - // Source: https://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/ - ((u64::from(x.0)).wrapping_mul(u64::from(n)) >> 32) as u32 - }) -} - /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; diff --git a/tests/mutex.rs b/tests/mutex.rs index fd1c07b..ebdd752 100644 --- a/tests/mutex.rs +++ b/tests/mutex.rs @@ -40,24 +40,33 @@ fn contention() { let tx = Arc::new(tx); let mutex = Arc::new(Mutex::new(0)); - let num_tasks = 10000; + let num_tasks = 10; //000; + let mut handles = Vec::new(); for _ in 0..num_tasks { let tx = tx.clone(); let mutex = mutex.clone(); - task::spawn(async move { + dbg!("spawn"); + handles.push(task::spawn(async move { let mut lock = mutex.lock().await; *lock += 1; tx.unbounded_send(()).unwrap(); drop(lock); - }); + })); } - for _ in 0..num_tasks { + for i in 0..num_tasks { + dbg!(i); rx.next().await.unwrap(); } + for handle in handles.into_iter() { + handle.await; + } + + dbg!("wait"); + let lock = mutex.lock().await; assert_eq!(num_tasks, *lock); }); From fc9ee0dfdd377952eb3b93ccb81ee52bb11d25af Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 24 Apr 2020 20:08:31 +0200 Subject: [PATCH 26/60] keep std::sync::Arc --- Cargo.toml | 2 -- src/sync/mod.rs | 5 +---- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 3bc9084..45662bd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -48,7 +48,6 @@ std = [ "once_cell", "pin-utils", "slab", - "piper", ] alloc = [ "futures-core/alloc", @@ -77,7 +76,6 @@ pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } smol = { path = "../smol", optional = true } -piper = { git = "https://github.com/stjepang/piper.git", branch = "master", optional = true } [dev-dependencies] femme = "1.3.0" diff --git a/src/sync/mod.rs b/src/sync/mod.rs index 217709b..bccc6ec 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -174,10 +174,7 @@ #![allow(clippy::needless_doctest_main)] #[doc(inline)] -pub use std::sync::Weak; - -#[doc(inline)] -pub use piper::Arc; +pub use std::sync::{Arc, Weak}; pub use mutex::{Mutex, MutexGuard}; pub use rwlock::{RwLock, RwLockReadGuard, RwLockWriteGuard}; From e082634b5e4ce0e89c6107ae61be3ab5b0e95445 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 24 Apr 2020 22:53:39 +0200 Subject: [PATCH 27/60] fix spawning --- Cargo.toml | 2 +- src/task/block_on.rs | 6 +++--- src/task/builder.rs | 12 +++++++----- src/task/join_handle.rs | 29 +++++++++++++++++++++++------ src/task/spawn_blocking.rs | 6 +++--- 5 files changed, 37 insertions(+), 18 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 45662bd..b850b54 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -56,7 +56,7 @@ alloc = [ [dependencies] async-attributes = { version = "1.1.1", optional = true } -async-task = { version = "1.3.1", optional = true } +async-task = { version = "3.0.0", optional = true } broadcaster = { version = "1.0.0", optional = true } crossbeam-channel = { version = "0.4.2", optional = true } crossbeam-deque = { version = "0.7.3", optional = true } diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 41e0ca0..65d6541 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -42,9 +42,9 @@ where let wrapped_future = async move { // Drop task-locals on exit. - // defer! { - // Task::get_current(|t| unsafe { t.drop_locals() }); - // } + defer! { + Task::get_current(|t| unsafe { t.drop_locals() }); + } // Log completion on exit. defer! { diff --git a/src/task/builder.rs b/src/task/builder.rs index 898308c..de3e6ac 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -42,9 +42,9 @@ impl Builder { let wrapped_future = async move { // Drop task-locals on exit. - // defer! { - // Task::get_current(|t| unsafe { t.drop_locals() }); - // } + defer! { + Task::get_current(|t| unsafe { t.drop_locals() }); + } // Log completion on exit. defer! { @@ -57,7 +57,9 @@ impl Builder { once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - let task = smol::Task::spawn(wrapped_future); - Ok(JoinHandle::new(task)) + // FIXME: figure out how to set the current task. + + let smol_task = smol::Task::spawn(wrapped_future).detach(); + Ok(JoinHandle::new(smol_task, task)) } } diff --git a/src/task/join_handle.rs b/src/task/join_handle.rs index 72fb2e0..3a63271 100644 --- a/src/task/join_handle.rs +++ b/src/task/join_handle.rs @@ -12,12 +12,18 @@ use crate::task::{Context, Poll, Task}; /// /// [spawned]: fn.spawn.html #[derive(Debug)] -pub struct JoinHandle(smol::Task); +pub struct JoinHandle { + handle: Option>, + task: Task, +} impl JoinHandle { /// Creates a new `JoinHandle`. - pub(crate) fn new(inner: smol::Task) -> JoinHandle { - JoinHandle(inner) + pub(crate) fn new(inner: async_task::JoinHandle, task: Task) -> JoinHandle { + JoinHandle { + handle: Some(inner), + task, + } } /// Returns a handle to the underlying task. @@ -36,7 +42,14 @@ impl JoinHandle { /// # /// # }) pub fn task(&self) -> &Task { - todo!() + &self.task + } + + /// Cancel this task. + pub async fn cancel(mut self) -> Option { + let handle = self.handle.take().unwrap(); + handle.cancel(); + handle.await } } @@ -44,7 +57,11 @@ impl Future for JoinHandle { type Output = T; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - dbg!("poll joinhandle"); - Pin::new(&mut self.0).poll(cx) + match Pin::new(&mut self.handle.as_mut().unwrap()).poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(output) => { + Poll::Ready(output.expect("cannot await the result of a panicked task")) + } + } } } diff --git a/src/task/spawn_blocking.rs b/src/task/spawn_blocking.rs index d7b4fd0..054f3fd 100644 --- a/src/task/spawn_blocking.rs +++ b/src/task/spawn_blocking.rs @@ -1,4 +1,4 @@ -use crate::task::JoinHandle; +use crate::task::{JoinHandle, Task}; /// Spawns a blocking task. /// @@ -37,6 +37,6 @@ where { once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - let handle = smol::Task::blocking(async move { f() }); - JoinHandle::new(handle) + let handle = smol::Task::blocking(async move { f() }).detach(); + JoinHandle::new(handle, Task::new(None)) } From 75ab7219df9db0db8425992e238dc0c17de0d6be Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 24 Apr 2020 23:31:13 +0200 Subject: [PATCH 28/60] bring back random --- src/utils.rs | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/utils.rs b/src/utils.rs index 13ee16d..33e6604 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -20,6 +20,40 @@ pub fn abort_on_panic(f: impl FnOnce() -> T) -> T { t } +/// Generates a random number in `0..n`. +#[cfg(feature = "unstable")] +pub fn random(n: u32) -> u32 { + use std::cell::Cell; + use std::num::Wrapping; + + thread_local! { + static RNG: Cell> = { + // Take the address of a local value as seed. + let mut x = 0i32; + let r = &mut x; + let addr = r as *mut i32 as usize; + Cell::new(Wrapping(addr as u32)) + } + } + + RNG.with(|rng| { + // This is the 32-bit variant of Xorshift. + // + // Source: https://en.wikipedia.org/wiki/Xorshift + let mut x = rng.get(); + x ^= x << 13; + x ^= x >> 17; + x ^= x << 5; + rng.set(x); + + // This is a fast alternative to `x % n`. + // + // Author: Daniel Lemire + // Source: https://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/ + ((u64::from(x.0)).wrapping_mul(u64::from(n)) >> 32) as u32 + }) +} + /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; From b96afc41dc8f69cad8f9b5a61454b439612e4b4a Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 01:40:54 +0200 Subject: [PATCH 29/60] implement task locals --- src/task/block_on.rs | 34 +------ src/task/builder.rs | 91 ++++++++++++++----- src/task/current.rs | 4 +- src/task/mod.rs | 2 + src/task/task.rs | 152 +++----------------------------- src/task/task_local.rs | 4 +- src/task/task_locals_wrapper.rs | 84 ++++++++++++++++++ 7 files changed, 172 insertions(+), 199 deletions(-) create mode 100644 src/task/task_locals_wrapper.rs diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 65d6541..92a1187 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -1,8 +1,6 @@ use std::future::Future; -use kv_log_macro::trace; - -use crate::task::Task; +use crate::task::Builder; /// Spawns a task and blocks the current thread on its result. /// @@ -31,33 +29,5 @@ pub fn block_on(future: F) -> T where F: Future, { - // Create a new task handle. - let task = Task::new(None); - - // Log this `block_on` operation. - trace!("block_on", { - task_id: task.id().0, - parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), - }); - - let wrapped_future = async move { - // Drop task-locals on exit. - defer! { - Task::get_current(|t| unsafe { t.drop_locals() }); - } - - // Log completion on exit. - defer! { - trace!("completed", { - task_id: Task::get_current(|t| t.id().0), - }); - } - - future.await - }; - - once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - - // Run the future as a task. - unsafe { Task::set_current(&task, || smol::block_on(wrapped_future)) } + Builder::new().blocking(future) } diff --git a/src/task/builder.rs b/src/task/builder.rs index de3e6ac..4936d4b 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -1,9 +1,12 @@ use std::future::Future; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; use kv_log_macro::trace; use crate::io; -use crate::task::{JoinHandle, Task}; +use crate::task::{JoinHandle, Task, TaskLocalsWrapper}; /// Task builder that configures the settings of a new task. #[derive(Debug, Default)] @@ -25,41 +28,83 @@ impl Builder { self } + fn build(self, future: F) -> SupportTaskLocals + where + F: Future, + { + let name = self.name.map(Arc::new); + + // Create a new task handle. + let task = Task::new(name); + + once_cell::sync::Lazy::force(&crate::rt::RUNTIME); + + let tag = TaskLocalsWrapper::new(task.clone()); + + // FIXME: do not require all futures to be boxed. + SupportTaskLocals { + tag, + future: Box::pin(future), + } + } + /// Spawns a task with the configured settings. pub fn spawn(self, future: F) -> io::Result> where F: Future + Send + 'static, T: Send + 'static, { - // Create a new task handle. - let task = Task::new(self.name); + let wrapped = self.build(future); // Log this `spawn` operation. trace!("spawn", { - task_id: task.id().0, - parent_task_id: Task::get_current(|t| t.id().0).unwrap_or(0), + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), }); - let wrapped_future = async move { - // Drop task-locals on exit. - defer! { - Task::get_current(|t| unsafe { t.drop_locals() }); - } - - // Log completion on exit. - defer! { - trace!("completed", { - task_id: Task::get_current(|t| t.id().0), - }); - } - future.await - }; + let task = wrapped.tag.task().clone(); + let smol_task = smol::Task::spawn(wrapped).detach(); - once_cell::sync::Lazy::force(&crate::rt::RUNTIME); + Ok(JoinHandle::new(smol_task, task)) + } - // FIXME: figure out how to set the current task. + /// Spawns a task with the configured settings, blocking on its execution. + pub fn blocking(self, future: F) -> T + where + F: Future, + { + let wrapped = self.build(future); - let smol_task = smol::Task::spawn(wrapped_future).detach(); - Ok(JoinHandle::new(smol_task, task)) + // Log this `block_on` operation. + trace!("block_on", { + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), + }); + + // Run the future as a task. + unsafe { TaskLocalsWrapper::set_current(&wrapped.tag, || smol::block_on(wrapped)) } + } +} + +/// Wrapper to add support for task locals. +struct SupportTaskLocals { + tag: TaskLocalsWrapper, + future: Pin>, +} + +impl Drop for SupportTaskLocals { + fn drop(&mut self) { + // Log completion on exit. + trace!("completed", { + task_id: self.tag.id().0, + }); + } +} + +impl Future for SupportTaskLocals { + type Output = F::Output; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + unsafe { TaskLocalsWrapper::set_current(&self.tag, || Pin::new(&mut self.future).poll(cx)) } } } diff --git a/src/task/current.rs b/src/task/current.rs index 0dc3699..e4624e1 100644 --- a/src/task/current.rs +++ b/src/task/current.rs @@ -1,4 +1,4 @@ -use crate::task::Task; +use crate::task::{Task, TaskLocalsWrapper}; /// Returns a handle to the current task. /// @@ -23,6 +23,6 @@ use crate::task::Task; /// # }) /// ``` pub fn current() -> Task { - Task::get_current(|t| t.clone()) + TaskLocalsWrapper::get_current(|t| t.task().clone()) .expect("`task::current()` called outside the context of a task") } diff --git a/src/task/mod.rs b/src/task/mod.rs index 61917cd..d4fccea 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -142,6 +142,7 @@ cfg_default! { pub use task_local::{AccessError, LocalKey}; pub(crate) use task_local::LocalsMap; + pub(crate) use task_locals_wrapper::TaskLocalsWrapper; mod block_on; mod builder; @@ -153,6 +154,7 @@ cfg_default! { mod task; mod task_id; mod task_local; + mod task_locals_wrapper; #[cfg(any(feature = "unstable", test))] pub use spawn_blocking::spawn_blocking; diff --git a/src/task/task.rs b/src/task/task.rs index bcec2e0..eba99c7 100644 --- a/src/task/task.rs +++ b/src/task/task.rs @@ -1,74 +1,32 @@ -use std::cell::Cell; use std::fmt; -use std::mem::ManuallyDrop; -use std::ptr; -use std::sync::atomic::{AtomicPtr, Ordering}; use std::sync::Arc; -use crate::task::{LocalsMap, TaskId}; -use crate::utils::abort_on_panic; +use crate::task::TaskId; -thread_local! { - /// A pointer to the currently running task. - static CURRENT: Cell<*const Task> = Cell::new(ptr::null_mut()); -} - -/// The inner representation of a task handle. -struct Inner { +/// A handle to a task. +#[derive(Clone)] +pub struct Task { /// The task ID. id: TaskId, /// The optional task name. - name: Option>, - - /// The map holding task-local values. - locals: LocalsMap, -} - -impl Inner { - #[inline] - fn new(name: Option) -> Inner { - Inner { - id: TaskId::generate(), - name: name.map(String::into_boxed_str), - locals: LocalsMap::new(), - } - } + name: Option>, } -/// A handle to a task. -pub struct Task { - /// The inner representation. - /// - /// This pointer is lazily initialized on first use. In most cases, the inner representation is - /// never touched and therefore we don't allocate it unless it's really needed. - inner: AtomicPtr, -} - -unsafe impl Send for Task {} -unsafe impl Sync for Task {} - impl Task { /// Creates a new task handle. - /// - /// If the task is unnamed, the inner representation of the task will be lazily allocated on - /// demand. #[inline] - pub(crate) fn new(name: Option) -> Task { - let inner = match name { - None => AtomicPtr::default(), - Some(name) => { - let raw = Arc::into_raw(Arc::new(Inner::new(Some(name)))); - AtomicPtr::new(raw as *mut Inner) - } - }; - Task { inner } + pub(crate) fn new(name: Option>) -> Task { + Task { + id: TaskId::generate(), + name, + } } /// Gets the task's unique identifier. #[inline] pub fn id(&self) -> TaskId { - self.inner().id + self.id } /// Returns the name of this task. @@ -77,93 +35,7 @@ impl Task { /// /// [`Builder::name`]: struct.Builder.html#method.name pub fn name(&self) -> Option<&str> { - self.inner().name.as_ref().map(|s| &**s) - } - - /// Returns the map holding task-local values. - pub(crate) fn locals(&self) -> &LocalsMap { - &self.inner().locals - } - - /// Drops all task-local values. - /// - /// This method is only safe to call at the end of the task. - #[inline] - pub(crate) unsafe fn drop_locals(&self) { - let raw = self.inner.load(Ordering::Acquire); - if let Some(inner) = raw.as_mut() { - // Abort the process if dropping task-locals panics. - abort_on_panic(|| { - inner.locals.clear(); - }); - } - } - - /// Returns the inner representation, initializing it on first use. - fn inner(&self) -> &Inner { - loop { - let raw = self.inner.load(Ordering::Acquire); - if !raw.is_null() { - return unsafe { &*raw }; - } - - let new = Arc::into_raw(Arc::new(Inner::new(None))) as *mut Inner; - if self.inner.compare_and_swap(raw, new, Ordering::AcqRel) != raw { - unsafe { - drop(Arc::from_raw(new)); - } - } - } - } - - /// Set a reference to the current task. - pub(crate) unsafe fn set_current(task: *const Task, f: F) -> R - where - F: FnOnce() -> R, - { - CURRENT.with(|current| { - let old_task = current.replace(task); - defer! { - current.set(old_task); - } - f() - }) - } - - /// Gets a reference to the current task. - pub(crate) fn get_current(f: F) -> Option - where - F: FnOnce(&Task) -> R, - { - let res = CURRENT.try_with(|current| unsafe { current.get().as_ref().map(f) }); - match res { - Ok(Some(val)) => Some(val), - Ok(None) | Err(_) => None, - } - } -} - -impl Drop for Task { - fn drop(&mut self) { - // Deallocate the inner representation if it was initialized. - let raw = *self.inner.get_mut(); - if !raw.is_null() { - unsafe { - drop(Arc::from_raw(raw)); - } - } - } -} - -impl Clone for Task { - fn clone(&self) -> Task { - // We need to make sure the inner representation is initialized now so that this instance - // and the clone have raw pointers that point to the same `Arc`. - let arc = unsafe { ManuallyDrop::new(Arc::from_raw(self.inner())) }; - let raw = Arc::into_raw(Arc::clone(&arc)); - Task { - inner: AtomicPtr::new(raw as *mut Inner), - } + self.name.as_ref().map(|s| s.as_str()) } } diff --git a/src/task/task_local.rs b/src/task/task_local.rs index 72e53d7..4e2ba83 100644 --- a/src/task/task_local.rs +++ b/src/task/task_local.rs @@ -3,7 +3,7 @@ use std::error::Error; use std::fmt; use std::sync::atomic::{AtomicU32, Ordering}; -use crate::task::Task; +use crate::task::TaskLocalsWrapper; /// The key for accessing a task-local value. /// @@ -98,7 +98,7 @@ impl LocalKey { where F: FnOnce(&T) -> R, { - Task::get_current(|task| unsafe { + TaskLocalsWrapper::get_current(|task| unsafe { // Prepare the numeric key, initialization function, and the map of task-locals. let key = self.key(); let init = || Box::new((self.__init)()) as Box; diff --git a/src/task/task_locals_wrapper.rs b/src/task/task_locals_wrapper.rs new file mode 100644 index 0000000..2a7ddb7 --- /dev/null +++ b/src/task/task_locals_wrapper.rs @@ -0,0 +1,84 @@ +use std::cell::Cell; +use std::ptr; + +use crate::task::{LocalsMap, Task, TaskId}; +use crate::utils::abort_on_panic; + +thread_local! { + /// A pointer to the currently running task. + static CURRENT: Cell<*const TaskLocalsWrapper> = Cell::new(ptr::null_mut()); +} + +/// A wrapper to store task local data. +pub(crate) struct TaskLocalsWrapper { + /// The actual task details. + task: Task, + + /// The map holding task-local values. + locals: LocalsMap, +} + +impl TaskLocalsWrapper { + /// Creates a new task handle. + /// + /// If the task is unnamed, the inner representation of the task will be lazily allocated on + /// demand. + #[inline] + pub(crate) fn new(task: Task) -> Self { + Self { + task, + locals: LocalsMap::new(), + } + } + + /// Gets the task's unique identifier. + #[inline] + pub fn id(&self) -> TaskId { + self.task.id() + } + + /// Returns a reference to the inner `Task`. + pub(crate) fn task(&self) -> &Task { + &self.task + } + + /// Returns the map holding task-local values. + pub(crate) fn locals(&self) -> &LocalsMap { + &self.locals + } + + /// Set a reference to the current task. + pub(crate) unsafe fn set_current(task: *const TaskLocalsWrapper, f: F) -> R + where + F: FnOnce() -> R, + { + CURRENT.with(|current| { + let old_task = current.replace(task); + defer! { + current.set(old_task); + } + f() + }) + } + + /// Gets a reference to the current task. + pub(crate) fn get_current(f: F) -> Option + where + F: FnOnce(&TaskLocalsWrapper) -> R, + { + let res = CURRENT.try_with(|current| unsafe { current.get().as_ref().map(f) }); + match res { + Ok(Some(val)) => Some(val), + Ok(None) | Err(_) => None, + } + } +} + +impl Drop for TaskLocalsWrapper { + fn drop(&mut self) { + // Abort the process if dropping task-locals panics. + abort_on_panic(|| { + unsafe { self.locals.clear() }; + }); + } +} From f5fa0d7e4e2a40bc07d4c916c9524c7970b61071 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 16:02:32 +0200 Subject: [PATCH 30/60] avoid boxing futures --- src/task/builder.rs | 39 +++++++++++++++------------------------ 1 file changed, 15 insertions(+), 24 deletions(-) diff --git a/src/task/builder.rs b/src/task/builder.rs index 4936d4b..aa9f0c0 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -4,6 +4,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use kv_log_macro::trace; +use pin_project_lite::pin_project; use crate::io; use crate::task::{JoinHandle, Task, TaskLocalsWrapper}; @@ -42,10 +43,7 @@ impl Builder { let tag = TaskLocalsWrapper::new(task.clone()); // FIXME: do not require all futures to be boxed. - SupportTaskLocals { - tag, - future: Box::pin(future), - } + SupportTaskLocals { tag, future } } /// Spawns a task with the configured settings. @@ -56,12 +54,6 @@ impl Builder { { let wrapped = self.build(future); - // Log this `spawn` operation. - trace!("spawn", { - task_id: wrapped.tag.id().0, - parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), - }); - let task = wrapped.tag.task().clone(); let smol_task = smol::Task::spawn(wrapped).detach(); @@ -86,25 +78,24 @@ impl Builder { } } -/// Wrapper to add support for task locals. -struct SupportTaskLocals { - tag: TaskLocalsWrapper, - future: Pin>, -} - -impl Drop for SupportTaskLocals { - fn drop(&mut self) { - // Log completion on exit. - trace!("completed", { - task_id: self.tag.id().0, - }); +pin_project! { + /// Wrapper to add support for task locals. + struct SupportTaskLocals { + tag: TaskLocalsWrapper, + #[pin] + future: F, } } impl Future for SupportTaskLocals { type Output = F::Output; - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - unsafe { TaskLocalsWrapper::set_current(&self.tag, || Pin::new(&mut self.future).poll(cx)) } + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + unsafe { + TaskLocalsWrapper::set_current(&self.tag, || { + let this = self.project(); + this.future.poll(cx) + }) + } } } From ab9d6554aaf5c95b070a377ffc3f1a0b4b034986 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 17:07:10 +0200 Subject: [PATCH 31/60] switch to smol::Timer --- Cargo.toml | 4 +--- src/future/future/delay.rs | 6 +++--- src/future/timeout.rs | 17 ++++++++--------- src/io/timeout.rs | 8 ++++---- src/os/unix/net/datagram.rs | 4 ++-- src/stream/interval.rs | 10 +++++----- src/stream/stream/delay.rs | 5 +++-- src/stream/stream/throttle.rs | 8 ++++---- src/stream/stream/timeout.rs | 6 +++--- 9 files changed, 33 insertions(+), 35 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index b850b54..404e5fb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,7 +27,6 @@ default = [ "crossbeam-channel", "crossbeam-deque", "crossbeam-queue", - "futures-timer", "kv-log-macro", "log", "mio", @@ -37,7 +36,7 @@ default = [ "smol", ] docs = ["attributes", "unstable", "default"] -unstable = ["std", "broadcaster", "futures-timer"] +unstable = ["std", "broadcaster"] attributes = ["async-attributes"] std = [ "alloc", @@ -64,7 +63,6 @@ crossbeam-queue = { version = "0.2.0", optional = true } crossbeam-utils = { version = "0.7.2", optional = true } futures-core = { version = "0.3.4", optional = true, default-features = false } futures-io = { version = "0.3.4", optional = true } -futures-timer = { version = "3.0.2", optional = true } kv-log-macro = { version = "1.0.4", optional = true } log = { version = "0.4.8", features = ["kv_unstable"], optional = true } memchr = { version = "2.3.3", optional = true } diff --git a/src/future/future/delay.rs b/src/future/future/delay.rs index 641084f..e194470 100644 --- a/src/future/future/delay.rs +++ b/src/future/future/delay.rs @@ -2,8 +2,8 @@ use std::future::Future; use std::pin::Pin; use std::time::Duration; -use futures_timer::Delay; use pin_project_lite::pin_project; +use smol::Timer; use crate::task::{Context, Poll}; @@ -14,13 +14,13 @@ pin_project! { #[pin] future: F, #[pin] - delay: Delay, + delay: Timer, } } impl DelayFuture { pub fn new(future: F, dur: Duration) -> DelayFuture { - let delay = Delay::new(dur); + let delay = Timer::after(dur); DelayFuture { future, delay } } diff --git a/src/future/timeout.rs b/src/future/timeout.rs index 05aaa45..ec547f8 100644 --- a/src/future/timeout.rs +++ b/src/future/timeout.rs @@ -1,11 +1,11 @@ use std::error::Error; use std::fmt; +use std::future::Future; use std::pin::Pin; use std::time::Duration; -use std::future::Future; -use futures_timer::Delay; use pin_project_lite::pin_project; +use smol::Timer; use crate::task::{Context, Poll}; @@ -33,11 +33,7 @@ pub async fn timeout(dur: Duration, f: F) -> Result where F: Future, { - let f = TimeoutFuture { - future: f, - delay: Delay::new(dur), - }; - f.await + TimeoutFuture::new(f, dur).await } pin_project! { @@ -46,14 +42,17 @@ pin_project! { #[pin] future: F, #[pin] - delay: Delay, + delay: Timer, } } impl TimeoutFuture { #[allow(dead_code)] pub(super) fn new(future: F, dur: Duration) -> TimeoutFuture { - TimeoutFuture { future: future, delay: Delay::new(dur) } + TimeoutFuture { + future, + delay: Timer::after(dur), + } } } diff --git a/src/io/timeout.rs b/src/io/timeout.rs index 6e22dbf..c19d25d 100644 --- a/src/io/timeout.rs +++ b/src/io/timeout.rs @@ -1,10 +1,10 @@ +use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; use std::time::Duration; -use std::future::Future; -use futures_timer::Delay; use pin_project_lite::pin_project; +use smol::Timer; use crate::io; @@ -37,7 +37,7 @@ where F: Future>, { Timeout { - timeout: Delay::new(dur), + timeout: Timer::after(dur), future: f, } .await @@ -53,7 +53,7 @@ pin_project! { #[pin] future: F, #[pin] - timeout: Delay, + timeout: Timer, } } diff --git a/src/os/unix/net/datagram.rs b/src/os/unix/net/datagram.rs index 6a98736..c73c9ce 100644 --- a/src/os/unix/net/datagram.rs +++ b/src/os/unix/net/datagram.rs @@ -319,8 +319,8 @@ impl AsRawFd for UnixDatagram { impl FromRawFd for UnixDatagram { unsafe fn from_raw_fd(fd: RawFd) -> UnixDatagram { - let datagram = std::os::unix::net::UnixDatagram::from_raw_fd(fd); - datagram.into() + let datagram = Async::::from_raw_fd(fd); + UnixDatagram { watcher: datagram } } } diff --git a/src/stream/interval.rs b/src/stream/interval.rs index be94b06..0161240 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -1,10 +1,10 @@ +use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; use std::time::Duration; -use crate::future::Future; use crate::stream::Stream; -use futures_timer::Delay; +use smol::Timer; /// Creates a new stream that yields at a set interval. /// @@ -45,7 +45,7 @@ use futures_timer::Delay; #[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub fn interval(dur: Duration) -> Interval { Interval { - delay: Delay::new(dur), + delay: Timer::after(dur), interval: dur, } } @@ -60,7 +60,7 @@ pub fn interval(dur: Duration) -> Interval { #[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Interval { - delay: Delay, + delay: Timer, interval: Duration, } @@ -72,7 +72,7 @@ impl Stream for Interval { return Poll::Pending; } let interval = self.interval; - self.delay.reset(interval); + std::mem::replace(&mut self.delay, Timer::after(interval)); Poll::Ready(Some(())) } } diff --git a/src/stream/stream/delay.rs b/src/stream/stream/delay.rs index ff4c937..754bef8 100644 --- a/src/stream/stream/delay.rs +++ b/src/stream/stream/delay.rs @@ -3,6 +3,7 @@ use core::pin::Pin; use core::time::Duration; use pin_project_lite::pin_project; +use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; @@ -14,7 +15,7 @@ pin_project! { #[pin] stream: S, #[pin] - delay: futures_timer::Delay, + delay: Timer, delay_done: bool, } } @@ -23,7 +24,7 @@ impl Delay { pub(super) fn new(stream: S, dur: Duration) -> Self { Delay { stream, - delay: futures_timer::Delay::new(dur), + delay: Timer::after(dur), delay_done: false, } } diff --git a/src/stream/stream/throttle.rs b/src/stream/stream/throttle.rs index 554ca30..15a0f31 100644 --- a/src/stream/stream/throttle.rs +++ b/src/stream/stream/throttle.rs @@ -2,8 +2,8 @@ use std::future::Future; use std::pin::Pin; use std::time::Duration; -use futures_timer::Delay; use pin_project_lite::pin_project; +use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; @@ -25,7 +25,7 @@ pin_project! { #[pin] blocked: bool, #[pin] - delay: Delay, + delay: Timer, } } @@ -35,7 +35,7 @@ impl Throttle { stream, duration, blocked: false, - delay: Delay::new(Duration::default()), + delay: Timer::after(Duration::default()), } } } @@ -59,7 +59,7 @@ impl Stream for Throttle { Poll::Ready(None) => Poll::Ready(None), Poll::Ready(Some(v)) => { *this.blocked = true; - this.delay.reset(*this.duration); + std::mem::replace(&mut *this.delay, Timer::after(*this.duration)); Poll::Ready(Some(v)) } } diff --git a/src/stream/stream/timeout.rs b/src/stream/stream/timeout.rs index ce658c8..f49aed3 100644 --- a/src/stream/stream/timeout.rs +++ b/src/stream/stream/timeout.rs @@ -4,8 +4,8 @@ use std::future::Future; use std::pin::Pin; use std::time::Duration; -use futures_timer::Delay; use pin_project_lite::pin_project; +use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; @@ -17,13 +17,13 @@ pin_project! { #[pin] stream: S, #[pin] - delay: Delay, + delay: Timer, } } impl Timeout { pub(crate) fn new(stream: S, dur: Duration) -> Self { - let delay = Delay::new(dur); + let delay = Timer::after(dur); Self { stream, delay } } From fd6ae40817e42031d19ff53ef74eaf9da5727c01 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 17:36:20 +0200 Subject: [PATCH 32/60] add timeout stress test --- tests/timeout.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 tests/timeout.rs diff --git a/tests/timeout.rs b/tests/timeout.rs new file mode 100644 index 0000000..c9694f8 --- /dev/null +++ b/tests/timeout.rs @@ -0,0 +1,22 @@ +use std::time::Duration; + +use async_std::future::timeout; +use async_std::task; + +#[test] +fn timeout_future_many() { + task::block_on(async { + let futures = (0..100) + .map(|i| { + timeout(Duration::from_millis(i * 10), async move { + task::sleep(Duration::from_millis(i)).await; + Ok::<(), async_std::future::TimeoutError>(()) + }) + }) + .collect::>(); + + for future in futures { + future.await.unwrap().unwrap(); + } + }); +} From 10c8b9a6d893608828e304b386dfa7b86b65b158 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 17:38:41 +0200 Subject: [PATCH 33/60] silence must use --- src/stream/interval.rs | 2 +- src/stream/stream/throttle.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/stream/interval.rs b/src/stream/interval.rs index 0161240..fe249fb 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -72,7 +72,7 @@ impl Stream for Interval { return Poll::Pending; } let interval = self.interval; - std::mem::replace(&mut self.delay, Timer::after(interval)); + let _ = std::mem::replace(&mut self.delay, Timer::after(interval)); Poll::Ready(Some(())) } } diff --git a/src/stream/stream/throttle.rs b/src/stream/stream/throttle.rs index 15a0f31..4d4cc87 100644 --- a/src/stream/stream/throttle.rs +++ b/src/stream/stream/throttle.rs @@ -59,7 +59,7 @@ impl Stream for Throttle { Poll::Ready(None) => Poll::Ready(None), Poll::Ready(Some(v)) => { *this.blocked = true; - std::mem::replace(&mut *this.delay, Timer::after(*this.duration)); + let _ = std::mem::replace(&mut *this.delay, Timer::after(*this.duration)); Poll::Ready(Some(v)) } } From 0a7a52aed50ff288e81ae861e97928a8ac280436 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 19:05:05 +0200 Subject: [PATCH 34/60] update to work on smol/master --- src/os/unix/net/datagram.rs | 3 ++- src/task/builder.rs | 2 +- src/task/spawn_blocking.rs | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/os/unix/net/datagram.rs b/src/os/unix/net/datagram.rs index c73c9ce..6a30b02 100644 --- a/src/os/unix/net/datagram.rs +++ b/src/os/unix/net/datagram.rs @@ -319,7 +319,8 @@ impl AsRawFd for UnixDatagram { impl FromRawFd for UnixDatagram { unsafe fn from_raw_fd(fd: RawFd) -> UnixDatagram { - let datagram = Async::::from_raw_fd(fd); + let raw = StdUnixDatagram::from_raw_fd(fd); + let datagram = Async::::new(raw).expect("invalid file descriptor"); UnixDatagram { watcher: datagram } } } diff --git a/src/task/builder.rs b/src/task/builder.rs index aa9f0c0..51a5898 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -55,7 +55,7 @@ impl Builder { let wrapped = self.build(future); let task = wrapped.tag.task().clone(); - let smol_task = smol::Task::spawn(wrapped).detach(); + let smol_task = smol::Task::spawn(wrapped).into(); Ok(JoinHandle::new(smol_task, task)) } diff --git a/src/task/spawn_blocking.rs b/src/task/spawn_blocking.rs index 054f3fd..e9ed0c5 100644 --- a/src/task/spawn_blocking.rs +++ b/src/task/spawn_blocking.rs @@ -37,6 +37,6 @@ where { once_cell::sync::Lazy::force(&crate::rt::RUNTIME); - let handle = smol::Task::blocking(async move { f() }).detach(); + let handle = smol::Task::blocking(async move { f() }).into(); JoinHandle::new(handle, Task::new(None)) } From 228cc59b3bcd341b5a8cfc11cca3a8cd18866f98 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 23:00:00 +0200 Subject: [PATCH 35/60] feat: add spawn_local --- src/task/builder.rs | 14 ++++++++++++++ src/task/mod.rs | 2 ++ src/task/spawn_local.rs | 32 ++++++++++++++++++++++++++++++++ 3 files changed, 48 insertions(+) create mode 100644 src/task/spawn_local.rs diff --git a/src/task/builder.rs b/src/task/builder.rs index 51a5898..f1bf791 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -60,6 +60,20 @@ impl Builder { Ok(JoinHandle::new(smol_task, task)) } + /// Spawns a task locally with the configured settings. + pub fn local(self, future: F) -> io::Result> + where + F: Future + 'static, + T: 'static, + { + let wrapped = self.build(future); + + let task = wrapped.tag.task().clone(); + let smol_task = smol::Task::local(wrapped).into(); + + Ok(JoinHandle::new(smol_task, task)) + } + /// Spawns a task with the configured settings, blocking on its execution. pub fn blocking(self, future: F) -> T where diff --git a/src/task/mod.rs b/src/task/mod.rs index d4fccea..f5bc864 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -139,6 +139,7 @@ cfg_default! { pub use join_handle::JoinHandle; pub use sleep::sleep; pub use spawn::spawn; + pub use spawn_local::spawn_local; pub use task_local::{AccessError, LocalKey}; pub(crate) use task_local::LocalsMap; @@ -151,6 +152,7 @@ cfg_default! { mod sleep; mod spawn; mod spawn_blocking; + mod spawn_local; mod task; mod task_id; mod task_local; diff --git a/src/task/spawn_local.rs b/src/task/spawn_local.rs new file mode 100644 index 0000000..5ed7226 --- /dev/null +++ b/src/task/spawn_local.rs @@ -0,0 +1,32 @@ +use std::future::Future; + +use crate::task::{Builder, JoinHandle}; + +/// Spawns a task onto the thread-local executor. +/// +/// This function is similar to [`std::thread::spawn`], except it spawns an asynchronous task. +/// +/// [`std::thread`]: https://doc.rust-lang.org/std/thread/fn.spawn.html +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::task; +/// +/// let handle = task::spawn_local(async { +/// 1 + 2 +/// }); +/// +/// assert_eq!(handle.await, 3); +/// # +/// # }) +/// ``` +pub fn spawn_local(future: F) -> JoinHandle +where + F: Future + 'static, + T: 'static, +{ + Builder::new().local(future).expect("cannot spawn task") +} From 3161a4e449d4d4ec0e536dff74da93d923dd177f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 25 Apr 2020 23:09:40 +0200 Subject: [PATCH 36/60] add some missing windows imports --- src/net/tcp/listener.rs | 2 +- src/net/tcp/stream.rs | 9 ++++++--- src/net/udp/mod.rs | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index 290da0d..f31f135 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -229,7 +229,7 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, AsRawSocket, RawSocket, FromRawSocket}; impl AsRawSocket for TcpListener { fn as_raw_socket(&self) -> RawSocket { diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 1d2d0ce..0dc43f5 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -388,7 +388,7 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, RawSocket, AsRawSocket, FromRawSocket, IntoRawSocket}; impl AsRawSocket for TcpStream { fn as_raw_socket(&self) -> RawSocket { @@ -402,9 +402,12 @@ cfg_windows! { } } - impl IntoRawSocket for TcpListener { + impl IntoRawSocket for crate::net::tcp::TcpListener { fn into_raw_socket(self) -> RawSocket { - self.raw_socket + // TODO(stjepang): This does not mean `RawFd` is now the sole owner of the file + // descriptor because it's possible that there are other clones of this `TcpStream` + // using it at the same time. We should probably document that behavior. + self.as_raw_socket() } } } diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 3bc9ad7..53add8e 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -482,7 +482,7 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; + use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, RawSocket, AsRawSocket, IntoRawSocket, FromRawSocket}; impl AsRawSocket for UdpSocket { fn as_raw_socket(&self) -> RawSocket { From 2cd2ba3530fc75b8cff0b6ad542fec6dbd176031 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sun, 26 Apr 2020 15:29:45 +0200 Subject: [PATCH 37/60] remove unused dependencies --- Cargo.toml | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 404e5fb..dffdc5c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,13 +24,8 @@ rustdoc-args = ["--cfg", "feature=\"docs\""] default = [ "std", "async-task", - "crossbeam-channel", - "crossbeam-deque", - "crossbeam-queue", "kv-log-macro", "log", - "mio", - "mio-uds", "num_cpus", "pin-project-lite", "smol", @@ -57,17 +52,12 @@ alloc = [ async-attributes = { version = "1.1.1", optional = true } async-task = { version = "3.0.0", optional = true } broadcaster = { version = "1.0.0", optional = true } -crossbeam-channel = { version = "0.4.2", optional = true } -crossbeam-deque = { version = "0.7.3", optional = true } -crossbeam-queue = { version = "0.2.0", optional = true } crossbeam-utils = { version = "0.7.2", optional = true } futures-core = { version = "0.3.4", optional = true, default-features = false } futures-io = { version = "0.3.4", optional = true } kv-log-macro = { version = "1.0.4", optional = true } log = { version = "0.4.8", features = ["kv_unstable"], optional = true } memchr = { version = "2.3.3", optional = true } -mio = { version = "0.6.19", optional = true } -mio-uds = { version = "0.6.7", optional = true } num_cpus = { version = "1.12.0", optional = true } once_cell = { version = "1.3.1", optional = true } pin-project-lite = { version = "0.1.4", optional = true } From e4df1405c1a04b9e4a65f878b2bb1a86f855e986 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sun, 26 Apr 2020 18:00:00 +0200 Subject: [PATCH 38/60] feat: add basic wasm support --- .github/workflows/ci.yml | 8 ++++++ Cargo.toml | 11 ++++++++ src/future/future/delay.rs | 2 +- src/future/timeout.rs | 2 +- src/io/mod.rs | 11 ++++++++ src/io/read/mod.rs | 11 ++++---- src/io/read/take.rs | 2 +- src/io/timeout.rs | 2 +- src/lib.rs | 3 +++ src/net/mod.rs | 6 +++++ src/path/path.rs | 12 +++++++-- src/stream/interval.rs | 2 +- src/stream/stream/delay.rs | 2 +- src/stream/stream/throttle.rs | 2 +- src/stream/stream/timeout.rs | 2 +- src/sync/barrier.rs | 2 +- src/task/block_on.rs | 11 ++++++++ src/task/builder.rs | 29 ++++++++++++++++++--- src/task/join_handle.rs | 18 ++++++++++++-- src/task/mod.rs | 5 ++++ src/utils.rs | 31 +++++++++++++++++++++++ tests/addr.rs | 2 ++ tests/block_on.rs | 2 ++ tests/buf_writer.rs | 24 +++++++++++------- tests/channel.rs | 47 ++++++++++++++++++++++++----------- tests/condvar.rs | 16 ++++++++++-- tests/io_timeout.rs | 1 + tests/mutex.rs | 21 ++++++++++++---- tests/rwlock.rs | 25 +++++++++++++++---- tests/stream.rs | 20 +++++++++++---- tests/task_local.rs | 11 +++++++- tests/tcp.rs | 2 ++ tests/timeout.rs | 4 +++ tests/udp.rs | 2 ++ tests/uds.rs | 2 +- tests/verbose_errors.rs | 2 ++ wasm-test.sh | 10 ++++++++ 37 files changed, 301 insertions(+), 64 deletions(-) create mode 100755 wasm-test.sh diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e9fcdcc..8f519e5 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -58,6 +58,14 @@ jobs: with: command: check args: --features unstable --all --bins --examples --tests + + - name: check wasm + uses: actions-rs/cargo@v1 + with: + command: check + target: wasm32-unknown-unknown + override: true + args: --features unstable --all --bins --tests - name: check bench uses: actions-rs/cargo@v1 diff --git a/Cargo.toml b/Cargo.toml index dffdc5c..e74a2ed 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -63,14 +63,25 @@ once_cell = { version = "1.3.1", optional = true } pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } + +[target.'cfg(not(target_os = "unknown"))'.dependencies] smol = { path = "../smol", optional = true } +[target.'cfg(target_arch = "wasm32")'.dependencies] +wasm-timer = "0.2.4" +wasm-bindgen-futures = "0.4.10" +futures-channel = "0.3.4" + +[target.'cfg(target_arch = "wasm32")'.dev-dependencies] +wasm-bindgen-test = "0.3.10" + [dev-dependencies] femme = "1.3.0" rand = "0.7.3" surf = "1.0.3" tempdir = "0.3.7" futures = "0.3.4" +rand_xorshift = "0.2.0" [[test]] name = "stream" diff --git a/src/future/future/delay.rs b/src/future/future/delay.rs index e194470..b6c30bc 100644 --- a/src/future/future/delay.rs +++ b/src/future/future/delay.rs @@ -3,9 +3,9 @@ use std::pin::Pin; use std::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::task::{Context, Poll}; +use crate::utils::Timer; pin_project! { #[doc(hidden)] diff --git a/src/future/timeout.rs b/src/future/timeout.rs index ec547f8..4a9d93c 100644 --- a/src/future/timeout.rs +++ b/src/future/timeout.rs @@ -5,9 +5,9 @@ use std::pin::Pin; use std::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::task::{Context, Poll}; +use crate::utils::Timer; /// Awaits a future or times out after a duration of time. /// diff --git a/src/io/mod.rs b/src/io/mod.rs index dd97567..f5dd9e2 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -307,22 +307,33 @@ cfg_std! { cfg_default! { // For use in the print macros. #[doc(hidden)] + #[cfg(not(target_os = "unknown"))] pub use stdio::{_eprint, _print}; + #[cfg(not(target_os = "unknown"))] pub use stderr::{stderr, Stderr}; + #[cfg(not(target_os = "unknown"))] pub use stdin::{stdin, Stdin}; + #[cfg(not(target_os = "unknown"))] pub use stdout::{stdout, Stdout}; pub use timeout::timeout; mod timeout; + #[cfg(not(target_os = "unknown"))] mod stderr; + #[cfg(not(target_os = "unknown"))] mod stdin; + #[cfg(not(target_os = "unknown"))] mod stdio; + #[cfg(not(target_os = "unknown"))] mod stdout; } cfg_unstable_default! { + #[cfg(not(target_os = "unknown"))] pub use stderr::StderrLock; + #[cfg(not(target_os = "unknown"))] pub use stdin::StdinLock; + #[cfg(not(target_os = "unknown"))] pub use stdout::StdoutLock; } diff --git a/src/io/read/mod.rs b/src/io/read/mod.rs index 8aade18..0d42920 100644 --- a/src/io/read/mod.rs +++ b/src/io/read/mod.rs @@ -17,9 +17,9 @@ use std::mem; use crate::io::IoSliceMut; -pub use take::Take; pub use bytes::Bytes; pub use chain::Chain; +pub use take::Take; extension_trait! { use std::pin::Pin; @@ -483,7 +483,7 @@ mod tests { use crate::prelude::*; #[test] - fn test_read_by_ref() -> io::Result<()> { + fn test_read_by_ref() { crate::task::block_on(async { let mut f = io::Cursor::new(vec![0u8, 1, 2, 3, 4, 5, 6, 7, 8]); let mut buffer = Vec::new(); @@ -493,14 +493,13 @@ mod tests { let reference = f.by_ref(); // read at most 5 bytes - assert_eq!(reference.take(5).read_to_end(&mut buffer).await?, 5); + assert_eq!(reference.take(5).read_to_end(&mut buffer).await.unwrap(), 5); assert_eq!(&buffer, &[0, 1, 2, 3, 4]) } // drop our &mut reference so we can use f again // original file still usable, read the rest - assert_eq!(f.read_to_end(&mut other_buffer).await?, 4); + assert_eq!(f.read_to_end(&mut other_buffer).await.unwrap(), 4); assert_eq!(&other_buffer, &[5, 6, 7, 8]); - Ok(()) - }) + }); } } diff --git a/src/io/read/take.rs b/src/io/read/take.rs index 09b02c2..ba9a9e3 100644 --- a/src/io/read/take.rs +++ b/src/io/read/take.rs @@ -218,7 +218,7 @@ impl BufRead for Take { } } -#[cfg(test)] +#[cfg(all(test, not(target_os = "unknown")))] mod tests { use crate::io; use crate::prelude::*; diff --git a/src/io/timeout.rs b/src/io/timeout.rs index c19d25d..ce33fea 100644 --- a/src/io/timeout.rs +++ b/src/io/timeout.rs @@ -4,9 +4,9 @@ use std::task::{Context, Poll}; use std::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::io; +use crate::utils::Timer; /// Awaits an I/O future or times out after a duration of time. /// diff --git a/src/lib.rs b/src/lib.rs index 7e0e98d..408a7ab 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -267,14 +267,17 @@ cfg_std! { } cfg_default! { + #[cfg(not(target_os = "unknown"))] pub mod fs; pub mod path; pub mod net; + #[cfg(not(target_os = "unknown"))] pub(crate) mod rt; } cfg_unstable! { pub mod pin; + #[cfg(not(target_os = "unknown"))] pub mod process; mod unit; diff --git a/src/net/mod.rs b/src/net/mod.rs index fe83d3b..1814073 100644 --- a/src/net/mod.rs +++ b/src/net/mod.rs @@ -61,10 +61,16 @@ pub use std::net::Shutdown; pub use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; pub use std::net::{SocketAddr, SocketAddrV4, SocketAddrV6}; +#[cfg(not(target_os = "unknown"))] pub use addr::ToSocketAddrs; +#[cfg(not(target_os = "unknown"))] pub use tcp::{Incoming, TcpListener, TcpStream}; +#[cfg(not(target_os = "unknown"))] pub use udp::UdpSocket; +#[cfg(not(target_os = "unknown"))] mod addr; +#[cfg(not(target_os = "unknown"))] mod tcp; +#[cfg(not(target_os = "unknown"))] mod udp; diff --git a/src/path/path.rs b/src/path/path.rs index dfe9426..185bfaf 100644 --- a/src/path/path.rs +++ b/src/path/path.rs @@ -4,9 +4,9 @@ use std::ffi::{OsStr, OsString}; use std::rc::Rc; use std::sync::Arc; -use crate::fs; -use crate::io; use crate::path::{Ancestors, Components, Display, Iter, PathBuf, StripPrefixError}; +#[cfg(not(target_os = "unknown"))] +use crate::{fs, io}; /// A slice of a path. /// @@ -584,6 +584,7 @@ impl Path { /// # /// # Ok(()) }) } /// ``` + #[cfg(not(target_os = "unknown"))] pub async fn metadata(&self) -> io::Result { fs::metadata(self).await } @@ -607,6 +608,7 @@ impl Path { /// # /// # Ok(()) }) } /// ``` + #[cfg(not(target_os = "unknown"))] pub async fn symlink_metadata(&self) -> io::Result { fs::symlink_metadata(self).await } @@ -632,6 +634,7 @@ impl Path { /// # /// # Ok(()) }) } /// ``` + #[cfg(not(target_os = "unknown"))] pub async fn canonicalize(&self) -> io::Result { fs::canonicalize(self).await } @@ -654,6 +657,7 @@ impl Path { /// # /// # Ok(()) }) } /// ``` + #[cfg(not(target_os = "unknown"))] pub async fn read_link(&self) -> io::Result { fs::read_link(self).await } @@ -688,6 +692,7 @@ impl Path { /// # /// # Ok(()) }) } /// ``` + #[cfg(not(target_os = "unknown"))] pub async fn read_dir(&self) -> io::Result { fs::read_dir(self).await } @@ -717,6 +722,7 @@ impl Path { /// check errors, call [fs::metadata]. /// /// [fs::metadata]: ../fs/fn.metadata.html + #[cfg(not(target_os = "unknown"))] pub async fn exists(&self) -> bool { fs::metadata(self).await.is_ok() } @@ -749,6 +755,7 @@ impl Path { /// /// [fs::metadata]: ../fs/fn.metadata.html /// [fs::Metadata::is_file]: ../fs/struct.Metadata.html#method.is_file + #[cfg(not(target_os = "unknown"))] pub async fn is_file(&self) -> bool { fs::metadata(self) .await @@ -785,6 +792,7 @@ impl Path { /// /// [fs::metadata]: ../fs/fn.metadata.html /// [fs::Metadata::is_dir]: ../fs/struct.Metadata.html#method.is_dir + #[cfg(not(target_os = "unknown"))] pub async fn is_dir(&self) -> bool { fs::metadata(self) .await diff --git a/src/stream/interval.rs b/src/stream/interval.rs index fe249fb..4e5c92b 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -4,7 +4,7 @@ use std::task::{Context, Poll}; use std::time::Duration; use crate::stream::Stream; -use smol::Timer; +use crate::utils::Timer; /// Creates a new stream that yields at a set interval. /// diff --git a/src/stream/stream/delay.rs b/src/stream/stream/delay.rs index 754bef8..0ba42b0 100644 --- a/src/stream/stream/delay.rs +++ b/src/stream/stream/delay.rs @@ -3,10 +3,10 @@ use core::pin::Pin; use core::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; +use crate::utils::Timer; pin_project! { #[doc(hidden)] diff --git a/src/stream/stream/throttle.rs b/src/stream/stream/throttle.rs index 4d4cc87..2f9333a 100644 --- a/src/stream/stream/throttle.rs +++ b/src/stream/stream/throttle.rs @@ -3,10 +3,10 @@ use std::pin::Pin; use std::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; +use crate::utils::Timer; pin_project! { /// A stream that only yields one element once every `duration`. diff --git a/src/stream/stream/timeout.rs b/src/stream/stream/timeout.rs index f49aed3..28e52ae 100644 --- a/src/stream/stream/timeout.rs +++ b/src/stream/stream/timeout.rs @@ -5,10 +5,10 @@ use std::pin::Pin; use std::time::Duration; use pin_project_lite::pin_project; -use smol::Timer; use crate::stream::Stream; use crate::task::{Context, Poll}; +use crate::utils::Timer; pin_project! { /// A stream with timeout time set diff --git a/src/sync/barrier.rs b/src/sync/barrier.rs index 2822d54..86e9a2d 100644 --- a/src/sync/barrier.rs +++ b/src/sync/barrier.rs @@ -202,7 +202,7 @@ impl BarrierWaitResult { } } -#[cfg(test)] +#[cfg(all(test, not(target_os = "unknown")))] mod test { use futures::channel::mpsc::unbounded; use futures::sink::SinkExt; diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 92a1187..fa66f91 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -25,9 +25,20 @@ use crate::task::Builder; /// }) /// } /// ``` +#[cfg(not(target_os = "unknown"))] pub fn block_on(future: F) -> T where F: Future, { Builder::new().blocking(future) } + +/// Spawns a task and waits for it to finish. +#[cfg(target_os = "unknown")] +pub fn block_on(future: F) +where + F: Future + 'static, + T: 'static, +{ + Builder::new().local(future).unwrap(); +} diff --git a/src/task/builder.rs b/src/task/builder.rs index f1bf791..f48b6b4 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -3,7 +3,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use kv_log_macro::trace; use pin_project_lite::pin_project; use crate::io; @@ -38,15 +37,16 @@ impl Builder { // Create a new task handle. let task = Task::new(name); + #[cfg(not(target_os = "unknown"))] once_cell::sync::Lazy::force(&crate::rt::RUNTIME); let tag = TaskLocalsWrapper::new(task.clone()); - // FIXME: do not require all futures to be boxed. SupportTaskLocals { tag, future } } /// Spawns a task with the configured settings. + #[cfg(not(target_os = "unknown"))] pub fn spawn(self, future: F) -> io::Result> where F: Future + Send + 'static, @@ -61,6 +61,7 @@ impl Builder { } /// Spawns a task locally with the configured settings. + #[cfg(not(target_os = "unknown"))] pub fn local(self, future: F) -> io::Result> where F: Future + 'static, @@ -74,7 +75,29 @@ impl Builder { Ok(JoinHandle::new(smol_task, task)) } + /// Spawns a task locally with the configured settings. + #[cfg(target_arch = "wasm32")] + pub fn local(self, future: F) -> io::Result> + where + F: Future + 'static, + T: 'static, + { + use futures_channel::oneshot::channel; + let (sender, receiver) = channel(); + + let wrapped = self.build(async move { + let res = future.await; + let _ = sender.send(res); + }); + + let task = wrapped.tag.task().clone(); + wasm_bindgen_futures::spawn_local(wrapped); + + Ok(JoinHandle::new(receiver, task)) + } + /// Spawns a task with the configured settings, blocking on its execution. + #[cfg(not(target_os = "unknown"))] pub fn blocking(self, future: F) -> T where F: Future, @@ -82,7 +105,7 @@ impl Builder { let wrapped = self.build(future); // Log this `block_on` operation. - trace!("block_on", { + kv_log_macro::trace!("block_on", { task_id: wrapped.tag.id().0, parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), }); diff --git a/src/task/join_handle.rs b/src/task/join_handle.rs index 3a63271..110b827 100644 --- a/src/task/join_handle.rs +++ b/src/task/join_handle.rs @@ -13,13 +13,18 @@ use crate::task::{Context, Poll, Task}; /// [spawned]: fn.spawn.html #[derive(Debug)] pub struct JoinHandle { - handle: Option>, + handle: Option>, task: Task, } +#[cfg(not(target_os = "unknown"))] +type InnerHandle = async_task::JoinHandle; +#[cfg(target_arch = "wasm32")] +type InnerHandle = futures_channel::oneshot::Receiver; + impl JoinHandle { /// Creates a new `JoinHandle`. - pub(crate) fn new(inner: async_task::JoinHandle, task: Task) -> JoinHandle { + pub(crate) fn new(inner: InnerHandle, task: Task) -> JoinHandle { JoinHandle { handle: Some(inner), task, @@ -46,11 +51,20 @@ impl JoinHandle { } /// Cancel this task. + #[cfg(not(target_os = "unknown"))] pub async fn cancel(mut self) -> Option { let handle = self.handle.take().unwrap(); handle.cancel(); handle.await } + + /// Cancel this task. + #[cfg(target_arch = "wasm32")] + pub async fn cancel(mut self) -> Option { + let mut handle = self.handle.take().unwrap(); + handle.close(); + handle.await.ok() + } } impl Future for JoinHandle { diff --git a/src/task/mod.rs b/src/task/mod.rs index f5bc864..6a142ff 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -138,6 +138,7 @@ cfg_default! { pub use task_id::TaskId; pub use join_handle::JoinHandle; pub use sleep::sleep; + #[cfg(not(target_os = "unknown"))] pub use spawn::spawn; pub use spawn_local::spawn_local; pub use task_local::{AccessError, LocalKey}; @@ -150,7 +151,9 @@ cfg_default! { mod current; mod join_handle; mod sleep; + #[cfg(not(target_os = "unknown"))] mod spawn; + #[cfg(not(target_os = "unknown"))] mod spawn_blocking; mod spawn_local; mod task; @@ -158,8 +161,10 @@ cfg_default! { mod task_local; mod task_locals_wrapper; + #[cfg(not(target_os = "unknown"))] #[cfg(any(feature = "unstable", test))] pub use spawn_blocking::spawn_blocking; + #[cfg(not(target_os = "unknown"))] #[cfg(not(any(feature = "unstable", test)))] pub(crate) use spawn_blocking::spawn_blocking; } diff --git a/src/utils.rs b/src/utils.rs index 33e6604..2ae5488 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -59,6 +59,37 @@ pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; } +#[cfg(not(target_os = "unknown"))] +pub(crate) type Timer = smol::Timer; + +#[cfg(target_arch = "wasm32")] +#[derive(Debug)] +pub(crate) struct Timer(wasm_timer::Delay); + +#[cfg(target_arch = "wasm32")] +impl Timer { + pub(crate) fn after(dur: std::time::Duration) -> Self { + Timer(wasm_timer::Delay::new(dur)) + } +} + +#[cfg(target_arch = "wasm32")] +use std::pin::Pin; +#[cfg(target_arch = "wasm32")] +use std::task::Poll; + +#[cfg(target_arch = "wasm32")] +impl std::future::Future for Timer { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, cx: &mut std::task::Context<'_>) -> Poll { + match Pin::new(&mut self.0).poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(_) => Poll::Ready(()), + } + } +} + /// Defers evaluation of a block of code until the end of the scope. #[cfg(feature = "default")] #[doc(hidden)] diff --git a/tests/addr.rs b/tests/addr.rs index aada557..fcd5aa1 100644 --- a/tests/addr.rs +++ b/tests/addr.rs @@ -1,3 +1,5 @@ +#![cfg(not(target_os = "unknown"))] + use std::net::{Ipv4Addr, Ipv6Addr, SocketAddr, SocketAddrV4, SocketAddrV6}; use async_std::net::ToSocketAddrs; diff --git a/tests/block_on.rs b/tests/block_on.rs index c422d06..28902b0 100644 --- a/tests/block_on.rs +++ b/tests/block_on.rs @@ -1,3 +1,5 @@ +#![cfg(not(target_os = "unknown"))] + use async_std::task; #[test] diff --git a/tests/buf_writer.rs b/tests/buf_writer.rs index 5df90e0..442cf8a 100644 --- a/tests/buf_writer.rs +++ b/tests/buf_writer.rs @@ -2,15 +2,19 @@ use async_std::io::{self, BufWriter, SeekFrom}; use async_std::prelude::*; use async_std::task; +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn test_buffered_writer() { #![allow(clippy::cognitive_complexity)] task::block_on(async { - let inner = Vec::new(); - let mut writer = BufWriter::with_capacity(2, inner); + let inner: Vec = Vec::new(); + let mut writer = BufWriter::>::with_capacity(2, inner); writer.write(&[0, 1]).await.unwrap(); - assert_eq!(writer.buffer(), []); + assert!(writer.buffer().is_empty()); assert_eq!(*writer.get_ref(), [0, 1]); writer.write(&[2]).await.unwrap(); @@ -22,7 +26,7 @@ fn test_buffered_writer() { assert_eq!(*writer.get_ref(), [0, 1]); writer.flush().await.unwrap(); - assert_eq!(writer.buffer(), []); + assert!(writer.buffer().is_empty()); assert_eq!(*writer.get_ref(), [0, 1, 2, 3]); writer.write(&[4]).await.unwrap(); @@ -35,31 +39,33 @@ fn test_buffered_writer() { assert_eq!(*writer.get_ref(), [0, 1, 2, 3, 4, 5]); writer.write(&[7, 8]).await.unwrap(); - assert_eq!(writer.buffer(), []); + assert!(writer.buffer().is_empty()); assert_eq!(*writer.get_ref(), [0, 1, 2, 3, 4, 5, 6, 7, 8]); writer.write(&[9, 10, 11]).await.unwrap(); - assert_eq!(writer.buffer(), []); + assert!(writer.buffer().is_empty()); assert_eq!(*writer.get_ref(), [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]); writer.flush().await.unwrap(); - assert_eq!(writer.buffer(), []); + assert!(writer.buffer().is_empty()); assert_eq!(*writer.get_ref(), [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]); }) } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn test_buffered_writer_inner_into_inner_flushes() { task::block_on(async { - let mut w = BufWriter::with_capacity(3, Vec::new()); + let mut w = BufWriter::with_capacity(3, Vec::::new()); w.write(&[0, 1]).await.unwrap(); - assert_eq!(*w.get_ref(), []); + assert!(w.get_ref().is_empty()); let w = w.into_inner().await.unwrap(); assert_eq!(w, [0, 1]); }) } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn test_buffered_writer_seek() { task::block_on(async { let mut w = BufWriter::with_capacity(3, io::Cursor::new(Vec::new())); diff --git a/tests/channel.rs b/tests/channel.rs index f302906..a218ea2 100644 --- a/tests/channel.rs +++ b/tests/channel.rs @@ -6,13 +6,22 @@ use std::time::Duration; use async_std::sync::channel; use async_std::task; -use rand::{thread_rng, Rng}; +use rand::{Rng, SeedableRng}; + +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); fn ms(ms: u64) -> Duration { Duration::from_millis(ms) } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn smoke() { task::block_on(async { let (s, r) = channel(1); @@ -35,6 +44,7 @@ fn smoke() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn capacity() { for i in 1..10 { let (s, r) = channel::<()>(i); @@ -44,6 +54,7 @@ fn capacity() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn len_empty_full() { #![allow(clippy::cognitive_complexity)] task::block_on(async { @@ -86,11 +97,12 @@ fn len_empty_full() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn recv() { task::block_on(async { let (s, r) = channel(100); - task::spawn(async move { + spawn(async move { assert_eq!(r.recv().await.unwrap(), 7); task::sleep(ms(1000)).await; assert_eq!(r.recv().await.unwrap(), 8); @@ -107,11 +119,12 @@ fn recv() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn send() { task::block_on(async { let (s, r) = channel(1); - task::spawn(async move { + spawn(async move { s.send(7).await; task::sleep(ms(1000)).await; s.send(8).await; @@ -129,6 +142,7 @@ fn send() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn recv_after_disconnect() { task::block_on(async { let (s, r) = channel(100); @@ -147,6 +161,7 @@ fn recv_after_disconnect() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn len() { const COUNT: usize = 25_000; const CAP: usize = 1000; @@ -184,7 +199,7 @@ fn len() { assert_eq!(s.len(), 0); assert_eq!(r.len(), 0); - let child = task::spawn({ + let child = spawn({ let r = r.clone(); async move { for i in 0..COUNT { @@ -209,11 +224,12 @@ fn len() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn disconnect_wakes_receiver() { task::block_on(async { let (s, r) = channel::<()>(1); - let child = task::spawn(async move { + let child = spawn(async move { assert!(r.recv().await.is_err()); }); @@ -225,13 +241,14 @@ fn disconnect_wakes_receiver() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn spsc() { const COUNT: usize = 100_000; task::block_on(async { let (s, r) = channel(3); - let child = task::spawn(async move { + let child = spawn(async move { for i in 0..COUNT { assert_eq!(r.recv().await.unwrap(), i); } @@ -248,6 +265,7 @@ fn spsc() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn mpmc() { const COUNT: usize = 25_000; const TASKS: usize = 4; @@ -262,7 +280,7 @@ fn mpmc() { for _ in 0..TASKS { let r = r.clone(); let v = v.clone(); - tasks.push(task::spawn(async move { + tasks.push(spawn(async move { for _ in 0..COUNT { let n = r.recv().await.unwrap(); v[n].fetch_add(1, Ordering::SeqCst); @@ -272,7 +290,7 @@ fn mpmc() { for _ in 0..TASKS { let s = s.clone(); - tasks.push(task::spawn(async move { + tasks.push(spawn(async move { for i in 0..COUNT { s.send(i).await; } @@ -290,6 +308,7 @@ fn mpmc() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn oneshot() { const COUNT: usize = 10_000; @@ -297,8 +316,8 @@ fn oneshot() { for _ in 0..COUNT { let (s, r) = channel(1); - let c1 = task::spawn(async move { r.recv().await.unwrap() }); - let c2 = task::spawn(async move { s.send(0).await }); + let c1 = spawn(async move { r.recv().await.unwrap() }); + let c2 = spawn(async move { s.send(0).await }); c1.await; c2.await; @@ -307,6 +326,7 @@ fn oneshot() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn drops() { const RUNS: usize = 100; @@ -321,17 +341,16 @@ fn drops() { } } - let mut rng = thread_rng(); - for _ in 0..RUNS { - task::block_on(async { + let mut rng = rand_xorshift::XorShiftRng::seed_from_u64(0); + task::block_on(async move { let steps = rng.gen_range(0, 10_000); let additional = rng.gen_range(0, 50); DROPS.store(0, Ordering::SeqCst); let (s, r) = channel::(50); - let child = task::spawn({ + let child = spawn({ let r = r.clone(); async move { for _ in 0..steps { diff --git a/tests/condvar.rs b/tests/condvar.rs index c4d680f..7b05b28 100644 --- a/tests/condvar.rs +++ b/tests/condvar.rs @@ -5,13 +5,22 @@ use std::time::Duration; use async_std::sync::{Condvar, Mutex}; use async_std::task::{self, JoinHandle}; +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn wait_timeout_with_lock() { task::block_on(async { let pair = Arc::new((Mutex::new(false), Condvar::new())); let pair2 = pair.clone(); - task::spawn(async move { + spawn(async move { let (m, c) = &*pair2; let _g = m.lock().await; task::sleep(Duration::from_millis(20)).await; @@ -27,6 +36,7 @@ fn wait_timeout_with_lock() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn wait_timeout_without_lock() { task::block_on(async { let m = Mutex::new(false); @@ -40,6 +50,7 @@ fn wait_timeout_without_lock() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn wait_timeout_until_timed_out() { task::block_on(async { let m = Mutex::new(false); @@ -55,6 +66,7 @@ fn wait_timeout_until_timed_out() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn notify_all() { task::block_on(async { let mut tasks: Vec> = Vec::new(); @@ -62,7 +74,7 @@ fn notify_all() { for _ in 0..10 { let pair = pair.clone(); - tasks.push(task::spawn(async move { + tasks.push(spawn(async move { let (m, c) = &*pair; let mut count = m.lock().await; while *count == 0 { diff --git a/tests/io_timeout.rs b/tests/io_timeout.rs index 85a17ab..fa30a68 100644 --- a/tests/io_timeout.rs +++ b/tests/io_timeout.rs @@ -5,6 +5,7 @@ use async_std::task; #[test] #[should_panic(expected = "timed out")] +#[cfg(not(target_os = "unknown"))] fn io_timeout_timedout() { task::block_on(async { io::timeout(Duration::from_secs(1), async { diff --git a/tests/mutex.rs b/tests/mutex.rs index ebdd752..76f42e2 100644 --- a/tests/mutex.rs +++ b/tests/mutex.rs @@ -5,7 +5,16 @@ use async_std::sync::Mutex; use async_std::task; use futures::channel::mpsc; +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn smoke() { task::block_on(async { let m = Mutex::new(()); @@ -15,18 +24,21 @@ fn smoke() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn try_lock() { let m = Mutex::new(()); *m.try_lock().unwrap() = (); } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn into_inner() { let m = Mutex::new(10); assert_eq!(m.into_inner(), 10); } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn get_mut() { let mut m = Mutex::new(10); *m.get_mut() = 20; @@ -34,21 +46,21 @@ fn get_mut() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn contention() { task::block_on(async { let (tx, mut rx) = mpsc::unbounded(); let tx = Arc::new(tx); let mutex = Arc::new(Mutex::new(0)); - let num_tasks = 10; //000; + let num_tasks = 10000; let mut handles = Vec::new(); for _ in 0..num_tasks { let tx = tx.clone(); let mutex = mutex.clone(); - dbg!("spawn"); - handles.push(task::spawn(async move { + handles.push(spawn(async move { let mut lock = mutex.lock().await; *lock += 1; tx.unbounded_send(()).unwrap(); @@ -56,8 +68,7 @@ fn contention() { })); } - for i in 0..num_tasks { - dbg!(i); + for _ in 0..num_tasks { rx.next().await.unwrap(); } diff --git a/tests/rwlock.rs b/tests/rwlock.rs index 370dcb9..1d33a45 100644 --- a/tests/rwlock.rs +++ b/tests/rwlock.rs @@ -10,6 +10,14 @@ use async_std::sync::RwLock; use async_std::task; use futures::channel::mpsc; +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + /// Generates a random number in `0..n`. pub fn random(n: u32) -> u32 { thread_local! { @@ -35,6 +43,7 @@ pub fn random(n: u32) -> u32 { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn smoke() { task::block_on(async { let lock = RwLock::new(()); @@ -46,6 +55,7 @@ fn smoke() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn try_write() { task::block_on(async { let lock = RwLock::new(0isize); @@ -56,12 +66,14 @@ fn try_write() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn into_inner() { let lock = RwLock::new(10); assert_eq!(lock.into_inner(), 10); } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn into_inner_and_drop() { struct Counter(Arc); @@ -84,6 +96,7 @@ fn into_inner_and_drop() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn get_mut() { let mut lock = RwLock::new(10); *lock.get_mut() = 20; @@ -91,6 +104,7 @@ fn get_mut() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn contention() { const N: u32 = 10; const M: usize = 1000; @@ -104,7 +118,7 @@ fn contention() { let tx = tx.clone(); let rw = rw.clone(); - task::spawn(async move { + spawn(async move { for _ in 0..M { if random(N) == 0 { drop(rw.write().await); @@ -116,7 +130,7 @@ fn contention() { }); } - task::block_on(async { + task::block_on(async move { for _ in 0..N { rx.next().await.unwrap(); } @@ -124,6 +138,7 @@ fn contention() { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn writer_and_readers() { #[derive(Default)] struct Yield(Cell); @@ -146,7 +161,7 @@ fn writer_and_readers() { let (tx, mut rx) = mpsc::unbounded(); // Spawn a writer task. - task::spawn({ + spawn({ let lock = lock.clone(); async move { let mut lock = lock.write().await; @@ -164,13 +179,13 @@ fn writer_and_readers() { let mut readers = Vec::new(); for _ in 0..5 { let lock = lock.clone(); - readers.push(task::spawn(async move { + readers.push(spawn(async move { let lock = lock.read().await; assert!(*lock >= 0); })); } - task::block_on(async { + task::block_on(async move { // Wait for readers to pass their asserts. for r in readers { r.await; diff --git a/tests/stream.rs b/tests/stream.rs index 42a6191..3576cb9 100644 --- a/tests/stream.rs +++ b/tests/stream.rs @@ -8,14 +8,23 @@ use async_std::stream; use async_std::sync::channel; use async_std::task; +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] /// Checks that streams are merged fully even if one of the components /// experiences delay. fn merging_delayed_streams_work() { let (sender, receiver) = channel::(10); let mut s = receiver.merge(stream::empty()); - let t = task::spawn(async move { + let t = spawn(async move { let mut xs = Vec::new(); while let Some(x) = s.next().await { xs.push(x); @@ -34,7 +43,7 @@ fn merging_delayed_streams_work() { let (sender, receiver) = channel::(10); let mut s = stream::empty().merge(receiver); - let t = task::spawn(async move { + let t = spawn(async move { let mut xs = Vec::new(); while let Some(x) = s.next().await { xs.push(x); @@ -85,16 +94,17 @@ fn explode(s: S) -> Explode { } #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn merge_works_with_unfused_streams() { let s1 = explode(stream::once(92)); let s2 = explode(stream::once(92)); let mut s = s1.merge(s2); - let xs = task::block_on(async move { + + task::block_on(async move { let mut xs = Vec::new(); while let Some(x) = s.next().await { xs.push(x) } - xs + assert_eq!(xs, vec![92, 92]); }); - assert_eq!(xs, vec![92, 92]); } diff --git a/tests/task_local.rs b/tests/task_local.rs index 813185c..b5345fe 100644 --- a/tests/task_local.rs +++ b/tests/task_local.rs @@ -3,7 +3,16 @@ use std::sync::atomic::{AtomicBool, Ordering}; use async_std::task; use async_std::task_local; +#[cfg(not(target_os = "unknown"))] +use async_std::task::spawn; +#[cfg(target_os = "unknown")] +use async_std::task::spawn_local as spawn; + +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn drop_local() { static DROP_LOCAL: AtomicBool = AtomicBool::new(false); @@ -20,7 +29,7 @@ fn drop_local() { } // Spawn a task that just touches its task-local. - let handle = task::spawn(async { + let handle = spawn(async { LOCAL.with(|_| ()); }); let task = handle.task().clone(); diff --git a/tests/tcp.rs b/tests/tcp.rs index d92cff0..f21737e 100644 --- a/tests/tcp.rs +++ b/tests/tcp.rs @@ -1,3 +1,5 @@ +#![cfg(not(target_os = "unknown"))] + use async_std::io; use async_std::net::{TcpListener, TcpStream}; use async_std::prelude::*; diff --git a/tests/timeout.rs b/tests/timeout.rs index c9694f8..8ad358a 100644 --- a/tests/timeout.rs +++ b/tests/timeout.rs @@ -3,7 +3,11 @@ use std::time::Duration; use async_std::future::timeout; use async_std::task; +#[cfg(target_arch = "wasm32")] +wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); + #[test] +#[cfg_attr(target_arch = "wasm32", wasm_bindgen_test::wasm_bindgen_test)] fn timeout_future_many() { task::block_on(async { let futures = (0..100) diff --git a/tests/udp.rs b/tests/udp.rs index 319dc74..15404f8 100644 --- a/tests/udp.rs +++ b/tests/udp.rs @@ -1,3 +1,5 @@ +#![cfg(not(target_os = "unknown"))] + use async_std::io; use async_std::net::UdpSocket; use async_std::task; diff --git a/tests/uds.rs b/tests/uds.rs index 3ab4d6b..038ac0e 100644 --- a/tests/uds.rs +++ b/tests/uds.rs @@ -1,4 +1,4 @@ -#![cfg(unix)] +#![cfg(all(unix, not(target_os = "unknown")))] use async_std::io; use async_std::os::unix::net::{UnixDatagram, UnixListener, UnixStream}; diff --git a/tests/verbose_errors.rs b/tests/verbose_errors.rs index 17d4261..2876183 100644 --- a/tests/verbose_errors.rs +++ b/tests/verbose_errors.rs @@ -1,3 +1,5 @@ +#![cfg(not(target_os = "unknown"))] + use async_std::{fs, io, net::ToSocketAddrs, task}; #[test] diff --git a/wasm-test.sh b/wasm-test.sh new file mode 100755 index 0000000..3d8be9f --- /dev/null +++ b/wasm-test.sh @@ -0,0 +1,10 @@ +#!/bin/sh + +wasm-pack test --chrome --headless -- --features unstable --test buf_writer +wasm-pack test --chrome --headless -- --features unstable --test channel +wasm-pack test --chrome --headless -- --features unstable --test condvar +wasm-pack test --chrome --headless -- --features unstable --test mutex +wasm-pack test --chrome --headless -- --features unstable --test rwlock +wasm-pack test --chrome --headless -- --features unstable --test stream +wasm-pack test --chrome --headless -- --features unstable --test task_local +wasm-pack test --chrome --headless -- --features unstable --test timeout From 804a52b7fd538d1a6e6ee0bc9aefd100549a7826 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sun, 26 Apr 2020 20:49:33 +0200 Subject: [PATCH 39/60] use published smol --- Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index e74a2ed..5b59bee 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,6 +4,7 @@ version = "1.5.0" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", + "Friedel Ziegelmayer ", "Contributors to async-std", ] edition = "2018" @@ -65,7 +66,7 @@ pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } [target.'cfg(not(target_os = "unknown"))'.dependencies] -smol = { path = "../smol", optional = true } +smol = { version = "0.1", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] wasm-timer = "0.2.4" From 48dd683535b33a8b950e594257ff7808e491bb8a Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sun, 26 Apr 2020 21:02:01 +0200 Subject: [PATCH 40/60] fix feature settings --- Cargo.toml | 2 +- src/utils.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 5b59bee..4cdc5cf 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -29,7 +29,6 @@ default = [ "log", "num_cpus", "pin-project-lite", - "smol", ] docs = ["attributes", "unstable", "default"] unstable = ["std", "broadcaster"] @@ -43,6 +42,7 @@ std = [ "once_cell", "pin-utils", "slab", + "smol", ] alloc = [ "futures-core/alloc", diff --git a/src/utils.rs b/src/utils.rs index 2ae5488..ef068cb 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -59,14 +59,14 @@ pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; } -#[cfg(not(target_os = "unknown"))] +#[cfg(all(not(target_os = "unknown"), feature = "default"))] pub(crate) type Timer = smol::Timer; -#[cfg(target_arch = "wasm32")] +#[cfg(all(target_arch = "wasm32", feature = "default"))] #[derive(Debug)] pub(crate) struct Timer(wasm_timer::Delay); -#[cfg(target_arch = "wasm32")] +#[cfg(all(target_arch = "wasm32", feature = "default"))] impl Timer { pub(crate) fn after(dur: std::time::Duration) -> Self { Timer(wasm_timer::Delay::new(dur)) From 280b1a4344514dd20205c983e6d72b3b14ea2457 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Mon, 27 Apr 2020 11:11:16 +0200 Subject: [PATCH 41/60] remove invalid doc comment --- src/task/spawn_local.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/task/spawn_local.rs b/src/task/spawn_local.rs index 5ed7226..04da020 100644 --- a/src/task/spawn_local.rs +++ b/src/task/spawn_local.rs @@ -4,10 +4,6 @@ use crate::task::{Builder, JoinHandle}; /// Spawns a task onto the thread-local executor. /// -/// This function is similar to [`std::thread::spawn`], except it spawns an asynchronous task. -/// -/// [`std::thread`]: https://doc.rust-lang.org/std/thread/fn.spawn.html -/// /// # Examples /// /// ``` From 7a9afbd81c856a84a8940ff5c26e5e360afe4d06 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Mon, 27 Apr 2020 12:17:12 +0200 Subject: [PATCH 42/60] update smol --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 4cdc5cf..57084ab 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -66,7 +66,7 @@ pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } [target.'cfg(not(target_os = "unknown"))'.dependencies] -smol = { version = "0.1", optional = true } +smol = { version = "0.1.1", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] wasm-timer = "0.2.4" From 1a6d4f6a2f589e01ffe33facbf98d091056e2d07 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Mon, 27 Apr 2020 22:05:08 +0200 Subject: [PATCH 43/60] fix windows trait declarations for rawsocket --- src/net/tcp/listener.rs | 5 ++++- src/net/tcp/stream.rs | 2 +- src/os/windows/io.rs | 32 +++++++++++++++++++++++++++++++- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index f31f135..49ee4f4 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -229,7 +229,10 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, AsRawSocket, RawSocket, FromRawSocket}; + use crate::os::windows::io::{ + AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, + AsRawSocket, FromRawSocket, IntoRawSocket, RawSocket, + }; impl AsRawSocket for TcpListener { fn as_raw_socket(&self) -> RawSocket { diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 0dc43f5..7b71f98 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -402,7 +402,7 @@ cfg_windows! { } } - impl IntoRawSocket for crate::net::tcp::TcpListener { + impl IntoRawSocket for TcpStream { fn into_raw_socket(self) -> RawSocket { // TODO(stjepang): This does not mean `RawFd` is now the sole owner of the file // descriptor because it's possible that there are other clones of this `TcpStream` diff --git a/src/os/windows/io.rs b/src/os/windows/io.rs index e83d557..30d37a0 100644 --- a/src/os/windows/io.rs +++ b/src/os/windows/io.rs @@ -2,7 +2,8 @@ cfg_not_docs! { pub use std::os::windows::io::{ - AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, RawSocket, + AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, + AsRawSocket, FromRawSocket, IntoRawSocket, RawSocket, }; } @@ -45,4 +46,33 @@ cfg_docs! { /// it once it's no longer needed. fn into_raw_handle(self) -> RawHandle; } + + /// Creates I/O objects from raw sockets. + pub trait FromRawSocket { + /// Creates a new I/O object from the given raw socket. + /// + /// This function will consume ownership of the socket provided and it will be closed when the returned object goes out of scope. + /// + /// This function is also unsafe as the primitives currently returned have the contract that they are the sole owner of the + /// file descriptor they are wrapping. Usage of this function could accidentally allow violating this contract which can cause + /// memory unsafety in code that relies on it being true. + unsafe fn from_raw_socket(sock: RawSocket) -> Self; + } + + /// Extracts raw sockets. + pub trait AsRawSocket { + /// Extracts the underlying raw socket from this object. + fn as_raw_socket(&self) -> RawSocket; + } + + /// A trait to express the ability to consume an object and acquire ownership of + /// its raw `SOCKET`. + pub trait IntoRawSocket { + /// Consumes this object, returning the raw underlying socket. + /// + /// This function **transfers ownership** of the underlying socket to the + /// caller. Callers are then the unique owners of the socket and must close + /// it once it's no longer needed. + fn into_raw_socket(self) -> RawSocket; + } } From 92532612b75d1e2d1ff93440f760e5718acb1824 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Mon, 27 Apr 2020 22:15:04 +0200 Subject: [PATCH 44/60] mark spawn_local unstable --- src/io/read/mod.rs | 2 +- src/task/builder.rs | 25 +++++++++++++++++++++++-- src/task/mod.rs | 8 ++++++-- 3 files changed, 30 insertions(+), 5 deletions(-) diff --git a/src/io/read/mod.rs b/src/io/read/mod.rs index 0d42920..388237c 100644 --- a/src/io/read/mod.rs +++ b/src/io/read/mod.rs @@ -477,7 +477,7 @@ unsafe fn initialize(_reader: &R, buf: &mut [u8]) { std::ptr::write_bytes(buf.as_mut_ptr(), 0, buf.len()) } -#[cfg(test)] +#[cfg(all(test, not(target_os = "unknown")))] mod tests { use crate::io; use crate::prelude::*; diff --git a/src/task/builder.rs b/src/task/builder.rs index f48b6b4..91e2cb6 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -61,7 +61,7 @@ impl Builder { } /// Spawns a task locally with the configured settings. - #[cfg(not(target_os = "unknown"))] + #[cfg(all(not(target_os = "unknown"), feature = "unstable"))] pub fn local(self, future: F) -> io::Result> where F: Future + 'static, @@ -76,7 +76,7 @@ impl Builder { } /// Spawns a task locally with the configured settings. - #[cfg(target_arch = "wasm32")] + #[cfg(all(target_arch = "wasm32", feature = "unstable"))] pub fn local(self, future: F) -> io::Result> where F: Future + 'static, @@ -96,6 +96,27 @@ impl Builder { Ok(JoinHandle::new(receiver, task)) } + /// Spawns a task locally with the configured settings. + #[cfg(all(target_arch = "wasm32", not(feature = "unstable")))] + pub(crate) fn local(self, future: F) -> io::Result> + where + F: Future + 'static, + T: 'static, + { + use futures_channel::oneshot::channel; + let (sender, receiver) = channel(); + + let wrapped = self.build(async move { + let res = future.await; + let _ = sender.send(res); + }); + + let task = wrapped.tag.task().clone(); + wasm_bindgen_futures::spawn_local(wrapped); + + Ok(JoinHandle::new(receiver, task)) + } + /// Spawns a task with the configured settings, blocking on its execution. #[cfg(not(target_os = "unknown"))] pub fn blocking(self, future: F) -> T diff --git a/src/task/mod.rs b/src/task/mod.rs index 6a142ff..eefc7c2 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -140,7 +140,6 @@ cfg_default! { pub use sleep::sleep; #[cfg(not(target_os = "unknown"))] pub use spawn::spawn; - pub use spawn_local::spawn_local; pub use task_local::{AccessError, LocalKey}; pub(crate) use task_local::LocalsMap; @@ -155,7 +154,6 @@ cfg_default! { mod spawn; #[cfg(not(target_os = "unknown"))] mod spawn_blocking; - mod spawn_local; mod task; mod task_id; mod task_local; @@ -168,3 +166,9 @@ cfg_default! { #[cfg(not(any(feature = "unstable", test)))] pub(crate) use spawn_blocking::spawn_blocking; } + +cfg_unstable! { + pub use spawn_local::spawn_local; + + mod spawn_local; +} From e0928463b166aba813c23c6fcc79f1a0204980a7 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Tue, 28 Apr 2020 11:16:36 +0200 Subject: [PATCH 45/60] fix windows traits --- src/net/tcp/listener.rs | 3 +-- src/net/tcp/stream.rs | 8 +++++--- src/net/udp/mod.rs | 8 +++++--- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index 49ee4f4..72c5d3a 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -230,7 +230,6 @@ cfg_unix! { cfg_windows! { use crate::os::windows::io::{ - AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, AsRawSocket, FromRawSocket, IntoRawSocket, RawSocket, }; @@ -242,7 +241,7 @@ cfg_windows! { impl FromRawSocket for TcpListener { unsafe fn from_raw_socket(handle: RawSocket) -> TcpListener { - net::TcpListener::from_raw_socket(handle).try_into().unwrap() + std::net::TcpListener::from_raw_socket(handle).into() } } diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 7b71f98..b854143 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -388,17 +388,19 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, RawSocket, AsRawSocket, FromRawSocket, IntoRawSocket}; + use crate::os::windows::io::{ + RawSocket, AsRawSocket, FromRawSocket, IntoRawSocket + }; impl AsRawSocket for TcpStream { fn as_raw_socket(&self) -> RawSocket { - self.raw_socket + self.watcher.get_ref().as_raw_socket() } } impl FromRawSocket for TcpStream { unsafe fn from_raw_socket(handle: RawSocket) -> TcpStream { - net::TcpStream::from_raw_socket(handle).try_into().unwrap() + std::net::TcpStream::from_raw_socket(handle).into() } } diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 53add8e..30cceb7 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -482,17 +482,19 @@ cfg_unix! { } cfg_windows! { - use crate::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle, RawSocket, AsRawSocket, IntoRawSocket, FromRawSocket}; + use crate::os::windows::io::{ + RawSocket, AsRawSocket, IntoRawSocket, FromRawSocket + }; impl AsRawSocket for UdpSocket { fn as_raw_socket(&self) -> RawSocket { - self.watcher.as_raw_socket() + self.watcher.get_ref().as_raw_socket() } } impl FromRawSocket for UdpSocket { unsafe fn from_raw_socket(handle: RawSocket) -> UdpSocket { - net::UdpSocket::from_raw_socket(handle).into() + std::net::UdpSocket::from_raw_socket(handle).into() } } From 26f62aafd98e5e373142c7945fc78f8fa46b2618 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Tue, 28 Apr 2020 20:41:04 +0200 Subject: [PATCH 46/60] make wasm deps part of std --- Cargo.toml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 57084ab..630b373 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,6 +43,9 @@ std = [ "pin-utils", "slab", "smol", + "wasm-timer", + "wasm-bindgen-futures", + "futures-channel", ] alloc = [ "futures-core/alloc", @@ -69,9 +72,9 @@ slab = { version = "0.4.2", optional = true } smol = { version = "0.1.1", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] -wasm-timer = "0.2.4" -wasm-bindgen-futures = "0.4.10" -futures-channel = "0.3.4" +wasm-timer = { version = "0.2.4", optional = true } +wasm-bindgen-futures = { version = "0.4.10", optional = true } +futures-channel = { version = "0.3.4", optional = true } [target.'cfg(target_arch = "wasm32")'.dev-dependencies] wasm-bindgen-test = "0.3.10" From 1214bc2dee891cc4d2713c6da7882a11cdedd87f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 29 Apr 2020 18:45:07 +0200 Subject: [PATCH 47/60] increase timeouts --- tests/condvar.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/condvar.rs b/tests/condvar.rs index 7b05b28..76574a1 100644 --- a/tests/condvar.rs +++ b/tests/condvar.rs @@ -23,13 +23,13 @@ fn wait_timeout_with_lock() { spawn(async move { let (m, c) = &*pair2; let _g = m.lock().await; - task::sleep(Duration::from_millis(20)).await; + task::sleep(Duration::from_millis(200)).await; c.notify_one(); }); let (m, c) = &*pair; let (_, wait_result) = c - .wait_timeout(m.lock().await, Duration::from_millis(10)) + .wait_timeout(m.lock().await, Duration::from_millis(100)) .await; assert!(wait_result.timed_out()); }) @@ -57,7 +57,7 @@ fn wait_timeout_until_timed_out() { let c = Condvar::new(); let (_, wait_result) = c - .wait_timeout_until(m.lock().await, Duration::from_millis(10), |&mut started| { + .wait_timeout_until(m.lock().await, Duration::from_millis(100), |&mut started| { started }) .await; @@ -85,7 +85,7 @@ fn notify_all() { } // Give some time for tasks to start up - task::sleep(Duration::from_millis(5)).await; + task::sleep(Duration::from_millis(50)).await; let (m, c) = &*pair; { From faea222b9cb5e8cc8b0d3d69520f363af0ace2a7 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 2 May 2020 20:24:59 +0200 Subject: [PATCH 48/60] fix: use run instead of block_on --- src/task/builder.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/task/builder.rs b/src/task/builder.rs index 91e2cb6..3b71a53 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -132,7 +132,7 @@ impl Builder { }); // Run the future as a task. - unsafe { TaskLocalsWrapper::set_current(&wrapped.tag, || smol::block_on(wrapped)) } + unsafe { TaskLocalsWrapper::set_current(&wrapped.tag, || smol::run(wrapped)) } } } From 27c605b4c99e1435d9d46aa790fde3cb3b7bfa43 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 7 May 2020 20:56:52 +0200 Subject: [PATCH 49/60] cr: bring back trace call --- src/task/builder.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/task/builder.rs b/src/task/builder.rs index 3b71a53..cbb3187 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -54,6 +54,11 @@ impl Builder { { let wrapped = self.build(future); + kv_log_macro::trace!("spawn", { + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), + }); + let task = wrapped.tag.task().clone(); let smol_task = smol::Task::spawn(wrapped).into(); @@ -69,6 +74,11 @@ impl Builder { { let wrapped = self.build(future); + kv_log_macro::trace!("spawn_local", { + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), + }); + let task = wrapped.tag.task().clone(); let smol_task = smol::Task::local(wrapped).into(); @@ -89,6 +99,10 @@ impl Builder { let res = future.await; let _ = sender.send(res); }); + kv_log_macro::trace!("spawn_local", { + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), + }); let task = wrapped.tag.task().clone(); wasm_bindgen_futures::spawn_local(wrapped); @@ -111,6 +125,11 @@ impl Builder { let _ = sender.send(res); }); + kv_log_macro::trace!("spawn_local", { + task_id: wrapped.tag.id().0, + parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), + }); + let task = wrapped.tag.task().clone(); wasm_bindgen_futures::spawn_local(wrapped); From 6f6fced1034ae8ef8c2bf91540630d2cc8e774d8 Mon Sep 17 00:00:00 2001 From: Thayne McCombs Date: Thu, 7 May 2020 14:26:46 -0600 Subject: [PATCH 50/60] feat: implement Barrier using Condvar --- Cargo.toml | 3 +-- src/sync/barrier.rs | 59 ++++++++++++--------------------------------- 2 files changed, 16 insertions(+), 46 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 630b373..db26625 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -31,7 +31,7 @@ default = [ "pin-project-lite", ] docs = ["attributes", "unstable", "default"] -unstable = ["std", "broadcaster"] +unstable = ["std"] attributes = ["async-attributes"] std = [ "alloc", @@ -55,7 +55,6 @@ alloc = [ [dependencies] async-attributes = { version = "1.1.1", optional = true } async-task = { version = "3.0.0", optional = true } -broadcaster = { version = "1.0.0", optional = true } crossbeam-utils = { version = "0.7.2", optional = true } futures-core = { version = "0.3.4", optional = true, default-features = false } futures-io = { version = "0.3.4", optional = true } diff --git a/src/sync/barrier.rs b/src/sync/barrier.rs index 86e9a2d..f492ebe 100644 --- a/src/sync/barrier.rs +++ b/src/sync/barrier.rs @@ -1,6 +1,4 @@ -use broadcaster::BroadcastChannel; - -use crate::sync::Mutex; +use crate::sync::{Condvar,Mutex}; /// A barrier enables multiple tasks to synchronize the beginning /// of some computation. @@ -36,14 +34,13 @@ use crate::sync::Mutex; #[derive(Debug)] pub struct Barrier { state: Mutex, - wait: BroadcastChannel<(usize, usize)>, - n: usize, + cvar: Condvar, + num_tasks: usize, } // The inner state of a double barrier #[derive(Debug)] struct BarrierState { - waker: BroadcastChannel<(usize, usize)>, count: usize, generation_id: usize, } @@ -81,25 +78,14 @@ impl Barrier { /// /// let barrier = Barrier::new(10); /// ``` - pub fn new(mut n: usize) -> Barrier { - let waker = BroadcastChannel::new(); - let wait = waker.clone(); - - if n == 0 { - // if n is 0, it's not clear what behavior the user wants. - // in std::sync::Barrier, an n of 0 exhibits the same behavior as n == 1, where every - // .wait() immediately unblocks, so we adopt that here as well. - n = 1; - } - + pub fn new(n: usize) -> Barrier { Barrier { state: Mutex::new(BarrierState { - waker, count: 0, generation_id: 1, }), - n, - wait, + cvar: Condvar::new(), + num_tasks: n, } } @@ -143,35 +129,20 @@ impl Barrier { /// # }); /// ``` pub async fn wait(&self) -> BarrierWaitResult { - let mut lock = self.state.lock().await; - let local_gen = lock.generation_id; - - lock.count += 1; + let mut state = self.state.lock().await; + let local_gen = state.generation_id; + state.count += 1; - if lock.count < self.n { - let mut wait = self.wait.clone(); - - let mut generation_id = lock.generation_id; - let mut count = lock.count; - - drop(lock); - - while local_gen == generation_id && count < self.n { - let (g, c) = wait.recv().await.expect("sender has not been closed"); - generation_id = g; - count = c; + if state.count < self.num_tasks { + while local_gen == state.generation_id && state.count < self.num_tasks { + state = self.cvar.wait(state).await; } BarrierWaitResult(false) } else { - lock.count = 0; - lock.generation_id = lock.generation_id.wrapping_add(1); - - lock.waker - .send(&(lock.generation_id, lock.count)) - .await - .expect("there should be at least one receiver"); - + state.count = 0; + state.generation_id = state.generation_id.wrapping_add(1); + self.cvar.notify_all(); BarrierWaitResult(true) } } From e4c4c93d29dac93d33eafcc0c677bf13e40bf5e4 Mon Sep 17 00:00:00 2001 From: Friedel Ziegelmayer Date: Thu, 7 May 2020 23:20:44 +0200 Subject: [PATCH 51/60] Test and fix 32 bit targets --- .github/workflows/ci.yml | 33 +++++++++++++++++++++++++++++++++ Cargo.toml | 8 +++++++- src/task/task_id.rs | 9 +++++---- tests/io_timeout.rs | 9 ++++++++- tests/timeout.rs | 2 +- 5 files changed, 54 insertions(+), 7 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 8f519e5..1a0c432 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -110,6 +110,39 @@ jobs: command: check args: --no-default-features --features alloc --target thumbv7m-none-eabi -Z avoid-dev-deps + cross: + name: Cross compile + runs-on: ubuntu-latest + strategy: + matrix: + target: + - i686-unknown-linux-gnu + - powerpc-unknown-linux-gnu + - powerpc64-unknown-linux-gnu + - mips-unknown-linux-gnu + - arm-linux-androideabi + + steps: + - uses: actions/checkout@master + + - name: Install nightly + uses: actions-rs/toolchain@v1 + with: + toolchain: nightly + override: true + + - name: Install cross + run: cargo install cross + + - name: check + run: cross check --all --target ${{ matrix.target }} + + - name: check unstable + run: cross check --all --features unstable --target ${{ matrix.target }} + + - name: test + run: cross test --all --features unstable --target ${{ matrix.target }} + check_fmt_and_docs: name: Checking fmt and docs runs-on: ubuntu-latest diff --git a/Cargo.toml b/Cargo.toml index db26625..e6e810f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -67,6 +67,9 @@ pin-project-lite = { version = "0.1.4", optional = true } pin-utils = { version = "0.1.0-alpha.4", optional = true } slab = { version = "0.4.2", optional = true } +# Devdepencency, but they are not allowed to be optional :/ +surf = { version = "1.0.3", optional = true } + [target.'cfg(not(target_os = "unknown"))'.dependencies] smol = { version = "0.1.1", optional = true } @@ -81,7 +84,6 @@ wasm-bindgen-test = "0.3.10" [dev-dependencies] femme = "1.3.0" rand = "0.7.3" -surf = "1.0.3" tempdir = "0.3.7" futures = "0.3.4" rand_xorshift = "0.2.0" @@ -93,3 +95,7 @@ required-features = ["unstable"] [[example]] name = "tcp-ipv4-and-6-echo" required-features = ["unstable"] + +[[example]] +name = "surf-web" +required-features = ["surf"] \ No newline at end of file diff --git a/src/task/task_id.rs b/src/task/task_id.rs index 67eee15..92c607c 100644 --- a/src/task/task_id.rs +++ b/src/task/task_id.rs @@ -1,5 +1,5 @@ use std::fmt; -use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::atomic::{AtomicUsize, Ordering}; /// A unique identifier for a task. /// @@ -13,15 +13,16 @@ use std::sync::atomic::{AtomicU64, Ordering}; /// }) /// ``` #[derive(Eq, PartialEq, Clone, Copy, Hash, Debug)] -pub struct TaskId(pub(crate) u64); +pub struct TaskId(pub(crate) usize); impl TaskId { /// Generates a new `TaskId`. pub(crate) fn generate() -> TaskId { - static COUNTER: AtomicU64 = AtomicU64::new(1); + // TODO: find a good version to emulate u64 atomics on 32 bit systems. + static COUNTER: AtomicUsize = AtomicUsize::new(1); let id = COUNTER.fetch_add(1, Ordering::Relaxed); - if id > u64::max_value() / 2 { + if id > usize::max_value() / 2 { std::process::abort(); } TaskId(id) diff --git a/tests/io_timeout.rs b/tests/io_timeout.rs index fa30a68..3711506 100644 --- a/tests/io_timeout.rs +++ b/tests/io_timeout.rs @@ -5,7 +5,14 @@ use async_std::task; #[test] #[should_panic(expected = "timed out")] -#[cfg(not(target_os = "unknown"))] +#[cfg(not(any( + target_os = "unknown", + target_arch = "arm", + target_arch = "mips", + target_arch = "powerpc", + target_arch = "powerpc64", + target_arch = "x86", +)))] // stdin tests fail when running through cross fn io_timeout_timedout() { task::block_on(async { io::timeout(Duration::from_secs(1), async { diff --git a/tests/timeout.rs b/tests/timeout.rs index 8ad358a..e09acdf 100644 --- a/tests/timeout.rs +++ b/tests/timeout.rs @@ -12,7 +12,7 @@ fn timeout_future_many() { task::block_on(async { let futures = (0..100) .map(|i| { - timeout(Duration::from_millis(i * 10), async move { + timeout(Duration::from_millis(i * 20), async move { task::sleep(Duration::from_millis(i)).await; Ok::<(), async_std::future::TimeoutError>(()) }) From bd6a7e200bd380042eb811e37cd2b81089f0d55f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 7 May 2020 23:02:55 +0200 Subject: [PATCH 52/60] prepare v1.6.0-beta.1 --- CHANGELOG.md | 20 ++++++++++++++++++++ Cargo.toml | 2 +- src/lib.rs | 8 ++++---- 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e464ed7..44a36f3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,26 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +# [1.6.0-beta.1] - 2020-05-07 + +## Added + +- Added `task::spawn_local`. ([#757](https://github.com/async-rs/async-std/pull/757)) +- Added out of the box support for `wasm`. ([#757](https://github.com/async-rs/async-std/pull/757)) +- Added `JoinHandle::cancel` ([#757](https://github.com/async-rs/async-std/pull/757)) +- Added `sync::Condvar` ([#369](https://github.com/async-rs/async-std/pull/369)) +- Added `sync::Sender::try_send` and `sync::Receiver::try_recv` ([#585](https://github.com/async-rs/async-std/pull/585)) +- Added `no_std` support for `task`, `future` and `stream` ([#680](https://github.com/async-rs/async-std/pull/680)) + +## Changed + +- Switched underlying runtime to [`smol`](https://github.com/stjepang/smol/). ([#757](https://github.com/async-rs/async-std/pull/757)) +- Switched implementation of `sync::Barrier` to use `sync::Condvar` like `std` does. ([#581](https://github.com/async-rs/async-std/pull/581)) + +## Fixed + +- Allow compilation on 32 bit targets, by using `AtomicUsize` for `TaskId`. ([#756](https://github.com/async-rs/async-std/pull/756)) + # [1.5.0] - 2020-02-03 [API Documentation](https://docs.rs/async-std/1.5.0/async-std) diff --git a/Cargo.toml b/Cargo.toml index e6e810f..a14a32b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "async-std" -version = "1.5.0" +version = "1.6.0-beta.1" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", diff --git a/src/lib.rs b/src/lib.rs index 408a7ab..4be05e1 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -194,7 +194,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.0.0" +//! version = "1.6.0-beta.1" //! features = ["unstable"] //! ``` //! @@ -207,7 +207,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.0.0" +//! version = "1.6.0-beta.1" //! features = ["attributes"] //! ``` //! @@ -216,7 +216,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.0.0" +//! version = "1.6.0-beta.1" //! default-features = false //! features = ["std"] //! ``` @@ -226,7 +226,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.5.0" +//! version = "1.6.0-beta.1" //! default-features = false //! features = ["alloc"] //! ``` From 247c94ca06d6d04bc0ef061176b05b9a8e664842 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 7 May 2020 23:34:49 +0200 Subject: [PATCH 53/60] docs(changelog): add missing link --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 44a36f3..e7ac8b1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -698,6 +698,7 @@ task::blocking(async { - Initial beta release [Unreleased]: https://github.com/async-rs/async-std/compare/v1.5.0...HEAD +[1.6.0-beta.1]: https://github.com/async-rs/async-std/compare/v1.4.0...v1.6.0-beta.1 [1.5.0]: https://github.com/async-rs/async-std/compare/v1.4.0...v1.5.0 [1.4.0]: https://github.com/async-rs/async-std/compare/v1.3.0...v1.4.0 [1.3.0]: https://github.com/async-rs/async-std/compare/v1.2.0...v1.3.0 From 2762ec5800d94e7891c7b01f39f57c0b79eb3088 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 9 May 2020 11:36:13 +0200 Subject: [PATCH 54/60] fix(fs): use smol::block_on for drop handling of File Ref #766 --- src/fs/file.rs | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/src/fs/file.rs b/src/fs/file.rs index 7fe99ee..74d2bfd 100644 --- a/src/fs/file.rs +++ b/src/fs/file.rs @@ -12,7 +12,7 @@ use crate::future; use crate::io::{self, Read, Seek, SeekFrom, Write}; use crate::path::Path; use crate::prelude::*; -use crate::task::{self, spawn_blocking, Context, Poll, Waker}; +use crate::task::{spawn_blocking, Context, Poll, Waker}; use crate::utils::Context as _; /// An open file on the filesystem. @@ -315,7 +315,7 @@ impl Drop for File { // non-blocking fashion, but our only other option here is losing data remaining in the // write cache. Good task schedulers should be resilient to occasional blocking hiccups in // file destructors so we don't expect this to be a common problem in practice. - let _ = task::block_on(self.flush()); + let _ = smol::block_on(self.flush()); } } @@ -867,3 +867,15 @@ impl LockGuard { Poll::Ready(Ok(())) } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn async_file_drop() { + crate::task::block_on(async move { + File::open(".").await.unwrap(); + }); + } +} From 19170aead40d73e773fe6784521cda92689cffef Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sat, 9 May 2020 11:44:16 +0200 Subject: [PATCH 55/60] use local file --- src/fs/file.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/fs/file.rs b/src/fs/file.rs index 74d2bfd..1930fdd 100644 --- a/src/fs/file.rs +++ b/src/fs/file.rs @@ -875,7 +875,7 @@ mod tests { #[test] fn async_file_drop() { crate::task::block_on(async move { - File::open(".").await.unwrap(); + File::open(file!()).await.unwrap(); }); } } From cd5e17fe87746f10c823bf8a6103183fefbb82ec Mon Sep 17 00:00:00 2001 From: Jacob Rothstein Date: Sun, 10 May 2020 18:18:50 -0700 Subject: [PATCH 56/60] make UnixStream Clone --- src/os/unix/net/listener.rs | 3 ++- src/os/unix/net/stream.rs | 26 ++++++++++++++++---------- tests/uds.rs | 24 ++++++++++++++++++++++++ 3 files changed, 42 insertions(+), 11 deletions(-) diff --git a/src/os/unix/net/listener.rs b/src/os/unix/net/listener.rs index 4099bd6..ac03307 100644 --- a/src/os/unix/net/listener.rs +++ b/src/os/unix/net/listener.rs @@ -13,6 +13,7 @@ use crate::io; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; use crate::stream::Stream; +use crate::sync::Arc; use crate::task::{Context, Poll}; /// A Unix domain socket server, listening for connections. @@ -92,7 +93,7 @@ impl UnixListener { pub async fn accept(&self) -> io::Result<(UnixStream, SocketAddr)> { let (stream, addr) = self.watcher.accept().await?; - Ok((UnixStream { watcher: stream }, addr)) + Ok((UnixStream { watcher: Arc::new(stream) }, addr)) } /// Returns a stream of incoming connections. diff --git a/src/os/unix/net/stream.rs b/src/os/unix/net/stream.rs index 7320c85..b1ba5bc 100644 --- a/src/os/unix/net/stream.rs +++ b/src/os/unix/net/stream.rs @@ -11,6 +11,7 @@ use super::SocketAddr; use crate::io::{self, Read, Write}; use crate::os::unix::io::{AsRawFd, FromRawFd, IntoRawFd, RawFd}; use crate::path::Path; +use crate::sync::Arc; use crate::task::{Context, Poll}; /// A Unix stream socket. @@ -36,8 +37,9 @@ use crate::task::{Context, Poll}; /// # /// # Ok(()) }) } /// ``` +#[derive(Clone)] pub struct UnixStream { - pub(super) watcher: Async, + pub(super) watcher: Arc>, } impl UnixStream { @@ -56,7 +58,7 @@ impl UnixStream { /// ``` pub async fn connect>(path: P) -> io::Result { let path = path.as_ref().to_owned(); - let stream = Async::::connect(path).await?; + let stream = Arc::new(Async::::connect(path).await?); Ok(UnixStream { watcher: stream }) } @@ -78,8 +80,12 @@ impl UnixStream { /// ``` pub fn pair() -> io::Result<(UnixStream, UnixStream)> { let (a, b) = Async::::pair()?; - let a = UnixStream { watcher: a }; - let b = UnixStream { watcher: b }; + let a = UnixStream { + watcher: Arc::new(a), + }; + let b = UnixStream { + watcher: Arc::new(b), + }; Ok((a, b)) } @@ -158,7 +164,7 @@ impl Read for &UnixStream { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - Pin::new(&mut &self.watcher).poll_read(cx, buf) + Pin::new(&mut &*self.watcher).poll_read(cx, buf) } } @@ -186,15 +192,15 @@ impl Write for &UnixStream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - Pin::new(&mut &self.watcher).poll_write(cx, buf) + Pin::new(&mut &*self.watcher).poll_write(cx, buf) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut &self.watcher).poll_flush(cx) + Pin::new(&mut &*self.watcher).poll_flush(cx) } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut &self.watcher).poll_close(cx) + Pin::new(&mut &*self.watcher).poll_close(cx) } } @@ -219,7 +225,7 @@ impl From for UnixStream { /// Converts a `std::os::unix::net::UnixStream` into its asynchronous equivalent. fn from(stream: StdUnixStream) -> UnixStream { let stream = Async::new(stream).expect("UnixStream is known to be good"); - UnixStream { watcher: stream } + UnixStream { watcher: Arc::new(stream) } } } @@ -238,6 +244,6 @@ impl FromRawFd for UnixStream { impl IntoRawFd for UnixStream { fn into_raw_fd(self) -> RawFd { - self.watcher.into_raw_fd() + self.as_raw_fd() } } diff --git a/tests/uds.rs b/tests/uds.rs index 038ac0e..d081bda 100644 --- a/tests/uds.rs +++ b/tests/uds.rs @@ -94,3 +94,27 @@ async fn ping_pong_client(socket: &std::path::PathBuf, iterations: u32) -> std:: } Ok(()) } + +#[test] +fn uds_clone() -> io::Result<()> { + task::block_on(async { + let tmp_dir = TempDir::new("socket_ping_pong").expect("Temp dir not created"); + let sock_path = tmp_dir.as_ref().join("sock"); + let input = UnixListener::bind(&sock_path).await?; + + let mut writer = UnixStream::connect(&sock_path).await?; + let mut reader = input.incoming().next().await.unwrap()?; + + writer.write(b"original").await.unwrap(); + let mut original_buf = [0; 8]; + reader.read(&mut original_buf).await?; + assert_eq!(&original_buf, b"original"); + + writer.clone().write(b"clone").await.unwrap(); + let mut clone_buf = [0; 5]; + reader.clone().read(&mut clone_buf).await?; + assert_eq!(&clone_buf, b"clone"); + + Ok(()) + }) +} From d3e59370e78ce279f45a3943457e22ebf46292fe Mon Sep 17 00:00:00 2001 From: Azriel Hoh Date: Wed, 13 May 2020 10:14:05 +1200 Subject: [PATCH 57/60] Switches `wasm-timer` for `futures-timer`. --- Cargo.toml | 4 ++-- src/utils.rs | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a14a32b..5a1d31e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,7 +43,7 @@ std = [ "pin-utils", "slab", "smol", - "wasm-timer", + "futures-timer", "wasm-bindgen-futures", "futures-channel", ] @@ -74,7 +74,7 @@ surf = { version = "1.0.3", optional = true } smol = { version = "0.1.1", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] -wasm-timer = { version = "0.2.4", optional = true } +futures-timer = { version = "3.0.2", optional = true, features = ["wasm-bindgen"] } wasm-bindgen-futures = { version = "0.4.10", optional = true } futures-channel = { version = "0.3.4", optional = true } diff --git a/src/utils.rs b/src/utils.rs index ef068cb..7c9aa99 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -64,12 +64,12 @@ pub(crate) type Timer = smol::Timer; #[cfg(all(target_arch = "wasm32", feature = "default"))] #[derive(Debug)] -pub(crate) struct Timer(wasm_timer::Delay); +pub(crate) struct Timer(futures_timer::Delay); #[cfg(all(target_arch = "wasm32", feature = "default"))] impl Timer { pub(crate) fn after(dur: std::time::Duration) -> Self { - Timer(wasm_timer::Delay::new(dur)) + Timer(futures_timer::Delay::new(dur)) } } From e9621af345d854bcf1e71690b54cc270558c4940 Mon Sep 17 00:00:00 2001 From: Azriel Hoh Date: Wed, 13 May 2020 10:37:19 +1200 Subject: [PATCH 58/60] Updates `CHANGELOG.md`. --- CHANGELOG.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e7ac8b1..70d8ed8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,10 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +## Changed + +- For `wasm`, switched underlying `Timer` implementation to [`futures-timer`](https://github.com/async-rs/futures-timer). ([#776](https://github.com/async-rs/async-std/pull/776)) + # [1.6.0-beta.1] - 2020-05-07 ## Added From baead51a282e077103a17b30fc6f25726613e0b6 Mon Sep 17 00:00:00 2001 From: Azriel Hoh Date: Wed, 13 May 2020 10:38:40 +1200 Subject: [PATCH 59/60] Reduces duration in timeout test. Tries to get CI to pass. --- tests/condvar.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/condvar.rs b/tests/condvar.rs index 76574a1..b5ec12a 100644 --- a/tests/condvar.rs +++ b/tests/condvar.rs @@ -29,7 +29,7 @@ fn wait_timeout_with_lock() { let (m, c) = &*pair; let (_, wait_result) = c - .wait_timeout(m.lock().await, Duration::from_millis(100)) + .wait_timeout(m.lock().await, Duration::from_millis(50)) .await; assert!(wait_result.timed_out()); }) From 9e6a76af04a08d3e3d13f4366c5d254dc2c22b94 Mon Sep 17 00:00:00 2001 From: Jacob Rothstein Date: Tue, 19 May 2020 02:16:01 -0700 Subject: [PATCH 60/60] feat: add env vars to configure the runtime threadpool size and name --- src/lib.rs | 15 +++++++++++++++ src/rt/mod.rs | 17 ++++++++++++++--- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index 4be05e1..4704a9d 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -230,6 +230,21 @@ //! default-features = false //! features = ["alloc"] //! ``` +//! +//! # Runtime configuration +//! +//! Several environment variables are available to tune the async-std +//! runtime: +//! +//! * `ASYNC_STD_THREAD_COUNT`: The number of threads that the +//! async-std runtime will start. By default, this is one per logical +//! cpu as reported by the [num_cpus](num_cpus) crate, which may be +//! different than the number of physical cpus. Async-std _will panic_ +//! if this is set to any value other than a positive integer. +//! * `ASYNC_STD_THREAD_NAME`: The name that async-std's runtime +//! threads report to the operating system. The default value is +//! `"async-std/runtime"`. +//! #![cfg_attr(not(feature = "std"), no_std)] #![cfg_attr(feature = "docs", feature(doc_cfg))] diff --git a/src/rt/mod.rs b/src/rt/mod.rs index d5d0d61..65f4e24 100644 --- a/src/rt/mod.rs +++ b/src/rt/mod.rs @@ -1,5 +1,6 @@ //! The runtime. +use std::env; use std::thread; use once_cell::sync::Lazy; @@ -12,10 +13,20 @@ pub struct Runtime {} /// The global runtime. pub static RUNTIME: Lazy = Lazy::new(|| { // Create an executor thread pool. - let num_threads = num_cpus::get().max(1); - for _ in 0..num_threads { + + let thread_count = env::var("ASYNC_STD_THREAD_COUNT") + .map(|env| { + env.parse() + .expect("ASYNC_STD_THREAD_COUNT must be a number") + }) + .unwrap_or_else(|_| num_cpus::get()) + .max(1); + + let thread_name = env::var("ASYNC_STD_THREAD_NAME").unwrap_or("async-std/runtime".to_string()); + + for _ in 0..thread_count { thread::Builder::new() - .name("async-std/runtime".to_string()) + .name(thread_name.clone()) .spawn(|| smol::run(future::pending::<()>())) .expect("cannot start a runtime thread"); }