From 96f3bf811c8730946102b41f3c52c4b5d1237bf6 Mon Sep 17 00:00:00 2001 From: jtnunley Date: Wed, 7 Sep 2022 13:33:56 -0700 Subject: [PATCH 1/6] Split the crate into io and timer features --- .github/workflows/ci.yml | 6 + Cargo.toml | 14 +- examples/linux-inotify.rs | 4 +- examples/linux-timerfd.rs | 4 +- examples/unix-signal.rs | 4 +- examples/windows-uds.rs | 4 +- src/driver.rs | 423 +++---- src/io.rs | 1468 +++++++++++++++++++++++ src/lib.rs | 1861 +---------------------------- src/{reactor.rs => reactor/io.rs} | 205 +--- src/reactor/mod.rs | 106 ++ src/reactor/timer.rs | 260 ++++ src/timer.rs | 385 ++++++ tests/async.rs | 11 +- tests/timer.rs | 2 + 15 files changed, 2551 insertions(+), 2206 deletions(-) create mode 100644 src/io.rs rename src/{reactor.rs => reactor/io.rs} (74%) create mode 100644 src/reactor/mod.rs create mode 100644 src/reactor/timer.rs create mode 100644 src/timer.rs diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 0e50e07..2729951 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -30,6 +30,9 @@ jobs: if: startsWith(matrix.rust, 'nightly') run: cargo check -Z features=dev_dep - run: cargo test + - run: cargo test --no-default-features + - run: cargo test --no-default-features --features io + - run: cargo test --no-default-features --features timer # Copied from: https://github.com/rust-lang/stacker/pull/19/files windows_gnu: @@ -98,6 +101,9 @@ jobs: - name: Install Rust run: rustup update ${{ matrix.rust }} && rustup default ${{ matrix.rust }} - run: cargo build + - run: cargo build --no-default-features + - run: cargo build --no-default-features --features io + - run: cargo build --no-default-features --features timer clippy: runs-on: ubuntu-latest diff --git a/Cargo.toml b/Cargo.toml index 91dcd50..6529cde 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -19,16 +19,22 @@ name = "io" harness = false [dependencies] -concurrent-queue = "1.2.2" +cfg-if = "1.0.0" +concurrent-queue = { version = "1.2.2", optional = true } futures-lite = "1.11.0" log = "0.4.11" once_cell = "1.4.1" parking = "2.0.0" -polling = "2.0.0" -slab = "0.4.2" -socket2 = { version = "0.4.2", features = ["all"] } +polling = { version = "2.0.0", optional = true } +slab = { version = "0.4.2", optional = true } +socket2 = { version = "0.4.2", features = ["all"], optional = true } waker-fn = "1.1.0" +[features] +default = ["io", "timer"] +io = ["polling", "slab", "socket2"] +timer = ["concurrent-queue"] + [build-dependencies] autocfg = "1" diff --git a/examples/linux-inotify.rs b/examples/linux-inotify.rs index c168f63..f846f87 100644 --- a/examples/linux-inotify.rs +++ b/examples/linux-inotify.rs @@ -6,7 +6,7 @@ //! cargo run --example linux-inotify //! ``` -#[cfg(target_os = "linux")] +#[cfg(all(feature = "io", target_os = "linux"))] fn main() -> std::io::Result<()> { use std::ffi::OsString; use std::io; @@ -51,7 +51,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(target_os = "linux"))] +#[cfg(not(all(feature = "io", target_os = "linux")))] fn main() { println!("This example works only on Linux!"); } diff --git a/examples/linux-timerfd.rs b/examples/linux-timerfd.rs index e57053b..045b2a4 100644 --- a/examples/linux-timerfd.rs +++ b/examples/linux-timerfd.rs @@ -6,7 +6,7 @@ //! cargo run --example linux-timerfd //! ``` -#[cfg(target_os = "linux")] +#[cfg(all(feature = "io", target_os = "linux"))] fn main() -> std::io::Result<()> { use std::io; use std::os::unix::io::AsRawFd; @@ -41,7 +41,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(target_os = "linux"))] +#[cfg(not(all(feature = "io", target_os = "linux")))] fn main() { println!("This example works only on Linux!"); } diff --git a/examples/unix-signal.rs b/examples/unix-signal.rs index e712893..ed485aa 100644 --- a/examples/unix-signal.rs +++ b/examples/unix-signal.rs @@ -6,7 +6,7 @@ //! cargo run --example unix-signal //! ``` -#[cfg(unix)] +#[cfg(all(feature = "io", unix))] fn main() -> std::io::Result<()> { use std::os::unix::{io::AsRawFd, net::UnixStream}; @@ -27,7 +27,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(unix))] +#[cfg(not(all(feature = "io", unix)))] fn main() { println!("This example works only on Unix systems!"); } diff --git a/examples/windows-uds.rs b/examples/windows-uds.rs index 0980d1b..44d97d7 100644 --- a/examples/windows-uds.rs +++ b/examples/windows-uds.rs @@ -6,7 +6,7 @@ //! cargo run --example windows-uds //! ``` -#[cfg(windows)] +#[cfg(all(feature = "io", windows))] fn main() -> std::io::Result<()> { use std::path::PathBuf; @@ -53,7 +53,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(windows))] +#[cfg(not(all(feature = "io", windows)))] fn main() { println!("This example works only on Windows!"); } diff --git a/src/driver.rs b/src/driver.rs index dd52621..83947e0 100644 --- a/src/driver.rs +++ b/src/driver.rs @@ -1,230 +1,253 @@ -use std::cell::Cell; -use std::future::Future; -use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; -use std::sync::Arc; -use std::task::{Context, Poll}; -use std::thread; -use std::time::{Duration, Instant}; - -use futures_lite::pin; -use once_cell::sync::Lazy; -use waker_fn::waker_fn; - -use crate::reactor::Reactor; - -/// Number of currently active `block_on()` invocations. -static BLOCK_ON_COUNT: AtomicUsize = AtomicUsize::new(0); - -/// Unparker for the "async-io" thread. -static UNPARKER: Lazy = Lazy::new(|| { - let (parker, unparker) = parking::pair(); - - // Spawn a helper thread driving the reactor. - // - // Note that this thread is not exactly necessary, it's only here to help push things - // forward if there are no `Parker`s around or if `Parker`s are just idling and never - // parking. - thread::Builder::new() - .name("async-io".to_string()) - .spawn(move || main_loop(parker)) - .expect("cannot spawn async-io thread"); - - unparker -}); - -/// Initializes the "async-io" thread. -pub(crate) fn init() { - Lazy::force(&UNPARKER); -} +#[cfg(any(feature = "io", feature = "timer"))] +mod driver_impl { + use std::cell::Cell; + use std::future::Future; + use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; + use std::sync::Arc; + use std::task::{Context, Poll}; + use std::thread; + use std::time::{Duration, Instant}; + + use futures_lite::pin; + use once_cell::sync::Lazy; + use waker_fn::waker_fn; + + use crate::reactor::Reactor; + + /// Number of currently active `block_on()` invocations. + static BLOCK_ON_COUNT: AtomicUsize = AtomicUsize::new(0); + + /// Unparker for the "async-io" thread. + static UNPARKER: Lazy = Lazy::new(|| { + let (parker, unparker) = parking::pair(); + + // Spawn a helper thread driving the reactor. + // + // Note that this thread is not exactly necessary, it's only here to help push things + // forward if there are no `Parker`s around or if `Parker`s are just idling and never + // parking. + thread::Builder::new() + .name("async-io".to_string()) + .spawn(move || main_loop(parker)) + .expect("cannot spawn async-io thread"); + + unparker + }); + + /// Initializes the "async-io" thread. + pub(crate) fn init() { + Lazy::force(&UNPARKER); + } -/// The main loop for the "async-io" thread. -fn main_loop(parker: parking::Parker) { - // The last observed reactor tick. - let mut last_tick = 0; - // Number of sleeps since this thread has called `react()`. - let mut sleeps = 0u64; - - loop { - let tick = Reactor::get().ticker(); - - if last_tick == tick { - let reactor_lock = if sleeps >= 10 { - // If no new ticks have occurred for a while, stop sleeping and spinning in - // this loop and just block on the reactor lock. - Some(Reactor::get().lock()) + /// The main loop for the "async-io" thread. + fn main_loop(parker: parking::Parker) { + // The last observed reactor tick. + let mut last_tick = 0; + // Number of sleeps since this thread has called `react()`. + let mut sleeps = 0u64; + + loop { + let tick = Reactor::get().ticker(); + + if last_tick == tick { + let reactor_lock = if sleeps >= 10 { + // If no new ticks have occurred for a while, stop sleeping and spinning in + // this loop and just block on the reactor lock. + Some(Reactor::get().lock()) + } else { + Reactor::get().try_lock() + }; + + if let Some(mut reactor_lock) = reactor_lock { + log::trace!("main_loop: waiting on I/O"); + reactor_lock.react(None).ok(); + last_tick = Reactor::get().ticker(); + sleeps = 0; + } } else { - Reactor::get().try_lock() - }; - - if let Some(mut reactor_lock) = reactor_lock { - log::trace!("main_loop: waiting on I/O"); - reactor_lock.react(None).ok(); - last_tick = Reactor::get().ticker(); - sleeps = 0; + last_tick = tick; } - } else { - last_tick = tick; - } - - if BLOCK_ON_COUNT.load(Ordering::SeqCst) > 0 { - // Exponential backoff from 50us to 10ms. - let delay_us = [50, 75, 100, 250, 500, 750, 1000, 2500, 5000] - .get(sleeps as usize) - .unwrap_or(&10_000); - - log::trace!("main_loop: sleeping for {} us", delay_us); - if parker.park_timeout(Duration::from_micros(*delay_us)) { - log::trace!("main_loop: notified"); - // If notified before timeout, reset the last tick and the sleep counter. - last_tick = Reactor::get().ticker(); - sleeps = 0; - } else { - sleeps += 1; + if BLOCK_ON_COUNT.load(Ordering::SeqCst) > 0 { + // Exponential backoff from 50us to 10ms. + let delay_us = [50, 75, 100, 250, 500, 750, 1000, 2500, 5000] + .get(sleeps as usize) + .unwrap_or(&10_000); + + log::trace!("main_loop: sleeping for {} us", delay_us); + if parker.park_timeout(Duration::from_micros(*delay_us)) { + log::trace!("main_loop: notified"); + + // If notified before timeout, reset the last tick and the sleep counter. + last_tick = Reactor::get().ticker(); + sleeps = 0; + } else { + sleeps += 1; + } } } } -} -/// Blocks the current thread on a future, processing I/O events when idle. -/// -/// # Examples -/// -/// ``` -/// use async_io::Timer; -/// use std::time::Duration; -/// -/// async_io::block_on(async { -/// // This timer will likely be processed by the current -/// // thread rather than the fallback "async-io" thread. -/// Timer::after(Duration::from_millis(1)).await; -/// }); -/// ``` -pub fn block_on(future: impl Future) -> T { - log::trace!("block_on()"); + pub(crate) fn block_on(future: impl Future) -> T { + log::trace!("block_on()"); - // Increment `BLOCK_ON_COUNT` so that the "async-io" thread becomes less aggressive. - BLOCK_ON_COUNT.fetch_add(1, Ordering::SeqCst); + // Increment `BLOCK_ON_COUNT` so that the "async-io" thread becomes less aggressive. + BLOCK_ON_COUNT.fetch_add(1, Ordering::SeqCst); - // Make sure to decrement `BLOCK_ON_COUNT` at the end and wake the "async-io" thread. - let _guard = CallOnDrop(|| { - BLOCK_ON_COUNT.fetch_sub(1, Ordering::SeqCst); - UNPARKER.unpark(); - }); + // Make sure to decrement `BLOCK_ON_COUNT` at the end and wake the "async-io" thread. + let _guard = CallOnDrop(|| { + BLOCK_ON_COUNT.fetch_sub(1, Ordering::SeqCst); + UNPARKER.unpark(); + }); - // Parker and unparker for notifying the current thread. - let (p, u) = parking::pair(); - // This boolean is set to `true` when the current thread is blocked on I/O. - let io_blocked = Arc::new(AtomicBool::new(false)); + // Parker and unparker for notifying the current thread. + let (p, u) = parking::pair(); + // This boolean is set to `true` when the current thread is blocked on I/O. + let io_blocked = Arc::new(AtomicBool::new(false)); - thread_local! { - // Indicates that the current thread is polling I/O, but not necessarily blocked on it. - static IO_POLLING: Cell = Cell::new(false); - } + thread_local! { + // Indicates that the current thread is polling I/O, but not necessarily blocked on it. + static IO_POLLING: Cell = Cell::new(false); + } - // Prepare the waker. - let waker = waker_fn({ - let io_blocked = io_blocked.clone(); - move || { - if u.unpark() { - // Check if waking from another thread and if currently blocked on I/O. - if !IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst) { - Reactor::get().notify(); + // Prepare the waker. + let waker = waker_fn({ + let io_blocked = io_blocked.clone(); + move || { + if u.unpark() { + // Check if waking from another thread and if currently blocked on I/O. + if !IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst) { + Reactor::get().notify(); + } } } - } - }); - let cx = &mut Context::from_waker(&waker); - pin!(future); - - loop { - // Poll the future. - if let Poll::Ready(t) = future.as_mut().poll(cx) { - log::trace!("block_on: completed"); - return t; - } - - // Check if a notification was received. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); - - // Try grabbing a lock on the reactor to process I/O events. - if let Some(mut reactor_lock) = Reactor::get().try_lock() { - // First let wakers know this parker is processing I/O events. - IO_POLLING.with(|io| io.set(true)); - let _guard = CallOnDrop(|| { - IO_POLLING.with(|io| io.set(false)); - }); - - // Process available I/O events. - reactor_lock.react(Some(Duration::from_secs(0))).ok(); + }); + let cx = &mut Context::from_waker(&waker); + pin!(future); + + loop { + // Poll the future. + if let Poll::Ready(t) = future.as_mut().poll(cx) { + log::trace!("block_on: completed"); + return t; } - continue; - } - // Try grabbing a lock on the reactor to wait on I/O. - if let Some(mut reactor_lock) = Reactor::get().try_lock() { - // Record the instant at which the lock was grabbed. - let start = Instant::now(); - - loop { - // First let wakers know this parker is blocked on I/O. - IO_POLLING.with(|io| io.set(true)); - io_blocked.store(true, Ordering::SeqCst); - let _guard = CallOnDrop(|| { - IO_POLLING.with(|io| io.set(false)); - io_blocked.store(false, Ordering::SeqCst); - }); - - // Check if a notification has been received before `io_blocked` was updated - // because in that case the reactor won't receive a wakeup. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); - break; - } + // Check if a notification was received. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); - // Wait for I/O events. - log::trace!("block_on: waiting on I/O"); - reactor_lock.react(None).ok(); + // Try grabbing a lock on the reactor to process I/O events. + if let Some(mut reactor_lock) = Reactor::get().try_lock() { + // First let wakers know this parker is processing I/O events. + IO_POLLING.with(|io| io.set(true)); + let _guard = CallOnDrop(|| { + IO_POLLING.with(|io| io.set(false)); + }); - // Check if a notification has been received. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); - break; + // Process available I/O events. + reactor_lock.react(Some(Duration::from_secs(0))).ok(); } + continue; + } - // Check if this thread been handling I/O events for a long time. - if start.elapsed() > Duration::from_micros(500) { - log::trace!("block_on: stops hogging the reactor"); - - // This thread is clearly processing I/O events for some other threads - // because it didn't get a notification yet. It's best to stop hogging the - // reactor and give other threads a chance to process I/O events for - // themselves. - drop(reactor_lock); - - // Unpark the "async-io" thread in case no other thread is ready to start - // processing I/O events. This way we prevent a potential latency spike. - UNPARKER.unpark(); - - // Wait for a notification. - p.park(); - break; + // Try grabbing a lock on the reactor to wait on I/O. + if let Some(mut reactor_lock) = Reactor::get().try_lock() { + // Record the instant at which the lock was grabbed. + let start = Instant::now(); + + loop { + // First let wakers know this parker is blocked on I/O. + IO_POLLING.with(|io| io.set(true)); + io_blocked.store(true, Ordering::SeqCst); + let _guard = CallOnDrop(|| { + IO_POLLING.with(|io| io.set(false)); + io_blocked.store(false, Ordering::SeqCst); + }); + + // Check if a notification has been received before `io_blocked` was updated + // because in that case the reactor won't receive a wakeup. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); + break; + } + + // Wait for I/O events. + log::trace!("block_on: waiting on I/O"); + reactor_lock.react(None).ok(); + + // Check if a notification has been received. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); + break; + } + + // Check if this thread been handling I/O events for a long time. + if start.elapsed() > Duration::from_micros(500) { + log::trace!("block_on: stops hogging the reactor"); + + // This thread is clearly processing I/O events for some other threads + // because it didn't get a notification yet. It's best to stop hogging the + // reactor and give other threads a chance to process I/O events for + // themselves. + drop(reactor_lock); + + // Unpark the "async-io" thread in case no other thread is ready to start + // processing I/O events. This way we prevent a potential latency spike. + UNPARKER.unpark(); + + // Wait for a notification. + p.park(); + break; + } } + } else { + // Wait for an actual notification. + log::trace!("block_on: sleep until notification"); + p.park(); } - } else { - // Wait for an actual notification. - log::trace!("block_on: sleep until notification"); - p.park(); + } + } + + /// Runs a closure when dropped. + struct CallOnDrop(F); + + impl Drop for CallOnDrop { + fn drop(&mut self) { + (self.0)(); } } } -/// Runs a closure when dropped. -struct CallOnDrop(F); +#[cfg(not(any(feature = "io", feature = "timer")))] +mod driver_impl { + use std::future::Future; -impl Drop for CallOnDrop { - fn drop(&mut self) { - (self.0)(); + #[inline] + pub(crate) fn block_on(future: impl Future) -> T { + futures_lite::future::block_on(future) } } + +use std::future::Future; + +/// Blocks the current thread on a future, processing I/O events when idle. +/// +/// # Examples +/// +#[cfg_attr(feature = "timer", doc = "```")] +#[cfg_attr(not(feature = "timer"), doc = "```no_compile")] +/// use async_io::Timer; +/// use std::time::Duration; +/// +/// async_io::block_on(async { +/// // This timer will likely be processed by the current +/// // thread rather than the fallback "async-io" thread. +/// Timer::after(Duration::from_millis(1)).await; +/// }); +/// ``` +pub fn block_on(future: impl Future) -> T { + driver_impl::block_on(future) +} + +#[cfg(any(feature = "io", feature = "timer"))] +pub(crate) use driver_impl::init; diff --git a/src/io.rs b/src/io.rs new file mode 100644 index 0000000..35e28a8 --- /dev/null +++ b/src/io.rs @@ -0,0 +1,1468 @@ +use std::convert::TryFrom; +use std::future::Future; +use std::io::{self, IoSlice, IoSliceMut, Read, Write}; +use std::net::{SocketAddr, TcpListener, TcpStream, UdpSocket}; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use futures_lite::io::{AsyncRead, AsyncWrite}; +use futures_lite::stream::{self, Stream}; +use futures_lite::{future, pin, ready}; +use socket2::{Domain, Protocol, SockAddr, Socket, Type}; + +use crate::reactor::{Reactor, Readable, ReadableOwned, Source, Writable, WritableOwned}; + +#[cfg(all(not(async_io_no_io_safety), unix))] +use std::os::unix::io::{AsFd, BorrowedFd, OwnedFd}; +#[cfg(unix)] +use std::{ + os::unix::io::{AsRawFd, RawFd}, + os::unix::net::{SocketAddr as UnixSocketAddr, UnixDatagram, UnixListener, UnixStream}, + path::Path, +}; + +#[cfg(windows)] +use std::os::windows::io::{AsRawSocket, RawSocket}; +#[cfg(all(not(async_io_no_io_safety), windows))] +use std::os::windows::io::{AsSocket, BorrowedSocket, OwnedSocket}; + +/// Async adapter for I/O types. +/// +/// This type puts an I/O handle into non-blocking mode, registers it in +/// [epoll]/[kqueue]/[event ports]/[wepoll], and then provides an async interface for it. +/// +/// [epoll]: https://en.wikipedia.org/wiki/Epoll +/// [kqueue]: https://en.wikipedia.org/wiki/Kqueue +/// [event ports]: https://illumos.org/man/port_create +/// [wepoll]: https://github.com/piscisaureus/wepoll +/// +/// # Caveats +/// +/// [`Async`] is a low-level primitive, and as such it comes with some caveats. +/// +/// For higher-level primitives built on top of [`Async`], look into [`async-net`] or +/// [`async-process`] (on Unix). +/// +/// [`async-net`]: https://github.com/smol-rs/async-net +/// [`async-process`]: https://github.com/smol-rs/async-process +/// +/// ### Supported types +/// +/// [`Async`] supports all networking types, as well as some OS-specific file descriptors like +/// [timerfd] and [inotify]. +/// +/// However, do not use [`Async`] with types like [`File`][`std::fs::File`], +/// [`Stdin`][`std::io::Stdin`], [`Stdout`][`std::io::Stdout`], or [`Stderr`][`std::io::Stderr`] +/// because all operating systems have issues with them when put in non-blocking mode. +/// +/// [timerfd]: https://github.com/smol-rs/async-io/blob/master/examples/linux-timerfd.rs +/// [inotify]: https://github.com/smol-rs/async-io/blob/master/examples/linux-inotify.rs +/// +/// ### Concurrent I/O +/// +/// Note that [`&Async`][`Async`] implements [`AsyncRead`] and [`AsyncWrite`] if `&T` +/// implements those traits, which means tasks can concurrently read and write using shared +/// references. +/// +/// But there is a catch: only one task can read a time, and only one task can write at a time. It +/// is okay to have two tasks where one is reading and the other is writing at the same time, but +/// it is not okay to have two tasks reading at the same time or writing at the same time. If you +/// try to do that, conflicting tasks will just keep waking each other in turn, thus wasting CPU +/// time. +/// +/// Besides [`AsyncRead`] and [`AsyncWrite`], this caveat also applies to +/// [`poll_readable()`][`Async::poll_readable()`] and +/// [`poll_writable()`][`Async::poll_writable()`]. +/// +/// However, any number of tasks can be concurrently calling other methods like +/// [`readable()`][`Async::readable()`] or [`read_with()`][`Async::read_with()`]. +/// +/// ### Closing +/// +/// Closing the write side of [`Async`] with [`close()`][`futures_lite::AsyncWriteExt::close()`] +/// simply flushes. If you want to shutdown a TCP or Unix socket, use +/// [`Shutdown`][`std::net::Shutdown`]. +/// +/// # Examples +/// +/// Connect to a server and echo incoming messages back to the server: +/// +/// ```no_run +/// use async_io::Async; +/// use futures_lite::io; +/// use std::net::TcpStream; +/// +/// # futures_lite::future::block_on(async { +/// // Connect to a local server. +/// let stream = Async::::connect(([127, 0, 0, 1], 8000)).await?; +/// +/// // Echo all messages from the read side of the stream into the write side. +/// io::copy(&stream, &stream).await?; +/// # std::io::Result::Ok(()) }); +/// ``` +/// +/// You can use either predefined async methods or wrap blocking I/O operations in +/// [`Async::read_with()`], [`Async::read_with_mut()`], [`Async::write_with()`], and +/// [`Async::write_with_mut()`]: +/// +/// ```no_run +/// use async_io::Async; +/// use std::net::TcpListener; +/// +/// # futures_lite::future::block_on(async { +/// let listener = Async::::bind(([127, 0, 0, 1], 0))?; +/// +/// // These two lines are equivalent: +/// let (stream, addr) = listener.accept().await?; +/// let (stream, addr) = listener.read_with(|inner| inner.accept()).await?; +/// # std::io::Result::Ok(()) }); +/// ``` +#[derive(Debug)] +pub struct Async { + /// A source registered in the reactor. + pub(crate) source: Arc, + + /// The inner I/O handle. + io: Option, +} + +impl Unpin for Async {} + +#[cfg(unix)] +impl Async { + /// Creates an async I/O handle. + /// + /// This method will put the handle in non-blocking mode and register it in + /// [epoll]/[kqueue]/[event ports]/[wepoll]. + /// + /// On Unix systems, the handle must implement `AsRawFd`, while on Windows it must implement + /// `AsRawSocket`. + /// + /// [epoll]: https://en.wikipedia.org/wiki/Epoll + /// [kqueue]: https://en.wikipedia.org/wiki/Kqueue + /// [event ports]: https://illumos.org/man/port_create + /// [wepoll]: https://github.com/piscisaureus/wepoll + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::{SocketAddr, TcpListener}; + /// + /// # futures_lite::future::block_on(async { + /// let listener = TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0)))?; + /// let listener = Async::new(listener)?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn new(io: T) -> io::Result> { + let fd = io.as_raw_fd(); + + // Put the file descriptor in non-blocking mode. + unsafe { + let mut res = libc::fcntl(fd, libc::F_GETFL); + if res != -1 { + res = libc::fcntl(fd, libc::F_SETFL, res | libc::O_NONBLOCK); + } + if res == -1 { + return Err(io::Error::last_os_error()); + } + } + + Ok(Async { + source: Reactor::get().insert_io(fd)?, + io: Some(io), + }) + } +} + +#[cfg(unix)] +impl AsRawFd for Async { + fn as_raw_fd(&self) -> RawFd { + self.source.raw + } +} + +#[cfg(all(not(async_io_no_io_safety), unix))] +impl AsFd for Async { + fn as_fd(&self) -> BorrowedFd<'_> { + self.get_ref().as_fd() + } +} + +#[cfg(all(not(async_io_no_io_safety), unix))] +impl> TryFrom for Async { + type Error = io::Error; + + fn try_from(value: OwnedFd) -> Result { + Async::new(value.into()) + } +} + +#[cfg(all(not(async_io_no_io_safety), unix))] +impl> TryFrom> for OwnedFd { + type Error = io::Error; + + fn try_from(value: Async) -> Result { + value.into_inner().map(Into::into) + } +} + +#[cfg(windows)] +impl Async { + /// Creates an async I/O handle. + /// + /// This method will put the handle in non-blocking mode and register it in + /// [epoll]/[kqueue]/[event ports]/[wepoll]. + /// + /// On Unix systems, the handle must implement `AsRawFd`, while on Windows it must implement + /// `AsRawSocket`. + /// + /// [epoll]: https://en.wikipedia.org/wiki/Epoll + /// [kqueue]: https://en.wikipedia.org/wiki/Kqueue + /// [event ports]: https://illumos.org/man/port_create + /// [wepoll]: https://github.com/piscisaureus/wepoll + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::{SocketAddr, TcpListener}; + /// + /// # futures_lite::future::block_on(async { + /// let listener = TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0)))?; + /// let listener = Async::new(listener)?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn new(io: T) -> io::Result> { + let sock = io.as_raw_socket(); + + // Put the socket in non-blocking mode. + + use winapi::ctypes; + use winapi::um::winsock2; + + let mut nonblocking = true as ctypes::c_ulong; + let res = unsafe { + winsock2::ioctlsocket( + sock as winsock2::SOCKET, + winsock2::FIONBIO, + &mut nonblocking, + ) + }; + if res != 0 { + return Err(io::Error::last_os_error()); + } + + Ok(Async { + source: Reactor::get().insert_io(sock)?, + io: Some(io), + }) + } +} + +#[cfg(windows)] +impl AsRawSocket for Async { + fn as_raw_socket(&self) -> RawSocket { + self.source.raw + } +} + +#[cfg(all(not(async_io_no_io_safety), windows))] +impl AsSocket for Async { + fn as_socket(&self) -> BorrowedSocket<'_> { + self.get_ref().as_socket() + } +} + +#[cfg(all(not(async_io_no_io_safety), windows))] +impl> TryFrom for Async { + type Error = io::Error; + + fn try_from(value: OwnedSocket) -> Result { + Async::new(value.into()) + } +} + +#[cfg(all(not(async_io_no_io_safety), windows))] +impl> TryFrom> for OwnedSocket { + type Error = io::Error; + + fn try_from(value: Async) -> Result { + value.into_inner().map(Into::into) + } +} + +impl Async { + /// Gets a reference to the inner I/O handle. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// let inner = listener.get_ref(); + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn get_ref(&self) -> &T { + self.io.as_ref().unwrap() + } + + /// Gets a mutable reference to the inner I/O handle. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// let inner = listener.get_mut(); + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn get_mut(&mut self) -> &mut T { + self.io.as_mut().unwrap() + } + + /// Unwraps the inner I/O handle. + /// + /// This method will **not** put the I/O handle back into blocking mode. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// let inner = listener.into_inner()?; + /// + /// // Put the listener back into blocking mode. + /// inner.set_nonblocking(false)?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn into_inner(mut self) -> io::Result { + let io = self.io.take().unwrap(); + Reactor::get().remove_io(&self.source)?; + Ok(io) + } + + /// Waits until the I/O handle is readable. + /// + /// This method completes when a read operation on this I/O handle wouldn't block. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// + /// // Wait until a client can be accepted. + /// listener.readable().await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn readable(&self) -> Readable<'_, T> { + Source::readable(self) + } + + /// Waits until the I/O handle is readable. + /// + /// This method completes when a read operation on this I/O handle wouldn't block. + pub fn readable_owned(self: Arc) -> ReadableOwned { + Source::readable_owned(self) + } + + /// Waits until the I/O handle is writable. + /// + /// This method completes when a write operation on this I/O handle wouldn't block. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::{TcpStream, ToSocketAddrs}; + /// + /// # futures_lite::future::block_on(async { + /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); + /// let stream = Async::::connect(addr).await?; + /// + /// // Wait until the stream is writable. + /// stream.writable().await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn writable(&self) -> Writable<'_, T> { + Source::writable(self) + } + + /// Waits until the I/O handle is writable. + /// + /// This method completes when a write operation on this I/O handle wouldn't block. + pub fn writable_owned(self: Arc) -> WritableOwned { + Source::writable_owned(self) + } + + /// Polls the I/O handle for readability. + /// + /// When this method returns [`Poll::Ready`], that means the OS has delivered an event + /// indicating readability since the last time this task has called the method and received + /// [`Poll::Pending`]. + /// + /// # Caveats + /// + /// Two different tasks should not call this method concurrently. Otherwise, conflicting tasks + /// will just keep waking each other in turn, thus wasting CPU time. + /// + /// Note that the [`AsyncRead`] implementation for [`Async`] also uses this method. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use futures_lite::future; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// + /// // Wait until a client can be accepted. + /// future::poll_fn(|cx| listener.poll_readable(cx)).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn poll_readable(&self, cx: &mut Context<'_>) -> Poll> { + self.source.poll_readable(cx) + } + + /// Polls the I/O handle for writability. + /// + /// When this method returns [`Poll::Ready`], that means the OS has delivered an event + /// indicating writability since the last time this task has called the method and received + /// [`Poll::Pending`]. + /// + /// # Caveats + /// + /// Two different tasks should not call this method concurrently. Otherwise, conflicting tasks + /// will just keep waking each other in turn, thus wasting CPU time. + /// + /// Note that the [`AsyncWrite`] implementation for [`Async`] also uses this method. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use futures_lite::future; + /// use std::net::{TcpStream, ToSocketAddrs}; + /// + /// # futures_lite::future::block_on(async { + /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); + /// let stream = Async::::connect(addr).await?; + /// + /// // Wait until the stream is writable. + /// future::poll_fn(|cx| stream.poll_writable(cx)).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn poll_writable(&self, cx: &mut Context<'_>) -> Poll> { + self.source.poll_writable(cx) + } + + /// Performs a read operation asynchronously. + /// + /// The I/O handle is registered in the reactor and put in non-blocking mode. This method + /// invokes the `op` closure in a loop until it succeeds or returns an error other than + /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS + /// sends a notification that the I/O handle is readable. + /// + /// The closure receives a shared reference to the I/O handle. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// + /// // Accept a new client asynchronously. + /// let (stream, addr) = listener.read_with(|l| l.accept()).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn read_with(&self, op: impl FnMut(&T) -> io::Result) -> io::Result { + let mut op = op; + loop { + match op(self.get_ref()) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return res, + } + optimistic(self.readable()).await?; + } + } + + /// Performs a read operation asynchronously. + /// + /// The I/O handle is registered in the reactor and put in non-blocking mode. This method + /// invokes the `op` closure in a loop until it succeeds or returns an error other than + /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS + /// sends a notification that the I/O handle is readable. + /// + /// The closure receives a mutable reference to the I/O handle. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// + /// // Accept a new client asynchronously. + /// let (stream, addr) = listener.read_with_mut(|l| l.accept()).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn read_with_mut( + &mut self, + op: impl FnMut(&mut T) -> io::Result, + ) -> io::Result { + let mut op = op; + loop { + match op(self.get_mut()) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return res, + } + optimistic(self.readable()).await?; + } + } + + /// Performs a write operation asynchronously. + /// + /// The I/O handle is registered in the reactor and put in non-blocking mode. This method + /// invokes the `op` closure in a loop until it succeeds or returns an error other than + /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS + /// sends a notification that the I/O handle is writable. + /// + /// The closure receives a shared reference to the I/O handle. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// socket.get_ref().connect("127.0.0.1:9000")?; + /// + /// let msg = b"hello"; + /// let len = socket.write_with(|s| s.send(msg)).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn write_with(&self, op: impl FnMut(&T) -> io::Result) -> io::Result { + let mut op = op; + loop { + match op(self.get_ref()) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return res, + } + optimistic(self.writable()).await?; + } + } + + /// Performs a write operation asynchronously. + /// + /// The I/O handle is registered in the reactor and put in non-blocking mode. This method + /// invokes the `op` closure in a loop until it succeeds or returns an error other than + /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS + /// sends a notification that the I/O handle is writable. + /// + /// The closure receives a mutable reference to the I/O handle. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let mut socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// socket.get_ref().connect("127.0.0.1:9000")?; + /// + /// let msg = b"hello"; + /// let len = socket.write_with_mut(|s| s.send(msg)).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn write_with_mut( + &mut self, + op: impl FnMut(&mut T) -> io::Result, + ) -> io::Result { + let mut op = op; + loop { + match op(self.get_mut()) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return res, + } + optimistic(self.writable()).await?; + } + } +} + +impl AsRef for Async { + fn as_ref(&self) -> &T { + self.get_ref() + } +} + +impl AsMut for Async { + fn as_mut(&mut self) -> &mut T { + self.get_mut() + } +} + +impl Drop for Async { + fn drop(&mut self) { + if self.io.is_some() { + // Deregister and ignore errors because destructors should not panic. + Reactor::get().remove_io(&self.source).ok(); + + // Drop the I/O handle to close it. + self.io.take(); + } + } +} + +impl AsyncRead for Async { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + loop { + match (&mut *self).get_mut().read(buf) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_readable(cx))?; + } + } + + fn poll_read_vectored( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { + loop { + match (&mut *self).get_mut().read_vectored(bufs) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_readable(cx))?; + } + } +} + +impl AsyncRead for &Async +where + for<'a> &'a T: Read, +{ + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + loop { + match (*self).get_ref().read(buf) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_readable(cx))?; + } + } + + fn poll_read_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &mut [IoSliceMut<'_>], + ) -> Poll> { + loop { + match (*self).get_ref().read_vectored(bufs) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_readable(cx))?; + } + } +} + +impl AsyncWrite for Async { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + loop { + match (&mut *self).get_mut().write(buf) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_write_vectored( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { + loop { + match (&mut *self).get_mut().write_vectored(bufs) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + loop { + match (&mut *self).get_mut().flush() { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} + +impl AsyncWrite for &Async +where + for<'a> &'a T: Write, +{ + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + loop { + match (*self).get_ref().write(buf) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_write_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[IoSlice<'_>], + ) -> Poll> { + loop { + match (*self).get_ref().write_vectored(bufs) { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + loop { + match (*self).get_ref().flush() { + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + res => return Poll::Ready(res), + } + ready!(self.poll_writable(cx))?; + } + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_flush(cx) + } +} + +impl Async { + /// Creates a TCP listener bound to the specified address. + /// + /// Binding with port number 0 will request an available port from the OS. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; + /// println!("Listening on {}", listener.get_ref().local_addr()?); + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn bind>(addr: A) -> io::Result> { + let addr = addr.into(); + Async::new(TcpListener::bind(addr)?) + } + + /// Accepts a new incoming TCP connection. + /// + /// When a connection is established, it will be returned as a TCP stream together with its + /// remote address. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 8000))?; + /// let (stream, addr) = listener.accept().await?; + /// println!("Accepted client: {}", addr); + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn accept(&self) -> io::Result<(Async, SocketAddr)> { + let (stream, addr) = self.read_with(|io| io.accept()).await?; + Ok((Async::new(stream)?, addr)) + } + + /// Returns a stream of incoming TCP connections. + /// + /// The stream is infinite, i.e. it never stops with a [`None`]. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use futures_lite::{pin, stream::StreamExt}; + /// use std::net::TcpListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind(([127, 0, 0, 1], 8000))?; + /// let incoming = listener.incoming(); + /// pin!(incoming); + /// + /// while let Some(stream) = incoming.next().await { + /// let stream = stream?; + /// println!("Accepted client: {}", stream.get_ref().peer_addr()?); + /// } + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn incoming(&self) -> impl Stream>> + Send + '_ { + stream::unfold(self, |listener| async move { + let res = listener.accept().await.map(|(stream, _)| stream); + Some((res, listener)) + }) + } +} + +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(listener: std::net::TcpListener) -> io::Result { + Async::new(listener) + } +} + +impl Async { + /// Creates a TCP connection to the specified address. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::{TcpStream, ToSocketAddrs}; + /// + /// # futures_lite::future::block_on(async { + /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); + /// let stream = Async::::connect(addr).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn connect>(addr: A) -> io::Result> { + // Begin async connect. + let addr = addr.into(); + let domain = Domain::for_address(addr); + let socket = connect(addr.into(), domain, Some(Protocol::TCP))?; + let stream = Async::new(TcpStream::from(socket))?; + + // The stream becomes writable when connected. + stream.writable().await?; + + // Check if there was an error while connecting. + match stream.get_ref().take_error()? { + None => Ok(stream), + Some(err) => Err(err), + } + } + + /// Reads data from the stream without removing it from the buffer. + /// + /// Returns the number of bytes read. Successive calls of this method read the same data. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use futures_lite::{io::AsyncWriteExt, stream::StreamExt}; + /// use std::net::{TcpStream, ToSocketAddrs}; + /// + /// # futures_lite::future::block_on(async { + /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); + /// let mut stream = Async::::connect(addr).await?; + /// + /// stream + /// .write_all(b"GET / HTTP/1.1\r\nHost: example.com\r\n\r\n") + /// .await?; + /// + /// let mut buf = [0u8; 1024]; + /// let len = stream.peek(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn peek(&self, buf: &mut [u8]) -> io::Result { + self.read_with(|io| io.peek(buf)).await + } +} + +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(stream: std::net::TcpStream) -> io::Result { + Async::new(stream) + } +} + +impl Async { + /// Creates a UDP socket bound to the specified address. + /// + /// Binding with port number 0 will request an available port from the OS. + /// + /// # Examples + /// + /// ``` + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 0))?; + /// println!("Bound to {}", socket.get_ref().local_addr()?); + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn bind>(addr: A) -> io::Result> { + let addr = addr.into(); + Async::new(UdpSocket::bind(addr)?) + } + + /// Receives a single datagram message. + /// + /// Returns the number of bytes read and the address the message came from. + /// + /// This method must be called with a valid byte slice of sufficient size to hold the message. + /// If the message is too long to fit, excess bytes may get discarded. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// + /// let mut buf = [0u8; 1024]; + /// let (len, addr) = socket.recv_from(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { + self.read_with(|io| io.recv_from(buf)).await + } + + /// Receives a single datagram message without removing it from the queue. + /// + /// Returns the number of bytes read and the address the message came from. + /// + /// This method must be called with a valid byte slice of sufficient size to hold the message. + /// If the message is too long to fit, excess bytes may get discarded. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// + /// let mut buf = [0u8; 1024]; + /// let (len, addr) = socket.peek_from(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn peek_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { + self.read_with(|io| io.peek_from(buf)).await + } + + /// Sends data to the specified address. + /// + /// Returns the number of bytes writen. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 0))?; + /// let addr = socket.get_ref().local_addr()?; + /// + /// let msg = b"hello"; + /// let len = socket.send_to(msg, addr).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn send_to>(&self, buf: &[u8], addr: A) -> io::Result { + let addr = addr.into(); + self.write_with(|io| io.send_to(buf, addr)).await + } + + /// Receives a single datagram message from the connected peer. + /// + /// Returns the number of bytes read. + /// + /// This method must be called with a valid byte slice of sufficient size to hold the message. + /// If the message is too long to fit, excess bytes may get discarded. + /// + /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// socket.get_ref().connect("127.0.0.1:9000")?; + /// + /// let mut buf = [0u8; 1024]; + /// let len = socket.recv(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn recv(&self, buf: &mut [u8]) -> io::Result { + self.read_with(|io| io.recv(buf)).await + } + + /// Receives a single datagram message from the connected peer without removing it from the + /// queue. + /// + /// Returns the number of bytes read and the address the message came from. + /// + /// This method must be called with a valid byte slice of sufficient size to hold the message. + /// If the message is too long to fit, excess bytes may get discarded. + /// + /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// socket.get_ref().connect("127.0.0.1:9000")?; + /// + /// let mut buf = [0u8; 1024]; + /// let len = socket.peek(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn peek(&self, buf: &mut [u8]) -> io::Result { + self.read_with(|io| io.peek(buf)).await + } + + /// Sends data to the connected peer. + /// + /// Returns the number of bytes written. + /// + /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::net::UdpSocket; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; + /// socket.get_ref().connect("127.0.0.1:9000")?; + /// + /// let msg = b"hello"; + /// let len = socket.send(msg).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn send(&self, buf: &[u8]) -> io::Result { + self.write_with(|io| io.send(buf)).await + } +} + +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(socket: std::net::UdpSocket) -> io::Result { + Async::new(socket) + } +} + +#[cfg(unix)] +impl Async { + /// Creates a UDS listener bound to the specified path. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind("/tmp/socket")?; + /// println!("Listening on {:?}", listener.get_ref().local_addr()?); + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn bind>(path: P) -> io::Result> { + let path = path.as_ref().to_owned(); + Async::new(UnixListener::bind(path)?) + } + + /// Accepts a new incoming UDS stream connection. + /// + /// When a connection is established, it will be returned as a stream together with its remote + /// address. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind("/tmp/socket")?; + /// let (stream, addr) = listener.accept().await?; + /// println!("Accepted client: {:?}", addr); + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn accept(&self) -> io::Result<(Async, UnixSocketAddr)> { + let (stream, addr) = self.read_with(|io| io.accept()).await?; + Ok((Async::new(stream)?, addr)) + } + + /// Returns a stream of incoming UDS connections. + /// + /// The stream is infinite, i.e. it never stops with a [`None`] item. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use futures_lite::{pin, stream::StreamExt}; + /// use std::os::unix::net::UnixListener; + /// + /// # futures_lite::future::block_on(async { + /// let listener = Async::::bind("/tmp/socket")?; + /// let incoming = listener.incoming(); + /// pin!(incoming); + /// + /// while let Some(stream) = incoming.next().await { + /// let stream = stream?; + /// println!("Accepted client: {:?}", stream.get_ref().peer_addr()?); + /// } + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn incoming(&self) -> impl Stream>> + Send + '_ { + stream::unfold(self, |listener| async move { + let res = listener.accept().await.map(|(stream, _)| stream); + Some((res, listener)) + }) + } +} + +#[cfg(unix)] +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(listener: std::os::unix::net::UnixListener) -> io::Result { + Async::new(listener) + } +} + +#[cfg(unix)] +impl Async { + /// Creates a UDS stream connected to the specified path. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixStream; + /// + /// # futures_lite::future::block_on(async { + /// let stream = Async::::connect("/tmp/socket").await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn connect>(path: P) -> io::Result> { + // Begin async connect. + let socket = connect(SockAddr::unix(path)?, Domain::UNIX, None)?; + let stream = Async::new(UnixStream::from(socket))?; + + // The stream becomes writable when connected. + stream.writable().await?; + + // On Linux, it appears the socket may become writable even when connecting fails, so we + // must do an extra check here and see if the peer address is retrievable. + stream.get_ref().peer_addr()?; + Ok(stream) + } + + /// Creates an unnamed pair of connected UDS stream sockets. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixStream; + /// + /// # futures_lite::future::block_on(async { + /// let (stream1, stream2) = Async::::pair()?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn pair() -> io::Result<(Async, Async)> { + let (stream1, stream2) = UnixStream::pair()?; + Ok((Async::new(stream1)?, Async::new(stream2)?)) + } +} + +#[cfg(unix)] +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(stream: std::os::unix::net::UnixStream) -> io::Result { + Async::new(stream) + } +} + +#[cfg(unix)] +impl Async { + /// Creates a UDS datagram socket bound to the specified path. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind("/tmp/socket")?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn bind>(path: P) -> io::Result> { + let path = path.as_ref().to_owned(); + Async::new(UnixDatagram::bind(path)?) + } + + /// Creates a UDS datagram socket not bound to any address. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::unbound()?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn unbound() -> io::Result> { + Async::new(UnixDatagram::unbound()?) + } + + /// Creates an unnamed pair of connected Unix datagram sockets. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let (socket1, socket2) = Async::::pair()?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub fn pair() -> io::Result<(Async, Async)> { + let (socket1, socket2) = UnixDatagram::pair()?; + Ok((Async::new(socket1)?, Async::new(socket2)?)) + } + + /// Receives data from the socket. + /// + /// Returns the number of bytes read and the address the message came from. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind("/tmp/socket")?; + /// + /// let mut buf = [0u8; 1024]; + /// let (len, addr) = socket.recv_from(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, UnixSocketAddr)> { + self.read_with(|io| io.recv_from(buf)).await + } + + /// Sends data to the specified address. + /// + /// Returns the number of bytes written. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::unbound()?; + /// + /// let msg = b"hello"; + /// let addr = "/tmp/socket"; + /// let len = socket.send_to(msg, addr).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn send_to>(&self, buf: &[u8], path: P) -> io::Result { + self.write_with(|io| io.send_to(buf, &path)).await + } + + /// Receives data from the connected peer. + /// + /// Returns the number of bytes read and the address the message came from. + /// + /// The [`connect`][`UnixDatagram::connect()`] method connects this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind("/tmp/socket1")?; + /// socket.get_ref().connect("/tmp/socket2")?; + /// + /// let mut buf = [0u8; 1024]; + /// let len = socket.recv(&mut buf).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn recv(&self, buf: &mut [u8]) -> io::Result { + self.read_with(|io| io.recv(buf)).await + } + + /// Sends data to the connected peer. + /// + /// Returns the number of bytes written. + /// + /// The [`connect`][`UnixDatagram::connect()`] method connects this socket to a remote address. + /// This method will fail if the socket is not connected. + /// + /// # Examples + /// + /// ```no_run + /// use async_io::Async; + /// use std::os::unix::net::UnixDatagram; + /// + /// # futures_lite::future::block_on(async { + /// let socket = Async::::bind("/tmp/socket1")?; + /// socket.get_ref().connect("/tmp/socket2")?; + /// + /// let msg = b"hello"; + /// let len = socket.send(msg).await?; + /// # std::io::Result::Ok(()) }); + /// ``` + pub async fn send(&self, buf: &[u8]) -> io::Result { + self.write_with(|io| io.send(buf)).await + } +} + +#[cfg(unix)] +impl TryFrom for Async { + type Error = io::Error; + + fn try_from(socket: std::os::unix::net::UnixDatagram) -> io::Result { + Async::new(socket) + } +} + +/// Polls a future once, waits for a wakeup, and then optimistically assumes the future is ready. +async fn optimistic(fut: impl Future>) -> io::Result<()> { + let mut polled = false; + pin!(fut); + + future::poll_fn(|cx| { + if !polled { + polled = true; + fut.as_mut().poll(cx) + } else { + Poll::Ready(Ok(())) + } + }) + .await +} + +fn connect(addr: SockAddr, domain: Domain, protocol: Option) -> io::Result { + let sock_type = Type::STREAM; + #[cfg(any( + target_os = "android", + target_os = "dragonfly", + target_os = "freebsd", + target_os = "fuchsia", + target_os = "illumos", + target_os = "linux", + target_os = "netbsd", + target_os = "openbsd" + ))] + // If we can, set nonblocking at socket creation for unix + let sock_type = sock_type.nonblocking(); + // This automatically handles cloexec on unix, no_inherit on windows and nosigpipe on macos + let socket = Socket::new(domain, sock_type, protocol)?; + #[cfg(not(any( + target_os = "android", + target_os = "dragonfly", + target_os = "freebsd", + target_os = "fuchsia", + target_os = "illumos", + target_os = "linux", + target_os = "netbsd", + target_os = "openbsd" + )))] + // If the current platform doesn't support nonblocking at creation, enable it after creation + socket.set_nonblocking(true)?; + match socket.connect(&addr) { + Ok(_) => {} + #[cfg(unix)] + Err(err) if err.raw_os_error() == Some(libc::EINPROGRESS) => {} + Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} + Err(err) => return Err(err), + } + Ok(socket) +} diff --git a/src/lib.rs b/src/lib.rs index 5b2de42..37e64af 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -35,7 +35,8 @@ //! //! Connect to `example.com:80`, or time out after 10 seconds. //! -//! ``` +#![cfg_attr(all(feature = "timer", feature = "io"), doc = "```")] +#![cfg_attr(not(all(feature = "timer", feature = "io")), doc = "```no_compile")] //! use async_io::{Async, Timer}; //! use futures_lite::{future::FutureExt, io}; //! @@ -55,1856 +56,18 @@ #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms)] -use std::convert::TryFrom; -use std::future::Future; -use std::io::{self, IoSlice, IoSliceMut, Read, Write}; -use std::net::{SocketAddr, TcpListener, TcpStream, UdpSocket}; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll, Waker}; -use std::time::{Duration, Instant}; - -#[cfg(all(not(async_io_no_io_safety), unix))] -use std::os::unix::io::{AsFd, BorrowedFd, OwnedFd}; -#[cfg(unix)] -use std::{ - os::unix::io::{AsRawFd, RawFd}, - os::unix::net::{SocketAddr as UnixSocketAddr, UnixDatagram, UnixListener, UnixStream}, - path::Path, -}; - -#[cfg(windows)] -use std::os::windows::io::{AsRawSocket, RawSocket}; -#[cfg(all(not(async_io_no_io_safety), windows))] -use std::os::windows::io::{AsSocket, BorrowedSocket, OwnedSocket}; - -use futures_lite::io::{AsyncRead, AsyncWrite}; -use futures_lite::stream::{self, Stream}; -use futures_lite::{future, pin, ready}; -use socket2::{Domain, Protocol, SockAddr, Socket, Type}; - -use crate::reactor::{Reactor, Source}; - mod driver; +#[cfg(feature = "io")] +mod io; +#[cfg(any(feature = "io", feature = "timer"))] mod reactor; +#[cfg(feature = "timer")] +mod timer; pub use driver::block_on; +#[cfg(feature = "io")] +pub use io::Async; +#[cfg(feature = "io")] pub use reactor::{Readable, ReadableOwned, Writable, WritableOwned}; - -/// Use `Duration::MAX` once `duration_constants` are stabilized. -fn duration_max() -> Duration { - Duration::new(std::u64::MAX, 1_000_000_000 - 1) -} - -/// A future or stream that emits timed events. -/// -/// Timers are futures that output a single [`Instant`] when they fire. -/// -/// Timers are also streams that can output [`Instant`]s periodically. -/// -/// # Examples -/// -/// Sleep for 1 second: -/// -/// ``` -/// use async_io::Timer; -/// use std::time::Duration; -/// -/// # futures_lite::future::block_on(async { -/// Timer::after(Duration::from_secs(1)).await; -/// # }); -/// ``` -/// -/// Timeout after 1 second: -/// -/// ``` -/// use async_io::Timer; -/// use futures_lite::FutureExt; -/// use std::time::Duration; -/// -/// # futures_lite::future::block_on(async { -/// let addrs = async_net::resolve("google.com:80") -/// .or(async { -/// Timer::after(Duration::from_secs(10)).await; -/// Err(std::io::ErrorKind::TimedOut.into()) -/// }) -/// .await?; -/// # std::io::Result::Ok(()) }); -/// ``` -#[derive(Debug)] -pub struct Timer { - /// This timer's ID and last waker that polled it. - /// - /// When this field is set to `None`, this timer is not registered in the reactor. - id_and_waker: Option<(usize, Waker)>, - - /// The next instant at which this timer fires. - /// - /// If this timer is a blank timer, this value is None. If the timer - /// must be set, this value contains the next instant at which the - /// timer must fire. - when: Option, - - /// The period. - period: Duration, -} - -impl Timer { - /// Creates a timer that will never fire. - /// - /// # Examples - /// - /// This function may also be useful for creating a function with an optional timeout. - /// - /// ``` - /// # futures_lite::future::block_on(async { - /// use async_io::Timer; - /// use futures_lite::prelude::*; - /// use std::time::Duration; - /// - /// async fn run_with_timeout(timeout: Option) { - /// let timer = timeout - /// .map(|timeout| Timer::after(timeout)) - /// .unwrap_or_else(Timer::never); - /// - /// run_lengthy_operation().or(timer).await; - /// } - /// # // Note that since a Timer as a Future returns an Instant, - /// # // this function needs to return an Instant to be used - /// # // in "or". - /// # async fn run_lengthy_operation() -> std::time::Instant { - /// # std::time::Instant::now() - /// # } - /// - /// // Times out after 5 seconds. - /// run_with_timeout(Some(Duration::from_secs(5))).await; - /// // Does not time out. - /// run_with_timeout(None).await; - /// # }); - /// ``` - pub fn never() -> Timer { - Timer { - id_and_waker: None, - when: None, - period: duration_max(), - } - } - - /// Creates a timer that emits an event once after the given duration of time. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use std::time::Duration; - /// - /// # futures_lite::future::block_on(async { - /// Timer::after(Duration::from_secs(1)).await; - /// # }); - /// ``` - pub fn after(duration: Duration) -> Timer { - Instant::now() - .checked_add(duration) - .map_or_else(Timer::never, Timer::at) - } - - /// Creates a timer that emits an event once at the given time instant. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let now = Instant::now(); - /// let when = now + Duration::from_secs(1); - /// Timer::at(when).await; - /// # }); - /// ``` - pub fn at(instant: Instant) -> Timer { - // Use Duration::MAX once duration_constants are stabilized. - Timer::interval_at(instant, duration_max()) - } - - /// Creates a timer that emits events periodically. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use futures_lite::StreamExt; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let period = Duration::from_secs(1); - /// Timer::interval(period).next().await; - /// # }); - /// ``` - pub fn interval(period: Duration) -> Timer { - Instant::now() - .checked_add(period) - .map_or_else(Timer::never, |at| Timer::interval_at(at, period)) - } - - /// Creates a timer that emits events periodically, starting at `start`. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use futures_lite::StreamExt; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let start = Instant::now(); - /// let period = Duration::from_secs(1); - /// Timer::interval_at(start, period).next().await; - /// # }); - /// ``` - pub fn interval_at(start: Instant, period: Duration) -> Timer { - Timer { - id_and_waker: None, - when: Some(start), - period, - } - } - - /// Sets the timer to emit an en event once after the given duration of time. - /// - /// Note that resetting a timer is different from creating a new timer because - /// [`set_after()`][`Timer::set_after()`] does not remove the waker associated with the task - /// that is polling the timer. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use std::time::Duration; - /// - /// # futures_lite::future::block_on(async { - /// let mut t = Timer::after(Duration::from_secs(1)); - /// t.set_after(Duration::from_millis(100)); - /// # }); - /// ``` - pub fn set_after(&mut self, duration: Duration) { - match Instant::now().checked_add(duration) { - Some(instant) => self.set_at(instant), - None => { - // Overflow to never going off. - self.clear(); - self.when = None; - } - } - } - - /// Sets the timer to emit an event once at the given time instant. - /// - /// Note that resetting a timer is different from creating a new timer because - /// [`set_at()`][`Timer::set_at()`] does not remove the waker associated with the task - /// that is polling the timer. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let mut t = Timer::after(Duration::from_secs(1)); - /// - /// let now = Instant::now(); - /// let when = now + Duration::from_secs(1); - /// t.set_at(when); - /// # }); - /// ``` - pub fn set_at(&mut self, instant: Instant) { - self.clear(); - - // Update the timeout. - self.when = Some(instant); - - if let Some((id, waker)) = self.id_and_waker.as_mut() { - // Re-register the timer with the new timeout. - *id = Reactor::get().insert_timer(instant, waker); - } - } - - /// Sets the timer to emit events periodically. - /// - /// Note that resetting a timer is different from creating a new timer because - /// [`set_interval()`][`Timer::set_interval()`] does not remove the waker associated with the - /// task that is polling the timer. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use futures_lite::StreamExt; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let mut t = Timer::after(Duration::from_secs(1)); - /// - /// let period = Duration::from_secs(2); - /// t.set_interval(period); - /// # }); - /// ``` - pub fn set_interval(&mut self, period: Duration) { - match Instant::now().checked_add(period) { - Some(instant) => self.set_interval_at(instant, period), - None => { - // Overflow to never going off. - self.clear(); - self.when = None; - } - } - } - - /// Sets the timer to emit events periodically, starting at `start`. - /// - /// Note that resetting a timer is different from creating a new timer because - /// [`set_interval_at()`][`Timer::set_interval_at()`] does not remove the waker associated with - /// the task that is polling the timer. - /// - /// # Examples - /// - /// ``` - /// use async_io::Timer; - /// use futures_lite::StreamExt; - /// use std::time::{Duration, Instant}; - /// - /// # futures_lite::future::block_on(async { - /// let mut t = Timer::after(Duration::from_secs(1)); - /// - /// let start = Instant::now(); - /// let period = Duration::from_secs(2); - /// t.set_interval_at(start, period); - /// # }); - /// ``` - pub fn set_interval_at(&mut self, start: Instant, period: Duration) { - self.clear(); - - self.when = Some(start); - self.period = period; - - if let Some((id, waker)) = self.id_and_waker.as_mut() { - // Re-register the timer with the new timeout. - *id = Reactor::get().insert_timer(start, waker); - } - } - - /// Helper function to clear the current timer. - fn clear(&mut self) { - if let (Some(when), Some((id, _))) = (self.when, self.id_and_waker.as_ref()) { - // Deregister the timer from the reactor. - Reactor::get().remove_timer(when, *id); - } - } -} - -impl Drop for Timer { - fn drop(&mut self) { - if let (Some(when), Some((id, _))) = (self.when, self.id_and_waker.take()) { - // Deregister the timer from the reactor. - Reactor::get().remove_timer(when, id); - } - } -} - -impl Future for Timer { - type Output = Instant; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - match self.poll_next(cx) { - Poll::Ready(Some(when)) => Poll::Ready(when), - Poll::Pending => Poll::Pending, - Poll::Ready(None) => unreachable!(), - } - } -} - -impl Stream for Timer { - type Item = Instant; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let this = self.get_mut(); - - if let Some(ref mut when) = this.when { - // Check if the timer has already fired. - if Instant::now() >= *when { - if let Some((id, _)) = this.id_and_waker.take() { - // Deregister the timer from the reactor. - Reactor::get().remove_timer(*when, id); - } - let result_time = *when; - if let Some(next) = (*when).checked_add(this.period) { - *when = next; - // Register the timer in the reactor. - let id = Reactor::get().insert_timer(next, cx.waker()); - this.id_and_waker = Some((id, cx.waker().clone())); - } - return Poll::Ready(Some(result_time)); - } else { - match &this.id_and_waker { - None => { - // Register the timer in the reactor. - let id = Reactor::get().insert_timer(*when, cx.waker()); - this.id_and_waker = Some((id, cx.waker().clone())); - } - Some((id, w)) if !w.will_wake(cx.waker()) => { - // Deregister the timer from the reactor to remove the old waker. - Reactor::get().remove_timer(*when, *id); - - // Register the timer in the reactor with the new waker. - let id = Reactor::get().insert_timer(*when, cx.waker()); - this.id_and_waker = Some((id, cx.waker().clone())); - } - Some(_) => {} - } - } - } - - Poll::Pending - } -} - -/// Async adapter for I/O types. -/// -/// This type puts an I/O handle into non-blocking mode, registers it in -/// [epoll]/[kqueue]/[event ports]/[wepoll], and then provides an async interface for it. -/// -/// [epoll]: https://en.wikipedia.org/wiki/Epoll -/// [kqueue]: https://en.wikipedia.org/wiki/Kqueue -/// [event ports]: https://illumos.org/man/port_create -/// [wepoll]: https://github.com/piscisaureus/wepoll -/// -/// # Caveats -/// -/// [`Async`] is a low-level primitive, and as such it comes with some caveats. -/// -/// For higher-level primitives built on top of [`Async`], look into [`async-net`] or -/// [`async-process`] (on Unix). -/// -/// [`async-net`]: https://github.com/smol-rs/async-net -/// [`async-process`]: https://github.com/smol-rs/async-process -/// -/// ### Supported types -/// -/// [`Async`] supports all networking types, as well as some OS-specific file descriptors like -/// [timerfd] and [inotify]. -/// -/// However, do not use [`Async`] with types like [`File`][`std::fs::File`], -/// [`Stdin`][`std::io::Stdin`], [`Stdout`][`std::io::Stdout`], or [`Stderr`][`std::io::Stderr`] -/// because all operating systems have issues with them when put in non-blocking mode. -/// -/// [timerfd]: https://github.com/smol-rs/async-io/blob/master/examples/linux-timerfd.rs -/// [inotify]: https://github.com/smol-rs/async-io/blob/master/examples/linux-inotify.rs -/// -/// ### Concurrent I/O -/// -/// Note that [`&Async`][`Async`] implements [`AsyncRead`] and [`AsyncWrite`] if `&T` -/// implements those traits, which means tasks can concurrently read and write using shared -/// references. -/// -/// But there is a catch: only one task can read a time, and only one task can write at a time. It -/// is okay to have two tasks where one is reading and the other is writing at the same time, but -/// it is not okay to have two tasks reading at the same time or writing at the same time. If you -/// try to do that, conflicting tasks will just keep waking each other in turn, thus wasting CPU -/// time. -/// -/// Besides [`AsyncRead`] and [`AsyncWrite`], this caveat also applies to -/// [`poll_readable()`][`Async::poll_readable()`] and -/// [`poll_writable()`][`Async::poll_writable()`]. -/// -/// However, any number of tasks can be concurrently calling other methods like -/// [`readable()`][`Async::readable()`] or [`read_with()`][`Async::read_with()`]. -/// -/// ### Closing -/// -/// Closing the write side of [`Async`] with [`close()`][`futures_lite::AsyncWriteExt::close()`] -/// simply flushes. If you want to shutdown a TCP or Unix socket, use -/// [`Shutdown`][`std::net::Shutdown`]. -/// -/// # Examples -/// -/// Connect to a server and echo incoming messages back to the server: -/// -/// ```no_run -/// use async_io::Async; -/// use futures_lite::io; -/// use std::net::TcpStream; -/// -/// # futures_lite::future::block_on(async { -/// // Connect to a local server. -/// let stream = Async::::connect(([127, 0, 0, 1], 8000)).await?; -/// -/// // Echo all messages from the read side of the stream into the write side. -/// io::copy(&stream, &stream).await?; -/// # std::io::Result::Ok(()) }); -/// ``` -/// -/// You can use either predefined async methods or wrap blocking I/O operations in -/// [`Async::read_with()`], [`Async::read_with_mut()`], [`Async::write_with()`], and -/// [`Async::write_with_mut()`]: -/// -/// ```no_run -/// use async_io::Async; -/// use std::net::TcpListener; -/// -/// # futures_lite::future::block_on(async { -/// let listener = Async::::bind(([127, 0, 0, 1], 0))?; -/// -/// // These two lines are equivalent: -/// let (stream, addr) = listener.accept().await?; -/// let (stream, addr) = listener.read_with(|inner| inner.accept()).await?; -/// # std::io::Result::Ok(()) }); -/// ``` -#[derive(Debug)] -pub struct Async { - /// A source registered in the reactor. - source: Arc, - - /// The inner I/O handle. - io: Option, -} - -impl Unpin for Async {} - -#[cfg(unix)] -impl Async { - /// Creates an async I/O handle. - /// - /// This method will put the handle in non-blocking mode and register it in - /// [epoll]/[kqueue]/[event ports]/[wepoll]. - /// - /// On Unix systems, the handle must implement `AsRawFd`, while on Windows it must implement - /// `AsRawSocket`. - /// - /// [epoll]: https://en.wikipedia.org/wiki/Epoll - /// [kqueue]: https://en.wikipedia.org/wiki/Kqueue - /// [event ports]: https://illumos.org/man/port_create - /// [wepoll]: https://github.com/piscisaureus/wepoll - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::{SocketAddr, TcpListener}; - /// - /// # futures_lite::future::block_on(async { - /// let listener = TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0)))?; - /// let listener = Async::new(listener)?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn new(io: T) -> io::Result> { - let fd = io.as_raw_fd(); - - // Put the file descriptor in non-blocking mode. - unsafe { - let mut res = libc::fcntl(fd, libc::F_GETFL); - if res != -1 { - res = libc::fcntl(fd, libc::F_SETFL, res | libc::O_NONBLOCK); - } - if res == -1 { - return Err(io::Error::last_os_error()); - } - } - - Ok(Async { - source: Reactor::get().insert_io(fd)?, - io: Some(io), - }) - } -} - -#[cfg(unix)] -impl AsRawFd for Async { - fn as_raw_fd(&self) -> RawFd { - self.source.raw - } -} - -#[cfg(all(not(async_io_no_io_safety), unix))] -impl AsFd for Async { - fn as_fd(&self) -> BorrowedFd<'_> { - self.get_ref().as_fd() - } -} - -#[cfg(all(not(async_io_no_io_safety), unix))] -impl> TryFrom for Async { - type Error = io::Error; - - fn try_from(value: OwnedFd) -> Result { - Async::new(value.into()) - } -} - -#[cfg(all(not(async_io_no_io_safety), unix))] -impl> TryFrom> for OwnedFd { - type Error = io::Error; - - fn try_from(value: Async) -> Result { - value.into_inner().map(Into::into) - } -} - -#[cfg(windows)] -impl Async { - /// Creates an async I/O handle. - /// - /// This method will put the handle in non-blocking mode and register it in - /// [epoll]/[kqueue]/[event ports]/[wepoll]. - /// - /// On Unix systems, the handle must implement `AsRawFd`, while on Windows it must implement - /// `AsRawSocket`. - /// - /// [epoll]: https://en.wikipedia.org/wiki/Epoll - /// [kqueue]: https://en.wikipedia.org/wiki/Kqueue - /// [event ports]: https://illumos.org/man/port_create - /// [wepoll]: https://github.com/piscisaureus/wepoll - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::{SocketAddr, TcpListener}; - /// - /// # futures_lite::future::block_on(async { - /// let listener = TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0)))?; - /// let listener = Async::new(listener)?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn new(io: T) -> io::Result> { - let sock = io.as_raw_socket(); - - // Put the socket in non-blocking mode. - - use winapi::ctypes; - use winapi::um::winsock2; - - let mut nonblocking = true as ctypes::c_ulong; - let res = unsafe { - winsock2::ioctlsocket( - sock as winsock2::SOCKET, - winsock2::FIONBIO, - &mut nonblocking, - ) - }; - if res != 0 { - return Err(io::Error::last_os_error()); - } - - Ok(Async { - source: Reactor::get().insert_io(sock)?, - io: Some(io), - }) - } -} - -#[cfg(windows)] -impl AsRawSocket for Async { - fn as_raw_socket(&self) -> RawSocket { - self.source.raw - } -} - -#[cfg(all(not(async_io_no_io_safety), windows))] -impl AsSocket for Async { - fn as_socket(&self) -> BorrowedSocket<'_> { - self.get_ref().as_socket() - } -} - -#[cfg(all(not(async_io_no_io_safety), windows))] -impl> TryFrom for Async { - type Error = io::Error; - - fn try_from(value: OwnedSocket) -> Result { - Async::new(value.into()) - } -} - -#[cfg(all(not(async_io_no_io_safety), windows))] -impl> TryFrom> for OwnedSocket { - type Error = io::Error; - - fn try_from(value: Async) -> Result { - value.into_inner().map(Into::into) - } -} - -impl Async { - /// Gets a reference to the inner I/O handle. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// let inner = listener.get_ref(); - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn get_ref(&self) -> &T { - self.io.as_ref().unwrap() - } - - /// Gets a mutable reference to the inner I/O handle. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// let inner = listener.get_mut(); - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn get_mut(&mut self) -> &mut T { - self.io.as_mut().unwrap() - } - - /// Unwraps the inner I/O handle. - /// - /// This method will **not** put the I/O handle back into blocking mode. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// let inner = listener.into_inner()?; - /// - /// // Put the listener back into blocking mode. - /// inner.set_nonblocking(false)?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn into_inner(mut self) -> io::Result { - let io = self.io.take().unwrap(); - Reactor::get().remove_io(&self.source)?; - Ok(io) - } - - /// Waits until the I/O handle is readable. - /// - /// This method completes when a read operation on this I/O handle wouldn't block. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// - /// // Wait until a client can be accepted. - /// listener.readable().await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn readable(&self) -> Readable<'_, T> { - Source::readable(self) - } - - /// Waits until the I/O handle is readable. - /// - /// This method completes when a read operation on this I/O handle wouldn't block. - pub fn readable_owned(self: Arc) -> ReadableOwned { - Source::readable_owned(self) - } - - /// Waits until the I/O handle is writable. - /// - /// This method completes when a write operation on this I/O handle wouldn't block. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::{TcpStream, ToSocketAddrs}; - /// - /// # futures_lite::future::block_on(async { - /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); - /// let stream = Async::::connect(addr).await?; - /// - /// // Wait until the stream is writable. - /// stream.writable().await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn writable(&self) -> Writable<'_, T> { - Source::writable(self) - } - - /// Waits until the I/O handle is writable. - /// - /// This method completes when a write operation on this I/O handle wouldn't block. - pub fn writable_owned(self: Arc) -> WritableOwned { - Source::writable_owned(self) - } - - /// Polls the I/O handle for readability. - /// - /// When this method returns [`Poll::Ready`], that means the OS has delivered an event - /// indicating readability since the last time this task has called the method and received - /// [`Poll::Pending`]. - /// - /// # Caveats - /// - /// Two different tasks should not call this method concurrently. Otherwise, conflicting tasks - /// will just keep waking each other in turn, thus wasting CPU time. - /// - /// Note that the [`AsyncRead`] implementation for [`Async`] also uses this method. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use futures_lite::future; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// - /// // Wait until a client can be accepted. - /// future::poll_fn(|cx| listener.poll_readable(cx)).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn poll_readable(&self, cx: &mut Context<'_>) -> Poll> { - self.source.poll_readable(cx) - } - - /// Polls the I/O handle for writability. - /// - /// When this method returns [`Poll::Ready`], that means the OS has delivered an event - /// indicating writability since the last time this task has called the method and received - /// [`Poll::Pending`]. - /// - /// # Caveats - /// - /// Two different tasks should not call this method concurrently. Otherwise, conflicting tasks - /// will just keep waking each other in turn, thus wasting CPU time. - /// - /// Note that the [`AsyncWrite`] implementation for [`Async`] also uses this method. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use futures_lite::future; - /// use std::net::{TcpStream, ToSocketAddrs}; - /// - /// # futures_lite::future::block_on(async { - /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); - /// let stream = Async::::connect(addr).await?; - /// - /// // Wait until the stream is writable. - /// future::poll_fn(|cx| stream.poll_writable(cx)).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn poll_writable(&self, cx: &mut Context<'_>) -> Poll> { - self.source.poll_writable(cx) - } - - /// Performs a read operation asynchronously. - /// - /// The I/O handle is registered in the reactor and put in non-blocking mode. This method - /// invokes the `op` closure in a loop until it succeeds or returns an error other than - /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS - /// sends a notification that the I/O handle is readable. - /// - /// The closure receives a shared reference to the I/O handle. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// - /// // Accept a new client asynchronously. - /// let (stream, addr) = listener.read_with(|l| l.accept()).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn read_with(&self, op: impl FnMut(&T) -> io::Result) -> io::Result { - let mut op = op; - loop { - match op(self.get_ref()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return res, - } - optimistic(self.readable()).await?; - } - } - - /// Performs a read operation asynchronously. - /// - /// The I/O handle is registered in the reactor and put in non-blocking mode. This method - /// invokes the `op` closure in a loop until it succeeds or returns an error other than - /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS - /// sends a notification that the I/O handle is readable. - /// - /// The closure receives a mutable reference to the I/O handle. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let mut listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// - /// // Accept a new client asynchronously. - /// let (stream, addr) = listener.read_with_mut(|l| l.accept()).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn read_with_mut( - &mut self, - op: impl FnMut(&mut T) -> io::Result, - ) -> io::Result { - let mut op = op; - loop { - match op(self.get_mut()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return res, - } - optimistic(self.readable()).await?; - } - } - - /// Performs a write operation asynchronously. - /// - /// The I/O handle is registered in the reactor and put in non-blocking mode. This method - /// invokes the `op` closure in a loop until it succeeds or returns an error other than - /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS - /// sends a notification that the I/O handle is writable. - /// - /// The closure receives a shared reference to the I/O handle. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// socket.get_ref().connect("127.0.0.1:9000")?; - /// - /// let msg = b"hello"; - /// let len = socket.write_with(|s| s.send(msg)).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn write_with(&self, op: impl FnMut(&T) -> io::Result) -> io::Result { - let mut op = op; - loop { - match op(self.get_ref()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return res, - } - optimistic(self.writable()).await?; - } - } - - /// Performs a write operation asynchronously. - /// - /// The I/O handle is registered in the reactor and put in non-blocking mode. This method - /// invokes the `op` closure in a loop until it succeeds or returns an error other than - /// [`io::ErrorKind::WouldBlock`]. In between iterations of the loop, it waits until the OS - /// sends a notification that the I/O handle is writable. - /// - /// The closure receives a mutable reference to the I/O handle. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let mut socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// socket.get_ref().connect("127.0.0.1:9000")?; - /// - /// let msg = b"hello"; - /// let len = socket.write_with_mut(|s| s.send(msg)).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn write_with_mut( - &mut self, - op: impl FnMut(&mut T) -> io::Result, - ) -> io::Result { - let mut op = op; - loop { - match op(self.get_mut()) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return res, - } - optimistic(self.writable()).await?; - } - } -} - -impl AsRef for Async { - fn as_ref(&self) -> &T { - self.get_ref() - } -} - -impl AsMut for Async { - fn as_mut(&mut self) -> &mut T { - self.get_mut() - } -} - -impl Drop for Async { - fn drop(&mut self) { - if self.io.is_some() { - // Deregister and ignore errors because destructors should not panic. - Reactor::get().remove_io(&self.source).ok(); - - // Drop the I/O handle to close it. - self.io.take(); - } - } -} - -impl AsyncRead for Async { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - loop { - match (&mut *self).get_mut().read(buf) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_readable(cx))?; - } - } - - fn poll_read_vectored( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - bufs: &mut [IoSliceMut<'_>], - ) -> Poll> { - loop { - match (&mut *self).get_mut().read_vectored(bufs) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_readable(cx))?; - } - } -} - -impl AsyncRead for &Async -where - for<'a> &'a T: Read, -{ - fn poll_read( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - loop { - match (*self).get_ref().read(buf) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_readable(cx))?; - } - } - - fn poll_read_vectored( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - bufs: &mut [IoSliceMut<'_>], - ) -> Poll> { - loop { - match (*self).get_ref().read_vectored(bufs) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_readable(cx))?; - } - } -} - -impl AsyncWrite for Async { - fn poll_write( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - loop { - match (&mut *self).get_mut().write(buf) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_write_vectored( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - bufs: &[IoSlice<'_>], - ) -> Poll> { - loop { - match (&mut *self).get_mut().write_vectored(bufs) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - loop { - match (&mut *self).get_mut().flush() { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} - -impl AsyncWrite for &Async -where - for<'a> &'a T: Write, -{ - fn poll_write( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - loop { - match (*self).get_ref().write(buf) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_write_vectored( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - bufs: &[IoSlice<'_>], - ) -> Poll> { - loop { - match (*self).get_ref().write_vectored(bufs) { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - loop { - match (*self).get_ref().flush() { - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - res => return Poll::Ready(res), - } - ready!(self.poll_writable(cx))?; - } - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} - -impl Async { - /// Creates a TCP listener bound to the specified address. - /// - /// Binding with port number 0 will request an available port from the OS. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 0))?; - /// println!("Listening on {}", listener.get_ref().local_addr()?); - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn bind>(addr: A) -> io::Result> { - let addr = addr.into(); - Async::new(TcpListener::bind(addr)?) - } - - /// Accepts a new incoming TCP connection. - /// - /// When a connection is established, it will be returned as a TCP stream together with its - /// remote address. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 8000))?; - /// let (stream, addr) = listener.accept().await?; - /// println!("Accepted client: {}", addr); - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn accept(&self) -> io::Result<(Async, SocketAddr)> { - let (stream, addr) = self.read_with(|io| io.accept()).await?; - Ok((Async::new(stream)?, addr)) - } - - /// Returns a stream of incoming TCP connections. - /// - /// The stream is infinite, i.e. it never stops with a [`None`]. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use futures_lite::{pin, stream::StreamExt}; - /// use std::net::TcpListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind(([127, 0, 0, 1], 8000))?; - /// let incoming = listener.incoming(); - /// pin!(incoming); - /// - /// while let Some(stream) = incoming.next().await { - /// let stream = stream?; - /// println!("Accepted client: {}", stream.get_ref().peer_addr()?); - /// } - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn incoming(&self) -> impl Stream>> + Send + '_ { - stream::unfold(self, |listener| async move { - let res = listener.accept().await.map(|(stream, _)| stream); - Some((res, listener)) - }) - } -} - -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(listener: std::net::TcpListener) -> io::Result { - Async::new(listener) - } -} - -impl Async { - /// Creates a TCP connection to the specified address. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::{TcpStream, ToSocketAddrs}; - /// - /// # futures_lite::future::block_on(async { - /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); - /// let stream = Async::::connect(addr).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn connect>(addr: A) -> io::Result> { - // Begin async connect. - let addr = addr.into(); - let domain = Domain::for_address(addr); - let socket = connect(addr.into(), domain, Some(Protocol::TCP))?; - let stream = Async::new(TcpStream::from(socket))?; - - // The stream becomes writable when connected. - stream.writable().await?; - - // Check if there was an error while connecting. - match stream.get_ref().take_error()? { - None => Ok(stream), - Some(err) => Err(err), - } - } - - /// Reads data from the stream without removing it from the buffer. - /// - /// Returns the number of bytes read. Successive calls of this method read the same data. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use futures_lite::{io::AsyncWriteExt, stream::StreamExt}; - /// use std::net::{TcpStream, ToSocketAddrs}; - /// - /// # futures_lite::future::block_on(async { - /// let addr = "example.com:80".to_socket_addrs()?.next().unwrap(); - /// let mut stream = Async::::connect(addr).await?; - /// - /// stream - /// .write_all(b"GET / HTTP/1.1\r\nHost: example.com\r\n\r\n") - /// .await?; - /// - /// let mut buf = [0u8; 1024]; - /// let len = stream.peek(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn peek(&self, buf: &mut [u8]) -> io::Result { - self.read_with(|io| io.peek(buf)).await - } -} - -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(stream: std::net::TcpStream) -> io::Result { - Async::new(stream) - } -} - -impl Async { - /// Creates a UDP socket bound to the specified address. - /// - /// Binding with port number 0 will request an available port from the OS. - /// - /// # Examples - /// - /// ``` - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 0))?; - /// println!("Bound to {}", socket.get_ref().local_addr()?); - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn bind>(addr: A) -> io::Result> { - let addr = addr.into(); - Async::new(UdpSocket::bind(addr)?) - } - - /// Receives a single datagram message. - /// - /// Returns the number of bytes read and the address the message came from. - /// - /// This method must be called with a valid byte slice of sufficient size to hold the message. - /// If the message is too long to fit, excess bytes may get discarded. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// - /// let mut buf = [0u8; 1024]; - /// let (len, addr) = socket.recv_from(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - self.read_with(|io| io.recv_from(buf)).await - } - - /// Receives a single datagram message without removing it from the queue. - /// - /// Returns the number of bytes read and the address the message came from. - /// - /// This method must be called with a valid byte slice of sufficient size to hold the message. - /// If the message is too long to fit, excess bytes may get discarded. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// - /// let mut buf = [0u8; 1024]; - /// let (len, addr) = socket.peek_from(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn peek_from(&self, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - self.read_with(|io| io.peek_from(buf)).await - } - - /// Sends data to the specified address. - /// - /// Returns the number of bytes writen. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 0))?; - /// let addr = socket.get_ref().local_addr()?; - /// - /// let msg = b"hello"; - /// let len = socket.send_to(msg, addr).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn send_to>(&self, buf: &[u8], addr: A) -> io::Result { - let addr = addr.into(); - self.write_with(|io| io.send_to(buf, addr)).await - } - - /// Receives a single datagram message from the connected peer. - /// - /// Returns the number of bytes read. - /// - /// This method must be called with a valid byte slice of sufficient size to hold the message. - /// If the message is too long to fit, excess bytes may get discarded. - /// - /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. - /// This method will fail if the socket is not connected. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// socket.get_ref().connect("127.0.0.1:9000")?; - /// - /// let mut buf = [0u8; 1024]; - /// let len = socket.recv(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn recv(&self, buf: &mut [u8]) -> io::Result { - self.read_with(|io| io.recv(buf)).await - } - - /// Receives a single datagram message from the connected peer without removing it from the - /// queue. - /// - /// Returns the number of bytes read and the address the message came from. - /// - /// This method must be called with a valid byte slice of sufficient size to hold the message. - /// If the message is too long to fit, excess bytes may get discarded. - /// - /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. - /// This method will fail if the socket is not connected. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// socket.get_ref().connect("127.0.0.1:9000")?; - /// - /// let mut buf = [0u8; 1024]; - /// let len = socket.peek(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn peek(&self, buf: &mut [u8]) -> io::Result { - self.read_with(|io| io.peek(buf)).await - } - - /// Sends data to the connected peer. - /// - /// Returns the number of bytes written. - /// - /// The [`connect`][`UdpSocket::connect()`] method connects this socket to a remote address. - /// This method will fail if the socket is not connected. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::net::UdpSocket; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind(([127, 0, 0, 1], 8000))?; - /// socket.get_ref().connect("127.0.0.1:9000")?; - /// - /// let msg = b"hello"; - /// let len = socket.send(msg).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn send(&self, buf: &[u8]) -> io::Result { - self.write_with(|io| io.send(buf)).await - } -} - -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(socket: std::net::UdpSocket) -> io::Result { - Async::new(socket) - } -} - -#[cfg(unix)] -impl Async { - /// Creates a UDS listener bound to the specified path. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind("/tmp/socket")?; - /// println!("Listening on {:?}", listener.get_ref().local_addr()?); - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn bind>(path: P) -> io::Result> { - let path = path.as_ref().to_owned(); - Async::new(UnixListener::bind(path)?) - } - - /// Accepts a new incoming UDS stream connection. - /// - /// When a connection is established, it will be returned as a stream together with its remote - /// address. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind("/tmp/socket")?; - /// let (stream, addr) = listener.accept().await?; - /// println!("Accepted client: {:?}", addr); - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn accept(&self) -> io::Result<(Async, UnixSocketAddr)> { - let (stream, addr) = self.read_with(|io| io.accept()).await?; - Ok((Async::new(stream)?, addr)) - } - - /// Returns a stream of incoming UDS connections. - /// - /// The stream is infinite, i.e. it never stops with a [`None`] item. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use futures_lite::{pin, stream::StreamExt}; - /// use std::os::unix::net::UnixListener; - /// - /// # futures_lite::future::block_on(async { - /// let listener = Async::::bind("/tmp/socket")?; - /// let incoming = listener.incoming(); - /// pin!(incoming); - /// - /// while let Some(stream) = incoming.next().await { - /// let stream = stream?; - /// println!("Accepted client: {:?}", stream.get_ref().peer_addr()?); - /// } - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn incoming(&self) -> impl Stream>> + Send + '_ { - stream::unfold(self, |listener| async move { - let res = listener.accept().await.map(|(stream, _)| stream); - Some((res, listener)) - }) - } -} - -#[cfg(unix)] -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(listener: std::os::unix::net::UnixListener) -> io::Result { - Async::new(listener) - } -} - -#[cfg(unix)] -impl Async { - /// Creates a UDS stream connected to the specified path. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixStream; - /// - /// # futures_lite::future::block_on(async { - /// let stream = Async::::connect("/tmp/socket").await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn connect>(path: P) -> io::Result> { - // Begin async connect. - let socket = connect(SockAddr::unix(path)?, Domain::UNIX, None)?; - let stream = Async::new(UnixStream::from(socket))?; - - // The stream becomes writable when connected. - stream.writable().await?; - - // On Linux, it appears the socket may become writable even when connecting fails, so we - // must do an extra check here and see if the peer address is retrievable. - stream.get_ref().peer_addr()?; - Ok(stream) - } - - /// Creates an unnamed pair of connected UDS stream sockets. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixStream; - /// - /// # futures_lite::future::block_on(async { - /// let (stream1, stream2) = Async::::pair()?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn pair() -> io::Result<(Async, Async)> { - let (stream1, stream2) = UnixStream::pair()?; - Ok((Async::new(stream1)?, Async::new(stream2)?)) - } -} - -#[cfg(unix)] -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(stream: std::os::unix::net::UnixStream) -> io::Result { - Async::new(stream) - } -} - -#[cfg(unix)] -impl Async { - /// Creates a UDS datagram socket bound to the specified path. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind("/tmp/socket")?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn bind>(path: P) -> io::Result> { - let path = path.as_ref().to_owned(); - Async::new(UnixDatagram::bind(path)?) - } - - /// Creates a UDS datagram socket not bound to any address. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::unbound()?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn unbound() -> io::Result> { - Async::new(UnixDatagram::unbound()?) - } - - /// Creates an unnamed pair of connected Unix datagram sockets. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let (socket1, socket2) = Async::::pair()?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub fn pair() -> io::Result<(Async, Async)> { - let (socket1, socket2) = UnixDatagram::pair()?; - Ok((Async::new(socket1)?, Async::new(socket2)?)) - } - - /// Receives data from the socket. - /// - /// Returns the number of bytes read and the address the message came from. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind("/tmp/socket")?; - /// - /// let mut buf = [0u8; 1024]; - /// let (len, addr) = socket.recv_from(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn recv_from(&self, buf: &mut [u8]) -> io::Result<(usize, UnixSocketAddr)> { - self.read_with(|io| io.recv_from(buf)).await - } - - /// Sends data to the specified address. - /// - /// Returns the number of bytes written. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::unbound()?; - /// - /// let msg = b"hello"; - /// let addr = "/tmp/socket"; - /// let len = socket.send_to(msg, addr).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn send_to>(&self, buf: &[u8], path: P) -> io::Result { - self.write_with(|io| io.send_to(buf, &path)).await - } - - /// Receives data from the connected peer. - /// - /// Returns the number of bytes read and the address the message came from. - /// - /// The [`connect`][`UnixDatagram::connect()`] method connects this socket to a remote address. - /// This method will fail if the socket is not connected. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind("/tmp/socket1")?; - /// socket.get_ref().connect("/tmp/socket2")?; - /// - /// let mut buf = [0u8; 1024]; - /// let len = socket.recv(&mut buf).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn recv(&self, buf: &mut [u8]) -> io::Result { - self.read_with(|io| io.recv(buf)).await - } - - /// Sends data to the connected peer. - /// - /// Returns the number of bytes written. - /// - /// The [`connect`][`UnixDatagram::connect()`] method connects this socket to a remote address. - /// This method will fail if the socket is not connected. - /// - /// # Examples - /// - /// ```no_run - /// use async_io::Async; - /// use std::os::unix::net::UnixDatagram; - /// - /// # futures_lite::future::block_on(async { - /// let socket = Async::::bind("/tmp/socket1")?; - /// socket.get_ref().connect("/tmp/socket2")?; - /// - /// let msg = b"hello"; - /// let len = socket.send(msg).await?; - /// # std::io::Result::Ok(()) }); - /// ``` - pub async fn send(&self, buf: &[u8]) -> io::Result { - self.write_with(|io| io.send(buf)).await - } -} - -#[cfg(unix)] -impl TryFrom for Async { - type Error = io::Error; - - fn try_from(socket: std::os::unix::net::UnixDatagram) -> io::Result { - Async::new(socket) - } -} - -/// Polls a future once, waits for a wakeup, and then optimistically assumes the future is ready. -async fn optimistic(fut: impl Future>) -> io::Result<()> { - let mut polled = false; - pin!(fut); - - future::poll_fn(|cx| { - if !polled { - polled = true; - fut.as_mut().poll(cx) - } else { - Poll::Ready(Ok(())) - } - }) - .await -} - -fn connect(addr: SockAddr, domain: Domain, protocol: Option) -> io::Result { - let sock_type = Type::STREAM; - #[cfg(any( - target_os = "android", - target_os = "dragonfly", - target_os = "freebsd", - target_os = "fuchsia", - target_os = "illumos", - target_os = "linux", - target_os = "netbsd", - target_os = "openbsd" - ))] - // If we can, set nonblocking at socket creation for unix - let sock_type = sock_type.nonblocking(); - // This automatically handles cloexec on unix, no_inherit on windows and nosigpipe on macos - let socket = Socket::new(domain, sock_type, protocol)?; - #[cfg(not(any( - target_os = "android", - target_os = "dragonfly", - target_os = "freebsd", - target_os = "fuchsia", - target_os = "illumos", - target_os = "linux", - target_os = "netbsd", - target_os = "openbsd" - )))] - // If the current platform doesn't support nonblocking at creation, enable it after creation - socket.set_nonblocking(true)?; - match socket.connect(&addr) { - Ok(_) => {} - #[cfg(unix)] - Err(err) if err.raw_os_error() == Some(libc::EINPROGRESS) => {} - Err(err) if err.kind() == io::ErrorKind::WouldBlock => {} - Err(err) => return Err(err), - } - Ok(socket) -} +#[cfg(feature = "timer")] +pub use timer::Timer; diff --git a/src/reactor.rs b/src/reactor/io.rs similarity index 74% rename from src/reactor.rs rename to src/reactor/io.rs index b6ae153..6b26184 100644 --- a/src/reactor.rs +++ b/src/reactor/io.rs @@ -1,24 +1,26 @@ +#[cfg(feature = "timer")] +use super::timer::Reactor as Timers; +#[cfg(feature = "timer")] +use std::time::Instant; + use std::borrow::Borrow; -use std::collections::BTreeMap; use std::fmt; use std::future::Future; use std::io; use std::marker::PhantomData; -use std::mem; #[cfg(unix)] use std::os::unix::io::RawFd; #[cfg(windows)] use std::os::windows::io::RawSocket; use std::panic; use std::pin::Pin; +#[cfg(not(feature = "timer"))] use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Arc, Mutex, MutexGuard}; use std::task::{Context, Poll, Waker}; -use std::time::{Duration, Instant}; +use std::time::Duration; -use concurrent_queue::ConcurrentQueue; use futures_lite::ready; -use once_cell::sync::Lazy; use polling::{Event, Poller}; use slab::Slab; @@ -34,14 +36,6 @@ pub(crate) struct Reactor { /// This is where I/O is polled, producing I/O events. poller: Poller, - /// Ticker bumped before polling. - /// - /// This is useful for checking what is the current "round" of `ReactorLock::react()` when - /// synchronizing things in `Source::readable()` and `Source::writable()`. Both of those - /// methods must make sure they don't receive stale I/O events - they only accept events from a - /// fresh "round" of `ReactorLock::react()`. - ticker: AtomicUsize, - /// Registered sources. sources: Mutex>>, @@ -50,40 +44,24 @@ pub(crate) struct Reactor { /// Holding a lock on this event list implies the exclusive right to poll I/O. events: Mutex>, - /// An ordered map of registered timers. - /// - /// Timers are in the order in which they fire. The `usize` in this type is a timer ID used to - /// distinguish timers that fire at the same time. The `Waker` represents the task awaiting the - /// timer. - timers: Mutex>, - - /// A queue of timer operations (insert and remove). - /// - /// When inserting or removing a timer, we don't process it immediately - we just push it into - /// this queue. Timers actually get processed when the queue fills up or the reactor is polled. - timer_ops: ConcurrentQueue, + /// The object used to manage timers within the reactor. + timers: Timers, } impl Reactor { - /// Returns a reference to the reactor. - pub(crate) fn get() -> &'static Reactor { - static REACTOR: Lazy = Lazy::new(|| { - crate::driver::init(); - Reactor { - poller: Poller::new().expect("cannot initialize I/O event notification"), - ticker: AtomicUsize::new(0), - sources: Mutex::new(Slab::new()), - events: Mutex::new(Vec::new()), - timers: Mutex::new(BTreeMap::new()), - timer_ops: ConcurrentQueue::bounded(1000), - } - }); - &REACTOR + /// Create a new `Reactor`. + pub(crate) fn new() -> Reactor { + Reactor { + poller: Poller::new().expect("cannot initialize I/O event notification"), + sources: Mutex::new(Slab::new()), + events: Mutex::new(Vec::new()), + timers: Timers::new(), + } } /// Returns the current ticker. pub(crate) fn ticker(&self) -> usize { - self.ticker.load(Ordering::SeqCst) + self.timers.ticker() } /// Registers an I/O source in the reactor. @@ -125,36 +103,15 @@ impl Reactor { /// Registers a timer in the reactor. /// /// Returns the inserted timer's ID. + #[cfg(feature = "timer")] pub(crate) fn insert_timer(&self, when: Instant, waker: &Waker) -> usize { - // Generate a new timer ID. - static ID_GENERATOR: AtomicUsize = AtomicUsize::new(1); - let id = ID_GENERATOR.fetch_add(1, Ordering::Relaxed); - - // Push an insert operation. - while self - .timer_ops - .push(TimerOp::Insert(when, id, waker.clone())) - .is_err() - { - // If the queue is full, drain it and try again. - let mut timers = self.timers.lock().unwrap(); - self.process_timer_ops(&mut timers); - } - - // Notify that a timer has been inserted. - self.notify(); - - id + self.timers.insert_timer(when, waker) } /// Deregisters a timer from the reactor. + #[cfg(feature = "timer")] pub(crate) fn remove_timer(&self, when: Instant, id: usize) { - // Push a remove operation. - while self.timer_ops.push(TimerOp::Remove(when, id)).is_err() { - // If the queue is full, drain it and try again. - let mut timers = self.timers.lock().unwrap(); - self.process_timer_ops(&mut timers); - } + self.timers.remove_timer(when, id); } /// Notifies the thread blocked on the reactor. @@ -176,63 +133,6 @@ impl Reactor { ReactorLock { reactor, events } }) } - - /// Processes ready timers and extends the list of wakers to wake. - /// - /// Returns the duration until the next timer before this method was called. - fn process_timers(&self, wakers: &mut Vec) -> Option { - let mut timers = self.timers.lock().unwrap(); - self.process_timer_ops(&mut timers); - - let now = Instant::now(); - - // Split timers into ready and pending timers. - // - // Careful to split just *after* `now`, so that a timer set for exactly `now` is considered - // ready. - let pending = timers.split_off(&(now + Duration::from_nanos(1), 0)); - let ready = mem::replace(&mut *timers, pending); - - // Calculate the duration until the next event. - let dur = if ready.is_empty() { - // Duration until the next timer. - timers - .keys() - .next() - .map(|(when, _)| when.saturating_duration_since(now)) - } else { - // Timers are about to fire right now. - Some(Duration::from_secs(0)) - }; - - // Drop the lock before waking. - drop(timers); - - // Add wakers to the list. - log::trace!("process_timers: {} ready wakers", ready.len()); - for (_, waker) in ready { - wakers.push(waker); - } - - dur - } - - /// Processes queued timer operations. - fn process_timer_ops(&self, timers: &mut MutexGuard<'_, BTreeMap<(Instant, usize), Waker>>) { - // Process only as much as fits into the queue, or else this loop could in theory run - // forever. - for _ in 0..self.timer_ops.capacity().unwrap() { - match self.timer_ops.pop() { - Ok(TimerOp::Insert(when, id, waker)) => { - timers.insert((when, id), waker); - } - Ok(TimerOp::Remove(when, id)) => { - timers.remove(&(when, id)); - } - Err(_) => break, - } - } - } } /// A lock on the reactor. @@ -246,22 +146,11 @@ impl ReactorLock<'_> { pub(crate) fn react(&mut self, timeout: Option) -> io::Result<()> { let mut wakers = Vec::new(); - // Process ready timers. - let next_timer = self.reactor.process_timers(&mut wakers); - // compute the timeout for blocking on I/O events. - let timeout = match (next_timer, timeout) { - (None, None) => None, - (Some(t), None) | (None, Some(t)) => Some(t), - (Some(a), Some(b)) => Some(a.min(b)), - }; + let timeout = self.reactor.timers.timeout_duration(timeout, &mut wakers); // Bump the ticker before polling I/O. - let tick = self - .reactor - .ticker - .fetch_add(1, Ordering::SeqCst) - .wrapping_add(1); + let tick = self.reactor.timers.bump_ticker(); self.events.clear(); @@ -271,7 +160,7 @@ impl ReactorLock<'_> { Ok(0) => { if timeout != Some(Duration::from_secs(0)) { // The non-zero timeout was hit so fire ready timers. - self.reactor.process_timers(&mut wakers); + self.reactor.timers.process_timers(&mut wakers); } Ok(()) } @@ -331,10 +220,38 @@ impl ReactorLock<'_> { } } -/// A single timer operation. -enum TimerOp { - Insert(Instant, usize, Waker), - Remove(Instant, usize), +/// Shim for `Timers` on targets with timers disabled. +#[cfg(not(feature = "timer"))] +struct Timers { + /// The ticker. + ticker: AtomicUsize, +} + +#[cfg(not(feature = "timer"))] +impl Timers { + fn new() -> Self { + Self { + ticker: AtomicUsize::new(0), + } + } + + fn timeout_duration( + &self, + timeout: Option, + _wakers: &mut Vec, + ) -> Option { + timeout + } + + fn process_timers(&self, _wakers: &mut Vec) {} + + fn ticker(&self) -> usize { + self.ticker.load(Ordering::SeqCst) + } + + fn bump_ticker(&self) -> usize { + self.ticker.fetch_add(1, Ordering::SeqCst).wrapping_add(1) + } } /// A registered source of I/O events. @@ -431,11 +348,11 @@ impl Source { panic::catch_unwind(|| w.wake()).ok(); } state[dir].waker = Some(cx.waker().clone()); - state[dir].ticks = Some((Reactor::get().ticker(), state[dir].tick)); + state[dir].ticks = Some((crate::reactor::Reactor::get().ticker(), state[dir].tick)); // Update interest in this I/O handle. if was_empty { - Reactor::get().poller.modify( + crate::reactor::Reactor::get().0.poller.modify( self.raw, Event { key: self.key, @@ -608,7 +525,7 @@ impl> + Clone, T> Future for Ready { _marker: PhantomData, }); *index = Some(i); - *ticks = Some((Reactor::get().ticker(), state[*dir].tick)); + *ticks = Some((crate::reactor::Reactor::get().ticker(), state[*dir].tick)); i } }; @@ -616,7 +533,7 @@ impl> + Clone, T> Future for Ready { // Update interest in this I/O handle. if was_empty { - Reactor::get().poller.modify( + crate::reactor::Reactor::get().0.poller.modify( handle.borrow().source.raw, Event { key: handle.borrow().source.key, diff --git a/src/reactor/mod.rs b/src/reactor/mod.rs new file mode 100644 index 0000000..4d0c2e4 --- /dev/null +++ b/src/reactor/mod.rs @@ -0,0 +1,106 @@ +use std::io::Result; +#[cfg(all(feature = "io", unix))] +use std::os::unix::io::RawFd; +#[cfg(all(feature = "io", windows))] +use std::os::windows::io::RawSocket; +#[cfg(feature = "io")] +use std::sync::Arc; + +#[cfg(feature = "timer")] +use std::task::Waker; +#[cfg(feature = "timer")] +use std::time::Instant; + +use once_cell::sync::Lazy; +use std::time::Duration; + +#[cfg(feature = "timer")] +mod timer; + +cfg_if::cfg_if! { + if #[cfg(feature = "io")] { + mod io; + use io as sys; + + pub use io::{Readable, ReadableOwned, Writable, WritableOwned}; + pub(crate) use io::Source; + } else { + use timer as sys; + } +} + +/// The reactor. +/// +/// There is only one global instance of this type, accessible by [`Reactor::get()`]. +pub(crate) struct Reactor(sys::Reactor); + +impl Reactor { + /// Returns a reference to the reactor. + pub(crate) fn get() -> &'static Reactor { + static REACTOR: Lazy = Lazy::new(|| { + crate::driver::init(); + Reactor(sys::Reactor::new()) + }); + &REACTOR + } + + /// Returns the current ticker. + pub(crate) fn ticker(&self) -> usize { + self.0.ticker() + } + + /// Registers an I/O source in the reactor. + #[cfg(feature = "io")] + pub(crate) fn insert_io( + &self, + #[cfg(unix)] raw: RawFd, + #[cfg(windows)] raw: RawSocket, + ) -> Result> { + self.0.insert_io(raw) + } + + /// Deregisters an I/O source from the reactor. + #[cfg(feature = "io")] + pub(crate) fn remove_io(&self, source: &Source) -> Result<()> { + self.0.remove_io(source) + } + + /// Registers a timer in the reactor. + /// + /// Returns the inserted timer's ID. + #[cfg(feature = "timer")] + pub(crate) fn insert_timer(&self, when: Instant, waker: &Waker) -> usize { + self.0.insert_timer(when, waker) + } + + /// Deregisters a timer from the reactor. + #[cfg(feature = "timer")] + pub(crate) fn remove_timer(&self, when: Instant, id: usize) { + self.0.remove_timer(when, id) + } + + /// Notifies the thread blocked on the reactor. + pub(crate) fn notify(&self) { + self.0.notify() + } + + /// Locks the reactor, potentially blocking if the lock is held by another thread. + pub(crate) fn lock(&self) -> ReactorLock<'_> { + ReactorLock(self.0.lock()) + } + + /// Attempts to lock the reactor. + pub(crate) fn try_lock(&self) -> Option> { + self.0.try_lock().map(ReactorLock) + } +} + +/// A lock on the reactor. +pub(crate) struct ReactorLock<'a>(sys::ReactorLock<'a>); + +impl ReactorLock<'_> { + /// Processes new events, blocking until the first event or the timeout. + pub(crate) fn react(&mut self, timeout: Option) -> Result<()> { + self.0.react(timeout) + } +} diff --git a/src/reactor/timer.rs b/src/reactor/timer.rs new file mode 100644 index 0000000..ef9e171 --- /dev/null +++ b/src/reactor/timer.rs @@ -0,0 +1,260 @@ +use std::cmp; +use std::collections::BTreeMap; +#[cfg(not(feature = "io"))] +use std::io; +use std::mem; +#[cfg(not(feature = "io"))] +use std::panic; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Mutex, MutexGuard}; +use std::task::Waker; +use std::time::{Duration, Instant}; + +use concurrent_queue::ConcurrentQueue; +#[cfg(not(feature = "io"))] +use parking::{pair, Parker, Unparker}; + +/// A reactor that is capable of processing timers. +pub(crate) struct Reactor { + /// An ordered map of registered timers. + /// + /// Timers are in the order in which they fire. The `usize` in this type is a timer ID used to + /// distinguish timers that fire at the same time. The `Waker` represents the task awaiting the + /// timer. + timers: Mutex>, + + /// A queue of timer operations (insert and remove). + /// + /// When inserting or removing a timer, we don't process it immediately - we just push it into + /// this queue. Timers actually get processed when the queue fills up or the reactor is polled. + timer_ops: ConcurrentQueue, + + /// Ticker bumped before polling. + /// + /// This is useful for checking what is the current "round" of `ReactorLock::react()` when + /// synchronizing things in `Source::readable()` and `Source::writable()`. Both of those + /// methods must make sure they don't receive stale I/O events - they only accept events from a + /// fresh "round" of `ReactorLock::react()`. + ticker: AtomicUsize, + + /// The object used to unpark the reactor. + /// + /// Not needed when I/O is available since we block on `polling` instead. + #[cfg(not(feature = "io"))] + unparker: Unparker, + + /// The object used to park the reactor. + /// + /// The mutex used to hold this value implies the exclusive right to poll the reactor. + #[cfg(not(feature = "io"))] + parker: Mutex, +} + +impl Reactor { + /// Create a new `Reactor`. + pub(crate) fn new() -> Reactor { + #[cfg(not(feature = "io"))] + let (parker, unparker) = pair(); + + Reactor { + timers: Mutex::new(BTreeMap::new()), + timer_ops: ConcurrentQueue::bounded(1000), + ticker: AtomicUsize::new(0), + #[cfg(not(feature = "io"))] + unparker, + #[cfg(not(feature = "io"))] + parker: Mutex::new(parker), + } + } + + /// Returns the current ticker. + pub(crate) fn ticker(&self) -> usize { + self.ticker.load(Ordering::SeqCst) + } + + /// Bump the current ticker and return the new value. + pub(crate) fn bump_ticker(&self) -> usize { + self.ticker.fetch_add(1, Ordering::SeqCst).wrapping_add(1) + } + + /// Registers a timer in the reactor. + /// + /// Returns the inserted timer's ID. + pub(crate) fn insert_timer(&self, when: Instant, waker: &Waker) -> usize { + // Generate a new timer ID. + static ID_GENERATOR: AtomicUsize = AtomicUsize::new(1); + let id = ID_GENERATOR.fetch_add(1, Ordering::Relaxed); + + // Push an insert operation. + while self + .timer_ops + .push(TimerOp::Insert(when, id, waker.clone())) + .is_err() + { + // If the queue is full, drain it and try again. + let mut timers = self.timers.lock().unwrap(); + self.process_timer_ops(&mut timers); + } + + // Notify that a timer has been inserted. + crate::reactor::Reactor::get().notify(); + + id + } + + /// Deregisters a timer from the reactor. + pub(crate) fn remove_timer(&self, when: Instant, id: usize) { + // Push a remove operation. + while self.timer_ops.push(TimerOp::Remove(when, id)).is_err() { + // If the queue is full, drain it and try again. + let mut timers = self.timers.lock().unwrap(); + self.process_timer_ops(&mut timers); + } + } + + /// Notify the thread blocked on this reactor. + #[cfg(not(feature = "io"))] + pub(crate) fn notify(&self) { + self.unparker.unpark(); + } + + /// Acquire a lock on the reactor. + #[cfg(not(feature = "io"))] + pub(crate) fn lock(&self) -> ReactorLock<'_> { + let reactor = self; + let parker = reactor.parker.lock().unwrap(); + ReactorLock { reactor, parker } + } + + /// Try to acquire a lock on the reactor. + #[cfg(not(feature = "io"))] + pub(crate) fn try_lock(&self) -> Option> { + let reactor = self; + let parker = reactor.parker.try_lock().ok()?; + Some(ReactorLock { reactor, parker }) + } + + /// Processes ready timers and extends the list of wakers to wake. + /// + /// Returns the duration until the next timer before this method was called. + pub(crate) fn process_timers(&self, wakers: &mut Vec) -> Option { + let mut timers = self.timers.lock().unwrap(); + self.process_timer_ops(&mut timers); + + let now = Instant::now(); + + // Split timers into ready and pending timers. + // + // Careful to split just *after* `now`, so that a timer set for exactly `now` is considered + // ready. + let pending = timers.split_off(&(now + Duration::from_nanos(1), 0)); + let ready = mem::replace(&mut *timers, pending); + + // Calculate the duration until the next event. + let dur = if ready.is_empty() { + // Duration until the next timer. + timers + .keys() + .next() + .map(|(when, _)| when.saturating_duration_since(now)) + } else { + // Timers are about to fire right now. + Some(Duration::from_secs(0)) + }; + + // Drop the lock before waking. + drop(timers); + + // Add wakers to the list. + log::trace!("process_timers: {} ready wakers", ready.len()); + for (_, waker) in ready { + wakers.push(waker); + } + + dur + } + + /// Processes queued timer operations. + fn process_timer_ops(&self, timers: &mut MutexGuard<'_, BTreeMap<(Instant, usize), Waker>>) { + // Process only as much as fits into the queue, or else this loop could in theory run + // forever. + for _ in 0..self.timer_ops.capacity().unwrap() { + match self.timer_ops.pop() { + Ok(TimerOp::Insert(when, id, waker)) => { + timers.insert((when, id), waker); + } + Ok(TimerOp::Remove(when, id)) => { + timers.remove(&(when, id)); + } + Err(_) => break, + } + } + } + + /// Determine the timeout duration for the reactor. + pub(crate) fn timeout_duration( + &self, + timeout: Option, + wakers: &mut Vec, + ) -> Option { + let next_ready_timer = self.process_timers(wakers); + + // Determine the timeout duration. + match (next_ready_timer, timeout) { + (None, None) => None, + (Some(dur), None) | (None, Some(dur)) => Some(dur), + (Some(dur1), Some(dur2)) => Some(cmp::min(dur1, dur2)), + } + } +} + +#[cfg(not(feature = "io"))] +pub(crate) struct ReactorLock<'a> { + /// The reference to the reactor. + reactor: &'a Reactor, + + /// The guard used to lock the reactor. + parker: MutexGuard<'a, Parker>, +} + +#[cfg(not(feature = "io"))] +impl ReactorLock<'_> { + /// Processes timers and then blocks until the next timer is ready. + pub(crate) fn react(&self, timeout: Option) -> io::Result<()> { + // Process the upcoming timers and determine the timeout duration. + let mut wakers = vec![]; + let timeout = self.reactor.timeout_duration(timeout, &mut wakers); + + // Bump the ticker. + self.reactor.bump_ticker(); + + // Park the thread until either the timeout duration has elapsed or a timer is ready. + let timers_ready = match timeout { + None => { + // Park until a notification. + self.parker.park(); + false + } + Some(timeout) => !self.parker.park_timeout(timeout), + }; + + // If we hit the timeout, we may need to process more timers. + if timers_ready { + self.reactor.process_timers(&mut wakers); + } + + // Fire off wakers. + log::trace!("react: {} wakers", wakers.len()); + for waker in wakers { + panic::catch_unwind(|| waker.wake()).ok(); + } + + Ok(()) + } +} + +/// A single timer operation. +enum TimerOp { + Insert(Instant, usize, Waker), + Remove(Instant, usize), +} diff --git a/src/timer.rs b/src/timer.rs new file mode 100644 index 0000000..503ce80 --- /dev/null +++ b/src/timer.rs @@ -0,0 +1,385 @@ +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll, Waker}; +use std::time::{Duration, Instant}; + +use crate::reactor::Reactor; +use futures_lite::Stream; + +/// Use `Duration::MAX` once `duration_constants` are stabilized. +fn duration_max() -> Duration { + Duration::new(std::u64::MAX, 1_000_000_000 - 1) +} + +/// A future or stream that emits timed events. +/// +/// Timers are futures that output a single [`Instant`] when they fire. +/// +/// Timers are also streams that can output [`Instant`]s periodically. +/// +/// # Examples +/// +/// Sleep for 1 second: +/// +/// ``` +/// use async_io::Timer; +/// use std::time::Duration; +/// +/// # futures_lite::future::block_on(async { +/// Timer::after(Duration::from_secs(1)).await; +/// # }); +/// ``` +/// +/// Timeout after 1 second: +/// +/// ``` +/// use async_io::Timer; +/// use futures_lite::FutureExt; +/// use std::time::Duration; +/// +/// # futures_lite::future::block_on(async { +/// let addrs = async_net::resolve("google.com:80") +/// .or(async { +/// Timer::after(Duration::from_secs(10)).await; +/// Err(std::io::ErrorKind::TimedOut.into()) +/// }) +/// .await?; +/// # std::io::Result::Ok(()) }); +/// ``` +#[derive(Debug)] +pub struct Timer { + /// This timer's ID and last waker that polled it. + /// + /// When this field is set to `None`, this timer is not registered in the reactor. + id_and_waker: Option<(usize, Waker)>, + + /// The next instant at which this timer fires. + /// + /// If this timer is a blank timer, this value is None. If the timer + /// must be set, this value contains the next instant at which the + /// timer must fire. + when: Option, + + /// The period. + period: Duration, +} + +impl Timer { + /// Creates a timer that will never fire. + /// + /// # Examples + /// + /// This function may also be useful for creating a function with an optional timeout. + /// + /// ``` + /// # futures_lite::future::block_on(async { + /// use async_io::Timer; + /// use futures_lite::prelude::*; + /// use std::time::Duration; + /// + /// async fn run_with_timeout(timeout: Option) { + /// let timer = timeout + /// .map(|timeout| Timer::after(timeout)) + /// .unwrap_or_else(Timer::never); + /// + /// run_lengthy_operation().or(timer).await; + /// } + /// # // Note that since a Timer as a Future returns an Instant, + /// # // this function needs to return an Instant to be used + /// # // in "or". + /// # async fn run_lengthy_operation() -> std::time::Instant { + /// # std::time::Instant::now() + /// # } + /// + /// // Times out after 5 seconds. + /// run_with_timeout(Some(Duration::from_secs(5))).await; + /// // Does not time out. + /// run_with_timeout(None).await; + /// # }); + /// ``` + pub fn never() -> Timer { + Timer { + id_and_waker: None, + when: None, + period: duration_max(), + } + } + + /// Creates a timer that emits an event once after the given duration of time. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use std::time::Duration; + /// + /// # futures_lite::future::block_on(async { + /// Timer::after(Duration::from_secs(1)).await; + /// # }); + /// ``` + pub fn after(duration: Duration) -> Timer { + Instant::now() + .checked_add(duration) + .map_or_else(Timer::never, Timer::at) + } + + /// Creates a timer that emits an event once at the given time instant. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let now = Instant::now(); + /// let when = now + Duration::from_secs(1); + /// Timer::at(when).await; + /// # }); + /// ``` + pub fn at(instant: Instant) -> Timer { + // Use Duration::MAX once duration_constants are stabilized. + Timer::interval_at(instant, duration_max()) + } + + /// Creates a timer that emits events periodically. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use futures_lite::StreamExt; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let period = Duration::from_secs(1); + /// Timer::interval(period).next().await; + /// # }); + /// ``` + pub fn interval(period: Duration) -> Timer { + Instant::now() + .checked_add(period) + .map_or_else(Timer::never, |at| Timer::interval_at(at, period)) + } + + /// Creates a timer that emits events periodically, starting at `start`. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use futures_lite::StreamExt; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let start = Instant::now(); + /// let period = Duration::from_secs(1); + /// Timer::interval_at(start, period).next().await; + /// # }); + /// ``` + pub fn interval_at(start: Instant, period: Duration) -> Timer { + Timer { + id_and_waker: None, + when: Some(start), + period, + } + } + + /// Sets the timer to emit an en event once after the given duration of time. + /// + /// Note that resetting a timer is different from creating a new timer because + /// [`set_after()`][`Timer::set_after()`] does not remove the waker associated with the task + /// that is polling the timer. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use std::time::Duration; + /// + /// # futures_lite::future::block_on(async { + /// let mut t = Timer::after(Duration::from_secs(1)); + /// t.set_after(Duration::from_millis(100)); + /// # }); + /// ``` + pub fn set_after(&mut self, duration: Duration) { + match Instant::now().checked_add(duration) { + Some(instant) => self.set_at(instant), + None => { + // Overflow to never going off. + self.clear(); + self.when = None; + } + } + } + + /// Sets the timer to emit an event once at the given time instant. + /// + /// Note that resetting a timer is different from creating a new timer because + /// [`set_at()`][`Timer::set_at()`] does not remove the waker associated with the task + /// that is polling the timer. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let mut t = Timer::after(Duration::from_secs(1)); + /// + /// let now = Instant::now(); + /// let when = now + Duration::from_secs(1); + /// t.set_at(when); + /// # }); + /// ``` + pub fn set_at(&mut self, instant: Instant) { + self.clear(); + + // Update the timeout. + self.when = Some(instant); + + if let Some((id, waker)) = self.id_and_waker.as_mut() { + // Re-register the timer with the new timeout. + *id = Reactor::get().insert_timer(instant, waker); + } + } + + /// Sets the timer to emit events periodically. + /// + /// Note that resetting a timer is different from creating a new timer because + /// [`set_interval()`][`Timer::set_interval()`] does not remove the waker associated with the + /// task that is polling the timer. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use futures_lite::StreamExt; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let mut t = Timer::after(Duration::from_secs(1)); + /// + /// let period = Duration::from_secs(2); + /// t.set_interval(period); + /// # }); + /// ``` + pub fn set_interval(&mut self, period: Duration) { + match Instant::now().checked_add(period) { + Some(instant) => self.set_interval_at(instant, period), + None => { + // Overflow to never going off. + self.clear(); + self.when = None; + } + } + } + + /// Sets the timer to emit events periodically, starting at `start`. + /// + /// Note that resetting a timer is different from creating a new timer because + /// [`set_interval_at()`][`Timer::set_interval_at()`] does not remove the waker associated with + /// the task that is polling the timer. + /// + /// # Examples + /// + /// ``` + /// use async_io::Timer; + /// use futures_lite::StreamExt; + /// use std::time::{Duration, Instant}; + /// + /// # futures_lite::future::block_on(async { + /// let mut t = Timer::after(Duration::from_secs(1)); + /// + /// let start = Instant::now(); + /// let period = Duration::from_secs(2); + /// t.set_interval_at(start, period); + /// # }); + /// ``` + pub fn set_interval_at(&mut self, start: Instant, period: Duration) { + self.clear(); + + self.when = Some(start); + self.period = period; + + if let Some((id, waker)) = self.id_and_waker.as_mut() { + // Re-register the timer with the new timeout. + *id = Reactor::get().insert_timer(start, waker); + } + } + + /// Helper function to clear the current timer. + fn clear(&mut self) { + if let (Some(when), Some((id, _))) = (self.when, self.id_and_waker.as_ref()) { + // Deregister the timer from the reactor. + Reactor::get().remove_timer(when, *id); + } + } +} + +impl Drop for Timer { + fn drop(&mut self) { + if let (Some(when), Some((id, _))) = (self.when, self.id_and_waker.take()) { + // Deregister the timer from the reactor. + Reactor::get().remove_timer(when, id); + } + } +} + +impl Future for Timer { + type Output = Instant; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.poll_next(cx) { + Poll::Ready(Some(when)) => Poll::Ready(when), + Poll::Pending => Poll::Pending, + Poll::Ready(None) => unreachable!(), + } + } +} + +impl Stream for Timer { + type Item = Instant; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + + if let Some(ref mut when) = this.when { + // Check if the timer has already fired. + if Instant::now() >= *when { + if let Some((id, _)) = this.id_and_waker.take() { + // Deregister the timer from the reactor. + Reactor::get().remove_timer(*when, id); + } + let result_time = *when; + if let Some(next) = (*when).checked_add(this.period) { + *when = next; + // Register the timer in the reactor. + let id = Reactor::get().insert_timer(next, cx.waker()); + this.id_and_waker = Some((id, cx.waker().clone())); + } + return Poll::Ready(Some(result_time)); + } else { + match &this.id_and_waker { + None => { + // Register the timer in the reactor. + let id = Reactor::get().insert_timer(*when, cx.waker()); + this.id_and_waker = Some((id, cx.waker().clone())); + } + Some((id, w)) if !w.will_wake(cx.waker()) => { + // Deregister the timer from the reactor to remove the old waker. + Reactor::get().remove_timer(*when, *id); + + // Register the timer in the reactor with the new waker. + let id = Reactor::get().insert_timer(*when, cx.waker()); + this.id_and_waker = Some((id, cx.waker().clone())); + } + Some(_) => {} + } + } + } + + Poll::Pending + } +} \ No newline at end of file diff --git a/tests/async.rs b/tests/async.rs index c856760..6894494 100644 --- a/tests/async.rs +++ b/tests/async.rs @@ -1,3 +1,5 @@ +#![cfg(feature = "io")] + use std::future::Future; use std::io; use std::net::{Shutdown, TcpListener, TcpStream, UdpSocket}; @@ -7,7 +9,9 @@ use std::sync::Arc; use std::thread; use std::time::Duration; -use async_io::{Async, Timer}; +use async_io::Async; +#[cfg(feature = "timer")] +use async_io::Timer; use futures_lite::{future, prelude::*}; #[cfg(unix)] use tempfile::tempdir; @@ -82,6 +86,7 @@ fn tcp_peek_read() -> io::Result<()> { }) } +#[cfg(feature = "timer")] #[test] fn tcp_reader_hangup() -> io::Result<()> { future::block_on(async { @@ -104,6 +109,7 @@ fn tcp_reader_hangup() -> io::Result<()> { }) } +#[cfg(feature = "timer")] #[test] fn tcp_writer_hangup() -> io::Result<()> { future::block_on(async { @@ -244,6 +250,7 @@ fn uds_send_to_recv_from() -> io::Result<()> { }) } +#[cfg(feature = "timer")] #[cfg(unix)] #[test] fn uds_reader_hangup() -> io::Result<()> { @@ -262,6 +269,7 @@ fn uds_reader_hangup() -> io::Result<()> { }) } +#[cfg(feature = "timer")] #[cfg(unix)] #[test] fn uds_writer_hangup() -> io::Result<()> { @@ -285,6 +293,7 @@ fn uds_writer_hangup() -> io::Result<()> { // Test that we correctly re-register interests after we've previously been // interested in both readable and writable events and then we get only one of // those (we need to re-register interest on the other). +#[cfg(feature = "timer")] #[test] fn tcp_duplex() -> io::Result<()> { future::block_on(async { diff --git a/tests/timer.rs b/tests/timer.rs index cdd90db..e7c150b 100644 --- a/tests/timer.rs +++ b/tests/timer.rs @@ -1,3 +1,5 @@ +#![cfg(feature = "timer")] + use std::future::Future; use std::pin::Pin; use std::sync::{Arc, Mutex}; From d88759a20aa720f600042c3987d19d6f7225007b Mon Sep 17 00:00:00 2001 From: jtnunley Date: Wed, 7 Sep 2022 13:43:44 -0700 Subject: [PATCH 2/6] Fix the warnings that appear in I/O tests --- tests/async.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/async.rs b/tests/async.rs index 6894494..9fd6d22 100644 --- a/tests/async.rs +++ b/tests/async.rs @@ -5,8 +5,10 @@ use std::io; use std::net::{Shutdown, TcpListener, TcpStream, UdpSocket}; #[cfg(unix)] use std::os::unix::net::{UnixDatagram, UnixListener, UnixStream}; +#[cfg(feature = "timer")] use std::sync::Arc; use std::thread; +#[cfg(feature = "timer")] use std::time::Duration; use async_io::Async; From 58c1a0337a1be4c0d59d2c742da0812fb2bd35f9 Mon Sep 17 00:00:00 2001 From: jtnunley Date: Thu, 8 Sep 2022 09:31:42 -0700 Subject: [PATCH 3/6] convert to non-breaking wasm --- .github/workflows/ci.yml | 17 +- Cargo.toml | 22 +- src/driver.rs | 442 +++++++++++++++++++-------------------- src/lib.rs | 9 +- src/reactor/io.rs | 40 ---- src/reactor/mod.rs | 21 +- src/reactor/timer.rs | 28 +-- tests/async.rs | 13 +- tests/timer.rs | 7 +- 9 files changed, 273 insertions(+), 326 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 2729951..e97a7e9 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -30,9 +30,6 @@ jobs: if: startsWith(matrix.rust, 'nightly') run: cargo check -Z features=dev_dep - run: cargo test - - run: cargo test --no-default-features - - run: cargo test --no-default-features --features io - - run: cargo test --no-default-features --features timer # Copied from: https://github.com/rust-lang/stacker/pull/19/files windows_gnu: @@ -89,6 +86,17 @@ jobs: # if: startsWith(matrix.os, 'ubuntu') # run: cross build --target x86_64-unknown-illumos + wasm: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Install Rust + run: rustup update stable && rustup default stable + - run: rustup target add wasm32-unknown-unknown + - name: Install wasm-pack + uses: taiki-e/install-action@wasm-pack + - run: wasm-pack test --headless --chrome + msrv: runs-on: ubuntu-latest strategy: @@ -101,9 +109,6 @@ jobs: - name: Install Rust run: rustup update ${{ matrix.rust }} && rustup default ${{ matrix.rust }} - run: cargo build - - run: cargo build --no-default-features - - run: cargo build --no-default-features --features io - - run: cargo build --no-default-features --features timer clippy: runs-on: ubuntu-latest diff --git a/Cargo.toml b/Cargo.toml index 6529cde..a7b22a7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -20,20 +20,20 @@ harness = false [dependencies] cfg-if = "1.0.0" -concurrent-queue = { version = "1.2.2", optional = true } +concurrent-queue = { version = "1.2.2" } futures-lite = "1.11.0" log = "0.4.11" once_cell = "1.4.1" parking = "2.0.0" -polling = { version = "2.0.0", optional = true } -slab = { version = "0.4.2", optional = true } -socket2 = { version = "0.4.2", features = ["all"], optional = true } waker-fn = "1.1.0" -[features] -default = ["io", "timer"] -io = ["polling", "slab", "socket2"] -timer = ["concurrent-queue"] +[target.'cfg(not(target_family = "wasm"))'.dependencies] +polling = { version = "2.0.0"} +slab = { version = "0.4.2" } +socket2 = { version = "0.4.2", features = ["all"] } + +[target.'cfg(target_family = "wasm")'.dependencies] +wasm-bindgen-test = "0.2" [build-dependencies] autocfg = "1" @@ -46,12 +46,14 @@ winapi = { version = "0.3.9", features = ["winsock2"] } [dev-dependencies] async-channel = "1" -async-net = "1" blocking = "1" criterion = "0.3.6" +tempfile = "3" + +[target.'cfg(not(target_family = "wasm"))'.dev-dependencies] +async-net = "1" getrandom = "0.2.7" signal-hook = "0.3" -tempfile = "3" [target.'cfg(target_os = "linux")'.dev-dependencies] inotify = { version = "0.10", default-features = false } diff --git a/src/driver.rs b/src/driver.rs index 83947e0..2f7929f 100644 --- a/src/driver.rs +++ b/src/driver.rs @@ -1,241 +1,107 @@ -#[cfg(any(feature = "io", feature = "timer"))] -mod driver_impl { - use std::cell::Cell; - use std::future::Future; - use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; - use std::sync::Arc; - use std::task::{Context, Poll}; - use std::thread; - use std::time::{Duration, Instant}; - - use futures_lite::pin; - use once_cell::sync::Lazy; - use waker_fn::waker_fn; - - use crate::reactor::Reactor; - - /// Number of currently active `block_on()` invocations. - static BLOCK_ON_COUNT: AtomicUsize = AtomicUsize::new(0); - - /// Unparker for the "async-io" thread. - static UNPARKER: Lazy = Lazy::new(|| { - let (parker, unparker) = parking::pair(); - - // Spawn a helper thread driving the reactor. - // - // Note that this thread is not exactly necessary, it's only here to help push things - // forward if there are no `Parker`s around or if `Parker`s are just idling and never - // parking. - thread::Builder::new() - .name("async-io".to_string()) - .spawn(move || main_loop(parker)) - .expect("cannot spawn async-io thread"); - - unparker - }); - - /// Initializes the "async-io" thread. - pub(crate) fn init() { - Lazy::force(&UNPARKER); - } +use std::cell::Cell; +use std::future::Future; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; +use std::task::{Context, Poll}; +use std::time::{Duration, Instant}; + +#[cfg(not(target_family = "wasm"))] +use std::sync::atomic::AtomicUsize; +#[cfg(not(target_family = "wasm"))] +use std::thread; +#[cfg(not(target_family = "wasm"))] +use once_cell::sync::Lazy; + +use futures_lite::pin; +use waker_fn::waker_fn; + +use crate::reactor::Reactor; + +/// Number of currently active `block_on()` invocations. +#[cfg(not(target_family = "wasm"))] +static BLOCK_ON_COUNT: AtomicUsize = AtomicUsize::new(0); + +/// Unparker for the "async-io" thread. +/// +/// The thread is not available on WASM. +#[cfg(not(target_family = "wasm"))] +static UNPARKER: Lazy = Lazy::new(|| { + let (parker, unparker) = parking::pair(); + + // Spawn a helper thread driving the reactor. + // + // Note that this thread is not exactly necessary, it's only here to help push things + // forward if there are no `Parker`s around or if `Parker`s are just idling and never + // parking. + thread::Builder::new() + .name("async-io".to_string()) + .spawn(move || main_loop(parker)) + .expect("cannot spawn async-io thread"); + + unparker +}); + +/// Initializes the "async-io" thread. +pub(crate) fn init() { + #[cfg(not(target_family = "wasm"))] + Lazy::force(&UNPARKER); +} - /// The main loop for the "async-io" thread. - fn main_loop(parker: parking::Parker) { - // The last observed reactor tick. - let mut last_tick = 0; - // Number of sleeps since this thread has called `react()`. - let mut sleeps = 0u64; - - loop { - let tick = Reactor::get().ticker(); - - if last_tick == tick { - let reactor_lock = if sleeps >= 10 { - // If no new ticks have occurred for a while, stop sleeping and spinning in - // this loop and just block on the reactor lock. - Some(Reactor::get().lock()) - } else { - Reactor::get().try_lock() - }; - - if let Some(mut reactor_lock) = reactor_lock { - log::trace!("main_loop: waiting on I/O"); - reactor_lock.react(None).ok(); - last_tick = Reactor::get().ticker(); - sleeps = 0; - } +/// The main loop for the "async-io" thread. +#[cfg(not(target_family = "wasm"))] +fn main_loop(parker: parking::Parker) { + // The last observed reactor tick. + let mut last_tick = 0; + // Number of sleeps since this thread has called `react()`. + let mut sleeps = 0u64; + + loop { + let tick = Reactor::get().ticker(); + + if last_tick == tick { + let reactor_lock = if sleeps >= 10 { + // If no new ticks have occurred for a while, stop sleeping and spinning in + // this loop and just block on the reactor lock. + Some(Reactor::get().lock()) } else { - last_tick = tick; - } - - if BLOCK_ON_COUNT.load(Ordering::SeqCst) > 0 { - // Exponential backoff from 50us to 10ms. - let delay_us = [50, 75, 100, 250, 500, 750, 1000, 2500, 5000] - .get(sleeps as usize) - .unwrap_or(&10_000); - - log::trace!("main_loop: sleeping for {} us", delay_us); - if parker.park_timeout(Duration::from_micros(*delay_us)) { - log::trace!("main_loop: notified"); - - // If notified before timeout, reset the last tick and the sleep counter. - last_tick = Reactor::get().ticker(); - sleeps = 0; - } else { - sleeps += 1; - } + Reactor::get().try_lock() + }; + + if let Some(mut reactor_lock) = reactor_lock { + log::trace!("main_loop: waiting on I/O"); + reactor_lock.react(None).ok(); + last_tick = Reactor::get().ticker(); + sleeps = 0; } + } else { + last_tick = tick; } - } - - pub(crate) fn block_on(future: impl Future) -> T { - log::trace!("block_on()"); - - // Increment `BLOCK_ON_COUNT` so that the "async-io" thread becomes less aggressive. - BLOCK_ON_COUNT.fetch_add(1, Ordering::SeqCst); - - // Make sure to decrement `BLOCK_ON_COUNT` at the end and wake the "async-io" thread. - let _guard = CallOnDrop(|| { - BLOCK_ON_COUNT.fetch_sub(1, Ordering::SeqCst); - UNPARKER.unpark(); - }); - - // Parker and unparker for notifying the current thread. - let (p, u) = parking::pair(); - // This boolean is set to `true` when the current thread is blocked on I/O. - let io_blocked = Arc::new(AtomicBool::new(false)); - - thread_local! { - // Indicates that the current thread is polling I/O, but not necessarily blocked on it. - static IO_POLLING: Cell = Cell::new(false); - } - - // Prepare the waker. - let waker = waker_fn({ - let io_blocked = io_blocked.clone(); - move || { - if u.unpark() { - // Check if waking from another thread and if currently blocked on I/O. - if !IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst) { - Reactor::get().notify(); - } - } - } - }); - let cx = &mut Context::from_waker(&waker); - pin!(future); - - loop { - // Poll the future. - if let Poll::Ready(t) = future.as_mut().poll(cx) { - log::trace!("block_on: completed"); - return t; - } - // Check if a notification was received. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); + if BLOCK_ON_COUNT.load(Ordering::SeqCst) > 0 { + // Exponential backoff from 50us to 10ms. + let delay_us = [50, 75, 100, 250, 500, 750, 1000, 2500, 5000] + .get(sleeps as usize) + .unwrap_or(&10_000); - // Try grabbing a lock on the reactor to process I/O events. - if let Some(mut reactor_lock) = Reactor::get().try_lock() { - // First let wakers know this parker is processing I/O events. - IO_POLLING.with(|io| io.set(true)); - let _guard = CallOnDrop(|| { - IO_POLLING.with(|io| io.set(false)); - }); + log::trace!("main_loop: sleeping for {} us", delay_us); + if parker.park_timeout(Duration::from_micros(*delay_us)) { + log::trace!("main_loop: notified"); - // Process available I/O events. - reactor_lock.react(Some(Duration::from_secs(0))).ok(); - } - continue; - } - - // Try grabbing a lock on the reactor to wait on I/O. - if let Some(mut reactor_lock) = Reactor::get().try_lock() { - // Record the instant at which the lock was grabbed. - let start = Instant::now(); - - loop { - // First let wakers know this parker is blocked on I/O. - IO_POLLING.with(|io| io.set(true)); - io_blocked.store(true, Ordering::SeqCst); - let _guard = CallOnDrop(|| { - IO_POLLING.with(|io| io.set(false)); - io_blocked.store(false, Ordering::SeqCst); - }); - - // Check if a notification has been received before `io_blocked` was updated - // because in that case the reactor won't receive a wakeup. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); - break; - } - - // Wait for I/O events. - log::trace!("block_on: waiting on I/O"); - reactor_lock.react(None).ok(); - - // Check if a notification has been received. - if p.park_timeout(Duration::from_secs(0)) { - log::trace!("block_on: notified"); - break; - } - - // Check if this thread been handling I/O events for a long time. - if start.elapsed() > Duration::from_micros(500) { - log::trace!("block_on: stops hogging the reactor"); - - // This thread is clearly processing I/O events for some other threads - // because it didn't get a notification yet. It's best to stop hogging the - // reactor and give other threads a chance to process I/O events for - // themselves. - drop(reactor_lock); - - // Unpark the "async-io" thread in case no other thread is ready to start - // processing I/O events. This way we prevent a potential latency spike. - UNPARKER.unpark(); - - // Wait for a notification. - p.park(); - break; - } - } + // If notified before timeout, reset the last tick and the sleep counter. + last_tick = Reactor::get().ticker(); + sleeps = 0; } else { - // Wait for an actual notification. - log::trace!("block_on: sleep until notification"); - p.park(); + sleeps += 1; } } } - - /// Runs a closure when dropped. - struct CallOnDrop(F); - - impl Drop for CallOnDrop { - fn drop(&mut self) { - (self.0)(); - } - } -} - -#[cfg(not(any(feature = "io", feature = "timer")))] -mod driver_impl { - use std::future::Future; - - #[inline] - pub(crate) fn block_on(future: impl Future) -> T { - futures_lite::future::block_on(future) - } } -use std::future::Future; - /// Blocks the current thread on a future, processing I/O events when idle. /// /// # Examples /// -#[cfg_attr(feature = "timer", doc = "```")] -#[cfg_attr(not(feature = "timer"), doc = "```no_compile")] +/// ``` /// use async_io::Timer; /// use std::time::Duration; /// @@ -246,8 +112,136 @@ use std::future::Future; /// }); /// ``` pub fn block_on(future: impl Future) -> T { - driver_impl::block_on(future) + log::trace!("block_on()"); + + cfg_if::cfg_if! { + if #[cfg(not(target_family = "wasm"))] { + // Increment `BLOCK_ON_COUNT` so that the "async-io" thread becomes less aggressive. + BLOCK_ON_COUNT.fetch_add(1, Ordering::SeqCst); + + // Make sure to decrement `BLOCK_ON_COUNT` at the end and wake the "async-io" thread. + let _guard = CallOnDrop(|| { + BLOCK_ON_COUNT.fetch_sub(1, Ordering::SeqCst); + UNPARKER.unpark(); + }); + } + } + + // Parker and unparker for notifying the current thread. + let (p, u) = parking::pair(); + // This boolean is set to `true` when the current thread is blocked on I/O. + let io_blocked = Arc::new(AtomicBool::new(false)); + + thread_local! { + // Indicates that the current thread is polling I/O, but not necessarily blocked on it. + static IO_POLLING: Cell = Cell::new(false); + } + + // Prepare the waker. + let waker = waker_fn({ + let io_blocked = io_blocked.clone(); + move || { + if u.unpark() { + // Check if waking from another thread and if currently blocked on I/O. + // Always wake up on WASM. + if cfg!(target_family = "wasm") || (!IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst)) { + Reactor::get().notify(); + } + } + } + }); + let cx = &mut Context::from_waker(&waker); + pin!(future); + + loop { + // Poll the future. + if let Poll::Ready(t) = future.as_mut().poll(cx) { + log::trace!("block_on: completed"); + return t; + } + + // Check if a notification was received. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); + + // Try grabbing a lock on the reactor to process I/O events. + if let Some(mut reactor_lock) = Reactor::get().try_lock() { + // First let wakers know this parker is processing I/O events. + IO_POLLING.with(|io| io.set(true)); + let _guard = CallOnDrop(|| { + IO_POLLING.with(|io| io.set(false)); + }); + + // Process available I/O events. + reactor_lock.react(Some(Duration::from_secs(0))).ok(); + } + continue; + } + + // Try grabbing a lock on the reactor to wait on I/O. + if let Some(mut reactor_lock) = Reactor::get().try_lock() { + // Record the instant at which the lock was grabbed. + let start = Instant::now(); + + loop { + // First let wakers know this parker is blocked on I/O. + IO_POLLING.with(|io| io.set(true)); + io_blocked.store(true, Ordering::SeqCst); + let _guard = CallOnDrop(|| { + IO_POLLING.with(|io| io.set(false)); + io_blocked.store(false, Ordering::SeqCst); + }); + + // Check if a notification has been received before `io_blocked` was updated + // because in that case the reactor won't receive a wakeup. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); + break; + } + + // Wait for I/O events. + log::trace!("block_on: waiting on I/O"); + reactor_lock.react(None).ok(); + + // Check if a notification has been received. + if p.park_timeout(Duration::from_secs(0)) { + log::trace!("block_on: notified"); + break; + } + + // Check if this thread been handling I/O events for a long time. + #[cfg(not(target_family = "wasm"))] + if start.elapsed() > Duration::from_micros(500) { + log::trace!("block_on: stops hogging the reactor"); + + // This thread is clearly processing I/O events for some other threads + // because it didn't get a notification yet. It's best to stop hogging the + // reactor and give other threads a chance to process I/O events for + // themselves. + drop(reactor_lock); + + // Unpark the "async-io" thread in case no other thread is ready to start + // processing I/O events. This way we prevent a potential latency spike. + UNPARKER.unpark(); + + // Wait for a notification. + p.park(); + break; + } + } + } else { + // Wait for an actual notification. + log::trace!("block_on: sleep until notification"); + p.park(); + } + } } -#[cfg(any(feature = "io", feature = "timer"))] -pub(crate) use driver_impl::init; +/// Runs a closure when dropped. +struct CallOnDrop(F); + +impl Drop for CallOnDrop { + fn drop(&mut self) { + (self.0)(); + } +} \ No newline at end of file diff --git a/src/lib.rs b/src/lib.rs index 37e64af..e961756 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -57,17 +57,14 @@ #![warn(missing_docs, missing_debug_implementations, rust_2018_idioms)] mod driver; -#[cfg(feature = "io")] +#[cfg(not(target_family = "wasm"))] mod io; -#[cfg(any(feature = "io", feature = "timer"))] mod reactor; -#[cfg(feature = "timer")] mod timer; pub use driver::block_on; -#[cfg(feature = "io")] +#[cfg(not(target_family = "wasm"))] pub use io::Async; -#[cfg(feature = "io")] +#[cfg(not(target_family = "wasm"))] pub use reactor::{Readable, ReadableOwned, Writable, WritableOwned}; -#[cfg(feature = "timer")] pub use timer::Timer; diff --git a/src/reactor/io.rs b/src/reactor/io.rs index 6b26184..b7c5f29 100644 --- a/src/reactor/io.rs +++ b/src/reactor/io.rs @@ -1,6 +1,4 @@ -#[cfg(feature = "timer")] use super::timer::Reactor as Timers; -#[cfg(feature = "timer")] use std::time::Instant; use std::borrow::Borrow; @@ -14,8 +12,6 @@ use std::os::unix::io::RawFd; use std::os::windows::io::RawSocket; use std::panic; use std::pin::Pin; -#[cfg(not(feature = "timer"))] -use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Arc, Mutex, MutexGuard}; use std::task::{Context, Poll, Waker}; use std::time::Duration; @@ -103,13 +99,11 @@ impl Reactor { /// Registers a timer in the reactor. /// /// Returns the inserted timer's ID. - #[cfg(feature = "timer")] pub(crate) fn insert_timer(&self, when: Instant, waker: &Waker) -> usize { self.timers.insert_timer(when, waker) } /// Deregisters a timer from the reactor. - #[cfg(feature = "timer")] pub(crate) fn remove_timer(&self, when: Instant, id: usize) { self.timers.remove_timer(when, id); } @@ -220,40 +214,6 @@ impl ReactorLock<'_> { } } -/// Shim for `Timers` on targets with timers disabled. -#[cfg(not(feature = "timer"))] -struct Timers { - /// The ticker. - ticker: AtomicUsize, -} - -#[cfg(not(feature = "timer"))] -impl Timers { - fn new() -> Self { - Self { - ticker: AtomicUsize::new(0), - } - } - - fn timeout_duration( - &self, - timeout: Option, - _wakers: &mut Vec, - ) -> Option { - timeout - } - - fn process_timers(&self, _wakers: &mut Vec) {} - - fn ticker(&self) -> usize { - self.ticker.load(Ordering::SeqCst) - } - - fn bump_ticker(&self) -> usize { - self.ticker.fetch_add(1, Ordering::SeqCst).wrapping_add(1) - } -} - /// A registered source of I/O events. #[derive(Debug)] pub(crate) struct Source { diff --git a/src/reactor/mod.rs b/src/reactor/mod.rs index 4d0c2e4..cc03ccb 100644 --- a/src/reactor/mod.rs +++ b/src/reactor/mod.rs @@ -1,31 +1,28 @@ use std::io::Result; -#[cfg(all(feature = "io", unix))] +#[cfg(unix)] use std::os::unix::io::RawFd; -#[cfg(all(feature = "io", windows))] +#[cfg(windows)] use std::os::windows::io::RawSocket; -#[cfg(feature = "io")] +#[cfg(not(target_family = "wasm"))] use std::sync::Arc; -#[cfg(feature = "timer")] use std::task::Waker; -#[cfg(feature = "timer")] use std::time::Instant; use once_cell::sync::Lazy; use std::time::Duration; -#[cfg(feature = "timer")] mod timer; cfg_if::cfg_if! { - if #[cfg(feature = "io")] { + if #[cfg(target_family = "wasm")] { + use timer as sys; + } else { mod io; use io as sys; pub use io::{Readable, ReadableOwned, Writable, WritableOwned}; pub(crate) use io::Source; - } else { - use timer as sys; } } @@ -50,7 +47,7 @@ impl Reactor { } /// Registers an I/O source in the reactor. - #[cfg(feature = "io")] + #[cfg(not(target_family = "wasm"))] pub(crate) fn insert_io( &self, #[cfg(unix)] raw: RawFd, @@ -60,7 +57,7 @@ impl Reactor { } /// Deregisters an I/O source from the reactor. - #[cfg(feature = "io")] + #[cfg(not(target_family = "wasm"))] pub(crate) fn remove_io(&self, source: &Source) -> Result<()> { self.0.remove_io(source) } @@ -68,13 +65,11 @@ impl Reactor { /// Registers a timer in the reactor. /// /// Returns the inserted timer's ID. - #[cfg(feature = "timer")] pub(crate) fn insert_timer(&self, when: Instant, waker: &Waker) -> usize { self.0.insert_timer(when, waker) } /// Deregisters a timer from the reactor. - #[cfg(feature = "timer")] pub(crate) fn remove_timer(&self, when: Instant, id: usize) { self.0.remove_timer(when, id) } diff --git a/src/reactor/timer.rs b/src/reactor/timer.rs index ef9e171..aa91894 100644 --- a/src/reactor/timer.rs +++ b/src/reactor/timer.rs @@ -1,9 +1,9 @@ use std::cmp; use std::collections::BTreeMap; -#[cfg(not(feature = "io"))] +#[cfg(target_family = "wasm")] use std::io; use std::mem; -#[cfg(not(feature = "io"))] +#[cfg(target_family = "wasm")] use std::panic; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Mutex, MutexGuard}; @@ -11,7 +11,7 @@ use std::task::Waker; use std::time::{Duration, Instant}; use concurrent_queue::ConcurrentQueue; -#[cfg(not(feature = "io"))] +#[cfg(target_family = "wasm")] use parking::{pair, Parker, Unparker}; /// A reactor that is capable of processing timers. @@ -40,29 +40,29 @@ pub(crate) struct Reactor { /// The object used to unpark the reactor. /// /// Not needed when I/O is available since we block on `polling` instead. - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] unparker: Unparker, /// The object used to park the reactor. /// /// The mutex used to hold this value implies the exclusive right to poll the reactor. - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] parker: Mutex, } impl Reactor { /// Create a new `Reactor`. - pub(crate) fn new() -> Reactor { - #[cfg(not(feature = "io"))] + pub(crate) fn new() -> Reactor { + #[cfg(target_family = "wasm")] let (parker, unparker) = pair(); Reactor { timers: Mutex::new(BTreeMap::new()), timer_ops: ConcurrentQueue::bounded(1000), ticker: AtomicUsize::new(0), - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] unparker, - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] parker: Mutex::new(parker), } } @@ -113,13 +113,13 @@ impl Reactor { } /// Notify the thread blocked on this reactor. - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] pub(crate) fn notify(&self) { self.unparker.unpark(); } /// Acquire a lock on the reactor. - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] pub(crate) fn lock(&self) -> ReactorLock<'_> { let reactor = self; let parker = reactor.parker.lock().unwrap(); @@ -127,7 +127,7 @@ impl Reactor { } /// Try to acquire a lock on the reactor. - #[cfg(not(feature = "io"))] + #[cfg(target_family = "wasm")] pub(crate) fn try_lock(&self) -> Option> { let reactor = self; let parker = reactor.parker.try_lock().ok()?; @@ -208,7 +208,7 @@ impl Reactor { } } -#[cfg(not(feature = "io"))] +#[cfg(target_family = "wasm")] pub(crate) struct ReactorLock<'a> { /// The reference to the reactor. reactor: &'a Reactor, @@ -217,7 +217,7 @@ pub(crate) struct ReactorLock<'a> { parker: MutexGuard<'a, Parker>, } -#[cfg(not(feature = "io"))] +#[cfg(target_family = "wasm")] impl ReactorLock<'_> { /// Processes timers and then blocks until the next timer is ready. pub(crate) fn react(&self, timeout: Option) -> io::Result<()> { diff --git a/tests/async.rs b/tests/async.rs index 9fd6d22..5d9bd12 100644 --- a/tests/async.rs +++ b/tests/async.rs @@ -1,19 +1,15 @@ -#![cfg(feature = "io")] +#![cfg(not(target_family = "wasm"))] use std::future::Future; use std::io; use std::net::{Shutdown, TcpListener, TcpStream, UdpSocket}; #[cfg(unix)] use std::os::unix::net::{UnixDatagram, UnixListener, UnixStream}; -#[cfg(feature = "timer")] use std::sync::Arc; use std::thread; -#[cfg(feature = "timer")] use std::time::Duration; -use async_io::Async; -#[cfg(feature = "timer")] -use async_io::Timer; +use async_io::{Async, Timer}; use futures_lite::{future, prelude::*}; #[cfg(unix)] use tempfile::tempdir; @@ -88,7 +84,6 @@ fn tcp_peek_read() -> io::Result<()> { }) } -#[cfg(feature = "timer")] #[test] fn tcp_reader_hangup() -> io::Result<()> { future::block_on(async { @@ -111,7 +106,6 @@ fn tcp_reader_hangup() -> io::Result<()> { }) } -#[cfg(feature = "timer")] #[test] fn tcp_writer_hangup() -> io::Result<()> { future::block_on(async { @@ -252,7 +246,6 @@ fn uds_send_to_recv_from() -> io::Result<()> { }) } -#[cfg(feature = "timer")] #[cfg(unix)] #[test] fn uds_reader_hangup() -> io::Result<()> { @@ -271,7 +264,6 @@ fn uds_reader_hangup() -> io::Result<()> { }) } -#[cfg(feature = "timer")] #[cfg(unix)] #[test] fn uds_writer_hangup() -> io::Result<()> { @@ -295,7 +287,6 @@ fn uds_writer_hangup() -> io::Result<()> { // Test that we correctly re-register interests after we've previously been // interested in both readable and writable events and then we get only one of // those (we need to re-register interest on the other). -#[cfg(feature = "timer")] #[test] fn tcp_duplex() -> io::Result<()> { future::block_on(async { diff --git a/tests/timer.rs b/tests/timer.rs index e7c150b..5424705 100644 --- a/tests/timer.rs +++ b/tests/timer.rs @@ -1,5 +1,3 @@ -#![cfg(feature = "timer")] - use std::future::Future; use std::pin::Pin; use std::sync::{Arc, Mutex}; @@ -9,6 +7,9 @@ use std::time::{Duration, Instant}; use async_io::Timer; use futures_lite::{future, FutureExt, StreamExt}; +#[cfg(target_family = "wasm")] +use wasm_bindgen_test::*; + fn spawn( f: impl Future + Send + 'static, ) -> impl Future + Send + 'static { @@ -23,6 +24,7 @@ fn spawn( Box::pin(async move { r.recv().await.unwrap() }) } +#[cfg_attr(target_family = "wasm", wasm_bindgen_test)] #[test] fn smoke() { future::block_on(async { @@ -32,6 +34,7 @@ fn smoke() { }); } +#[cfg_attr(target_family = "wasm", wasm_bindgen_test)] #[test] fn interval() { future::block_on(async { From aec0234bf613bf92f402badb7e2681a5112fee32 Mon Sep 17 00:00:00 2001 From: jtnunley Date: Thu, 8 Sep 2022 10:46:57 -0700 Subject: [PATCH 4/6] fmt + fix wasm warns --- src/driver.rs | 15 ++++++++++----- src/reactor/io.rs | 18 ++++++++++++++++-- src/reactor/mod.rs | 2 ++ src/reactor/timer.rs | 32 +------------------------------- src/timer.rs | 2 +- 5 files changed, 30 insertions(+), 39 deletions(-) diff --git a/src/driver.rs b/src/driver.rs index 2f7929f..e295856 100644 --- a/src/driver.rs +++ b/src/driver.rs @@ -3,14 +3,16 @@ use std::future::Future; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::task::{Context, Poll}; -use std::time::{Duration, Instant}; +use std::time::Duration; +#[cfg(not(target_family = "wasm"))] +use once_cell::sync::Lazy; #[cfg(not(target_family = "wasm"))] use std::sync::atomic::AtomicUsize; #[cfg(not(target_family = "wasm"))] use std::thread; #[cfg(not(target_family = "wasm"))] -use once_cell::sync::Lazy; +use std::time::Instant; use futures_lite::pin; use waker_fn::waker_fn; @@ -22,7 +24,7 @@ use crate::reactor::Reactor; static BLOCK_ON_COUNT: AtomicUsize = AtomicUsize::new(0); /// Unparker for the "async-io" thread. -/// +/// /// The thread is not available on WASM. #[cfg(not(target_family = "wasm"))] static UNPARKER: Lazy = Lazy::new(|| { @@ -144,7 +146,9 @@ pub fn block_on(future: impl Future) -> T { if u.unpark() { // Check if waking from another thread and if currently blocked on I/O. // Always wake up on WASM. - if cfg!(target_family = "wasm") || (!IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst)) { + if cfg!(target_family = "wasm") + || (!IO_POLLING.with(Cell::get) && io_blocked.load(Ordering::SeqCst)) + { Reactor::get().notify(); } } @@ -181,6 +185,7 @@ pub fn block_on(future: impl Future) -> T { // Try grabbing a lock on the reactor to wait on I/O. if let Some(mut reactor_lock) = Reactor::get().try_lock() { // Record the instant at which the lock was grabbed. + #[cfg(not(target_family = "wasm"))] let start = Instant::now(); loop { @@ -244,4 +249,4 @@ impl Drop for CallOnDrop { fn drop(&mut self) { (self.0)(); } -} \ No newline at end of file +} diff --git a/src/reactor/io.rs b/src/reactor/io.rs index b7c5f29..4a80c5a 100644 --- a/src/reactor/io.rs +++ b/src/reactor/io.rs @@ -12,6 +12,7 @@ use std::os::unix::io::RawFd; use std::os::windows::io::RawSocket; use std::panic; use std::pin::Pin; +use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Arc, Mutex, MutexGuard}; use std::task::{Context, Poll, Waker}; use std::time::Duration; @@ -35,6 +36,14 @@ pub(crate) struct Reactor { /// Registered sources. sources: Mutex>>, + /// Ticker bumped before polling. + /// + /// This is useful for checking what is the current "round" of `ReactorLock::react()` when + /// synchronizing things in `Source::readable()` and `Source::writable()`. Both of those + /// methods must make sure they don't receive stale I/O events - they only accept events from a + /// fresh "round" of `ReactorLock::react()`. + ticker: AtomicUsize, + /// Temporary storage for I/O events when polling the reactor. /// /// Holding a lock on this event list implies the exclusive right to poll I/O. @@ -50,6 +59,7 @@ impl Reactor { Reactor { poller: Poller::new().expect("cannot initialize I/O event notification"), sources: Mutex::new(Slab::new()), + ticker: AtomicUsize::new(0), events: Mutex::new(Vec::new()), timers: Timers::new(), } @@ -57,7 +67,7 @@ impl Reactor { /// Returns the current ticker. pub(crate) fn ticker(&self) -> usize { - self.timers.ticker() + self.ticker.load(Ordering::SeqCst) } /// Registers an I/O source in the reactor. @@ -144,7 +154,11 @@ impl ReactorLock<'_> { let timeout = self.reactor.timers.timeout_duration(timeout, &mut wakers); // Bump the ticker before polling I/O. - let tick = self.reactor.timers.bump_ticker(); + let tick = self + .reactor + .ticker + .fetch_add(1, Ordering::SeqCst) + .wrapping_add(1); self.events.clear(); diff --git a/src/reactor/mod.rs b/src/reactor/mod.rs index cc03ccb..06906c2 100644 --- a/src/reactor/mod.rs +++ b/src/reactor/mod.rs @@ -42,6 +42,7 @@ impl Reactor { } /// Returns the current ticker. + #[cfg(not(target_family = "wasm"))] pub(crate) fn ticker(&self) -> usize { self.0.ticker() } @@ -80,6 +81,7 @@ impl Reactor { } /// Locks the reactor, potentially blocking if the lock is held by another thread. + #[cfg(not(target_family = "wasm"))] pub(crate) fn lock(&self) -> ReactorLock<'_> { ReactorLock(self.0.lock()) } diff --git a/src/reactor/timer.rs b/src/reactor/timer.rs index aa91894..5a403a9 100644 --- a/src/reactor/timer.rs +++ b/src/reactor/timer.rs @@ -29,14 +29,6 @@ pub(crate) struct Reactor { /// this queue. Timers actually get processed when the queue fills up or the reactor is polled. timer_ops: ConcurrentQueue, - /// Ticker bumped before polling. - /// - /// This is useful for checking what is the current "round" of `ReactorLock::react()` when - /// synchronizing things in `Source::readable()` and `Source::writable()`. Both of those - /// methods must make sure they don't receive stale I/O events - they only accept events from a - /// fresh "round" of `ReactorLock::react()`. - ticker: AtomicUsize, - /// The object used to unpark the reactor. /// /// Not needed when I/O is available since we block on `polling` instead. @@ -52,14 +44,13 @@ pub(crate) struct Reactor { impl Reactor { /// Create a new `Reactor`. - pub(crate) fn new() -> Reactor { + pub(crate) fn new() -> Reactor { #[cfg(target_family = "wasm")] let (parker, unparker) = pair(); Reactor { timers: Mutex::new(BTreeMap::new()), timer_ops: ConcurrentQueue::bounded(1000), - ticker: AtomicUsize::new(0), #[cfg(target_family = "wasm")] unparker, #[cfg(target_family = "wasm")] @@ -67,16 +58,6 @@ impl Reactor { } } - /// Returns the current ticker. - pub(crate) fn ticker(&self) -> usize { - self.ticker.load(Ordering::SeqCst) - } - - /// Bump the current ticker and return the new value. - pub(crate) fn bump_ticker(&self) -> usize { - self.ticker.fetch_add(1, Ordering::SeqCst).wrapping_add(1) - } - /// Registers a timer in the reactor. /// /// Returns the inserted timer's ID. @@ -118,14 +99,6 @@ impl Reactor { self.unparker.unpark(); } - /// Acquire a lock on the reactor. - #[cfg(target_family = "wasm")] - pub(crate) fn lock(&self) -> ReactorLock<'_> { - let reactor = self; - let parker = reactor.parker.lock().unwrap(); - ReactorLock { reactor, parker } - } - /// Try to acquire a lock on the reactor. #[cfg(target_family = "wasm")] pub(crate) fn try_lock(&self) -> Option> { @@ -225,9 +198,6 @@ impl ReactorLock<'_> { let mut wakers = vec![]; let timeout = self.reactor.timeout_duration(timeout, &mut wakers); - // Bump the ticker. - self.reactor.bump_ticker(); - // Park the thread until either the timeout duration has elapsed or a timer is ready. let timers_ready = match timeout { None => { diff --git a/src/timer.rs b/src/timer.rs index 503ce80..dc8a4d6 100644 --- a/src/timer.rs +++ b/src/timer.rs @@ -382,4 +382,4 @@ impl Stream for Timer { Poll::Pending } -} \ No newline at end of file +} From 03f3f04bdd84fc0a85a8f00b4bd2eff7ac029b2b Mon Sep 17 00:00:00 2001 From: jtnunley Date: Tue, 25 Oct 2022 09:41:17 -0700 Subject: [PATCH 5/6] Review from taiki-e --- Cargo.toml | 2 +- examples/linux-inotify.rs | 4 ++-- examples/linux-timerfd.rs | 4 ++-- examples/unix-signal.rs | 4 ++-- examples/windows-uds.rs | 4 ++-- src/lib.rs | 4 ++-- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a7b22a7..60c67fc 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,7 +32,7 @@ polling = { version = "2.0.0"} slab = { version = "0.4.2" } socket2 = { version = "0.4.2", features = ["all"] } -[target.'cfg(target_family = "wasm")'.dependencies] +[target.'cfg(target_family = "wasm")'.dev-dependencies] wasm-bindgen-test = "0.2" [build-dependencies] diff --git a/examples/linux-inotify.rs b/examples/linux-inotify.rs index f846f87..c168f63 100644 --- a/examples/linux-inotify.rs +++ b/examples/linux-inotify.rs @@ -6,7 +6,7 @@ //! cargo run --example linux-inotify //! ``` -#[cfg(all(feature = "io", target_os = "linux"))] +#[cfg(target_os = "linux")] fn main() -> std::io::Result<()> { use std::ffi::OsString; use std::io; @@ -51,7 +51,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(all(feature = "io", target_os = "linux")))] +#[cfg(not(target_os = "linux"))] fn main() { println!("This example works only on Linux!"); } diff --git a/examples/linux-timerfd.rs b/examples/linux-timerfd.rs index 045b2a4..e57053b 100644 --- a/examples/linux-timerfd.rs +++ b/examples/linux-timerfd.rs @@ -6,7 +6,7 @@ //! cargo run --example linux-timerfd //! ``` -#[cfg(all(feature = "io", target_os = "linux"))] +#[cfg(target_os = "linux")] fn main() -> std::io::Result<()> { use std::io; use std::os::unix::io::AsRawFd; @@ -41,7 +41,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(all(feature = "io", target_os = "linux")))] +#[cfg(not(target_os = "linux"))] fn main() { println!("This example works only on Linux!"); } diff --git a/examples/unix-signal.rs b/examples/unix-signal.rs index ed485aa..e712893 100644 --- a/examples/unix-signal.rs +++ b/examples/unix-signal.rs @@ -6,7 +6,7 @@ //! cargo run --example unix-signal //! ``` -#[cfg(all(feature = "io", unix))] +#[cfg(unix)] fn main() -> std::io::Result<()> { use std::os::unix::{io::AsRawFd, net::UnixStream}; @@ -27,7 +27,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(all(feature = "io", unix)))] +#[cfg(not(unix))] fn main() { println!("This example works only on Unix systems!"); } diff --git a/examples/windows-uds.rs b/examples/windows-uds.rs index 44d97d7..0980d1b 100644 --- a/examples/windows-uds.rs +++ b/examples/windows-uds.rs @@ -6,7 +6,7 @@ //! cargo run --example windows-uds //! ``` -#[cfg(all(feature = "io", windows))] +#[cfg(windows)] fn main() -> std::io::Result<()> { use std::path::PathBuf; @@ -53,7 +53,7 @@ fn main() -> std::io::Result<()> { }) } -#[cfg(not(all(feature = "io", windows)))] +#[cfg(not(windows))] fn main() { println!("This example works only on Windows!"); } diff --git a/src/lib.rs b/src/lib.rs index e961756..62deab9 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -35,8 +35,8 @@ //! //! Connect to `example.com:80`, or time out after 10 seconds. //! -#![cfg_attr(all(feature = "timer", feature = "io"), doc = "```")] -#![cfg_attr(not(all(feature = "timer", feature = "io")), doc = "```no_compile")] +#![cfg_attr(not(target_family = "wasm"), doc = "```")] +#![cfg_attr(target_family = "wasm", doc = "```no_compile")] //! use async_io::{Async, Timer}; //! use futures_lite::{future::FutureExt, io}; //! From b4f13bbfd6079abf364231dc978af6b41590934c Mon Sep 17 00:00:00 2001 From: jtnunley Date: Sun, 27 Nov 2022 13:10:41 -0800 Subject: [PATCH 6/6] Incomplete support --- .github/workflows/ci.yml | 6 ++++-- Cargo.toml | 19 ++++++++++++------- src/reactor/io.rs | 2 +- src/reactor/mod.rs | 3 +-- src/reactor/timer.rs | 3 ++- src/timer.rs | 2 +- tests/timer.rs | 2 +- 7 files changed, 22 insertions(+), 15 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e97a7e9..52eb734 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -91,11 +91,13 @@ jobs: steps: - uses: actions/checkout@v3 - name: Install Rust - run: rustup update stable && rustup default stable + run: rustup update nightly && rustup default nightly - run: rustup target add wasm32-unknown-unknown - name: Install wasm-pack uses: taiki-e/install-action@wasm-pack - - run: wasm-pack test --headless --chrome + - run: wasm-pack test --headless --chrome -Zbuild-std + env: + RUSTFLAGS: "${{ env.RUSTFLAGS }} -Ctarget-feature=+atomics,+bulk-memory,+mutable-globals" msrv: runs-on: ubuntu-latest diff --git a/Cargo.toml b/Cargo.toml index 60c67fc..5a4343a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,10 +18,15 @@ exclude = ["/.*"] name = "io" harness = false +[features] +stdweb = ["instant/stdweb"] +wasm-bindgen = ["instant/wasm-bindgen"] + [dependencies] cfg-if = "1.0.0" concurrent-queue = { version = "1.2.2" } futures-lite = "1.11.0" +instant = "0.1.12" log = "0.4.11" once_cell = "1.4.1" parking = "2.0.0" @@ -32,12 +37,6 @@ polling = { version = "2.0.0"} slab = { version = "0.4.2" } socket2 = { version = "0.4.2", features = ["all"] } -[target.'cfg(target_family = "wasm")'.dev-dependencies] -wasm-bindgen-test = "0.2" - -[build-dependencies] -autocfg = "1" - [target."cfg(unix)".dependencies] libc = "0.2.77" @@ -47,7 +46,7 @@ winapi = { version = "0.3.9", features = ["winsock2"] } [dev-dependencies] async-channel = "1" blocking = "1" -criterion = "0.3.6" +criterion = { version = "0.4", default-features = false, features = ["cargo_bench_support"] } tempfile = "3" [target.'cfg(not(target_family = "wasm"))'.dev-dependencies] @@ -55,6 +54,9 @@ async-net = "1" getrandom = "0.2.7" signal-hook = "0.3" +[target.'cfg(target_family = "wasm")'.dev-dependencies] +wasm-bindgen-test = "0.3" + [target.'cfg(target_os = "linux")'.dev-dependencies] inotify = { version = "0.10", default-features = false } nix = { version = "0.24", default-features = false } @@ -62,3 +64,6 @@ timerfd = "1" [target.'cfg(windows)'.dev-dependencies] uds_windows = "1" + +[build-dependencies] +autocfg = "1" diff --git a/src/reactor/io.rs b/src/reactor/io.rs index 4a80c5a..ddd4e3f 100644 --- a/src/reactor/io.rs +++ b/src/reactor/io.rs @@ -15,9 +15,9 @@ use std::pin::Pin; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Arc, Mutex, MutexGuard}; use std::task::{Context, Poll, Waker}; -use std::time::Duration; use futures_lite::ready; +use instant::{Duration, Instant}; use polling::{Event, Poller}; use slab::Slab; diff --git a/src/reactor/mod.rs b/src/reactor/mod.rs index 06906c2..0a2ddc0 100644 --- a/src/reactor/mod.rs +++ b/src/reactor/mod.rs @@ -7,10 +7,9 @@ use std::os::windows::io::RawSocket; use std::sync::Arc; use std::task::Waker; -use std::time::Instant; +use instant::{Duration, Instant}; use once_cell::sync::Lazy; -use std::time::Duration; mod timer; diff --git a/src/reactor/timer.rs b/src/reactor/timer.rs index 5a403a9..077f8e1 100644 --- a/src/reactor/timer.rs +++ b/src/reactor/timer.rs @@ -8,7 +8,8 @@ use std::panic; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::{Mutex, MutexGuard}; use std::task::Waker; -use std::time::{Duration, Instant}; + +use instant::{Duration, Instant}; use concurrent_queue::ConcurrentQueue; #[cfg(target_family = "wasm")] diff --git a/src/timer.rs b/src/timer.rs index dc8a4d6..ee7f319 100644 --- a/src/timer.rs +++ b/src/timer.rs @@ -1,10 +1,10 @@ use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll, Waker}; -use std::time::{Duration, Instant}; use crate::reactor::Reactor; use futures_lite::Stream; +use instant::{Duration, Instant}; /// Use `Duration::MAX` once `duration_constants` are stabilized. fn duration_max() -> Duration { diff --git a/tests/timer.rs b/tests/timer.rs index 5424705..a297b3d 100644 --- a/tests/timer.rs +++ b/tests/timer.rs @@ -2,10 +2,10 @@ use std::future::Future; use std::pin::Pin; use std::sync::{Arc, Mutex}; use std::thread; -use std::time::{Duration, Instant}; use async_io::Timer; use futures_lite::{future, FutureExt, StreamExt}; +use instant::{Duration, Instant}; #[cfg(target_family = "wasm")] use wasm_bindgen_test::*;