From b2aaa8b8259a1acbf09b656176b046fd3820f4bd Mon Sep 17 00:00:00 2001 From: sclaire-1 <54961957+sclaire-1@users.noreply.github.com> Date: Sat, 16 Nov 2019 13:02:17 -0800 Subject: [PATCH 001/149] Edit tutorial: implementing_a_client.md Edited to improve reading flow --- docs/src/tutorial/implementing_a_client.md | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/docs/src/tutorial/implementing_a_client.md b/docs/src/tutorial/implementing_a_client.md index fd72855..ba9d6f3 100644 --- a/docs/src/tutorial/implementing_a_client.md +++ b/docs/src/tutorial/implementing_a_client.md @@ -1,18 +1,16 @@ ## Implementing a client -Let's now implement the client for the chat. -Because the protocol is line-based, the implementation is pretty straightforward: +Since the protocol is line-based, implementing a client for the chat is straightforward: * Lines read from stdin should be sent over the socket. * Lines read from the socket should be echoed to stdout. -Unlike the server, the client needs only limited concurrency, as it interacts with only a single user. -For this reason, async doesn't bring a lot of performance benefits in this case. +Although async does not significantly affect client performance (as unlike the server, the client interacts solely with one user and only needs limited concurrency), async is still useful for managing concurrency! + +The client has to read from stdin and the socket *simultaneously*. +Programming this with threads is cumbersome, especially when implementing a clean shutdown. +With async, the `select!` macro is all that is needed. -However, async is still useful for managing concurrency! -Specifically, the client should *simultaneously* read from stdin and from the socket. -Programming this with threads is cumbersome, especially when implementing clean shutdown. -With async, we can just use the `select!` macro. ```rust,edition2018 # extern crate async_std; From 36d24cd0e1f5049be4caa2461a43c704eb61975e Mon Sep 17 00:00:00 2001 From: Stjepan Glavina Date: Mon, 16 Dec 2019 13:57:27 +0100 Subject: [PATCH 002/149] 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 003/149] 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 c8c075615c0a44ec10c210292caf25899d8b8b0a Mon Sep 17 00:00:00 2001 From: Paul Colomiets Date: Sun, 12 Jan 2020 03:45:41 +0200 Subject: [PATCH 004/149] book: Add Production-ready Accept Loop section Part of the #658 work --- docs/src/SUMMARY.md | 3 +- docs/src/patterns/accept-loop.md | 256 +++++++++++++++++++++++++++++++ 2 files changed, 258 insertions(+), 1 deletion(-) create mode 100644 docs/src/patterns/accept-loop.md diff --git a/docs/src/SUMMARY.md b/docs/src/SUMMARY.md index d679825..8b49ce7 100644 --- a/docs/src/SUMMARY.md +++ b/docs/src/SUMMARY.md @@ -19,8 +19,9 @@ - [Clean Shutdown](./tutorial/clean_shutdown.md) - [Handling Disconnection](./tutorial/handling_disconnection.md) - [Implementing a Client](./tutorial/implementing_a_client.md) -- [TODO: Async Patterns](./patterns.md) +- [Async Patterns](./patterns.md) - [TODO: Collected Small Patterns](./patterns/small-patterns.md) + - [Production-Ready Accept Loop](./patterns/accept-loop.md) - [Security practices](./security/index.md) - [Security Disclosures and Policy](./security/policy.md) - [Glossary](./glossary.md) diff --git a/docs/src/patterns/accept-loop.md b/docs/src/patterns/accept-loop.md new file mode 100644 index 0000000..acc0d75 --- /dev/null +++ b/docs/src/patterns/accept-loop.md @@ -0,0 +1,256 @@ +# Production-Ready Accept Loop + +Production-ready accept loop needs the following things: +1. Handling errors +2. Limiting the number of simultanteous connections to avoid deny-of-service + (DoS) attacks + + +## Handling errors + +There are two kinds of errors in accept loop: +1. Per-connection errors. System uses them to notify that there was a + connection in the queue and it's dropped by peer. Subsequent connection + can be already queued so next connection must be accepted immediately. +2. Resource shortages. When these are encountered it doesn't make sense to + accept next socket immediately. But listener stays active, so you server + should try to accept socket later. + +Here is the example of per-connection error (printed in normal and debug mode): +``` +Error: Connection reset by peer (os error 104) +Error: Os { code: 104, kind: ConnectionReset, message: "Connection reset by peer" } +``` + +And the following is the most common example of a resource shortage error: +``` +Error: Too many open files (os error 24) +Error: Os { code: 24, kind: Other, message: "Too many open files" } +``` + +### Testing Application + +To test your application on these errors try the following (this works +on unixes only). + +Lower limit and start the application: +``` +$ ulimit -n 100 +$ cargo run --example your_app + Compiling your_app v0.1.0 (/work) + Finished dev [unoptimized + debuginfo] target(s) in 5.47s + Running `target/debug/examples/your_app` +Server is listening on: http://127.0.0.1:1234 +``` +Then in another console run [`wrk`] benchmark tool: +``` +$ wrk -c 1000 http://127.0.0.1:1234 +Running 10s test @ http://localhost:8080/ + 2 threads and 1000 connections +$ telnet localhost 1234 +Trying ::1... +Connected to localhost. +``` + +Important is to check the following things: + +1. Application doesn't crash on error (but may log errors, see below) +2. It's possible to connect to the application again once load is stopped + (few seconds after `wrk`). This is what `telnet` does in example above, + make sure it prints `Connected to `. +3. The `Too many open files` error is logged in the appropriate log. This + requires to set "maximum number of simultaneous connections" parameter (see + below) of your application to a value greater that `100` for this example. +4. Check CPU usage of the app while doing a test. It should not occupy 100% + of a single CPU core (it's unlikely that you can exhaust CPU by 1000 + connections in Rust, so this means error handling is not right). + +#### Testing non-HTTP applications + +If it's possible, use the appropriate benchmark tool and set the appropriate +number of connections. For example `redis-benchmark` has `-c` parameter for +that, if you implement redis protocol. + +Alternatively, can still use `wrk`, just make sure that connection is not +immediately closed. If it is, put a temporary timeout before handing +connection to the protocol handler, like this: + +```rust,edition2018 +# extern crate async_std; +# use std::time::Duration; +# use async_std::{ +# net::{TcpListener, ToSocketAddrs}, +# prelude::*, +# }; +# +# type Result = std::result::Result>; +# +#async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { +# let listener = TcpListener::bind(addr).await?; +# let mut incoming = listener.incoming(); +while let Some(stream) = incoming.next().await { + task::spawn(async { + task:sleep(Duration::from_secs(10)).await; // 1 + connection_loop(stream).await; + }); +} +# Ok(()) +# } +``` + +1. Make sure the sleep coroutine is inside the spawned task, not in the loop. + +[`wrk`]: https://github.com/wg/wrk + + +### Handling Errors Manually + +Here is how basic accept loop could look like: + +```rust,edition2018 +# extern crate async_std; +# use std::time::Duration; +# use async_std::{ +# net::{TcpListener, ToSocketAddrs}, +# prelude::*, +# }; +# +# type Result = std::result::Result>; +# +async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { + let listener = TcpListener::bind(addr).await?; + let mut incoming = listener.incoming(); + while let Some(result) = incoming.next().await { + let stream = match stream { + Err(ref e) if is_connection_error(e) => continue, // 1 + Err(e) => { + eprintln!("Error: {}. Pausing for 500ms."); // 3 + task::sleep(Duration::from_millis(500)).await; // 2 + continue; + } + Ok(s) => s, + }; + // body + } + Ok(()) +} +``` + +1. Ignore per-connection errors. +2. Sleep and continue on resource shortage. +3. It's important to log the message, because these errors commonly mean the + misconfiguration of the system and are helpful for operations people running + the application. + +Be sure to [test your application](#testing-application). + + +### External Crates + +The crate [`async-listen`] have a helper to achieve this task: +```rust,edition2018 +# extern crate async_std; +# extern crate async_listen; +# use std::time::Duration; +# use async_std::{ +# net::{TcpListener, ToSocketAddrs}, +# prelude::*, +# }; +# +# type Result = std::result::Result>; +# +use async_listen::ListenExt; + +async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { + + let listener = TcpListener::bind(addr).await?; + let mut incoming = listener + .incoming() + .log_warnings(|e| eprintln!("Error: {}. Pausing for 500ms.", e)) // 1 + .handle_errors(Duration::from_millis(500)); + while let Some(socket) = incoming.next().await { // 2 + // body + } + Ok(()) +} +``` + +1. Logs resource shortages (`async-listen` calls them warnings). If you use + `log` crate or any other in your app this should go to the log. +2. Stream yields sockets without `Result` wrapper after `handle_errors` because + all errors are already handled. + +[`async-listen`]: https://crates.io/crates/async-listen/ + +Be sure to [test your application](#testing-application). + + +## Connections Limit + +Even if you've applied everything described in +[Handling Errors](#handling-errors) section, there is still a problem. + +Let's imagine you have a server that needs to open a file to process +client request. At some point, you might encounter the following situation: + +1. There are as much client connection as max file descriptors allowed for + the application. +2. Listener gets `Too many open files` error so it sleeps. +3. Some client sends a request via the previously open connection. +4. Opening a file to serve request fails, because of the same + `Too many open files` error, until some other client drops a connection. + +There are many more possible situations, this is just a small illustation that +limiting number of connections is very useful. Generally, it's one of the ways +to control resources used by a server and avoiding some kinds of deny of +service (DoS) attacks. + +### `async-listen` crate + +Limiting maximum number of simultaneous connections with [`async-listen`] +looks like the following: + +```rust,edition2018 +# extern crate async_std; +# extern crate async_listen; +# use std::time::Duration; +# use async_std::{ +# net::{TcpListener, TcpStream, ToSocketAddrs}, +# prelude::*, +# }; +# +# type Result = std::result::Result>; +# +use async_listen::{ListenExt, Token}; + +async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { + + let listener = TcpListener::bind(addr).await?; + let mut incoming = listener + .incoming() + .log_warnings(|e| eprintln!("Error: {}. Pausing for 500ms.", e)) + .handle_errors(Duration::from_millis(500)) // 1 + .backpressure(100); + while let Some((token, socket)) = incoming.next().await { // 2 + task::spawn(async move { + connection_loop(&token, stream).await; // 3 + }); + } + Ok(()) +} +async fn connection_loop(_token: &Token, stream: TcpStream) { // 4 + // ... +} +``` + +1. We need to handle errors first, because [`backpressure`] helper expects + stream of `TcpStream` rather than `Result`. +2. The token yielded by a new stream is what is counted by backpressure helper. + I.e. if you drop a token, new connection can be established. +3. We give connection loop a reference to token to bind token's lifetime to + the lifetime of the connection. +4. The token itsellf in the function can be ignored, hence `_token` + +[`backpressure`]: https://docs.rs/async-listen/0.1.2/async_listen/trait.ListenExt.html#method.backpressure + +Be sure to [test this behavior](#testing-application). From fb567a3a09da32569f05712bdb07b4f31f2f9526 Mon Sep 17 00:00:00 2001 From: Oleg Nosov Date: Sun, 12 Jan 2020 17:53:16 +0300 Subject: [PATCH 005/149] Recovered comments --- src/option/from_stream.rs | 1 + src/option/product.rs | 1 + src/option/sum.rs | 1 + 3 files changed, 3 insertions(+) diff --git a/src/option/from_stream.rs b/src/option/from_stream.rs index dc6d841..f0aafea 100644 --- a/src/option/from_stream.rs +++ b/src/option/from_stream.rs @@ -24,6 +24,7 @@ where .take_while(|elem| { elem.is_some() || { found_none = true; + // Stop processing the stream on `None` false } }) diff --git a/src/option/product.rs b/src/option/product.rs index ff61eaa..2238a91 100644 --- a/src/option/product.rs +++ b/src/option/product.rs @@ -48,6 +48,7 @@ where .take_while(|elem| { elem.is_some() || { found_none = true; + // Stop processing the stream on `None` false } }) diff --git a/src/option/sum.rs b/src/option/sum.rs index cd92f78..0570a1e 100644 --- a/src/option/sum.rs +++ b/src/option/sum.rs @@ -43,6 +43,7 @@ where .take_while(|elem| { elem.is_some() || { found_none = true; + // Stop processing the stream on error false } }) From 002903788319328d2627a60da08d1c78db7b062e Mon Sep 17 00:00:00 2001 From: Paul Colomiets Date: Mon, 13 Jan 2020 08:46:32 +0200 Subject: [PATCH 006/149] async-listen crate: Add `error_hint()` invocation --- docs/src/patterns/accept-loop.md | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/src/patterns/accept-loop.md b/docs/src/patterns/accept-loop.md index acc0d75..46d6cc6 100644 --- a/docs/src/patterns/accept-loop.md +++ b/docs/src/patterns/accept-loop.md @@ -159,26 +159,33 @@ The crate [`async-listen`] have a helper to achieve this task: # # type Result = std::result::Result>; # -use async_listen::ListenExt; +use async_listen::{ListenExt, error_hint}; async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { let listener = TcpListener::bind(addr).await?; let mut incoming = listener .incoming() - .log_warnings(|e| eprintln!("Error: {}. Pausing for 500ms.", e)) // 1 + .log_warnings(log_accept_error) // 1 .handle_errors(Duration::from_millis(500)); while let Some(socket) = incoming.next().await { // 2 // body } Ok(()) } + +fn log_accept_error(e: &io::Error) { + eprintln!("Error: {}. Listener paused for 0.5s. {}", e, error_hint(e)) // 3 +} ``` 1. Logs resource shortages (`async-listen` calls them warnings). If you use `log` crate or any other in your app this should go to the log. 2. Stream yields sockets without `Result` wrapper after `handle_errors` because all errors are already handled. +3. Together with the error we print a hint, which explains some errors for end + users. For example, it recommends increasing open file limit and gives + a link. [`async-listen`]: https://crates.io/crates/async-listen/ @@ -221,14 +228,14 @@ looks like the following: # # type Result = std::result::Result>; # -use async_listen::{ListenExt, Token}; +use async_listen::{ListenExt, Token, error_hint}; async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { let listener = TcpListener::bind(addr).await?; let mut incoming = listener .incoming() - .log_warnings(|e| eprintln!("Error: {}. Pausing for 500ms.", e)) + .log_warnings(log_accept_error) .handle_errors(Duration::from_millis(500)) // 1 .backpressure(100); while let Some((token, socket)) = incoming.next().await { // 2 @@ -241,6 +248,9 @@ async fn accept_loop(addr: impl ToSocketAddrs) -> Result<()> { async fn connection_loop(_token: &Token, stream: TcpStream) { // 4 // ... } +# fn log_accept_error(e: &io::Error) { +# eprintln!("Error: {}. Listener paused for 0.5s. {}", e, error_hint(e)); +# } ``` 1. We need to handle errors first, because [`backpressure`] helper expects From 0ed0d63094c00abdebb9f027754974703248e419 Mon Sep 17 00:00:00 2001 From: nasa Date: Tue, 14 Jan 2020 03:49:52 +0900 Subject: [PATCH 007/149] Remove unnecessary trait bound on FlatMap (#651) * Remove unnecessary trait bound on FlatMap * test: upgrade test code --- src/stream/stream/flat_map.rs | 1 - src/stream/stream/mod.rs | 18 +++++++++++------- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/stream/stream/flat_map.rs b/src/stream/stream/flat_map.rs index 6c828c9..9a7d921 100644 --- a/src/stream/stream/flat_map.rs +++ b/src/stream/stream/flat_map.rs @@ -41,7 +41,6 @@ where impl Stream for FlatMap where S: Stream, - S::Item: IntoStream, U: Stream, F: FnMut(S::Item) -> U, { diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index b98950b..4cf88c7 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -791,18 +791,22 @@ extension_trait! { # async_std::task::block_on(async { use async_std::prelude::*; - use async_std::stream::IntoStream; use async_std::stream; - let inner1 = stream::from_iter(vec![1,2,3]); - let inner2 = stream::from_iter(vec![4,5,6]); + let words = stream::from_iter(&["alpha", "beta", "gamma"]); - let s = stream::from_iter(vec![inner1, inner2]); + let merged: String = words + .flat_map(|s| stream::from_iter(s.chars())) + .collect().await; + assert_eq!(merged, "alphabetagamma"); - let v :Vec<_> = s.flat_map(|s| s.into_stream()).collect().await; - - assert_eq!(v, vec![1,2,3,4,5,6]); + let d3 = stream::from_iter(&[[[1, 2], [3, 4]], [[5, 6], [7, 8]]]); + let d1: Vec<_> = d3 + .flat_map(|item| stream::from_iter(item)) + .flat_map(|item| stream::from_iter(item)) + .collect().await; + assert_eq!(d1, [&1, &2, &3, &4, &5, &6, &7, &8]); # }); ``` "#] From 5d5064b871317b28e6a992650e350974acd36e25 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 13 Jan 2020 21:42:31 +0100 Subject: [PATCH 008/149] add FromStream Result example (#643) Signed-off-by: Yoshua Wuyts --- src/result/from_stream.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/result/from_stream.rs b/src/result/from_stream.rs index a8490d6..d6e790e 100644 --- a/src/result/from_stream.rs +++ b/src/result/from_stream.rs @@ -10,6 +10,23 @@ where /// Takes each element in the stream: if it is an `Err`, no further /// elements are taken, and the `Err` is returned. Should no `Err` /// occur, a container with the values of each `Result` is returned. + /// + /// # Examples + /// + /// ``` + /// # fn main() { async_std::task::block_on(async { + /// # + /// use async_std::prelude::*; + /// use async_std::stream; + /// + /// let v = stream::from_iter(vec![1, 2]); + /// let res: Result, &'static str> = v.map(|x: u32| + /// x.checked_add(1).ok_or("Overflow!") + /// ).collect().await; + /// assert_eq!(res, Ok(vec![2, 3])); + /// # + /// # }) } + /// ``` #[inline] fn from_stream<'a, S: IntoStream> + 'a>( stream: S, From a4f6806605d86fcd9056beb8bfd462a4d9377f7d Mon Sep 17 00:00:00 2001 From: noah Date: Mon, 13 Jan 2020 17:47:51 -0600 Subject: [PATCH 009/149] Fixes https://github.com/async-rs/async-std/issues/652 --- src/net/udp/mod.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 1b6d0d7..4ff3563 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -244,9 +244,12 @@ impl UdpSocket { })) } - /// Sends data on the socket to the given address. + /// Sends data on the socket to the remote address to which it is connected. /// - /// On success, returns the number of bytes written. + /// The [`connect`] method will connect this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// [`connect`]: #method.connect /// /// # Examples /// @@ -297,12 +300,11 @@ impl UdpSocket { /// # /// use async_std::net::UdpSocket; /// - /// let socket = UdpSocket::bind("127.0.0.1:0").await?; + /// let socket = UdpSocket::bind("127.0.0.1:7878").await?; /// socket.connect("127.0.0.1:8080").await?; + /// let bytes = socket.send(b"Hi there!").await?; /// - /// let mut buf = vec![0; 1024]; - /// let n = socket.recv(&mut buf).await?; - /// println!("Received {} bytes", n); + /// println!("Sent {} bytes", bytes); /// # /// # Ok(()) }) } /// ``` From f8dd3d98169ef5efca51d37e5995b626cf324b6a Mon Sep 17 00:00:00 2001 From: Qifan Lu Date: Tue, 10 Dec 2019 18:18:15 -0800 Subject: [PATCH 010/149] Add stream::pending::{pending, Pending} --- src/stream/mod.rs | 2 ++ src/stream/pending.rs | 49 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 src/stream/pending.rs diff --git a/src/stream/mod.rs b/src/stream/mod.rs index d8b96ec..0bfd4e8 100644 --- a/src/stream/mod.rs +++ b/src/stream/mod.rs @@ -325,6 +325,7 @@ cfg_unstable! { mod fused_stream; mod interval; mod into_stream; + mod pending; mod product; mod successors; mod sum; @@ -336,6 +337,7 @@ cfg_unstable! { pub use fused_stream::FusedStream; pub use interval::{interval, Interval}; pub use into_stream::IntoStream; + pub use pending::{pending, Pending}; pub use product::Product; pub use stream::Merge; pub use successors::{successors, Successors}; diff --git a/src/stream/pending.rs b/src/stream/pending.rs new file mode 100644 index 0000000..1fd0851 --- /dev/null +++ b/src/stream/pending.rs @@ -0,0 +1,49 @@ +use std::marker::PhantomData; +use std::pin::Pin; +use std::task::{Context, Poll}; + +use crate::stream::{Stream, DoubleEndedStream, ExactSizeStream, FusedStream}; + +/// A stream that never returns any items. +/// +/// This stream is created by the [`pending`] function. See its +/// documentation for more. +/// +/// [`pending`]: fn.pending.html +#[derive(Debug)] +pub struct Pending { + _marker: PhantomData +} + +/// Creates a stream that never returns any items. +/// +/// The returned stream will always return `Pending` when polled. +pub fn pending() -> Pending { + Pending { _marker: PhantomData } +} + +impl Stream for Pending { + type Item = T; + + fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { + Poll::Pending + } + + fn size_hint(&self) -> (usize, Option) { + (0, Some(0)) + } +} + +impl DoubleEndedStream for Pending { + fn poll_next_back(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { + Poll::Pending + } +} + +impl FusedStream for Pending {} + +impl ExactSizeStream for Pending { + fn len(&self) -> usize { + 0 + } +} From 879e14c6abf7d246871a7882075e8e8eb8c37d18 Mon Sep 17 00:00:00 2001 From: Qifan Lu Date: Tue, 10 Dec 2019 18:31:06 -0800 Subject: [PATCH 011/149] Remove size_hint from Stream impl --- src/stream/pending.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/stream/pending.rs b/src/stream/pending.rs index 1fd0851..943513e 100644 --- a/src/stream/pending.rs +++ b/src/stream/pending.rs @@ -28,10 +28,6 @@ impl Stream for Pending { fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { Poll::Pending } - - fn size_hint(&self) -> (usize, Option) { - (0, Some(0)) - } } impl DoubleEndedStream for Pending { From e9357c0307b8ea6bfaccccb0bb10ed4406960573 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 14 Jan 2020 09:49:34 +0900 Subject: [PATCH 012/149] style: Run `cargo fmt` --- src/stream/pending.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/stream/pending.rs b/src/stream/pending.rs index 943513e..5563d30 100644 --- a/src/stream/pending.rs +++ b/src/stream/pending.rs @@ -2,24 +2,26 @@ use std::marker::PhantomData; use std::pin::Pin; use std::task::{Context, Poll}; -use crate::stream::{Stream, DoubleEndedStream, ExactSizeStream, FusedStream}; +use crate::stream::{DoubleEndedStream, ExactSizeStream, FusedStream, Stream}; /// A stream that never returns any items. -/// +/// /// This stream is created by the [`pending`] function. See its /// documentation for more. -/// +/// /// [`pending`]: fn.pending.html #[derive(Debug)] pub struct Pending { - _marker: PhantomData + _marker: PhantomData, } /// Creates a stream that never returns any items. -/// +/// /// The returned stream will always return `Pending` when polled. pub fn pending() -> Pending { - Pending { _marker: PhantomData } + Pending { + _marker: PhantomData, + } } impl Stream for Pending { From f53fcbb7060390757b91edd1bc195ecc804be730 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 14 Jan 2020 10:18:14 +0900 Subject: [PATCH 013/149] test,docs: Add stream::pending example code --- src/stream/pending.rs | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/src/stream/pending.rs b/src/stream/pending.rs index 5563d30..922a540 100644 --- a/src/stream/pending.rs +++ b/src/stream/pending.rs @@ -18,6 +18,27 @@ pub struct Pending { /// Creates a stream that never returns any items. /// /// The returned stream will always return `Pending` when polled. +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use std::time::Duration; +/// +/// use async_std::prelude::*; +/// use async_std::stream; +/// +/// let dur = Duration::from_millis(100); +/// let mut s = stream::pending::<()>().timeout(dur); +/// +/// let item = s.next().await; +/// +/// assert!(item.is_some()); +/// assert!(item.unwrap().is_err()); +/// +/// # +/// # }) +/// ``` pub fn pending() -> Pending { Pending { _marker: PhantomData, From 76ed174fd5e3e60386dcbbd810613a27613207d1 Mon Sep 17 00:00:00 2001 From: nasa Date: Tue, 14 Jan 2020 23:26:22 +0900 Subject: [PATCH 014/149] Version up of dependent crate (#672) --- Cargo.toml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 63d599a..60ea91a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -60,18 +60,18 @@ futures-io = { version = "0.3.1", 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.2.1", optional = true } +memchr = { version = "2.3.0", 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.1", optional = true } +pin-project-lite = { version = "0.1.2", 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.2" +rand = "0.7.3" surf = "1.0.3" tempdir = "0.3.7" futures = "0.3.1" From 76993dd7557ca0c0c11f73b5d4e10cc91abd33de Mon Sep 17 00:00:00 2001 From: noah Date: Tue, 14 Jan 2020 10:55:10 -0600 Subject: [PATCH 015/149] Revert "Fixes https://github.com/async-rs/async-std/issues/652" This reverts commit a4f68066 --- src/net/udp/mod.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 4ff3563..1b6d0d7 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -244,12 +244,9 @@ impl UdpSocket { })) } - /// Sends data on the socket to the remote address to which it is connected. - /// - /// The [`connect`] method will connect this socket to a remote address. - /// This method will fail if the socket is not connected. + /// Sends data on the socket to the given address. /// - /// [`connect`]: #method.connect + /// On success, returns the number of bytes written. /// /// # Examples /// @@ -300,11 +297,12 @@ impl UdpSocket { /// # /// use async_std::net::UdpSocket; /// - /// let socket = UdpSocket::bind("127.0.0.1:7878").await?; + /// let socket = UdpSocket::bind("127.0.0.1:0").await?; /// socket.connect("127.0.0.1:8080").await?; - /// let bytes = socket.send(b"Hi there!").await?; /// - /// println!("Sent {} bytes", bytes); + /// let mut buf = vec![0; 1024]; + /// let n = socket.recv(&mut buf).await?; + /// println!("Received {} bytes", n); /// # /// # Ok(()) }) } /// ``` From 0a528647649971453aa5a724023a2cba84638418 Mon Sep 17 00:00:00 2001 From: noah Date: Tue, 14 Jan 2020 10:59:17 -0600 Subject: [PATCH 016/149] Revert "Fixes https://github.com/async-rs/async-std/issues/652" This reverts commit a4f68066 --- src/net/udp/mod.rs | 22 +++++++++------------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/src/net/udp/mod.rs b/src/net/udp/mod.rs index 1b6d0d7..e3bd714 100644 --- a/src/net/udp/mod.rs +++ b/src/net/udp/mod.rs @@ -244,9 +244,12 @@ impl UdpSocket { })) } - /// Sends data on the socket to the given address. + /// Sends data on the socket to the remote address to which it is connected. /// - /// On success, returns the number of bytes written. + /// The [`connect`] method will connect this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// [`connect`]: #method.connect /// /// # Examples /// @@ -255,18 +258,11 @@ impl UdpSocket { /// # /// use async_std::net::UdpSocket; /// - /// const THE_MERCHANT_OF_VENICE: &[u8] = b" - /// If you prick us, do we not bleed? - /// If you tickle us, do we not laugh? - /// If you poison us, do we not die? - /// And if you wrong us, shall we not revenge? - /// "; - /// - /// let socket = UdpSocket::bind("127.0.0.1:0").await?; + /// let socket = UdpSocket::bind("127.0.0.1:34254").await?; + /// socket.connect("127.0.0.1:8080").await?; + /// let bytes = socket.send(b"Hi there!").await?; /// - /// let addr = "127.0.0.1:7878"; - /// let sent = socket.send_to(THE_MERCHANT_OF_VENICE, &addr).await?; - /// println!("Sent {} bytes to {}", sent, addr); + /// println!("Sent {} bytes", bytes); /// # /// # Ok(()) }) } /// ``` From ee102dfc9e6d76593527524ecc36cfbbdb14f32d Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 15 Jan 2020 10:41:39 +0900 Subject: [PATCH 017/149] docs: Add stream::timeout example when timeout error --- src/stream/stream/mod.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index 4cf88c7..6c84ac2 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -1645,6 +1645,13 @@ extension_trait! { while let Some(v) = s.next().await { assert_eq!(v, Ok(1)); } + + // when timeout + let mut s = stream::pending::<()>().timeout(Duration::from_millis(10)); + match s.next().await { + Some(item) => assert!(item.is_err()), + None => panic!() + }; # # Ok(()) }) } ``` From b72dd83726f89a740f393389ff4c9ef0c4791ec4 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 15 Jan 2020 11:00:03 +0900 Subject: [PATCH 018/149] update async-task to 1.2.1 --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 63d599a..339db1f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -50,7 +50,7 @@ std = [ [dependencies] async-attributes = { version = "1.1.1", optional = true } -async-task = { version = "1.0.0", optional = true } +async-task = { version = "1.2.1", 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 } From 134089af2c711e3786beb4174e0f1592d9de7752 Mon Sep 17 00:00:00 2001 From: Oleg Nosov Date: Wed, 15 Jan 2020 08:55:18 +0300 Subject: [PATCH 019/149] Use `filter_map(identity)` + other fixes --- src/option/from_stream.rs | 3 ++- src/option/product.rs | 3 ++- src/option/sum.rs | 5 +++-- src/result/from_stream.rs | 2 +- src/result/product.rs | 28 +++++++++++++++------------- src/result/sum.rs | 26 ++++++++++++++------------ src/unit/from_stream.rs | 2 +- src/utils.rs | 8 ++++++++ 8 files changed, 46 insertions(+), 31 deletions(-) diff --git a/src/option/from_stream.rs b/src/option/from_stream.rs index f0aafea..2194f29 100644 --- a/src/option/from_stream.rs +++ b/src/option/from_stream.rs @@ -2,6 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{FromStream, IntoStream}; +use crate::utils::identity; impl FromStream> for Option where @@ -28,7 +29,7 @@ where false } }) - .map(Option::unwrap) + .filter_map(identity) .collect() .await; diff --git a/src/option/product.rs b/src/option/product.rs index 2238a91..f733bef 100644 --- a/src/option/product.rs +++ b/src/option/product.rs @@ -2,6 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{Product, Stream}; +use crate::utils::identity; impl Product> for Option where @@ -52,7 +53,7 @@ where false } }) - .map(Option::unwrap), + .filter_map(identity), ) .await; diff --git a/src/option/sum.rs b/src/option/sum.rs index 0570a1e..64fdb07 100644 --- a/src/option/sum.rs +++ b/src/option/sum.rs @@ -2,6 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{Stream, Sum}; +use crate::utils::identity; impl Sum> for Option where @@ -43,11 +44,11 @@ where .take_while(|elem| { elem.is_some() || { found_none = true; - // Stop processing the stream on error + // Stop processing the stream on `None` false } }) - .map(Option::unwrap), + .filter_map(identity), ) .await; diff --git a/src/result/from_stream.rs b/src/result/from_stream.rs index a8490d6..424a53b 100644 --- a/src/result/from_stream.rs +++ b/src/result/from_stream.rs @@ -21,7 +21,7 @@ where // if a failure occurs let mut found_error = None; let out: V = stream - .scan((), |_, elem| { + .scan((), |(), elem| { match elem { Ok(elem) => Some(elem), Err(err) => { diff --git a/src/result/product.rs b/src/result/product.rs index ec9d94a..ad2e014 100644 --- a/src/result/product.rs +++ b/src/result/product.rs @@ -1,7 +1,7 @@ use std::pin::Pin; use crate::prelude::*; -use crate::stream::{Stream, Product}; +use crate::stream::{Product, Stream}; impl Product> for Result where @@ -36,26 +36,28 @@ where ``` "#] fn product<'a, S>(stream: S) -> Pin> + 'a>> - where S: Stream> + 'a + where + S: Stream> + 'a, { Box::pin(async move { // Using `scan` here because it is able to stop the stream early // if a failure occurs let mut found_error = None; - let out = >::product(stream - .scan((), |_, elem| { - match elem { - Ok(elem) => Some(elem), - Err(err) => { - found_error = Some(err); - // Stop processing the stream on error - None - } + let out = >::product(stream.scan((), |(), elem| { + match elem { + Ok(elem) => Some(elem), + Err(err) => { + found_error = Some(err); + // Stop processing the stream on error + None } - })).await; + } + })) + .await; + match found_error { Some(err) => Err(err), - None => Ok(out) + None => Ok(out), } }) } diff --git a/src/result/sum.rs b/src/result/sum.rs index ccc4240..1cf5b7a 100644 --- a/src/result/sum.rs +++ b/src/result/sum.rs @@ -36,26 +36,28 @@ where ``` "#] fn sum<'a, S>(stream: S) -> Pin> + 'a>> - where S: Stream> + 'a + where + S: Stream> + 'a, { Box::pin(async move { // Using `scan` here because it is able to stop the stream early // if a failure occurs let mut found_error = None; - let out = >::sum(stream - .scan((), |_, elem| { - match elem { - Ok(elem) => Some(elem), - Err(err) => { - found_error = Some(err); - // Stop processing the stream on error - None - } + let out = >::sum(stream.scan((), |(), elem| { + match elem { + Ok(elem) => Some(elem), + Err(err) => { + found_error = Some(err); + // Stop processing the stream on error + None } - })).await; + } + })) + .await; + match found_error { Some(err) => Err(err), - None => Ok(out) + None => Ok(out), } }) } diff --git a/src/unit/from_stream.rs b/src/unit/from_stream.rs index da216e2..7b78438 100644 --- a/src/unit/from_stream.rs +++ b/src/unit/from_stream.rs @@ -8,6 +8,6 @@ impl FromStream<()> for () { fn from_stream<'a, S: IntoStream + 'a>( stream: S, ) -> Pin + 'a>> { - Box::pin(stream.into_stream().for_each(|_| ())) + Box::pin(stream.into_stream().for_each(drop)) } } diff --git a/src/utils.rs b/src/utils.rs index ef50ed0..780e733 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -52,6 +52,14 @@ pub fn random(n: u32) -> u32 { }) } +/// Returns given argument without changes. +#[allow(dead_code)] +#[doc(hidden)] +#[inline(always)] +pub(crate) fn identity(arg: T) -> T { + arg +} + /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; From 38de0bfd2267cb299952046f129eb6cc896bde9a Mon Sep 17 00:00:00 2001 From: Oleg Nosov Date: Wed, 15 Jan 2020 09:42:30 +0300 Subject: [PATCH 020/149] Use `std::convert::identity` --- src/option/from_stream.rs | 8 ++------ src/option/product.rs | 8 ++------ src/option/sum.rs | 8 ++------ src/utils.rs | 8 -------- 4 files changed, 6 insertions(+), 26 deletions(-) diff --git a/src/option/from_stream.rs b/src/option/from_stream.rs index 2194f29..de929ca 100644 --- a/src/option/from_stream.rs +++ b/src/option/from_stream.rs @@ -2,7 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{FromStream, IntoStream}; -use crate::utils::identity; +use std::convert::identity; impl FromStream> for Option where @@ -33,11 +33,7 @@ where .collect() .await; - if found_none { - None - } else { - Some(out) - } + if found_none { None } else { Some(out) } }) } } diff --git a/src/option/product.rs b/src/option/product.rs index f733bef..b446c1f 100644 --- a/src/option/product.rs +++ b/src/option/product.rs @@ -2,7 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{Product, Stream}; -use crate::utils::identity; +use std::convert::identity; impl Product> for Option where @@ -57,11 +57,7 @@ where ) .await; - if found_none { - None - } else { - Some(out) - } + if found_none { None } else { Some(out) } }) } } diff --git a/src/option/sum.rs b/src/option/sum.rs index 64fdb07..de404f4 100644 --- a/src/option/sum.rs +++ b/src/option/sum.rs @@ -2,7 +2,7 @@ use std::pin::Pin; use crate::prelude::*; use crate::stream::{Stream, Sum}; -use crate::utils::identity; +use std::convert::identity; impl Sum> for Option where @@ -52,11 +52,7 @@ where ) .await; - if found_none { - None - } else { - Some(out) - } + if found_none { None } else { Some(out) } }) } } diff --git a/src/utils.rs b/src/utils.rs index 780e733..ef50ed0 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -52,14 +52,6 @@ pub fn random(n: u32) -> u32 { }) } -/// Returns given argument without changes. -#[allow(dead_code)] -#[doc(hidden)] -#[inline(always)] -pub(crate) fn identity(arg: T) -> T { - arg -} - /// Add additional context to errors pub(crate) trait Context { fn context(self, message: impl Fn() -> String) -> Self; From ed248017b476f17334260db31daf82cc07c2a465 Mon Sep 17 00:00:00 2001 From: Oleg Nosov Date: Wed, 15 Jan 2020 12:06:50 +0300 Subject: [PATCH 021/149] Use internal `scan` state in `Result`s implementation --- src/result/from_stream.rs | 4 ++-- src/result/product.rs | 4 ++-- src/result/sum.rs | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/result/from_stream.rs b/src/result/from_stream.rs index 424a53b..86405ce 100644 --- a/src/result/from_stream.rs +++ b/src/result/from_stream.rs @@ -21,11 +21,11 @@ where // if a failure occurs let mut found_error = None; let out: V = stream - .scan((), |(), elem| { + .scan(&mut found_error, |error, elem| { match elem { Ok(elem) => Some(elem), Err(err) => { - found_error = Some(err); + **error = Some(err); // Stop processing the stream on error None } diff --git a/src/result/product.rs b/src/result/product.rs index ad2e014..89ddacb 100644 --- a/src/result/product.rs +++ b/src/result/product.rs @@ -43,11 +43,11 @@ where // Using `scan` here because it is able to stop the stream early // if a failure occurs let mut found_error = None; - let out = >::product(stream.scan((), |(), elem| { + let out = >::product(stream.scan(&mut found_error, |error, elem| { match elem { Ok(elem) => Some(elem), Err(err) => { - found_error = Some(err); + **error = Some(err); // Stop processing the stream on error None } diff --git a/src/result/sum.rs b/src/result/sum.rs index 1cf5b7a..c0ef4c2 100644 --- a/src/result/sum.rs +++ b/src/result/sum.rs @@ -43,11 +43,11 @@ where // Using `scan` here because it is able to stop the stream early // if a failure occurs let mut found_error = None; - let out = >::sum(stream.scan((), |(), elem| { + let out = >::sum(stream.scan(&mut found_error, |error, elem| { match elem { Ok(elem) => Some(elem), Err(err) => { - found_error = Some(err); + **error = Some(err); // Stop processing the stream on error None } From d283352a9add80f722c272238c6f9e122976aedb Mon Sep 17 00:00:00 2001 From: nasa Date: Fri, 17 Jan 2020 03:07:11 +0900 Subject: [PATCH 022/149] update broadcastor to 1.0.0 (#681) --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 65541db..b1155bb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -51,7 +51,7 @@ std = [ [dependencies] async-attributes = { version = "1.1.1", optional = true } async-task = { version = "1.2.1", optional = true } -broadcaster = { version = "0.2.6", optional = true, default-features = false, features = ["default-channels"] } +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 } From ed7ddacb28f1cb3a5c4b7b2a92afa6003dc320ef Mon Sep 17 00:00:00 2001 From: Oleg Nosov Date: Fri, 17 Jan 2020 17:13:52 +0300 Subject: [PATCH 023/149] Rewrote `Result`s implementation using `take_while` and `filter_map` --- src/result/from_stream.rs | 32 ++++++++++++++++++++------------ src/result/product.rs | 39 +++++++++++++++++++++++++-------------- src/result/sum.rs | 39 +++++++++++++++++++++++++-------------- 3 files changed, 70 insertions(+), 40 deletions(-) diff --git a/src/result/from_stream.rs b/src/result/from_stream.rs index 86405ce..8ce4b85 100644 --- a/src/result/from_stream.rs +++ b/src/result/from_stream.rs @@ -17,26 +17,34 @@ where let stream = stream.into_stream(); Box::pin(async move { - // Using `scan` here because it is able to stop the stream early + // Using `take_while` here because it is able to stop the stream early // if a failure occurs + let mut is_error = false; let mut found_error = None; let out: V = stream - .scan(&mut found_error, |error, elem| { - match elem { - Ok(elem) => Some(elem), - Err(err) => { - **error = Some(err); - // Stop processing the stream on error - None - } + .take_while(|elem| { + // Stop processing the stream on `Err` + !is_error + && (elem.is_ok() || { + is_error = true; + // Capture first `Err` + true + }) + }) + .filter_map(|elem| match elem { + Ok(value) => Some(value), + Err(err) => { + found_error = Some(err); + None } }) .collect() .await; - match found_error { - Some(err) => Err(err), - None => Ok(out), + if is_error { + Err(found_error.unwrap()) + } else { + Ok(out) } }) } diff --git a/src/result/product.rs b/src/result/product.rs index 89ddacb..45782ff 100644 --- a/src/result/product.rs +++ b/src/result/product.rs @@ -40,24 +40,35 @@ where S: Stream> + 'a, { Box::pin(async move { - // Using `scan` here because it is able to stop the stream early + // Using `take_while` here because it is able to stop the stream early // if a failure occurs + let mut is_error = false; let mut found_error = None; - let out = >::product(stream.scan(&mut found_error, |error, elem| { - match elem { - Ok(elem) => Some(elem), - Err(err) => { - **error = Some(err); - // Stop processing the stream on error - None - } - } - })) + let out = >::product( + stream + .take_while(|elem| { + // Stop processing the stream on `Err` + !is_error + && (elem.is_ok() || { + is_error = true; + // Capture first `Err` + true + }) + }) + .filter_map(|elem| match elem { + Ok(value) => Some(value), + Err(err) => { + found_error = Some(err); + None + } + }), + ) .await; - match found_error { - Some(err) => Err(err), - None => Ok(out), + if is_error { + Err(found_error.unwrap()) + } else { + Ok(out) } }) } diff --git a/src/result/sum.rs b/src/result/sum.rs index c0ef4c2..b6d84a0 100644 --- a/src/result/sum.rs +++ b/src/result/sum.rs @@ -40,24 +40,35 @@ where S: Stream> + 'a, { Box::pin(async move { - // Using `scan` here because it is able to stop the stream early + // Using `take_while` here because it is able to stop the stream early // if a failure occurs + let mut is_error = false; let mut found_error = None; - let out = >::sum(stream.scan(&mut found_error, |error, elem| { - match elem { - Ok(elem) => Some(elem), - Err(err) => { - **error = Some(err); - // Stop processing the stream on error - None - } - } - })) + let out = >::sum( + stream + .take_while(|elem| { + // Stop processing the stream on `Err` + !is_error + && (elem.is_ok() || { + is_error = true; + // Capture first `Err` + true + }) + }) + .filter_map(|elem| match elem { + Ok(value) => Some(value), + Err(err) => { + found_error = Some(err); + None + } + }), + ) .await; - match found_error { - Some(err) => Err(err), - None => Ok(out), + if is_error { + Err(found_error.unwrap()) + } else { + Ok(out) } }) } From 2221441a4c40ce5f4117d92d34f986d93a71a66b Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 18 Jan 2020 08:36:54 +0900 Subject: [PATCH 024/149] feat: Implement Clone trait for DirEntry --- src/fs/dir_entry.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/fs/dir_entry.rs b/src/fs/dir_entry.rs index 527fab4..e1622ee 100644 --- a/src/fs/dir_entry.rs +++ b/src/fs/dir_entry.rs @@ -158,6 +158,12 @@ impl fmt::Debug for DirEntry { } } +impl Clone for DirEntry { + fn clone(&self) -> Self { + DirEntry(self.0.clone()) + } +} + cfg_unix! { use crate::os::unix::fs::DirEntryExt; From 81aa6d152a6be24f1ba19e63ad78318315a67c07 Mon Sep 17 00:00:00 2001 From: Katharina Fey Date: Mon, 20 Jan 2020 20:40:01 +0100 Subject: [PATCH 025/149] Changing task::block_on to park after a single poll (#684) This was previously discussed in #605 and others as a source of high CPU load when sleeping tasks because of the overhead created by retrying a future in short succession. --- src/task/block_on.rs | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/src/task/block_on.rs b/src/task/block_on.rs index 80259c5..a994ee7 100644 --- a/src/task/block_on.rs +++ b/src/task/block_on.rs @@ -3,7 +3,6 @@ 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; @@ -125,7 +124,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 +131,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(); } }) } From 6b860c370a182803bd39cc37867dcb8bafa5810e Mon Sep 17 00:00:00 2001 From: Taiki Endo Date: Tue, 21 Jan 2020 05:41:48 +0900 Subject: [PATCH 026/149] Remove usage of unstable format_code_in_doc_comments option (#685) --- rustfmt.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/rustfmt.toml b/rustfmt.toml index c6d404e..1082fd8 100644 --- a/rustfmt.toml +++ b/rustfmt.toml @@ -1,2 +1 @@ version = "Two" -format_code_in_doc_comments = true From f9fe5c90cf704887e78b0487cfd44bc5f7d10a11 Mon Sep 17 00:00:00 2001 From: Florian Gilcher Date: Wed, 22 Jan 2020 12:47:18 +0100 Subject: [PATCH 027/149] Fix some typos in accept-loop pattern chapter --- docs/src/patterns/accept-loop.md | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/src/patterns/accept-loop.md b/docs/src/patterns/accept-loop.md index 46d6cc6..43c3f41 100644 --- a/docs/src/patterns/accept-loop.md +++ b/docs/src/patterns/accept-loop.md @@ -1,6 +1,6 @@ # Production-Ready Accept Loop -Production-ready accept loop needs the following things: +A production-ready accept loop needs the following things: 1. Handling errors 2. Limiting the number of simultanteous connections to avoid deny-of-service (DoS) attacks @@ -8,15 +8,15 @@ Production-ready accept loop needs the following things: ## Handling errors -There are two kinds of errors in accept loop: -1. Per-connection errors. System uses them to notify that there was a - connection in the queue and it's dropped by peer. Subsequent connection +There are two kinds of errors in an accept loop: +1. Per-connection errors. The system uses them to notify that there was a + connection in the queue and it's dropped by the peer. Subsequent connections can be already queued so next connection must be accepted immediately. 2. Resource shortages. When these are encountered it doesn't make sense to - accept next socket immediately. But listener stays active, so you server + accept the next socket immediately. But the listener stays active, so you server should try to accept socket later. -Here is the example of per-connection error (printed in normal and debug mode): +Here is the example of a per-connection error (printed in normal and debug mode): ``` Error: Connection reset by peer (os error 104) Error: Os { code: 104, kind: ConnectionReset, message: "Connection reset by peer" } @@ -30,10 +30,10 @@ Error: Os { code: 24, kind: Other, message: "Too many open files" } ### Testing Application -To test your application on these errors try the following (this works +To test your application for these errors try the following (this works on unixes only). -Lower limit and start the application: +Lower limits and start the application: ``` $ ulimit -n 100 $ cargo run --example your_app @@ -42,7 +42,7 @@ $ cargo run --example your_app Running `target/debug/examples/your_app` Server is listening on: http://127.0.0.1:1234 ``` -Then in another console run [`wrk`] benchmark tool: +Then in another console run the [`wrk`] benchmark tool: ``` $ wrk -c 1000 http://127.0.0.1:1234 Running 10s test @ http://localhost:8080/ @@ -54,13 +54,13 @@ Connected to localhost. Important is to check the following things: -1. Application doesn't crash on error (but may log errors, see below) +1. The application doesn't crash on error (but may log errors, see below) 2. It's possible to connect to the application again once load is stopped (few seconds after `wrk`). This is what `telnet` does in example above, make sure it prints `Connected to `. 3. The `Too many open files` error is logged in the appropriate log. This requires to set "maximum number of simultaneous connections" parameter (see - below) of your application to a value greater that `100` for this example. + below) of your application to a value greater then `100` for this example. 4. Check CPU usage of the app while doing a test. It should not occupy 100% of a single CPU core (it's unlikely that you can exhaust CPU by 1000 connections in Rust, so this means error handling is not right). @@ -68,12 +68,12 @@ Important is to check the following things: #### Testing non-HTTP applications If it's possible, use the appropriate benchmark tool and set the appropriate -number of connections. For example `redis-benchmark` has `-c` parameter for +number of connections. For example `redis-benchmark` has a `-c` parameter for that, if you implement redis protocol. Alternatively, can still use `wrk`, just make sure that connection is not immediately closed. If it is, put a temporary timeout before handing -connection to the protocol handler, like this: +the connection to the protocol handler, like this: ```rust,edition2018 # extern crate async_std; @@ -147,7 +147,7 @@ Be sure to [test your application](#testing-application). ### External Crates -The crate [`async-listen`] have a helper to achieve this task: +The crate [`async-listen`] has a helper to achieve this task: ```rust,edition2018 # extern crate async_std; # extern crate async_listen; @@ -200,7 +200,7 @@ Even if you've applied everything described in Let's imagine you have a server that needs to open a file to process client request. At some point, you might encounter the following situation: -1. There are as much client connection as max file descriptors allowed for +1. There are as many client connection as max file descriptors allowed for the application. 2. Listener gets `Too many open files` error so it sleeps. 3. Some client sends a request via the previously open connection. @@ -257,7 +257,7 @@ async fn connection_loop(_token: &Token, stream: TcpStream) { // 4 stream of `TcpStream` rather than `Result`. 2. The token yielded by a new stream is what is counted by backpressure helper. I.e. if you drop a token, new connection can be established. -3. We give connection loop a reference to token to bind token's lifetime to +3. We give the connection loop a reference to token to bind token's lifetime to the lifetime of the connection. 4. The token itsellf in the function can be ignored, hence `_token` From b258215952b5819c08b05c838c285395b5f88ebd Mon Sep 17 00:00:00 2001 From: ninj Date: Sat, 25 Jan 2020 22:13:26 +0000 Subject: [PATCH 028/149] fix syntax problem for task::sleep --- docs/src/patterns/accept-loop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/src/patterns/accept-loop.md b/docs/src/patterns/accept-loop.md index 43c3f41..4508baf 100644 --- a/docs/src/patterns/accept-loop.md +++ b/docs/src/patterns/accept-loop.md @@ -90,7 +90,7 @@ the connection to the protocol handler, like this: # let mut incoming = listener.incoming(); while let Some(stream) = incoming.next().await { task::spawn(async { - task:sleep(Duration::from_secs(10)).await; // 1 + task::sleep(Duration::from_secs(10)).await; // 1 connection_loop(stream).await; }); } From 57974ae0b7be220d9e96e0262fd8bd7d8d1e851d Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Mon, 27 Jan 2020 23:13:13 +0100 Subject: [PATCH 029/149] Use non-blocking connect for TcpStream. (#687) * Use non-blocking connect for TcpStream. Instead of spawning a background thread which is unaware of any timeouts but continues to run until the TCP stack decides that the remote is not reachable we use mio's non-blocking connect. mio's `TcpStream::connect` returns immediately but the actual connection is usually just in progress and we have to be sure the socket is writeable before we can consider the connection as established. * Add Watcher::{poll_read_ready, poll_write_ready}. Following a suggestion of @stjepang we offer methods to check for read/write readiness of a `Watcher` instead of the previous approach to accept a set of `Waker`s when registering an event source. The changes relative to master are smaller and both methods look more useful in other contexts. Also the code is more robust w.r.t. wakeups of the `Waker` from clones outside the `Reactor`. I am not sure if we need to add protection mechanisms against spurious wakeups from mio. Currently we treat the `Poll::Ready(())` of `Watcher::poll_write_ready` as proof that the non-blocking connect has finished, but if the event from mio was a spurious one, it might still be ongoing. --- src/net/driver/mod.rs | 89 +++++++++++++++++++++++++++++++++++++------ src/net/tcp/stream.rs | 33 ++++++++-------- 2 files changed, 95 insertions(+), 27 deletions(-) diff --git a/src/net/driver/mod.rs b/src/net/driver/mod.rs index 7f33e85..07ef2c7 100644 --- a/src/net/driver/mod.rs +++ b/src/net/driver/mod.rs @@ -16,10 +16,30 @@ struct Entry { token: mio::Token, /// Tasks that are blocked on reading from this I/O handle. - readers: Mutex>, + readers: Mutex, /// Thasks that are blocked on writing to this I/O handle. - writers: Mutex>, + writers: Mutex, +} + +/// 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 } /// The state of a networking driver. @@ -68,8 +88,8 @@ impl Reactor { // Allocate an entry and insert it into the slab. let entry = Arc::new(Entry { token, - readers: Mutex::new(Vec::new()), - writers: Mutex::new(Vec::new()), + readers: Mutex::new(Readers { ready: false, wakers: Vec::new() }), + writers: Mutex::new(Writers { ready: false, wakers: Vec::new() }), }); vacant.insert(entry.clone()); @@ -144,14 +164,18 @@ fn main_loop() -> io::Result<()> { // Wake up reader tasks blocked on this I/O handle. if !(readiness & reader_interests()).is_empty() { - for w in entry.readers.lock().unwrap().drain(..) { + let mut readers = entry.readers.lock().unwrap(); + readers.ready = true; + for w in readers.wakers.drain(..) { w.wake(); } } // Wake up writer tasks blocked on this I/O handle. if !(readiness & writer_interests()).is_empty() { - for w in entry.writers.lock().unwrap().drain(..) { + let mut writers = entry.writers.lock().unwrap(); + writers.ready = true; + for w in writers.wakers.drain(..) { w.wake(); } } @@ -207,7 +231,7 @@ impl Watcher { } // Lock the waker list. - let mut list = self.entry.readers.lock().unwrap(); + let mut readers = self.entry.readers.lock().unwrap(); // Try running the operation again. match f(self.source.as_ref().unwrap()) { @@ -216,10 +240,12 @@ impl Watcher { } // Register the task if it isn't registered already. - if list.iter().all(|w| !w.will_wake(cx.waker())) { - list.push(cx.waker().clone()); + if readers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { + readers.wakers.push(cx.waker().clone()); } + readers.ready = false; + Poll::Pending } @@ -242,7 +268,7 @@ impl Watcher { } // Lock the waker list. - let mut list = self.entry.writers.lock().unwrap(); + let mut writers = self.entry.writers.lock().unwrap(); // Try running the operation again. match f(self.source.as_ref().unwrap()) { @@ -251,10 +277,49 @@ impl Watcher { } // Register the task if it isn't registered already. - if list.iter().all(|w| !w.will_wake(cx.waker())) { - list.push(cx.waker().clone()); + if writers.wakers.iter().all(|w| !w.will_wake(cx.waker())) { + writers.wakers.push(cx.waker().clone()); } + writers.ready = false; + + 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 } diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index ae8ca7d..537bd4c 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -6,8 +6,7 @@ use crate::future; use crate::io::{self, Read, Write}; use crate::net::driver::Watcher; use crate::net::ToSocketAddrs; -use crate::task::{spawn_blocking, Context, Poll}; -use crate::utils::Context as _; +use crate::task::{Context, Poll}; /// A TCP stream between a local and a remote socket. /// @@ -77,20 +76,24 @@ impl TcpStream { .await?; for addr in addrs { - let res = spawn_blocking(move || { - let std_stream = std::net::TcpStream::connect(addr) - .context(|| format!("could not connect to {}", addr))?; - let mio_stream = mio::net::TcpStream::from_stream(std_stream) - .context(|| format!("could not open async connection to {}", addr))?; - Ok(TcpStream { - watcher: Watcher::new(mio_stream), - }) - }) - .await; + // 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), + Err(e) => { + last_err = Some(e); + continue + } + }; - match res { - Ok(stream) => return Ok(stream), - Err(err) => last_err = Some(err), + future::poll_fn(|cx| watcher.poll_write_ready(cx)).await; + + match watcher.get_ref().take_error() { + Ok(None) => return Ok(TcpStream { watcher }), + Ok(Some(e)) => last_err = Some(e), + Err(e) => last_err = Some(e) } } From 4996f297788542074d6fe37675d059a70cad85b8 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 10:30:27 +0900 Subject: [PATCH 030/149] feat: Add no-std feature --- Cargo.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index b1155bb..fbcf5f4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,7 +46,9 @@ std = [ "pin-project-lite", "pin-utils", "slab", + "no-std", ] +no-std = [] [dependencies] async-attributes = { version = "1.1.1", optional = true } From 51b84a7620823d8b1c081d0d1d994e95b2001f17 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 10:31:27 +0900 Subject: [PATCH 031/149] feat: Add no_std attribute when not std feature --- src/lib.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/lib.rs b/src/lib.rs index c9d1277..237d73f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -221,6 +221,7 @@ //! features = ["std"] //! ``` +#![cfg_attr(not(feature = "std"), no_std)] #![cfg_attr(feature = "docs", feature(doc_cfg))] #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms)] #![allow(clippy::mutex_atomic, clippy::module_inception)] From 3d32fd81f4a2993086c97da6dad8a529c7c597f4 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 10:33:00 +0900 Subject: [PATCH 032/149] feat: Make the utils module no_std --- src/utils.rs | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/src/utils.rs b/src/utils.rs index ef50ed0..ccf5e84 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -1,3 +1,9 @@ +#[cfg(feature = "no-std")] +extern crate alloc; + +#[cfg(feature = "no-std")] +use alloc::string::String; + /// Calls a function and aborts if it panics. /// /// This is useful in unsafe code where we can't recover from panics. @@ -104,6 +110,7 @@ macro_rules! cfg_unstable_default { /// Declares Unix-specific items. #[doc(hidden)] +#[allow(unused_macros)] macro_rules! cfg_unix { ($($item:item)*) => { $( @@ -116,6 +123,7 @@ macro_rules! cfg_unix { /// Declares Windows-specific items. #[doc(hidden)] +#[allow(unused_macros)] macro_rules! cfg_windows { ($($item:item)*) => { $( @@ -128,6 +136,7 @@ macro_rules! cfg_windows { /// Declares items when the "docs" feature is enabled. #[doc(hidden)] +#[allow(unused_macros)] macro_rules! cfg_docs { ($($item:item)*) => { $( @@ -139,6 +148,7 @@ macro_rules! cfg_docs { /// Declares items when the "docs" feature is disabled. #[doc(hidden)] +#[allow(unused_macros)] macro_rules! cfg_not_docs { ($($item:item)*) => { $( @@ -160,6 +170,18 @@ macro_rules! cfg_std { } } +/// Declares no-std items. +#[allow(unused_macros)] +#[doc(hidden)] +macro_rules! cfg_no_std { + ($($item:item)*) => { + $( + #[cfg(feature = "no-std")] + $item + )* + } +} + /// Declares default items. #[allow(unused_macros)] #[doc(hidden)] @@ -180,6 +202,7 @@ macro_rules! cfg_default { /// /// Inside invocations of this macro, we write a definitions that looks similar to the final /// rendered docs, and the macro then generates all the boilerplate for us. +#[allow(unused_macros)] #[doc(hidden)] macro_rules! extension_trait { ( @@ -204,14 +227,14 @@ macro_rules! extension_trait { #[allow(dead_code)] mod owned { #[doc(hidden)] - pub struct ImplFuture(std::marker::PhantomData); + pub struct ImplFuture(core::marker::PhantomData); } // A fake `impl Future` type that borrows its environment. #[allow(dead_code)] mod borrowed { #[doc(hidden)] - pub struct ImplFuture<'a, T>(std::marker::PhantomData<&'a T>); + pub struct ImplFuture<'a, T>(core::marker::PhantomData<&'a T>); } // Render a fake trait combining the bodies of the base trait and the extension trait. From 41f114d9fe17a9a0822570bfa4f90365b35f5d66 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 10:37:22 +0900 Subject: [PATCH 033/149] ci: Add no-std check --- .github/workflows/ci.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index bee2562..0b0e84f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -40,6 +40,7 @@ jobs: with: command: check args: --features unstable --all --bins --examples --tests + - name: check bench uses: actions-rs/cargo@v1 if: matrix.rust == 'nightly' @@ -53,6 +54,12 @@ jobs: command: check args: --no-default-features --features std + - name: check no_std + uses: actions-rs/cargo@v1 + with: + command: check + args: --no-default-features --features no-std + - name: check attributes uses: actions-rs/cargo@v1 with: From 6aa55fde59bea45c1e5e7184782c882a4259813f Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 11:18:09 +0900 Subject: [PATCH 034/149] feat: Make the task module no_std --- Cargo.toml | 5 +++-- src/lib.rs | 5 ++++- src/task/mod.rs | 11 +++++++---- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index fbcf5f4..4492048 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,7 +39,6 @@ unstable = ["std", "broadcaster", "futures-timer"] attributes = ["async-attributes"] std = [ "crossbeam-utils", - "futures-core", "futures-io", "memchr", "once_cell", @@ -48,7 +47,9 @@ std = [ "slab", "no-std", ] -no-std = [] +no-std = [ + "futures-core", +] [dependencies] async-attributes = { version = "1.1.1", optional = true } diff --git a/src/lib.rs b/src/lib.rs index 237d73f..fc0755d 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -241,6 +241,10 @@ pub use async_attributes::{main, test}; #[cfg(feature = "std")] mod macros; +cfg_no_std! { + pub mod task; +} + cfg_std! { pub mod future; pub mod io; @@ -248,7 +252,6 @@ cfg_std! { pub mod prelude; pub mod stream; pub mod sync; - pub mod task; } cfg_default! { diff --git a/src/task/mod.rs b/src/task/mod.rs index f738fca..680e2a5 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -117,13 +117,16 @@ //! [`task_local!`]: ../macro.task_local.html //! [`with`]: struct.LocalKey.html#method.with -cfg_std! { +cfg_no_std! { #[doc(inline)] - pub use std::task::{Context, Poll, Waker}; - + pub use core::task::{Context, Poll, Waker}; pub use ready::ready; - pub use yield_now::yield_now; + mod ready; +} + +cfg_std! { + pub use yield_now::yield_now; mod yield_now; } From 1762de285b8bd96eed278c6c0bf8b13edc5d0b62 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 15:40:28 +0900 Subject: [PATCH 035/149] feat: Make the future module no_std --- src/future/future/mod.rs | 6 +++--- src/future/mod.rs | 21 +++++++++++++-------- src/lib.rs | 2 +- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/future/future/mod.rs b/src/future/future/mod.rs index d610096..24f3fb5 100644 --- a/src/future/future/mod.rs +++ b/src/future/future/mod.rs @@ -21,8 +21,8 @@ cfg_unstable_default! { } extension_trait! { - use std::pin::Pin; - use std::ops::{Deref, DerefMut}; + use core::pin::Pin; + use core::ops::{Deref, DerefMut}; use crate::task::{Context, Poll}; @@ -136,7 +136,7 @@ extension_trait! { [`Future`]: ../future/trait.Future.html "#] - pub trait FutureExt: std::future::Future { + pub trait FutureExt: core::future::Future { /// Returns a Future that delays execution for a specified time. /// /// # Examples diff --git a/src/future/mod.rs b/src/future/mod.rs index 9936276..3d325be 100644 --- a/src/future/mod.rs +++ b/src/future/mod.rs @@ -46,15 +46,20 @@ //! [`Future::race`]: trait.Future.html#method.race //! [`Future::try_race`]: trait.Future.html#method.try_race -pub use future::Future; -pub use pending::pending; -pub use poll_fn::poll_fn; -pub use ready::ready; +cfg_no_std! { + pub use future::Future; + pub(crate) mod future; +} + +cfg_std! { + pub use pending::pending; + pub use poll_fn::poll_fn; + pub use ready::ready; -pub(crate) mod future; -mod pending; -mod poll_fn; -mod ready; + mod pending; + mod poll_fn; + mod ready; +} cfg_default! { pub use timeout::{timeout, TimeoutError}; diff --git a/src/lib.rs b/src/lib.rs index fc0755d..6fe13e6 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -243,10 +243,10 @@ mod macros; cfg_no_std! { pub mod task; + pub mod future; } cfg_std! { - pub mod future; pub mod io; pub mod os; pub mod prelude; From 880b7ee987576f24a6dceb955a96365479ae8101 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 20:34:51 +0900 Subject: [PATCH 036/149] remove crate::prelude import --- src/stream/extend.rs | 2 +- src/stream/interval.rs | 4 ++-- src/stream/stream/chain.rs | 3 ++- src/stream/stream/cmp.rs | 4 ++-- src/stream/stream/eq.rs | 4 ++-- src/stream/stream/flat_map.rs | 2 +- src/stream/stream/ge.rs | 4 ++-- src/stream/stream/gt.rs | 4 ++-- src/stream/stream/le.rs | 4 ++-- src/stream/stream/lt.rs | 4 ++-- src/stream/stream/merge.rs | 3 ++- src/stream/stream/ne.rs | 4 ++-- src/stream/stream/partial_cmp.rs | 2 +- 13 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/stream/extend.rs b/src/stream/extend.rs index c48fe1e..fab8e6e 100644 --- a/src/stream/extend.rs +++ b/src/stream/extend.rs @@ -1,6 +1,6 @@ use std::pin::Pin; -use crate::prelude::*; +use crate::future::Future; use crate::stream::IntoStream; /// Extends a collection with the contents of a stream. diff --git a/src/stream/interval.rs b/src/stream/interval.rs index f8a6ef6..7a0c174 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -2,10 +2,10 @@ use std::pin::Pin; use std::task::{Context, Poll}; use std::time::{Duration, Instant}; +use crate::future::Future; +use crate::stream::Stream; use futures_timer::Delay; -use crate::prelude::*; - /// Creates a new stream that yields at a set interval. /// /// The stream first yields after `dur`, and continues to yield every diff --git a/src/stream/stream/chain.rs b/src/stream/stream/chain.rs index 909fc19..1d62f38 100644 --- a/src/stream/stream/chain.rs +++ b/src/stream/stream/chain.rs @@ -3,7 +3,8 @@ use std::pin::Pin; use pin_project_lite::pin_project; use super::fuse::Fuse; -use crate::prelude::*; +use crate::stream::stream::StreamExt; +use crate::stream::Stream; use crate::task::{Context, Poll}; pin_project! { diff --git a/src/stream/stream/cmp.rs b/src/stream/stream/cmp.rs index 2be0c1a..191219d 100644 --- a/src/stream/stream/cmp.rs +++ b/src/stream/stream/cmp.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::fuse::Fuse; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/eq.rs b/src/stream/stream/eq.rs index 58ccc90..54745af 100644 --- a/src/stream/stream/eq.rs +++ b/src/stream/stream/eq.rs @@ -1,10 +1,10 @@ -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::fuse::Fuse; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/flat_map.rs b/src/stream/stream/flat_map.rs index 9a7d921..cc0f665 100644 --- a/src/stream/stream/flat_map.rs +++ b/src/stream/stream/flat_map.rs @@ -2,8 +2,8 @@ use std::pin::Pin; use pin_project_lite::pin_project; -use crate::prelude::*; use crate::stream::stream::map::Map; +use crate::stream::stream::StreamExt; use crate::stream::{IntoStream, Stream}; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/ge.rs b/src/stream/stream/ge.rs index 67b20be..50ea21d 100644 --- a/src/stream/stream/ge.rs +++ b/src/stream/stream/ge.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::partial_cmp::PartialCmpFuture; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/gt.rs b/src/stream/stream/gt.rs index 1c12189..d42b8d4 100644 --- a/src/stream/stream/gt.rs +++ b/src/stream/stream/gt.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::partial_cmp::PartialCmpFuture; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/le.rs b/src/stream/stream/le.rs index 7b86161..9ee44cb 100644 --- a/src/stream/stream/le.rs +++ b/src/stream/stream/le.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::partial_cmp::PartialCmpFuture; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/lt.rs b/src/stream/stream/lt.rs index 100a003..4364ffe 100644 --- a/src/stream/stream/lt.rs +++ b/src/stream/stream/lt.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::partial_cmp::PartialCmpFuture; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/merge.rs b/src/stream/stream/merge.rs index 84ac432..ad697a8 100644 --- a/src/stream/stream/merge.rs +++ b/src/stream/stream/merge.rs @@ -3,8 +3,9 @@ use std::task::{Context, Poll}; use pin_project_lite::pin_project; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Fuse; +use crate::stream::Stream; use crate::utils; pin_project! { diff --git a/src/stream/stream/ne.rs b/src/stream/stream/ne.rs index ec11d1f..7cad3b7 100644 --- a/src/stream/stream/ne.rs +++ b/src/stream/stream/ne.rs @@ -1,10 +1,10 @@ -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use pin_project_lite::pin_project; use super::fuse::Fuse; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/partial_cmp.rs b/src/stream/stream/partial_cmp.rs index 85587c9..247413b 100644 --- a/src/stream/stream/partial_cmp.rs +++ b/src/stream/stream/partial_cmp.rs @@ -5,7 +5,7 @@ use std::pin::Pin; use pin_project_lite::pin_project; use super::fuse::Fuse; -use crate::prelude::*; +use crate::stream::stream::StreamExt; use crate::stream::Stream; use crate::task::{Context, Poll}; From d622ec5d357ce8797fecdd480479ca92560e6c41 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 21:01:47 +0900 Subject: [PATCH 037/149] feat: Make the stream module no_std --- Cargo.toml | 2 +- src/lib.rs | 2 +- src/stream/double_ended_stream/next_back.rs | 4 ++-- src/stream/double_ended_stream/nth_back.rs | 6 +++--- src/stream/double_ended_stream/rfind.rs | 6 +++--- src/stream/double_ended_stream/rfold.rs | 6 +++--- src/stream/double_ended_stream/try_rfold.rs | 2 +- src/stream/empty.rs | 4 ++-- src/stream/extend.rs | 2 +- src/stream/from_fn.rs | 2 +- src/stream/from_iter.rs | 2 +- src/stream/from_stream.rs | 4 ++-- src/stream/once.rs | 2 +- src/stream/pending.rs | 6 +++--- src/stream/product.rs | 4 ++-- src/stream/repeat.rs | 2 +- src/stream/repeat_with.rs | 2 +- src/stream/stream/all.rs | 6 +++--- src/stream/stream/any.rs | 6 +++--- src/stream/stream/chain.rs | 2 +- src/stream/stream/cloned.rs | 2 +- src/stream/stream/cmp.rs | 6 +++--- src/stream/stream/copied.rs | 2 +- src/stream/stream/count.rs | 4 ++-- src/stream/stream/cycle.rs | 4 ++-- src/stream/stream/delay.rs | 6 +++--- src/stream/stream/enumerate.rs | 2 +- src/stream/stream/eq.rs | 4 ++-- src/stream/stream/filter.rs | 2 +- src/stream/stream/filter_map.rs | 4 ++-- src/stream/stream/find.rs | 4 ++-- src/stream/stream/find_map.rs | 6 +++--- src/stream/stream/flat_map.rs | 2 +- src/stream/stream/flatten.rs | 4 ++-- src/stream/stream/fold.rs | 4 ++-- src/stream/stream/for_each.rs | 4 ++-- src/stream/stream/fuse.rs | 2 +- src/stream/stream/ge.rs | 6 +++--- src/stream/stream/gt.rs | 6 +++--- src/stream/stream/inspect.rs | 2 +- src/stream/stream/last.rs | 4 ++-- src/stream/stream/le.rs | 6 +++--- src/stream/stream/lt.rs | 6 +++--- src/stream/stream/map.rs | 2 +- src/stream/stream/max.rs | 8 ++++---- src/stream/stream/max_by.rs | 6 +++--- src/stream/stream/max_by_key.rs | 6 +++--- src/stream/stream/merge.rs | 4 ++-- src/stream/stream/min.rs | 8 ++++---- src/stream/stream/min_by.rs | 6 +++--- src/stream/stream/min_by_key.rs | 6 +++--- src/stream/stream/mod.rs | 8 ++++---- src/stream/stream/ne.rs | 4 ++-- src/stream/stream/next.rs | 4 ++-- src/stream/stream/nth.rs | 6 +++--- src/stream/stream/partial_cmp.rs | 6 +++--- src/stream/stream/partition.rs | 6 +++--- src/stream/stream/position.rs | 4 ++-- src/stream/stream/scan.rs | 2 +- src/stream/stream/skip.rs | 4 ++-- src/stream/stream/skip_while.rs | 2 +- src/stream/stream/step_by.rs | 2 +- src/stream/stream/take.rs | 2 +- src/stream/stream/take_while.rs | 2 +- src/stream/stream/timeout.rs | 2 +- src/stream/stream/try_fold.rs | 2 +- src/stream/stream/try_for_each.rs | 4 ++-- src/stream/stream/unzip.rs | 4 ++-- src/stream/stream/zip.rs | 4 ++-- src/stream/successors.rs | 4 ++-- src/stream/sum.rs | 4 ++-- 71 files changed, 143 insertions(+), 143 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 4492048..1206129 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -42,13 +42,13 @@ std = [ "futures-io", "memchr", "once_cell", - "pin-project-lite", "pin-utils", "slab", "no-std", ] no-std = [ "futures-core", + "pin-project-lite", ] [dependencies] diff --git a/src/lib.rs b/src/lib.rs index 6fe13e6..f35d6cb 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -244,13 +244,13 @@ mod macros; cfg_no_std! { pub mod task; pub mod future; + pub mod stream; } cfg_std! { pub mod io; pub mod os; pub mod prelude; - pub mod stream; pub mod sync; } diff --git a/src/stream/double_ended_stream/next_back.rs b/src/stream/double_ended_stream/next_back.rs index aa642d0..9fb52b7 100644 --- a/src/stream/double_ended_stream/next_back.rs +++ b/src/stream/double_ended_stream/next_back.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::future::Future; +use core::pin::Pin; +use core::future::Future; use crate::stream::DoubleEndedStream; use crate::task::{Context, Poll}; diff --git a/src/stream/double_ended_stream/nth_back.rs b/src/stream/double_ended_stream/nth_back.rs index e32a28f..2701e9b 100644 --- a/src/stream/double_ended_stream/nth_back.rs +++ b/src/stream/double_ended_stream/nth_back.rs @@ -1,6 +1,6 @@ -use std::future::Future; -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::future::Future; +use core::pin::Pin; +use core::task::{Context, Poll}; use crate::stream::DoubleEndedStream; diff --git a/src/stream/double_ended_stream/rfind.rs b/src/stream/double_ended_stream/rfind.rs index 9472693..3666551 100644 --- a/src/stream/double_ended_stream/rfind.rs +++ b/src/stream/double_ended_stream/rfind.rs @@ -1,6 +1,6 @@ -use std::task::{Context, Poll}; -use std::future::Future; -use std::pin::Pin; +use core::task::{Context, Poll}; +use core::future::Future; +use core::pin::Pin; use crate::stream::DoubleEndedStream; diff --git a/src/stream/double_ended_stream/rfold.rs b/src/stream/double_ended_stream/rfold.rs index 9002f8d..9bc1878 100644 --- a/src/stream/double_ended_stream/rfold.rs +++ b/src/stream/double_ended_stream/rfold.rs @@ -1,6 +1,6 @@ -use std::future::Future; -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::future::Future; +use core::pin::Pin; +use core::task::{Context, Poll}; use pin_project_lite::pin_project; diff --git a/src/stream/double_ended_stream/try_rfold.rs b/src/stream/double_ended_stream/try_rfold.rs index 9e6295a..d67b6ec 100644 --- a/src/stream/double_ended_stream/try_rfold.rs +++ b/src/stream/double_ended_stream/try_rfold.rs @@ -1,5 +1,5 @@ use crate::future::Future; -use std::pin::Pin; +use core::pin::Pin; use crate::task::{Context, Poll}; use pin_project_lite::pin_project; diff --git a/src/stream/empty.rs b/src/stream/empty.rs index 4909070..a11337a 100644 --- a/src/stream/empty.rs +++ b/src/stream/empty.rs @@ -1,5 +1,5 @@ -use std::marker::PhantomData; -use std::pin::Pin; +use core::marker::PhantomData; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/extend.rs b/src/stream/extend.rs index fab8e6e..dbf7911 100644 --- a/src/stream/extend.rs +++ b/src/stream/extend.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use crate::future::Future; use crate::stream::IntoStream; diff --git a/src/stream/from_fn.rs b/src/stream/from_fn.rs index 8067176..d373645 100644 --- a/src/stream/from_fn.rs +++ b/src/stream/from_fn.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/from_iter.rs b/src/stream/from_iter.rs index 705d150..ea2ed8e 100644 --- a/src/stream/from_iter.rs +++ b/src/stream/from_iter.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/from_stream.rs b/src/stream/from_stream.rs index 67b9b3d..12d89e8 100644 --- a/src/stream/from_stream.rs +++ b/src/stream/from_stream.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use crate::stream::IntoStream; diff --git a/src/stream/once.rs b/src/stream/once.rs index 939722d..b86f181 100644 --- a/src/stream/once.rs +++ b/src/stream/once.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/pending.rs b/src/stream/pending.rs index 922a540..edb6be4 100644 --- a/src/stream/pending.rs +++ b/src/stream/pending.rs @@ -1,6 +1,6 @@ -use std::marker::PhantomData; -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::marker::PhantomData; +use core::pin::Pin; +use core::task::{Context, Poll}; use crate::stream::{DoubleEndedStream, ExactSizeStream, FusedStream, Stream}; diff --git a/src/stream/product.rs b/src/stream/product.rs index 2f5bf4c..15497e8 100644 --- a/src/stream/product.rs +++ b/src/stream/product.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::future::Future; +use core::pin::Pin; +use core::future::Future; use crate::stream::Stream; diff --git a/src/stream/repeat.rs b/src/stream/repeat.rs index f3dfdbd..e73a2f8 100644 --- a/src/stream/repeat.rs +++ b/src/stream/repeat.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/repeat_with.rs b/src/stream/repeat_with.rs index e183a77..39984a3 100644 --- a/src/stream/repeat_with.rs +++ b/src/stream/repeat_with.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/all.rs b/src/stream/stream/all.rs index d2ac5ca..06f4d7f 100644 --- a/src/stream/stream/all.rs +++ b/src/stream/stream/all.rs @@ -1,6 +1,6 @@ -use std::marker::PhantomData; -use std::pin::Pin; -use std::future::Future; +use core::marker::PhantomData; +use core::pin::Pin; +use core::future::Future; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/any.rs b/src/stream/stream/any.rs index 34168e6..15154c5 100644 --- a/src/stream/stream/any.rs +++ b/src/stream/stream/any.rs @@ -1,6 +1,6 @@ -use std::marker::PhantomData; -use std::pin::Pin; -use std::future::Future; +use core::marker::PhantomData; +use core::pin::Pin; +use core::future::Future; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/chain.rs b/src/stream/stream/chain.rs index 1d62f38..c034a53 100644 --- a/src/stream/stream/chain.rs +++ b/src/stream/stream/chain.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/cloned.rs b/src/stream/stream/cloned.rs index 4c77c5c..eac992d 100644 --- a/src/stream/stream/cloned.rs +++ b/src/stream/stream/cloned.rs @@ -1,7 +1,7 @@ use crate::stream::Stream; use crate::task::{Context, Poll}; use pin_project_lite::pin_project; -use std::pin::Pin; +use core::pin::Pin; pin_project! { /// A stream that clones the elements of an underlying stream. diff --git a/src/stream/stream/cmp.rs b/src/stream/stream/cmp.rs index 191219d..9d2b0ec 100644 --- a/src/stream/stream/cmp.rs +++ b/src/stream/stream/cmp.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/copied.rs b/src/stream/stream/copied.rs index 651c31b..19296f5 100644 --- a/src/stream/stream/copied.rs +++ b/src/stream/stream/copied.rs @@ -1,7 +1,7 @@ use crate::stream::Stream; use crate::task::{Context, Poll}; use pin_project_lite::pin_project; -use std::pin::Pin; +use core::pin::Pin; pin_project! { /// A stream that copies the elements of an underlying stream. diff --git a/src/stream/stream/count.rs b/src/stream/stream/count.rs index 1ca8aef..63e0449 100644 --- a/src/stream/stream/count.rs +++ b/src/stream/stream/count.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/cycle.rs b/src/stream/stream/cycle.rs index 5f8eaa2..ef46d1a 100644 --- a/src/stream/stream/cycle.rs +++ b/src/stream/stream/cycle.rs @@ -1,5 +1,5 @@ -use std::mem::ManuallyDrop; -use std::pin::Pin; +use core::mem::ManuallyDrop; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/delay.rs b/src/stream/stream/delay.rs index 5768792..ff4c937 100644 --- a/src/stream/stream/delay.rs +++ b/src/stream/stream/delay.rs @@ -1,6 +1,6 @@ -use std::future::Future; -use std::pin::Pin; -use std::time::Duration; +use core::future::Future; +use core::pin::Pin; +use core::time::Duration; use pin_project_lite::pin_project; diff --git a/src/stream/stream/enumerate.rs b/src/stream/stream/enumerate.rs index c4a37d6..093fefb 100644 --- a/src/stream/stream/enumerate.rs +++ b/src/stream/stream/enumerate.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/eq.rs b/src/stream/stream/eq.rs index 54745af..3d8307b 100644 --- a/src/stream/stream/eq.rs +++ b/src/stream/stream/eq.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/filter.rs b/src/stream/stream/filter.rs index 00344b0..2dc7dd4 100644 --- a/src/stream/stream/filter.rs +++ b/src/stream/stream/filter.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/filter_map.rs b/src/stream/stream/filter_map.rs index 3cd1e47..e43e8f0 100644 --- a/src/stream/stream/filter_map.rs +++ b/src/stream/stream/filter_map.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::pin::Pin; +use core::task::{Context, Poll}; use pin_project_lite::pin_project; diff --git a/src/stream/stream/find.rs b/src/stream/stream/find.rs index 4a0749b..8652ac0 100644 --- a/src/stream/stream/find.rs +++ b/src/stream/stream/find.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/find_map.rs b/src/stream/stream/find_map.rs index c794943..f7e3c1e 100644 --- a/src/stream/stream/find_map.rs +++ b/src/stream/stream/find_map.rs @@ -1,6 +1,6 @@ -use std::future::Future; -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::future::Future; +use core::pin::Pin; +use core::task::{Context, Poll}; use crate::stream::Stream; diff --git a/src/stream/stream/flat_map.rs b/src/stream/stream/flat_map.rs index cc0f665..e07893a 100644 --- a/src/stream/stream/flat_map.rs +++ b/src/stream/stream/flat_map.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/flatten.rs b/src/stream/stream/flatten.rs index 1d6fcae..f2e275c 100644 --- a/src/stream/stream/flatten.rs +++ b/src/stream/stream/flatten.rs @@ -1,5 +1,5 @@ -use std::fmt; -use std::pin::Pin; +use core::fmt; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/fold.rs b/src/stream/stream/fold.rs index a346eb6..3938a37 100644 --- a/src/stream/stream/fold.rs +++ b/src/stream/stream/fold.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/for_each.rs b/src/stream/stream/for_each.rs index dce5cda..dbada10 100644 --- a/src/stream/stream/for_each.rs +++ b/src/stream/stream/for_each.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::future::Future; +use core::pin::Pin; +use core::future::Future; use pin_project_lite::pin_project; diff --git a/src/stream/stream/fuse.rs b/src/stream/stream/fuse.rs index c7449c2..f3a0596 100644 --- a/src/stream/stream/fuse.rs +++ b/src/stream/stream/fuse.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/ge.rs b/src/stream/stream/ge.rs index 50ea21d..1e1b70d 100644 --- a/src/stream/stream/ge.rs +++ b/src/stream/stream/ge.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/gt.rs b/src/stream/stream/gt.rs index d42b8d4..d58e7e9 100644 --- a/src/stream/stream/gt.rs +++ b/src/stream/stream/gt.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/inspect.rs b/src/stream/stream/inspect.rs index bb39662..481810d 100644 --- a/src/stream/stream/inspect.rs +++ b/src/stream/stream/inspect.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/last.rs b/src/stream/stream/last.rs index d037efc..ebf1a48 100644 --- a/src/stream/stream/last.rs +++ b/src/stream/stream/last.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/le.rs b/src/stream/stream/le.rs index 9ee44cb..169f9ec 100644 --- a/src/stream/stream/le.rs +++ b/src/stream/stream/le.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/lt.rs b/src/stream/stream/lt.rs index 4364ffe..1851b8e 100644 --- a/src/stream/stream/lt.rs +++ b/src/stream/stream/lt.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/map.rs b/src/stream/stream/map.rs index 8e074a7..0eab3ce 100644 --- a/src/stream/stream/map.rs +++ b/src/stream/stream/map.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/max.rs b/src/stream/stream/max.rs index d8ff119..8a4d594 100644 --- a/src/stream/stream/max.rs +++ b/src/stream/stream/max.rs @@ -1,7 +1,7 @@ -use std::cmp::{Ord, Ordering}; -use std::marker::PhantomData; -use std::pin::Pin; -use std::future::Future; +use core::cmp::{Ord, Ordering}; +use core::marker::PhantomData; +use core::pin::Pin; +use core::future::Future; use pin_project_lite::pin_project; diff --git a/src/stream/stream/max_by.rs b/src/stream/stream/max_by.rs index 36b876b..8f98645 100644 --- a/src/stream/stream/max_by.rs +++ b/src/stream/stream/max_by.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::pin::Pin; -use std::future::Future; +use core::cmp::Ordering; +use core::pin::Pin; +use core::future::Future; use pin_project_lite::pin_project; diff --git a/src/stream/stream/max_by_key.rs b/src/stream/stream/max_by_key.rs index e421f94..8fa91ab 100644 --- a/src/stream/stream/max_by_key.rs +++ b/src/stream/stream/max_by_key.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/merge.rs b/src/stream/stream/merge.rs index ad697a8..2320972 100644 --- a/src/stream/stream/merge.rs +++ b/src/stream/stream/merge.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::pin::Pin; +use core::task::{Context, Poll}; use pin_project_lite::pin_project; diff --git a/src/stream/stream/min.rs b/src/stream/stream/min.rs index 4ce52be..4fe2a67 100644 --- a/src/stream/stream/min.rs +++ b/src/stream/stream/min.rs @@ -1,7 +1,7 @@ -use std::cmp::{Ord, Ordering}; -use std::marker::PhantomData; -use std::pin::Pin; -use std::future::Future; +use core::cmp::{Ord, Ordering}; +use core::marker::PhantomData; +use core::pin::Pin; +use core::future::Future; use pin_project_lite::pin_project; diff --git a/src/stream/stream/min_by.rs b/src/stream/stream/min_by.rs index e35719e..fe1d40e 100644 --- a/src/stream/stream/min_by.rs +++ b/src/stream/stream/min_by.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::pin::Pin; -use std::future::Future; +use core::cmp::Ordering; +use core::pin::Pin; +use core::future::Future; use pin_project_lite::pin_project; diff --git a/src/stream/stream/min_by_key.rs b/src/stream/stream/min_by_key.rs index 07c3642..549b798 100644 --- a/src/stream/stream/min_by_key.rs +++ b/src/stream/stream/min_by_key.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index 6c84ac2..d0cc718 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -110,12 +110,12 @@ pub use take::Take; pub use take_while::TakeWhile; pub use zip::Zip; -use std::cmp::Ordering; +use core::cmp::Ordering; cfg_unstable! { - use std::future::Future; - use std::pin::Pin; - use std::time::Duration; + use core::future::Future; + use core::pin::Pin; + use core::time::Duration; use crate::stream::into_stream::IntoStream; use crate::stream::{FromStream, Product, Sum}; diff --git a/src/stream/stream/ne.rs b/src/stream/stream/ne.rs index 7cad3b7..c51ab31 100644 --- a/src/stream/stream/ne.rs +++ b/src/stream/stream/ne.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/next.rs b/src/stream/stream/next.rs index 23abb0b..7bd2083 100644 --- a/src/stream/stream/next.rs +++ b/src/stream/stream/next.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::future::Future; +use core::pin::Pin; +use core::future::Future; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/nth.rs b/src/stream/stream/nth.rs index 267bd40..8cdabb6 100644 --- a/src/stream/stream/nth.rs +++ b/src/stream/stream/nth.rs @@ -1,6 +1,6 @@ -use std::pin::Pin; -use std::task::{Context, Poll}; -use std::future::Future; +use core::pin::Pin; +use core::task::{Context, Poll}; +use core::future::Future; use crate::stream::Stream; diff --git a/src/stream/stream/partial_cmp.rs b/src/stream/stream/partial_cmp.rs index 247413b..928a03b 100644 --- a/src/stream/stream/partial_cmp.rs +++ b/src/stream/stream/partial_cmp.rs @@ -1,6 +1,6 @@ -use std::cmp::Ordering; -use std::future::Future; -use std::pin::Pin; +use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/partition.rs b/src/stream/stream/partition.rs index aaf58ac..69268ec 100644 --- a/src/stream/stream/partition.rs +++ b/src/stream/stream/partition.rs @@ -1,7 +1,7 @@ use pin_project_lite::pin_project; -use std::default::Default; -use std::future::Future; -use std::pin::Pin; +use core::default::Default; +use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/position.rs b/src/stream/stream/position.rs index df60eaa..2811b6d 100644 --- a/src/stream/stream/position.rs +++ b/src/stream/stream/position.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/scan.rs b/src/stream/stream/scan.rs index 385edf8..d72b0dc 100644 --- a/src/stream/stream/scan.rs +++ b/src/stream/stream/scan.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/skip.rs b/src/stream/stream/skip.rs index bcff50d..52b137d 100644 --- a/src/stream/stream/skip.rs +++ b/src/stream/stream/skip.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::pin::Pin; +use core::task::{Context, Poll}; use pin_project_lite::pin_project; diff --git a/src/stream/stream/skip_while.rs b/src/stream/stream/skip_while.rs index 2334713..d139de4 100644 --- a/src/stream/stream/skip_while.rs +++ b/src/stream/stream/skip_while.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/step_by.rs b/src/stream/stream/step_by.rs index 2149cda..3dd3d62 100644 --- a/src/stream/stream/step_by.rs +++ b/src/stream/stream/step_by.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/take.rs b/src/stream/stream/take.rs index 8c85227..ffc3e99 100644 --- a/src/stream/stream/take.rs +++ b/src/stream/stream/take.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/take_while.rs b/src/stream/stream/take_while.rs index 2ba8490..60eb8c5 100644 --- a/src/stream/stream/take_while.rs +++ b/src/stream/stream/take_while.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/timeout.rs b/src/stream/stream/timeout.rs index f580360..ce658c8 100644 --- a/src/stream/stream/timeout.rs +++ b/src/stream/stream/timeout.rs @@ -1,8 +1,8 @@ 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; diff --git a/src/stream/stream/try_fold.rs b/src/stream/stream/try_fold.rs index 3b92d95..73312ff 100644 --- a/src/stream/stream/try_fold.rs +++ b/src/stream/stream/try_fold.rs @@ -1,4 +1,4 @@ -use std::pin::Pin; +use core::pin::Pin; use crate::future::Future; use crate::stream::Stream; diff --git a/src/stream/stream/try_for_each.rs b/src/stream/stream/try_for_each.rs index 86f1674..917bfd1 100644 --- a/src/stream/stream/try_for_each.rs +++ b/src/stream/stream/try_for_each.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/stream/unzip.rs b/src/stream/stream/unzip.rs index cb57578..7771509 100644 --- a/src/stream/stream/unzip.rs +++ b/src/stream/stream/unzip.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/stream/zip.rs b/src/stream/stream/zip.rs index 597691b..83d613c 100644 --- a/src/stream/stream/zip.rs +++ b/src/stream/stream/zip.rs @@ -1,5 +1,5 @@ -use std::fmt; -use std::pin::Pin; +use core::fmt; +use core::pin::Pin; use pin_project_lite::pin_project; diff --git a/src/stream/successors.rs b/src/stream/successors.rs index 4421564..a9ce40f 100644 --- a/src/stream/successors.rs +++ b/src/stream/successors.rs @@ -1,5 +1,5 @@ -use std::mem; -use std::pin::Pin; +use core::mem; +use core::pin::Pin; use crate::stream::Stream; use crate::task::{Context, Poll}; diff --git a/src/stream/sum.rs b/src/stream/sum.rs index 9607baf..3b3144e 100644 --- a/src/stream/sum.rs +++ b/src/stream/sum.rs @@ -1,5 +1,5 @@ -use std::future::Future; -use std::pin::Pin; +use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; From 22d929d481b8748111a51907be1ca716a95c183c Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 16 Jan 2020 21:13:23 +0900 Subject: [PATCH 038/149] fix import Future --- src/stream/extend.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/extend.rs b/src/stream/extend.rs index dbf7911..702cbca 100644 --- a/src/stream/extend.rs +++ b/src/stream/extend.rs @@ -1,6 +1,6 @@ use core::pin::Pin; +use core::future::Future; -use crate::future::Future; use crate::stream::IntoStream; /// Extends a collection with the contents of a stream. From 7efe7caf66012ae6eb92e667f372f7a6ad6bd282 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 28 Jan 2020 15:58:06 +0900 Subject: [PATCH 039/149] fix: Change feature name no-std to alloc --- .github/workflows/ci.yml | 2 +- Cargo.toml | 4 ++-- src/future/mod.rs | 2 +- src/lib.rs | 2 +- src/task/mod.rs | 2 +- src/utils.rs | 8 ++++---- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 0b0e84f..597c2b6 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -58,7 +58,7 @@ jobs: uses: actions-rs/cargo@v1 with: command: check - args: --no-default-features --features no-std + args: --no-default-features --features alloc - name: check attributes uses: actions-rs/cargo@v1 diff --git a/Cargo.toml b/Cargo.toml index 1206129..da6187a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -44,9 +44,9 @@ std = [ "once_cell", "pin-utils", "slab", - "no-std", + "alloc", ] -no-std = [ +alloc = [ "futures-core", "pin-project-lite", ] diff --git a/src/future/mod.rs b/src/future/mod.rs index 3d325be..9b75533 100644 --- a/src/future/mod.rs +++ b/src/future/mod.rs @@ -46,7 +46,7 @@ //! [`Future::race`]: trait.Future.html#method.race //! [`Future::try_race`]: trait.Future.html#method.try_race -cfg_no_std! { +cfg_alloc! { pub use future::Future; pub(crate) mod future; } diff --git a/src/lib.rs b/src/lib.rs index f35d6cb..b5a3fff 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -241,7 +241,7 @@ pub use async_attributes::{main, test}; #[cfg(feature = "std")] mod macros; -cfg_no_std! { +cfg_alloc! { pub mod task; pub mod future; pub mod stream; diff --git a/src/task/mod.rs b/src/task/mod.rs index 680e2a5..13fe903 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -117,7 +117,7 @@ //! [`task_local!`]: ../macro.task_local.html //! [`with`]: struct.LocalKey.html#method.with -cfg_no_std! { +cfg_alloc! { #[doc(inline)] pub use core::task::{Context, Poll, Waker}; pub use ready::ready; diff --git a/src/utils.rs b/src/utils.rs index ccf5e84..e257d2f 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -1,7 +1,7 @@ -#[cfg(feature = "no-std")] +#[cfg(feature = "alloc")] extern crate alloc; -#[cfg(feature = "no-std")] +#[cfg(feature = "alloc")] use alloc::string::String; /// Calls a function and aborts if it panics. @@ -173,10 +173,10 @@ macro_rules! cfg_std { /// Declares no-std items. #[allow(unused_macros)] #[doc(hidden)] -macro_rules! cfg_no_std { +macro_rules! cfg_alloc { ($($item:item)*) => { $( - #[cfg(feature = "no-std")] + #[cfg(feature = "alloc")] $item )* } From 1d875836a2302681a395ee44512a518f0222da4a Mon Sep 17 00:00:00 2001 From: Stjepan Glavina Date: Tue, 28 Jan 2020 18:14:16 +0100 Subject: [PATCH 040/149] Implement Clone for TcpStream (#689) * Implement Clone for TcpStream * Update examples * Remove accidentally added examples --- examples/tcp-echo.rs | 5 +++-- examples/tcp-ipv4-and-6-echo.rs | 5 +++-- src/net/tcp/listener.rs | 7 +++---- src/net/tcp/stream.rs | 26 ++++++++++++++++---------- tests/tcp.rs | 22 ++++++++++++++++++++++ 5 files changed, 47 insertions(+), 18 deletions(-) diff --git a/examples/tcp-echo.rs b/examples/tcp-echo.rs index 7c50be0..c04f077 100644 --- a/examples/tcp-echo.rs +++ b/examples/tcp-echo.rs @@ -14,8 +14,9 @@ use async_std::task; async fn process(stream: TcpStream) -> io::Result<()> { println!("Accepted from: {}", stream.peer_addr()?); - let (reader, writer) = &mut (&stream, &stream); - io::copy(reader, writer).await?; + let mut reader = stream.clone(); + let mut writer = stream; + io::copy(&mut reader, &mut writer).await?; Ok(()) } diff --git a/examples/tcp-ipv4-and-6-echo.rs b/examples/tcp-ipv4-and-6-echo.rs index aef5e15..a00e1f3 100644 --- a/examples/tcp-ipv4-and-6-echo.rs +++ b/examples/tcp-ipv4-and-6-echo.rs @@ -15,8 +15,9 @@ use async_std::task; async fn process(stream: TcpStream) -> io::Result<()> { println!("Accepted from: {}", stream.peer_addr()?); - let (reader, writer) = &mut (&stream, &stream); - io::copy(reader, writer).await?; + let mut reader = stream.clone(); + let mut writer = stream; + io::copy(&mut reader, &mut writer).await?; Ok(()) } diff --git a/src/net/tcp/listener.rs b/src/net/tcp/listener.rs index fe06a96..1d7e91a 100644 --- a/src/net/tcp/listener.rs +++ b/src/net/tcp/listener.rs @@ -1,6 +1,7 @@ use std::future::Future; use std::net::SocketAddr; use std::pin::Pin; +use std::sync::Arc; use crate::future; use crate::io; @@ -75,9 +76,7 @@ impl TcpListener { /// [`local_addr`]: #method.local_addr 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::TcpListener::bind(&addr) { @@ -121,7 +120,7 @@ impl TcpListener { let mio_stream = mio::net::TcpStream::from_stream(io)?; let stream = TcpStream { - watcher: Watcher::new(mio_stream), + watcher: Arc::new(Watcher::new(mio_stream)), }; Ok((stream, addr)) } diff --git a/src/net/tcp/stream.rs b/src/net/tcp/stream.rs index 537bd4c..c9cdf5e 100644 --- a/src/net/tcp/stream.rs +++ b/src/net/tcp/stream.rs @@ -1,6 +1,7 @@ use std::io::{IoSlice, IoSliceMut, Read as _, Write as _}; use std::net::SocketAddr; use std::pin::Pin; +use std::sync::Arc; use crate::future; use crate::io::{self, Read, Write}; @@ -44,9 +45,9 @@ use crate::task::{Context, Poll}; /// # /// # Ok(()) }) } /// ``` -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct TcpStream { - pub(super) watcher: Watcher, + pub(super) watcher: Arc>, } impl TcpStream { @@ -71,9 +72,7 @@ impl TcpStream { /// ``` pub async fn connect(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 { // mio's TcpStream::connect is non-blocking and may just be in progress @@ -84,16 +83,20 @@ impl TcpStream { Ok(s) => Watcher::new(s), Err(e) => { last_err = Some(e); - continue + continue; } }; future::poll_fn(|cx| watcher.poll_write_ready(cx)).await; match watcher.get_ref().take_error() { - Ok(None) => return Ok(TcpStream { watcher }), + Ok(None) => { + return Ok(TcpStream { + watcher: Arc::new(watcher), + }); + } Ok(Some(e)) => last_err = Some(e), - Err(e) => last_err = Some(e) + Err(e) => last_err = Some(e), } } @@ -369,7 +372,7 @@ impl From for TcpStream { fn from(stream: std::net::TcpStream) -> TcpStream { let mio_stream = mio::net::TcpStream::from_stream(stream).unwrap(); TcpStream { - watcher: Watcher::new(mio_stream), + watcher: Arc::new(Watcher::new(mio_stream)), } } } @@ -391,7 +394,10 @@ cfg_unix! { impl IntoRawFd for TcpStream { fn into_raw_fd(self) -> RawFd { - self.watcher.into_inner().into_raw_fd() + // 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_fd() } } } diff --git a/tests/tcp.rs b/tests/tcp.rs index 00fa3a0..d92cff0 100644 --- a/tests/tcp.rs +++ b/tests/tcp.rs @@ -94,3 +94,25 @@ fn smoke_async_stream_to_std_listener() -> io::Result<()> { Ok(()) } + +#[test] +fn cloned_streams() -> io::Result<()> { + task::block_on(async { + let listener = TcpListener::bind("127.0.0.1:0").await?; + let addr = listener.local_addr()?; + + let mut stream = TcpStream::connect(&addr).await?; + let mut cloned_stream = stream.clone(); + let mut incoming = listener.incoming(); + let mut write_stream = incoming.next().await.unwrap()?; + write_stream.write_all(b"Each your doing").await?; + + let mut buf = [0; 15]; + stream.read_exact(&mut buf[..8]).await?; + cloned_stream.read_exact(&mut buf[8..]).await?; + + assert_eq!(&buf[..15], b"Each your doing"); + + Ok(()) + }) +} From ef985bc72ef770b338480769cdf5ce42edcbc7b8 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 1 Feb 2020 09:45:41 +0900 Subject: [PATCH 041/149] ci: fix no_std ci --- .github/workflows/ci.yml | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 597c2b6..e99f508 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -54,12 +54,6 @@ jobs: command: check args: --no-default-features --features std - - name: check no_std - uses: actions-rs/cargo@v1 - with: - command: check - args: --no-default-features --features alloc - - name: check attributes uses: actions-rs/cargo@v1 with: @@ -78,6 +72,22 @@ jobs: command: test args: --doc --features "unstable attributes" + build__with_no_std: + name: Build with no-std + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@master + + - name: setup + run: rustup target add thumbv7m-none-eabi + + - name: check no_std + uses: actions-rs/cargo@v1 + with: + command: check + args: --no-default-features --features alloc --target thumbv7m-none-eabi -v + check_fmt_and_docs: name: Checking fmt and docs runs-on: ubuntu-latest From f789f9d4f68af64eaee47c0db931de5632a294f3 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 1 Feb 2020 09:47:33 +0900 Subject: [PATCH 042/149] Select future-core featue according to feature --- Cargo.toml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index da6187a..6ae4d82 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -38,16 +38,17 @@ docs = ["attributes", "unstable", "default"] unstable = ["std", "broadcaster", "futures-timer"] attributes = ["async-attributes"] std = [ + "alloc", "crossbeam-utils", + "futures-core/std", "futures-io", "memchr", "once_cell", "pin-utils", "slab", - "alloc", ] alloc = [ - "futures-core", + "futures-core/alloc", "pin-project-lite", ] @@ -58,7 +59,7 @@ 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 } +futures-core = { version = "0.3.1", optional = true, default-features = false } futures-io = { version = "0.3.1", optional = true } futures-timer = { version = "2.0.2", optional = true } kv-log-macro = { version = "1.0.4", optional = true } From 0d90cb07b9afa8d741100483020be79b4acd6409 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 1 Feb 2020 09:49:54 +0900 Subject: [PATCH 043/149] fix: Move `extern crate alloc` to lib.rs --- src/lib.rs | 2 ++ src/utils.rs | 4 ---- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index b5a3fff..05073a2 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -230,6 +230,8 @@ #![doc(html_logo_url = "https://async.rs/images/logo--hero.svg")] #![recursion_limit = "2048"] +extern crate alloc; + #[macro_use] mod utils; diff --git a/src/utils.rs b/src/utils.rs index e257d2f..f18b74d 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -1,7 +1,3 @@ -#[cfg(feature = "alloc")] -extern crate alloc; - -#[cfg(feature = "alloc")] use alloc::string::String; /// Calls a function and aborts if it panics. From 3e24e0ba4ed99c185013a26eb0943cf00037429e Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 1 Feb 2020 16:43:12 +0900 Subject: [PATCH 044/149] ci: fix no-std check --- .github/workflows/ci.yml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e99f508..bcda990 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -80,13 +80,15 @@ jobs: - uses: actions/checkout@master - name: setup - run: rustup target add thumbv7m-none-eabi + run: | + rustup default nightly + rustup target add thumbv7m-none-eabi - name: check no_std uses: actions-rs/cargo@v1 with: command: check - args: --no-default-features --features alloc --target thumbv7m-none-eabi -v + args: --no-default-features --features alloc --target thumbv7m-none-eabi -Z avoid-dev-deps check_fmt_and_docs: name: Checking fmt and docs From 39f2c6da784fdbf4c945f9541f77a8df96a90b6c Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 3 Feb 2020 16:45:00 +0100 Subject: [PATCH 045/149] V1.5.0 (#694) * Update CHANGELOG.md * v1.5.0 * Update CHANGELOG.md --- CHANGELOG.md | 43 ++++++++++++++++++++++++++++++++++++++++++- Cargo.toml | 2 +- 2 files changed, 43 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 831c45c..e464ed7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,46 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +# [1.5.0] - 2020-02-03 + +[API Documentation](https://docs.rs/async-std/1.5.0/async-std) + +This patch includes various quality of life improvements to async-std. +Including improved performance, stability, and the addition of various +`Clone` impls that replace the use of `Arc` in many cases. + +## Added + +- Added links to various ecosystem projects from the README ([#660](https://github.com/async-rs/async-std/pull/660)) +- Added an example on `FromStream` for `Result` ([#643](https://github.com/async-rs/async-std/pull/643)) +- Added `stream::pending` as "unstable" ([#615](https://github.com/async-rs/async-std/pull/615)) +- Added an example of `stream::timeout` to document the error flow ([#675](https://github.com/async-rs/async-std/pull/675)) +- Implement `Clone` for `DirEntry` ([#682](https://github.com/async-rs/async-std/pull/682)) +- Implement `Clone` for `TcpStream` ([#689](https://github.com/async-rs/async-std/pull/689)) + +## Changed + +- Removed internal comment on `stream::Interval` ([#645](https://github.com/async-rs/async-std/pull/645)) +- The "unstable" feature can now be used without requiring the "default" feature ([#647](https://github.com/async-rs/async-std/pull/647)) +- Removed unnecessary trait bound on `stream::FlatMap` ([#651](https://github.com/async-rs/async-std/pull/651)) +- Updated the "broadcaster" dependency used by "unstable" to `1.0.0` ([#681](https://github.com/async-rs/async-std/pull/681)) +- Updated `async-task` to 1.2.1 ([#676](https://github.com/async-rs/async-std/pull/676)) +- `task::block_on` now parks after a single poll, improving performance in many cases ([#684](https://github.com/async-rs/async-std/pull/684)) +- Improved reading flow of the "client" part of the async-std tutorial ([#550](https://github.com/async-rs/async-std/pull/550)) +- Use `take_while` instead of `scan` in `impl` of `Product`, `Sum` and `FromStream` ([#667](https://github.com/async-rs/async-std/pull/667)) +- `TcpStream::connect` no longer uses a thread from the threadpool, improving performance ([#687](https://github.com/async-rs/async-std/pull/687)) + +## Fixed + +- Fixed crate documentation typo ([#655](https://github.com/async-rs/async-std/pull/655)) +- Fixed documentation for `UdpSocket::recv` ([#648](https://github.com/async-rs/async-std/pull/648)) +- Fixed documentation for `UdpSocket::send` ([#671](https://github.com/async-rs/async-std/pull/671)) +- Fixed typo in stream documentation ([#650](https://github.com/async-rs/async-std/pull/650)) +- Fixed typo on `sync::JoinHandle` documentation ([#659](https://github.com/async-rs/async-std/pull/659)) +- Removed use of `std::error::Error::description` which failed CI ([#661](https://github.com/async-rs/async-std/pull/662)) +- Removed the use of rustfmt's unstable `format_code_in_doc_comments` option which failed CI ([#685](https://github.com/async-rs/async-std/pull/685)) +- Fixed a code typo in the `task::sleep` example ([#688](https://github.com/async-rs/async-std/pull/688)) + # [1.4.0] - 2019-12-20 [API Documentation](https://docs.rs/async-std/1.4.0/async-std) @@ -637,7 +677,8 @@ task::blocking(async { - Initial beta release -[Unreleased]: https://github.com/async-rs/async-std/compare/v1.3.0...HEAD +[Unreleased]: https://github.com/async-rs/async-std/compare/v1.5.0...HEAD +[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 [1.2.0]: https://github.com/async-rs/async-std/compare/v1.1.0...v1.2.0 diff --git a/Cargo.toml b/Cargo.toml index b1155bb..9948e8e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "async-std" -version = "1.4.0" +version = "1.5.0" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", From d026c44ea344e14a64320df80064950843c03a30 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Tue, 4 Feb 2020 11:07:50 +0100 Subject: [PATCH 046/149] Document the core feature Follow-up to https://github.com/async-rs/async-std/pull/680 --- src/lib.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/lib.rs b/src/lib.rs index 05073a2..7eb3050 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -220,6 +220,16 @@ //! default-features = false //! features = ["std"] //! ``` +//! +//! And to use async-std on `no_std` targets that only support `alloc` only +//! enable the `core` Cargo feature: +//! +//! ```toml +//! [dependencies.async-std] +//! version = "1.0.0" +//! default-features = false +//! features = ["core"] +//! ``` #![cfg_attr(not(feature = "std"), no_std)] #![cfg_attr(feature = "docs", feature(doc_cfg))] From aae835cc146c3eac301647ee170cc53dbeb913da Mon Sep 17 00:00:00 2001 From: Katharina Fey Date: Wed, 12 Feb 2020 01:38:20 +0100 Subject: [PATCH 047/149] channel/recv: improving function docs and code example At the moment it's not clear when and why recv returns Option, instead of just T. This changed comment makes it clear that None will only be returned once no data will ever be sent again (i.e. after all senders are gone). --- src/sync/channel.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index 2647f65..b42326b 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -346,8 +346,9 @@ pub struct Receiver { impl Receiver { /// Receives a message from the channel. /// - /// If the channel is empty and still has senders, this method will wait until a message is - /// sent into the channel or until all senders get dropped. + /// If the channel is emtpy and still has senders, this method + /// will wait until a message is sent into it. Once all senders + /// have been dropped it will return `None`. /// /// # Examples /// @@ -362,10 +363,13 @@ impl Receiver { /// task::spawn(async move { /// s.send(1).await; /// s.send(2).await; + /// // Then we drop the sender /// }); /// /// assert_eq!(r.recv().await, Some(1)); /// assert_eq!(r.recv().await, Some(2)); + /// + /// // recv() returns `None` /// assert_eq!(r.recv().await, None); /// # /// # }) From 3719484ebae68e0ea1b917dfdc49234193974f89 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 17 Feb 2020 13:36:23 +0100 Subject: [PATCH 048/149] Update src/lib.rs Co-Authored-By: nasa --- src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index 7eb3050..0e7844b 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -228,7 +228,7 @@ //! [dependencies.async-std] //! version = "1.0.0" //! default-features = false -//! features = ["core"] +//! features = ["alloc"] //! ``` #![cfg_attr(not(feature = "std"), no_std)] From 283a54a1559b2660c82f4ddd0d03d7bd97a37f57 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 17 Feb 2020 13:36:59 +0100 Subject: [PATCH 049/149] Update src/lib.rs --- src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index 0e7844b..f874045 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -222,7 +222,7 @@ //! ``` //! //! And to use async-std on `no_std` targets that only support `alloc` only -//! enable the `core` Cargo feature: +//! enable the `alloc` Cargo feature: //! //! ```toml //! [dependencies.async-std] From d87e2832150c249a6cce4210613f736819ecfe78 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 17 Feb 2020 13:38:24 +0100 Subject: [PATCH 050/149] Update src/lib.rs --- src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index f874045..a6a8f31 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -226,7 +226,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.0.0" +//! version = "1.5.0" //! default-features = false //! features = ["alloc"] //! ``` From b9e4b6da3e541417143a8ee36608d8adfe39a800 Mon Sep 17 00:00:00 2001 From: sunli Date: Wed, 19 Feb 2020 14:36:07 +0800 Subject: [PATCH 051/149] Add Xactor to the ecosystems inside the readme --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 8cd3ac5..4ebf592 100644 --- a/README.md +++ b/README.md @@ -134,6 +134,8 @@ documentation] on how to enable them. * [Surf](https://crates.io/crates/surf) — Surf the web. Surf is a friendly **HTTP client** built for casual Rustaceans and veterans alike. + * [Xactor](https://crates.io/crates/xactor) — Xactor is a rust actors framework based on async-std. + ## License From bd60cd9f81a16681761372fe57a8613c37e6adbd Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 20 Feb 2020 09:03:36 +0900 Subject: [PATCH 052/149] run `cargo fmt` --- src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index a6a8f31..d498792 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -220,7 +220,7 @@ //! default-features = false //! features = ["std"] //! ``` -//! +//! //! And to use async-std on `no_std` targets that only support `alloc` only //! enable the `alloc` Cargo feature: //! From 4742f461fe0d6e3219ed3d71d5d97468627a7d08 Mon Sep 17 00:00:00 2001 From: abhi Date: Sat, 22 Feb 2020 15:17:06 +0530 Subject: [PATCH 053/149] Add missing ? operator after handle.await According to line#118, there should be a `?` operator after `await`. --- docs/src/tutorial/receiving_messages.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/src/tutorial/receiving_messages.md b/docs/src/tutorial/receiving_messages.md index 4f70529..f62b65d 100644 --- a/docs/src/tutorial/receiving_messages.md +++ b/docs/src/tutorial/receiving_messages.md @@ -111,7 +111,7 @@ We can "fix" it by waiting for the task to be joined, like this: # # async move |stream| { let handle = task::spawn(connection_loop(stream)); -handle.await +handle.await? # }; ``` From 23b7c174f36f6632b0c29302f79094a23db325ec Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 3 Mar 2020 22:46:18 +0900 Subject: [PATCH 054/149] feat: Stabilize io::Std*Lock --- src/io/mod.rs | 12 +++--------- src/io/stderr.rs | 14 ++------------ src/io/stdin.rs | 14 ++------------ src/io/stdout.rs | 14 ++------------ 4 files changed, 9 insertions(+), 45 deletions(-) diff --git a/src/io/mod.rs b/src/io/mod.rs index 51c473d..3734c84 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -309,9 +309,9 @@ cfg_default! { #[doc(hidden)] pub use stdio::{_eprint, _print}; - pub use stderr::{stderr, Stderr}; - pub use stdin::{stdin, Stdin}; - pub use stdout::{stdout, Stdout}; + pub use stderr::{stderr, Stderr, StderrLock}; + pub use stdin::{stdin, Stdin, StdinLock}; + pub use stdout::{stdout, Stdout, StdoutLock}; pub use timeout::timeout; mod timeout; @@ -320,9 +320,3 @@ cfg_default! { mod stdio; mod stdout; } - -cfg_unstable_default! { - pub use stderr::StderrLock; - pub use stdin::StdinLock; - pub use stdout::StdoutLock; -} diff --git a/src/io/stderr.rs b/src/io/stderr.rs index 5ff8a02..fc5f4a0 100644 --- a/src/io/stderr.rs +++ b/src/io/stderr.rs @@ -1,14 +1,12 @@ use std::pin::Pin; use std::sync::Mutex; use std::future::Future; +use std::io::Write as _; use crate::io::{self, Write}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Write as _; -} +use once_cell::sync::Lazy; /// Constructs a new handle to the standard error of the current process. /// @@ -65,13 +63,9 @@ pub struct Stderr(Mutex); /// /// [`Write`]: trait.Read.html /// [`Stderr::lock`]: struct.Stderr.html#method.lock -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct StderrLock<'a>(std::io::StderrLock<'a>); -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] unsafe impl Send for StderrLock<'_> {} /// The state of the asynchronous stderr. @@ -128,8 +122,6 @@ impl Stderr { /// # /// # Ok(()) }) } /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] pub async fn lock(&self) -> StderrLock<'static> { static STDERR: Lazy = Lazy::new(std::io::stderr); @@ -240,8 +232,6 @@ cfg_windows! { } } -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] impl io::Write for StderrLock<'_> { fn poll_write( mut self: Pin<&mut Self>, diff --git a/src/io/stdin.rs b/src/io/stdin.rs index 369ccae..b299d09 100644 --- a/src/io/stdin.rs +++ b/src/io/stdin.rs @@ -1,16 +1,14 @@ use std::future::Future; use std::pin::Pin; use std::sync::Mutex; +use std::io::Read as _; use crate::future; use crate::io::{self, Read}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; use crate::utils::Context as _; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Read as _; -} +use once_cell::sync::Lazy; /// Constructs a new handle to the standard input of the current process. /// @@ -67,13 +65,9 @@ pub struct Stdin(Mutex); /// /// [`Read`]: trait.Read.html /// [`Stdin::lock`]: struct.Stdin.html#method.lock -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[cfg(feature = "unstable")] #[derive(Debug)] pub struct StdinLock<'a>(std::io::StdinLock<'a>); -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] unsafe impl Send for StdinLock<'_> {} /// The state of the asynchronous stdin. @@ -187,8 +181,6 @@ impl Stdin { /// # /// # Ok(()) }) } /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] pub async fn lock(&self) -> StdinLock<'static> { static STDIN: Lazy = Lazy::new(std::io::stdin); @@ -266,8 +258,6 @@ cfg_windows! { } } -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] impl Read for StdinLock<'_> { fn poll_read( mut self: Pin<&mut Self>, diff --git a/src/io/stdout.rs b/src/io/stdout.rs index 1711c09..d98ef13 100644 --- a/src/io/stdout.rs +++ b/src/io/stdout.rs @@ -1,14 +1,12 @@ use std::pin::Pin; use std::sync::Mutex; use std::future::Future; +use std::io::Write as _; use crate::io::{self, Write}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Write as _; -} +use once_cell::sync::Lazy; /// Constructs a new handle to the standard output of the current process. /// @@ -65,13 +63,9 @@ pub struct Stdout(Mutex); /// /// [`Write`]: trait.Read.html /// [`Stdout::lock`]: struct.Stdout.html#method.lock -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct StdoutLock<'a>(std::io::StdoutLock<'a>); -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] unsafe impl Send for StdoutLock<'_> {} /// The state of the asynchronous stdout. @@ -128,8 +122,6 @@ impl Stdout { /// # /// # Ok(()) }) } /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] pub async fn lock(&self) -> StdoutLock<'static> { static STDOUT: Lazy = Lazy::new(std::io::stdout); @@ -240,8 +232,6 @@ cfg_windows! { } } -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] impl Write for StdoutLock<'_> { fn poll_write( mut self: Pin<&mut Self>, From be60dd9fe7081ad89ca592a783e3329d6626ecf7 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 3 Mar 2020 22:49:59 +0900 Subject: [PATCH 055/149] fix: Remove unnecessary re-export and macros --- src/future/into_future.rs | 3 +- src/io/mod.rs | 4 - src/lib.rs | 3 - src/macros.rs | 168 -------------------------------------- 4 files changed, 2 insertions(+), 176 deletions(-) diff --git a/src/future/into_future.rs b/src/future/into_future.rs index 8e5e5e0..473ed45 100644 --- a/src/future/into_future.rs +++ b/src/future/into_future.rs @@ -5,9 +5,10 @@ use std::future::Future; /// # Examples /// /// ``` +/// use std::pin::Pin; +/// /// use async_std::future::{Future, IntoFuture}; /// use async_std::io; -/// use async_std::pin::Pin; /// /// struct Client; /// diff --git a/src/io/mod.rs b/src/io/mod.rs index 3734c84..c7c970c 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -305,10 +305,6 @@ cfg_std! { } cfg_default! { - // For use in the print macros. - #[doc(hidden)] - pub use stdio::{_eprint, _print}; - pub use stderr::{stderr, Stderr, StderrLock}; pub use stdin::{stdin, Stdin, StdinLock}; pub use stdout::{stdout, Stdout, StdoutLock}; diff --git a/src/lib.rs b/src/lib.rs index d498792..8cd0d30 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -273,9 +273,6 @@ cfg_default! { } cfg_unstable! { - pub mod pin; - pub mod process; - mod unit; mod vec; mod result; diff --git a/src/macros.rs b/src/macros.rs index 638a234..22cd00d 100644 --- a/src/macros.rs +++ b/src/macros.rs @@ -1,171 +1,3 @@ -/// Prints to the standard output. -/// -/// Equivalent to the [`println!`] macro except that a newline is not printed at -/// the end of the message. -/// -/// Note that stdout is frequently line-buffered by default so it may be -/// necessary to use [`io::stdout().flush()`][flush] to ensure the output is emitted -/// immediately. -/// -/// Use `print!` only for the primary output of your program. Use -/// [`eprint!`] instead to print error and progress messages. -/// -/// [`println!`]: macro.println.html -/// [flush]: io/trait.Write.html#tymethod.flush -/// [`eprint!`]: macro.eprint.html -/// -/// # Panics -/// -/// Panics if writing to `io::stdout()` fails. -/// -/// # Examples -/// -/// ``` -/// # async_std::task::block_on(async { -/// # -/// use async_std::io; -/// use async_std::prelude::*; -/// use async_std::print; -/// -/// print!("this ").await; -/// print!("will ").await; -/// print!("be ").await; -/// print!("on ").await; -/// print!("the ").await; -/// print!("same ").await; -/// print!("line ").await; -/// -/// io::stdout().flush().await.unwrap(); -/// -/// print!("this string has a newline, why not choose println! instead?\n").await; -/// -/// io::stdout().flush().await.unwrap(); -/// # -/// # }) -/// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[macro_export] -macro_rules! print { - ($($arg:tt)*) => ($crate::io::_print(format_args!($($arg)*))) -} - -/// Prints to the standard output, with a newline. -/// -/// On all platforms, the newline is the LINE FEED character (`\n`/`U+000A`) alone -/// (no additional CARRIAGE RETURN (`\r`/`U+000D`)). -/// -/// Use the [`format!`] syntax to write data to the standard output. -/// See [`std::fmt`] for more information. -/// -/// Use `println!` only for the primary output of your program. Use -/// [`eprintln!`] instead to print error and progress messages. -/// -/// [`format!`]: macro.format.html -/// [`std::fmt`]: https://doc.rust-lang.org/std/fmt/index.html -/// [`eprintln!`]: macro.eprintln.html -/// # Panics -/// -/// Panics if writing to `io::stdout` fails. -/// -/// # Examples -/// -/// ``` -/// # async_std::task::block_on(async { -/// # -/// use async_std::println; -/// -/// println!().await; // prints just a newline -/// println!("hello there!").await; -/// println!("format {} arguments", "some").await; -/// # -/// # }) -/// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[macro_export] -macro_rules! println { - () => ($crate::print!("\n")); - ($($arg:tt)*) => (async { - $crate::io::_print(format_args!($($arg)*)).await; - $crate::io::_print(format_args!("\n")).await; - }) -} - -/// Prints to the standard error. -/// -/// Equivalent to the [`print!`] macro, except that output goes to -/// [`io::stderr`] instead of `io::stdout`. See [`print!`] for -/// example usage. -/// -/// Use `eprint!` only for error and progress messages. Use `print!` -/// instead for the primary output of your program. -/// -/// [`io::stderr`]: io/struct.Stderr.html -/// [`print!`]: macro.print.html -/// -/// # Panics -/// -/// Panics if writing to `io::stderr` fails. -/// -/// # Examples -/// -/// ``` -/// # async_std::task::block_on(async { -/// # -/// use async_std::eprint; -/// -/// eprint!("Error: Could not complete task").await; -/// # -/// # }) -/// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[macro_export] -macro_rules! eprint { - ($($arg:tt)*) => ($crate::io::_eprint(format_args!($($arg)*))) -} - -/// Prints to the standard error, with a newline. -/// -/// Equivalent to the [`println!`] macro, except that output goes to -/// [`io::stderr`] instead of `io::stdout`. See [`println!`] for -/// example usage. -/// -/// Use `eprintln!` only for error and progress messages. Use `println!` -/// instead for the primary output of your program. -/// -/// [`io::stderr`]: io/struct.Stderr.html -/// [`println!`]: macro.println.html -/// -/// # Panics -/// -/// Panics if writing to `io::stderr` fails. -/// -/// # Examples -/// -/// ``` -/// # async_std::task::block_on(async { -/// # -/// use async_std::eprintln; -/// -/// eprintln!("Error: Could not complete task").await; -/// # -/// # }) -/// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[macro_export] -macro_rules! eprintln { - () => (async { $crate::eprint!("\n").await; }); - ($($arg:tt)*) => ( - async { - $crate::io::_eprint(format_args!($($arg)*)).await; - $crate::io::_eprint(format_args!("\n")).await; - } - ); -} - /// Declares task-local values. /// /// The macro wraps any number of static declarations and makes them task-local. Attributes and From 75223905bd1a61a064ccf30eb0f1f2f403910a8f Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 3 Mar 2020 23:12:09 +0900 Subject: [PATCH 056/149] fix: Stabilize stream most method --- src/stream/double_ended_stream/mod.rs | 2 - src/stream/exact_size_stream.rs | 2 - src/stream/extend.rs | 2 - src/stream/fused_stream.rs | 2 - src/stream/interval.rs | 4 -- src/stream/mod.rs | 36 +++++++++--------- src/stream/once.rs | 4 +- src/stream/product.rs | 12 +++--- src/stream/stream/count.rs | 2 - src/stream/stream/merge.rs | 2 - src/stream/stream/mod.rs | 54 ++++++++++----------------- src/stream/stream/timeout.rs | 2 - src/stream/stream/unzip.rs | 2 - src/stream/successors.rs | 4 -- src/stream/sum.rs | 2 - 15 files changed, 43 insertions(+), 89 deletions(-) diff --git a/src/stream/double_ended_stream/mod.rs b/src/stream/double_ended_stream/mod.rs index a177865..ed10fd2 100644 --- a/src/stream/double_ended_stream/mod.rs +++ b/src/stream/double_ended_stream/mod.rs @@ -22,8 +22,6 @@ use try_rfold::TryRFoldFuture; /// `Item`s from the back, as well as the front. /// /// [`Stream`]: trait.Stream.html -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait DoubleEndedStream: Stream { #[doc = r#" Attempts to receive the next item from the back of the stream. diff --git a/src/stream/exact_size_stream.rs b/src/stream/exact_size_stream.rs index 8b6ba97..28792c6 100644 --- a/src/stream/exact_size_stream.rs +++ b/src/stream/exact_size_stream.rs @@ -76,8 +76,6 @@ pub use crate::stream::Stream; /// # }); /// ``` #[allow(clippy::len_without_is_empty)] // ExactSizeIterator::is_empty is unstable -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait ExactSizeStream: Stream { /// Returns the exact number of times the stream will iterate. /// diff --git a/src/stream/extend.rs b/src/stream/extend.rs index 702cbca..9f54007 100644 --- a/src/stream/extend.rs +++ b/src/stream/extend.rs @@ -27,8 +27,6 @@ use crate::stream::IntoStream; /// # /// # }) /// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Extend { /// Extends a collection with the contents of a stream. fn extend<'a, T: IntoStream + 'a>( diff --git a/src/stream/fused_stream.rs b/src/stream/fused_stream.rs index e14ab5b..5d02f1d 100644 --- a/src/stream/fused_stream.rs +++ b/src/stream/fused_stream.rs @@ -14,8 +14,6 @@ use crate::stream::Stream; /// [`None`]: https://doc.rust-lang.org/std/option/enum.Option.html#variant.None /// [`Stream::fuse`]: trait.Stream.html#method.fuse /// [`Fuse`]: struct.Fuse.html -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait FusedStream: Stream {} impl FusedStream for &mut S {} diff --git a/src/stream/interval.rs b/src/stream/interval.rs index 7a0c174..8dd6b5e 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -41,8 +41,6 @@ use futures_timer::Delay; /// # /// # Ok(()) }) } /// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub fn interval(dur: Duration) -> Interval { Interval { delay: Delay::new(dur), @@ -56,8 +54,6 @@ pub fn interval(dur: Duration) -> Interval { /// documentation for more. /// /// [`interval`]: fn.interval.html -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Interval { delay: Delay, diff --git a/src/stream/mod.rs b/src/stream/mod.rs index 0bfd4e8..8984b22 100644 --- a/src/stream/mod.rs +++ b/src/stream/mod.rs @@ -300,46 +300,46 @@ //! [`take`]: trait.Stream.html#method.take //! [`min`]: trait.Stream.html#method.min +pub use double_ended_stream::DoubleEndedStream; pub use empty::{empty, Empty}; +pub use exact_size_stream::ExactSizeStream; pub use from_fn::{from_fn, FromFn}; pub use from_iter::{from_iter, FromIter}; +pub use fused_stream::FusedStream; +pub use interval::{interval, Interval}; pub use once::{once, Once}; +pub use pending::{pending, Pending}; +pub use product::Product; pub use repeat::{repeat, Repeat}; pub use repeat_with::{repeat_with, RepeatWith}; +pub use stream::Merge; pub use stream::*; +pub use successors::{successors, Successors}; +pub use sum::Sum; pub(crate) mod stream; +mod double_ended_stream; mod empty; +mod exact_size_stream; mod from_fn; mod from_iter; +mod fused_stream; +mod interval; mod once; +mod pending; +mod product; mod repeat; mod repeat_with; +mod successors; +mod sum; cfg_unstable! { - mod double_ended_stream; - mod exact_size_stream; - mod extend; mod from_stream; - mod fused_stream; - mod interval; mod into_stream; - mod pending; - mod product; - mod successors; - mod sum; + mod extend; - pub use double_ended_stream::DoubleEndedStream; - pub use exact_size_stream::ExactSizeStream; pub use extend::{extend, Extend}; pub use from_stream::FromStream; - pub use fused_stream::FusedStream; - pub use interval::{interval, Interval}; pub use into_stream::IntoStream; - pub use pending::{pending, Pending}; - pub use product::Product; - pub use stream::Merge; - pub use successors::{successors, Successors}; - pub use sum::Sum; } diff --git a/src/stream/once.rs b/src/stream/once.rs index b86f181..c21a1e6 100644 --- a/src/stream/once.rs +++ b/src/stream/once.rs @@ -5,7 +5,6 @@ use pin_project_lite::pin_project; use crate::stream::Stream; use crate::task::{Context, Poll}; -#[cfg(feature = "unstable")] use crate::stream::DoubleEndedStream; /// Creates a stream that yields a single item. @@ -50,8 +49,7 @@ impl Stream for Once { } } -#[cfg(feature = "unstable")] -impl DoubleEndedStream for Once { +impl DoubleEndedStream for Once { fn poll_next_back(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(self.project().value.take()) } diff --git a/src/stream/product.rs b/src/stream/product.rs index 15497e8..44aef4e 100644 --- a/src/stream/product.rs +++ b/src/stream/product.rs @@ -1,5 +1,5 @@ -use core::pin::Pin; use core::future::Future; +use core::pin::Pin; use crate::stream::Stream; @@ -13,8 +13,6 @@ use crate::stream::Stream; /// [`product`]: trait.Product.html#tymethod.product /// [`FromStream`]: trait.FromStream.html /// [`Stream::product`]: trait.Stream.html#method.product -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Product: Sized { /// Method which takes a stream and generates `Self` from the elements by /// multiplying the items. @@ -23,9 +21,9 @@ pub trait Product: Sized { S: Stream + 'a; } -use core::ops::Mul; -use core::num::Wrapping; use crate::stream::stream::StreamExt; +use core::num::Wrapping; +use core::ops::Mul; macro_rules! integer_product { (@impls $one: expr, $($a:ty)*) => ($( @@ -75,5 +73,5 @@ macro_rules! float_product { ); } -integer_product!{ i8 i16 i32 i64 i128 isize u8 u16 u32 u64 u128 usize } -float_product!{ f32 f64 } +integer_product! { i8 i16 i32 i64 i128 isize u8 u16 u32 u64 u128 usize } +float_product! { f32 f64 } diff --git a/src/stream/stream/count.rs b/src/stream/stream/count.rs index 63e0449..ae6c78c 100644 --- a/src/stream/stream/count.rs +++ b/src/stream/stream/count.rs @@ -9,8 +9,6 @@ use crate::task::{Context, Poll}; pin_project! { #[doc(hidden)] #[allow(missing_debug_implementations)] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub struct CountFuture { #[pin] stream: S, diff --git a/src/stream/stream/merge.rs b/src/stream/stream/merge.rs index 2320972..d1eea9d 100644 --- a/src/stream/stream/merge.rs +++ b/src/stream/stream/merge.rs @@ -16,8 +16,6 @@ pin_project! { /// /// [`merge`]: trait.Stream.html#method.merge /// [`Stream`]: trait.Stream.html - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Merge { #[pin] diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index d0cc718..da85269 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -27,7 +27,9 @@ mod chain; mod cloned; mod cmp; mod copied; +mod count; mod cycle; +mod delay; mod enumerate; mod eq; mod filter; @@ -47,6 +49,7 @@ mod map; mod max; mod max_by; mod max_by_key; +mod merge; mod min; mod min_by; mod min_by_key; @@ -61,13 +64,17 @@ mod skip_while; mod step_by; mod take; mod take_while; +mod throttle; +mod timeout; mod try_fold; mod try_for_each; +mod unzip; mod zip; use all::AllFuture; use any::AnyFuture; use cmp::CmpFuture; +use count::CountFuture; use cycle::Cycle; use enumerate::Enumerate; use eq::EqFuture; @@ -94,53 +101,48 @@ use partial_cmp::PartialCmpFuture; use position::PositionFuture; use try_fold::TryFoldFuture; use try_for_each::TryForEachFuture; +use unzip::UnzipFuture; pub use chain::Chain; pub use cloned::Cloned; pub use copied::Copied; +pub use delay::Delay; pub use filter::Filter; pub use fuse::Fuse; pub use inspect::Inspect; pub use map::Map; +pub use merge::Merge; pub use scan::Scan; pub use skip::Skip; pub use skip_while::SkipWhile; pub use step_by::StepBy; pub use take::Take; pub use take_while::TakeWhile; +pub use throttle::Throttle; +pub use timeout::{Timeout, TimeoutError}; pub use zip::Zip; use core::cmp::Ordering; +use core::future::Future; +use core::pin::Pin; +use core::time::Duration; -cfg_unstable! { - use core::future::Future; - use core::pin::Pin; - use core::time::Duration; +use crate::stream::{Product, Sum}; +cfg_unstable! { + use crate::stream::FromStream; use crate::stream::into_stream::IntoStream; - use crate::stream::{FromStream, Product, Sum}; use crate::stream::Extend; - use count::CountFuture; use partition::PartitionFuture; - use unzip::UnzipFuture; - pub use merge::Merge; pub use flatten::Flatten; pub use flat_map::FlatMap; - pub use timeout::{TimeoutError, Timeout}; - pub use throttle::Throttle; - pub use delay::Delay; - mod count; - mod merge; + mod flatten; mod flat_map; mod partition; - mod timeout; - mod throttle; - mod delay; - mod unzip; } extension_trait! { @@ -355,8 +357,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn throttle(self, d: Duration) -> Throttle where Self: Sized, @@ -598,8 +598,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(any(feature = "unstable", feature = "docs"))] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn delay(self, dur: std::time::Duration) -> Delay where Self: Sized, @@ -1511,8 +1509,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn by_ref(&mut self) -> &mut Self { self } @@ -1656,8 +1652,6 @@ extension_trait! { # Ok(()) }) } ``` "#] - #[cfg(any(feature = "unstable", feature = "docs"))] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn timeout(self, dur: Duration) -> Timeout where Self: Stream + Sized, @@ -1822,8 +1816,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn unzip(self) -> impl Future [UnzipFuture] where FromA: Default + Extend, @@ -1921,8 +1913,6 @@ extension_trait! { # }); ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn merge(self, other: U) -> Merge where Self: Sized, @@ -2068,8 +2058,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn count(self) -> impl Future [CountFuture] where Self: Sized, @@ -2330,8 +2318,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn sum<'a, S>( self, ) -> impl Future + 'a [Pin + 'a>>] @@ -2376,8 +2362,6 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn product<'a, P>( self, ) -> impl Future + 'a [Pin + 'a>>] diff --git a/src/stream/stream/timeout.rs b/src/stream/stream/timeout.rs index ce658c8..411be7e 100644 --- a/src/stream/stream/timeout.rs +++ b/src/stream/stream/timeout.rs @@ -47,8 +47,6 @@ impl Stream for Timeout { } /// An error returned when a stream times out. -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[cfg(any(feature = "unstable", feature = "docs"))] #[derive(Clone, Copy, Debug, Eq, PartialEq)] pub struct TimeoutError { _private: (), diff --git a/src/stream/stream/unzip.rs b/src/stream/stream/unzip.rs index 7771509..94cbc0a 100644 --- a/src/stream/stream/unzip.rs +++ b/src/stream/stream/unzip.rs @@ -8,8 +8,6 @@ use crate::task::{Context, Poll}; pin_project! { #[derive(Clone, Debug)] - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub struct UnzipFuture { #[pin] stream: S, diff --git a/src/stream/successors.rs b/src/stream/successors.rs index a9ce40f..fb450c6 100644 --- a/src/stream/successors.rs +++ b/src/stream/successors.rs @@ -27,8 +27,6 @@ use pin_project_lite::pin_project; /// # /// # }) } /// ``` -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub fn successors(first: Option, succ: F) -> Successors where F: FnMut(&T) -> Option, @@ -43,8 +41,6 @@ pin_project! { /// This stream is constructed by [`successors`] function /// /// [`successors`]: fn.succssors.html - #[cfg(feature = "unstable")] - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Successors where diff --git a/src/stream/sum.rs b/src/stream/sum.rs index 3b3144e..eee41bd 100644 --- a/src/stream/sum.rs +++ b/src/stream/sum.rs @@ -13,8 +13,6 @@ use crate::stream::Stream; /// [`sum`]: trait.Sum.html#tymethod.sum /// [`FromStream`]: trait.FromStream.html /// [`Stream::sum`]: trait.Stream.html#method.sum -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Sum: Sized { /// Method which takes a stream and generates `Self` from the elements by /// "summing up" the items. From 9a62df143f94b58c5e5b7707877e982c9b724d9a Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 3 Mar 2020 23:14:25 +0900 Subject: [PATCH 057/149] add whitespace --- src/future/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/future/mod.rs b/src/future/mod.rs index 9b75533..56cd71d 100644 --- a/src/future/mod.rs +++ b/src/future/mod.rs @@ -63,12 +63,14 @@ cfg_std! { cfg_default! { pub use timeout::{timeout, TimeoutError}; + mod timeout; } cfg_unstable! { pub use into_future::IntoFuture; pub(crate) use maybe_done::MaybeDone; + mod into_future; mod maybe_done; } From f31878655ed6d93a846ea7ec8dbed58a314e6fe8 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 4 Mar 2020 08:30:45 +0900 Subject: [PATCH 058/149] fix: Stabilize stream method --- Cargo.toml | 2 +- src/prelude.rs | 12 +++---- src/stream/double_ended_stream/mod.rs | 4 +-- src/stream/mod.rs | 37 +++++++++++---------- src/stream/once.rs | 2 ++ src/stream/product.rs | 1 + src/stream/stream/mod.rs | 46 +++++++++++++++++---------- src/stream/sum.rs | 1 + src/utils.rs | 2 +- 9 files changed, 63 insertions(+), 44 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 2d4f602..00766b4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -35,7 +35,7 @@ default = [ "pin-project-lite", ] docs = ["attributes", "unstable", "default"] -unstable = ["std", "broadcaster", "futures-timer"] +unstable = ["std", "broadcaster"] attributes = ["async-attributes"] std = [ "alloc", diff --git a/src/prelude.rs b/src/prelude.rs index a2a14a1..a522745 100644 --- a/src/prelude.rs +++ b/src/prelude.rs @@ -38,16 +38,14 @@ cfg_std! { pub use crate::io::prelude::SeekExt as _; #[doc(no_inline)] pub use crate::io::prelude::WriteExt as _; -} - -cfg_default! { - #[doc(no_inline)] - pub use crate::task_local; -} -cfg_unstable! { #[doc(no_inline)] pub use crate::stream::DoubleEndedStream; #[doc(no_inline)] pub use crate::stream::ExactSizeStream; } + +cfg_default! { + #[doc(no_inline)] + pub use crate::task_local; +} diff --git a/src/stream/double_ended_stream/mod.rs b/src/stream/double_ended_stream/mod.rs index ed10fd2..1563e1b 100644 --- a/src/stream/double_ended_stream/mod.rs +++ b/src/stream/double_ended_stream/mod.rs @@ -1,7 +1,7 @@ use crate::stream::Stream; -use std::pin::Pin; -use std::task::{Context, Poll}; +use core::pin::Pin; +use core::task::{Context, Poll}; mod next_back; mod nth_back; diff --git a/src/stream/mod.rs b/src/stream/mod.rs index 8984b22..f1c5fdf 100644 --- a/src/stream/mod.rs +++ b/src/stream/mod.rs @@ -300,39 +300,42 @@ //! [`take`]: trait.Stream.html#method.take //! [`min`]: trait.Stream.html#method.min -pub use double_ended_stream::DoubleEndedStream; pub use empty::{empty, Empty}; -pub use exact_size_stream::ExactSizeStream; pub use from_fn::{from_fn, FromFn}; pub use from_iter::{from_iter, FromIter}; -pub use fused_stream::FusedStream; -pub use interval::{interval, Interval}; pub use once::{once, Once}; -pub use pending::{pending, Pending}; -pub use product::Product; pub use repeat::{repeat, Repeat}; pub use repeat_with::{repeat_with, RepeatWith}; -pub use stream::Merge; pub use stream::*; -pub use successors::{successors, Successors}; -pub use sum::Sum; pub(crate) mod stream; -mod double_ended_stream; mod empty; -mod exact_size_stream; mod from_fn; mod from_iter; -mod fused_stream; -mod interval; mod once; -mod pending; -mod product; mod repeat; mod repeat_with; -mod successors; -mod sum; + +cfg_std! { + pub use double_ended_stream::DoubleEndedStream; + pub use exact_size_stream::ExactSizeStream; + pub use fused_stream::FusedStream; + pub use interval::{interval, Interval}; + pub use pending::{pending, Pending}; + pub use product::Product; + pub use successors::{successors, Successors}; + pub use sum::Sum; + + mod double_ended_stream; + mod exact_size_stream; + mod fused_stream; + mod interval; + mod pending; + mod product; + mod successors; + mod sum; +} cfg_unstable! { mod from_stream; diff --git a/src/stream/once.rs b/src/stream/once.rs index c21a1e6..9daeac5 100644 --- a/src/stream/once.rs +++ b/src/stream/once.rs @@ -5,6 +5,7 @@ use pin_project_lite::pin_project; use crate::stream::Stream; use crate::task::{Context, Poll}; +#[cfg(feature = "std")] use crate::stream::DoubleEndedStream; /// Creates a stream that yields a single item. @@ -49,6 +50,7 @@ impl Stream for Once { } } +#[cfg(feature = "std")] impl DoubleEndedStream for Once { fn poll_next_back(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(self.project().value.take()) diff --git a/src/stream/product.rs b/src/stream/product.rs index 44aef4e..9794846 100644 --- a/src/stream/product.rs +++ b/src/stream/product.rs @@ -1,3 +1,4 @@ +use alloc::boxed::Box; use core::future::Future; use core::pin::Pin; diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index da85269..5cdf530 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -27,9 +27,7 @@ mod chain; mod cloned; mod cmp; mod copied; -mod count; mod cycle; -mod delay; mod enumerate; mod eq; mod filter; @@ -49,7 +47,6 @@ mod map; mod max; mod max_by; mod max_by_key; -mod merge; mod min; mod min_by; mod min_by_key; @@ -64,17 +61,13 @@ mod skip_while; mod step_by; mod take; mod take_while; -mod throttle; -mod timeout; mod try_fold; mod try_for_each; -mod unzip; mod zip; use all::AllFuture; use any::AnyFuture; use cmp::CmpFuture; -use count::CountFuture; use cycle::Cycle; use enumerate::Enumerate; use eq::EqFuture; @@ -101,33 +94,46 @@ use partial_cmp::PartialCmpFuture; use position::PositionFuture; use try_fold::TryFoldFuture; use try_for_each::TryForEachFuture; -use unzip::UnzipFuture; pub use chain::Chain; pub use cloned::Cloned; pub use copied::Copied; -pub use delay::Delay; pub use filter::Filter; pub use fuse::Fuse; pub use inspect::Inspect; pub use map::Map; -pub use merge::Merge; pub use scan::Scan; pub use skip::Skip; pub use skip_while::SkipWhile; pub use step_by::StepBy; pub use take::Take; pub use take_while::TakeWhile; -pub use throttle::Throttle; -pub use timeout::{Timeout, TimeoutError}; pub use zip::Zip; use core::cmp::Ordering; -use core::future::Future; -use core::pin::Pin; -use core::time::Duration; -use crate::stream::{Product, Sum}; +cfg_std! { + use core::time::Duration; + use crate::stream::{Product, Sum}; + use alloc::boxed::Box; + use core::future::Future; + use core::pin::Pin; + + use unzip::UnzipFuture; + use count::CountFuture; + + pub use throttle::Throttle; + pub use merge::Merge; + pub use delay::Delay; + pub use timeout::{Timeout, TimeoutError}; + + mod timeout; + mod throttle; + mod merge; + mod delay; + mod unzip; + mod count; +} cfg_unstable! { use crate::stream::FromStream; @@ -357,6 +363,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn throttle(self, d: Duration) -> Throttle where Self: Sized, @@ -598,6 +605,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn delay(self, dur: std::time::Duration) -> Delay where Self: Sized, @@ -1652,6 +1660,7 @@ extension_trait! { # Ok(()) }) } ``` "#] + #[cfg(feature = "std")] fn timeout(self, dur: Duration) -> Timeout where Self: Stream + Sized, @@ -1816,6 +1825,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn unzip(self) -> impl Future [UnzipFuture] where FromA: Default + Extend, @@ -1913,6 +1923,7 @@ extension_trait! { # }); ``` "#] + #[cfg(feature = "std")] fn merge(self, other: U) -> Merge where Self: Sized, @@ -2058,6 +2069,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn count(self) -> impl Future [CountFuture] where Self: Sized, @@ -2318,6 +2330,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn sum<'a, S>( self, ) -> impl Future + 'a [Pin + 'a>>] @@ -2362,6 +2375,7 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "std")] fn product<'a, P>( self, ) -> impl Future + 'a [Pin + 'a>>] diff --git a/src/stream/sum.rs b/src/stream/sum.rs index eee41bd..dc41a0f 100644 --- a/src/stream/sum.rs +++ b/src/stream/sum.rs @@ -1,3 +1,4 @@ +use alloc::boxed::Box; use core::future::Future; use core::pin::Pin; diff --git a/src/utils.rs b/src/utils.rs index f18b74d..ab2b3db 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -21,7 +21,7 @@ pub fn abort_on_panic(f: impl FnOnce() -> T) -> T { } /// Generates a random number in `0..n`. -#[cfg(any(feature = "unstable", feature = "default"))] +#[cfg(any(feature = "std", feature = "default"))] pub fn random(n: u32) -> u32 { use std::cell::Cell; use std::num::Wrapping; From 1e18839f1f4eaf1cde34bc893cee26adeb1ae5d9 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 4 Mar 2020 08:38:31 +0900 Subject: [PATCH 059/149] fix warning --- Cargo.toml | 2 +- src/utils.rs | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 00766b4..c1479c2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,7 +26,6 @@ default = [ "async-task", "crossbeam-channel", "crossbeam-deque", - "futures-timer", "kv-log-macro", "log", "mio", @@ -42,6 +41,7 @@ std = [ "crossbeam-utils", "futures-core/std", "futures-io", + "futures-timer", "memchr", "once_cell", "pin-utils", diff --git a/src/utils.rs b/src/utils.rs index ab2b3db..a930a84 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -257,11 +257,6 @@ macro_rules! extension_trait { $(#[cfg(feature = "docs")] $imp)* }; - // Optimization: expand `$head` eagerly before starting a new method definition. - (@ext ($($head:tt)*) #[doc = $d:literal] $($tail:tt)*) => { - $($head)* extension_trait!(@ext (#[doc = $d]) $($tail)*); - }; - // Parse the return type in an extension method. (@doc ($($head:tt)*) -> impl Future $(+ $lt:lifetime)? [$f:ty] $($tail:tt)*) => { extension_trait!(@doc ($($head)* -> owned::ImplFuture<$out>) $($tail)*); From b95bd6c1fe6403db3becb776bf0613577980c97c Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 5 Mar 2020 09:22:00 +0900 Subject: [PATCH 060/149] fix: Remove unnecessary io modules --- src/io/copy.rs | 10 +- src/io/mod.rs | 74 ++----------- src/io/stderr.rs | 251 ------------------------------------------ src/io/stdin.rs | 269 ---------------------------------------------- src/io/stdio.rs | 21 ---- src/io/stdout.rs | 251 ------------------------------------------ src/io/timeout.rs | 4 +- 7 files changed, 18 insertions(+), 862 deletions(-) delete mode 100644 src/io/stderr.rs delete mode 100644 src/io/stdin.rs delete mode 100644 src/io/stdio.rs delete mode 100644 src/io/stdout.rs diff --git a/src/io/copy.rs b/src/io/copy.rs index f05ed0e..f946c8b 100644 --- a/src/io/copy.rs +++ b/src/io/copy.rs @@ -32,13 +32,14 @@ use crate::utils::Context as _; /// /// # Examples /// -/// ``` +/// ```no_run /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { /// # /// use async_std::io; +/// use async_std::fs::File; /// /// let mut reader: &[u8] = b"hello"; -/// let mut writer = io::stdout(); +/// let mut writer = File::open("foo.txt").await?; /// /// io::copy(&mut reader, &mut writer).await?; /// # @@ -119,13 +120,14 @@ where /// /// # Examples /// -/// ``` +/// ```no_run /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { /// # /// use async_std::io; +/// use async_std::fs::File; /// /// let mut reader: &[u8] = b"hello"; -/// let mut writer = io::stdout(); +/// let mut writer = File::open("foo.txt").await?; /// /// io::copy(&mut reader, &mut writer).await?; /// # diff --git a/src/io/mod.rs b/src/io/mod.rs index c7c970c..65c204c 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -122,56 +122,6 @@ //! # Ok(()) }) } //! ``` //! -//! ## Standard input and output -//! -//! A very common source of input is standard input: -//! -//! ```no_run -//! use async_std::io; -//! -//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -//! # -//! let mut input = String::new(); -//! -//! io::stdin().read_line(&mut input).await?; -//! -//! println!("You typed: {}", input.trim()); -//! # -//! # Ok(()) }) } -//! ``` -//! -//! Note that you cannot use the [`?` operator] in functions that do not return -//! a [`Result`][`Result`]. Instead, you can call [`.unwrap()`] -//! or `match` on the return value to catch any possible errors: -//! -//! ```no_run -//! use async_std::io; -//! -//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -//! # -//! let mut input = String::new(); -//! -//! io::stdin().read_line(&mut input).await.unwrap(); -//! # -//! # Ok(()) }) } -//! ``` -//! -//! And a very common source of output is standard output: -//! -//! ```no_run -//! use async_std::io; -//! use async_std::io::prelude::*; -//! -//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -//! # -//! io::stdout().write(&[42]).await?; -//! # -//! # Ok(()) }) } -//! ``` -//! -//! Of course, using [`io::stdout`] directly is less common than something like -//! [`println!`]. -//! //! ## Iterator types //! //! A large number of the structures provided by `std::io` are for various @@ -204,10 +154,14 @@ //! //! ```no_run //! use async_std::io; +//! use async_std::fs::File; //! //! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { //! # -//! io::copy(&mut io::stdin(), &mut io::stdout()).await?; +//! let mut reader: &[u8] = b"hello"; +//! let mut writer = File::open("foo.txt").await?; +//! +//! io::copy(&mut reader, &mut writer).await?; //! # //! # Ok(()) }) } //! ``` @@ -224,13 +178,14 @@ //! ``` //! #![allow(dead_code)] //! use async_std::io; +//! use std::time::Duration; //! //! async fn read_input() -> io::Result<()> { -//! let mut input = String::new(); -//! -//! io::stdin().read_line(&mut input).await?; +//! let f = io::timeout(Duration::from_secs(5), async { +//! Ok(()) +//! }); //! -//! println!("You typed: {}", input.trim()); +//! assert_eq!(f.await?, ()); //! //! Ok(()) //! } @@ -260,8 +215,6 @@ //! [`BufReader`]: struct.BufReader.html //! [`BufWriter`]: struct.BufWriter.html //! [`Write::write`]: trait.Write.html#tymethod.write -//! [`io::stdout`]: fn.stdout.html -//! [`println!`]: ../macro.println.html //! [`Lines`]: struct.Lines.html //! [`io::Result`]: type.Result.html //! [`?` operator]: https://doc.rust-lang.org/stable/book/appendix-02-operators.html @@ -305,14 +258,7 @@ cfg_std! { } cfg_default! { - pub use stderr::{stderr, Stderr, StderrLock}; - pub use stdin::{stdin, Stdin, StdinLock}; - pub use stdout::{stdout, Stdout, StdoutLock}; pub use timeout::timeout; mod timeout; - mod stderr; - mod stdin; - mod stdio; - mod stdout; } diff --git a/src/io/stderr.rs b/src/io/stderr.rs deleted file mode 100644 index fc5f4a0..0000000 --- a/src/io/stderr.rs +++ /dev/null @@ -1,251 +0,0 @@ -use std::pin::Pin; -use std::sync::Mutex; -use std::future::Future; -use std::io::Write as _; - -use crate::io::{self, Write}; -use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; - -use once_cell::sync::Lazy; - -/// Constructs a new handle to the standard error of the current process. -/// -/// This function is an async version of [`std::io::stderr`]. -/// -/// [`std::io::stderr`]: https://doc.rust-lang.org/std/io/fn.stderr.html -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// # Examples -/// -/// ```no_run -/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -/// # -/// use async_std::io; -/// use async_std::prelude::*; -/// -/// let mut stderr = io::stderr(); -/// stderr.write_all(b"Hello, world!").await?; -/// # -/// # Ok(()) }) } -/// ``` -pub fn stderr() -> Stderr { - Stderr(Mutex::new(State::Idle(Some(Inner { - stderr: std::io::stderr(), - buf: Vec::new(), - last_op: None, - })))) -} - -/// A handle to the standard error of the current process. -/// -/// This writer is created by the [`stderr`] function. See its documentation for -/// more. -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// [`stderr`]: fn.stderr.html -#[derive(Debug)] -pub struct Stderr(Mutex); - -/// A locked reference to the Stderr handle. -/// -/// This handle implements the [`Write`] traits, and is constructed via the [`Stderr::lock`] -/// method. -/// -/// [`Write`]: trait.Read.html -/// [`Stderr::lock`]: struct.Stderr.html#method.lock -#[derive(Debug)] -pub struct StderrLock<'a>(std::io::StderrLock<'a>); - -unsafe impl Send for StderrLock<'_> {} - -/// The state of the asynchronous stderr. -/// -/// The stderr can be either idle or busy performing an asynchronous operation. -#[derive(Debug)] -enum State { - /// The stderr is idle. - Idle(Option), - - /// The stderr is blocked on an asynchronous operation. - /// - /// Awaiting this operation will result in the new state of the stderr. - Busy(JoinHandle), -} - -/// Inner representation of the asynchronous stderr. -#[derive(Debug)] -struct Inner { - /// The blocking stderr handle. - stderr: std::io::Stderr, - - /// The write buffer. - buf: Vec, - - /// The result of the last asynchronous operation on the stderr. - last_op: Option, -} - -/// Possible results of an asynchronous operation on the stderr. -#[derive(Debug)] -enum Operation { - Write(io::Result), - Flush(io::Result<()>), -} - -impl Stderr { - /// Locks this handle to the standard error stream, returning a writable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let stderr = io::stderr(); - /// let mut handle = stderr.lock().await; - /// - /// handle.write_all(b"hello world").await?; - /// # - /// # Ok(()) }) } - /// ``` - pub async fn lock(&self) -> StderrLock<'static> { - static STDERR: Lazy = Lazy::new(std::io::stderr); - - spawn_blocking(move || StderrLock(STDERR.lock())).await - } -} - -impl Write for Stderr { - fn poll_write( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::Write(res)) = inner.last_op.take() { - let n = res?; - - // If more data was written than is available in the buffer, let's retry - // the write operation. - if n <= buf.len() { - return Poll::Ready(Ok(n)); - } - } else { - let mut inner = opt.take().unwrap(); - - // Set the length of the inner buffer to the length of the provided buffer. - if inner.buf.len() < buf.len() { - inner.buf.reserve(buf.len() - inner.buf.len()); - } - unsafe { - inner.buf.set_len(buf.len()); - } - - // Copy the data to write into the inner buffer. - inner.buf[..buf.len()].copy_from_slice(buf); - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - let res = std::io::Write::write(&mut inner.stderr, &inner.buf); - inner.last_op = Some(Operation::Write(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stderr is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - } - - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::Flush(res)) = inner.last_op.take() { - return Poll::Ready(res); - } else { - let mut inner = opt.take().unwrap(); - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - let res = std::io::Write::flush(&mut inner.stderr); - inner.last_op = Some(Operation::Flush(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stderr is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} - -cfg_unix! { - use crate::os::unix::io::{AsRawFd, RawFd}; - - impl AsRawFd for Stderr { - fn as_raw_fd(&self) -> RawFd { - std::io::stderr().as_raw_fd() - } - } -} - -cfg_windows! { - use crate::os::windows::io::{AsRawHandle, RawHandle}; - - impl AsRawHandle for Stderr { - fn as_raw_handle(&self) -> RawHandle { - std::io::stderr().as_raw_handle() - } - } -} - -impl io::Write for StderrLock<'_> { - fn poll_write( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - Poll::Ready(self.0.write(buf)) - } - - fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(self.0.flush()) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} diff --git a/src/io/stdin.rs b/src/io/stdin.rs deleted file mode 100644 index b299d09..0000000 --- a/src/io/stdin.rs +++ /dev/null @@ -1,269 +0,0 @@ -use std::future::Future; -use std::pin::Pin; -use std::sync::Mutex; -use std::io::Read as _; - -use crate::future; -use crate::io::{self, Read}; -use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; -use crate::utils::Context as _; - -use once_cell::sync::Lazy; - -/// Constructs a new handle to the standard input of the current process. -/// -/// This function is an async version of [`std::io::stdin`]. -/// -/// [`std::io::stdin`]: https://doc.rust-lang.org/std/io/fn.stdin.html -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// # Examples -/// -/// ```no_run -/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -/// # -/// use async_std::io; -/// -/// let stdin = io::stdin(); -/// let mut line = String::new(); -/// stdin.read_line(&mut line).await?; -/// # -/// # Ok(()) }) } -/// ``` -pub fn stdin() -> Stdin { - Stdin(Mutex::new(State::Idle(Some(Inner { - stdin: std::io::stdin(), - line: String::new(), - buf: Vec::new(), - last_op: None, - })))) -} - -/// A handle to the standard input of the current process. -/// -/// This reader is created by the [`stdin`] function. See its documentation for -/// more. -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// [`stdin`]: fn.stdin.html -#[derive(Debug)] -pub struct Stdin(Mutex); - -/// A locked reference to the Stdin handle. -/// -/// This handle implements the [`Read`] traits, and is constructed via the [`Stdin::lock`] method. -/// -/// [`Read`]: trait.Read.html -/// [`Stdin::lock`]: struct.Stdin.html#method.lock -#[derive(Debug)] -pub struct StdinLock<'a>(std::io::StdinLock<'a>); - -unsafe impl Send for StdinLock<'_> {} - -/// The state of the asynchronous stdin. -/// -/// The stdin can be either idle or busy performing an asynchronous operation. -#[derive(Debug)] -enum State { - /// The stdin is idle. - Idle(Option), - - /// The stdin is blocked on an asynchronous operation. - /// - /// Awaiting this operation will result in the new state of the stdin. - Busy(JoinHandle), -} - -/// Inner representation of the asynchronous stdin. -#[derive(Debug)] -struct Inner { - /// The blocking stdin handle. - stdin: std::io::Stdin, - - /// The line buffer. - line: String, - - /// The write buffer. - buf: Vec, - - /// The result of the last asynchronous operation on the stdin. - last_op: Option, -} - -/// Possible results of an asynchronous operation on the stdin. -#[derive(Debug)] -enum Operation { - ReadLine(io::Result), - Read(io::Result), -} - -impl Stdin { - /// Reads a line of input into the specified buffer. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// - /// let stdin = io::stdin(); - /// let mut line = String::new(); - /// stdin.read_line(&mut line).await?; - /// # - /// # Ok(()) }) } - /// ``` - pub async fn read_line(&self, buf: &mut String) -> io::Result { - future::poll_fn(|cx| { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::ReadLine(res)) = inner.last_op.take() { - let n = res?; - - // Copy the read data into the buffer and return. - buf.push_str(&inner.line); - return Poll::Ready(Ok(n)); - } else { - let mut inner = opt.take().unwrap(); - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - inner.line.clear(); - let res = inner.stdin.read_line(&mut inner.line); - inner.last_op = Some(Operation::ReadLine(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stdin is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - }) - .await - .context(|| String::from("could not read line on stdin")) - } - - /// Locks this handle to the standard input stream, returning a readable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Read trait for accessing the underlying data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let mut buffer = String::new(); - /// - /// let stdin = io::stdin(); - /// let mut handle = stdin.lock().await; - /// - /// handle.read_to_string(&mut buffer).await?; - /// # - /// # Ok(()) }) } - /// ``` - pub async fn lock(&self) -> StdinLock<'static> { - static STDIN: Lazy = Lazy::new(std::io::stdin); - - spawn_blocking(move || StdinLock(STDIN.lock())).await - } -} - -impl Read for Stdin { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::Read(res)) = inner.last_op.take() { - let n = res?; - - // If more data was read than fits into the buffer, let's retry the read - // operation. - if n <= buf.len() { - // Copy the read data into the buffer and return. - buf[..n].copy_from_slice(&inner.buf[..n]); - return Poll::Ready(Ok(n)); - } - } else { - let mut inner = opt.take().unwrap(); - - // Set the length of the inner buffer to the length of the provided buffer. - if inner.buf.len() < buf.len() { - inner.buf.reserve(buf.len() - inner.buf.len()); - } - unsafe { - inner.buf.set_len(buf.len()); - } - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - let res = std::io::Read::read(&mut inner.stdin, &mut inner.buf); - inner.last_op = Some(Operation::Read(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stdin is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - } -} - -cfg_unix! { - use crate::os::unix::io::{AsRawFd, RawFd}; - - impl AsRawFd for Stdin { - fn as_raw_fd(&self) -> RawFd { - std::io::stdin().as_raw_fd() - } - } -} - -cfg_windows! { - use crate::os::windows::io::{AsRawHandle, RawHandle}; - - impl AsRawHandle for Stdin { - fn as_raw_handle(&self) -> RawHandle { - std::io::stdin().as_raw_handle() - } - } -} - -impl Read for StdinLock<'_> { - fn poll_read( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - Poll::Ready(self.0.read(buf)) - } -} diff --git a/src/io/stdio.rs b/src/io/stdio.rs deleted file mode 100644 index 0e11e1a..0000000 --- a/src/io/stdio.rs +++ /dev/null @@ -1,21 +0,0 @@ -//! Internal types for stdio. -//! -//! This module is a port of `libstd/io/stdio.rs`,and contains internal types for `print`/`eprint`. - -use crate::io::{stderr, stdout}; -use crate::prelude::*; -use std::fmt; - -#[doc(hidden)] -pub async fn _print(args: fmt::Arguments<'_>) { - if let Err(e) = stdout().write_fmt(args).await { - panic!("failed printing to stdout: {}", e); - } -} - -#[doc(hidden)] -pub async fn _eprint(args: fmt::Arguments<'_>) { - if let Err(e) = stderr().write_fmt(args).await { - panic!("failed printing to stderr: {}", e); - } -} diff --git a/src/io/stdout.rs b/src/io/stdout.rs deleted file mode 100644 index d98ef13..0000000 --- a/src/io/stdout.rs +++ /dev/null @@ -1,251 +0,0 @@ -use std::pin::Pin; -use std::sync::Mutex; -use std::future::Future; -use std::io::Write as _; - -use crate::io::{self, Write}; -use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; - -use once_cell::sync::Lazy; - -/// Constructs a new handle to the standard output of the current process. -/// -/// This function is an async version of [`std::io::stdout`]. -/// -/// [`std::io::stdout`]: https://doc.rust-lang.org/std/io/fn.stdout.html -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// # Examples -/// -/// ```no_run -/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { -/// # -/// use async_std::io; -/// use async_std::prelude::*; -/// -/// let mut stdout = io::stdout(); -/// stdout.write_all(b"Hello, world!").await?; -/// # -/// # Ok(()) }) } -/// ``` -pub fn stdout() -> Stdout { - Stdout(Mutex::new(State::Idle(Some(Inner { - stdout: std::io::stdout(), - buf: Vec::new(), - last_op: None, - })))) -} - -/// A handle to the standard output of the current process. -/// -/// This writer is created by the [`stdout`] function. See its documentation -/// for more. -/// -/// ### Note: Windows Portability Consideration -/// -/// When operating in a console, the Windows implementation of this stream does not support -/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return -/// an error. -/// -/// [`stdout`]: fn.stdout.html -#[derive(Debug)] -pub struct Stdout(Mutex); - -/// A locked reference to the Stderr handle. -/// -/// This handle implements the [`Write`] traits, and is constructed via the [`Stdout::lock`] -/// method. -/// -/// [`Write`]: trait.Read.html -/// [`Stdout::lock`]: struct.Stdout.html#method.lock -#[derive(Debug)] -pub struct StdoutLock<'a>(std::io::StdoutLock<'a>); - -unsafe impl Send for StdoutLock<'_> {} - -/// The state of the asynchronous stdout. -/// -/// The stdout can be either idle or busy performing an asynchronous operation. -#[derive(Debug)] -enum State { - /// The stdout is idle. - Idle(Option), - - /// The stdout is blocked on an asynchronous operation. - /// - /// Awaiting this operation will result in the new state of the stdout. - Busy(JoinHandle), -} - -/// Inner representation of the asynchronous stdout. -#[derive(Debug)] -struct Inner { - /// The blocking stdout handle. - stdout: std::io::Stdout, - - /// The write buffer. - buf: Vec, - - /// The result of the last asynchronous operation on the stdout. - last_op: Option, -} - -/// Possible results of an asynchronous operation on the stdout. -#[derive(Debug)] -enum Operation { - Write(io::Result), - Flush(io::Result<()>), -} - -impl Stdout { - /// Locks this handle to the standard error stream, returning a writable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let stdout = io::stdout(); - /// let mut handle = stdout.lock().await; - /// - /// handle.write_all(b"hello world").await?; - /// # - /// # Ok(()) }) } - /// ``` - pub async fn lock(&self) -> StdoutLock<'static> { - static STDOUT: Lazy = Lazy::new(std::io::stdout); - - spawn_blocking(move || StdoutLock(STDOUT.lock())).await - } -} - -impl Write for Stdout { - fn poll_write( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::Write(res)) = inner.last_op.take() { - let n = res?; - - // If more data was written than is available in the buffer, let's retry - // the write operation. - if n <= buf.len() { - return Poll::Ready(Ok(n)); - } - } else { - let mut inner = opt.take().unwrap(); - - // Set the length of the inner buffer to the length of the provided buffer. - if inner.buf.len() < buf.len() { - inner.buf.reserve(buf.len() - inner.buf.len()); - } - unsafe { - inner.buf.set_len(buf.len()); - } - - // Copy the data to write into the inner buffer. - inner.buf[..buf.len()].copy_from_slice(buf); - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - let res = std::io::Write::write(&mut inner.stdout, &inner.buf); - inner.last_op = Some(Operation::Write(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stdout is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - } - - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let state = &mut *self.0.lock().unwrap(); - - loop { - match state { - State::Idle(opt) => { - let inner = opt.as_mut().unwrap(); - - // Check if the operation has completed. - if let Some(Operation::Flush(res)) = inner.last_op.take() { - return Poll::Ready(res); - } else { - let mut inner = opt.take().unwrap(); - - // Start the operation asynchronously. - *state = State::Busy(spawn_blocking(move || { - let res = std::io::Write::flush(&mut inner.stdout); - inner.last_op = Some(Operation::Flush(res)); - State::Idle(Some(inner)) - })); - } - } - // Poll the asynchronous operation the stdout is currently blocked on. - State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), - } - } - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} - -cfg_unix! { - use crate::os::unix::io::{AsRawFd, RawFd}; - - impl AsRawFd for Stdout { - fn as_raw_fd(&self) -> RawFd { - std::io::stdout().as_raw_fd() - } - } -} - -cfg_windows! { - use crate::os::windows::io::{AsRawHandle, RawHandle}; - - impl AsRawHandle for Stdout { - fn as_raw_handle(&self) -> RawHandle { - std::io::stdout().as_raw_handle() - } - } -} - -impl Write for StdoutLock<'_> { - fn poll_write( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - Poll::Ready(self.0.write(buf)) - } - - fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(self.0.flush()) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} diff --git a/src/io/timeout.rs b/src/io/timeout.rs index 6e22dbf..3627439 100644 --- a/src/io/timeout.rs +++ b/src/io/timeout.rs @@ -23,9 +23,9 @@ use crate::io; /// use async_std::io; /// /// io::timeout(Duration::from_secs(5), async { -/// let stdin = io::stdin(); +/// let stdin = std::io::stdin(); /// let mut line = String::new(); -/// let n = stdin.read_line(&mut line).await?; +/// let n = stdin.read_line(&mut line)?; /// Ok(()) /// }) /// .await?; From ec4b09ecd07de02dca1a2e3763477b61dfd927f0 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 5 Mar 2020 10:38:19 +0900 Subject: [PATCH 061/149] fix test code --- examples/a-chat/client.rs | 9 ++++++--- examples/print-file.rs | 7 ++++--- examples/stdin-echo.rs | 12 ++++++------ examples/stdin-timeout.rs | 4 ++-- tests/io_timeout.rs | 4 ++-- 5 files changed, 20 insertions(+), 16 deletions(-) diff --git a/examples/a-chat/client.rs b/examples/a-chat/client.rs index 48634ba..c8d5a6c 100644 --- a/examples/a-chat/client.rs +++ b/examples/a-chat/client.rs @@ -1,10 +1,12 @@ use futures::select; use futures::FutureExt; +use std::io::{self, BufReader as StdBufReader, BufRead}; use async_std::{ - io::{stdin, BufReader}, + io::{BufReader}, net::{TcpStream, ToSocketAddrs}, prelude::*, + stream, task, }; @@ -20,8 +22,9 @@ async fn try_main(addr: impl ToSocketAddrs) -> Result<()> { let reader = BufReader::new(reader); let mut lines_from_server = futures::StreamExt::fuse(reader.lines()); - let stdin = BufReader::new(stdin()); - let mut lines_from_stdin = futures::StreamExt::fuse(stdin.lines()); + let stdin = StdBufReader::new(io::stdin()); + let mut lines_from_stdin = stream::from_iter(stdin.lines()); + loop { select! { line = lines_from_server.next().fuse() => match line { diff --git a/examples/print-file.rs b/examples/print-file.rs index e2cdde7..794390b 100644 --- a/examples/print-file.rs +++ b/examples/print-file.rs @@ -1,6 +1,7 @@ //! Prints a file given as an argument to stdout. use std::env::args; +use std::io::Write; use async_std::fs::File; use async_std::io; @@ -14,7 +15,7 @@ fn main() -> io::Result<()> { task::block_on(async { let mut file = File::open(&path).await?; - let mut stdout = io::stdout(); + let mut stdout = std::io::stdout(); let mut buf = vec![0u8; LEN]; loop { @@ -23,12 +24,12 @@ fn main() -> io::Result<()> { // If this is the end of file, clean up and return. if n == 0 { - stdout.flush().await?; + stdout.flush()?; return Ok(()); } // Write the buffer into stdout. - stdout.write_all(&buf[..n]).await?; + stdout.write_all(&buf[..n])?; } }) } diff --git a/examples/stdin-echo.rs b/examples/stdin-echo.rs index 9514219..c9b571c 100644 --- a/examples/stdin-echo.rs +++ b/examples/stdin-echo.rs @@ -1,18 +1,18 @@ //! Echoes lines read on stdin to stdout. +use std::io::Write; use async_std::io; -use async_std::prelude::*; use async_std::task; fn main() -> io::Result<()> { task::block_on(async { - let stdin = io::stdin(); - let mut stdout = io::stdout(); + let stdin = std::io::stdin(); + let mut stdout = std::io::stdout(); let mut line = String::new(); loop { // Read a line from stdin. - let n = stdin.read_line(&mut line).await?; + let n = stdin.read_line(&mut line)?; // If this is the end of stdin, return. if n == 0 { @@ -20,8 +20,8 @@ fn main() -> io::Result<()> { } // Write the line to stdout. - stdout.write_all(line.as_bytes()).await?; - stdout.flush().await?; + stdout.write_all(line.as_bytes())?; + stdout.flush()?; line.clear(); } }) diff --git a/examples/stdin-timeout.rs b/examples/stdin-timeout.rs index f13c387..2bcab5e 100644 --- a/examples/stdin-timeout.rs +++ b/examples/stdin-timeout.rs @@ -8,11 +8,11 @@ use async_std::task; fn main() -> io::Result<()> { // This async scope times out after 5 seconds. task::block_on(io::timeout(Duration::from_secs(5), async { - let stdin = io::stdin(); + let stdin = std::io::stdin(); // Read a line from the standard input and display it. let mut line = String::new(); - stdin.read_line(&mut line).await?; + stdin.read_line(&mut line)?; dbg!(line); Ok(()) diff --git a/tests/io_timeout.rs b/tests/io_timeout.rs index 85a17ab..46d9d39 100644 --- a/tests/io_timeout.rs +++ b/tests/io_timeout.rs @@ -8,9 +8,9 @@ use async_std::task; fn io_timeout_timedout() { task::block_on(async { io::timeout(Duration::from_secs(1), async { - let stdin = io::stdin(); + let stdin = std::io::stdin(); let mut line = String::new(); - let _n = stdin.read_line(&mut line).await?; + let _n = stdin.read_line(&mut line)?; Ok(()) }) .await From e3bf89fc0520d529c015971fc8a4326f5acea32d Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 5 Mar 2020 18:49:58 +0900 Subject: [PATCH 062/149] $cargo fmt --- examples/a-chat/client.rs | 7 +++---- examples/stdin-echo.rs | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/examples/a-chat/client.rs b/examples/a-chat/client.rs index c8d5a6c..1964297 100644 --- a/examples/a-chat/client.rs +++ b/examples/a-chat/client.rs @@ -1,13 +1,12 @@ use futures::select; use futures::FutureExt; -use std::io::{self, BufReader as StdBufReader, BufRead}; +use std::io::{self, BufRead, BufReader as StdBufReader}; use async_std::{ - io::{BufReader}, + io::BufReader, net::{TcpStream, ToSocketAddrs}, prelude::*, - stream, - task, + stream, task, }; type Result = std::result::Result>; diff --git a/examples/stdin-echo.rs b/examples/stdin-echo.rs index c9b571c..cf0674a 100644 --- a/examples/stdin-echo.rs +++ b/examples/stdin-echo.rs @@ -1,8 +1,8 @@ //! Echoes lines read on stdin to stdout. -use std::io::Write; use async_std::io; use async_std::task; +use std::io::Write; fn main() -> io::Result<()> { task::block_on(async { From f33d7f40ab35406230262c188bd1a55b62befa61 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Fri, 6 Mar 2020 09:20:08 +0900 Subject: [PATCH 063/149] fix test --- tests/io_timeout.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/io_timeout.rs b/tests/io_timeout.rs index 46d9d39..aa464f4 100644 --- a/tests/io_timeout.rs +++ b/tests/io_timeout.rs @@ -7,10 +7,9 @@ use async_std::task; #[should_panic(expected = "timed out")] fn io_timeout_timedout() { task::block_on(async { - io::timeout(Duration::from_secs(1), async { - let stdin = std::io::stdin(); - let mut line = String::new(); - let _n = stdin.read_line(&mut line)?; + io::timeout(Duration::from_millis(100), async { + task::sleep(Duration::from_secs(1)).await; + Ok(()) }) .await From 0b0531057d023c71dcf20b475d264c244aeda581 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sun, 8 Mar 2020 20:46:26 +0900 Subject: [PATCH 064/149] 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 065/149] 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 cc19592f80dd62d7a44dd37fb6796f3b48e5dbfa Mon Sep 17 00:00:00 2001 From: nasa Date: Thu, 12 Mar 2020 18:34:09 +0900 Subject: [PATCH 066/149] Revert "Stabilize most stream method and remove unnecessary macros" --- Cargo.toml | 4 +- examples/a-chat/client.rs | 10 +- examples/print-file.rs | 7 +- examples/stdin-echo.rs | 12 +- examples/stdin-timeout.rs | 4 +- src/future/into_future.rs | 3 +- src/future/mod.rs | 2 - src/io/copy.rs | 10 +- src/io/mod.rs | 84 +++++++- src/io/stderr.rs | 261 ++++++++++++++++++++++++ src/io/stdin.rs | 279 ++++++++++++++++++++++++++ src/io/stdio.rs | 21 ++ src/io/stdout.rs | 261 ++++++++++++++++++++++++ src/io/timeout.rs | 4 +- src/lib.rs | 3 + src/macros.rs | 168 ++++++++++++++++ src/prelude.rs | 12 +- src/stream/double_ended_stream/mod.rs | 6 +- src/stream/exact_size_stream.rs | 2 + src/stream/extend.rs | 2 + src/stream/fused_stream.rs | 2 + src/stream/interval.rs | 4 + src/stream/mod.rs | 29 ++- src/stream/once.rs | 6 +- src/stream/product.rs | 13 +- src/stream/stream/count.rs | 2 + src/stream/stream/merge.rs | 2 + src/stream/stream/mod.rs | 64 +++--- src/stream/stream/timeout.rs | 2 + src/stream/stream/unzip.rs | 2 + src/stream/successors.rs | 4 + src/stream/sum.rs | 3 +- src/utils.rs | 7 +- tests/io_timeout.rs | 7 +- 34 files changed, 1192 insertions(+), 110 deletions(-) create mode 100644 src/io/stderr.rs create mode 100644 src/io/stdin.rs create mode 100644 src/io/stdio.rs create mode 100644 src/io/stdout.rs diff --git a/Cargo.toml b/Cargo.toml index c1479c2..2d4f602 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,6 +26,7 @@ default = [ "async-task", "crossbeam-channel", "crossbeam-deque", + "futures-timer", "kv-log-macro", "log", "mio", @@ -34,14 +35,13 @@ default = [ "pin-project-lite", ] docs = ["attributes", "unstable", "default"] -unstable = ["std", "broadcaster"] +unstable = ["std", "broadcaster", "futures-timer"] attributes = ["async-attributes"] std = [ "alloc", "crossbeam-utils", "futures-core/std", "futures-io", - "futures-timer", "memchr", "once_cell", "pin-utils", diff --git a/examples/a-chat/client.rs b/examples/a-chat/client.rs index 1964297..48634ba 100644 --- a/examples/a-chat/client.rs +++ b/examples/a-chat/client.rs @@ -1,12 +1,11 @@ use futures::select; use futures::FutureExt; -use std::io::{self, BufRead, BufReader as StdBufReader}; use async_std::{ - io::BufReader, + io::{stdin, BufReader}, net::{TcpStream, ToSocketAddrs}, prelude::*, - stream, task, + task, }; type Result = std::result::Result>; @@ -21,9 +20,8 @@ async fn try_main(addr: impl ToSocketAddrs) -> Result<()> { let reader = BufReader::new(reader); let mut lines_from_server = futures::StreamExt::fuse(reader.lines()); - let stdin = StdBufReader::new(io::stdin()); - let mut lines_from_stdin = stream::from_iter(stdin.lines()); - + let stdin = BufReader::new(stdin()); + let mut lines_from_stdin = futures::StreamExt::fuse(stdin.lines()); loop { select! { line = lines_from_server.next().fuse() => match line { diff --git a/examples/print-file.rs b/examples/print-file.rs index 794390b..e2cdde7 100644 --- a/examples/print-file.rs +++ b/examples/print-file.rs @@ -1,7 +1,6 @@ //! Prints a file given as an argument to stdout. use std::env::args; -use std::io::Write; use async_std::fs::File; use async_std::io; @@ -15,7 +14,7 @@ fn main() -> io::Result<()> { task::block_on(async { let mut file = File::open(&path).await?; - let mut stdout = std::io::stdout(); + let mut stdout = io::stdout(); let mut buf = vec![0u8; LEN]; loop { @@ -24,12 +23,12 @@ fn main() -> io::Result<()> { // If this is the end of file, clean up and return. if n == 0 { - stdout.flush()?; + stdout.flush().await?; return Ok(()); } // Write the buffer into stdout. - stdout.write_all(&buf[..n])?; + stdout.write_all(&buf[..n]).await?; } }) } diff --git a/examples/stdin-echo.rs b/examples/stdin-echo.rs index cf0674a..9514219 100644 --- a/examples/stdin-echo.rs +++ b/examples/stdin-echo.rs @@ -1,18 +1,18 @@ //! Echoes lines read on stdin to stdout. use async_std::io; +use async_std::prelude::*; use async_std::task; -use std::io::Write; fn main() -> io::Result<()> { task::block_on(async { - let stdin = std::io::stdin(); - let mut stdout = std::io::stdout(); + let stdin = io::stdin(); + let mut stdout = io::stdout(); let mut line = String::new(); loop { // Read a line from stdin. - let n = stdin.read_line(&mut line)?; + let n = stdin.read_line(&mut line).await?; // If this is the end of stdin, return. if n == 0 { @@ -20,8 +20,8 @@ fn main() -> io::Result<()> { } // Write the line to stdout. - stdout.write_all(line.as_bytes())?; - stdout.flush()?; + stdout.write_all(line.as_bytes()).await?; + stdout.flush().await?; line.clear(); } }) diff --git a/examples/stdin-timeout.rs b/examples/stdin-timeout.rs index 2bcab5e..f13c387 100644 --- a/examples/stdin-timeout.rs +++ b/examples/stdin-timeout.rs @@ -8,11 +8,11 @@ use async_std::task; fn main() -> io::Result<()> { // This async scope times out after 5 seconds. task::block_on(io::timeout(Duration::from_secs(5), async { - let stdin = std::io::stdin(); + let stdin = io::stdin(); // Read a line from the standard input and display it. let mut line = String::new(); - stdin.read_line(&mut line)?; + stdin.read_line(&mut line).await?; dbg!(line); Ok(()) diff --git a/src/future/into_future.rs b/src/future/into_future.rs index 473ed45..8e5e5e0 100644 --- a/src/future/into_future.rs +++ b/src/future/into_future.rs @@ -5,10 +5,9 @@ use std::future::Future; /// # Examples /// /// ``` -/// use std::pin::Pin; -/// /// use async_std::future::{Future, IntoFuture}; /// use async_std::io; +/// use async_std::pin::Pin; /// /// struct Client; /// diff --git a/src/future/mod.rs b/src/future/mod.rs index 56cd71d..9b75533 100644 --- a/src/future/mod.rs +++ b/src/future/mod.rs @@ -63,14 +63,12 @@ cfg_std! { cfg_default! { pub use timeout::{timeout, TimeoutError}; - mod timeout; } cfg_unstable! { pub use into_future::IntoFuture; pub(crate) use maybe_done::MaybeDone; - mod into_future; mod maybe_done; } diff --git a/src/io/copy.rs b/src/io/copy.rs index f946c8b..f05ed0e 100644 --- a/src/io/copy.rs +++ b/src/io/copy.rs @@ -32,14 +32,13 @@ use crate::utils::Context as _; /// /// # Examples /// -/// ```no_run +/// ``` /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { /// # /// use async_std::io; -/// use async_std::fs::File; /// /// let mut reader: &[u8] = b"hello"; -/// let mut writer = File::open("foo.txt").await?; +/// let mut writer = io::stdout(); /// /// io::copy(&mut reader, &mut writer).await?; /// # @@ -120,14 +119,13 @@ where /// /// # Examples /// -/// ```no_run +/// ``` /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { /// # /// use async_std::io; -/// use async_std::fs::File; /// /// let mut reader: &[u8] = b"hello"; -/// let mut writer = File::open("foo.txt").await?; +/// let mut writer = io::stdout(); /// /// io::copy(&mut reader, &mut writer).await?; /// # diff --git a/src/io/mod.rs b/src/io/mod.rs index 65c204c..51c473d 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -122,6 +122,56 @@ //! # Ok(()) }) } //! ``` //! +//! ## Standard input and output +//! +//! A very common source of input is standard input: +//! +//! ```no_run +//! use async_std::io; +//! +//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +//! # +//! let mut input = String::new(); +//! +//! io::stdin().read_line(&mut input).await?; +//! +//! println!("You typed: {}", input.trim()); +//! # +//! # Ok(()) }) } +//! ``` +//! +//! Note that you cannot use the [`?` operator] in functions that do not return +//! a [`Result`][`Result`]. Instead, you can call [`.unwrap()`] +//! or `match` on the return value to catch any possible errors: +//! +//! ```no_run +//! use async_std::io; +//! +//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +//! # +//! let mut input = String::new(); +//! +//! io::stdin().read_line(&mut input).await.unwrap(); +//! # +//! # Ok(()) }) } +//! ``` +//! +//! And a very common source of output is standard output: +//! +//! ```no_run +//! use async_std::io; +//! use async_std::io::prelude::*; +//! +//! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +//! # +//! io::stdout().write(&[42]).await?; +//! # +//! # Ok(()) }) } +//! ``` +//! +//! Of course, using [`io::stdout`] directly is less common than something like +//! [`println!`]. +//! //! ## Iterator types //! //! A large number of the structures provided by `std::io` are for various @@ -154,14 +204,10 @@ //! //! ```no_run //! use async_std::io; -//! use async_std::fs::File; //! //! # fn main() -> std::io::Result<()> { async_std::task::block_on(async { //! # -//! let mut reader: &[u8] = b"hello"; -//! let mut writer = File::open("foo.txt").await?; -//! -//! io::copy(&mut reader, &mut writer).await?; +//! io::copy(&mut io::stdin(), &mut io::stdout()).await?; //! # //! # Ok(()) }) } //! ``` @@ -178,14 +224,13 @@ //! ``` //! #![allow(dead_code)] //! use async_std::io; -//! use std::time::Duration; //! //! async fn read_input() -> io::Result<()> { -//! let f = io::timeout(Duration::from_secs(5), async { -//! Ok(()) -//! }); +//! let mut input = String::new(); +//! +//! io::stdin().read_line(&mut input).await?; //! -//! assert_eq!(f.await?, ()); +//! println!("You typed: {}", input.trim()); //! //! Ok(()) //! } @@ -215,6 +260,8 @@ //! [`BufReader`]: struct.BufReader.html //! [`BufWriter`]: struct.BufWriter.html //! [`Write::write`]: trait.Write.html#tymethod.write +//! [`io::stdout`]: fn.stdout.html +//! [`println!`]: ../macro.println.html //! [`Lines`]: struct.Lines.html //! [`io::Result`]: type.Result.html //! [`?` operator]: https://doc.rust-lang.org/stable/book/appendix-02-operators.html @@ -258,7 +305,24 @@ cfg_std! { } cfg_default! { + // For use in the print macros. + #[doc(hidden)] + pub use stdio::{_eprint, _print}; + + pub use stderr::{stderr, Stderr}; + pub use stdin::{stdin, Stdin}; + pub use stdout::{stdout, Stdout}; pub use timeout::timeout; mod timeout; + mod stderr; + mod stdin; + mod stdio; + mod stdout; +} + +cfg_unstable_default! { + pub use stderr::StderrLock; + pub use stdin::StdinLock; + pub use stdout::StdoutLock; } diff --git a/src/io/stderr.rs b/src/io/stderr.rs new file mode 100644 index 0000000..5ff8a02 --- /dev/null +++ b/src/io/stderr.rs @@ -0,0 +1,261 @@ +use std::pin::Pin; +use std::sync::Mutex; +use std::future::Future; + +use crate::io::{self, Write}; +use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; + +cfg_unstable! { + use once_cell::sync::Lazy; + use std::io::Write as _; +} + +/// Constructs a new handle to the standard error of the current process. +/// +/// This function is an async version of [`std::io::stderr`]. +/// +/// [`std::io::stderr`]: https://doc.rust-lang.org/std/io/fn.stderr.html +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// # Examples +/// +/// ```no_run +/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +/// # +/// use async_std::io; +/// use async_std::prelude::*; +/// +/// let mut stderr = io::stderr(); +/// stderr.write_all(b"Hello, world!").await?; +/// # +/// # Ok(()) }) } +/// ``` +pub fn stderr() -> Stderr { + Stderr(Mutex::new(State::Idle(Some(Inner { + stderr: std::io::stderr(), + buf: Vec::new(), + last_op: None, + })))) +} + +/// A handle to the standard error of the current process. +/// +/// This writer is created by the [`stderr`] function. See its documentation for +/// more. +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// [`stderr`]: fn.stderr.html +#[derive(Debug)] +pub struct Stderr(Mutex); + +/// A locked reference to the Stderr handle. +/// +/// This handle implements the [`Write`] traits, and is constructed via the [`Stderr::lock`] +/// method. +/// +/// [`Write`]: trait.Read.html +/// [`Stderr::lock`]: struct.Stderr.html#method.lock +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[derive(Debug)] +pub struct StderrLock<'a>(std::io::StderrLock<'a>); + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +unsafe impl Send for StderrLock<'_> {} + +/// The state of the asynchronous stderr. +/// +/// The stderr can be either idle or busy performing an asynchronous operation. +#[derive(Debug)] +enum State { + /// The stderr is idle. + Idle(Option), + + /// The stderr is blocked on an asynchronous operation. + /// + /// Awaiting this operation will result in the new state of the stderr. + Busy(JoinHandle), +} + +/// Inner representation of the asynchronous stderr. +#[derive(Debug)] +struct Inner { + /// The blocking stderr handle. + stderr: std::io::Stderr, + + /// The write buffer. + buf: Vec, + + /// The result of the last asynchronous operation on the stderr. + last_op: Option, +} + +/// Possible results of an asynchronous operation on the stderr. +#[derive(Debug)] +enum Operation { + Write(io::Result), + Flush(io::Result<()>), +} + +impl Stderr { + /// Locks this handle to the standard error stream, returning a writable guard. + /// + /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. + /// + /// # Examples + /// + /// ```no_run + /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { + /// # + /// use async_std::io; + /// use async_std::prelude::*; + /// + /// let stderr = io::stderr(); + /// let mut handle = stderr.lock().await; + /// + /// handle.write_all(b"hello world").await?; + /// # + /// # Ok(()) }) } + /// ``` + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] + #[cfg(any(feature = "unstable", feature = "docs"))] + pub async fn lock(&self) -> StderrLock<'static> { + static STDERR: Lazy = Lazy::new(std::io::stderr); + + spawn_blocking(move || StderrLock(STDERR.lock())).await + } +} + +impl Write for Stderr { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::Write(res)) = inner.last_op.take() { + let n = res?; + + // If more data was written than is available in the buffer, let's retry + // the write operation. + if n <= buf.len() { + return Poll::Ready(Ok(n)); + } + } else { + let mut inner = opt.take().unwrap(); + + // Set the length of the inner buffer to the length of the provided buffer. + if inner.buf.len() < buf.len() { + inner.buf.reserve(buf.len() - inner.buf.len()); + } + unsafe { + inner.buf.set_len(buf.len()); + } + + // Copy the data to write into the inner buffer. + inner.buf[..buf.len()].copy_from_slice(buf); + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + let res = std::io::Write::write(&mut inner.stderr, &inner.buf); + inner.last_op = Some(Operation::Write(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stderr is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::Flush(res)) = inner.last_op.take() { + return Poll::Ready(res); + } else { + let mut inner = opt.take().unwrap(); + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + let res = std::io::Write::flush(&mut inner.stderr); + inner.last_op = Some(Operation::Flush(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stderr is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} + +cfg_unix! { + use crate::os::unix::io::{AsRawFd, RawFd}; + + impl AsRawFd for Stderr { + fn as_raw_fd(&self) -> RawFd { + std::io::stderr().as_raw_fd() + } + } +} + +cfg_windows! { + use crate::os::windows::io::{AsRawHandle, RawHandle}; + + impl AsRawHandle for Stderr { + fn as_raw_handle(&self) -> RawHandle { + std::io::stderr().as_raw_handle() + } + } +} + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +impl io::Write for StderrLock<'_> { + fn poll_write( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + Poll::Ready(self.0.write(buf)) + } + + fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + Poll::Ready(self.0.flush()) + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} diff --git a/src/io/stdin.rs b/src/io/stdin.rs new file mode 100644 index 0000000..369ccae --- /dev/null +++ b/src/io/stdin.rs @@ -0,0 +1,279 @@ +use std::future::Future; +use std::pin::Pin; +use std::sync::Mutex; + +use crate::future; +use crate::io::{self, Read}; +use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; +use crate::utils::Context as _; + +cfg_unstable! { + use once_cell::sync::Lazy; + use std::io::Read as _; +} + +/// Constructs a new handle to the standard input of the current process. +/// +/// This function is an async version of [`std::io::stdin`]. +/// +/// [`std::io::stdin`]: https://doc.rust-lang.org/std/io/fn.stdin.html +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// # Examples +/// +/// ```no_run +/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +/// # +/// use async_std::io; +/// +/// let stdin = io::stdin(); +/// let mut line = String::new(); +/// stdin.read_line(&mut line).await?; +/// # +/// # Ok(()) }) } +/// ``` +pub fn stdin() -> Stdin { + Stdin(Mutex::new(State::Idle(Some(Inner { + stdin: std::io::stdin(), + line: String::new(), + buf: Vec::new(), + last_op: None, + })))) +} + +/// A handle to the standard input of the current process. +/// +/// This reader is created by the [`stdin`] function. See its documentation for +/// more. +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// [`stdin`]: fn.stdin.html +#[derive(Debug)] +pub struct Stdin(Mutex); + +/// A locked reference to the Stdin handle. +/// +/// This handle implements the [`Read`] traits, and is constructed via the [`Stdin::lock`] method. +/// +/// [`Read`]: trait.Read.html +/// [`Stdin::lock`]: struct.Stdin.html#method.lock +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[cfg(feature = "unstable")] +#[derive(Debug)] +pub struct StdinLock<'a>(std::io::StdinLock<'a>); + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +unsafe impl Send for StdinLock<'_> {} + +/// The state of the asynchronous stdin. +/// +/// The stdin can be either idle or busy performing an asynchronous operation. +#[derive(Debug)] +enum State { + /// The stdin is idle. + Idle(Option), + + /// The stdin is blocked on an asynchronous operation. + /// + /// Awaiting this operation will result in the new state of the stdin. + Busy(JoinHandle), +} + +/// Inner representation of the asynchronous stdin. +#[derive(Debug)] +struct Inner { + /// The blocking stdin handle. + stdin: std::io::Stdin, + + /// The line buffer. + line: String, + + /// The write buffer. + buf: Vec, + + /// The result of the last asynchronous operation on the stdin. + last_op: Option, +} + +/// Possible results of an asynchronous operation on the stdin. +#[derive(Debug)] +enum Operation { + ReadLine(io::Result), + Read(io::Result), +} + +impl Stdin { + /// Reads a line of input into the specified buffer. + /// + /// # Examples + /// + /// ```no_run + /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { + /// # + /// use async_std::io; + /// + /// let stdin = io::stdin(); + /// let mut line = String::new(); + /// stdin.read_line(&mut line).await?; + /// # + /// # Ok(()) }) } + /// ``` + pub async fn read_line(&self, buf: &mut String) -> io::Result { + future::poll_fn(|cx| { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::ReadLine(res)) = inner.last_op.take() { + let n = res?; + + // Copy the read data into the buffer and return. + buf.push_str(&inner.line); + return Poll::Ready(Ok(n)); + } else { + let mut inner = opt.take().unwrap(); + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + inner.line.clear(); + let res = inner.stdin.read_line(&mut inner.line); + inner.last_op = Some(Operation::ReadLine(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stdin is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + }) + .await + .context(|| String::from("could not read line on stdin")) + } + + /// Locks this handle to the standard input stream, returning a readable guard. + /// + /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Read trait for accessing the underlying data. + /// + /// # Examples + /// + /// ```no_run + /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { + /// # + /// use async_std::io; + /// use async_std::prelude::*; + /// + /// let mut buffer = String::new(); + /// + /// let stdin = io::stdin(); + /// let mut handle = stdin.lock().await; + /// + /// handle.read_to_string(&mut buffer).await?; + /// # + /// # Ok(()) }) } + /// ``` + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] + #[cfg(any(feature = "unstable", feature = "docs"))] + pub async fn lock(&self) -> StdinLock<'static> { + static STDIN: Lazy = Lazy::new(std::io::stdin); + + spawn_blocking(move || StdinLock(STDIN.lock())).await + } +} + +impl Read for Stdin { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::Read(res)) = inner.last_op.take() { + let n = res?; + + // If more data was read than fits into the buffer, let's retry the read + // operation. + if n <= buf.len() { + // Copy the read data into the buffer and return. + buf[..n].copy_from_slice(&inner.buf[..n]); + return Poll::Ready(Ok(n)); + } + } else { + let mut inner = opt.take().unwrap(); + + // Set the length of the inner buffer to the length of the provided buffer. + if inner.buf.len() < buf.len() { + inner.buf.reserve(buf.len() - inner.buf.len()); + } + unsafe { + inner.buf.set_len(buf.len()); + } + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + let res = std::io::Read::read(&mut inner.stdin, &mut inner.buf); + inner.last_op = Some(Operation::Read(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stdin is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + } +} + +cfg_unix! { + use crate::os::unix::io::{AsRawFd, RawFd}; + + impl AsRawFd for Stdin { + fn as_raw_fd(&self) -> RawFd { + std::io::stdin().as_raw_fd() + } + } +} + +cfg_windows! { + use crate::os::windows::io::{AsRawHandle, RawHandle}; + + impl AsRawHandle for Stdin { + fn as_raw_handle(&self) -> RawHandle { + std::io::stdin().as_raw_handle() + } + } +} + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +impl Read for StdinLock<'_> { + fn poll_read( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + Poll::Ready(self.0.read(buf)) + } +} diff --git a/src/io/stdio.rs b/src/io/stdio.rs new file mode 100644 index 0000000..0e11e1a --- /dev/null +++ b/src/io/stdio.rs @@ -0,0 +1,21 @@ +//! Internal types for stdio. +//! +//! This module is a port of `libstd/io/stdio.rs`,and contains internal types for `print`/`eprint`. + +use crate::io::{stderr, stdout}; +use crate::prelude::*; +use std::fmt; + +#[doc(hidden)] +pub async fn _print(args: fmt::Arguments<'_>) { + if let Err(e) = stdout().write_fmt(args).await { + panic!("failed printing to stdout: {}", e); + } +} + +#[doc(hidden)] +pub async fn _eprint(args: fmt::Arguments<'_>) { + if let Err(e) = stderr().write_fmt(args).await { + panic!("failed printing to stderr: {}", e); + } +} diff --git a/src/io/stdout.rs b/src/io/stdout.rs new file mode 100644 index 0000000..1711c09 --- /dev/null +++ b/src/io/stdout.rs @@ -0,0 +1,261 @@ +use std::pin::Pin; +use std::sync::Mutex; +use std::future::Future; + +use crate::io::{self, Write}; +use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; + +cfg_unstable! { + use once_cell::sync::Lazy; + use std::io::Write as _; +} + +/// Constructs a new handle to the standard output of the current process. +/// +/// This function is an async version of [`std::io::stdout`]. +/// +/// [`std::io::stdout`]: https://doc.rust-lang.org/std/io/fn.stdout.html +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// # Examples +/// +/// ```no_run +/// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { +/// # +/// use async_std::io; +/// use async_std::prelude::*; +/// +/// let mut stdout = io::stdout(); +/// stdout.write_all(b"Hello, world!").await?; +/// # +/// # Ok(()) }) } +/// ``` +pub fn stdout() -> Stdout { + Stdout(Mutex::new(State::Idle(Some(Inner { + stdout: std::io::stdout(), + buf: Vec::new(), + last_op: None, + })))) +} + +/// A handle to the standard output of the current process. +/// +/// This writer is created by the [`stdout`] function. See its documentation +/// for more. +/// +/// ### Note: Windows Portability Consideration +/// +/// When operating in a console, the Windows implementation of this stream does not support +/// non-UTF-8 byte sequences. Attempting to write bytes that are not valid UTF-8 will return +/// an error. +/// +/// [`stdout`]: fn.stdout.html +#[derive(Debug)] +pub struct Stdout(Mutex); + +/// A locked reference to the Stderr handle. +/// +/// This handle implements the [`Write`] traits, and is constructed via the [`Stdout::lock`] +/// method. +/// +/// [`Write`]: trait.Read.html +/// [`Stdout::lock`]: struct.Stdout.html#method.lock +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[derive(Debug)] +pub struct StdoutLock<'a>(std::io::StdoutLock<'a>); + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +unsafe impl Send for StdoutLock<'_> {} + +/// The state of the asynchronous stdout. +/// +/// The stdout can be either idle or busy performing an asynchronous operation. +#[derive(Debug)] +enum State { + /// The stdout is idle. + Idle(Option), + + /// The stdout is blocked on an asynchronous operation. + /// + /// Awaiting this operation will result in the new state of the stdout. + Busy(JoinHandle), +} + +/// Inner representation of the asynchronous stdout. +#[derive(Debug)] +struct Inner { + /// The blocking stdout handle. + stdout: std::io::Stdout, + + /// The write buffer. + buf: Vec, + + /// The result of the last asynchronous operation on the stdout. + last_op: Option, +} + +/// Possible results of an asynchronous operation on the stdout. +#[derive(Debug)] +enum Operation { + Write(io::Result), + Flush(io::Result<()>), +} + +impl Stdout { + /// Locks this handle to the standard error stream, returning a writable guard. + /// + /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. + /// + /// # Examples + /// + /// ```no_run + /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { + /// # + /// use async_std::io; + /// use async_std::prelude::*; + /// + /// let stdout = io::stdout(); + /// let mut handle = stdout.lock().await; + /// + /// handle.write_all(b"hello world").await?; + /// # + /// # Ok(()) }) } + /// ``` + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] + #[cfg(any(feature = "unstable", feature = "docs"))] + pub async fn lock(&self) -> StdoutLock<'static> { + static STDOUT: Lazy = Lazy::new(std::io::stdout); + + spawn_blocking(move || StdoutLock(STDOUT.lock())).await + } +} + +impl Write for Stdout { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::Write(res)) = inner.last_op.take() { + let n = res?; + + // If more data was written than is available in the buffer, let's retry + // the write operation. + if n <= buf.len() { + return Poll::Ready(Ok(n)); + } + } else { + let mut inner = opt.take().unwrap(); + + // Set the length of the inner buffer to the length of the provided buffer. + if inner.buf.len() < buf.len() { + inner.buf.reserve(buf.len() - inner.buf.len()); + } + unsafe { + inner.buf.set_len(buf.len()); + } + + // Copy the data to write into the inner buffer. + inner.buf[..buf.len()].copy_from_slice(buf); + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + let res = std::io::Write::write(&mut inner.stdout, &inner.buf); + inner.last_op = Some(Operation::Write(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stdout is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let state = &mut *self.0.lock().unwrap(); + + loop { + match state { + State::Idle(opt) => { + let inner = opt.as_mut().unwrap(); + + // Check if the operation has completed. + if let Some(Operation::Flush(res)) = inner.last_op.take() { + return Poll::Ready(res); + } else { + let mut inner = opt.take().unwrap(); + + // Start the operation asynchronously. + *state = State::Busy(spawn_blocking(move || { + let res = std::io::Write::flush(&mut inner.stdout); + inner.last_op = Some(Operation::Flush(res)); + State::Idle(Some(inner)) + })); + } + } + // Poll the asynchronous operation the stdout is currently blocked on. + State::Busy(task) => *state = futures_core::ready!(Pin::new(task).poll(cx)), + } + } + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} + +cfg_unix! { + use crate::os::unix::io::{AsRawFd, RawFd}; + + impl AsRawFd for Stdout { + fn as_raw_fd(&self) -> RawFd { + std::io::stdout().as_raw_fd() + } + } +} + +cfg_windows! { + use crate::os::windows::io::{AsRawHandle, RawHandle}; + + impl AsRawHandle for Stdout { + fn as_raw_handle(&self) -> RawHandle { + std::io::stdout().as_raw_handle() + } + } +} + +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +impl Write for StdoutLock<'_> { + fn poll_write( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + Poll::Ready(self.0.write(buf)) + } + + fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + Poll::Ready(self.0.flush()) + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} diff --git a/src/io/timeout.rs b/src/io/timeout.rs index 3627439..6e22dbf 100644 --- a/src/io/timeout.rs +++ b/src/io/timeout.rs @@ -23,9 +23,9 @@ use crate::io; /// use async_std::io; /// /// io::timeout(Duration::from_secs(5), async { -/// let stdin = std::io::stdin(); +/// let stdin = io::stdin(); /// let mut line = String::new(); -/// let n = stdin.read_line(&mut line)?; +/// let n = stdin.read_line(&mut line).await?; /// Ok(()) /// }) /// .await?; diff --git a/src/lib.rs b/src/lib.rs index 8cd0d30..d498792 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -273,6 +273,9 @@ cfg_default! { } cfg_unstable! { + pub mod pin; + pub mod process; + mod unit; mod vec; mod result; diff --git a/src/macros.rs b/src/macros.rs index 22cd00d..638a234 100644 --- a/src/macros.rs +++ b/src/macros.rs @@ -1,3 +1,171 @@ +/// Prints to the standard output. +/// +/// Equivalent to the [`println!`] macro except that a newline is not printed at +/// the end of the message. +/// +/// Note that stdout is frequently line-buffered by default so it may be +/// necessary to use [`io::stdout().flush()`][flush] to ensure the output is emitted +/// immediately. +/// +/// Use `print!` only for the primary output of your program. Use +/// [`eprint!`] instead to print error and progress messages. +/// +/// [`println!`]: macro.println.html +/// [flush]: io/trait.Write.html#tymethod.flush +/// [`eprint!`]: macro.eprint.html +/// +/// # Panics +/// +/// Panics if writing to `io::stdout()` fails. +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::io; +/// use async_std::prelude::*; +/// use async_std::print; +/// +/// print!("this ").await; +/// print!("will ").await; +/// print!("be ").await; +/// print!("on ").await; +/// print!("the ").await; +/// print!("same ").await; +/// print!("line ").await; +/// +/// io::stdout().flush().await.unwrap(); +/// +/// print!("this string has a newline, why not choose println! instead?\n").await; +/// +/// io::stdout().flush().await.unwrap(); +/// # +/// # }) +/// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[macro_export] +macro_rules! print { + ($($arg:tt)*) => ($crate::io::_print(format_args!($($arg)*))) +} + +/// Prints to the standard output, with a newline. +/// +/// On all platforms, the newline is the LINE FEED character (`\n`/`U+000A`) alone +/// (no additional CARRIAGE RETURN (`\r`/`U+000D`)). +/// +/// Use the [`format!`] syntax to write data to the standard output. +/// See [`std::fmt`] for more information. +/// +/// Use `println!` only for the primary output of your program. Use +/// [`eprintln!`] instead to print error and progress messages. +/// +/// [`format!`]: macro.format.html +/// [`std::fmt`]: https://doc.rust-lang.org/std/fmt/index.html +/// [`eprintln!`]: macro.eprintln.html +/// # Panics +/// +/// Panics if writing to `io::stdout` fails. +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::println; +/// +/// println!().await; // prints just a newline +/// println!("hello there!").await; +/// println!("format {} arguments", "some").await; +/// # +/// # }) +/// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[macro_export] +macro_rules! println { + () => ($crate::print!("\n")); + ($($arg:tt)*) => (async { + $crate::io::_print(format_args!($($arg)*)).await; + $crate::io::_print(format_args!("\n")).await; + }) +} + +/// Prints to the standard error. +/// +/// Equivalent to the [`print!`] macro, except that output goes to +/// [`io::stderr`] instead of `io::stdout`. See [`print!`] for +/// example usage. +/// +/// Use `eprint!` only for error and progress messages. Use `print!` +/// instead for the primary output of your program. +/// +/// [`io::stderr`]: io/struct.Stderr.html +/// [`print!`]: macro.print.html +/// +/// # Panics +/// +/// Panics if writing to `io::stderr` fails. +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::eprint; +/// +/// eprint!("Error: Could not complete task").await; +/// # +/// # }) +/// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[macro_export] +macro_rules! eprint { + ($($arg:tt)*) => ($crate::io::_eprint(format_args!($($arg)*))) +} + +/// Prints to the standard error, with a newline. +/// +/// Equivalent to the [`println!`] macro, except that output goes to +/// [`io::stderr`] instead of `io::stdout`. See [`println!`] for +/// example usage. +/// +/// Use `eprintln!` only for error and progress messages. Use `println!` +/// instead for the primary output of your program. +/// +/// [`io::stderr`]: io/struct.Stderr.html +/// [`println!`]: macro.println.html +/// +/// # Panics +/// +/// Panics if writing to `io::stderr` fails. +/// +/// # Examples +/// +/// ``` +/// # async_std::task::block_on(async { +/// # +/// use async_std::eprintln; +/// +/// eprintln!("Error: Could not complete task").await; +/// # +/// # }) +/// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[macro_export] +macro_rules! eprintln { + () => (async { $crate::eprint!("\n").await; }); + ($($arg:tt)*) => ( + async { + $crate::io::_eprint(format_args!($($arg)*)).await; + $crate::io::_eprint(format_args!("\n")).await; + } + ); +} + /// Declares task-local values. /// /// The macro wraps any number of static declarations and makes them task-local. Attributes and diff --git a/src/prelude.rs b/src/prelude.rs index a522745..a2a14a1 100644 --- a/src/prelude.rs +++ b/src/prelude.rs @@ -38,14 +38,16 @@ cfg_std! { pub use crate::io::prelude::SeekExt as _; #[doc(no_inline)] pub use crate::io::prelude::WriteExt as _; - - #[doc(no_inline)] - pub use crate::stream::DoubleEndedStream; - #[doc(no_inline)] - pub use crate::stream::ExactSizeStream; } cfg_default! { #[doc(no_inline)] pub use crate::task_local; } + +cfg_unstable! { + #[doc(no_inline)] + pub use crate::stream::DoubleEndedStream; + #[doc(no_inline)] + pub use crate::stream::ExactSizeStream; +} diff --git a/src/stream/double_ended_stream/mod.rs b/src/stream/double_ended_stream/mod.rs index 1563e1b..a177865 100644 --- a/src/stream/double_ended_stream/mod.rs +++ b/src/stream/double_ended_stream/mod.rs @@ -1,7 +1,7 @@ use crate::stream::Stream; -use core::pin::Pin; -use core::task::{Context, Poll}; +use std::pin::Pin; +use std::task::{Context, Poll}; mod next_back; mod nth_back; @@ -22,6 +22,8 @@ use try_rfold::TryRFoldFuture; /// `Item`s from the back, as well as the front. /// /// [`Stream`]: trait.Stream.html +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait DoubleEndedStream: Stream { #[doc = r#" Attempts to receive the next item from the back of the stream. diff --git a/src/stream/exact_size_stream.rs b/src/stream/exact_size_stream.rs index 28792c6..8b6ba97 100644 --- a/src/stream/exact_size_stream.rs +++ b/src/stream/exact_size_stream.rs @@ -76,6 +76,8 @@ pub use crate::stream::Stream; /// # }); /// ``` #[allow(clippy::len_without_is_empty)] // ExactSizeIterator::is_empty is unstable +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait ExactSizeStream: Stream { /// Returns the exact number of times the stream will iterate. /// diff --git a/src/stream/extend.rs b/src/stream/extend.rs index 9f54007..702cbca 100644 --- a/src/stream/extend.rs +++ b/src/stream/extend.rs @@ -27,6 +27,8 @@ use crate::stream::IntoStream; /// # /// # }) /// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Extend { /// Extends a collection with the contents of a stream. fn extend<'a, T: IntoStream + 'a>( diff --git a/src/stream/fused_stream.rs b/src/stream/fused_stream.rs index 5d02f1d..e14ab5b 100644 --- a/src/stream/fused_stream.rs +++ b/src/stream/fused_stream.rs @@ -14,6 +14,8 @@ use crate::stream::Stream; /// [`None`]: https://doc.rust-lang.org/std/option/enum.Option.html#variant.None /// [`Stream::fuse`]: trait.Stream.html#method.fuse /// [`Fuse`]: struct.Fuse.html +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait FusedStream: Stream {} impl FusedStream for &mut S {} diff --git a/src/stream/interval.rs b/src/stream/interval.rs index 8dd6b5e..7a0c174 100644 --- a/src/stream/interval.rs +++ b/src/stream/interval.rs @@ -41,6 +41,8 @@ use futures_timer::Delay; /// # /// # Ok(()) }) } /// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub fn interval(dur: Duration) -> Interval { Interval { delay: Delay::new(dur), @@ -54,6 +56,8 @@ pub fn interval(dur: Duration) -> Interval { /// documentation for more. /// /// [`interval`]: fn.interval.html +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Interval { delay: Delay, diff --git a/src/stream/mod.rs b/src/stream/mod.rs index f1c5fdf..0bfd4e8 100644 --- a/src/stream/mod.rs +++ b/src/stream/mod.rs @@ -317,32 +317,29 @@ mod once; mod repeat; mod repeat_with; -cfg_std! { - pub use double_ended_stream::DoubleEndedStream; - pub use exact_size_stream::ExactSizeStream; - pub use fused_stream::FusedStream; - pub use interval::{interval, Interval}; - pub use pending::{pending, Pending}; - pub use product::Product; - pub use successors::{successors, Successors}; - pub use sum::Sum; - +cfg_unstable! { mod double_ended_stream; mod exact_size_stream; + mod extend; + mod from_stream; mod fused_stream; mod interval; + mod into_stream; mod pending; mod product; mod successors; mod sum; -} - -cfg_unstable! { - mod from_stream; - mod into_stream; - mod extend; + pub use double_ended_stream::DoubleEndedStream; + pub use exact_size_stream::ExactSizeStream; pub use extend::{extend, Extend}; pub use from_stream::FromStream; + pub use fused_stream::FusedStream; + pub use interval::{interval, Interval}; pub use into_stream::IntoStream; + pub use pending::{pending, Pending}; + pub use product::Product; + pub use stream::Merge; + pub use successors::{successors, Successors}; + pub use sum::Sum; } diff --git a/src/stream/once.rs b/src/stream/once.rs index 9daeac5..b86f181 100644 --- a/src/stream/once.rs +++ b/src/stream/once.rs @@ -5,7 +5,7 @@ use pin_project_lite::pin_project; use crate::stream::Stream; use crate::task::{Context, Poll}; -#[cfg(feature = "std")] +#[cfg(feature = "unstable")] use crate::stream::DoubleEndedStream; /// Creates a stream that yields a single item. @@ -50,8 +50,8 @@ impl Stream for Once { } } -#[cfg(feature = "std")] -impl DoubleEndedStream for Once { +#[cfg(feature = "unstable")] +impl DoubleEndedStream for Once { fn poll_next_back(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(self.project().value.take()) } diff --git a/src/stream/product.rs b/src/stream/product.rs index 9794846..15497e8 100644 --- a/src/stream/product.rs +++ b/src/stream/product.rs @@ -1,6 +1,5 @@ -use alloc::boxed::Box; -use core::future::Future; use core::pin::Pin; +use core::future::Future; use crate::stream::Stream; @@ -14,6 +13,8 @@ use crate::stream::Stream; /// [`product`]: trait.Product.html#tymethod.product /// [`FromStream`]: trait.FromStream.html /// [`Stream::product`]: trait.Stream.html#method.product +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Product: Sized { /// Method which takes a stream and generates `Self` from the elements by /// multiplying the items. @@ -22,9 +23,9 @@ pub trait Product: Sized { S: Stream + 'a; } -use crate::stream::stream::StreamExt; -use core::num::Wrapping; use core::ops::Mul; +use core::num::Wrapping; +use crate::stream::stream::StreamExt; macro_rules! integer_product { (@impls $one: expr, $($a:ty)*) => ($( @@ -74,5 +75,5 @@ macro_rules! float_product { ); } -integer_product! { i8 i16 i32 i64 i128 isize u8 u16 u32 u64 u128 usize } -float_product! { f32 f64 } +integer_product!{ i8 i16 i32 i64 i128 isize u8 u16 u32 u64 u128 usize } +float_product!{ f32 f64 } diff --git a/src/stream/stream/count.rs b/src/stream/stream/count.rs index ae6c78c..63e0449 100644 --- a/src/stream/stream/count.rs +++ b/src/stream/stream/count.rs @@ -9,6 +9,8 @@ use crate::task::{Context, Poll}; pin_project! { #[doc(hidden)] #[allow(missing_debug_implementations)] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub struct CountFuture { #[pin] stream: S, diff --git a/src/stream/stream/merge.rs b/src/stream/stream/merge.rs index d1eea9d..2320972 100644 --- a/src/stream/stream/merge.rs +++ b/src/stream/stream/merge.rs @@ -16,6 +16,8 @@ pin_project! { /// /// [`merge`]: trait.Stream.html#method.merge /// [`Stream`]: trait.Stream.html + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Merge { #[pin] diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index 5cdf530..d0cc718 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -112,43 +112,35 @@ pub use zip::Zip; use core::cmp::Ordering; -cfg_std! { - use core::time::Duration; - use crate::stream::{Product, Sum}; - use alloc::boxed::Box; +cfg_unstable! { use core::future::Future; use core::pin::Pin; + use core::time::Duration; - use unzip::UnzipFuture; - use count::CountFuture; - - pub use throttle::Throttle; - pub use merge::Merge; - pub use delay::Delay; - pub use timeout::{Timeout, TimeoutError}; - - mod timeout; - mod throttle; - mod merge; - mod delay; - mod unzip; - mod count; -} - -cfg_unstable! { - use crate::stream::FromStream; use crate::stream::into_stream::IntoStream; + use crate::stream::{FromStream, Product, Sum}; use crate::stream::Extend; + use count::CountFuture; use partition::PartitionFuture; + use unzip::UnzipFuture; + pub use merge::Merge; pub use flatten::Flatten; pub use flat_map::FlatMap; + pub use timeout::{TimeoutError, Timeout}; + pub use throttle::Throttle; + pub use delay::Delay; - + mod count; + mod merge; mod flatten; mod flat_map; mod partition; + mod timeout; + mod throttle; + mod delay; + mod unzip; } extension_trait! { @@ -363,7 +355,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn throttle(self, d: Duration) -> Throttle where Self: Sized, @@ -605,7 +598,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(any(feature = "unstable", feature = "docs"))] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn delay(self, dur: std::time::Duration) -> Delay where Self: Sized, @@ -1517,6 +1511,8 @@ extension_trait! { # }) } ``` "#] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn by_ref(&mut self) -> &mut Self { self } @@ -1660,7 +1656,8 @@ extension_trait! { # Ok(()) }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(any(feature = "unstable", feature = "docs"))] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn timeout(self, dur: Duration) -> Timeout where Self: Stream + Sized, @@ -1825,7 +1822,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn unzip(self) -> impl Future [UnzipFuture] where FromA: Default + Extend, @@ -1923,7 +1921,8 @@ extension_trait! { # }); ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn merge(self, other: U) -> Merge where Self: Sized, @@ -2069,7 +2068,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn count(self) -> impl Future [CountFuture] where Self: Sized, @@ -2330,7 +2330,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn sum<'a, S>( self, ) -> impl Future + 'a [Pin + 'a>>] @@ -2375,7 +2376,8 @@ extension_trait! { # }) } ``` "#] - #[cfg(feature = "std")] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] fn product<'a, P>( self, ) -> impl Future + 'a [Pin + 'a>>] diff --git a/src/stream/stream/timeout.rs b/src/stream/stream/timeout.rs index 411be7e..ce658c8 100644 --- a/src/stream/stream/timeout.rs +++ b/src/stream/stream/timeout.rs @@ -47,6 +47,8 @@ impl Stream for Timeout { } /// An error returned when a stream times out. +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[cfg(any(feature = "unstable", feature = "docs"))] #[derive(Clone, Copy, Debug, Eq, PartialEq)] pub struct TimeoutError { _private: (), diff --git a/src/stream/stream/unzip.rs b/src/stream/stream/unzip.rs index 94cbc0a..7771509 100644 --- a/src/stream/stream/unzip.rs +++ b/src/stream/stream/unzip.rs @@ -8,6 +8,8 @@ use crate::task::{Context, Poll}; pin_project! { #[derive(Clone, Debug)] + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub struct UnzipFuture { #[pin] stream: S, diff --git a/src/stream/successors.rs b/src/stream/successors.rs index fb450c6..a9ce40f 100644 --- a/src/stream/successors.rs +++ b/src/stream/successors.rs @@ -27,6 +27,8 @@ use pin_project_lite::pin_project; /// # /// # }) } /// ``` +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub fn successors(first: Option, succ: F) -> Successors where F: FnMut(&T) -> Option, @@ -41,6 +43,8 @@ pin_project! { /// This stream is constructed by [`successors`] function /// /// [`successors`]: fn.succssors.html + #[cfg(feature = "unstable")] + #[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub struct Successors where diff --git a/src/stream/sum.rs b/src/stream/sum.rs index dc41a0f..3b3144e 100644 --- a/src/stream/sum.rs +++ b/src/stream/sum.rs @@ -1,4 +1,3 @@ -use alloc::boxed::Box; use core::future::Future; use core::pin::Pin; @@ -14,6 +13,8 @@ use crate::stream::Stream; /// [`sum`]: trait.Sum.html#tymethod.sum /// [`FromStream`]: trait.FromStream.html /// [`Stream::sum`]: trait.Stream.html#method.sum +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub trait Sum: Sized { /// Method which takes a stream and generates `Self` from the elements by /// "summing up" the items. diff --git a/src/utils.rs b/src/utils.rs index a930a84..f18b74d 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -21,7 +21,7 @@ pub fn abort_on_panic(f: impl FnOnce() -> T) -> T { } /// Generates a random number in `0..n`. -#[cfg(any(feature = "std", feature = "default"))] +#[cfg(any(feature = "unstable", feature = "default"))] pub fn random(n: u32) -> u32 { use std::cell::Cell; use std::num::Wrapping; @@ -257,6 +257,11 @@ macro_rules! extension_trait { $(#[cfg(feature = "docs")] $imp)* }; + // Optimization: expand `$head` eagerly before starting a new method definition. + (@ext ($($head:tt)*) #[doc = $d:literal] $($tail:tt)*) => { + $($head)* extension_trait!(@ext (#[doc = $d]) $($tail)*); + }; + // Parse the return type in an extension method. (@doc ($($head:tt)*) -> impl Future $(+ $lt:lifetime)? [$f:ty] $($tail:tt)*) => { extension_trait!(@doc ($($head)* -> owned::ImplFuture<$out>) $($tail)*); diff --git a/tests/io_timeout.rs b/tests/io_timeout.rs index aa464f4..85a17ab 100644 --- a/tests/io_timeout.rs +++ b/tests/io_timeout.rs @@ -7,9 +7,10 @@ use async_std::task; #[should_panic(expected = "timed out")] fn io_timeout_timedout() { task::block_on(async { - io::timeout(Duration::from_millis(100), async { - task::sleep(Duration::from_secs(1)).await; - + io::timeout(Duration::from_secs(1), async { + let stdin = io::stdin(); + let mut line = String::new(); + let _n = stdin.read_line(&mut line).await?; Ok(()) }) .await From 8931d1464e6bdfbb8d6cd0ec7e066de9daaa2423 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Sat, 14 Mar 2020 22:46:22 +0900 Subject: [PATCH 067/149] fix ci --- src/utils.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/utils.rs b/src/utils.rs index f18b74d..4bdbd92 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -257,11 +257,6 @@ macro_rules! extension_trait { $(#[cfg(feature = "docs")] $imp)* }; - // Optimization: expand `$head` eagerly before starting a new method definition. - (@ext ($($head:tt)*) #[doc = $d:literal] $($tail:tt)*) => { - $($head)* extension_trait!(@ext (#[doc = $d]) $($tail)*); - }; - // Parse the return type in an extension method. (@doc ($($head:tt)*) -> impl Future $(+ $lt:lifetime)? [$f:ty] $($tail:tt)*) => { extension_trait!(@doc ($($head)* -> owned::ImplFuture<$out>) $($tail)*); From bb11c676a1e4884d564c99fb333586a6ca38ef67 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Sun, 15 Mar 2020 23:46:36 +0100 Subject: [PATCH 068/149] doctests pass --- src/sync/channel.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index b42326b..a957bec 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -41,7 +41,7 @@ use crate::sync::WakerSet; /// let (s, r) = channel(1); /// /// // This call returns immediately because there is enough space in the channel. -/// s.send(1).await; +/// s.send(1usize).await; /// /// task::spawn(async move { /// // This call will have to wait because the channel is full. @@ -323,7 +323,7 @@ impl fmt::Debug for Sender { /// let (s, r) = channel(100); /// /// task::spawn(async move { -/// s.send(1).await; +/// s.send(1usize).await; /// task::sleep(Duration::from_secs(1)).await; /// s.send(2).await; /// }); @@ -346,7 +346,7 @@ pub struct Receiver { impl Receiver { /// Receives a message from the channel. /// - /// If the channel is emtpy and still has senders, this method + /// If the channel is empty and still has senders, this method /// will wait until a message is sent into it. Once all senders /// have been dropped it will return `None`. /// @@ -361,7 +361,7 @@ impl Receiver { /// let (s, r) = channel(1); /// /// task::spawn(async move { - /// s.send(1).await; + /// s.send(1usize).await; /// s.send(2).await; /// // Then we drop the sender /// }); From 49dd02b4debe96029c4375b32df29059920001c3 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Sun, 15 Mar 2020 23:51:19 +0100 Subject: [PATCH 069/149] Make the split struct public --- src/io/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/io/mod.rs b/src/io/mod.rs index 51c473d..dd97567 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -275,7 +275,7 @@ cfg_std! { #[doc(inline)] pub use std::io::{Error, ErrorKind, IoSlice, IoSliceMut, Result, SeekFrom}; - pub use buf_read::{BufRead, Lines}; + pub use buf_read::{BufRead, Lines, Split}; pub use buf_reader::BufReader; pub use buf_writer::{BufWriter, IntoInnerError}; pub use copy::copy; From 32dce319d33f4b5279e8b421134fbecd0c5b2120 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 25 Nov 2019 19:35:50 +0100 Subject: [PATCH 070/149] expose try_recv and try_send on channels Signed-off-by: Yoshua Wuyts --- src/sync/channel.rs | 88 ++++++++++++++++++++++++++++++++++++++++++--- src/sync/mod.rs | 2 +- 2 files changed, 84 insertions(+), 6 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index a957bec..ff9f996 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -1,5 +1,6 @@ use std::cell::UnsafeCell; -use std::fmt; +use std::error::Error; +use std::fmt::{Debug, Display, self}; use std::future::Future; use std::isize; use std::marker::PhantomData; @@ -192,6 +193,27 @@ impl Sender { .await } + /// Attempts to send a message into the channel. + /// + /// If the channel is full, this method will return an error. + /// + /// # Examples + /// + /// ``` + /// # async_std::task::block_on(async { + /// # + /// use async_std::sync::channel; + /// + /// let (s, r) = channel(1); + /// assert!(s.try_send(1).is_ok()); + /// assert!(s.try_send(2).is_err()); + /// # + /// # }) + /// ``` + pub fn try_send(&self, msg: T) -> Result<(), TrySendError> { + self.channel.try_send(msg) + } + /// Returns the channel capacity. /// /// # Examples @@ -409,6 +431,30 @@ impl Receiver { .await } + /// Attempts to receive a message from the channel. + /// + /// If the channel is empty, this method will return an error. + /// + /// # Examples + /// + /// ``` + /// # async_std::task::block_on(async { + /// # + /// use async_std::sync::channel; + /// + /// let (s, r) = channel(1); + /// + /// s.send(1u8).await; + /// + /// assert!(r.try_recv().is_ok()); + /// assert!(r.try_recv().is_err()); + /// # + /// # }) + /// ``` + pub fn try_recv(&self) -> Result { + self.channel.try_recv() + } + /// Returns the channel capacity. /// /// # Examples @@ -936,8 +982,8 @@ impl Drop for Channel { } } -/// An error returned from the `try_send()` method. -enum TrySendError { +/// An error returned from the `try_send` method. +pub enum TrySendError { /// The channel is full but not disconnected. Full(T), @@ -945,11 +991,43 @@ enum TrySendError { Disconnected(T), } -/// An error returned from the `try_recv()` method. -enum TryRecvError { +impl Error for TrySendError {} + +impl Debug for TrySendError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Full(_) => Debug::fmt("Full", f), + Self::Disconnected(_) => Debug::fmt("Disconnected", f), + } + } +} + +impl Display for TrySendError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Full(_) => Display::fmt("The channel is full.", f), + Self::Disconnected(_) => Display::fmt("The channel is full and disconnected.", f), + } + } +} + +/// An error returned from the `try_recv` method. +#[derive(Debug)] +pub enum TryRecvError { /// The channel is empty but not disconnected. Empty, /// The channel is empty and disconnected. Disconnected, } + +impl Error for TryRecvError {} + +impl Display for TryRecvError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Empty => Display::fmt("The channel is empty.", f), + Self::Disconnected => Display::fmt("The channel is empty and disconnected.", f), + } + } +} diff --git a/src/sync/mod.rs b/src/sync/mod.rs index 088c520..1d6a93f 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -184,7 +184,7 @@ mod rwlock; cfg_unstable! { pub use barrier::{Barrier, BarrierWaitResult}; - pub use channel::{channel, Sender, Receiver}; + pub use channel::{channel, Sender, Receiver, TryRecvError, TrySendError}; mod barrier; mod channel; From 7b7b959a6efa4732d7284a7b41cc3220e94b5694 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Tue, 26 Nov 2019 10:58:53 +0100 Subject: [PATCH 071/149] mark channel errs as unstable Signed-off-by: Yoshua Wuyts --- src/sync/channel.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index ff9f996..a1f2183 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -983,6 +983,8 @@ impl Drop for Channel { } /// An error returned from the `try_send` method. +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] pub enum TrySendError { /// The channel is full but not disconnected. Full(T), @@ -1012,6 +1014,8 @@ impl Display for TrySendError { } /// An error returned from the `try_recv` method. +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] #[derive(Debug)] pub enum TryRecvError { /// The channel is empty but not disconnected. From 7885c245c54ebc7943b406624462fde8be1fc41c Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Thu, 12 Dec 2019 10:36:43 +0100 Subject: [PATCH 072/149] recverror Signed-off-by: Yoshua Wuyts --- src/sync/channel.rs | 39 ++++++++++++++++++++++++++------------- src/sync/mod.rs | 2 +- 2 files changed, 27 insertions(+), 14 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index a1f2183..ace18e7 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -1,6 +1,6 @@ use std::cell::UnsafeCell; use std::error::Error; -use std::fmt::{Debug, Display, self}; +use std::fmt::{self, Debug, Display}; use std::future::Future; use std::isize; use std::marker::PhantomData; @@ -388,22 +388,20 @@ impl Receiver { /// // Then we drop the sender /// }); /// - /// assert_eq!(r.recv().await, Some(1)); - /// assert_eq!(r.recv().await, Some(2)); - /// - /// // recv() returns `None` - /// assert_eq!(r.recv().await, None); + /// assert_eq!(r.recv().await, Ok(1)); + /// assert_eq!(r.recv().await, Ok(2)); + /// assert!(r.recv().await.is_err()); /// # /// # }) /// ``` - pub async fn recv(&self) -> Option { + pub async fn recv(&self) -> Result { struct RecvFuture<'a, T> { channel: &'a Channel, opt_key: Option, } impl Future for RecvFuture<'_, T> { - type Output = Option; + type Output = Result; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { poll_recv( @@ -569,12 +567,13 @@ impl Stream for Receiver { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = &mut *self; - poll_recv( + let res = futures_core::ready!(poll_recv( &this.channel, &this.channel.stream_wakers, &mut this.opt_key, cx, - ) + )); + Poll::Ready(res.ok()) } } @@ -593,7 +592,7 @@ fn poll_recv( wakers: &WakerSet, opt_key: &mut Option, cx: &mut Context<'_>, -) -> Poll> { +) -> Poll> { loop { // If the current task is in the set, remove it. if let Some(key) = opt_key.take() { @@ -602,8 +601,8 @@ fn poll_recv( // Try receiving a message. match channel.try_recv() { - Ok(msg) => return Poll::Ready(Some(msg)), - Err(TryRecvError::Disconnected) => return Poll::Ready(None), + Ok(msg) => return Poll::Ready(Ok(msg)), + Err(TryRecvError::Disconnected) => return Poll::Ready(Err(RecvError {})), Err(TryRecvError::Empty) => { // Insert this receive operation. *opt_key = Some(wakers.insert(cx)); @@ -1035,3 +1034,17 @@ impl Display for TryRecvError { } } } + +/// An error returned from the `recv` method. +#[cfg(feature = "unstable")] +#[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[derive(Debug)] +pub struct RecvError; + +impl Error for RecvError {} + +impl Display for RecvError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + Display::fmt("The channel is empty.", f) + } +} diff --git a/src/sync/mod.rs b/src/sync/mod.rs index 1d6a93f..c221165 100644 --- a/src/sync/mod.rs +++ b/src/sync/mod.rs @@ -184,7 +184,7 @@ mod rwlock; cfg_unstable! { pub use barrier::{Barrier, BarrierWaitResult}; - pub use channel::{channel, Sender, Receiver, TryRecvError, TrySendError}; + pub use channel::{channel, Sender, Receiver, RecvError, TryRecvError, TrySendError}; mod barrier; mod channel; From 19fd7a4084b937c1d3de04b044f73ec9e52d77e1 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Thu, 12 Dec 2019 11:24:52 +0100 Subject: [PATCH 073/149] fix channel tests Signed-off-by: Yoshua Wuyts --- tests/channel.rs | 40 ++++++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/tests/channel.rs b/tests/channel.rs index 34bd888..f302906 100644 --- a/tests/channel.rs +++ b/tests/channel.rs @@ -18,13 +18,13 @@ fn smoke() { let (s, r) = channel(1); s.send(7).await; - assert_eq!(r.recv().await, Some(7)); + assert_eq!(r.recv().await.unwrap(), 7); s.send(8).await; - assert_eq!(r.recv().await, Some(8)); + assert_eq!(r.recv().await.unwrap(), 8); drop(s); - assert_eq!(r.recv().await, None); + assert!(r.recv().await.is_err()); }); task::block_on(async { @@ -74,7 +74,7 @@ fn len_empty_full() { assert_eq!(r.is_empty(), false); assert_eq!(r.is_full(), true); - r.recv().await; + let _ = r.recv().await; assert_eq!(s.len(), 1); assert_eq!(s.is_empty(), false); @@ -91,12 +91,12 @@ fn recv() { let (s, r) = channel(100); task::spawn(async move { - assert_eq!(r.recv().await, Some(7)); + assert_eq!(r.recv().await.unwrap(), 7); task::sleep(ms(1000)).await; - assert_eq!(r.recv().await, Some(8)); + assert_eq!(r.recv().await.unwrap(), 8); task::sleep(ms(1000)).await; - assert_eq!(r.recv().await, Some(9)); - assert_eq!(r.recv().await, None); + assert_eq!(r.recv().await.unwrap(), 9); + assert!(r.recv().await.is_err()); }); task::sleep(ms(1500)).await; @@ -122,9 +122,9 @@ fn send() { }); task::sleep(ms(1500)).await; - assert_eq!(r.recv().await, Some(7)); - assert_eq!(r.recv().await, Some(8)); - assert_eq!(r.recv().await, Some(9)); + assert_eq!(r.recv().await.unwrap(), 7); + assert_eq!(r.recv().await.unwrap(), 8); + assert_eq!(r.recv().await.unwrap(), 9); }) } @@ -139,10 +139,10 @@ fn recv_after_disconnect() { drop(s); - assert_eq!(r.recv().await, Some(1)); - assert_eq!(r.recv().await, Some(2)); - assert_eq!(r.recv().await, Some(3)); - assert_eq!(r.recv().await, None); + assert_eq!(r.recv().await.unwrap(), 1); + assert_eq!(r.recv().await.unwrap(), 2); + assert_eq!(r.recv().await.unwrap(), 3); + assert!(r.recv().await.is_err()); }) } @@ -164,7 +164,7 @@ fn len() { } for i in 0..50 { - r.recv().await; + let _ = r.recv().await; assert_eq!(r.len(), 50 - i - 1); } } @@ -188,7 +188,7 @@ fn len() { let r = r.clone(); async move { for i in 0..COUNT { - assert_eq!(r.recv().await, Some(i)); + assert_eq!(r.recv().await.unwrap(), i); let len = r.len(); assert!(len <= CAP); } @@ -214,7 +214,7 @@ fn disconnect_wakes_receiver() { let (s, r) = channel::<()>(1); let child = task::spawn(async move { - assert_eq!(r.recv().await, None); + assert!(r.recv().await.is_err()); }); task::sleep(ms(1000)).await; @@ -233,9 +233,9 @@ fn spsc() { let child = task::spawn(async move { for i in 0..COUNT { - assert_eq!(r.recv().await, Some(i)); + assert_eq!(r.recv().await.unwrap(), i); } - assert_eq!(r.recv().await, None); + assert!(r.recv().await.is_err()); }); for i in 0..COUNT { From b7c7efc797a0a5d4bce5e1d80bc95bb189d160f1 Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Mon, 16 Mar 2020 00:05:39 +0100 Subject: [PATCH 074/149] Update try_channel doctests --- src/sync/channel.rs | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index ace18e7..8ab1cc1 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -32,6 +32,7 @@ use crate::sync::WakerSet; /// # Examples /// /// ``` +/// # fn main() -> Result<(), async_std::sync::RecvError> { /// # async_std::task::block_on(async { /// # /// use std::time::Duration; @@ -51,10 +52,11 @@ use crate::sync::WakerSet; /// }); /// /// task::sleep(Duration::from_secs(1)).await; -/// assert_eq!(r.recv().await, Some(1)); -/// assert_eq!(r.recv().await, Some(2)); +/// assert_eq!(r.recv().await?, 1); +/// assert_eq!(r.recv().await?, 2); +/// # Ok(()) /// # -/// # }) +/// # }) } /// ``` #[cfg(feature = "unstable")] #[cfg_attr(feature = "docs", doc(cfg(unstable)))] @@ -113,6 +115,7 @@ impl Sender { /// # Examples /// /// ``` + /// # fn main() -> Result<(), async_std::sync::RecvError> { /// # async_std::task::block_on(async { /// # /// use async_std::sync::channel; @@ -125,11 +128,12 @@ impl Sender { /// s.send(2).await; /// }); /// - /// assert_eq!(r.recv().await, Some(1)); - /// assert_eq!(r.recv().await, Some(2)); - /// assert_eq!(r.recv().await, None); + /// assert_eq!(r.recv().await?, 1); + /// assert_eq!(r.recv().await?, 2); + /// assert!(r.recv().await.is_err()); /// # - /// # }) + /// # Ok(()) + /// # }) } /// ``` pub async fn send(&self, msg: T) { struct SendFuture<'a, T> { @@ -335,6 +339,7 @@ impl fmt::Debug for Sender { /// # Examples /// /// ``` +/// # fn main() -> Result<(), async_std::sync::RecvError> { /// # async_std::task::block_on(async { /// # /// use std::time::Duration; @@ -350,10 +355,11 @@ impl fmt::Debug for Sender { /// s.send(2).await; /// }); /// -/// assert_eq!(r.recv().await, Some(1)); // Received immediately. -/// assert_eq!(r.recv().await, Some(2)); // Received after 1 second. +/// assert_eq!(r.recv().await?, 1); // Received immediately. +/// assert_eq!(r.recv().await?, 2); // Received after 1 second. /// # -/// # }) +/// # Ok(()) +/// # }) } /// ``` #[cfg(feature = "unstable")] #[cfg_attr(feature = "docs", doc(cfg(unstable)))] @@ -375,6 +381,7 @@ impl Receiver { /// # Examples /// /// ``` + /// # fn main() -> Result<(), async_std::sync::RecvError> { /// # async_std::task::block_on(async { /// # /// use async_std::sync::channel; @@ -388,11 +395,12 @@ impl Receiver { /// // Then we drop the sender /// }); /// - /// assert_eq!(r.recv().await, Ok(1)); - /// assert_eq!(r.recv().await, Ok(2)); + /// assert_eq!(r.recv().await?, 1); + /// assert_eq!(r.recv().await?, 2); /// assert!(r.recv().await.is_err()); /// # - /// # }) + /// # Ok(()) + /// # }) } /// ``` pub async fn recv(&self) -> Result { struct RecvFuture<'a, T> { From 98cbf7f8ebbab626d6cb8e46d2048b54687c554d Mon Sep 17 00:00:00 2001 From: k-nasa Date: Tue, 17 Mar 2020 20:39:30 +0900 Subject: [PATCH 075/149] 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 6c8237276b89646bac099874626af14accedc823 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 18 Mar 2020 23:02:59 +0900 Subject: [PATCH 076/149] fix doc test --- src/lib.rs | 9 ++++++--- src/stream/stream/mod.rs | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index d498792..e3cfb51 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -138,7 +138,8 @@ //! //! Call an async function from the main function: //! -//! ``` +#![cfg_attr(feature = "attributes", doc = "```")] +#![cfg_attr(not(feature = "attributes"), doc = "```ignore")] //! async fn say_hello() { //! println!("Hello, world!"); //! } @@ -151,7 +152,8 @@ //! //! Await two futures concurrently, and return a tuple of their output: //! -//! ``` +#![cfg_attr(feature = "attributes", doc = "```")] +#![cfg_attr(not(feature = "attributes"), doc = "```ignore")] //! use async_std::prelude::*; //! //! #[async_std::main] @@ -164,7 +166,8 @@ //! //! Create a UDP server that echoes back each received message to the sender: //! -//! ```no_run +#![cfg_attr(feature = "attributes", doc = "```no_run")] +#![cfg_attr(not(feature = "attributes"), doc = "```ignore")] //! use async_std::net::UdpSocket; //! //! #[async_std::main] diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index d0cc718..883ec04 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -1011,7 +1011,7 @@ extension_trait! { # Examples - ```ignore + ``` # fn main() { async_std::task::block_on(async { # use async_std::prelude::*; @@ -1044,7 +1044,7 @@ extension_trait! { # Examples - ```ignore + ``` # fn main() { async_std::task::block_on(async { # use async_std::prelude::*; From c0f18600cff225811313064b88375ee87d3b612a Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 18 Mar 2020 23:03:17 +0900 Subject: [PATCH 077/149] run ignored test --- src/stream/stream/max.rs | 17 +++++------------ src/stream/stream/min.rs | 17 +++++------------ src/stream/stream/mod.rs | 12 ++++++------ 3 files changed, 16 insertions(+), 30 deletions(-) diff --git a/src/stream/stream/max.rs b/src/stream/stream/max.rs index 8a4d594..03fe635 100644 --- a/src/stream/stream/max.rs +++ b/src/stream/stream/max.rs @@ -1,7 +1,6 @@ use core::cmp::{Ord, Ordering}; -use core::marker::PhantomData; -use core::pin::Pin; use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; @@ -11,29 +10,23 @@ use crate::task::{Context, Poll}; pin_project! { #[doc(hidden)] #[allow(missing_debug_implementations)] - pub struct MaxFuture { + pub struct MaxFuture { #[pin] stream: S, - _compare: PhantomData, max: Option, } } -impl MaxFuture { +impl MaxFuture { pub(super) fn new(stream: S) -> Self { - Self { - stream, - _compare: PhantomData, - max: None, - } + Self { stream, max: None } } } -impl Future for MaxFuture +impl Future for MaxFuture where S: Stream, S::Item: Ord, - F: FnMut(&S::Item, &S::Item) -> Ordering, { type Output = Option; diff --git a/src/stream/stream/min.rs b/src/stream/stream/min.rs index 4fe2a67..8430b94 100644 --- a/src/stream/stream/min.rs +++ b/src/stream/stream/min.rs @@ -1,7 +1,6 @@ use core::cmp::{Ord, Ordering}; -use core::marker::PhantomData; -use core::pin::Pin; use core::future::Future; +use core::pin::Pin; use pin_project_lite::pin_project; @@ -11,29 +10,23 @@ use crate::task::{Context, Poll}; pin_project! { #[doc(hidden)] #[allow(missing_debug_implementations)] - pub struct MinFuture { + pub struct MinFuture { #[pin] stream: S, - _compare: PhantomData, min: Option, } } -impl MinFuture { +impl MinFuture { pub(super) fn new(stream: S) -> Self { - Self { - stream, - _compare: PhantomData, - min: None, - } + Self { stream, min: None } } } -impl Future for MinFuture +impl Future for MinFuture where S: Stream, S::Item: Ord, - F: FnMut(&S::Item, &S::Item) -> Ordering, { type Output = Option; diff --git a/src/stream/stream/mod.rs b/src/stream/stream/mod.rs index 883ec04..4be0eb5 100644 --- a/src/stream/stream/mod.rs +++ b/src/stream/stream/mod.rs @@ -1028,12 +1028,12 @@ extension_trait! { # }) } ``` "#] - fn max( + fn max( self, - ) -> impl Future> [MaxFuture] + ) -> impl Future> [MaxFuture] where Self: Sized, - F: FnMut(&Self::Item, &Self::Item) -> Ordering, + Self::Item: Ord, { MaxFuture::new(self) } @@ -1061,12 +1061,12 @@ extension_trait! { # }) } ``` "#] - fn min( + fn min( self, - ) -> impl Future> [MinFuture] + ) -> impl Future> [MinFuture] where Self: Sized, - F: FnMut(&Self::Item, &Self::Item) -> Ordering, + Self::Item: Ord, { MinFuture::new(self) } From 2ab075d02796f0577316b4d22412030340a652e3 Mon Sep 17 00:00:00 2001 From: k-nasa Date: Thu, 19 Mar 2020 11:50:19 +0900 Subject: [PATCH 078/149] refactor --- src/os/unix/net/datagram.rs | 2 -- src/os/unix/net/listener.rs | 2 -- src/os/unix/net/stream.rs | 2 -- 3 files changed, 6 deletions(-) diff --git a/src/os/unix/net/datagram.rs b/src/os/unix/net/datagram.rs index fc426b7..dbf421b 100644 --- a/src/os/unix/net/datagram.rs +++ b/src/os/unix/net/datagram.rs @@ -3,8 +3,6 @@ use std::fmt; use std::net::Shutdown; -use mio_uds; - use super::SocketAddr; use crate::future; use crate::io; diff --git a/src/os/unix/net/listener.rs b/src/os/unix/net/listener.rs index 675ef48..01ecf3f 100644 --- a/src/os/unix/net/listener.rs +++ b/src/os/unix/net/listener.rs @@ -4,8 +4,6 @@ use std::fmt; use std::pin::Pin; use std::future::Future; -use mio_uds; - use super::SocketAddr; use super::UnixStream; use crate::future; diff --git a/src/os/unix/net/stream.rs b/src/os/unix/net/stream.rs index 647edc9..f74cb60 100644 --- a/src/os/unix/net/stream.rs +++ b/src/os/unix/net/stream.rs @@ -5,8 +5,6 @@ use std::io::{Read as _, Write as _}; use std::net::Shutdown; use std::pin::Pin; -use mio_uds; - use super::SocketAddr; use crate::io::{self, Read, Write}; use crate::net::driver::Watcher; From b1ec1ea9300de215f2b46d2adcd232602abd214e Mon Sep 17 00:00:00 2001 From: k-nasa Date: Wed, 18 Mar 2020 23:59:12 +0900 Subject: [PATCH 079/149] 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 080/149] 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 081/149] 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 082/149] 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 083/149] 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 084/149] 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 085/149] 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 086/149] 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 087/149] 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 088/149] 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 090/149] 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 091/149] 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 092/149] 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 093/149] 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 094/149] 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 095/149] 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 096/149] 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 097/149] 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 098/149] 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 099/149] 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 100/149] 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 101/149] 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 102/149] 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 103/149] 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 104/149] 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 105/149] 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 106/149] 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 107/149] 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 108/149] 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 109/149] 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 110/149] 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 111/149] 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 112/149] 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 113/149] 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 114/149] 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 115/149] 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 116/149] 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 117/149] 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 118/149] 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 119/149] 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 120/149] 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 121/149] 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 122/149] 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 123/149] 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 124/149] 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 125/149] 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 126/149] 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 127/149] 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 128/149] 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 129/149] 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 130/149] 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 131/149] 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 132/149] 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 133/149] 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 134/149] 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"); } From c9ecb5bbbdfaaececf369915852d748a73af726e Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Tue, 19 May 2020 11:29:36 +0200 Subject: [PATCH 135/149] prepare v1.6.0-beta.2 --- CHANGELOG.md | 16 ++++++++++++++-- Cargo.toml | 4 ++-- src/lib.rs | 8 ++++---- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 70d8ed8..fc96d1a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,10 +7,21 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +# [1.6.0-beta.2] - 2020-05-19 + +## Added + +- Added an environment variable to configure the thread pool size of the runtime. ([#774](https://github.com/async-rs/async-std/pull/774)) +- Implement `Clone` for `UnixStream` ([#772](https://github.com/async-rs/async-std/pull/772)) + ## 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)) +## Fixed + +- Use `smol::block_on` to handle drop of `File`, avoiding nested executor panic. ([#768](https://github.com/async-rs/async-std/pull/768)) + # [1.6.0-beta.1] - 2020-05-07 ## Added @@ -701,8 +712,9 @@ 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 +[Unreleased]: https://github.com/async-rs/async-std/compare/v1.6.0-beta.2...HEAD +[1.6.0-beta.2]: https://github.com/async-rs/async-std/compare/v1.6.0-beta.1...v1.6.0-beta.2 +[1.6.0-beta.1]: https://github.com/async-rs/async-std/compare/v1.5.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 diff --git a/Cargo.toml b/Cargo.toml index 5a1d31e..7dae8f0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "async-std" -version = "1.6.0-beta.1" +version = "1.6.0-beta.2" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", @@ -71,7 +71,7 @@ slab = { version = "0.4.2", optional = true } surf = { version = "1.0.3", optional = true } [target.'cfg(not(target_os = "unknown"))'.dependencies] -smol = { version = "0.1.1", optional = true } +smol = { version = "0.1.8", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] futures-timer = { version = "3.0.2", optional = true, features = ["wasm-bindgen"] } diff --git a/src/lib.rs b/src/lib.rs index 4704a9d..c669b45 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -194,7 +194,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.1" +//! version = "1.6.0-beta.2" //! features = ["unstable"] //! ``` //! @@ -207,7 +207,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.1" +//! version = "1.6.0-beta.2" //! features = ["attributes"] //! ``` //! @@ -216,7 +216,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.1" +//! version = "1.6.0-beta.2" //! default-features = false //! features = ["std"] //! ``` @@ -226,7 +226,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.1" +//! version = "1.6.0-beta.2" //! default-features = false //! features = ["alloc"] //! ``` From 69806403c6cfd92eb273cc7391a396c10d69e209 Mon Sep 17 00:00:00 2001 From: "Heinz N. Gies" Date: Wed, 20 May 2020 14:24:06 +0200 Subject: [PATCH 136/149] Fix readme for BufRead The `BufRead` readme points to `BufReadExt` being in `async_std::prelude` while it currently lives in `async_std::io::prelude` --- src/io/buf_read/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/io/buf_read/mod.rs b/src/io/buf_read/mod.rs index d919a78..7a0ecc6 100644 --- a/src/io/buf_read/mod.rs +++ b/src/io/buf_read/mod.rs @@ -29,7 +29,7 @@ extension_trait! { ``` # #[allow(unused_imports)] - use async_std::prelude::*; + use async_std::io::prelude::*; ``` [`std::io::BufRead`]: https://doc.rust-lang.org/std/io/trait.BufRead.html From 06eea4225b01329ac8faa9cc2a41a61d702b7d92 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 22 May 2020 22:08:23 +0200 Subject: [PATCH 137/149] feat: add PartialEq and Eq for channel Errors Closes #792 --- src/sync/channel.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/sync/channel.rs b/src/sync/channel.rs index 8ab1cc1..3207846 100644 --- a/src/sync/channel.rs +++ b/src/sync/channel.rs @@ -992,6 +992,7 @@ impl Drop for Channel { /// An error returned from the `try_send` method. #[cfg(feature = "unstable")] #[cfg_attr(feature = "docs", doc(cfg(unstable)))] +#[derive(PartialEq, Eq)] pub enum TrySendError { /// The channel is full but not disconnected. Full(T), @@ -1023,7 +1024,7 @@ impl Display for TrySendError { /// An error returned from the `try_recv` method. #[cfg(feature = "unstable")] #[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq)] pub enum TryRecvError { /// The channel is empty but not disconnected. Empty, @@ -1046,7 +1047,7 @@ impl Display for TryRecvError { /// An error returned from the `recv` method. #[cfg(feature = "unstable")] #[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq)] pub struct RecvError; impl Error for RecvError {} From e1c8638173e56f836f243f594079143804147f9e Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Fri, 22 May 2020 22:08:36 +0200 Subject: [PATCH 138/149] chore: release v1.6.0 --- CHANGELOG.md | 7 ++++++- Cargo.toml | 4 ++-- src/lib.rs | 8 ++++---- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fc96d1a..ccefc28 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,10 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +# [1.6.0] - 2020-05-22 + +See `1.6.0-beta.1` and `1.6.0-beta.2`. + # [1.6.0-beta.2] - 2020-05-19 ## Added @@ -712,7 +716,8 @@ task::blocking(async { - Initial beta release -[Unreleased]: https://github.com/async-rs/async-std/compare/v1.6.0-beta.2...HEAD +[Unreleased]: https://github.com/async-rs/async-std/compare/v1.6.0...HEAD +[1.6.0]: https://github.com/async-rs/async-std/compare/v1.5.0...v1.6.0 [1.6.0-beta.2]: https://github.com/async-rs/async-std/compare/v1.6.0-beta.1...v1.6.0-beta.2 [1.6.0-beta.1]: https://github.com/async-rs/async-std/compare/v1.5.0...v1.6.0-beta.1 [1.5.0]: https://github.com/async-rs/async-std/compare/v1.4.0...v1.5.0 diff --git a/Cargo.toml b/Cargo.toml index 7dae8f0..d02577e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "async-std" -version = "1.6.0-beta.2" +version = "1.6.0" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", @@ -71,7 +71,7 @@ slab = { version = "0.4.2", optional = true } surf = { version = "1.0.3", optional = true } [target.'cfg(not(target_os = "unknown"))'.dependencies] -smol = { version = "0.1.8", optional = true } +smol = { version = "0.1.10", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] futures-timer = { version = "3.0.2", optional = true, features = ["wasm-bindgen"] } diff --git a/src/lib.rs b/src/lib.rs index e3486d0..2dbd258 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -197,7 +197,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.2" +//! version = "1.6.0" //! features = ["unstable"] //! ``` //! @@ -210,7 +210,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.2" +//! version = "1.6.0" //! features = ["attributes"] //! ``` //! @@ -219,7 +219,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.2" +//! version = "1.6.0" //! default-features = false //! features = ["std"] //! ``` @@ -229,7 +229,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0-beta.2" +//! version = "1.6.0" //! default-features = false //! features = ["alloc"] //! ``` From d60e7cc27dffc57905e69c1a328cfba10b5eeddc Mon Sep 17 00:00:00 2001 From: jerry73204 Date: Fri, 29 May 2020 19:18:06 +0800 Subject: [PATCH 139/149] Fix wrong slice index when reading a 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 1930fdd..2ff5643 100644 --- a/src/fs/file.rs +++ b/src/fs/file.rs @@ -673,7 +673,7 @@ impl LockGuard { if available > 0 || self.cache.is_empty() { // Copy data from the cache into the buffer. let n = cmp::min(available, buf.len()); - buf[..n].copy_from_slice(&self.cache[start..n]); + buf[..n].copy_from_slice(&self.cache[start..(start + n)]); // Move the read cursor forward. self.mode = Mode::Reading(start + n); From 166c469d1c5a14a60b1f144cd40fae08e914678f Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Wed, 3 Jun 2020 12:09:33 +0200 Subject: [PATCH 140/149] Add the tokio02 feature flag --- Cargo.toml | 3 ++- src/lib.rs | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index d02577e..d51165b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -51,6 +51,7 @@ alloc = [ "futures-core/alloc", "pin-project-lite", ] +tokio02 = ["smol/tokio02"] [dependencies] async-attributes = { version = "1.1.1", optional = true } @@ -98,4 +99,4 @@ required-features = ["unstable"] [[example]] name = "surf-web" -required-features = ["surf"] \ No newline at end of file +required-features = ["surf"] diff --git a/src/lib.rs b/src/lib.rs index 2dbd258..e5b0438 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -214,6 +214,15 @@ //! features = ["attributes"] //! ``` //! +//! Compatibility with the `tokio` runtime is possible using the `tokio02` +//! Cargo feature: +//! +//! ```toml +//! [dependencies.async-std] +//! version = "1.6.0" +//! features = ["tokio02"] +//! ``` +//! //! Additionally it's possible to only use the core traits and combinators by //! only enabling the `std` Cargo feature: //! From 0df3c02b81f8b581dab0104fb13fdd662caa046f Mon Sep 17 00:00:00 2001 From: Yoshua Wuyts Date: Wed, 3 Jun 2020 12:40:02 +0200 Subject: [PATCH 141/149] check tokio02 features --- .github/workflows/ci.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 1a0c432..fcbc4bd 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -110,6 +110,17 @@ jobs: command: check args: --no-default-features --features alloc --target thumbv7m-none-eabi -Z avoid-dev-deps + check_tokio_02_feature: + name: Check tokio02 feature + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@master + - name: check tokio02 + uses: actions-rs/cargo@v1 + with: + command: check + args: --all --features tokio02 + cross: name: Cross compile runs-on: ubuntu-latest From 52c72426c1c377504addda2ffaccea6557f776ff Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 3 Jun 2020 18:38:20 +0200 Subject: [PATCH 142/149] fix: do not require the runtime to use unstable features --- Cargo.toml | 9 ++++++--- src/future/mod.rs | 8 ++++---- src/task/mod.rs | 2 ++ src/utils.rs | 44 +++++++++++++++++++++++--------------------- 4 files changed, 35 insertions(+), 28 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d02577e..bf08bee 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -29,9 +29,13 @@ default = [ "log", "num_cpus", "pin-project-lite", + "smol", ] docs = ["attributes", "unstable", "default"] -unstable = ["std"] +unstable = [ + "std", + "futures-timer", +] attributes = ["async-attributes"] std = [ "alloc", @@ -42,8 +46,6 @@ std = [ "once_cell", "pin-utils", "slab", - "smol", - "futures-timer", "wasm-bindgen-futures", "futures-channel", ] @@ -66,6 +68,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 } +futures-timer = { version = "3.0.2", optional = true } # Devdepencency, but they are not allowed to be optional :/ surf = { version = "1.0.3", optional = true } diff --git a/src/future/mod.rs b/src/future/mod.rs index 9b75533..db0607a 100644 --- a/src/future/mod.rs +++ b/src/future/mod.rs @@ -61,10 +61,10 @@ cfg_std! { mod ready; } -cfg_default! { - pub use timeout::{timeout, TimeoutError}; - mod timeout; -} +#[cfg(any(feature = "unstable", feature = "default"))] +pub use timeout::{timeout, TimeoutError}; +#[cfg(any(feature = "unstable", feature = "default"))] +mod timeout; cfg_unstable! { pub use into_future::IntoFuture; diff --git a/src/task/mod.rs b/src/task/mod.rs index eefc7c2..ca0b92a 100644 --- a/src/task/mod.rs +++ b/src/task/mod.rs @@ -168,7 +168,9 @@ cfg_default! { } cfg_unstable! { + #[cfg(feature = "default")] pub use spawn_local::spawn_local; + #[cfg(feature = "default")] mod spawn_local; } diff --git a/src/utils.rs b/src/utils.rs index 7c9aa99..e7ea9ec 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -60,36 +60,38 @@ pub(crate) trait Context { } #[cfg(all(not(target_os = "unknown"), feature = "default"))] -pub(crate) type Timer = smol::Timer; +mod timer { + pub type Timer = smol::Timer; +} -#[cfg(all(target_arch = "wasm32", feature = "default"))] -#[derive(Debug)] -pub(crate) struct Timer(futures_timer::Delay); +#[cfg(any(all(target_arch = "wasm32", feature = "default"), feature = "unstable"))] +mod timer { + use std::pin::Pin; + use std::task::Poll; -#[cfg(all(target_arch = "wasm32", feature = "default"))] -impl Timer { - pub(crate) fn after(dur: std::time::Duration) -> Self { - Timer(futures_timer::Delay::new(dur)) - } -} + #[derive(Debug)] + pub(crate) struct Timer(futures_timer::Delay); -#[cfg(target_arch = "wasm32")] -use std::pin::Pin; -#[cfg(target_arch = "wasm32")] -use std::task::Poll; + impl Timer { + pub(crate) fn after(dur: std::time::Duration) -> Self { + Timer(futures_timer::Delay::new(dur)) + } + } -#[cfg(target_arch = "wasm32")] -impl std::future::Future for Timer { - type Output = (); + 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(()), + 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(()), + } } } } +pub(crate) use timer::*; + /// Defers evaluation of a block of code until the end of the scope. #[cfg(feature = "default")] #[doc(hidden)] From 8943ba82dd0e4cb8a29f6750a519a53080093944 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 3 Jun 2020 18:43:19 +0200 Subject: [PATCH 143/149] fix nostd --- src/utils.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/utils.rs b/src/utils.rs index e7ea9ec..9ad4338 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -90,6 +90,7 @@ mod timer { } } +#[cfg(any(feature = "unstable", feature = "default"))] pub(crate) use timer::*; /// Defers evaluation of a block of code until the end of the scope. From 8389041414a4c39f3e899415d51d3cffdb121eea Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 3 Jun 2020 18:50:12 +0200 Subject: [PATCH 144/149] fix --- src/utils.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/utils.rs b/src/utils.rs index 9ad4338..e064570 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -64,7 +64,10 @@ mod timer { pub type Timer = smol::Timer; } -#[cfg(any(all(target_arch = "wasm32", feature = "default"), feature = "unstable"))] +#[cfg(any( + all(target_arch = "wasm32", feature = "default"), + all(feature = "unstable", not(feature = "default")) +))] mod timer { use std::pin::Pin; use std::task::Poll; From 721760a7a612eabce6a536945bafee27dfa5ae99 Mon Sep 17 00:00:00 2001 From: Konrad Borowski Date: Thu, 4 Jun 2020 09:05:14 +0200 Subject: [PATCH 145/149] Remove stdio lock methods Fixes #805. --- src/io/mod.rs | 9 ------- src/io/stderr.rs | 70 ------------------------------------------------ src/io/stdin.rs | 61 ----------------------------------------- src/io/stdout.rs | 70 ------------------------------------------------ 4 files changed, 210 deletions(-) diff --git a/src/io/mod.rs b/src/io/mod.rs index f5dd9e2..a673636 100644 --- a/src/io/mod.rs +++ b/src/io/mod.rs @@ -328,12 +328,3 @@ cfg_default! { #[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/stderr.rs b/src/io/stderr.rs index 5ff8a02..5067ed4 100644 --- a/src/io/stderr.rs +++ b/src/io/stderr.rs @@ -5,11 +5,6 @@ use std::future::Future; use crate::io::{self, Write}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Write as _; -} - /// Constructs a new handle to the standard error of the current process. /// /// This function is an async version of [`std::io::stderr`]. @@ -58,22 +53,6 @@ pub fn stderr() -> Stderr { #[derive(Debug)] pub struct Stderr(Mutex); -/// A locked reference to the Stderr handle. -/// -/// This handle implements the [`Write`] traits, and is constructed via the [`Stderr::lock`] -/// method. -/// -/// [`Write`]: trait.Read.html -/// [`Stderr::lock`]: struct.Stderr.html#method.lock -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[derive(Debug)] -pub struct StderrLock<'a>(std::io::StderrLock<'a>); - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -unsafe impl Send for StderrLock<'_> {} - /// The state of the asynchronous stderr. /// /// The stderr can be either idle or busy performing an asynchronous operation. @@ -108,35 +87,6 @@ enum Operation { Flush(io::Result<()>), } -impl Stderr { - /// Locks this handle to the standard error stream, returning a writable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let stderr = io::stderr(); - /// let mut handle = stderr.lock().await; - /// - /// handle.write_all(b"hello world").await?; - /// # - /// # Ok(()) }) } - /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] - pub async fn lock(&self) -> StderrLock<'static> { - static STDERR: Lazy = Lazy::new(std::io::stderr); - - spawn_blocking(move || StderrLock(STDERR.lock())).await - } -} - impl Write for Stderr { fn poll_write( mut self: Pin<&mut Self>, @@ -239,23 +189,3 @@ cfg_windows! { } } } - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -impl io::Write for StderrLock<'_> { - fn poll_write( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - Poll::Ready(self.0.write(buf)) - } - - fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(self.0.flush()) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} diff --git a/src/io/stdin.rs b/src/io/stdin.rs index 369ccae..fc280f8 100644 --- a/src/io/stdin.rs +++ b/src/io/stdin.rs @@ -7,11 +7,6 @@ use crate::io::{self, Read}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; use crate::utils::Context as _; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Read as _; -} - /// Constructs a new handle to the standard input of the current process. /// /// This function is an async version of [`std::io::stdin`]. @@ -61,21 +56,6 @@ pub fn stdin() -> Stdin { #[derive(Debug)] pub struct Stdin(Mutex); -/// A locked reference to the Stdin handle. -/// -/// This handle implements the [`Read`] traits, and is constructed via the [`Stdin::lock`] method. -/// -/// [`Read`]: trait.Read.html -/// [`Stdin::lock`]: struct.Stdin.html#method.lock -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[cfg(feature = "unstable")] -#[derive(Debug)] -pub struct StdinLock<'a>(std::io::StdinLock<'a>); - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -unsafe impl Send for StdinLock<'_> {} - /// The state of the asynchronous stdin. /// /// The stdin can be either idle or busy performing an asynchronous operation. @@ -165,35 +145,6 @@ impl Stdin { .await .context(|| String::from("could not read line on stdin")) } - - /// Locks this handle to the standard input stream, returning a readable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Read trait for accessing the underlying data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let mut buffer = String::new(); - /// - /// let stdin = io::stdin(); - /// let mut handle = stdin.lock().await; - /// - /// handle.read_to_string(&mut buffer).await?; - /// # - /// # Ok(()) }) } - /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] - pub async fn lock(&self) -> StdinLock<'static> { - static STDIN: Lazy = Lazy::new(std::io::stdin); - - spawn_blocking(move || StdinLock(STDIN.lock())).await - } } impl Read for Stdin { @@ -265,15 +216,3 @@ cfg_windows! { } } } - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -impl Read for StdinLock<'_> { - fn poll_read( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - Poll::Ready(self.0.read(buf)) - } -} diff --git a/src/io/stdout.rs b/src/io/stdout.rs index 1711c09..b3dfe64 100644 --- a/src/io/stdout.rs +++ b/src/io/stdout.rs @@ -5,11 +5,6 @@ use std::future::Future; use crate::io::{self, Write}; use crate::task::{spawn_blocking, Context, JoinHandle, Poll}; -cfg_unstable! { - use once_cell::sync::Lazy; - use std::io::Write as _; -} - /// Constructs a new handle to the standard output of the current process. /// /// This function is an async version of [`std::io::stdout`]. @@ -58,22 +53,6 @@ pub fn stdout() -> Stdout { #[derive(Debug)] pub struct Stdout(Mutex); -/// A locked reference to the Stderr handle. -/// -/// This handle implements the [`Write`] traits, and is constructed via the [`Stdout::lock`] -/// method. -/// -/// [`Write`]: trait.Read.html -/// [`Stdout::lock`]: struct.Stdout.html#method.lock -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -#[derive(Debug)] -pub struct StdoutLock<'a>(std::io::StdoutLock<'a>); - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -unsafe impl Send for StdoutLock<'_> {} - /// The state of the asynchronous stdout. /// /// The stdout can be either idle or busy performing an asynchronous operation. @@ -108,35 +87,6 @@ enum Operation { Flush(io::Result<()>), } -impl Stdout { - /// Locks this handle to the standard error stream, returning a writable guard. - /// - /// The lock is released when the returned lock goes out of scope. The returned guard also implements the Write trait for writing data. - /// - /// # Examples - /// - /// ```no_run - /// # fn main() -> std::io::Result<()> { async_std::task::block_on(async { - /// # - /// use async_std::io; - /// use async_std::prelude::*; - /// - /// let stdout = io::stdout(); - /// let mut handle = stdout.lock().await; - /// - /// handle.write_all(b"hello world").await?; - /// # - /// # Ok(()) }) } - /// ``` - #[cfg_attr(feature = "docs", doc(cfg(unstable)))] - #[cfg(any(feature = "unstable", feature = "docs"))] - pub async fn lock(&self) -> StdoutLock<'static> { - static STDOUT: Lazy = Lazy::new(std::io::stdout); - - spawn_blocking(move || StdoutLock(STDOUT.lock())).await - } -} - impl Write for Stdout { fn poll_write( mut self: Pin<&mut Self>, @@ -239,23 +189,3 @@ cfg_windows! { } } } - -#[cfg(feature = "unstable")] -#[cfg_attr(feature = "docs", doc(cfg(unstable)))] -impl Write for StdoutLock<'_> { - fn poll_write( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - Poll::Ready(self.0.write(buf)) - } - - fn poll_flush(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(self.0.flush()) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} From e12cf80ab0290fa766871982803e39882121e4b0 Mon Sep 17 00:00:00 2001 From: Friedel Ziegelmayer Date: Thu, 4 Jun 2020 13:19:03 +0200 Subject: [PATCH 146/149] fix: allow for recursive block-on calls Fixes #798,#795,#760 --- Cargo.toml | 3 ++- src/task/builder.rs | 26 ++++++++++++++++++++++- tests/block_on.rs | 51 ++++++++++++++++++++++++++++++++++++++++++--- 3 files changed, 75 insertions(+), 5 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d2daaf0..284c8b8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,7 +75,7 @@ futures-timer = { version = "3.0.2", optional = true } surf = { version = "1.0.3", optional = true } [target.'cfg(not(target_os = "unknown"))'.dependencies] -smol = { version = "0.1.10", optional = true } +smol = { version = "0.1.11", optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] futures-timer = { version = "3.0.2", optional = true, features = ["wasm-bindgen"] } @@ -103,3 +103,4 @@ required-features = ["unstable"] [[example]] name = "surf-web" required-features = ["surf"] + diff --git a/src/task/builder.rs b/src/task/builder.rs index cbb3187..0024f8a 100644 --- a/src/task/builder.rs +++ b/src/task/builder.rs @@ -1,3 +1,4 @@ +use std::cell::Cell; use std::future::Future; use std::pin::Pin; use std::sync::Arc; @@ -150,8 +151,31 @@ impl Builder { parent_task_id: TaskLocalsWrapper::get_current(|t| t.id().0).unwrap_or(0), }); + thread_local! { + /// Tracks the number of nested block_on calls. + static NUM_NESTED_BLOCKING: Cell = Cell::new(0); + } + // Run the future as a task. - unsafe { TaskLocalsWrapper::set_current(&wrapped.tag, || smol::run(wrapped)) } + NUM_NESTED_BLOCKING.with(|num_nested_blocking| { + let count = num_nested_blocking.get(); + let should_run = count == 0; + // increase the count + num_nested_blocking.replace(count + 1); + + unsafe { + TaskLocalsWrapper::set_current(&wrapped.tag, || { + let res = if should_run { + // The first call should use run. + smol::run(wrapped) + } else { + smol::block_on(wrapped) + }; + num_nested_blocking.replace(num_nested_blocking.get() - 1); + res + }) + } + }) } } diff --git a/tests/block_on.rs b/tests/block_on.rs index 28902b0..4c26480 100644 --- a/tests/block_on.rs +++ b/tests/block_on.rs @@ -1,18 +1,63 @@ #![cfg(not(target_os = "unknown"))] -use async_std::task; +use async_std::{future::ready, task::block_on}; #[test] fn smoke() { - let res = task::block_on(async { 1 + 2 }); + let res = block_on(async { 1 + 2 }); assert_eq!(res, 3); } #[test] #[should_panic = "boom"] fn panic() { - task::block_on(async { + block_on(async { // This panic should get propagated into the parent thread. panic!("boom"); }); } + +#[cfg(feature = "unstable")] +#[test] +fn nested_block_on_local() { + use async_std::task::spawn_local; + + let x = block_on(async { + let a = block_on(async { block_on(async { ready(3).await }) }); + let b = spawn_local(async { block_on(async { ready(2).await }) }).await; + let c = block_on(async { block_on(async { ready(1).await }) }); + a + b + c + }); + + assert_eq!(x, 3 + 2 + 1); + + let y = block_on(async { + let a = block_on(async { block_on(async { ready(3).await }) }); + let b = spawn_local(async { block_on(async { ready(2).await }) }).await; + let c = block_on(async { block_on(async { ready(1).await }) }); + a + b + c + }); + + assert_eq!(y, 3 + 2 + 1); +} + +#[test] +fn nested_block_on() { + let x = block_on(async { + let a = block_on(async { block_on(async { ready(3).await }) }); + let b = block_on(async { block_on(async { ready(2).await }) }); + let c = block_on(async { block_on(async { ready(1).await }) }); + a + b + c + }); + + assert_eq!(x, 3 + 2 + 1); + + let y = block_on(async { + let a = block_on(async { block_on(async { ready(3).await }) }); + let b = block_on(async { block_on(async { ready(2).await }) }); + let c = block_on(async { block_on(async { ready(1).await }) }); + a + b + c + }); + + assert_eq!(y, 3 + 2 + 1); +} From 5a1a681d685763011c258edb8b6e2a3e22bc418e Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 4 Jun 2020 18:25:07 +0200 Subject: [PATCH 147/149] fix(rt): use task::block_on on spawned threads This makes sure to capture threads into the recursive block_on detection. --- src/rt/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rt/mod.rs b/src/rt/mod.rs index 65f4e24..d8550aa 100644 --- a/src/rt/mod.rs +++ b/src/rt/mod.rs @@ -27,7 +27,7 @@ pub static RUNTIME: Lazy = Lazy::new(|| { for _ in 0..thread_count { thread::Builder::new() .name(thread_name.clone()) - .spawn(|| smol::run(future::pending::<()>())) + .spawn(|| crate::task::block_on(future::pending::<()>())) .expect("cannot start a runtime thread"); } Runtime {} From 4555f193a51f6c46ed2d0bab1b190dec94dc5d33 Mon Sep 17 00:00:00 2001 From: Thibault Martinez Date: Sun, 7 Jun 2020 18:15:43 +0200 Subject: [PATCH 148/149] ci: update actions/cache to v2 --- .github/workflows/ci.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index fcbc4bd..7b9439b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -30,19 +30,19 @@ jobs: override: true - name: Cache cargo registry - uses: actions/cache@v1 + uses: actions/cache@v2 with: path: ~/.cargo/registry key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-registry-${{ hashFiles('**/Cargo.toml') }} - name: Cache cargo index - uses: actions/cache@v1 + uses: actions/cache@v2 with: path: ~/.cargo/git key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-index-${{ hashFiles('**/Cargo.toml') }} - name: Cache cargo build - uses: actions/cache@v1 + uses: actions/cache@v2 with: path: target key: ${{ matrix.os }}-${{ matrix.rust }}-cargo-build-target-${{ hashFiles('**/Cargo.toml') }} @@ -58,7 +58,7 @@ jobs: with: command: check args: --features unstable --all --bins --examples --tests - + - name: check wasm uses: actions-rs/cargo@v1 with: From e9c6ea873c628a304b5f2d5a1306c603830e604f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 11 Jun 2020 13:17:31 +0200 Subject: [PATCH 149/149] chore: release v1.6.1 --- CHANGELOG.md | 19 ++++++++++++++++++- Cargo.toml | 2 +- src/lib.rs | 10 +++++----- 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ccefc28..d6bb2cc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,22 @@ and this project adheres to [Semantic Versioning](https://book.async.rs/overview ## [Unreleased] +# [1.6.1] - 2020-06-11 + +## Added + +- Added `tokio02` feature flag, to allow compatability usage with tokio@0.2 ([#804](https://github.com/async-rs/async-std/pull/804)). + +## Changed + +- Removed unstable `stdio` lock methods, due to their unsoundness ([#807](https://github.com/async-rs/async-std/pull/807)). + +## Fixed + +- Fixed wrong slice index for file reading ([#802](https://github.com/async-rs/async-std/pull/802)). +- Fixed recursive calls to `block_on` ([#799](https://github.com/async-rs/async-std/pull/799)) and ([#809](https://github.com/async-rs/async-std/pull/809)). +- Remove `default` feature requirement for the `unstable` feature ([#806](https://github.com/async-rs/async-std/pull/806)). + # [1.6.0] - 2020-05-22 See `1.6.0-beta.1` and `1.6.0-beta.2`. @@ -716,7 +732,8 @@ task::blocking(async { - Initial beta release -[Unreleased]: https://github.com/async-rs/async-std/compare/v1.6.0...HEAD +[Unreleased]: https://github.com/async-rs/async-std/compare/v1.6.1...HEAD +[1.6.1]: https://github.com/async-rs/async-std/compare/v1.6.0...v1.6.1 [1.6.0]: https://github.com/async-rs/async-std/compare/v1.5.0...v1.6.0 [1.6.0-beta.2]: https://github.com/async-rs/async-std/compare/v1.6.0-beta.1...v1.6.0-beta.2 [1.6.0-beta.1]: https://github.com/async-rs/async-std/compare/v1.5.0...v1.6.0-beta.1 diff --git a/Cargo.toml b/Cargo.toml index 284c8b8..bf86da8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "async-std" -version = "1.6.0" +version = "1.6.1" authors = [ "Stjepan Glavina ", "Yoshua Wuyts ", diff --git a/src/lib.rs b/src/lib.rs index e5b0438..6f76265 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -197,7 +197,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0" +//! version = "1.6.1" //! features = ["unstable"] //! ``` //! @@ -210,7 +210,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0" +//! version = "1.6.1" //! features = ["attributes"] //! ``` //! @@ -219,7 +219,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0" +//! version = "1.6.1" //! features = ["tokio02"] //! ``` //! @@ -228,7 +228,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0" +//! version = "1.6.1" //! default-features = false //! features = ["std"] //! ``` @@ -238,7 +238,7 @@ //! //! ```toml //! [dependencies.async-std] -//! version = "1.6.0" +//! version = "1.6.1" //! default-features = false //! features = ["alloc"] //! ```