From cd0e9732047ed2bde4bd70eac2cfd207cd7dea4c Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 13 Nov 2019 23:18:38 -0800 Subject: [PATCH 01/15] remove `futures` dependency from `tokio` In an effort to reach API stability, the `tokio` crate is shedding its _public_ dependencies on crates that are either a) do not provide a stable (1.0+) release with longevity guarantees or b) match the `tokio` release cadence. Of course, implmementing `std` traits fits the requirements. Instead of publically dependending on `futures` from `tokio`, the `tokio-util` crate provides the necessary bridge to go from Tokio types to `futures` traits (primarily `Stream`). Instead of implementing `Stream` for iteration, types that are logically streams provide `async fn next_*` functions to obtain the next value. Avoiding the `next()` function name will allow for forwards compatibility once the `Stream` trait is provided by `std`. Additionally, some misc cleanup is also done: - `tokio::io::io` -> `tokio::io::util`. - `delay` -> `delay_until`. - `Timeout::new` -> `timeout(...)`. - `signal::ctrl_c()` returns a future instead of a stream. - `{tcp,unix}::Incoming` is removed (due to lack of `Stream` trait). - `time::Throttle` is removed (due to lack of `Stream` trait). --- tests-integration/tests/process_stdio.rs | 2 +- tokio-test/src/io.rs | 4 +- tokio-test/tests/block_on.rs | 4 +- tokio-util/Cargo.toml | 2 +- tokio-util/src/fs/mod.rs | 1 + tokio-util/src/fs/read_dir.rs | 19 ++ tokio-util/src/io/lines.rs | 22 ++ tokio-util/src/io/mod.rs | 1 + tokio-util/src/io/split.rs | 21 ++ tokio-util/src/lib.rs | 5 + tokio-util/src/stream/as_std.rs | 0 tokio-util/src/stream/into_std.rs | 19 ++ tokio-util/src/stream/mod.rs | 117 +++++++++++ tokio-util/src/sync/mod.rs | 2 + tokio-util/src/sync/mpsc.rs | 21 ++ tokio-util/src/sync/watch.rs | 17 ++ tokio-util/src/time/interval.rs | 15 ++ tokio-util/src/time/mod.rs | 1 + tokio-util/tests/sync_watch.rs | 44 ++++ tokio/Cargo.toml | 3 - tokio/src/fs/blocking.rs | 1 - tokio/src/fs/file.rs | 3 +- tokio/src/fs/read_dir.rs | 37 ++-- tokio/src/future.rs | 69 ------- tokio/src/future/maybe_done.rs | 76 +++++++ tokio/src/future/mod.rs | 15 ++ tokio/src/future/pending.rs | 44 ++++ tokio/src/future/poll_fn.rs | 38 ++++ tokio/src/future/ready.rs | 27 +++ tokio/src/future/try_join.rs | 115 +++++++++++ tokio/src/io/async_read.rs | 1 - tokio/src/io/async_write.rs | 1 - tokio/src/io/io/lines.rs | 70 ------- tokio/src/io/io/split.rs | 67 ------ tokio/src/io/mod.rs | 15 +- tokio/src/io/split.rs | 1 - .../src/io/{io => util}/async_buf_read_ext.rs | 30 ++- tokio/src/io/{io => util}/async_read_ext.rs | 14 +- tokio/src/io/{io => util}/async_write_ext.rs | 8 +- tokio/src/io/{io => util}/buf_reader.rs | 3 +- tokio/src/io/{io => util}/buf_stream.rs | 2 +- tokio/src/io/{io => util}/buf_writer.rs | 3 +- tokio/src/io/{io => util}/chain.rs | 1 - tokio/src/io/{io => util}/copy.rs | 1 - tokio/src/io/{io => util}/empty.rs | 0 tokio/src/io/{io => util}/flush.rs | 0 tokio/src/io/util/lines.rs | 92 +++++++++ tokio/src/io/{io => util}/mod.rs | 66 +++--- tokio/src/io/{io => util}/read.rs | 0 tokio/src/io/{io => util}/read_exact.rs | 1 - tokio/src/io/{io => util}/read_line.rs | 3 +- tokio/src/io/{io => util}/read_to_end.rs | 1 - tokio/src/io/{io => util}/read_to_string.rs | 3 +- tokio/src/io/{io => util}/read_until.rs | 1 - tokio/src/io/{io => util}/repeat.rs | 0 tokio/src/io/{io => util}/shutdown.rs | 0 tokio/src/io/{io => util}/sink.rs | 0 tokio/src/io/util/split.rs | 88 ++++++++ tokio/src/io/{io => util}/take.rs | 1 - tokio/src/io/{io => util}/write.rs | 0 tokio/src/io/{io => util}/write_all.rs | 1 - tokio/src/lib.rs | 13 +- tokio/src/net/addr.rs | 9 +- tokio/src/net/tcp/incoming.rs | 31 --- tokio/src/net/tcp/listener.rs | 24 +-- tokio/src/net/tcp/mod.rs | 5 - tokio/src/net/tcp/stream.rs | 3 +- tokio/src/net/udp/socket.rs | 3 +- tokio/src/net/udp/split.rs | 4 +- tokio/src/net/unix/datagram.rs | 3 +- tokio/src/net/unix/incoming.rs | 29 --- tokio/src/net/unix/listener.rs | 12 +- tokio/src/net/unix/mod.rs | 3 - tokio/src/net/unix/stream.rs | 3 +- tokio/src/net/util/poll_evented.rs | 3 +- tokio/src/prelude.rs | 15 -- tokio/src/process/mod.rs | 32 +-- tokio/src/process/unix/mod.rs | 30 +-- tokio/src/process/unix/reap.rs | 38 ++-- tokio/src/runtime/thread_pool/tests/pool.rs | 2 +- tokio/src/runtime/thread_pool/tests/queue.rs | 2 +- tokio/src/signal/ctrl_c.rs | 37 +--- tokio/src/signal/mod.rs | 56 ++--- tokio/src/signal/registry.rs | 21 +- tokio/src/signal/unix.rs | 32 +-- tokio/src/stream.rs | 78 ------- tokio/src/stream/mod.rs | 10 + tokio/src/sync/barrier.rs | 3 +- tokio/src/sync/mpsc/bounded.rs | 36 +--- tokio/src/sync/mpsc/unbounded.rs | 32 +-- tokio/src/sync/mutex.rs | 2 +- tokio/src/sync/oneshot.rs | 3 +- tokio/src/sync/tests/loom_atomic_waker.rs | 2 +- tokio/src/sync/tests/loom_mpsc.rs | 2 +- tokio/src/sync/tests/loom_oneshot.rs | 2 +- tokio/src/sync/tests/loom_semaphore.rs | 3 +- tokio/src/sync/watch.rs | 112 +++------- tokio/src/task/tests/loom.rs | 6 +- tokio/src/task/tests/task.rs | 2 +- tokio/src/time/clock.rs | 4 +- tokio/src/time/delay.rs | 54 ++--- tokio/src/time/delay_queue.rs | 23 +-- tokio/src/time/driver/registration.rs | 8 - tokio/src/time/interval.rs | 151 ++++++++------ tokio/src/time/mod.rs | 32 +-- tokio/src/time/tests/test_delay.rs | 52 ++--- tokio/src/time/tests/test_queue.rs | 2 +- tokio/src/time/throttle.rs | 17 +- tokio/src/time/timeout.rs | 195 +++++++----------- tokio/tests/fs_dir.rs | 17 +- tokio/tests/fs_file_mocked.rs | 11 + tokio/tests/io_lines.rs | 10 +- tokio/tests/net_driver.rs | 2 +- tokio/tests/process_issue_42.rs | 4 +- tokio/tests/rt_common.rs | 2 +- tokio/tests/signal_ctrl_c.rs | 7 +- tokio/tests/signal_drop_recv.rs | 5 +- tokio/tests/signal_drop_rt.rs | 5 +- tokio/tests/signal_drop_signal.rs | 5 +- tokio/tests/signal_multi_rt.rs | 5 +- tokio/tests/signal_notify_both.rs | 11 +- tokio/tests/signal_twice.rs | 6 +- tokio/tests/signal_usr1.rs | 5 +- tokio/tests/sync_mpsc.rs | 67 ------ tokio/tests/sync_watch.rs | 120 +++-------- tokio/tests/time_interval.rs | 32 +-- tokio/tests/time_rt.rs | 11 +- tokio/tests/time_throttle.rs | 68 ------ tokio/tests/time_timeout.rs | 71 +------ 129 files changed, 1429 insertions(+), 1495 deletions(-) create mode 100644 tokio-util/src/fs/mod.rs create mode 100644 tokio-util/src/fs/read_dir.rs create mode 100644 tokio-util/src/io/lines.rs create mode 100644 tokio-util/src/io/mod.rs create mode 100644 tokio-util/src/io/split.rs create mode 100644 tokio-util/src/stream/as_std.rs create mode 100644 tokio-util/src/stream/into_std.rs create mode 100644 tokio-util/src/stream/mod.rs create mode 100644 tokio-util/src/sync/mod.rs create mode 100644 tokio-util/src/sync/mpsc.rs create mode 100644 tokio-util/src/sync/watch.rs create mode 100644 tokio-util/src/time/interval.rs create mode 100644 tokio-util/src/time/mod.rs create mode 100644 tokio-util/tests/sync_watch.rs delete mode 100644 tokio/src/future.rs create mode 100644 tokio/src/future/maybe_done.rs create mode 100644 tokio/src/future/mod.rs create mode 100644 tokio/src/future/pending.rs create mode 100644 tokio/src/future/poll_fn.rs create mode 100644 tokio/src/future/ready.rs create mode 100644 tokio/src/future/try_join.rs delete mode 100644 tokio/src/io/io/lines.rs delete mode 100644 tokio/src/io/io/split.rs rename tokio/src/io/{io => util}/async_buf_read_ext.rs (83%) rename tokio/src/io/{io => util}/async_read_ext.rs (88%) rename tokio/src/io/{io => util}/async_write_ext.rs (82%) rename tokio/src/io/{io => util}/buf_reader.rs (98%) rename tokio/src/io/{io => util}/buf_stream.rs (98%) rename tokio/src/io/{io => util}/buf_writer.rs (98%) rename tokio/src/io/{io => util}/chain.rs (99%) rename tokio/src/io/{io => util}/copy.rs (99%) rename tokio/src/io/{io => util}/empty.rs (100%) rename tokio/src/io/{io => util}/flush.rs (100%) create mode 100644 tokio/src/io/util/lines.rs rename tokio/src/io/{io => util}/mod.rs (88%) rename tokio/src/io/{io => util}/read.rs (100%) rename tokio/src/io/{io => util}/read_exact.rs (98%) rename tokio/src/io/{io => util}/read_line.rs (96%) rename tokio/src/io/{io => util}/read_to_end.rs (99%) rename tokio/src/io/{io => util}/read_to_string.rs (96%) rename tokio/src/io/{io => util}/read_until.rs (98%) rename tokio/src/io/{io => util}/repeat.rs (100%) rename tokio/src/io/{io => util}/shutdown.rs (100%) rename tokio/src/io/{io => util}/sink.rs (100%) create mode 100644 tokio/src/io/util/split.rs rename tokio/src/io/{io => util}/take.rs (99%) rename tokio/src/io/{io => util}/write.rs (100%) rename tokio/src/io/{io => util}/write_all.rs (98%) delete mode 100644 tokio/src/net/tcp/incoming.rs delete mode 100644 tokio/src/net/unix/incoming.rs delete mode 100644 tokio/src/stream.rs create mode 100644 tokio/src/stream/mod.rs delete mode 100644 tokio/tests/time_throttle.rs diff --git a/tests-integration/tests/process_stdio.rs b/tests-integration/tests/process_stdio.rs index 7e4f3d3e40b..5e284535e1b 100644 --- a/tests-integration/tests/process_stdio.rs +++ b/tests-integration/tests/process_stdio.rs @@ -47,7 +47,7 @@ async fn feed_cat(mut cat: Child, n: usize) -> io::Result { // (i.e. EOF is reached after `n` lines. loop { let data = reader - .next() + .next_line() .await .unwrap_or_else(|| Ok(String::new())) .expect("failed to read line"); diff --git a/tokio-test/src/io.rs b/tokio-test/src/io.rs index 1d42dd03f42..0cbc1dfecb4 100644 --- a/tokio-test/src/io.rs +++ b/tokio-test/src/io.rs @@ -298,7 +298,7 @@ impl AsyncRead for Mock { Err(ref e) if e.kind() == io::ErrorKind::WouldBlock => { if let Some(rem) = self.inner.remaining_wait() { let until = Instant::now() + rem; - self.inner.sleep = Some(time::delay(until)); + self.inner.sleep = Some(time::delay_until(until)); } else { self.inner.read_wait = Some(cx.waker().clone()); return Poll::Pending; @@ -340,7 +340,7 @@ impl AsyncWrite for Mock { Err(ref e) if e.kind() == io::ErrorKind::WouldBlock => { if let Some(rem) = self.inner.remaining_wait() { let until = Instant::now() + rem; - self.inner.sleep = Some(time::delay(until)); + self.inner.sleep = Some(time::delay_until(until)); } else { panic!("unexpected WouldBlock"); } diff --git a/tokio-test/tests/block_on.rs b/tokio-test/tests/block_on.rs index 3c0fe32f967..b50f9ae1486 100644 --- a/tokio-test/tests/block_on.rs +++ b/tokio-test/tests/block_on.rs @@ -1,6 +1,6 @@ #![warn(rust_2018_idioms)] -use tokio::time::{delay, Duration, Instant}; +use tokio::time::{delay_until, Duration, Instant}; use tokio_test::block_on; #[test] @@ -20,5 +20,5 @@ fn async_fn() { #[test] fn test_delay() { let deadline = Instant::now() + Duration::from_millis(100); - assert_eq!((), block_on(delay(deadline))); + assert_eq!((), block_on(delay_until(deadline))); } diff --git a/tokio-util/Cargo.toml b/tokio-util/Cargo.toml index 034dcd02ad8..91edb5501d9 100644 --- a/tokio-util/Cargo.toml +++ b/tokio-util/Cargo.toml @@ -31,7 +31,7 @@ log = "0.4" tokio = { version = "=0.2.0-alpha.6", path = "../tokio" } tokio-test = { version = "=0.2.0-alpha.6", path = "../tokio-test" } -futures-util = "0.3.0" +futures = "0.3.0" [package.metadata.docs.rs] all-features = true diff --git a/tokio-util/src/fs/mod.rs b/tokio-util/src/fs/mod.rs new file mode 100644 index 00000000000..8cc0135305e --- /dev/null +++ b/tokio-util/src/fs/mod.rs @@ -0,0 +1 @@ +mod read_dir; diff --git a/tokio-util/src/fs/read_dir.rs b/tokio-util/src/fs/read_dir.rs new file mode 100644 index 00000000000..4e5b38711a8 --- /dev/null +++ b/tokio-util/src/fs/read_dir.rs @@ -0,0 +1,19 @@ +use crate::stream::Stream; +use tokio::fs::{DirEntry, ReadDir}; + +use futures_core::ready; +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for ReadDir { + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(match ready!(self.poll_next_entry(cx)) { + Ok(Some(entry)) => Some(Ok(entry)), + Ok(None) => None, + Err(err) => Some(Err(err)), + }) + } +} diff --git a/tokio-util/src/io/lines.rs b/tokio-util/src/io/lines.rs new file mode 100644 index 00000000000..761ce119812 --- /dev/null +++ b/tokio-util/src/io/lines.rs @@ -0,0 +1,22 @@ +use crate::stream::Stream; +use tokio::io::{AsyncBufRead, Lines}; + +use futures_core::ready; +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for Lines +where + R: AsyncBufRead + Unpin, +{ + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match ready!(self.poll_next_line(cx))? { + Some(line) => Poll::Ready(Some(Ok(line))), + None => Poll::Ready(None), + } + } +} + diff --git a/tokio-util/src/io/mod.rs b/tokio-util/src/io/mod.rs new file mode 100644 index 00000000000..4cf30e13101 --- /dev/null +++ b/tokio-util/src/io/mod.rs @@ -0,0 +1 @@ +mod split; diff --git a/tokio-util/src/io/split.rs b/tokio-util/src/io/split.rs new file mode 100644 index 00000000000..3912dc3f072 --- /dev/null +++ b/tokio-util/src/io/split.rs @@ -0,0 +1,21 @@ +use crate::stream::Stream; +use tokio::io::{AsyncBufRead, Split}; + +use futures_core::ready; +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for Split +where + R: AsyncBufRead + Unpin, +{ + type Item = io::Result>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match ready!(self.poll_next_segment(cx))? { + Some(segment) => Poll::Ready(Some(Ok(segment))), + None => Poll::Ready(None), + } + } +} diff --git a/tokio-util/src/lib.rs b/tokio-util/src/lib.rs index 5a64673c8d1..783494bb46f 100644 --- a/tokio-util/src/lib.rs +++ b/tokio-util/src/lib.rs @@ -14,4 +14,9 @@ //! Utilities for working with Tokio. pub mod codec; +mod fs; +mod io; +pub mod stream; +mod sync; +mod time; pub mod udp; diff --git a/tokio-util/src/stream/as_std.rs b/tokio-util/src/stream/as_std.rs new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tokio-util/src/stream/into_std.rs b/tokio-util/src/stream/into_std.rs new file mode 100644 index 00000000000..9c3b3aa1c9b --- /dev/null +++ b/tokio-util/src/stream/into_std.rs @@ -0,0 +1,19 @@ +use crate::stream::Stream; + +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Stream for the [`into_std`](super::Stream::into_std) method. +#[derive(Debug)] +pub struct IntoStd { + pub(super) stream: T, +} + +impl futures_core::Stream for IntoStd { + type Item = T::Item; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let stream = unsafe { self.map_unchecked_mut(|me| &mut me.stream) }; + stream.poll_next(cx) + } +} diff --git a/tokio-util/src/stream/mod.rs b/tokio-util/src/stream/mod.rs new file mode 100644 index 00000000000..69407614fd4 --- /dev/null +++ b/tokio-util/src/stream/mod.rs @@ -0,0 +1,117 @@ +//! Asynchronous value iteration. + +mod into_std; +pub use into_std::IntoStd; + +use std::ops::DerefMut; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// A stream of values produced asynchronously. +/// +/// This trait is used to convert Tokio's "stream-like" types into +/// `futures::Stream`. This trait is not intended to be implemented by third +/// parties (use `futures::Stream` instead) and only exists to satisfy Rust's +/// coherence requirements. When `Stream` is stabilized in `std`, this trait +/// will be removed and `tokio` will directly implement the `std` trait. +#[must_use = "streams do nothing unless polled"] +pub trait Stream { + /// Values yielded by the stream. + type Item; + + /// Attempt to pull out the next value of this stream, registering the + /// current task for wakeup if the value is not yet available, and returning + /// `None` if the stream is exhausted. + /// + /// # Return value + /// + /// There are several possible return values, each indicating a distinct + /// stream state: + /// + /// - `Poll::Pending` means that this stream's next value is not ready + /// yet. Implementations will ensure that the current task will be notified + /// when the next value may be ready. + /// + /// - `Poll::Ready(Some(val))` means that the stream has successfully + /// produced a value, `val`, and may produce further values on subsequent + /// `poll_next` calls. + /// + /// - `Poll::Ready(None)` means that the stream has terminated, and + /// `poll_next` should not be invoked again. + /// + /// # Panics + /// + /// Once a stream is finished, i.e. `Ready(None)` has been returned, further + /// calls to `poll_next` may result in a panic or other "bad behavior". If + /// this is difficult to guard against then the `fuse` adapter can be used + /// to ensure that `poll_next` always returns `Ready(None)` in subsequent + /// calls. + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; + + /// Returns the bounds on the remaining length of the stream. + /// + /// Specifically, `size_hint()` returns a tuple where the first element + /// is the lower bound, and the second element is the upper bound. + /// + /// The second half of the tuple that is returned is an [`Option`]`<`[`usize`]`>`. + /// A [`None`] here means that either there is no known upper bound, or the + /// upper bound is larger than [`usize`]. + /// + /// # Implementation notes + /// + /// It is not enforced that a stream implementation yields the declared + /// number of elements. A buggy stream may yield less than the lower bound + /// or more than the upper bound of elements. + /// + /// `size_hint()` is primarily intended to be used for optimizations such as + /// reserving space for the elements of the stream, but must not be + /// trusted to e.g., omit bounds checks in unsafe code. An incorrect + /// implementation of `size_hint()` should not lead to memory safety + /// violations. + /// + /// That said, the implementation should provide a correct estimation, + /// because otherwise it would be a violation of the trait's protocol. + /// + /// The default implementation returns `(0, `[`None`]`)` which is correct for any + /// stream. + #[inline] + fn size_hint(&self) -> (usize, Option) { + (0, None) + } + + /// Convert the stream into a `futures::Stream` type. + fn into_std(self) -> IntoStd + where + Self: Sized, + { + IntoStd { stream: self } + } +} + +impl Stream for &mut S { + type Item = S::Item; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + S::poll_next(Pin::new(&mut **self), cx) + } + + fn size_hint(&self) -> (usize, Option) { + (**self).size_hint() + } +} + +impl

Stream for Pin

+where + P: DerefMut + Unpin, + P::Target: Stream, +{ + type Item = ::Item; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.get_mut().as_mut().poll_next(cx) + } + + fn size_hint(&self) -> (usize, Option) { + (**self).size_hint() + } +} diff --git a/tokio-util/src/sync/mod.rs b/tokio-util/src/sync/mod.rs new file mode 100644 index 00000000000..63a5c702901 --- /dev/null +++ b/tokio-util/src/sync/mod.rs @@ -0,0 +1,2 @@ +mod mpsc; +mod watch; diff --git a/tokio-util/src/sync/mpsc.rs b/tokio-util/src/sync/mpsc.rs new file mode 100644 index 00000000000..ea9e89d1215 --- /dev/null +++ b/tokio-util/src/sync/mpsc.rs @@ -0,0 +1,21 @@ +use crate::stream::Stream; +use tokio::sync::mpsc::{Receiver, UnboundedReceiver}; + +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for Receiver { + type Item = T; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.get_mut().poll_recv(cx) + } +} + +impl Stream for UnboundedReceiver { + type Item = T; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.get_mut().poll_recv(cx) + } +} diff --git a/tokio-util/src/sync/watch.rs b/tokio-util/src/sync/watch.rs new file mode 100644 index 00000000000..027b57dffb3 --- /dev/null +++ b/tokio-util/src/sync/watch.rs @@ -0,0 +1,17 @@ +use crate::stream::Stream; +use tokio::sync::watch::Receiver; + +use futures_core::ready; +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for Receiver { + type Item = T; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let ret = ready!(self.poll_recv_ref(cx)); + + #[allow(clippy::map_clone)] + Poll::Ready(ret.map(|v_ref| v_ref.clone())) + } +} diff --git a/tokio-util/src/time/interval.rs b/tokio-util/src/time/interval.rs new file mode 100644 index 00000000000..bc64b980c3d --- /dev/null +++ b/tokio-util/src/time/interval.rs @@ -0,0 +1,15 @@ +use crate::stream::Stream; +use tokio::time::{Instant, Interval}; + +use futures_core::ready; +use std::pin::Pin; +use std::task::{Context, Poll}; + +impl Stream for Interval { + type Item = Instant; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let instant = ready!(self.get_mut().poll_tick(cx)); + Poll::Ready(Some(instant)) + } +} diff --git a/tokio-util/src/time/mod.rs b/tokio-util/src/time/mod.rs new file mode 100644 index 00000000000..73c3dc6f161 --- /dev/null +++ b/tokio-util/src/time/mod.rs @@ -0,0 +1 @@ +mod interval; diff --git a/tokio-util/tests/sync_watch.rs b/tokio-util/tests/sync_watch.rs new file mode 100644 index 00000000000..93757acce4a --- /dev/null +++ b/tokio-util/tests/sync_watch.rs @@ -0,0 +1,44 @@ +#![warn(rust_2018_idioms)] + +use tokio::sync::watch; +use tokio_test::task::spawn; +use tokio_test::{assert_pending, assert_ready}; +use tokio_util::stream::Stream; + +use futures::prelude::*; + +#[test] +fn stream_impl() { + let (tx, rx) = watch::channel("one"); + let mut rx = rx.into_std(); + + { + let mut t = spawn(rx.next()); + let v = assert_ready!(t.poll()).unwrap(); + assert_eq!(v, "one"); + } + + { + let mut t = spawn(rx.next()); + + assert_pending!(t.poll()); + + tx.broadcast("two").unwrap(); + + assert!(t.is_woken()); + + let v = assert_ready!(t.poll()).unwrap(); + assert_eq!(v, "two"); + } + + { + let mut t = spawn(rx.next()); + + assert_pending!(t.poll()); + + drop(tx); + + let res = assert_ready!(t.poll()); + assert!(res.is_none()); + } +} diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 56ac0dc9703..18a970e4040 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -84,9 +84,6 @@ uds = ["io-driver", "mio-uds", "libc"] tokio-macros = { version = "=0.2.0-alpha.6", optional = true, path = "../tokio-macros" } bytes = "0.4" -futures-core = "0.3.0" -futures-sink = "0.3.0" -futures-util = { version = "0.3.0", features = ["sink", "channel"] } iovec = "0.1" # Everything else is optional... diff --git a/tokio/src/fs/blocking.rs b/tokio/src/fs/blocking.rs index 695358a3900..64398cbbd70 100644 --- a/tokio/src/fs/blocking.rs +++ b/tokio/src/fs/blocking.rs @@ -1,7 +1,6 @@ use crate::fs::sys; use crate::io::{AsyncRead, AsyncWrite}; -use futures_core::ready; use std::cmp; use std::future::Future; use std::io; diff --git a/tokio/src/fs/file.rs b/tokio/src/fs/file.rs index 3f18831e56c..0ff45025b2f 100644 --- a/tokio/src/fs/file.rs +++ b/tokio/src/fs/file.rs @@ -7,7 +7,6 @@ use crate::fs::blocking::Buf; use crate::fs::{asyncify, sys}; use crate::io::{AsyncRead, AsyncWrite}; -use futures_core::ready; use std::fmt; use std::fs::{Metadata, Permissions}; use std::future::Future; @@ -430,7 +429,7 @@ impl File { } async fn complete_inflight(&mut self) { - use futures_util::future::poll_fn; + use crate::future::poll_fn; if let Err(e) = poll_fn(|cx| Pin::new(&mut *self).poll_flush(cx)).await { self.last_write_err = Some(e.kind()); diff --git a/tokio/src/fs/read_dir.rs b/tokio/src/fs/read_dir.rs index 9492a2f4c69..627309eadf5 100644 --- a/tokio/src/fs/read_dir.rs +++ b/tokio/src/fs/read_dir.rs @@ -1,7 +1,5 @@ use crate::fs::{asyncify, sys}; -use futures_core::ready; -use futures_core::stream::Stream; use std::ffi::OsString; use std::fs::{FileType, Metadata}; use std::future::Future; @@ -50,10 +48,15 @@ enum State { Pending(sys::Blocking<(Option>, std::fs::ReadDir)>), } -impl Stream for ReadDir { - type Item = io::Result; +impl ReadDir { + /// TODO: Dox + pub async fn next_entry(&mut self) -> io::Result> { + use crate::future::poll_fn; + poll_fn(|cx| self.poll_next_entry(cx)).await + } - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + #[doc(hidden)] + pub fn poll_next_entry(&mut self, cx: &mut Context<'_>) -> Poll>> { loop { match self.0 { State::Idle(ref mut std) => { @@ -68,7 +71,11 @@ impl Stream for ReadDir { let (ret, std) = ready!(Pin::new(rx).poll(cx))?; self.0 = State::Idle(Some(std)); - let ret = ret.map(|res| res.map(|std| DirEntry(Arc::new(std)))); + let ret = match ret { + Some(Ok(std)) => Ok(Some(DirEntry(Arc::new(std)))), + Some(Err(e)) => Err(e), + None => Ok(None), + }; return Poll::Ready(ret); } @@ -100,13 +107,11 @@ impl DirEntry { /// /// ```no_run /// use tokio::fs; - /// use tokio::prelude::*; /// /// # async fn dox() -> std::io::Result<()> { /// let mut entries = fs::read_dir(".").await?; /// - /// while let Some(res) = entries.next().await { - /// let entry = res?; + /// while let Some(entry) = entries.next_entry().await? { /// println!("{:?}", entry.path()); /// } /// # Ok(()) @@ -133,13 +138,11 @@ impl DirEntry { /// /// ``` /// use tokio::fs; - /// use tokio::prelude::*; /// /// # async fn dox() -> std::io::Result<()> { /// let mut entries = fs::read_dir(".").await?; /// - /// while let Some(res) = entries.next().await { - /// let entry = res?; + /// while let Some(entry) = entries.next_entry().await? { /// println!("{:?}", entry.file_name()); /// } /// # Ok(()) @@ -164,14 +167,11 @@ impl DirEntry { /// /// ``` /// use tokio::fs; - /// use tokio::prelude::*; /// /// # async fn dox() -> std::io::Result<()> { /// let mut entries = fs::read_dir(".").await?; /// - /// while let Some(res) = entries.next().await { - /// let entry = res?; - /// + /// while let Some(entry) = entries.next_entry().await? { /// if let Ok(metadata) = entry.metadata().await { /// // Now let's show our entry's permissions! /// println!("{:?}: {:?}", entry.path(), metadata.permissions()); @@ -202,14 +202,11 @@ impl DirEntry { /// /// ``` /// use tokio::fs; - /// use tokio::prelude::*; /// /// # async fn dox() -> std::io::Result<()> { /// let mut entries = fs::read_dir(".").await?; /// - /// while let Some(res) = entries.next().await { - /// let entry = res?; - /// + /// while let Some(entry) = entries.next_entry().await? { /// if let Ok(file_type) = entry.file_type().await { /// // Now let's show our entry's file type! /// println!("{:?}: {:?}", entry.path(), file_type); diff --git a/tokio/src/future.rs b/tokio/src/future.rs deleted file mode 100644 index f6b7e4a71e3..00000000000 --- a/tokio/src/future.rs +++ /dev/null @@ -1,69 +0,0 @@ -//! Asynchronous values. - -#[cfg(feature = "time")] -use crate::time::Timeout; - -#[cfg(feature = "time")] -use std::time::Duration; - -#[doc(inline)] -pub use futures_util::future::{err, ok, pending, poll_fn, ready}; -#[doc(inline)] -pub use std::future::Future; - -/// An extension trait for `Future` that provides a variety of convenient -/// combinator functions. -/// -/// Currently, there only is a [`timeout`] function, but this will increase -/// over time. -/// -/// Users are not expected to implement this trait. All types that implement -/// `Future` already implement `FutureExt`. -/// -/// This trait can be imported directly or via the Tokio prelude: `use -/// tokio::prelude::*`. -/// -/// [`timeout`]: #method.timeout -pub trait FutureExt: Future { - /// Creates a new future which allows `self` until `timeout`. - /// - /// This combinator creates a new future which wraps the receiving future - /// with a timeout. The returned future is allowed to execute until it - /// completes or `timeout` has elapsed, whichever happens first. - /// - /// If the future completes before `timeout` then the future will resolve - /// with that item. Otherwise the future will resolve to an error. - /// - /// The future is guaranteed to be polled at least once, even if `timeout` - /// is set to zero. - /// - /// # Examples - /// - /// ``` - /// use tokio::prelude::*; - /// use std::time::Duration; - /// - /// async fn long_future() { - /// // do work here - /// } - /// - /// # async fn dox() { - /// let res = long_future() - /// .timeout(Duration::from_secs(1)) - /// .await; - /// - /// if res.is_err() { - /// println!("operation timed out"); - /// } - /// # } - /// ``` - #[cfg(feature = "time")] - fn timeout(self, timeout: Duration) -> Timeout - where - Self: Sized, - { - Timeout::new(self, timeout) - } -} - -impl FutureExt for T where T: Future {} diff --git a/tokio/src/future/maybe_done.rs b/tokio/src/future/maybe_done.rs new file mode 100644 index 00000000000..5011544c11a --- /dev/null +++ b/tokio/src/future/maybe_done.rs @@ -0,0 +1,76 @@ +//! Definition of the MaybeDone combinator + +use std::future::Future; +use std::mem; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// A future that may have completed. +#[derive(Debug)] +pub(crate) enum MaybeDone { + /// A not-yet-completed future + Future(Fut), + /// The output of the completed future + Done(Fut::Output), + /// The empty variant after the result of a [`MaybeDone`] has been + /// taken using the [`take_output`](MaybeDone::take_output) method. + Gone, +} + +// Safe because we never generate `Pin<&mut Fut::Output>` +impl Unpin for MaybeDone {} + +/// Wraps a future into a `MaybeDone` +pub(crate) fn maybe_done(future: Fut) -> MaybeDone { + MaybeDone::Future(future) +} + +impl MaybeDone { + /// Returns an [`Option`] containing a mutable reference to the output of the future. + /// The output of this method will be [`Some`] if and only if the inner + /// future has been completed and [`take_output`](MaybeDone::take_output) + /// has not yet been called. + pub(crate) fn output_mut(self: Pin<&mut Self>) -> Option<&mut Fut::Output> { + unsafe { + let this = self.get_unchecked_mut(); + match this { + MaybeDone::Done(res) => Some(res), + _ => None, + } + } + } + + /// Attempt to take the output of a `MaybeDone` without driving it + /// towards completion. + #[inline] + pub(crate) fn take_output(self: Pin<&mut Self>) -> Option { + unsafe { + let this = self.get_unchecked_mut(); + match this { + MaybeDone::Done(_) => {} + MaybeDone::Future(_) | MaybeDone::Gone => return None, + }; + if let MaybeDone::Done(output) = mem::replace(this, MaybeDone::Gone) { + Some(output) + } else { + unreachable!() + } + } + } +} + +impl Future for MaybeDone { + type Output = (); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let res = unsafe { + match self.as_mut().get_unchecked_mut() { + MaybeDone::Future(a) => ready!(Pin::new_unchecked(a).poll(cx)), + MaybeDone::Done(_) => return Poll::Ready(()), + MaybeDone::Gone => panic!("MaybeDone polled after value taken"), + } + }; + self.set(MaybeDone::Done(res)); + Poll::Ready(()) + } +} diff --git a/tokio/src/future/mod.rs b/tokio/src/future/mod.rs new file mode 100644 index 00000000000..9a155bf7f79 --- /dev/null +++ b/tokio/src/future/mod.rs @@ -0,0 +1,15 @@ +#![allow(unused_imports, dead_code)] + +//! Asynchronous values. + +mod maybe_done; +pub(crate) use maybe_done::{maybe_done, MaybeDone}; + +mod poll_fn; +pub(crate) use poll_fn::poll_fn; + +mod ready; +pub(crate) use ready::{ok, Ready}; + +mod try_join; +pub(crate) use try_join::try_join3; diff --git a/tokio/src/future/pending.rs b/tokio/src/future/pending.rs new file mode 100644 index 00000000000..c844ebc38ec --- /dev/null +++ b/tokio/src/future/pending.rs @@ -0,0 +1,44 @@ +use std::future::Future; +use std::marker; +use sdt::pin::Pin; +use std::task::{Context, Poll}; + +/// Future for the [`pending()`] function. +#[derive(Debug)] +#[must_use = "futures do nothing unless you `.await` or poll them"] +struct Pending { + _data: marker::PhantomData, +} + +/// Creates a future which never resolves, representing a computation that never +/// finishes. +/// +/// The returned future will forever return [`Poll::Pending`]. +/// +/// # Examples +/// +/// ```no_run +/// use tokio::future; +/// +/// #[tokio::main] +/// async fn main { +/// future::pending().await; +/// unreachable!(); +/// } +/// ``` +pub async fn pending() -> ! { + Pending { + _data: marker::PhantomData, + }.await +} + +impl Future for Pending { + type Output = !; + + fn poll(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll { + Poll::Pending + } +} + +impl Unpin for Pending { +} diff --git a/tokio/src/future/poll_fn.rs b/tokio/src/future/poll_fn.rs new file mode 100644 index 00000000000..ce2a552495d --- /dev/null +++ b/tokio/src/future/poll_fn.rs @@ -0,0 +1,38 @@ +//! Definition of the `PollFn` adapter combinator + +use std::fmt; +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Future for the [`poll_fn`] function. +pub(crate) struct PollFn { + f: F, +} + +impl Unpin for PollFn {} + +/// Creates a new future wrapping around a function returning [`Poll`]. +pub(crate) fn poll_fn(f: F) -> PollFn +where + F: FnMut(&mut Context<'_>) -> Poll, +{ + PollFn { f } +} + +impl fmt::Debug for PollFn { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("PollFn").finish() + } +} + +impl Future for PollFn +where + F: FnMut(&mut Context<'_>) -> Poll, +{ + type Output = T; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + (&mut self.f)(cx) + } +} diff --git a/tokio/src/future/ready.rs b/tokio/src/future/ready.rs new file mode 100644 index 00000000000..ba5d4804493 --- /dev/null +++ b/tokio/src/future/ready.rs @@ -0,0 +1,27 @@ +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Future for the [`ready`](ready()) function. +/// +/// `pub` in order to use the future as an associated type in a sealed trait. +#[derive(Debug)] +// Used as an associated type in a "sealed" trait. +#[allow(unreachable_pub)] +pub struct Ready(Option); + +impl Unpin for Ready {} + +impl Future for Ready { + type Output = T; + + #[inline] + fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { + Poll::Ready(self.0.take().unwrap()) + } +} + +/// Create a future that is immediately ready with a success value. +pub(crate) fn ok(t: T) -> Ready> { + Ready(Some(Ok(t))) +} diff --git a/tokio/src/future/try_join.rs b/tokio/src/future/try_join.rs new file mode 100644 index 00000000000..478c69dc457 --- /dev/null +++ b/tokio/src/future/try_join.rs @@ -0,0 +1,115 @@ +use crate::future::{maybe_done, MaybeDone}; + +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +pub(crate) fn try_join3( + future1: F1, + future2: F2, + future3: F3, +) -> TryJoin3 +where + F1: Future>, + F2: Future>, + F3: Future>, +{ + TryJoin3 { + future1: maybe_done(future1), + future2: maybe_done(future2), + future3: maybe_done(future3), + } +} + +pub(crate) struct TryJoin3 +where + F1: Future, + F2: Future, + F3: Future, +{ + future1: MaybeDone, + future2: MaybeDone, + future3: MaybeDone, +} + +impl Future for TryJoin3 +where + F1: Future>, + F2: Future>, + F3: Future>, +{ + type Output = Result<(T1, T2, T3), E>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let mut all_done = true; + + // Safety: the fn takes `Pin`, we don't move any data out of `self`. + unsafe { + let me = self.get_unchecked_mut(); + + if Pin::new_unchecked(&mut me.future1).poll(cx).is_pending() { + all_done = false; + } else if Pin::new_unchecked(&mut me.future1) + .output_mut() + .unwrap() + .is_err() + { + return Poll::Ready(Err(Pin::new_unchecked(&mut me.future1) + .take_output() + .unwrap() + .err() + .unwrap())); + } + + if Pin::new_unchecked(&mut me.future2).poll(cx).is_pending() { + all_done = false; + } else if Pin::new_unchecked(&mut me.future2) + .output_mut() + .unwrap() + .is_err() + { + return Poll::Ready(Err(Pin::new_unchecked(&mut me.future2) + .take_output() + .unwrap() + .err() + .unwrap())); + } + + if Pin::new_unchecked(&mut me.future3).poll(cx).is_pending() { + all_done = false; + } else if Pin::new_unchecked(&mut me.future3) + .output_mut() + .unwrap() + .is_err() + { + return Poll::Ready(Err(Pin::new_unchecked(&mut me.future3) + .take_output() + .unwrap() + .err() + .unwrap())); + } + + if all_done { + Poll::Ready(Ok(( + Pin::new_unchecked(&mut me.future1) + .take_output() + .unwrap() + .ok() + .unwrap(), + Pin::new_unchecked(&mut me.future2) + .take_output() + .unwrap() + .ok() + .unwrap(), + Pin::new_unchecked(&mut me.future3) + .take_output() + .unwrap() + .ok() + .unwrap(), + ))) + } else { + Poll::Pending + } + } + } +} diff --git a/tokio/src/io/async_read.rs b/tokio/src/io/async_read.rs index 8f6e0b98453..974cf346320 100644 --- a/tokio/src/io/async_read.rs +++ b/tokio/src/io/async_read.rs @@ -1,5 +1,4 @@ use bytes::BufMut; -use futures_core::ready; use std::io; use std::ops::DerefMut; use std::pin::Pin; diff --git a/tokio/src/io/async_write.rs b/tokio/src/io/async_write.rs index 0489fb7ce5a..2d5e45786be 100644 --- a/tokio/src/io/async_write.rs +++ b/tokio/src/io/async_write.rs @@ -1,5 +1,4 @@ use bytes::Buf; -use futures_core::ready; use std::io; use std::ops::DerefMut; use std::pin::Pin; diff --git a/tokio/src/io/io/lines.rs b/tokio/src/io/io/lines.rs deleted file mode 100644 index fe95e2a0fa6..00000000000 --- a/tokio/src/io/io/lines.rs +++ /dev/null @@ -1,70 +0,0 @@ -use crate::io::io::read_line::read_line_internal; -use crate::io::AsyncBufRead; - -use futures_core::{ready, Stream}; -use pin_project::{pin_project, project}; -use std::io; -use std::mem; -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Stream for the [`lines`](crate::io::AsyncBufReadExt::lines) method. -#[pin_project] -#[derive(Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct Lines { - #[pin] - reader: R, - buf: String, - bytes: Vec, - read: usize, -} - -pub(crate) fn lines(reader: R) -> Lines -where - R: AsyncBufRead, -{ - Lines { - reader, - buf: String::new(), - bytes: Vec::new(), - read: 0, - } -} - -impl Stream for Lines { - type Item = io::Result; - - #[project] - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let Lines { - reader, - buf, - bytes, - read, - } = self.project(); - - let n = ready!(read_line_internal(reader, cx, buf, bytes, read))?; - if n == 0 && buf.is_empty() { - return Poll::Ready(None); - } - if buf.ends_with('\n') { - buf.pop(); - if buf.ends_with('\r') { - buf.pop(); - } - } - Poll::Ready(Some(Ok(mem::replace(buf, String::new())))) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn assert_unpin() { - crate::is_unpin::>(); - } -} diff --git a/tokio/src/io/io/split.rs b/tokio/src/io/io/split.rs deleted file mode 100644 index c3d7430eb22..00000000000 --- a/tokio/src/io/io/split.rs +++ /dev/null @@ -1,67 +0,0 @@ -use crate::io::io::read_until::read_until_internal; -use crate::io::AsyncBufRead; - -use futures_core::{ready, Stream}; -use pin_project::{pin_project, project}; -use std::io; -use std::mem; -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Stream for the [`split`](crate::io::AsyncBufReadExt::split) method. -#[pin_project] -#[derive(Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct Split { - #[pin] - reader: R, - buf: Vec, - delim: u8, - read: usize, -} - -pub(crate) fn split(reader: R, delim: u8) -> Split -where - R: AsyncBufRead, -{ - Split { - reader, - buf: Vec::new(), - delim, - read: 0, - } -} - -impl Stream for Split { - type Item = io::Result>; - - #[project] - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let Split { - reader, - buf, - delim, - read, - } = self.project(); - - let n = ready!(read_until_internal(reader, cx, *delim, buf, read))?; - if n == 0 && buf.is_empty() { - return Poll::Ready(None); - } - if buf.last() == Some(&delim) { - buf.pop(); - } - Poll::Ready(Some(Ok(mem::replace(buf, Vec::new())))) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn assert_unpin() { - crate::is_unpin::>(); - } -} diff --git a/tokio/src/io/mod.rs b/tokio/src/io/mod.rs index 9e0567de84b..3f95c330cb5 100644 --- a/tokio/src/io/mod.rs +++ b/tokio/src/io/mod.rs @@ -45,19 +45,18 @@ pub use self::async_read::AsyncRead; mod async_write; pub use self::async_write::AsyncWrite; -#[allow(clippy::module_inception)] // TODO: remove #[cfg(feature = "io-util")] -mod io; +pub mod split; #[cfg(feature = "io-util")] -pub use self::io::{ - copy, empty, repeat, sink, AsyncBufReadExt, AsyncReadExt, AsyncWriteExt, BufReader, BufStream, - BufWriter, Copy, Empty, Repeat, Sink, Take, -}; +pub use self::split::split; #[cfg(feature = "io-util")] -pub mod split; +mod util; #[cfg(feature = "io-util")] -pub use self::split::split; +pub use self::util::{ + copy, empty, repeat, sink, AsyncBufReadExt, AsyncReadExt, AsyncWriteExt, BufReader, BufStream, + BufWriter, Copy, Empty, Lines, Repeat, Sink, Split, Take, +}; // TODO: These should not be guarded by `fs` diff --git a/tokio/src/io/split.rs b/tokio/src/io/split.rs index 28c7d5d4d2a..388ee4ed2ef 100644 --- a/tokio/src/io/split.rs +++ b/tokio/src/io/split.rs @@ -7,7 +7,6 @@ use crate::io::{AsyncRead, AsyncWrite}; use bytes::{Buf, BufMut}; -use futures_core::ready; use std::cell::UnsafeCell; use std::fmt; use std::io; diff --git a/tokio/src/io/io/async_buf_read_ext.rs b/tokio/src/io/util/async_buf_read_ext.rs similarity index 83% rename from tokio/src/io/io/async_buf_read_ext.rs rename to tokio/src/io/util/async_buf_read_ext.rs index b60f55d54e5..3b4677b41c7 100644 --- a/tokio/src/io/io/async_buf_read_ext.rs +++ b/tokio/src/io/util/async_buf_read_ext.rs @@ -1,7 +1,7 @@ -use crate::io::io::lines::{lines, Lines}; -use crate::io::io::read_line::{read_line, ReadLine}; -use crate::io::io::read_until::{read_until, ReadUntil}; -use crate::io::io::split::{split, Split}; +use crate::io::util::lines::{lines, Lines}; +use crate::io::util::read_line::{read_line, ReadLine}; +use crate::io::util::read_until::{read_until, ReadUntil}; +use crate::io::util::split::{split, Split}; use crate::io::AsyncBufRead; /// An extension trait which adds utility methods to `AsyncBufRead` types. @@ -59,7 +59,7 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// Returns a stream of the contents of this reader split on the byte /// `byte`. /// - /// This method is the async equivalent to + /// This method is the asynchronous equivalent to /// [`BufRead::split`](std::io::BufRead::split). /// /// The stream returned from this function will yield instances of @@ -73,9 +73,25 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// /// Each item of the stream has the same error semantics as /// [`AsyncBufReadExt::read_until`](AsyncBufReadExt::read_until). + /// + /// # Examples + /// + /// ``` + /// # use tokio::io::AsyncBufRead; + /// use tokio::io::AsyncBufReadExt; + /// + /// # async fn dox(my_buf_read: impl AsyncBufRead + Unpin) -> std::io::Result<()> { + /// let mut segments = my_buf_read.split(b'f'); + /// + /// while let Some(segment) = segments.next_segment().await? { + /// println!("length = {}", segment.len()) + /// } + /// # Ok(()) + /// # } + /// ``` fn split(self, byte: u8) -> Split where - Self: Sized, + Self: Sized + Unpin, { split(self, byte) } @@ -97,7 +113,7 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// [`AsyncBufReadExt::read_line`]: AsyncBufReadExt::read_line fn lines(self) -> Lines where - Self: Sized, + Self: Sized + Unpin, { lines(self) } diff --git a/tokio/src/io/io/async_read_ext.rs b/tokio/src/io/util/async_read_ext.rs similarity index 88% rename from tokio/src/io/io/async_read_ext.rs rename to tokio/src/io/util/async_read_ext.rs index c100a71fea4..0b7cdbf1394 100644 --- a/tokio/src/io/io/async_read_ext.rs +++ b/tokio/src/io/util/async_read_ext.rs @@ -1,10 +1,10 @@ -use crate::io::io::chain::{chain, Chain}; -use crate::io::io::copy::{copy, Copy}; -use crate::io::io::read::{read, Read}; -use crate::io::io::read_exact::{read_exact, ReadExact}; -use crate::io::io::read_to_end::{read_to_end, ReadToEnd}; -use crate::io::io::read_to_string::{read_to_string, ReadToString}; -use crate::io::io::take::{take, Take}; +use crate::io::util::chain::{chain, Chain}; +use crate::io::util::copy::{copy, Copy}; +use crate::io::util::read::{read, Read}; +use crate::io::util::read_exact::{read_exact, ReadExact}; +use crate::io::util::read_to_end::{read_to_end, ReadToEnd}; +use crate::io::util::read_to_string::{read_to_string, ReadToString}; +use crate::io::util::take::{take, Take}; use crate::io::{AsyncRead, AsyncWrite}; /// An extension trait which adds utility methods to `AsyncRead` types. diff --git a/tokio/src/io/io/async_write_ext.rs b/tokio/src/io/util/async_write_ext.rs similarity index 82% rename from tokio/src/io/io/async_write_ext.rs rename to tokio/src/io/util/async_write_ext.rs index ea6650d123f..9e40e2594aa 100644 --- a/tokio/src/io/io/async_write_ext.rs +++ b/tokio/src/io/util/async_write_ext.rs @@ -1,7 +1,7 @@ -use crate::io::io::flush::{flush, Flush}; -use crate::io::io::shutdown::{shutdown, Shutdown}; -use crate::io::io::write::{write, Write}; -use crate::io::io::write_all::{write_all, WriteAll}; +use crate::io::util::flush::{flush, Flush}; +use crate::io::util::shutdown::{shutdown, Shutdown}; +use crate::io::util::write::{write, Write}; +use crate::io::util::write_all::{write_all, WriteAll}; use crate::io::AsyncWrite; /// An extension trait which adds utility methods to `AsyncWrite` types. diff --git a/tokio/src/io/io/buf_reader.rs b/tokio/src/io/util/buf_reader.rs similarity index 98% rename from tokio/src/io/io/buf_reader.rs rename to tokio/src/io/util/buf_reader.rs index c9698e1672e..5ed10d22b5a 100644 --- a/tokio/src/io/io/buf_reader.rs +++ b/tokio/src/io/util/buf_reader.rs @@ -1,7 +1,6 @@ -use crate::io::io::DEFAULT_BUF_SIZE; +use crate::io::util::DEFAULT_BUF_SIZE; use crate::io::{AsyncBufRead, AsyncRead, AsyncWrite}; -use futures_core::ready; use pin_project::{pin_project, project}; use std::io::{self, Read}; use std::pin::Pin; diff --git a/tokio/src/io/io/buf_stream.rs b/tokio/src/io/util/buf_stream.rs similarity index 98% rename from tokio/src/io/io/buf_stream.rs rename to tokio/src/io/util/buf_stream.rs index 51b6bbe512e..472b7c658a2 100644 --- a/tokio/src/io/io/buf_stream.rs +++ b/tokio/src/io/util/buf_stream.rs @@ -1,4 +1,4 @@ -use crate::io::io::{BufReader, BufWriter}; +use crate::io::util::{BufReader, BufWriter}; use crate::io::{AsyncBufRead, AsyncRead, AsyncWrite}; use pin_project::pin_project; diff --git a/tokio/src/io/io/buf_writer.rs b/tokio/src/io/util/buf_writer.rs similarity index 98% rename from tokio/src/io/io/buf_writer.rs rename to tokio/src/io/util/buf_writer.rs index 7415c8788d6..e17281ac770 100644 --- a/tokio/src/io/io/buf_writer.rs +++ b/tokio/src/io/util/buf_writer.rs @@ -1,7 +1,6 @@ -use crate::io::io::DEFAULT_BUF_SIZE; +use crate::io::util::DEFAULT_BUF_SIZE; use crate::io::{AsyncBufRead, AsyncRead, AsyncWrite}; -use futures_core::ready; use pin_project::{pin_project, project}; use std::fmt; use std::io::{self, Write}; diff --git a/tokio/src/io/io/chain.rs b/tokio/src/io/util/chain.rs similarity index 99% rename from tokio/src/io/io/chain.rs rename to tokio/src/io/util/chain.rs index 3f997ff55aa..761440693e3 100644 --- a/tokio/src/io/io/chain.rs +++ b/tokio/src/io/util/chain.rs @@ -1,6 +1,5 @@ use crate::io::{AsyncBufRead, AsyncRead}; -use futures_core::ready; use pin_project::{pin_project, project}; use std::fmt; use std::io; diff --git a/tokio/src/io/io/copy.rs b/tokio/src/io/util/copy.rs similarity index 99% rename from tokio/src/io/io/copy.rs rename to tokio/src/io/util/copy.rs index 88c8763080d..65dfa9587b7 100644 --- a/tokio/src/io/io/copy.rs +++ b/tokio/src/io/util/copy.rs @@ -1,6 +1,5 @@ use crate::io::{AsyncRead, AsyncWrite}; -use futures_core::ready; use std::future::Future; use std::io; use std::pin::Pin; diff --git a/tokio/src/io/io/empty.rs b/tokio/src/io/util/empty.rs similarity index 100% rename from tokio/src/io/io/empty.rs rename to tokio/src/io/util/empty.rs diff --git a/tokio/src/io/io/flush.rs b/tokio/src/io/util/flush.rs similarity index 100% rename from tokio/src/io/io/flush.rs rename to tokio/src/io/util/flush.rs diff --git a/tokio/src/io/util/lines.rs b/tokio/src/io/util/lines.rs new file mode 100644 index 00000000000..29011aaf09c --- /dev/null +++ b/tokio/src/io/util/lines.rs @@ -0,0 +1,92 @@ +use crate::io::util::read_line::read_line_internal; +use crate::io::AsyncBufRead; + +use std::io; +use std::mem; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Stream for the [`lines`](crate::io::AsyncBufReadExt::lines) method. +#[derive(Debug)] +#[must_use = "streams do nothing unless polled"] +pub struct Lines { + reader: R, + buf: String, + bytes: Vec, + read: usize, +} + +pub(crate) fn lines(reader: R) -> Lines +where + R: AsyncBufRead, +{ + Lines { + reader, + buf: String::new(), + bytes: Vec::new(), + read: 0, + } +} + +impl Lines +where + R: AsyncBufRead + Unpin, +{ + /// Returns the next line in the stream. + /// + /// # Examples + /// + /// ``` + /// # use tokio::io::AsyncBufRead; + /// use tokio::io::AsyncBufReadExt; + /// + /// # async fn dox(my_buf_read: impl AsyncBufRead + Unpin) -> std::io::Result<()> { + /// let mut lines = my_buf_read.lines(); + /// + /// while let Some(line) = lines.next_line().await? { + /// println!("length = {}", line.len()) + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn next_line(&mut self) -> io::Result> { + use crate::future::poll_fn; + + poll_fn(|cx| self.poll_next_line(cx)).await + } + + #[doc(hidden)] + pub fn poll_next_line(&mut self, cx: &mut Context<'_>) -> Poll>> { + let n = ready!(read_line_internal( + Pin::new(&mut self.reader), + cx, + &mut self.buf, + &mut self.bytes, + &mut self.read + ))?; + + if n == 0 && self.buf.is_empty() { + return Poll::Ready(Ok(None)); + } + + if self.buf.ends_with('\n') { + self.buf.pop(); + + if self.buf.ends_with('\r') { + self.buf.pop(); + } + } + + Poll::Ready(Ok(Some(mem::replace(&mut self.buf, String::new())))) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn assert_unpin() { + crate::is_unpin::>(); + } +} diff --git a/tokio/src/io/io/mod.rs b/tokio/src/io/util/mod.rs similarity index 88% rename from tokio/src/io/io/mod.rs rename to tokio/src/io/util/mod.rs index 07ad0e7bcc4..8d4e7678f4c 100644 --- a/tokio/src/io/io/mod.rs +++ b/tokio/src/io/util/mod.rs @@ -1,51 +1,71 @@ mod async_buf_read_ext; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::async_buf_read_ext::AsyncBufReadExt; + mod async_read_ext; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::async_read_ext::AsyncReadExt; + mod async_write_ext; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::async_write_ext::AsyncWriteExt; + mod buf_reader; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::buf_reader::BufReader; + mod buf_stream; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::buf_stream::BufStream; + mod buf_writer; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::buf_writer::BufWriter; + mod chain; + mod copy; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::copy::{copy, Copy}; + mod empty; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::empty::{empty, Empty}; + mod flush; + mod lines; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::lines::Lines; + mod read; mod read_exact; mod read_line; mod read_to_end; mod read_to_string; mod read_until; -mod repeat; -mod shutdown; -mod sink; -mod split; -mod take; -mod write; -mod write_all; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::async_buf_read_ext::AsyncBufReadExt; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::async_read_ext::AsyncReadExt; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::async_write_ext::AsyncWriteExt; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::buf_reader::BufReader; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::buf_stream::BufStream; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::buf_writer::BufWriter; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::copy::{copy, Copy}; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::empty::{empty, Empty}; +mod repeat; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::repeat::{repeat, Repeat}; + +mod shutdown; + +mod sink; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::sink::{sink, Sink}; + +mod split; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::split::Split; + +mod take; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::take::Take; +mod write; +mod write_all; + // used by `BufReader` and `BufWriter` // https://github.com/rust-lang/rust/blob/master/src/libstd/sys_common/io.rs#L1 const DEFAULT_BUF_SIZE: usize = 8 * 1024; diff --git a/tokio/src/io/io/read.rs b/tokio/src/io/util/read.rs similarity index 100% rename from tokio/src/io/io/read.rs rename to tokio/src/io/util/read.rs diff --git a/tokio/src/io/io/read_exact.rs b/tokio/src/io/util/read_exact.rs similarity index 98% rename from tokio/src/io/io/read_exact.rs rename to tokio/src/io/util/read_exact.rs index fc57594a25a..545b993009b 100644 --- a/tokio/src/io/io/read_exact.rs +++ b/tokio/src/io/util/read_exact.rs @@ -1,6 +1,5 @@ use crate::io::AsyncRead; -use futures_core::ready; use std::future::Future; use std::io; use std::marker::Unpin; diff --git a/tokio/src/io/io/read_line.rs b/tokio/src/io/util/read_line.rs similarity index 96% rename from tokio/src/io/io/read_line.rs rename to tokio/src/io/util/read_line.rs index b5839a247ad..b917bd41fac 100644 --- a/tokio/src/io/io/read_line.rs +++ b/tokio/src/io/util/read_line.rs @@ -1,7 +1,6 @@ -use crate::io::io::read_until::read_until_internal; +use crate::io::util::read_until::read_until_internal; use crate::io::AsyncBufRead; -use futures_core::ready; use std::future::Future; use std::io; use std::mem; diff --git a/tokio/src/io/io/read_to_end.rs b/tokio/src/io/util/read_to_end.rs similarity index 99% rename from tokio/src/io/io/read_to_end.rs rename to tokio/src/io/util/read_to_end.rs index 32b2eff3357..36eba5bbf92 100644 --- a/tokio/src/io/io/read_to_end.rs +++ b/tokio/src/io/util/read_to_end.rs @@ -1,6 +1,5 @@ use crate::io::AsyncRead; -use futures_core::ready; use std::future::Future; use std::io; use std::pin::Pin; diff --git a/tokio/src/io/io/read_to_string.rs b/tokio/src/io/util/read_to_string.rs similarity index 96% rename from tokio/src/io/io/read_to_string.rs rename to tokio/src/io/util/read_to_string.rs index 55b2a9a7b29..0526d5ea1f6 100644 --- a/tokio/src/io/io/read_to_string.rs +++ b/tokio/src/io/util/read_to_string.rs @@ -1,7 +1,6 @@ -use crate::io::io::read_to_end::read_to_end_internal; +use crate::io::util::read_to_end::read_to_end_internal; use crate::io::AsyncRead; -use futures_core::ready; use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; diff --git a/tokio/src/io/io/read_until.rs b/tokio/src/io/util/read_until.rs similarity index 98% rename from tokio/src/io/io/read_until.rs rename to tokio/src/io/util/read_until.rs index 82264c84ab6..bdf1928bda1 100644 --- a/tokio/src/io/io/read_until.rs +++ b/tokio/src/io/util/read_until.rs @@ -1,6 +1,5 @@ use crate::io::AsyncBufRead; -use futures_core::ready; use std::future::Future; use std::io; use std::mem; diff --git a/tokio/src/io/io/repeat.rs b/tokio/src/io/util/repeat.rs similarity index 100% rename from tokio/src/io/io/repeat.rs rename to tokio/src/io/util/repeat.rs diff --git a/tokio/src/io/io/shutdown.rs b/tokio/src/io/util/shutdown.rs similarity index 100% rename from tokio/src/io/io/shutdown.rs rename to tokio/src/io/util/shutdown.rs diff --git a/tokio/src/io/io/sink.rs b/tokio/src/io/util/sink.rs similarity index 100% rename from tokio/src/io/io/sink.rs rename to tokio/src/io/util/sink.rs diff --git a/tokio/src/io/util/split.rs b/tokio/src/io/util/split.rs new file mode 100644 index 00000000000..23cd0bf6666 --- /dev/null +++ b/tokio/src/io/util/split.rs @@ -0,0 +1,88 @@ +use crate::io::util::read_until::read_until_internal; +use crate::io::AsyncBufRead; + +use std::io; +use std::mem; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Stream for the [`split`](crate::io::AsyncBufReadExt::split) method. +#[derive(Debug)] +#[must_use = "streams do nothing unless polled"] +pub struct Split { + reader: R, + buf: Vec, + delim: u8, + read: usize, +} + +pub(crate) fn split(reader: R, delim: u8) -> Split +where + R: AsyncBufRead, +{ + Split { + reader, + buf: Vec::new(), + delim, + read: 0, + } +} + +impl Split +where + R: AsyncBufRead + Unpin, +{ + /// Returns the next segment in the stream. + /// + /// # Examples + /// + /// ``` + /// # use tokio::io::AsyncBufRead; + /// use tokio::io::AsyncBufReadExt; + /// + /// # async fn dox(my_buf_read: impl AsyncBufRead + Unpin) -> std::io::Result<()> { + /// let mut segments = my_buf_read.split(b'f'); + /// + /// while let Some(segment) = segments.next_segment().await? { + /// println!("length = {}", segment.len()) + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn next_segment(&mut self) -> io::Result>> { + use crate::future::poll_fn; + + poll_fn(|cx| self.poll_next_segment(cx)).await + } + + #[doc(hidden)] + pub fn poll_next_segment(&mut self, cx: &mut Context<'_>) -> Poll>>> { + let n = ready!(read_until_internal( + Pin::new(&mut self.reader), + cx, + self.delim, + &mut self.buf, + &mut self.read + ))?; + + if n == 0 && self.buf.is_empty() { + return Poll::Ready(Ok(None)); + } + + if self.buf.last() == Some(&self.delim) { + self.buf.pop(); + } + + Poll::Ready(Ok(Some(mem::replace(&mut self.buf, Vec::new())))) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn assert_unpin() { + crate::is_unpin::>(); + } +} diff --git a/tokio/src/io/io/take.rs b/tokio/src/io/util/take.rs similarity index 99% rename from tokio/src/io/io/take.rs rename to tokio/src/io/util/take.rs index 255f835302f..e96009e2329 100644 --- a/tokio/src/io/io/take.rs +++ b/tokio/src/io/util/take.rs @@ -1,6 +1,5 @@ use crate::io::{AsyncBufRead, AsyncRead}; -use futures_core::ready; use pin_project::{pin_project, project}; use std::pin::Pin; use std::task::{Context, Poll}; diff --git a/tokio/src/io/io/write.rs b/tokio/src/io/util/write.rs similarity index 100% rename from tokio/src/io/io/write.rs rename to tokio/src/io/util/write.rs diff --git a/tokio/src/io/io/write_all.rs b/tokio/src/io/util/write_all.rs similarity index 98% rename from tokio/src/io/io/write_all.rs rename to tokio/src/io/util/write_all.rs index 78da8ecdd3f..d336e65db47 100644 --- a/tokio/src/io/io/write_all.rs +++ b/tokio/src/io/util/write_all.rs @@ -1,6 +1,5 @@ use crate::io::AsyncWrite; -use futures_core::ready; use std::future::Future; use std::io; use std::mem; diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index 3614bb223c0..0a651356e84 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -75,6 +75,15 @@ macro_rules! thread_local { ($($tts:tt)+) => { loom::thread_local!{ $($tts)+ } } } +macro_rules! ready { + ($e:expr $(,)?) => { + match $e { + std::task::Poll::Ready(t) => t, + std::task::Poll::Pending => return std::task::Poll::Pending, + } + }; +} + // At the top due to macros #[cfg(test)] #[macro_use] @@ -86,7 +95,7 @@ pub mod blocking; #[cfg(feature = "fs")] pub mod fs; -pub mod future; +mod future; pub mod io; @@ -107,7 +116,7 @@ pub mod runtime; #[cfg(not(loom))] pub mod signal; -pub mod stream; +mod stream; #[cfg(feature = "sync")] pub mod sync; diff --git a/tokio/src/net/addr.rs b/tokio/src/net/addr.rs index 9b3cc2bcb5c..33d30a22b6b 100644 --- a/tokio/src/net/addr.rs +++ b/tokio/src/net/addr.rs @@ -1,4 +1,5 @@ -use futures_util::future; +use crate::future; + use std::io; use std::net::{IpAddr, SocketAddr}; #[cfg(feature = "dns")] @@ -27,7 +28,7 @@ impl sealed::ToSocketAddrsPriv for SocketAddr { fn to_socket_addrs(&self) -> Self::Future { let iter = Some(*self).into_iter(); - future::ready(Ok(iter)) + future::ok(iter) } } @@ -111,7 +112,7 @@ impl sealed::ToSocketAddrsPriv for (IpAddr, u16) { fn to_socket_addrs(&self) -> Self::Future { let iter = Some(SocketAddr::from(*self)).into_iter(); - future::ready(Ok(iter)) + future::ok(iter) } } @@ -195,8 +196,6 @@ pub(crate) mod sealed { type Output = io::Result; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - use futures_core::ready; - match *self { MaybeReady::Ready(ref mut i) => { let iter = OneOrMore::One(i.take().into_iter()); diff --git a/tokio/src/net/tcp/incoming.rs b/tokio/src/net/tcp/incoming.rs deleted file mode 100644 index 0339615ae2e..00000000000 --- a/tokio/src/net/tcp/incoming.rs +++ /dev/null @@ -1,31 +0,0 @@ -use crate::net::tcp::TcpListener; -use crate::net::tcp::TcpStream; - -use futures_core::ready; -use futures_core::stream::Stream; -use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Stream returned by the `TcpListener::incoming` function representing the -/// stream of sockets received from a listener. -#[must_use = "streams do nothing unless polled"] -#[derive(Debug)] -pub struct Incoming { - inner: TcpListener, -} - -impl Incoming { - pub(crate) fn new(listener: TcpListener) -> Incoming { - Incoming { inner: listener } - } -} - -impl Stream for Incoming { - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let (socket, _) = ready!(self.inner.poll_accept(cx))?; - Poll::Ready(Some(Ok(socket))) - } -} diff --git a/tokio/src/net/tcp/listener.rs b/tokio/src/net/tcp/listener.rs index b4bf4fcb110..d8f7dfe08f8 100644 --- a/tokio/src/net/tcp/listener.rs +++ b/tokio/src/net/tcp/listener.rs @@ -1,9 +1,8 @@ -use crate::net::tcp::{Incoming, TcpStream}; +use crate::future::poll_fn; +use crate::net::tcp::TcpStream; use crate::net::util::PollEvented; use crate::net::ToSocketAddrs; -use futures_core::ready; -use futures_util::future::poll_fn; use std::convert::TryFrom; use std::fmt; use std::io; @@ -12,9 +11,6 @@ use std::task::{Context, Poll}; /// An I/O object representing a TCP socket listening for incoming connections. /// -/// This object can be converted into a stream of incoming connections for -/// various forms of processing. -/// /// # Examples /// /// ```no_run @@ -230,22 +226,6 @@ impl TcpListener { self.io.get_ref().local_addr() } - /// Consumes this listener, returning a stream of the sockets this listener - /// accepts. - /// - /// This method returns an implementation of the `Stream` trait which - /// resolves to the sockets the are accepted on this listener. - /// - /// # Errors - /// - /// Note that accepting a connection can lead to various errors and not all of them are - /// necessarily fatal ‒ for example having too many open file descriptors or the other side - /// closing the connection while it waits in an accept queue. These would terminate the stream - /// if not handled in any way. - pub fn incoming(self) -> Incoming { - Incoming::new(self) - } - /// Gets the value of the `IP_TTL` option for this socket. /// /// For more information about this option, see [`set_ttl`]. diff --git a/tokio/src/net/tcp/mod.rs b/tokio/src/net/tcp/mod.rs index f80eb833874..a4e5ab4cb48 100644 --- a/tokio/src/net/tcp/mod.rs +++ b/tokio/src/net/tcp/mod.rs @@ -14,11 +14,6 @@ //! [`TcpStream`]: struct.TcpStream.html //! [`connect`]: struct.TcpStream.html#method.connect //! [`TcpListener`]: struct.TcpListener.html -//! [incoming_method]: struct.TcpListener.html#method.incoming -//! [`Incoming`]: struct.Incoming.html - -mod incoming; -pub use self::incoming::Incoming; mod listener; pub use self::listener::TcpListener; diff --git a/tokio/src/net/tcp/stream.rs b/tokio/src/net/tcp/stream.rs index 09629cfb8cc..1a844e86c83 100644 --- a/tokio/src/net/tcp/stream.rs +++ b/tokio/src/net/tcp/stream.rs @@ -1,11 +1,10 @@ +use crate::future::poll_fn; use crate::io::{AsyncRead, AsyncWrite}; use crate::net::tcp::split::{split, ReadHalf, WriteHalf}; use crate::net::util::PollEvented; use crate::net::ToSocketAddrs; use bytes::{Buf, BufMut}; -use futures_core::ready; -use futures_util::future::poll_fn; use iovec::IoVec; use std::convert::TryFrom; use std::fmt; diff --git a/tokio/src/net/udp/socket.rs b/tokio/src/net/udp/socket.rs index 9cd861c86f1..b1d1e5cc499 100644 --- a/tokio/src/net/udp/socket.rs +++ b/tokio/src/net/udp/socket.rs @@ -1,9 +1,8 @@ +use crate::future::poll_fn; use crate::net::udp::split::{split, UdpSocketRecvHalf, UdpSocketSendHalf}; use crate::net::util::PollEvented; use crate::net::ToSocketAddrs; -use futures_core::ready; -use futures_util::future::poll_fn; use std::convert::TryFrom; use std::fmt; use std::io; diff --git a/tokio/src/net/udp/split.rs b/tokio/src/net/udp/split.rs index ad8ce061494..55ca082d81c 100644 --- a/tokio/src/net/udp/split.rs +++ b/tokio/src/net/udp/split.rs @@ -12,9 +12,9 @@ //! The halves can be reunited to the original socket with their `reunite` //! methods. -use super::UdpSocket; +use crate::future::poll_fn; +use crate::net::udp::UdpSocket; -use futures_util::future::poll_fn; use std::error::Error; use std::fmt; use std::io; diff --git a/tokio/src/net/unix/datagram.rs b/tokio/src/net/unix/datagram.rs index f9c47deccf4..55eebd6c1e4 100644 --- a/tokio/src/net/unix/datagram.rs +++ b/tokio/src/net/unix/datagram.rs @@ -1,7 +1,6 @@ +use crate::future::poll_fn; use crate::net::util::PollEvented; -use futures_core::ready; -use futures_util::future::poll_fn; use std::convert::TryFrom; use std::fmt; use std::io; diff --git a/tokio/src/net/unix/incoming.rs b/tokio/src/net/unix/incoming.rs deleted file mode 100644 index a66f21da0f0..00000000000 --- a/tokio/src/net/unix/incoming.rs +++ /dev/null @@ -1,29 +0,0 @@ -use super::{UnixListener, UnixStream}; - -use futures_core::ready; -use futures_core::stream::Stream; -use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Stream of listeners -#[derive(Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct Incoming { - inner: UnixListener, -} - -impl Incoming { - pub(crate) fn new(listener: UnixListener) -> Incoming { - Incoming { inner: listener } - } -} - -impl Stream for Incoming { - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let (socket, _) = ready!(Pin::new(&mut self.inner).poll_accept(cx))?; - Poll::Ready(Some(Ok(socket))) - } -} diff --git a/tokio/src/net/unix/listener.rs b/tokio/src/net/unix/listener.rs index 3cf8eff3235..72cd55bdf05 100644 --- a/tokio/src/net/unix/listener.rs +++ b/tokio/src/net/unix/listener.rs @@ -1,8 +1,7 @@ +use crate::future::poll_fn; use crate::net::unix::UnixStream; use crate::net::util::PollEvented; -use futures_core::ready; -use futures_util::future::poll_fn; use mio::Ready; use mio_uds; use std::convert::TryFrom; @@ -84,15 +83,6 @@ impl UnixListener { Err(err) => Err(err).into(), } } - - /// Consumes this listener, returning a stream of the sockets this listener - /// accepts. - /// - /// This method returns an implementation of the `Stream` trait which - /// resolves to the sockets the are accepted on this listener. - pub fn incoming(self) -> super::Incoming { - super::Incoming::new(self) - } } impl TryFrom for mio_uds::UnixListener { diff --git a/tokio/src/net/unix/mod.rs b/tokio/src/net/unix/mod.rs index 977e3a0f816..4fd0176be51 100644 --- a/tokio/src/net/unix/mod.rs +++ b/tokio/src/net/unix/mod.rs @@ -5,9 +5,6 @@ mod datagram; pub use self::datagram::UnixDatagram; -mod incoming; -pub use self::incoming::Incoming; - mod listener; pub use self::listener::UnixListener; diff --git a/tokio/src/net/unix/stream.rs b/tokio/src/net/unix/stream.rs index 2ea8b6c70ca..774d20f995f 100644 --- a/tokio/src/net/unix/stream.rs +++ b/tokio/src/net/unix/stream.rs @@ -1,11 +1,10 @@ +use crate::future::poll_fn; use crate::io::{AsyncRead, AsyncWrite}; use crate::net::unix::split::{split, ReadHalf, WriteHalf}; use crate::net::unix::ucred::{self, UCred}; use crate::net::util::PollEvented; use bytes::{Buf, BufMut}; -use futures_core::ready; -use futures_util::future::poll_fn; use iovec::IoVec; use std::convert::TryFrom; use std::fmt; diff --git a/tokio/src/net/util/poll_evented.rs b/tokio/src/net/util/poll_evented.rs index 619a35ad5c0..08dea3f3856 100644 --- a/tokio/src/net/util/poll_evented.rs +++ b/tokio/src/net/util/poll_evented.rs @@ -1,7 +1,6 @@ use crate::io::{AsyncRead, AsyncWrite}; use crate::net::driver::{platform, Registration}; -use futures_core::ready; use mio::event::Evented; use std::fmt; use std::io::{self, Read, Write}; @@ -55,7 +54,7 @@ use std::task::{Context, Poll}; /// ```rust /// use tokio::net::util::PollEvented; /// -/// use futures_core::ready; +/// use futures::ready; /// use mio::Ready; /// use mio::net::{TcpStream, TcpListener}; /// use std::io; diff --git a/tokio/src/prelude.rs b/tokio/src/prelude.rs index 956003ec6eb..8675c577886 100644 --- a/tokio/src/prelude.rs +++ b/tokio/src/prelude.rs @@ -11,21 +11,6 @@ //! //! The prelude may grow over time as additional items see ubiquitous use. -#[doc(no_inline)] -pub use crate::future::FutureExt as _; -#[doc(no_inline)] -pub use futures_util::future::FutureExt as _; -pub use std::future::Future; - -pub use crate::stream::Stream; -#[doc(no_inline)] -pub use crate::stream::StreamExt as _; -pub use futures_sink::Sink; -#[doc(no_inline)] -pub use futures_util::sink::SinkExt as _; -#[doc(no_inline)] -pub use futures_util::stream::StreamExt as _; - pub use crate::io::{AsyncBufRead, AsyncRead, AsyncWrite}; #[cfg(feature = "io-util")] #[doc(no_inline)] diff --git a/tokio/src/process/mod.rs b/tokio/src/process/mod.rs index e777da52502..3013a65a7d5 100644 --- a/tokio/src/process/mod.rs +++ b/tokio/src/process/mod.rs @@ -58,7 +58,6 @@ //! use tokio::io::{BufReader, AsyncBufReadExt}; //! use tokio::process::Command; //! -//! use futures_util::stream::StreamExt; //! use std::process::Stdio; //! //! #[tokio::main] @@ -89,8 +88,8 @@ //! println!("child status was: {}", status); //! }); //! -//! while let Some(line) = reader.next().await { -//! println!("Line: {}", line?); +//! while let Some(line) = reader.next_line().await? { +//! println!("Line: {}", line); //! } //! //! Ok(()) @@ -120,8 +119,6 @@ mod kill; use crate::io::{AsyncRead, AsyncReadExt, AsyncWrite}; use crate::process::kill::Kill; -use futures_core::TryFuture; -use futures_util::future::try_join3; use std::ffi::OsStr; use std::future::Future; use std::io; @@ -681,11 +678,14 @@ impl Drop for ChildDropGuard { } } -impl Future for ChildDropGuard { - type Output = Result; +impl Future for ChildDropGuard +where + F: Future> + Kill + Unpin, +{ + type Output = Result; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let ret = Pin::new(&mut self.inner).try_poll(cx); + let ret = Pin::new(&mut self.inner).poll(cx); if let Poll::Ready(Ok(_)) = ret { // Avoid the overhead of trying to kill a reaped process @@ -766,6 +766,8 @@ impl Child { /// new pipes between parent and child. Use `stdout(Stdio::piped())` or /// `stderr(Stdio::piped())`, respectively, when creating a `Command`. pub async fn wait_with_output(mut self) -> io::Result { + use crate::future::try_join3; + async fn read_to_end(io: Option) -> io::Result> { let mut vec = Vec::new(); if let Some(mut io) = io { @@ -940,16 +942,14 @@ mod sys { #[cfg(all(test, not(loom)))] mod test { + use super::kill::Kill; + use super::ChildDropGuard; + + use futures::future::FutureExt; use std::future::Future; use std::io; use std::pin::Pin; - use std::task::Context; - use std::task::Poll; - - use futures_util::future::FutureExt; - - use super::kill::Kill; - use super::ChildDropGuard; + use std::task::{Context, Poll}; struct Mock { num_kills: usize, @@ -1021,7 +1021,7 @@ mod test { let mut mock_reaped = Mock::with_result(Poll::Ready(Ok(()))); let mut mock_err = Mock::with_result(Poll::Ready(Err(()))); - let waker = futures_util::task::noop_waker(); + let waker = futures::task::noop_waker(); let mut context = Context::from_waker(&waker); { let mut guard = ChildDropGuard::new(&mut mock_pending); diff --git a/tokio/src/process/unix/mod.rs b/tokio/src/process/unix/mod.rs index dfb69f21d98..72f6f0bfd7a 100644 --- a/tokio/src/process/unix/mod.rs +++ b/tokio/src/process/unix/mod.rs @@ -22,14 +22,16 @@ //! bad in theory... mod orphan; +use orphan::{OrphanQueue, OrphanQueueImpl, Wait}; + mod reap; +use reap::Reaper; -use self::orphan::{OrphanQueue, OrphanQueueImpl, Wait}; -use self::reap::Reaper; -use super::SpawnedChild; use crate::net::util::PollEvented; use crate::process::kill::Kill; +use crate::process::SpawnedChild; use crate::signal::unix::{signal, Signal, SignalKind}; + use mio::event::Evented; use mio::unix::{EventedFd, UnixReady}; use mio::{Poll as MioPoll, PollOpt, Ready, Token}; @@ -38,11 +40,11 @@ use std::future::Future; use std::io; use std::os::unix::io::{AsRawFd, RawFd}; use std::pin::Pin; -use std::process::{self, ExitStatus}; +use std::process::ExitStatus; use std::task::Context; use std::task::Poll; -impl Wait for process::Child { +impl Wait for std::process::Child { fn id(&self) -> u32 { self.id() } @@ -52,14 +54,14 @@ impl Wait for process::Child { } } -impl Kill for process::Child { +impl Kill for std::process::Child { fn kill(&mut self) -> io::Result<()> { self.kill() } } lazy_static::lazy_static! { - static ref ORPHAN_QUEUE: OrphanQueueImpl = OrphanQueueImpl::new(); + static ref ORPHAN_QUEUE: OrphanQueueImpl = OrphanQueueImpl::new(); } struct GlobalOrphanQueue; @@ -70,8 +72,8 @@ impl fmt::Debug for GlobalOrphanQueue { } } -impl OrphanQueue for GlobalOrphanQueue { - fn push_orphan(&self, orphan: process::Child) { +impl OrphanQueue for GlobalOrphanQueue { + fn push_orphan(&self, orphan: std::process::Child) { ORPHAN_QUEUE.push_orphan(orphan) } @@ -82,7 +84,7 @@ impl OrphanQueue for GlobalOrphanQueue { #[must_use = "futures do nothing unless polled"] pub(crate) struct Child { - inner: Reaper, + inner: Reaper, } impl fmt::Debug for Child { @@ -93,7 +95,7 @@ impl fmt::Debug for Child { } } -pub(crate) fn spawn_child(cmd: &mut process::Command) -> io::Result { +pub(crate) fn spawn_child(cmd: &mut std::process::Command) -> io::Result { let mut child = cmd.spawn()?; let stdin = stdio(child.stdin.take())?; let stdout = stdio(child.stdout.take())?; @@ -196,9 +198,9 @@ where } } -pub(crate) type ChildStdin = PollEvented>; -pub(crate) type ChildStdout = PollEvented>; -pub(crate) type ChildStderr = PollEvented>; +pub(crate) type ChildStdin = PollEvented>; +pub(crate) type ChildStdout = PollEvented>; +pub(crate) type ChildStderr = PollEvented>; fn stdio(option: Option) -> io::Result>>> where diff --git a/tokio/src/process/unix/reap.rs b/tokio/src/process/unix/reap.rs index 631025d4dec..8963805afe3 100644 --- a/tokio/src/process/unix/reap.rs +++ b/tokio/src/process/unix/reap.rs @@ -1,6 +1,7 @@ -use super::orphan::{OrphanQueue, Wait}; +use crate::process::imp::orphan::{OrphanQueue, Wait}; use crate::process::kill::Kill; -use futures_core::stream::Stream; +use crate::signal::unix::Signal; + use std::future::Future; use std::io; use std::ops::Deref; @@ -22,6 +23,17 @@ where signal: S, } +// Work around removal of `futures_core` dependency +pub(crate) trait Stream: Unpin { + fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll>; +} + +impl Stream for Signal { + fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { + Signal::poll_recv(self, cx) + } +} + impl Deref for Reaper where W: Wait + Unpin, @@ -60,7 +72,7 @@ impl Future for Reaper where W: Wait + Unpin, Q: OrphanQueue + Unpin, - S: Stream + Unpin, + S: Stream, { type Output = io::Result; @@ -85,7 +97,7 @@ where // this future's task will be notified/woken up again. Since the // futures model allows for spurious wake ups this extra wakeup // should not cause significant issues with parent futures. - let registered_interest = Pin::new(&mut self.signal).poll_next(cx).is_pending(); + let registered_interest = self.signal.poll_recv(cx).is_pending(); self.orphan_queue.reap_orphans(); if let Some(status) = self.inner_mut().try_wait()? { @@ -134,11 +146,10 @@ where #[cfg(all(test, not(loom)))] mod test { use super::*; - use futures_core::stream::Stream; - use futures_util::future::FutureExt; + + use futures::future::FutureExt; use std::cell::{Cell, RefCell}; use std::os::unix::process::ExitStatusExt; - use std::pin::Pin; use std::process::ExitStatus; use std::task::Context; use std::task::Poll; @@ -201,13 +212,10 @@ mod test { } impl Stream for MockStream { - type Item = io::Result<()>; - - fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - let inner = Pin::get_mut(self); - inner.total_polls += 1; - match inner.values.remove(0) { - Some(()) => Poll::Ready(Some(Ok(()))), + fn poll_recv(&mut self, _cx: &mut Context<'_>) -> Poll> { + self.total_polls += 1; + match self.values.remove(0) { + Some(()) => Poll::Ready(Some(())), None => Poll::Pending, } } @@ -247,7 +255,7 @@ mod test { MockStream::new(vec![None, Some(()), None, None, None]), ); - let waker = futures_util::task::noop_waker(); + let waker = futures::task::noop_waker(); let mut context = Context::from_waker(&waker); // Not yet exited, interest registered diff --git a/tokio/src/runtime/thread_pool/tests/pool.rs b/tokio/src/runtime/thread_pool/tests/pool.rs index c11281f0ac9..25c11ea9d30 100644 --- a/tokio/src/runtime/thread_pool/tests/pool.rs +++ b/tokio/src/runtime/thread_pool/tests/pool.rs @@ -4,7 +4,7 @@ use crate::blocking; use crate::runtime::thread_pool::ThreadPool; use crate::runtime::{Park, Unpark}; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use std::future::Future; use std::pin::Pin; use std::sync::atomic::Ordering::Relaxed; diff --git a/tokio/src/runtime/thread_pool/tests/queue.rs b/tokio/src/runtime/thread_pool/tests/queue.rs index 7c0a65d5af5..ef0e373c7e0 100644 --- a/tokio/src/runtime/thread_pool/tests/queue.rs +++ b/tokio/src/runtime/thread_pool/tests/queue.rs @@ -246,7 +246,7 @@ fn val(num: u32) -> Task { } fn num(task: Task) -> u32 { - use futures_util::task::noop_waker_ref; + use futures::task::noop_waker_ref; use std::future::Future; use std::pin::Pin; use std::task::Context; diff --git a/tokio/src/signal/ctrl_c.rs b/tokio/src/signal/ctrl_c.rs index f9dd467924c..ed393b1dc0d 100644 --- a/tokio/src/signal/ctrl_c.rs +++ b/tokio/src/signal/ctrl_c.rs @@ -1,14 +1,11 @@ #[cfg(unix)] -use super::unix::{self as os_impl, Signal as Inner}; +use super::unix::{self as os_impl}; #[cfg(windows)] -use super::windows::{self as os_impl, Event as Inner}; +use super::windows::{self as os_impl}; -use futures_core::stream::Stream; use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; -/// Represents a stream which receives "ctrl-c" notifications sent to the process. +/// Completes when a "ctrl-c" notification is sent to the process. /// /// In general signals are handled very differently across Unix and Windows, but /// this is somewhat cross platform in terms of how it can be handled. A ctrl-c @@ -18,29 +15,7 @@ use std::task::{Context, Poll}; /// Note that there are a number of caveats listening for signals, and you may /// wish to read up on the documentation in the `unix` or `windows` module to /// take a peek. -/// -/// Notably, a notification to this process notifies *all* streams listening to -/// this event. Moreover, the notifications **are coalesced** if they aren't processed -/// quickly enough. This means that if two notifications are received back-to-back, -/// then the stream may only receive one item about the two notifications. -#[must_use = "streams do nothing unless polled"] -#[derive(Debug)] -pub struct CtrlC { - inner: Inner, -} - -/// Creates a new stream which receives "ctrl-c" notifications sent to the -/// process. -/// -/// This function binds to the default reactor. -pub fn ctrl_c() -> io::Result { - os_impl::ctrl_c().map(|inner| CtrlC { inner }) -} - -impl Stream for CtrlC { - type Item = (); - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut self.inner).poll_next(cx) - } +pub async fn ctrl_c() -> io::Result<()> { + os_impl::ctrl_c()?.recv().await; + Ok(()) } diff --git a/tokio/src/signal/mod.rs b/tokio/src/signal/mod.rs index f695b07cec4..6b36bc4e669 100644 --- a/tokio/src/signal/mod.rs +++ b/tokio/src/signal/mod.rs @@ -14,28 +14,15 @@ //! //! # Examples //! -//! Print out all ctrl-C notifications received +//! Print on "ctrl-c" notification. //! //! ```rust,no_run //! use tokio::signal; //! -//! use futures_util::future; -//! use futures_util::stream::StreamExt; -//! //! #[tokio::main] //! async fn main() -> Result<(), Box> { -//! // Create an infinite stream of "Ctrl+C" notifications. Each item received -//! // on this stream may represent multiple ctrl-c signals. -//! let ctrl_c = signal::ctrl_c()?; -//! -//! // Process each ctrl-c as it comes in -//! let prog = ctrl_c.for_each(|_| { -//! println!("ctrl-c received!"); -//! future::ready(()) -//! }); -//! -//! prog.await; -//! +//! signal::ctrl_c().await?; +//! println!("ctrl-c received!"); //! Ok(()) //! } //! ``` @@ -45,38 +32,25 @@ //! ```rust,no_run //! # #[cfg(unix)] { //! -//! use tokio::signal::{self, unix::{signal, SignalKind}}; -//! -//! use futures_util::future; -//! use futures_util::stream::StreamExt; +//! use tokio::signal::unix::{signal, SignalKind}; //! //! #[tokio::main] //! async fn main() -> Result<(), Box> { -//! // Create an infinite stream of "Ctrl+C" notifications. Each item received -//! // on this stream may represent multiple ctrl-c signals. -//! let ctrl_c = signal::ctrl_c()?; -//! -//! // Process each ctrl-c as it comes in -//! let prog = ctrl_c.for_each(|_| { -//! println!("ctrl-c received!"); -//! future::ready(()) -//! }); -//! -//! prog.await; -//! -//! // Like the previous example, this is an infinite stream of signals -//! // being received, and signals may be coalesced while pending. -//! let stream = signal(SignalKind::hangup())?; -//! -//! // Convert out stream into a future and block the program -//! let (signal, _stream) = stream.into_future().await; -//! println!("got signal {:?}", signal); -//! Ok(()) +//! // An infinite stream of hangup signals. +//! let mut stream = signal(SignalKind::hangup())?; +//! +//! // Print whenever a HUP signal is received +//! loop { +//! stream.recv().await; +//! println!("got signal HUP"); +//! } //! } //! # } //! ``` mod ctrl_c; +pub use ctrl_c::ctrl_c; + mod registry; mod os { @@ -89,5 +63,3 @@ mod os { pub mod unix; pub mod windows; - -pub use self::ctrl_c::{ctrl_c, CtrlC}; diff --git a/tokio/src/signal/registry.rs b/tokio/src/signal/registry.rs index d608539cf3b..ba247b00db4 100644 --- a/tokio/src/signal/registry.rs +++ b/tokio/src/signal/registry.rs @@ -180,7 +180,8 @@ mod tests { use super::*; use crate::runtime::{self, Runtime}; use crate::sync::{mpsc, oneshot}; - use futures::{future, StreamExt}; + + use futures::future; #[test] fn smoke() { @@ -220,11 +221,7 @@ mod tests { }); let _ = fire.send(()); - let all = future::join3( - first_rx.collect::>(), - second_rx.collect::>(), - third_rx.collect::>(), - ); + let all = future::join3(collect(first_rx), collect(second_rx), collect(third_rx)); let (first_results, second_results, third_results) = all.await; assert_eq!(2, first_results.len()); @@ -279,7 +276,7 @@ mod tests { }); let _ = fire.send(()); - let results: Vec<()> = third_rx.collect().await; + let results = collect(third_rx).await; assert_eq!(1, results.len()); }); @@ -311,4 +308,14 @@ mod tests { fn rt() -> Runtime { runtime::Builder::new().current_thread().build().unwrap() } + + async fn collect(mut rx: crate::sync::mpsc::Receiver<()>) -> Vec<()> { + let mut ret = vec![]; + + while let Some(v) = rx.recv().await { + ret.push(v); + } + + ret + } } diff --git a/tokio/src/signal/unix.rs b/tokio/src/signal/unix.rs index 87871503db7..17cb2e83578 100644 --- a/tokio/src/signal/unix.rs +++ b/tokio/src/signal/unix.rs @@ -10,10 +10,8 @@ use crate::net::util::PollEvented; use crate::signal::registry::{globals, EventId, EventInfo, Globals, Init, Storage}; use crate::sync::mpsc::{channel, Receiver}; -use futures_core::stream::Stream; use libc::c_int; use mio_uds::UnixStream; -use std::future::Future; use std::io::{self, Error, ErrorKind, Write}; use std::pin::Pin; use std::sync::atomic::{AtomicBool, Ordering}; @@ -262,10 +260,8 @@ struct Driver { wakeup: PollEvented, } -impl Future for Driver { - type Output = (); - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { +impl Driver { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll<()> { // Drain the data from the pipe and maintain interest in getting more self.drain(cx); // Broadcast any signals which were received @@ -302,7 +298,7 @@ impl Driver { /// We do *NOT* use the existence of any read bytes as evidence a sigal was /// received since the `pending` flags would have already been set if that /// was the case. See #38 for more info. - fn drain(mut self: Pin<&mut Self>, cx: &mut Context<'_>) { + fn drain(&mut self, cx: &mut Context<'_>) { loop { match Pin::new(&mut self.wakeup).poll_read(cx, &mut [0; 128]) { Poll::Ready(Ok(0)) => panic!("EOF on self-pipe"), @@ -395,20 +391,24 @@ pub fn signal(kind: SignalKind) -> io::Result { Ok(Signal { driver, rx }) } -pub(crate) fn ctrl_c() -> io::Result { - signal(SignalKind::interrupt()) -} - -impl Stream for Signal { - type Item = (); - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let _ = Pin::new(&mut self.driver).poll(cx); +impl Signal { + /// TODO: Dox + pub async fn recv(&mut self) -> Option<()> { + use crate::future::poll_fn; + poll_fn(|cx| self.poll_recv(cx)).await + } + #[doc(hidden)] // TODO: document + pub fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { + let _ = self.driver.poll(cx); self.rx.poll_recv(cx) } } +pub(crate) fn ctrl_c() -> io::Result { + signal(SignalKind::interrupt()) +} + #[cfg(all(test, not(loom)))] mod tests { use super::*; diff --git a/tokio/src/stream.rs b/tokio/src/stream.rs deleted file mode 100644 index 0a597a58f56..00000000000 --- a/tokio/src/stream.rs +++ /dev/null @@ -1,78 +0,0 @@ -//! A sequence of asynchronous values. - -#[cfg(feature = "time")] -use std::time::Duration; - -#[cfg(feature = "time")] -use crate::time::{throttle::Throttle, Timeout}; - -#[doc(inline)] -pub use futures_core::Stream; -#[doc(inline)] -pub use futures_util::stream::{empty, iter, once, pending, poll_fn, repeat, unfold}; - -/// An extension trait for `Stream` that provides a variety of convenient -/// combinator functions. -/// -/// Currently, there are only [`timeout`] and [`throttle`] functions, but -/// this will increase over time. -/// -/// Users are not expected to implement this trait. All types that implement -/// `Stream` already implement `StreamExt`. -/// -/// This trait can be imported directly or via the Tokio prelude: `use -/// tokio::prelude::*`. -/// -/// [`throttle`]: method.throttle -/// [`timeout`]: method.timeout -pub trait StreamExt: Stream { - /// Throttle down the stream by enforcing a fixed delay between items. - /// - /// Errors are also delayed. - #[cfg(feature = "time")] - fn throttle(self, duration: Duration) -> Throttle - where - Self: Sized, - { - Throttle::new(self, duration) - } - - /// Creates a new stream which allows `self` until `timeout`. - /// - /// This combinator creates a new stream which wraps the receiving stream - /// with a timeout. For each item, the returned stream is allowed to execute - /// until it completes or `timeout` has elapsed, whichever happens first. - /// - /// If an item completes before `timeout` then the stream will yield - /// with that item. Otherwise the stream will yield to an error. - /// - /// # Examples - /// - /// ``` - /// use tokio::prelude::*; - /// - /// use std::time::Duration; - /// - /// # fn slow_stream() -> impl Stream { - /// # tokio::stream::empty() - /// # } - /// # - /// # async fn dox() { - /// let mut stream = slow_stream() - /// .timeout(Duration::from_secs(1)); - /// - /// while let Some(value) = stream.next().await { - /// println!("value = {:?}", value); - /// } - /// # } - /// ``` - #[cfg(feature = "time")] - fn timeout(self, timeout: Duration) -> Timeout - where - Self: Sized, - { - Timeout::new(self, timeout) - } -} - -impl StreamExt for T where T: Stream {} diff --git a/tokio/src/stream/mod.rs b/tokio/src/stream/mod.rs new file mode 100644 index 00000000000..27c3d9154a2 --- /dev/null +++ b/tokio/src/stream/mod.rs @@ -0,0 +1,10 @@ +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Asynchronous iteration of values. +#[allow(unreachable_pub)] +pub trait Stream { + type Item; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; +} diff --git a/tokio/src/sync/barrier.rs b/tokio/src/sync/barrier.rs index 1582120e077..911e78fef42 100644 --- a/tokio/src/sync/barrier.rs +++ b/tokio/src/sync/barrier.rs @@ -8,8 +8,9 @@ use std::sync::Mutex; /// # #[tokio::main] /// # async fn main() { /// use tokio::sync::Barrier; +/// +/// use futures::future::join_all; /// use std::sync::Arc; -/// use futures_util::future::join_all; /// /// let mut handles = Vec::with_capacity(10); /// let barrier = Arc::new(Barrier::new(10)); diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 787dd5077c4..511f07ccfba 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -2,7 +2,6 @@ use crate::sync::mpsc::chan; use crate::sync::semaphore; use std::fmt; -use std::pin::Pin; use std::task::{Context, Poll}; /// Send values to the associated `Receiver`. @@ -161,7 +160,7 @@ impl Receiver { /// } /// ``` pub async fn recv(&mut self) -> Option { - use futures_util::future::poll_fn; + use crate::future::poll_fn; poll_fn(|cx| self.poll_recv(cx)).await } @@ -180,14 +179,6 @@ impl Receiver { } } -impl futures_core::Stream for Receiver { - type Item = T; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.get_mut().poll_recv(cx) - } -} - impl Sender { pub(crate) fn new(chan: chan::Tx) -> Sender { Sender { chan } @@ -234,7 +225,7 @@ impl Sender { /// } /// ``` pub async fn send(&mut self, value: T) -> Result<(), SendError> { - use futures_util::future::poll_fn; + use crate::future::poll_fn; poll_fn(|cx| self.poll_ready(cx)).await?; @@ -242,29 +233,6 @@ impl Sender { } } -impl futures_sink::Sink for Sender { - type Error = SendError; - - fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Sender::poll_ready(self.get_mut(), cx) - } - - fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::Error> { - self.as_mut().try_send(msg).map_err(|err| { - assert!(err.is_full(), "call `poll_ready` before sending"); - SendError(()) - }) - } - - fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) - } - - fn poll_close(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) - } -} - // ===== impl SendError ===== impl fmt::Display for SendError { diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index 4eb750efbe7..b347dee8797 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -4,8 +4,6 @@ use crate::sync::mpsc::chan; use std::fmt; use std::task::{Context, Poll}; -use std::pin::Pin; - /// Send values to the associated `UnboundedReceiver`. /// /// Instances are created by the @@ -131,7 +129,7 @@ impl UnboundedReceiver { /// } /// ``` pub async fn recv(&mut self) -> Option { - use futures_util::future::poll_fn; + use crate::future::poll_fn; poll_fn(|cx| self.poll_recv(cx)).await } @@ -145,14 +143,6 @@ impl UnboundedReceiver { } } -impl futures_core::Stream for UnboundedReceiver { - type Item = T; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.chan.recv(cx) - } -} - impl UnboundedSender { pub(crate) fn new(chan: chan::Tx) -> UnboundedSender { UnboundedSender { chan } @@ -165,26 +155,6 @@ impl UnboundedSender { } } -impl futures_sink::Sink for UnboundedSender { - type Error = UnboundedSendError; - - fn poll_ready(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) - } - - fn start_send(mut self: Pin<&mut Self>, msg: T) -> Result<(), Self::Error> { - self.try_send(msg).map_err(|_| UnboundedSendError(())) - } - - fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) - } - - fn poll_close(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) - } -} - // ===== impl UnboundedSendError ===== impl fmt::Display for UnboundedSendError { diff --git a/tokio/src/sync/mutex.rs b/tokio/src/sync/mutex.rs index ae45c666c42..b06f22b422a 100644 --- a/tokio/src/sync/mutex.rs +++ b/tokio/src/sync/mutex.rs @@ -29,9 +29,9 @@ //! [`Mutex`]: struct.Mutex.html //! [`MutexGuard`]: struct.MutexGuard.html +use crate::future::poll_fn; use crate::sync::semaphore; -use futures_util::future::poll_fn; use std::cell::UnsafeCell; use std::fmt; use std::ops::{Deref, DerefMut}; diff --git a/tokio/src/sync/oneshot.rs b/tokio/src/sync/oneshot.rs index 3c757e9e527..7b84f3194c7 100644 --- a/tokio/src/sync/oneshot.rs +++ b/tokio/src/sync/oneshot.rs @@ -4,7 +4,6 @@ use crate::loom::cell::CausalCell; use crate::loom::sync::atomic::AtomicUsize; use crate::loom::sync::Arc; -use futures_core::ready; use std::fmt; use std::future::Future; use std::mem::MaybeUninit; @@ -225,7 +224,7 @@ impl Sender { /// } /// ``` pub async fn closed(&mut self) { - use futures_util::future::poll_fn; + use crate::future::poll_fn; poll_fn(|cx| self.poll_closed(cx)).await } diff --git a/tokio/src/sync/tests/loom_atomic_waker.rs b/tokio/src/sync/tests/loom_atomic_waker.rs index 81e200ff9eb..c148bcbe117 100644 --- a/tokio/src/sync/tests/loom_atomic_waker.rs +++ b/tokio/src/sync/tests/loom_atomic_waker.rs @@ -1,6 +1,6 @@ use crate::sync::task::AtomicWaker; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use loom::future::block_on; use loom::sync::atomic::AtomicUsize; use loom::thread; diff --git a/tokio/src/sync/tests/loom_mpsc.rs b/tokio/src/sync/tests/loom_mpsc.rs index 748ae9e1cf4..8fd6d14b111 100644 --- a/tokio/src/sync/tests/loom_mpsc.rs +++ b/tokio/src/sync/tests/loom_mpsc.rs @@ -1,6 +1,6 @@ use crate::sync::mpsc; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use loom::future::block_on; use loom::thread; diff --git a/tokio/src/sync/tests/loom_oneshot.rs b/tokio/src/sync/tests/loom_oneshot.rs index 521047368ae..dfa7459da7f 100644 --- a/tokio/src/sync/tests/loom_oneshot.rs +++ b/tokio/src/sync/tests/loom_oneshot.rs @@ -1,6 +1,6 @@ use crate::sync::oneshot; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use loom::future::block_on; use loom::thread; use std::task::Poll::{Pending, Ready}; diff --git a/tokio/src/sync/tests/loom_semaphore.rs b/tokio/src/sync/tests/loom_semaphore.rs index d14c76686df..7b8de0f05d6 100644 --- a/tokio/src/sync/tests/loom_semaphore.rs +++ b/tokio/src/sync/tests/loom_semaphore.rs @@ -1,7 +1,6 @@ use crate::sync::semaphore::*; -use futures_core::ready; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use loom::future::block_on; use loom::thread; use std::future::Future; diff --git a/tokio/src/sync/watch.rs b/tokio/src/sync/watch.rs index 928c2c46410..a178720b9a3 100644 --- a/tokio/src/sync/watch.rs +++ b/tokio/src/sync/watch.rs @@ -51,20 +51,16 @@ //! [`Sender::closed`]: struct.Sender.html#method.closed //! [`Receiver::get_ref`]: struct.Receiver.html#method.get_ref +use crate::future::poll_fn; use crate::sync::task::AtomicWaker; -use core::task::Poll::{Pending, Ready}; -use core::task::{Context, Poll}; use fnv::FnvHashMap; -use futures_util::future::poll_fn; use std::ops; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use std::sync::{Arc, Mutex, RwLock, RwLockReadGuard, Weak}; - -use futures_core::ready; -use futures_util::pin_mut; -use std::pin::Pin; +use std::task::Poll::{Pending, Ready}; +use std::task::{Context, Poll}; /// Receives values from the associated [`Sender`](struct.Sender.html). /// @@ -235,77 +231,40 @@ impl Receiver { Ref { inner } } - /// Attempts to receive the latest value sent via the channel. - /// - /// If a new, unobserved, value has been sent, a reference to it is - /// returned. If no new value has been sent, then `Pending` is returned and - /// the current task is notified once a new value is sent. - /// - /// Only the **most recent** value is returned. If the receiver is falling - /// behind the sender, intermediate values are dropped. - pub async fn recv_ref(&mut self) -> Option> { - let shared = &self.shared; - let inner = &self.inner; - let version = self.ver; - - match poll_fn(|cx| poll_lock(cx, shared, inner, version)).await { - Some((lock, version)) => { - self.ver = version; - Some(lock) - } - None => None, - } - } -} + // TODO: document + #[doc(hidden)] + pub fn poll_recv_ref<'a>(&'a mut self, cx: &mut Context<'_>) -> Poll>> { + // Make sure the task is up to date + self.inner.waker.register_by_ref(cx.waker()); -fn poll_lock<'a, T>( - cx: &mut Context<'_>, - shared: &'a Arc>, - inner: &Arc, - ver: usize, -) -> Poll, usize)>> { - // Make sure the task is up to date - inner.waker.register_by_ref(cx.waker()); + let state = self.shared.version.load(SeqCst); + let version = state & !CLOSED; - let state = shared.version.load(SeqCst); - let version = state & !CLOSED; + if version != self.ver { + let inner = self.shared.value.read().unwrap(); + self.ver = version; - if version != ver { - let inner = shared.value.read().unwrap(); + return Ready(Some(Ref { inner })); + } - return Ready(Some((Ref { inner }, version))); - } + if CLOSED == state & CLOSED { + // The `Store` handle has been dropped. + return Ready(None); + } - if CLOSED == state & CLOSED { - // The `Store` handle has been dropped. - return Ready(None); + Pending } - - Pending } impl Receiver { /// Attempts to clone the latest value sent via the channel. - /// - /// This is equivalent to calling `clone()` on the value returned by - /// `recv_ref()`. #[allow(clippy::map_clone)] // false positive: https://github.com/rust-lang/rust-clippy/issues/3274 pub async fn recv(&mut self) -> Option { - self.recv_ref().await.map(|v_ref| v_ref.clone()) - } -} - -impl futures_core::Stream for Receiver { - type Item = T; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - use std::future::Future; - - let fut = self.get_mut().recv(); - pin_mut!(fut); - - let item = ready!(fut.poll(cx)); - Ready(item.map(|v_ref| v_ref)) + poll_fn(|cx| { + let v_ref = ready!(self.poll_recv_ref(cx)); + Poll::Ready(v_ref.map(|v| v.clone())) + }) + .await } } @@ -394,27 +353,6 @@ impl Sender { } } -impl futures_sink::Sink for Sender { - type Error = error::SendError; - - fn poll_ready(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Ready(Ok(())) - } - - fn start_send(self: Pin<&mut Self>, item: T) -> Result<(), Self::Error> { - self.as_ref().get_ref().broadcast(item)?; - Ok(()) - } - - fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Ready(Ok(())) - } - - fn poll_close(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Ready(Ok(())) - } -} - /// Notify all watchers of a change fn notify_all(shared: &Shared) { let watchers = shared.watchers.lock().unwrap(); diff --git a/tokio/src/task/tests/loom.rs b/tokio/src/task/tests/loom.rs index dd6fbbefcae..f1ccace2403 100644 --- a/tokio/src/task/tests/loom.rs +++ b/tokio/src/task/tests/loom.rs @@ -29,7 +29,7 @@ fn create_drop_join_handle() { #[test] fn poll_drop_handle_then_drop() { - use futures_util::future::poll_fn; + use futures::future::poll_fn; use std::pin::Pin; use std::task::Poll; @@ -196,7 +196,7 @@ fn shutdown_from_queue_after_poll() { } fn gated(n: usize, complete_first_poll: bool, by_val: bool) -> impl Future { - use futures_util::future::poll_fn; + use futures::future::poll_fn; use std::sync::Arc; use std::task::Poll; @@ -255,7 +255,7 @@ fn join_one_task(join_handle: T) -> loom::thread::JoinHandl fn join_two_tasks( join_handle: T, ) -> loom::thread::JoinHandle { - use futures_util::future::poll_fn; + use futures::future::poll_fn; use std::task::Poll; // Join handle diff --git a/tokio/src/task/tests/task.rs b/tokio/src/task/tests/task.rs index 95b1451cc93..9121c7df596 100644 --- a/tokio/src/task/tests/task.rs +++ b/tokio/src/task/tests/task.rs @@ -7,7 +7,7 @@ use crate::tests::track_drop::track_drop; use tokio_test::task::spawn; use tokio_test::{assert_pending, assert_ready_err, assert_ready_ok}; -use futures_util::future::poll_fn; +use futures::future::poll_fn; use std::sync::mpsc; #[test] diff --git a/tokio/src/time/clock.rs b/tokio/src/time/clock.rs index 982c79aae42..915301b82d6 100644 --- a/tokio/src/time/clock.rs +++ b/tokio/src/time/clock.rs @@ -7,7 +7,9 @@ #[cfg(feature = "test-util")] #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::variant::{advance, pause, resume}; -pub(crate) use self::variant::{now, Clock}; +pub(crate) use self::variant::Clock; +#[cfg(feature = "test-util")] +pub(crate) use self::variant::now; #[cfg(not(feature = "test-util"))] mod variant { diff --git a/tokio/src/time/delay.rs b/tokio/src/time/delay.rs index 83ab3c8fe3e..e3b605e74b7 100644 --- a/tokio/src/time/delay.rs +++ b/tokio/src/time/delay.rs @@ -1,25 +1,44 @@ use crate::time::driver::Registration; use crate::time::{Duration, Instant}; -use futures_core::ready; use std::future::Future; use std::pin::Pin; use std::task::{self, Poll}; -/// A future that completes at a specified instant in time. +/// Wait until `deadline` is reached. /// -/// Instances of `Delay` perform no work and complete with `()` once the -/// specified deadline has been reached. -/// -/// `Delay` has a resolution of one millisecond and should not be used for tasks -/// that require high-resolution timers. +/// No work is performed while awaiting on the delay to complete. The delay +/// operates at millisecond granularity and should not be used for tasks that +/// require high-resolution timers. /// /// # Cancellation /// -/// Canceling a `Delay` is done by dropping the value. No additional cleanup or -/// other work is required. +/// Canceling a delay is done by dropping the returned future. No additional +/// cleanup work is required. +pub fn delay_until(deadline: Instant) -> Delay { + let registration = Registration::new(deadline, Duration::from_millis(0)); + Delay { registration } +} + +/// Wait until `duration` has elapsed. +/// +/// Equivalent to `delay_until(Instant::now() + duration)`. An asynchronous +/// analog to `std::thread::sleep`. /// -/// [`new`]: #method.new +/// No work is performed while awaiting on the delay to complete. The delay +/// operates at millisecond granularity and should not be used for tasks that +/// require high-resolution timers. +/// +/// # Cancellation +/// +/// Canceling a delay is done by dropping the returned future. No additional +/// cleanup work is required. +pub fn delay_for(duration: Duration) -> Delay { + delay_until(Instant::now() + duration) +} + +/// Future returned by [`delay_until`](delay_until) and +/// [`delay_for`](delay_for). #[derive(Debug)] #[must_use = "futures do nothing unless you `.await` or poll them"] pub struct Delay { @@ -30,17 +49,6 @@ pub struct Delay { } impl Delay { - /// Create a new `Delay` instance that elapses at `deadline`. - /// - /// Only millisecond level resolution is guaranteed. There is no guarantee - /// as to how the sub-millisecond portion of `deadline` will be handled. - /// `Delay` should not be used for high-resolution timer use cases. - pub(crate) fn new(deadline: Instant) -> Delay { - let registration = Registration::new(deadline, Duration::from_millis(0)); - - Delay { registration } - } - pub(crate) fn new_timeout(deadline: Instant, duration: Duration) -> Delay { let registration = Registration::new(deadline, duration); Delay { registration } @@ -69,10 +77,6 @@ impl Delay { self.registration.reset(deadline); } - pub(crate) fn reset_timeout(&mut self) { - self.registration.reset_timeout(); - } - /// Register the delay with the timer instance for the current execution /// context. fn register(&mut self) { diff --git a/tokio/src/time/delay_queue.rs b/tokio/src/time/delay_queue.rs index 6fa455a67d8..6a7cc6b3b23 100644 --- a/tokio/src/time/delay_queue.rs +++ b/tokio/src/time/delay_queue.rs @@ -5,9 +5,8 @@ //! [`DelayQueue`]: struct.DelayQueue.html use crate::time::wheel::{self, Wheel}; -use crate::time::{Delay, Duration, Error, Instant}; +use crate::time::{delay_until, Delay, Duration, Error, Instant}; -use futures_core::ready; use slab::Slab; use std::cmp; use std::future::Future; @@ -69,7 +68,7 @@ use std::task::{self, Poll}; /// ```rust,no_run /// use tokio::time::{delay_queue, DelayQueue, Error}; /// -/// use futures_core::ready; +/// use futures::ready; /// use std::collections::HashMap; /// use std::task::{Context, Poll}; /// use std::time::Duration; @@ -103,7 +102,7 @@ use std::task::{self, Poll}; /// } /// /// fn poll_purge(&mut self, cx: &mut Context<'_>) -> Poll> { -/// while let Some(res) = ready!(self.expirations.poll_next(cx)) { +/// while let Some(res) = ready!(self.expirations.poll_expired(cx)) { /// let entry = res?; /// self.entries.remove(entry.get_ref()); /// } @@ -320,7 +319,7 @@ impl DelayQueue { }; if should_set_delay { - self.delay = Some(Delay::new(self.start + Duration::from_millis(when))); + self.delay = Some(delay_until(self.start + Duration::from_millis(when))); } Key::new(key) @@ -329,7 +328,7 @@ impl DelayQueue { /// Attempt to pull out the next value of the delay queue, registering the /// current task for wakeup if the value is not yet available, and returning /// None if the queue is exhausted. - pub fn poll_next( + pub fn poll_expired( &mut self, cx: &mut task::Context<'_>, ) -> Poll, Error>>> { @@ -676,7 +675,7 @@ impl DelayQueue { } if let Some(deadline) = self.next_deadline() { - self.delay = Some(Delay::new(deadline)); + self.delay = Some(delay_until(deadline)); } else { return Poll::Ready(None); } @@ -697,16 +696,6 @@ impl DelayQueue { // We never put `T` in a `Pin`... impl Unpin for DelayQueue {} -impl futures_core::Stream for DelayQueue { - // DelayQueue seems much more specific, where a user may care that it - // has reached capacity, so return those errors instead of panicking. - type Item = Result, Error>; - - fn poll_next(self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> Poll> { - DelayQueue::poll_next(self.get_mut(), cx) - } -} - impl Default for DelayQueue { fn default() -> DelayQueue { DelayQueue::new() diff --git a/tokio/src/time/driver/registration.rs b/tokio/src/time/driver/registration.rs index 3641e549b49..27b4c1cc296 100644 --- a/tokio/src/time/driver/registration.rs +++ b/tokio/src/time/driver/registration.rs @@ -37,15 +37,7 @@ impl Registration { unsafe { self.entry.time_mut().deadline = deadline; } - Entry::reset(&mut self.entry); - } - // Used by `Timeout` - pub(crate) fn reset_timeout(&mut self) { - let deadline = crate::time::clock::now() + self.entry.time_ref().duration; - unsafe { - self.entry.time_mut().deadline = deadline; - } Entry::reset(&mut self.entry); } diff --git a/tokio/src/time/interval.rs b/tokio/src/time/interval.rs index e60a8b8c6a6..9a0a4e69b8c 100644 --- a/tokio/src/time/interval.rs +++ b/tokio/src/time/interval.rs @@ -1,60 +1,93 @@ -use crate::time::{Delay, Duration, Instant}; +use crate::future::poll_fn; +use crate::time::{delay_until, Delay, Duration, Instant}; -use futures_core::ready; -use futures_util::future::poll_fn; use std::future::Future; use std::pin::Pin; use std::task::{self, Poll}; -/// A stream representing notifications at fixed interval +/// Creates new `Interval` that yields with interval of `duration`. The first +/// tick completes immediately. +/// +/// An interval will tick indefinitely. At any time, the `Interval` value can be +/// dropped. This cancels the interval. +/// +/// This function is equivalent to `interval_at(Instant::now(), period)`. +/// +/// # Panics +/// +/// This function panics if `period` is zero. +/// +/// # Examples +/// +/// ``` +/// use tokio::time::{self, Duration}; +/// +/// #[tokio::main] +/// async fn main() { +/// let mut interval = time::interval(Duration::from_millis(10)); +/// +/// interval.tick().await; +/// interval.tick().await; +/// interval.tick().await; +/// +/// // approximately 30ms have elapsed. +/// } +/// ``` +pub fn interval(period: Duration) -> Interval { + assert!(period > Duration::new(0, 0), "`period` must be non-zero."); + + interval_at(Instant::now(), period) +} + +/// Creates new `Interval` that yields with interval of `period` with the +/// first tick completing at `at`. +/// +/// An interval will tick indefinitely. At any time, the `Interval` value can be +/// dropped. This cancels the interval. +/// +/// # Panics +/// +/// This function panics if `period` is zero. +/// +/// # Examples +/// +/// ``` +/// use tokio::time::{interval_at, Duration, Instant}; +/// +/// #[tokio::main] +/// async fn main() { +/// let start = Instant::now() + Duration::from_millis(50); +/// let mut interval = interval_at(start, Duration::from_millis(10)); +/// +/// interval.tick().await; +/// interval.tick().await; +/// interval.tick().await; +/// +/// // approximately 70ms have elapsed. +/// } +/// ``` +pub fn interval_at(start: Instant, period: Duration) -> Interval { + assert!(period > Duration::new(0, 0), "`period` must be non-zero."); + + Interval { + delay: delay_until(start), + period, + } +} + +/// Stream returned by [`instant`](instant) and [`instant_at`](instant_at). #[derive(Debug)] pub struct Interval { /// Future that completes the next time the `Interval` yields a value. delay: Delay, /// The duration between values yielded by `Interval`. - duration: Duration, + period: Duration, } impl Interval { - /// Create a new `Interval` that starts at `at` and yields every `duration` - /// interval after that. - /// - /// Note that when it starts, it produces item too. - /// - /// The `duration` argument must be a non-zero duration. - /// - /// # Panics - /// - /// This function panics if `duration` is zero. - pub fn new(at: Instant, duration: Duration) -> Interval { - assert!( - duration > Duration::new(0, 0), - "`duration` must be non-zero." - ); - - Interval::new_with_delay(Delay::new(at), duration) - } - - /// Creates new `Interval` that yields with interval of `duration`. - /// - /// The function is shortcut for `Interval::new(tokio::time::clock::now() + duration, duration)`. - /// - /// The `duration` argument must be a non-zero duration. - /// - /// # Panics - /// - /// This function panics if `duration` is zero. - pub fn new_interval(duration: Duration) -> Interval { - Interval::new(Instant::now() + duration, duration) - } - - pub(crate) fn new_with_delay(delay: Delay, duration: Duration) -> Interval { - Interval { delay, duration } - } - - #[doc(hidden)] // TODO: remove - pub fn poll_next(&mut self, cx: &mut task::Context<'_>) -> Poll> { + #[doc(hidden)] // TODO: document + pub fn poll_tick(&mut self, cx: &mut task::Context<'_>) -> Poll { // Wait for the delay to be done ready!(Pin::new(&mut self.delay).poll(cx)); @@ -63,11 +96,11 @@ impl Interval { // The next interval value is `duration` after the one that just // yielded. - let next = now + self.duration; + let next = now + self.period; self.delay.reset(next); // Return the current instant - Poll::Ready(Some(now)) + Poll::Ready(now) } /// Completes when the next instant in the interval has been reached. @@ -75,37 +108,23 @@ impl Interval { /// # Examples /// /// ``` - /// use tokio::time::Interval; + /// use tokio::time; /// /// use std::time::Duration; /// /// #[tokio::main] /// async fn main() { - /// let mut interval = Interval::new_interval(Duration::from_millis(10)); + /// let mut interval = time::interval(Duration::from_millis(10)); /// - /// interval.next().await; - /// interval.next().await; - /// interval.next().await; + /// interval.tick().await; + /// interval.tick().await; + /// interval.tick().await; /// /// // approximately 30ms have elapsed. /// } /// ``` #[allow(clippy::should_implement_trait)] // TODO: rename (tokio-rs/tokio#1261) - pub async fn next(&mut self) -> Option { - poll_fn(|cx| self.poll_next(cx)).await - } -} - -impl futures_core::FusedStream for Interval { - fn is_terminated(&self) -> bool { - false - } -} - -impl futures_core::Stream for Interval { - type Item = Instant; - - fn poll_next(self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> Poll> { - Interval::poll_next(self.get_mut(), cx) + pub async fn tick(&mut self) -> Instant { + poll_fn(|cx| self.poll_tick(cx)).await } } diff --git a/tokio/src/time/mod.rs b/tokio/src/time/mod.rs index ddcf9b23df6..50b6a8feabf 100644 --- a/tokio/src/time/mod.rs +++ b/tokio/src/time/mod.rs @@ -45,17 +45,14 @@ //! included in the prelude. //! //! ``` -//! use tokio::prelude::*; -//! use std::time::Duration; +//! use tokio::time::{timeout, Duration}; //! //! async fn long_future() { //! // do work here //! } //! //! # async fn dox() { -//! let res = long_future() -//! .timeout(Duration::from_secs(1)) -//! .await; +//! let res = timeout(Duration::from_secs(1), long_future()).await; //! //! if res.is_err() { //! println!("operation timed out"); @@ -77,10 +74,10 @@ pub use clock::{advance, pause, resume}; pub mod delay_queue; #[doc(inline)] -pub use self::delay_queue::DelayQueue; +pub use delay_queue::DelayQueue; mod delay; -pub use self::delay::Delay; +pub use delay::{delay_for, delay_until, Delay}; pub(crate) mod driver; @@ -91,13 +88,11 @@ mod instant; pub use self::instant::Instant; mod interval; -pub use interval::Interval; - -pub mod throttle; +pub use interval::{interval, interval_at, Interval}; -pub mod timeout; +mod timeout; #[doc(inline)] -pub use timeout::Timeout; +pub use timeout::{timeout, timeout_at, Timeout}; mod wheel; @@ -105,20 +100,9 @@ mod wheel; #[cfg(not(loom))] mod tests; +// Re-export for convenience pub use std::time::Duration; -/// Create a Future that completes at `deadline`. -pub fn delay(deadline: Instant) -> Delay { - Delay::new(deadline) -} - -/// Create a Future that completes in `duration` from now. -/// -/// Equivalent to `delay(tokio::time::clock::now() + duration)`. Analogous to `std::thread::sleep`. -pub fn delay_for(duration: Duration) -> Delay { - delay(clock::now() + duration) -} - // ===== Internal utils ===== enum Round { diff --git a/tokio/src/time/tests/test_delay.rs b/tokio/src/time/tests/test_delay.rs index 797d2625c84..8b52e0a323a 100644 --- a/tokio/src/time/tests/test_delay.rs +++ b/tokio/src/time/tests/test_delay.rs @@ -1,7 +1,7 @@ #![warn(rust_2018_idioms)] use crate::time::tests::mock_clock::mock; -use crate::time::{delay, Duration, Instant}; +use crate::time::{delay_until, Duration, Instant}; use tokio_test::task; use tokio_test::{assert_pending, assert_ready}; @@ -9,7 +9,7 @@ use tokio_test::{assert_pending, assert_ready}; fn immediate_delay() { mock(|clock| { // Create `Delay` that elapsed immediately. - let mut fut = task::spawn(delay(clock.now())); + let mut fut = task::spawn(delay_until(clock.now())); // Ready! assert_ready!(fut.poll()); @@ -27,7 +27,7 @@ fn delayed_delay_level_0() { for &i in &[1, 10, 60] { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(i))); + let mut fut = task::spawn(delay_until(clock.now() + ms(i))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -46,7 +46,7 @@ fn sub_ms_delayed_delay() { for _ in 0..5 { let deadline = clock.now() + Duration::from_millis(1) + Duration::new(0, 1); - let mut fut = task::spawn(delay(deadline)); + let mut fut = task::spawn(delay_until(deadline)); assert_pending!(fut.poll()); @@ -66,7 +66,7 @@ fn delayed_delay_wrapping_level_0() { clock.turn_for(ms(5)); assert_eq!(clock.advanced(), ms(5)); - let mut fut = task::spawn(delay(clock.now() + ms(60))); + let mut fut = task::spawn(delay_until(clock.now() + ms(60))); assert_pending!(fut.poll()); @@ -85,14 +85,14 @@ fn delayed_delay_wrapping_level_0() { fn timer_wrapping_with_higher_levels() { mock(|clock| { // Set delay to hit level 1 - let mut s1 = task::spawn(delay(clock.now() + ms(64))); + let mut s1 = task::spawn(delay_until(clock.now() + ms(64))); assert_pending!(s1.poll()); // Turn a bit clock.turn_for(ms(5)); // Set timeout such that it will hit level 0, but wrap - let mut s2 = task::spawn(delay(clock.now() + ms(60))); + let mut s2 = task::spawn(delay_until(clock.now() + ms(60))); assert_pending!(s2.poll()); // This should result in s1 firing @@ -113,7 +113,7 @@ fn timer_wrapping_with_higher_levels() { fn delay_with_deadline_in_past() { mock(|clock| { // Create `Delay` that elapsed immediately. - let mut fut = task::spawn(delay(clock.now() - ms(100))); + let mut fut = task::spawn(delay_until(clock.now() - ms(100))); // Even though the delay expires in the past, it is not ready yet // because the timer must observe it. @@ -131,7 +131,7 @@ fn delay_with_deadline_in_past() { fn delayed_delay_level_1() { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(234))); + let mut fut = task::spawn(delay_until(clock.now() + ms(234))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -153,7 +153,7 @@ fn delayed_delay_level_1() { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(234))); + let mut fut = task::spawn(delay_until(clock.now() + ms(234))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -186,7 +186,7 @@ fn creating_delay_outside_of_context() { // This creates a delay outside of the context of a mock timer. This tests // that it will still expire. - let mut fut = task::spawn(delay(now + ms(500))); + let mut fut = task::spawn(delay_until(now + ms(500))); mock(|clock| { // This registers the delay with the timer @@ -209,8 +209,8 @@ fn creating_delay_outside_of_context() { #[test] fn concurrently_set_two_timers_second_one_shorter() { mock(|clock| { - let mut fut1 = task::spawn(delay(clock.now() + ms(500))); - let mut fut2 = task::spawn(delay(clock.now() + ms(200))); + let mut fut1 = task::spawn(delay_until(clock.now() + ms(500))); + let mut fut2 = task::spawn(delay_until(clock.now() + ms(200))); // The delay has not elapsed assert_pending!(fut1.poll()); @@ -245,7 +245,7 @@ fn concurrently_set_two_timers_second_one_shorter() { fn short_delay() { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(1))); + let mut fut = task::spawn(delay_until(clock.now() + ms(1))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -262,12 +262,12 @@ fn short_delay() { } #[test] -fn sorta_long_delay() { +fn sorta_long_delay_until() { const MIN_5: u64 = 5 * 60 * 1000; mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(MIN_5))); + let mut fut = task::spawn(delay_until(clock.now() + ms(MIN_5))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -295,7 +295,7 @@ fn very_long_delay() { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(MO_5))); + let mut fut = task::spawn(delay_until(clock.now() + ms(MO_5))); // The delay has not elapsed. assert_pending!(fut.poll()); @@ -332,7 +332,7 @@ fn greater_than_max() { mock(|clock| { // Create a `Delay` that elapses in the future - let mut fut = task::spawn(delay(clock.now() + ms(YR_5))); + let mut fut = task::spawn(delay_until(clock.now() + ms(YR_5))); assert_pending!(fut.poll()); @@ -346,9 +346,9 @@ fn greater_than_max() { #[test] fn unpark_is_delayed() { mock(|clock| { - let mut fut1 = task::spawn(delay(clock.now() + ms(100))); - let mut fut2 = task::spawn(delay(clock.now() + ms(101))); - let mut fut3 = task::spawn(delay(clock.now() + ms(200))); + let mut fut1 = task::spawn(delay_until(clock.now() + ms(100))); + let mut fut2 = task::spawn(delay_until(clock.now() + ms(101))); + let mut fut3 = task::spawn(delay_until(clock.now() + ms(200))); assert_pending!(fut1.poll()); assert_pending!(fut2.poll()); @@ -374,7 +374,7 @@ fn set_timeout_at_deadline_greater_than_max_timer() { clock.turn_for(ms(YR_1)); } - let mut fut = task::spawn(delay(clock.now() + ms(1))); + let mut fut = task::spawn(delay_until(clock.now() + ms(1))); assert_pending!(fut.poll()); clock.turn_for(ms(1000)); @@ -387,7 +387,7 @@ fn set_timeout_at_deadline_greater_than_max_timer() { #[test] fn reset_future_delay_before_fire() { mock(|clock| { - let mut fut = task::spawn(delay(clock.now() + ms(100))); + let mut fut = task::spawn(delay_until(clock.now() + ms(100))); assert_pending!(fut.poll()); @@ -408,7 +408,7 @@ fn reset_future_delay_before_fire() { #[test] fn reset_past_delay_before_turn() { mock(|clock| { - let mut fut = task::spawn(delay(clock.now() + ms(100))); + let mut fut = task::spawn(delay_until(clock.now() + ms(100))); assert_pending!(fut.poll()); @@ -429,7 +429,7 @@ fn reset_past_delay_before_turn() { #[test] fn reset_past_delay_before_fire() { mock(|clock| { - let mut fut = task::spawn(delay(clock.now() + ms(100))); + let mut fut = task::spawn(delay_until(clock.now() + ms(100))); assert_pending!(fut.poll()); clock.turn_for(ms(10)); @@ -452,7 +452,7 @@ fn reset_past_delay_before_fire() { #[test] fn reset_future_delay_after_fire() { mock(|clock| { - let mut fut = task::spawn(delay(clock.now() + ms(100))); + let mut fut = task::spawn(delay_until(clock.now() + ms(100))); assert_pending!(fut.poll()); diff --git a/tokio/src/time/tests/test_queue.rs b/tokio/src/time/tests/test_queue.rs index cfcbfc1a86f..0902ec4a483 100644 --- a/tokio/src/time/tests/test_queue.rs +++ b/tokio/src/time/tests/test_queue.rs @@ -6,7 +6,7 @@ use tokio_test::{assert_ok, assert_pending, assert_ready, task}; macro_rules! poll { ($queue:ident) => { - $queue.enter(|cx, mut queue| queue.poll_next(cx)) + $queue.enter(|cx, mut queue| queue.poll_expired(cx)) }; } diff --git a/tokio/src/time/throttle.rs b/tokio/src/time/throttle.rs index 07215cd166a..f81f0fcf67e 100644 --- a/tokio/src/time/throttle.rs +++ b/tokio/src/time/throttle.rs @@ -1,16 +1,11 @@ //! Slow down a stream by enforcing a delay between items. -use crate::time::{Delay, Instant}; - -use futures_core::ready; -use futures_core::Stream; -use std::{ - future::Future, - marker::Unpin, - pin::Pin, - task::{self, Poll}, - time::Duration, -}; +use crate::time::{Delay, Duration, Instant}; + +use std::future::Future; +use std::marker::Unpin; +use std::pin::Pin; +use std::task::{self, Poll}; /// Slow down a stream by enforcing a delay between items. #[derive(Debug)] diff --git a/tokio/src/time/timeout.rs b/tokio/src/time/timeout.rs index 2cc3508204d..3a66a8268a5 100644 --- a/tokio/src/time/timeout.rs +++ b/tokio/src/time/timeout.rs @@ -1,73 +1,104 @@ -//! Allows a future or stream to execute for a maximum amount of time. +//! Allows a future to execute for a maximum amount of time. //! //! See [`Timeout`] documentation for more details. //! //! [`Timeout`]: struct.Timeout.html -use crate::time::clock::now; -use crate::time::{Delay, Duration, Instant}; +use crate::time::{delay_until, Delay, Duration, Instant}; -use futures_core::ready; use std::fmt; use std::future::Future; use std::pin::Pin; use std::task::{self, Poll}; -/// Allows a `Future` or `Stream` to execute for a limited amount of time. +/// Require a `Future` to complete before the specified duration has elapsed. /// -/// If the future or stream completes before the timeout has expired, then -/// `Timeout` returns the completed value. Otherwise, `Timeout` returns an -/// [`Error`]. +/// If the future completes before the duration has elapsed, then the completed +/// value is returned. Otherwise, an error is returned. /// -/// # Futures and Streams +/// # Cancelation /// -/// The exact behavor depends on if the inner value is a `Future` or a `Stream`. -/// In the case of a `Future`, `Timeout` will require the future to complete by -/// a fixed deadline. In the case of a `Stream`, `Timeout` will allow each item -/// to take the entire timeout before returning an error. +/// Cancelling a timeout is done by dropping the future. No additional cleanup +/// or other work is required. /// -/// In order to set an upper bound on the processing of the *entire* stream, -/// then a timeout should be set on the future that processes the stream. For -/// example: +/// The original future may be obtained by calling [`Timeout::into_inner`]. This +/// consumes the `Timeout`. /// -/// ```rust,no_run -/// use tokio::prelude::*; -/// use tokio::sync::mpsc; +/// # Examples /// -/// use std::thread; -/// use std::time::Duration; +/// Create a new `Timeout` set to expire in 10 milliseconds. /// -/// # async fn dox() -> Result<(), Box> { -/// let (mut tx, rx) = mpsc::unbounded_channel(); +/// ```rust +/// use tokio::time::timeout; +/// use tokio::sync::oneshot; /// -/// thread::spawn(move || { -/// tx.try_send(()).unwrap(); -/// thread::sleep(Duration::from_millis(10)); -/// tx.try_send(()).unwrap(); -/// }); +/// use std::time::Duration; /// -/// let process = rx.for_each(|item| { -/// // do something with `item` -/// # drop(item); -/// # tokio::future::ready(()) -/// }); +/// # async fn dox() { +/// let (tx, rx) = oneshot::channel(); +/// # tx.send(()).unwrap(); /// /// // Wrap the future with a `Timeout` set to expire in 10 milliseconds. -/// process.timeout(Duration::from_millis(10)).await?; -/// # Ok(()) +/// if let Err(_) = timeout(Duration::from_millis(10), rx).await { +/// println!("did not receive value within 10 ms"); +/// } /// # } /// ``` +pub fn timeout(duration: Duration, future: T) -> Timeout +where + T: Future, +{ + let delay = Delay::new_timeout(Instant::now() + duration, duration); + Timeout::new_with_delay(future, delay) +} + +/// Require a `Future` to complete before the specified instant in time. +/// +/// If the future completes before the instant is reached, then the completed +/// value is returned. Otherwise, an error is returned. /// /// # Cancelation /// -/// Cancelling a `Timeout` is done by dropping the value. No additional cleanup +/// Cancelling a timeout is done by dropping the future. No additional cleanup /// or other work is required. /// -/// The original future or stream may be obtained by calling [`Timeout::into_inner`]. This +/// The original future may be obtained by calling [`Timeout::into_inner`]. This /// consumes the `Timeout`. /// -/// [`Error`]: struct.Error.html -/// [`Timeout::into_inner`]: struct.Timeout.html#method.into_iter +/// # Examples +/// +/// Create a new `Timeout` set to expire in 10 milliseconds. +/// +/// ```rust +/// use tokio::time::{Instant, timeout_at}; +/// use tokio::sync::oneshot; +/// +/// use std::time::Duration; +/// +/// # async fn dox() { +/// let (tx, rx) = oneshot::channel(); +/// # tx.send(()).unwrap(); +/// +/// // Wrap the future with a `Timeout` set to expire 10 milliseconds into the +/// // future. +/// if let Err(_) = timeout_at(Instant::now() + Duration::from_millis(10), rx).await { +/// println!("did not receive value within 10 ms"); +/// } +/// # } +/// ``` +pub fn timeout_at(deadline: Instant, future: T) -> Timeout +where + T: Future, +{ + let delay = delay_until(deadline); + + Timeout { + value: future, + delay, + } +} + +/// Future returned by [`timeout`](timeout) and [`timeout_at`](timeout_at). #[must_use = "futures do nothing unless you `.await` or poll them"] #[derive(Debug)] pub struct Timeout { @@ -80,39 +111,6 @@ pub struct Timeout { pub struct Elapsed(()); impl Timeout { - /// Create a new `Timeout` that allows `value` to execute for a duration of - /// at most `timeout`. - /// - /// The exact behavior depends on if `value` is a `Future` or a `Stream`. - /// - /// See [type] level documentation for more details. - /// - /// [type]: # - /// - /// # Examples - /// - /// Create a new `Timeout` set to expire in 10 milliseconds. - /// - /// ```rust - /// use tokio::time::Timeout; - /// use tokio::sync::oneshot; - /// - /// use std::time::Duration; - /// - /// # async fn dox() -> Result<(), Box> { - /// let (tx, rx) = oneshot::channel(); - /// # tx.send(()).unwrap(); - /// - /// // Wrap the future with a `Timeout` set to expire in 10 milliseconds. - /// Timeout::new(rx, Duration::from_millis(10)).await??; - /// # Ok(()) - /// # } - /// ``` - pub fn new(value: T, timeout: Duration) -> Timeout { - let delay = Delay::new_timeout(now() + timeout, timeout); - Timeout::new_with_delay(value, delay) - } - pub(crate) fn new_with_delay(value: T, delay: Delay) -> Timeout { Timeout { value, delay } } @@ -133,24 +131,6 @@ impl Timeout { } } -impl Timeout { - /// Create a new `Timeout` that completes when `future` completes or when - /// `deadline` is reached. - /// - /// This function differs from `new` in that: - /// - /// * It only accepts `Future` arguments. - /// * It sets an explicit `Instant` at which the timeout expires. - pub fn new_at(future: T, deadline: Instant) -> Timeout { - let delay = Delay::new(deadline); - - Timeout { - value: future, - delay, - } - } -} - impl Future for Timeout where T: Future, @@ -179,41 +159,6 @@ where } } -impl futures_core::Stream for Timeout -where - T: futures_core::Stream, -{ - type Item = Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut task::Context<'_>) -> Poll> { - // Safety: T might be !Unpin, but we never move neither `value` - // nor `delay`. - // - // ... X_X - unsafe { - // First, try polling the future - let v = self - .as_mut() - .map_unchecked_mut(|me| &mut me.value) - .poll_next(cx); - - if let Poll::Ready(v) = v { - if v.is_some() { - self.as_mut().get_unchecked_mut().delay.reset_timeout(); - } - return Poll::Ready(v.map(Ok)); - } - - // Now check the timer - ready!(self.as_mut().map_unchecked_mut(|me| &mut me.delay).poll(cx)); - - // if delay was ready, timeout elapsed! - self.as_mut().get_unchecked_mut().delay.reset_timeout(); - Poll::Ready(Some(Err(Elapsed(())))) - } - } -} - // ===== impl Elapsed ===== impl fmt::Display for Elapsed { diff --git a/tokio/tests/fs_dir.rs b/tokio/tests/fs_dir.rs index 7ef2db6f164..8f8f2717a46 100644 --- a/tokio/tests/fs_dir.rs +++ b/tokio/tests/fs_dir.rs @@ -3,8 +3,6 @@ use tokio::fs; use tokio_test::assert_ok; -use futures_util::future; -use futures_util::stream::TryStreamExt; use std::sync::{Arc, Mutex}; use tempfile::tempdir; @@ -55,15 +53,12 @@ async fn read() { let f = files.clone(); let p = p.to_path_buf(); - let read_dir_fut = fs::read_dir(p).await.unwrap(); - read_dir_fut - .try_for_each(move |e| { - let s = e.file_name().to_str().unwrap().to_string(); - f.lock().unwrap().push(s); - future::ok(()) - }) - .await - .unwrap(); + let mut entries = fs::read_dir(p).await.unwrap(); + + while let Some(e) = assert_ok!(entries.next_entry().await) { + let s = e.file_name().to_str().unwrap().to_string(); + f.lock().unwrap().push(s); + } let mut files = files.lock().unwrap(); files.sort(); // because the order is not guaranteed diff --git a/tokio/tests/fs_file_mocked.rs b/tokio/tests/fs_file_mocked.rs index 4697814cee6..d2eaadde93b 100644 --- a/tokio/tests/fs_file_mocked.rs +++ b/tokio/tests/fs_file_mocked.rs @@ -1,5 +1,16 @@ #![warn(rust_2018_idioms)] +macro_rules! ready { + ($e:expr $(,)?) => { + match $e { + std::task::Poll::Ready(t) => t, + std::task::Poll::Pending => return std::task::Poll::Pending, + } + }; +} + +use futures::future; + // Load source #[allow(warnings)] #[path = "../src/fs/file.rs"] diff --git a/tokio/tests/io_lines.rs b/tokio/tests/io_lines.rs index e85fbff7a7d..e479ee98888 100644 --- a/tokio/tests/io_lines.rs +++ b/tokio/tests/io_lines.rs @@ -3,18 +3,16 @@ use tokio::io::AsyncBufReadExt; use tokio_test::assert_ok; -use futures_util::StreamExt; - #[tokio::test] async fn lines() { let rd: &[u8] = b"hello\r\nworld\n\n"; let mut st = rd.lines(); - let b = assert_ok!(st.next().await.unwrap()); + let b = assert_ok!(st.next_line().await).unwrap(); assert_eq!(b, "hello"); - let b = assert_ok!(st.next().await.unwrap()); + let b = assert_ok!(st.next_line().await).unwrap(); assert_eq!(b, "world"); - let b = assert_ok!(st.next().await.unwrap()); + let b = assert_ok!(st.next_line().await).unwrap(); assert_eq!(b, ""); - assert!(st.next().await.is_none()); + assert!(assert_ok!(st.next_line().await).is_none()); } diff --git a/tokio/tests/net_driver.rs b/tokio/tests/net_driver.rs index 5285fd13572..5baa4eda849 100644 --- a/tokio/tests/net_driver.rs +++ b/tokio/tests/net_driver.rs @@ -4,7 +4,7 @@ use tokio::net::driver::Reactor; use tokio::net::TcpListener; use tokio_test::{assert_ok, assert_pending}; -use futures_util::task::{waker_ref, ArcWake}; +use futures::task::{waker_ref, ArcWake}; use std::future::Future; use std::net::TcpStream; use std::pin::Pin; diff --git a/tokio/tests/process_issue_42.rs b/tokio/tests/process_issue_42.rs index 9de9d0bf469..21651ac8e7a 100644 --- a/tokio/tests/process_issue_42.rs +++ b/tokio/tests/process_issue_42.rs @@ -5,8 +5,8 @@ use tokio::process::Command; use tokio::runtime; -use futures_util::future::FutureExt; -use futures_util::stream::FuturesOrdered; +use futures::future::FutureExt; +use futures::stream::FuturesOrdered; use std::process::Stdio; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index 73982ced3f8..e8d745e98b4 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -41,7 +41,7 @@ rt_test! { use tokio::time; use tokio_test::{assert_err, assert_ok}; - use futures_util::future::poll_fn; + use futures::future::poll_fn; use std::future::Future; use std::pin::Pin; use std::sync::{mpsc, Arc}; diff --git a/tokio/tests/signal_ctrl_c.rs b/tokio/tests/signal_ctrl_c.rs index ea4efaa2158..13eeaa81cfa 100644 --- a/tokio/tests/signal_ctrl_c.rs +++ b/tokio/tests/signal_ctrl_c.rs @@ -6,13 +6,13 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal; use tokio::sync::oneshot; +use tokio_test::assert_ok; #[tokio::test] async fn ctrl_c() { - let ctrl_c = signal::ctrl_c().expect("failed to init ctrl_c"); + let ctrl_c = signal::ctrl_c(); let (fire, wait) = oneshot::channel(); @@ -24,5 +24,6 @@ async fn ctrl_c() { }); let _ = fire.send(()); - let _ = ctrl_c.into_future().await; + + assert_ok!(ctrl_c.await); } diff --git a/tokio/tests/signal_drop_recv.rs b/tokio/tests/signal_drop_recv.rs index 2a5c047f044..06dffe1257b 100644 --- a/tokio/tests/signal_drop_recv.rs +++ b/tokio/tests/signal_drop_recv.rs @@ -6,7 +6,6 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal::unix::{signal, SignalKind}; #[tokio::test] @@ -16,7 +15,7 @@ async fn drop_then_get_a_signal() { drop(sig); send_signal(libc::SIGUSR1); - let sig = signal(kind).expect("failed to create second signal"); + let mut sig = signal(kind).expect("failed to create second signal"); - let _ = sig.into_future().await; + let _ = sig.recv().await; } diff --git a/tokio/tests/signal_drop_rt.rs b/tokio/tests/signal_drop_rt.rs index 1af8c0a7b8c..7387e312a20 100644 --- a/tokio/tests/signal_drop_rt.rs +++ b/tokio/tests/signal_drop_rt.rs @@ -6,7 +6,6 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::runtime::Runtime; use tokio::signal::unix::{signal, SignalKind}; @@ -25,7 +24,7 @@ fn dropping_loops_does_not_cause_starvation() { send_signal(libc::SIGUSR1); first_rt - .block_on(first_signal.next()) + .block_on(first_signal.recv()) .expect("failed to await first signal"); drop(first_rt); @@ -33,7 +32,7 @@ fn dropping_loops_does_not_cause_starvation() { send_signal(libc::SIGUSR1); - second_rt.block_on(second_signal.next()); + second_rt.block_on(second_signal.recv()); } fn rt() -> Runtime { diff --git a/tokio/tests/signal_drop_signal.rs b/tokio/tests/signal_drop_signal.rs index 3cf5611f87b..b5bc7dd8210 100644 --- a/tokio/tests/signal_drop_signal.rs +++ b/tokio/tests/signal_drop_signal.rs @@ -6,7 +6,6 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal::unix::{signal, SignalKind}; #[tokio::test] @@ -15,12 +14,12 @@ async fn dropping_signal_does_not_deregister_any_other_instances() { // Signals should not starve based on ordering let first_duplicate_signal = signal(kind).expect("failed to register first duplicate signal"); - let sig = signal(kind).expect("failed to register signal"); + let mut sig = signal(kind).expect("failed to register signal"); let second_duplicate_signal = signal(kind).expect("failed to register second duplicate signal"); drop(first_duplicate_signal); drop(second_duplicate_signal); send_signal(libc::SIGUSR1); - let _ = sig.into_future().await; + let _ = sig.recv().await; } diff --git a/tokio/tests/signal_multi_rt.rs b/tokio/tests/signal_multi_rt.rs index 6a16dd88454..fb5449f0f40 100644 --- a/tokio/tests/signal_multi_rt.rs +++ b/tokio/tests/signal_multi_rt.rs @@ -6,7 +6,6 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::runtime::Runtime; use tokio::signal::unix::{signal, SignalKind}; @@ -26,9 +25,9 @@ fn multi_loop() { thread::spawn(move || { let mut rt = rt(); let _ = rt.block_on(async { - let signal = signal(SignalKind::hangup()).unwrap(); + let mut signal = signal(SignalKind::hangup()).unwrap(); sender.send(()).unwrap(); - signal.into_future().await + signal.recv().await }); }) }) diff --git a/tokio/tests/signal_notify_both.rs b/tokio/tests/signal_notify_both.rs index 00385478660..7d830686307 100644 --- a/tokio/tests/signal_notify_both.rs +++ b/tokio/tests/signal_notify_both.rs @@ -6,18 +6,17 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal::unix::{signal, SignalKind}; -use futures::future; - #[tokio::test] async fn notify_both() { let kind = SignalKind::user_defined2(); - let signal1 = signal(kind).expect("failed to create signal1"); - let signal2 = signal(kind).expect("failed to create signal2"); + let mut signal1 = signal(kind).expect("failed to create signal1"); + let mut signal2 = signal(kind).expect("failed to create signal2"); send_signal(libc::SIGUSR2); - let _ = future::join(signal1.into_future(), signal2.into_future()).await; + + signal1.recv().await; + signal2.recv().await; } diff --git a/tokio/tests/signal_twice.rs b/tokio/tests/signal_twice.rs index d8e0facc6c7..171d18e6050 100644 --- a/tokio/tests/signal_twice.rs +++ b/tokio/tests/signal_twice.rs @@ -6,7 +6,6 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal::unix::{signal, SignalKind}; #[tokio::test] @@ -17,9 +16,6 @@ async fn twice() { for _ in 0..2 { send_signal(libc::SIGUSR1); - let (item, sig_next) = sig.into_future().await; - assert_eq!(item, Some(())); - - sig = sig_next; + assert!(sig.recv().await.is_some()); } } diff --git a/tokio/tests/signal_usr1.rs b/tokio/tests/signal_usr1.rs index 9b6a0dec4c0..95fc6c10478 100644 --- a/tokio/tests/signal_usr1.rs +++ b/tokio/tests/signal_usr1.rs @@ -6,18 +6,17 @@ mod support { } use support::signal::send_signal; -use tokio::prelude::*; use tokio::signal::unix::{signal, SignalKind}; use tokio_test::assert_ok; #[tokio::test] async fn signal_usr1() { - let signal = assert_ok!( + let mut signal = assert_ok!( signal(SignalKind::user_defined1()), "failed to create signal" ); send_signal(libc::SIGUSR1); - let _ = signal.into_future().await; + signal.recv().await; } diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index f724c564685..3d931aec89b 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -51,36 +51,6 @@ async fn async_send_recv_with_buffer() { assert_eq!(None, rx.recv().await); } -#[test] -fn send_sink_recv_with_buffer() { - use futures_core::Stream; - use futures_sink::Sink; - - let (tx, rx) = mpsc::channel::(16); - - task::spawn(tx).enter(|cx, mut tx| { - assert_ready_ok!(tx.as_mut().poll_ready(cx)); - assert_ok!(tx.as_mut().start_send(1)); - - assert_ready_ok!(tx.as_mut().poll_ready(cx)); - assert_ok!(tx.as_mut().start_send(2)); - - assert_ready_ok!(tx.as_mut().poll_flush(cx)); - assert_ready_ok!(tx.as_mut().poll_close(cx)); - }); - - task::spawn(rx).enter(|cx, mut rx| { - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert_eq!(val, Some(1)); - - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert_eq!(val, Some(2)); - - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert!(val.is_none()); - }); -} - #[test] fn start_send_past_cap() { let mut t1 = task::spawn(()); @@ -156,43 +126,6 @@ async fn async_send_recv_unbounded() { assert_eq!(None, rx.recv().await); } -#[test] -fn sink_send_recv_unbounded() { - use futures_core::Stream; - use futures_sink::Sink; - use futures_util::pin_mut; - - let mut t1 = task::spawn(()); - - let (tx, rx) = mpsc::unbounded_channel::(); - - t1.enter(|cx, _| { - pin_mut!(tx); - - assert_ready_ok!(tx.as_mut().poll_ready(cx)); - assert_ok!(tx.as_mut().start_send(1)); - - assert_ready_ok!(tx.as_mut().poll_ready(cx)); - assert_ok!(tx.as_mut().start_send(2)); - - assert_ready_ok!(tx.as_mut().poll_flush(cx)); - assert_ready_ok!(tx.as_mut().poll_close(cx)); - }); - - t1.enter(|cx, _| { - pin_mut!(rx); - - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert_eq!(val, Some(1)); - - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert_eq!(val, Some(2)); - - let val = assert_ready!(rx.as_mut().poll_next(cx)); - assert!(val.is_none()); - }); -} - #[test] fn no_t_bounds_buffer() { struct NoImpls; diff --git a/tokio/tests/sync_watch.rs b/tokio/tests/sync_watch.rs index 4d73bc81ff6..6c24ab01093 100644 --- a/tokio/tests/sync_watch.rs +++ b/tokio/tests/sync_watch.rs @@ -4,41 +4,6 @@ use tokio::sync::watch; use tokio_test::task::spawn; use tokio_test::{assert_pending, assert_ready}; -#[test] -fn single_rx_recv_ref() { - let (tx, mut rx) = watch::channel("one"); - - { - let mut t = spawn(rx.recv_ref()); - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(*v, "one"); - } - - { - let mut t = spawn(rx.recv_ref()); - - assert_pending!(t.poll()); - - tx.broadcast("two").unwrap(); - - assert!(t.is_woken()); - - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(*v, "two"); - } - - { - let mut t = spawn(rx.recv_ref()); - - assert_pending!(t.poll()); - - drop(tx); - - let res = assert_ready!(t.poll()); - assert!(res.is_none()); - } -} - #[test] fn single_rx_recv() { let (tx, mut rx) = watch::channel("one"); @@ -74,63 +39,26 @@ fn single_rx_recv() { } } -#[test] -fn stream_impl() { - use tokio::prelude::*; - - let (tx, mut rx) = watch::channel("one"); - - { - let mut t = spawn(rx.next()); - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(v, "one"); - } - - { - let mut t = spawn(rx.next()); - - assert_pending!(t.poll()); - - tx.broadcast("two").unwrap(); - - assert!(t.is_woken()); - - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(v, "two"); - } - - { - let mut t = spawn(rx.next()); - - assert_pending!(t.poll()); - - drop(tx); - - let res = assert_ready!(t.poll()); - assert!(res.is_none()); - } -} - #[test] fn multi_rx() { let (tx, mut rx1) = watch::channel("one"); let mut rx2 = rx1.clone(); { - let mut t1 = spawn(rx1.recv_ref()); - let mut t2 = spawn(rx2.recv_ref()); + let mut t1 = spawn(rx1.recv()); + let mut t2 = spawn(rx2.recv()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "one"); + assert_eq!(res.unwrap(), "one"); let res = assert_ready!(t2.poll()); - assert_eq!(*res.unwrap(), "one"); + assert_eq!(res.unwrap(), "one"); } - let mut t2 = spawn(rx2.recv_ref()); + let mut t2 = spawn(rx2.recv()); { - let mut t1 = spawn(rx1.recv_ref()); + let mut t1 = spawn(rx1.recv()); assert_pending!(t1.poll()); assert_pending!(t2.poll()); @@ -141,11 +69,11 @@ fn multi_rx() { assert!(t2.is_woken()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "two"); + assert_eq!(res.unwrap(), "two"); } { - let mut t1 = spawn(rx1.recv_ref()); + let mut t1 = spawn(rx1.recv()); assert_pending!(t1.poll()); @@ -155,17 +83,17 @@ fn multi_rx() { assert!(t2.is_woken()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "three"); + assert_eq!(res.unwrap(), "three"); let res = assert_ready!(t2.poll()); - assert_eq!(*res.unwrap(), "three"); + assert_eq!(res.unwrap(), "three"); } drop(t2); { - let mut t1 = spawn(rx1.recv_ref()); - let mut t2 = spawn(rx2.recv_ref()); + let mut t1 = spawn(rx1.recv()); + let mut t2 = spawn(rx2.recv()); assert_pending!(t1.poll()); assert_pending!(t2.poll()); @@ -173,10 +101,10 @@ fn multi_rx() { tx.broadcast("four").unwrap(); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "four"); + assert_eq!(res.unwrap(), "four"); drop(t1); - let mut t1 = spawn(rx1.recv_ref()); + let mut t1 = spawn(rx1.recv()); assert_pending!(t1.poll()); drop(tx); @@ -186,10 +114,10 @@ fn multi_rx() { assert!(res.is_none()); let res = assert_ready!(t2.poll()); - assert_eq!(*res.unwrap(), "four"); + assert_eq!(res.unwrap(), "four"); drop(t2); - let mut t2 = spawn(rx2.recv_ref()); + let mut t2 = spawn(rx2.recv()); let res = assert_ready!(t2.poll()); assert!(res.is_none()); } @@ -203,13 +131,13 @@ fn rx_observes_final_value() { drop(tx); { - let mut t1 = spawn(rx.recv_ref()); + let mut t1 = spawn(rx.recv()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "one"); + assert_eq!(res.unwrap(), "one"); } { - let mut t1 = spawn(rx.recv_ref()); + let mut t1 = spawn(rx.recv()); let res = assert_ready!(t1.poll()); assert!(res.is_none()); } @@ -221,13 +149,13 @@ fn rx_observes_final_value() { tx.broadcast("two").unwrap(); { - let mut t1 = spawn(rx.recv_ref()); + let mut t1 = spawn(rx.recv()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "two"); + assert_eq!(res.unwrap(), "two"); } { - let mut t1 = spawn(rx.recv_ref()); + let mut t1 = spawn(rx.recv()); assert_pending!(t1.poll()); tx.broadcast("three").unwrap(); @@ -236,11 +164,11 @@ fn rx_observes_final_value() { assert!(t1.is_woken()); let res = assert_ready!(t1.poll()); - assert_eq!(*res.unwrap(), "three"); + assert_eq!(res.unwrap(), "three"); } { - let mut t1 = spawn(rx.recv_ref()); + let mut t1 = spawn(rx.recv()); let res = assert_ready!(t1.poll()); assert!(res.is_none()); } diff --git a/tokio/tests/time_interval.rs b/tokio/tests/time_interval.rs index c884ca8e9f5..a5f2910b6de 100644 --- a/tokio/tests/time_interval.rs +++ b/tokio/tests/time_interval.rs @@ -1,12 +1,14 @@ #![warn(rust_2018_idioms)] -use tokio::time::{self, Duration, Instant, Interval}; +use tokio::time::{self, Duration, Instant}; use tokio_test::{assert_pending, assert_ready_eq, task}; +use std::task::Poll; + #[tokio::test] #[should_panic] async fn interval_zero_duration() { - let _ = Interval::new(Instant::now(), ms(0)); + let _ = time::interval_at(Instant::now(), ms(0)); } #[tokio::test] @@ -18,26 +20,30 @@ async fn usage() { // TODO: Skip this time::advance(ms(1)).await; - let mut int = task::spawn(Interval::new(start, ms(300))); + let mut i = task::spawn(time::interval_at(start, ms(300))); - assert_ready_eq!(int.poll_next(), Some(start)); - assert_pending!(int.poll_next()); + assert_ready_eq!(poll_next(&mut i), start); + assert_pending!(poll_next(&mut i)); time::advance(ms(100)).await; - assert_pending!(int.poll_next()); + assert_pending!(poll_next(&mut i)); time::advance(ms(200)).await; - assert_ready_eq!(int.poll_next(), Some(start + ms(300))); - assert_pending!(int.poll_next()); + assert_ready_eq!(poll_next(&mut i), start + ms(300)); + assert_pending!(poll_next(&mut i)); time::advance(ms(400)).await; - assert_ready_eq!(int.poll_next(), Some(start + ms(600))); - assert_pending!(int.poll_next()); + assert_ready_eq!(poll_next(&mut i), start + ms(600)); + assert_pending!(poll_next(&mut i)); time::advance(ms(500)).await; - assert_ready_eq!(int.poll_next(), Some(start + ms(900))); - assert_ready_eq!(int.poll_next(), Some(start + ms(1200))); - assert_pending!(int.poll_next()); + assert_ready_eq!(poll_next(&mut i), start + ms(900)); + assert_ready_eq!(poll_next(&mut i), start + ms(1200)); + assert_pending!(poll_next(&mut i)); +} + +fn poll_next(interval: &mut task::Spawn) -> Poll { + interval.enter(|cx, mut interval| interval.poll_tick(cx)) } fn ms(n: u64) -> Duration { diff --git a/tokio/tests/time_rt.rs b/tokio/tests/time_rt.rs index ecce72d6f0e..2576db4a6dc 100644 --- a/tokio/tests/time_rt.rs +++ b/tokio/tests/time_rt.rs @@ -1,6 +1,5 @@ #![warn(rust_2018_idioms)] -use tokio::prelude::*; use tokio::time::*; use std::sync::mpsc; @@ -15,7 +14,7 @@ fn timer_with_threaded_runtime() { rt.spawn(async move { let when = Instant::now() + Duration::from_millis(100); - delay(when).await; + delay_until(when).await; assert!(Instant::now() >= when); tx.send(()).unwrap(); @@ -34,7 +33,7 @@ fn timer_with_current_thread_runtime() { rt.block_on(async move { let when = Instant::now() + Duration::from_millis(100); - tokio::time::delay(when).await; + delay_until(when).await; assert!(Instant::now() >= when); tx.send(()).unwrap(); @@ -68,14 +67,14 @@ async fn starving() { } let when = Instant::now() + Duration::from_millis(20); - let starve = Starve(delay(when), 0); + let starve = Starve(delay_until(when), 0); starve.await; assert!(Instant::now() >= when); } #[tokio::test] -async fn timeout() { +async fn timeout_value() { use tokio::sync::oneshot; let (_tx, rx) = oneshot::channel::<()>(); @@ -83,7 +82,7 @@ async fn timeout() { let now = Instant::now(); let dur = Duration::from_millis(20); - let res = rx.timeout(dur).await; + let res = timeout(dur, rx).await; assert!(res.is_err()); assert!(Instant::now() >= now + dur); } diff --git a/tokio/tests/time_throttle.rs b/tokio/tests/time_throttle.rs deleted file mode 100644 index 0431a4f2c54..00000000000 --- a/tokio/tests/time_throttle.rs +++ /dev/null @@ -1,68 +0,0 @@ -#![warn(rust_2018_idioms)] - -use tokio::sync::mpsc; -use tokio::time::throttle::Throttle; -use tokio::time::Instant; -use tokio_test::{assert_pending, assert_ready_eq}; - -use futures::future::poll_fn; -use futures::StreamExt; -use std::task::Poll; -use std::time::Duration; - -#[tokio::test] -async fn throttle() { - let (mut tx, rx) = mpsc::unbounded_channel(); - let mut stream = Throttle::new(rx, ms(1)); - - poll_fn(|cx| { - assert_pending!(stream.poll_next_unpin(cx)); - Poll::Ready(()) - }) - .await; - - for i in 0..3 { - tx.try_send(i).unwrap(); - } - - drop(tx); - - let mut now = Instant::now(); - - while let Some(_) = stream.next().await { - assert!(Instant::now() >= now); - now += ms(1); - } -} - -#[tokio::test] -async fn throttle_dur_0() { - let (mut tx, rx) = mpsc::unbounded_channel(); - let mut stream = Throttle::new(rx, ms(0)); - - poll_fn(|cx| { - assert_pending!(stream.poll_next_unpin(cx)); - - for i in 0..3 { - tx.try_send(i).unwrap(); - } - - Poll::Ready(()) - }) - .await; - - poll_fn(|cx| { - for i in 0..3 { - assert_ready_eq!(stream.poll_next_unpin(cx), Some(i), "i = {}", i); - } - - assert_pending!(stream.poll_next_unpin(cx)); - - Poll::Ready(()) - }) - .await; -} - -fn ms(n: u64) -> Duration { - Duration::from_millis(n) -} diff --git a/tokio/tests/time_timeout.rs b/tokio/tests/time_timeout.rs index fe3298af600..408f794e3bf 100644 --- a/tokio/tests/time_timeout.rs +++ b/tokio/tests/time_timeout.rs @@ -1,7 +1,7 @@ #![warn(rust_2018_idioms)] use tokio::sync::oneshot; -use tokio::time::{self, Instant, Timeout}; +use tokio::time::{self, timeout, timeout_at, Instant}; use tokio_test::*; use futures::future::pending; @@ -10,7 +10,7 @@ use std::time::Duration; #[tokio::test] async fn simultaneous_deadline_future_completion() { // Create a future that is immediately ready - let mut fut = task::spawn(Timeout::new_at(async {}, Instant::now())); + let mut fut = task::spawn(timeout_at(Instant::now(), async {})); // Ready! assert_ready_ok!(fut.poll()); @@ -19,7 +19,7 @@ async fn simultaneous_deadline_future_completion() { #[tokio::test] async fn completed_future_past_deadline() { // Wrap it with a deadline - let mut fut = task::spawn(Timeout::new_at(async {}, Instant::now() - ms(1000))); + let mut fut = task::spawn(timeout_at(Instant::now() - ms(1000), async {})); // Ready! assert_ready_ok!(fut.poll()); @@ -33,7 +33,7 @@ async fn future_and_deadline_in_future() { let (tx, rx) = oneshot::channel(); // Wrap it with a deadline - let mut fut = task::spawn(Timeout::new_at(rx, Instant::now() + ms(100))); + let mut fut = task::spawn(timeout_at(Instant::now() + ms(100), rx)); assert_pending!(fut.poll()); @@ -57,7 +57,7 @@ async fn future_and_timeout_in_future() { let (tx, rx) = oneshot::channel(); // Wrap it with a deadline - let mut fut = task::spawn(Timeout::new(rx, ms(100))); + let mut fut = task::spawn(timeout(ms(100), rx)); // Ready! assert_pending!(fut.poll()); @@ -80,7 +80,7 @@ async fn deadline_now_elapses() { time::pause(); // Wrap it with a deadline - let mut fut = task::spawn(Timeout::new_at(pending::<()>(), Instant::now())); + let mut fut = task::spawn(timeout_at(Instant::now(), pending::<()>())); // Factor in jitter // TODO: don't require this @@ -94,7 +94,7 @@ async fn deadline_future_elapses() { time::pause(); // Wrap it with a deadline - let mut fut = task::spawn(Timeout::new_at(pending::<()>(), Instant::now() + ms(300))); + let mut fut = task::spawn(timeout_at(Instant::now() + ms(300), pending::<()>())); assert_pending!(fut.poll()); @@ -104,63 +104,6 @@ async fn deadline_future_elapses() { assert_ready_err!(fut.poll()); } -#[tokio::test] -async fn stream_and_timeout_in_future() { - use tokio::sync::mpsc; - - time::pause(); - - // Not yet complete - let (mut tx, rx) = mpsc::unbounded_channel(); - - // Wrap it with a deadline - let mut stream = task::spawn(Timeout::new(rx, ms(100))); - - // Not ready - assert_pending!(stream.poll_next()); - - // Turn the timer, it runs for the elapsed time - time::advance(ms(90)).await; - - assert_pending!(stream.poll_next()); - - // Complete the future - tx.try_send(()).unwrap(); - - let item = assert_ready!(stream.poll_next()); - assert!(item.is_some()); -} - -#[tokio::test] -async fn idle_stream_timesout_periodically() { - use tokio::sync::mpsc; - - time::pause(); - - // Not yet complete - let (_tx, rx) = mpsc::unbounded_channel::<()>(); - - // Wrap it with a deadline - let mut stream = task::spawn(Timeout::new(rx, ms(100))); - - // Not ready - assert_pending!(stream.poll_next()); - - // Turn the timer, it runs for the elapsed time - time::advance(ms(101)).await; - - let v = assert_ready!(stream.poll_next()).unwrap(); - assert_err!(v); - - // Stream's timeout should reset - assert_pending!(stream.poll_next()); - - // Turn the timer, it runs for the elapsed time - time::advance(ms(101)).await; - let v = assert_ready!(stream.poll_next()).unwrap(); - assert_err!(v); -} - fn ms(n: u64) -> Duration { Duration::from_millis(n) } From 8cef1fa4f140391f531aa162d7d158da1b16e330 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 14 Nov 2019 22:35:07 -0800 Subject: [PATCH 02/15] fmt --- tokio/src/time/clock.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tokio/src/time/clock.rs b/tokio/src/time/clock.rs index 915301b82d6..7727f4048cd 100644 --- a/tokio/src/time/clock.rs +++ b/tokio/src/time/clock.rs @@ -5,11 +5,11 @@ //! configurable. #[cfg(feature = "test-util")] -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::variant::{advance, pause, resume}; +pub(crate) use self::variant::now; pub(crate) use self::variant::Clock; #[cfg(feature = "test-util")] -pub(crate) use self::variant::now; +#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 +pub use self::variant::{advance, pause, resume}; #[cfg(not(feature = "test-util"))] mod variant { From 19006c3c6a89deaa58e44453dc3ab1ced62cee85 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 14 Nov 2019 23:01:22 -0800 Subject: [PATCH 03/15] fix test --- tests-integration/tests/process_stdio.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests-integration/tests/process_stdio.rs b/tests-integration/tests/process_stdio.rs index 5e284535e1b..3676525881b 100644 --- a/tests-integration/tests/process_stdio.rs +++ b/tests-integration/tests/process_stdio.rs @@ -5,7 +5,6 @@ use tokio::process::{Child, Command}; use tokio_test::assert_ok; use futures::future::{self, FutureExt}; -use futures::stream::StreamExt; use std::env; use std::io; use std::process::{ExitStatus, Stdio}; @@ -49,7 +48,7 @@ async fn feed_cat(mut cat: Child, n: usize) -> io::Result { let data = reader .next_line() .await - .unwrap_or_else(|| Ok(String::new())) + .unwrap_or_else(|_| Some(String::new())) .expect("failed to read line"); let num_read = data.len(); From fb7f60682bbb84cf11d0ca5d243afcba728d0909 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 14 Nov 2019 23:09:41 -0800 Subject: [PATCH 04/15] try to fix windows --- tokio/src/process/windows.rs | 8 +++----- tokio/src/signal/windows.rs | 22 +++++----------------- 2 files changed, 8 insertions(+), 22 deletions(-) diff --git a/tokio/src/process/windows.rs b/tokio/src/process/windows.rs index 013e9bb7a70..d25807d6310 100644 --- a/tokio/src/process/windows.rs +++ b/tokio/src/process/windows.rs @@ -20,8 +20,6 @@ use crate::process::kill::Kill; use crate::process::SpawnedChild; use crate::sync::oneshot; -use futures_util::future::Fuse; -use futures_util::future::FutureExt; use mio_named_pipes::NamedPipe; use std::fmt; use std::future::Future; @@ -59,7 +57,7 @@ impl fmt::Debug for Child { } struct Waiting { - rx: Fuse>, + rx: oneshot::Receiver<()>, wait_object: HANDLE, tx: *mut Option>, } @@ -103,7 +101,7 @@ impl Future for Child { let inner = Pin::get_mut(self); loop { if let Some(ref mut w) = inner.waiting { - match w.rx.poll_unpin(cx) { + match Pin::new(&mut w.rx).poll(cx) { Poll::Ready(Ok(())) => {} Poll::Ready(Err(_)) => panic!("should not be canceled"), Poll::Pending => return Poll::Pending, @@ -134,7 +132,7 @@ impl Future for Child { return Poll::Ready(Err(err)); } inner.waiting = Some(Waiting { - rx: rx.fuse(), + rx, wait_object, tx: ptr, }); diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index 96e585ba588..78795b187e5 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -10,10 +10,8 @@ use crate::signal::registry::{globals, EventId, EventInfo, Init, Storage}; use crate::sync::mpsc::{channel, Receiver}; -use futures_core::stream::Stream; use std::convert::TryFrom; use std::io; -use std::pin::Pin; use std::sync::Once; use std::task::{Context, Poll}; use winapi::shared::minwindef::*; @@ -97,11 +95,10 @@ pub(crate) fn ctrl_c() -> io::Result { Event::new(CTRL_C_EVENT) } -impl Stream for Event { - type Item = (); - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.rx.poll_recv(cx) +impl Event { + pub(crate) async fn recv(&mut self) -> Option<()> { + use crate::future::poll_fn; + poll_fn(|cx| self.rx.poll_recv(cx)).await } } @@ -109,6 +106,7 @@ fn global_init() -> io::Result<()> { static INIT: Once = Once::new(); let mut init = None; + INIT.call_once(|| unsafe { let rc = SetConsoleCtrlHandler(Some(handler), TRUE); let ret = if rc == 0 { @@ -161,16 +159,6 @@ pub fn ctrl_break() -> io::Result { Event::new(CTRL_BREAK_EVENT).map(|inner| CtrlBreak { inner }) } -impl Stream for CtrlBreak { - type Item = (); - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut self.inner) - .poll_next(cx) - .map(|item| item.map(|_| ())) - } -} - #[cfg(all(test, not(loom)))] mod tests { use super::*; From 4ae6d4b6480e7366ef51f1265ceaf6f0f8d3e425 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 11:24:04 -0800 Subject: [PATCH 05/15] bring back futures_core::Stream dep as feature flag --- tokio-test/src/io.rs | 4 +- tokio-util/src/fs/mod.rs | 1 - tokio-util/src/fs/read_dir.rs | 19 ---- tokio-util/src/io/lines.rs | 22 ----- tokio-util/src/io/mod.rs | 1 - tokio-util/src/io/split.rs | 21 ----- tokio-util/src/lib.rs | 5 - tokio-util/src/stream/as_std.rs | 0 tokio-util/src/stream/into_std.rs | 19 ---- tokio-util/src/stream/mod.rs | 117 ----------------------- tokio-util/src/sync/mod.rs | 2 - tokio-util/src/sync/mpsc.rs | 21 ----- tokio-util/src/sync/watch.rs | 17 ---- tokio-util/src/time/interval.rs | 15 --- tokio-util/src/time/mod.rs | 1 - tokio/Cargo.toml | 7 +- tokio/src/fs/read_dir.rs | 15 ++- tokio/src/io/util/async_buf_read_ext.rs | 2 +- tokio/src/io/util/lines.rs | 67 ++++++++----- tokio/src/io/util/split.rs | 61 ++++++++---- tokio/src/net/tcp/incoming.rs | 38 ++++++++ tokio/src/net/tcp/listener.rs | 21 ++++- tokio/src/net/tcp/mod.rs | 3 + tokio/src/net/unix/incoming.rs | 37 ++++++++ tokio/src/net/unix/listener.rs | 11 ++- tokio/src/net/unix/mod.rs | 3 + tokio/src/signal/registry.rs | 10 +- tokio/src/sync/mpsc/bounded.rs | 120 +++++------------------- tokio/src/sync/mpsc/chan.rs | 98 ++++++++++++++----- tokio/src/sync/mpsc/error.rs | 86 +++++++++++++++++ tokio/src/sync/mpsc/mod.rs | 7 +- tokio/src/sync/mpsc/unbounded.rs | 82 ++++------------ tokio/src/sync/watch.rs | 14 ++- tokio/src/time/interval.rs | 13 ++- tokio/tests/fs_dir.rs | 34 ++++++- tokio/tests/io_lines.rs | 18 +++- tokio/tests/rt_common.rs | 2 +- tokio/tests/sync_errors.rs | 5 +- tokio/tests/sync_mpsc.rs | 58 ++++++++++-- tokio/tests/sync_watch.rs | 37 ++++++++ tokio/tests/time_interval.rs | 14 +++ 41 files changed, 599 insertions(+), 529 deletions(-) delete mode 100644 tokio-util/src/fs/mod.rs delete mode 100644 tokio-util/src/fs/read_dir.rs delete mode 100644 tokio-util/src/io/lines.rs delete mode 100644 tokio-util/src/io/mod.rs delete mode 100644 tokio-util/src/io/split.rs delete mode 100644 tokio-util/src/stream/as_std.rs delete mode 100644 tokio-util/src/stream/into_std.rs delete mode 100644 tokio-util/src/stream/mod.rs delete mode 100644 tokio-util/src/sync/mod.rs delete mode 100644 tokio-util/src/sync/mpsc.rs delete mode 100644 tokio-util/src/sync/watch.rs delete mode 100644 tokio-util/src/time/interval.rs delete mode 100644 tokio-util/src/time/mod.rs create mode 100644 tokio/src/net/tcp/incoming.rs create mode 100644 tokio/src/net/unix/incoming.rs create mode 100644 tokio/src/sync/mpsc/error.rs diff --git a/tokio-test/src/io.rs b/tokio-test/src/io.rs index 0cbc1dfecb4..5a2b74bf71c 100644 --- a/tokio-test/src/io.rs +++ b/tokio-test/src/io.rs @@ -122,7 +122,7 @@ impl Handle { /// The next operation in the mock's script will be to expect a `read` call /// and return `buf`. pub fn read(&mut self, buf: &[u8]) -> &mut Self { - self.tx.try_send(Action::Read(buf.into())).unwrap(); + self.tx.send(Action::Read(buf.into())).unwrap(); self } @@ -131,7 +131,7 @@ impl Handle { /// The next operation in the mock's script will be to expect a `write` /// call. pub fn write(&mut self, buf: &[u8]) -> &mut Self { - self.tx.try_send(Action::Write(buf.into())).unwrap(); + self.tx.send(Action::Write(buf.into())).unwrap(); self } } diff --git a/tokio-util/src/fs/mod.rs b/tokio-util/src/fs/mod.rs deleted file mode 100644 index 8cc0135305e..00000000000 --- a/tokio-util/src/fs/mod.rs +++ /dev/null @@ -1 +0,0 @@ -mod read_dir; diff --git a/tokio-util/src/fs/read_dir.rs b/tokio-util/src/fs/read_dir.rs deleted file mode 100644 index 4e5b38711a8..00000000000 --- a/tokio-util/src/fs/read_dir.rs +++ /dev/null @@ -1,19 +0,0 @@ -use crate::stream::Stream; -use tokio::fs::{DirEntry, ReadDir}; - -use futures_core::ready; -use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for ReadDir { - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Poll::Ready(match ready!(self.poll_next_entry(cx)) { - Ok(Some(entry)) => Some(Ok(entry)), - Ok(None) => None, - Err(err) => Some(Err(err)), - }) - } -} diff --git a/tokio-util/src/io/lines.rs b/tokio-util/src/io/lines.rs deleted file mode 100644 index 761ce119812..00000000000 --- a/tokio-util/src/io/lines.rs +++ /dev/null @@ -1,22 +0,0 @@ -use crate::stream::Stream; -use tokio::io::{AsyncBufRead, Lines}; - -use futures_core::ready; -use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for Lines -where - R: AsyncBufRead + Unpin, -{ - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - match ready!(self.poll_next_line(cx))? { - Some(line) => Poll::Ready(Some(Ok(line))), - None => Poll::Ready(None), - } - } -} - diff --git a/tokio-util/src/io/mod.rs b/tokio-util/src/io/mod.rs deleted file mode 100644 index 4cf30e13101..00000000000 --- a/tokio-util/src/io/mod.rs +++ /dev/null @@ -1 +0,0 @@ -mod split; diff --git a/tokio-util/src/io/split.rs b/tokio-util/src/io/split.rs deleted file mode 100644 index 3912dc3f072..00000000000 --- a/tokio-util/src/io/split.rs +++ /dev/null @@ -1,21 +0,0 @@ -use crate::stream::Stream; -use tokio::io::{AsyncBufRead, Split}; - -use futures_core::ready; -use std::io; -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for Split -where - R: AsyncBufRead + Unpin, -{ - type Item = io::Result>; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - match ready!(self.poll_next_segment(cx))? { - Some(segment) => Poll::Ready(Some(Ok(segment))), - None => Poll::Ready(None), - } - } -} diff --git a/tokio-util/src/lib.rs b/tokio-util/src/lib.rs index 783494bb46f..5a64673c8d1 100644 --- a/tokio-util/src/lib.rs +++ b/tokio-util/src/lib.rs @@ -14,9 +14,4 @@ //! Utilities for working with Tokio. pub mod codec; -mod fs; -mod io; -pub mod stream; -mod sync; -mod time; pub mod udp; diff --git a/tokio-util/src/stream/as_std.rs b/tokio-util/src/stream/as_std.rs deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tokio-util/src/stream/into_std.rs b/tokio-util/src/stream/into_std.rs deleted file mode 100644 index 9c3b3aa1c9b..00000000000 --- a/tokio-util/src/stream/into_std.rs +++ /dev/null @@ -1,19 +0,0 @@ -use crate::stream::Stream; - -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Stream for the [`into_std`](super::Stream::into_std) method. -#[derive(Debug)] -pub struct IntoStd { - pub(super) stream: T, -} - -impl futures_core::Stream for IntoStd { - type Item = T::Item; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let stream = unsafe { self.map_unchecked_mut(|me| &mut me.stream) }; - stream.poll_next(cx) - } -} diff --git a/tokio-util/src/stream/mod.rs b/tokio-util/src/stream/mod.rs deleted file mode 100644 index 69407614fd4..00000000000 --- a/tokio-util/src/stream/mod.rs +++ /dev/null @@ -1,117 +0,0 @@ -//! Asynchronous value iteration. - -mod into_std; -pub use into_std::IntoStd; - -use std::ops::DerefMut; -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// A stream of values produced asynchronously. -/// -/// This trait is used to convert Tokio's "stream-like" types into -/// `futures::Stream`. This trait is not intended to be implemented by third -/// parties (use `futures::Stream` instead) and only exists to satisfy Rust's -/// coherence requirements. When `Stream` is stabilized in `std`, this trait -/// will be removed and `tokio` will directly implement the `std` trait. -#[must_use = "streams do nothing unless polled"] -pub trait Stream { - /// Values yielded by the stream. - type Item; - - /// Attempt to pull out the next value of this stream, registering the - /// current task for wakeup if the value is not yet available, and returning - /// `None` if the stream is exhausted. - /// - /// # Return value - /// - /// There are several possible return values, each indicating a distinct - /// stream state: - /// - /// - `Poll::Pending` means that this stream's next value is not ready - /// yet. Implementations will ensure that the current task will be notified - /// when the next value may be ready. - /// - /// - `Poll::Ready(Some(val))` means that the stream has successfully - /// produced a value, `val`, and may produce further values on subsequent - /// `poll_next` calls. - /// - /// - `Poll::Ready(None)` means that the stream has terminated, and - /// `poll_next` should not be invoked again. - /// - /// # Panics - /// - /// Once a stream is finished, i.e. `Ready(None)` has been returned, further - /// calls to `poll_next` may result in a panic or other "bad behavior". If - /// this is difficult to guard against then the `fuse` adapter can be used - /// to ensure that `poll_next` always returns `Ready(None)` in subsequent - /// calls. - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; - - /// Returns the bounds on the remaining length of the stream. - /// - /// Specifically, `size_hint()` returns a tuple where the first element - /// is the lower bound, and the second element is the upper bound. - /// - /// The second half of the tuple that is returned is an [`Option`]`<`[`usize`]`>`. - /// A [`None`] here means that either there is no known upper bound, or the - /// upper bound is larger than [`usize`]. - /// - /// # Implementation notes - /// - /// It is not enforced that a stream implementation yields the declared - /// number of elements. A buggy stream may yield less than the lower bound - /// or more than the upper bound of elements. - /// - /// `size_hint()` is primarily intended to be used for optimizations such as - /// reserving space for the elements of the stream, but must not be - /// trusted to e.g., omit bounds checks in unsafe code. An incorrect - /// implementation of `size_hint()` should not lead to memory safety - /// violations. - /// - /// That said, the implementation should provide a correct estimation, - /// because otherwise it would be a violation of the trait's protocol. - /// - /// The default implementation returns `(0, `[`None`]`)` which is correct for any - /// stream. - #[inline] - fn size_hint(&self) -> (usize, Option) { - (0, None) - } - - /// Convert the stream into a `futures::Stream` type. - fn into_std(self) -> IntoStd - where - Self: Sized, - { - IntoStd { stream: self } - } -} - -impl Stream for &mut S { - type Item = S::Item; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - S::poll_next(Pin::new(&mut **self), cx) - } - - fn size_hint(&self) -> (usize, Option) { - (**self).size_hint() - } -} - -impl

Stream for Pin

-where - P: DerefMut + Unpin, - P::Target: Stream, -{ - type Item = ::Item; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.get_mut().as_mut().poll_next(cx) - } - - fn size_hint(&self) -> (usize, Option) { - (**self).size_hint() - } -} diff --git a/tokio-util/src/sync/mod.rs b/tokio-util/src/sync/mod.rs deleted file mode 100644 index 63a5c702901..00000000000 --- a/tokio-util/src/sync/mod.rs +++ /dev/null @@ -1,2 +0,0 @@ -mod mpsc; -mod watch; diff --git a/tokio-util/src/sync/mpsc.rs b/tokio-util/src/sync/mpsc.rs deleted file mode 100644 index ea9e89d1215..00000000000 --- a/tokio-util/src/sync/mpsc.rs +++ /dev/null @@ -1,21 +0,0 @@ -use crate::stream::Stream; -use tokio::sync::mpsc::{Receiver, UnboundedReceiver}; - -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for Receiver { - type Item = T; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.get_mut().poll_recv(cx) - } -} - -impl Stream for UnboundedReceiver { - type Item = T; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.get_mut().poll_recv(cx) - } -} diff --git a/tokio-util/src/sync/watch.rs b/tokio-util/src/sync/watch.rs deleted file mode 100644 index 027b57dffb3..00000000000 --- a/tokio-util/src/sync/watch.rs +++ /dev/null @@ -1,17 +0,0 @@ -use crate::stream::Stream; -use tokio::sync::watch::Receiver; - -use futures_core::ready; -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for Receiver { - type Item = T; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let ret = ready!(self.poll_recv_ref(cx)); - - #[allow(clippy::map_clone)] - Poll::Ready(ret.map(|v_ref| v_ref.clone())) - } -} diff --git a/tokio-util/src/time/interval.rs b/tokio-util/src/time/interval.rs deleted file mode 100644 index bc64b980c3d..00000000000 --- a/tokio-util/src/time/interval.rs +++ /dev/null @@ -1,15 +0,0 @@ -use crate::stream::Stream; -use tokio::time::{Instant, Interval}; - -use futures_core::ready; -use std::pin::Pin; -use std::task::{Context, Poll}; - -impl Stream for Interval { - type Item = Instant; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let instant = ready!(self.get_mut().poll_tick(cx)); - Poll::Ready(Some(instant)) - } -} diff --git a/tokio-util/src/time/mod.rs b/tokio-util/src/time/mod.rs deleted file mode 100644 index 73c3dc6f161..00000000000 --- a/tokio-util/src/time/mod.rs +++ /dev/null @@ -1 +0,0 @@ -mod interval; diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 18a970e4040..f1a34c025b9 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -32,6 +32,7 @@ default = [ "process", "rt-full", "signal", + "stream", "sync", "time", ] @@ -40,7 +41,7 @@ blocking = ["rt-core"] dns = ["blocking"] fs = ["blocking"] io-driver = ["mio", "lazy_static", "sync"] # TODO: get rid of sync -io-util = ["pin-project", "memchr"] +io-util = ["pin-project", "pin-project-lite", "memchr"] macros = ["tokio-macros"] net = ["dns", "tcp", "udp", "uds"] process = [ @@ -55,6 +56,7 @@ process = [ ] # Includes basic task execution capabilities rt-core = [] +# TODO: rename this -> `rt-threaded` rt-full = [ "macros", "num_cpus", @@ -72,6 +74,7 @@ signal = [ "winapi/consoleapi", "winapi/minwindef", ] +stream = ["futures-core"] sync = ["fnv"] test-util = [] tcp = ["io-driver"] @@ -88,11 +91,13 @@ iovec = "0.1" # Everything else is optional... fnv = { version = "1.0.6", optional = true } +futures-core = { version = "0.3.0", optional = true } lazy_static = { version = "1.0.2", optional = true } memchr = { version = "2.2", optional = true } mio = { version = "0.6.14", optional = true } num_cpus = { version = "1.8.0", optional = true } pin-project = { version = "0.4", optional = true } +pin-project-lite = { version = "0.1", optional = true } # Backs `DelayQueue` slab = { version = "0.4.1", optional = true } diff --git a/tokio/src/fs/read_dir.rs b/tokio/src/fs/read_dir.rs index 627309eadf5..219c7b35435 100644 --- a/tokio/src/fs/read_dir.rs +++ b/tokio/src/fs/read_dir.rs @@ -49,7 +49,7 @@ enum State { } impl ReadDir { - /// TODO: Dox + /// Returns the next entry in the directory stream. pub async fn next_entry(&mut self) -> io::Result> { use crate::future::poll_fn; poll_fn(|cx| self.poll_next_entry(cx)).await @@ -84,6 +84,19 @@ impl ReadDir { } } +#[cfg(feature = "stream")] +impl futures_core::Stream for ReadDir { + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(match ready!(self.poll_next_entry(cx)) { + Ok(Some(entry)) => Some(Ok(entry)), + Ok(None) => None, + Err(err) => Some(Err(err)), + }) + } +} + /// Entries returned by the [`ReadDir`] stream. /// /// [`ReadDir`]: struct.ReadDir.html diff --git a/tokio/src/io/util/async_buf_read_ext.rs b/tokio/src/io/util/async_buf_read_ext.rs index 3b4677b41c7..7567d72ed9b 100644 --- a/tokio/src/io/util/async_buf_read_ext.rs +++ b/tokio/src/io/util/async_buf_read_ext.rs @@ -113,7 +113,7 @@ pub trait AsyncBufReadExt: AsyncBufRead { /// [`AsyncBufReadExt::read_line`]: AsyncBufReadExt::read_line fn lines(self) -> Lines where - Self: Sized + Unpin, + Self: Sized, { lines(self) } diff --git a/tokio/src/io/util/lines.rs b/tokio/src/io/util/lines.rs index 29011aaf09c..96c0cc27a95 100644 --- a/tokio/src/io/util/lines.rs +++ b/tokio/src/io/util/lines.rs @@ -1,19 +1,23 @@ use crate::io::util::read_line::read_line_internal; use crate::io::AsyncBufRead; +use pin_project_lite::pin_project; use std::io; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; -/// Stream for the [`lines`](crate::io::AsyncBufReadExt::lines) method. -#[derive(Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct Lines { - reader: R, - buf: String, - bytes: Vec, - read: usize, +pin_project! { + /// Stream for the [`lines`](crate::io::AsyncBufReadExt::lines) method. + #[derive(Debug)] + #[must_use = "streams do nothing unless polled"] + pub struct Lines { + #[pin] + reader: R, + buf: String, + bytes: Vec, + read: usize, + } } pub(crate) fn lines(reader: R) -> Lines @@ -52,32 +56,49 @@ where pub async fn next_line(&mut self) -> io::Result> { use crate::future::poll_fn; - poll_fn(|cx| self.poll_next_line(cx)).await + poll_fn(|cx| Pin::new(&mut *self).poll_next_line(cx)).await } +} +impl Lines +where + R: AsyncBufRead, +{ #[doc(hidden)] - pub fn poll_next_line(&mut self, cx: &mut Context<'_>) -> Poll>> { - let n = ready!(read_line_internal( - Pin::new(&mut self.reader), - cx, - &mut self.buf, - &mut self.bytes, - &mut self.read - ))?; + pub fn poll_next_line( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { + let me = self.project(); - if n == 0 && self.buf.is_empty() { + let n = ready!(read_line_internal(me.reader, cx, me.buf, me.bytes, me.read))?; + + if n == 0 && me.buf.is_empty() { return Poll::Ready(Ok(None)); } - if self.buf.ends_with('\n') { - self.buf.pop(); + if me.buf.ends_with('\n') { + me.buf.pop(); - if self.buf.ends_with('\r') { - self.buf.pop(); + if me.buf.ends_with('\r') { + me.buf.pop(); } } - Poll::Ready(Ok(Some(mem::replace(&mut self.buf, String::new())))) + Poll::Ready(Ok(Some(mem::replace(me.buf, String::new())))) + } +} + +#[cfg(feature = "stream")] +impl futures_core::Stream for Lines { + type Item = io::Result; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(match ready!(self.poll_next_line(cx)) { + Ok(Some(line)) => Some(Ok(line)), + Ok(None) => None, + Err(err) => Some(Err(err)), + }) } } diff --git a/tokio/src/io/util/split.rs b/tokio/src/io/util/split.rs index 23cd0bf6666..be66f70c3f8 100644 --- a/tokio/src/io/util/split.rs +++ b/tokio/src/io/util/split.rs @@ -1,19 +1,23 @@ use crate::io::util::read_until::read_until_internal; use crate::io::AsyncBufRead; +use pin_project_lite::pin_project; use std::io; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; -/// Stream for the [`split`](crate::io::AsyncBufReadExt::split) method. -#[derive(Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct Split { - reader: R, - buf: Vec, - delim: u8, - read: usize, +pin_project! { + /// Stream for the [`split`](crate::io::AsyncBufReadExt::split) method. + #[derive(Debug)] + #[must_use = "streams do nothing unless polled"] + pub struct Split { + #[pin] + reader: R, + buf: Vec, + delim: u8, + read: usize, + } } pub(crate) fn split(reader: R, delim: u8) -> Split @@ -52,28 +56,47 @@ where pub async fn next_segment(&mut self) -> io::Result>> { use crate::future::poll_fn; - poll_fn(|cx| self.poll_next_segment(cx)).await + poll_fn(|cx| Pin::new(&mut *self).poll_next_segment(cx)).await } +} +impl Split +where + R: AsyncBufRead, +{ #[doc(hidden)] - pub fn poll_next_segment(&mut self, cx: &mut Context<'_>) -> Poll>>> { + pub fn poll_next_segment( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>>> { + let me = self.project(); + let n = ready!(read_until_internal( - Pin::new(&mut self.reader), - cx, - self.delim, - &mut self.buf, - &mut self.read + me.reader, cx, *me.delim, me.buf, me.read, ))?; - if n == 0 && self.buf.is_empty() { + if n == 0 && me.buf.is_empty() { return Poll::Ready(Ok(None)); } - if self.buf.last() == Some(&self.delim) { - self.buf.pop(); + if me.buf.last() == Some(me.delim) { + me.buf.pop(); } - Poll::Ready(Ok(Some(mem::replace(&mut self.buf, Vec::new())))) + Poll::Ready(Ok(Some(mem::replace(me.buf, Vec::new())))) + } +} + +#[cfg(feature = "stream")] +impl futures_core::Stream for Split { + type Item = io::Result>; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(match ready!(self.poll_next_segment(cx)) { + Ok(Some(segment)) => Some(Ok(segment)), + Ok(None) => None, + Err(err) => Some(Err(err)), + }) } } diff --git a/tokio/src/net/tcp/incoming.rs b/tokio/src/net/tcp/incoming.rs new file mode 100644 index 00000000000..3033aefa33c --- /dev/null +++ b/tokio/src/net/tcp/incoming.rs @@ -0,0 +1,38 @@ +use crate::net::tcp::{TcpListener, TcpStream}; + +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Stream returned by the `TcpListener::incoming` function representing the +/// stream of sockets received from a listener. +#[must_use = "streams do nothing unless polled"] +#[derive(Debug)] +pub struct Incoming<'a> { + inner: &'a mut TcpListener, +} + +impl Incoming<'_> { + pub(crate) fn new(listener: &mut TcpListener) -> Incoming<'_> { + Incoming { inner: listener } + } + + #[doc(hidden)] // TODO: dox + pub fn poll_accept( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let (socket, _) = ready!(self.inner.poll_accept(cx))?; + Poll::Ready(Ok(socket)) + } +} + +#[cfg(feature = "stream")] +impl futures_core::Stream for Incoming<'_> { + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let (socket, _) = ready!(self.inner.poll_accept(cx))?; + Poll::Ready(Some(Ok(socket))) + } +} diff --git a/tokio/src/net/tcp/listener.rs b/tokio/src/net/tcp/listener.rs index d8f7dfe08f8..c0e7f1b3478 100644 --- a/tokio/src/net/tcp/listener.rs +++ b/tokio/src/net/tcp/listener.rs @@ -1,5 +1,5 @@ use crate::future::poll_fn; -use crate::net::tcp::TcpStream; +use crate::net::tcp::{Incoming, TcpStream}; use crate::net::util::PollEvented; use crate::net::ToSocketAddrs; @@ -122,7 +122,8 @@ impl TcpListener { poll_fn(|cx| self.poll_accept(cx)).await } - pub(crate) fn poll_accept( + #[doc(hidden)] // TODO: document + pub fn poll_accept( &mut self, cx: &mut Context<'_>, ) -> Poll> { @@ -226,6 +227,22 @@ impl TcpListener { self.io.get_ref().local_addr() } + /// Consumes this listener, returning a stream of the sockets this listener + /// accepts. + /// + /// This method returns an implementation of the `Stream` trait which + /// resolves to the sockets the are accepted on this listener. + /// + /// # Errors + /// + /// Note that accepting a connection can lead to various errors and not all of them are + /// necessarily fatal ‒ for example having too many open file descriptors or the other side + /// closing the connection while it waits in an accept queue. These would terminate the stream + /// if not handled in any way. + pub fn incoming(&mut self) -> Incoming<'_> { + Incoming::new(self) + } + /// Gets the value of the `IP_TTL` option for this socket. /// /// For more information about this option, see [`set_ttl`]. diff --git a/tokio/src/net/tcp/mod.rs b/tokio/src/net/tcp/mod.rs index a4e5ab4cb48..e3acf54217e 100644 --- a/tokio/src/net/tcp/mod.rs +++ b/tokio/src/net/tcp/mod.rs @@ -18,6 +18,9 @@ mod listener; pub use self::listener::TcpListener; +mod incoming; +pub use self::incoming::Incoming; + pub mod split; mod stream; diff --git a/tokio/src/net/unix/incoming.rs b/tokio/src/net/unix/incoming.rs new file mode 100644 index 00000000000..dbe964a8d29 --- /dev/null +++ b/tokio/src/net/unix/incoming.rs @@ -0,0 +1,37 @@ +use crate::net::unix::{UnixListener, UnixStream}; + +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// Stream of listeners +#[derive(Debug)] +#[must_use = "streams do nothing unless polled"] +pub struct Incoming<'a> { + inner: &'a mut UnixListener, +} + +impl Incoming<'_> { + pub(crate) fn new(listener: &mut UnixListener) -> Incoming<'_> { + Incoming { inner: listener } + } + + #[doc(hidden)] // TODO: dox + pub fn poll_accept( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let (socket, _) = ready!(self.inner.poll_accept(cx))?; + Poll::Ready(Ok(socket)) + } +} + +#[cfg(feature = "stream")] +impl futures_core::Stream for Incoming<'_> { + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let (socket, _) = ready!(self.inner.poll_accept(cx))?; + Poll::Ready(Some(Ok(socket))) + } +} diff --git a/tokio/src/net/unix/listener.rs b/tokio/src/net/unix/listener.rs index 72cd55bdf05..c7de68b1338 100644 --- a/tokio/src/net/unix/listener.rs +++ b/tokio/src/net/unix/listener.rs @@ -1,5 +1,5 @@ use crate::future::poll_fn; -use crate::net::unix::UnixStream; +use crate::net::unix::{Incoming, UnixStream}; use crate::net::util::PollEvented; use mio::Ready; @@ -83,6 +83,15 @@ impl UnixListener { Err(err) => Err(err).into(), } } + + /// Consumes this listener, returning a stream of the sockets this listener + /// accepts. + /// + /// This method returns an implementation of the `Stream` trait which + /// resolves to the sockets the are accepted on this listener. + pub fn incoming(&mut self) -> Incoming<'_> { + Incoming::new(self) + } } impl TryFrom for mio_uds::UnixListener { diff --git a/tokio/src/net/unix/mod.rs b/tokio/src/net/unix/mod.rs index 4fd0176be51..977e3a0f816 100644 --- a/tokio/src/net/unix/mod.rs +++ b/tokio/src/net/unix/mod.rs @@ -5,6 +5,9 @@ mod datagram; pub use self::datagram::UnixDatagram; +mod incoming; +pub use self::incoming::Incoming; + mod listener; pub use self::listener::UnixListener; diff --git a/tokio/src/signal/registry.rs b/tokio/src/signal/registry.rs index ba247b00db4..0e017965a1d 100644 --- a/tokio/src/signal/registry.rs +++ b/tokio/src/signal/registry.rs @@ -87,6 +87,8 @@ impl Registry { /// /// Returns true if an event was delivered to at least one listener. fn broadcast(&self) -> bool { + use crate::sync::mpsc::error::TrySendError; + let mut did_notify = false; self.storage.for_each(|event_info| { // Any signal of this kind arrived since we checked last? @@ -103,17 +105,13 @@ impl Registry { for i in (0..recipients.len()).rev() { match recipients[i].try_send(()) { Ok(()) => did_notify = true, - Err(ref e) if e.is_closed() => { + Err(TrySendError::Closed(..)) => { recipients.swap_remove(i); } // Channel is full, ignore the error since the // receiver has already been woken up - Err(e) => { - // Sanity check in case this error type ever gets - // additional variants we have not considered. - debug_assert!(e.is_full()); - } + Err(_) => {} } } }); diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 511f07ccfba..8c84eee4234 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -1,4 +1,5 @@ use crate::sync::mpsc::chan; +use crate::sync::mpsc::error::{ClosedError, SendError, TrySendError}; use crate::sync::semaphore; use std::fmt; @@ -43,27 +44,6 @@ impl fmt::Debug for Receiver { } } -/// Error returned by the `Sender`. -#[derive(Debug)] -pub struct SendError(()); - -/// Error returned by `Sender::try_send`. -#[derive(Debug)] -pub struct TrySendError { - kind: ErrorKind, - value: T, -} - -#[derive(Debug)] -enum ErrorKind { - Closed, - NoCapacity, -} - -/// Error returned by `Receiver`. -#[derive(Debug)] -pub struct RecvError(()); - /// Create a bounded mpsc channel for communicating between asynchronous tasks, /// returning the sender/receiver halves. /// @@ -165,7 +145,7 @@ impl Receiver { poll_fn(|cx| self.poll_recv(cx)).await } - #[doc(hidden)] // TODO: remove + #[doc(hidden)] // TODO: document pub fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { self.chan.recv(cx) } @@ -179,14 +159,25 @@ impl Receiver { } } +impl Unpin for Receiver {} + +#[cfg(feature = "stream")] +impl futures_core::Stream for Receiver { + type Item = T; + + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_recv(cx) + } +} + impl Sender { pub(crate) fn new(chan: chan::Tx) -> Sender { Sender { chan } } - #[doc(hidden)] // TODO: remove - pub fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { - self.chan.poll_ready(cx).map_err(|_| SendError(())) + #[doc(hidden)] // TODO: document + pub fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + self.chan.poll_ready(cx).map_err(|_| ClosedError::new()) } /// Attempts to send a message on this `Sender`, returning the message @@ -224,82 +215,17 @@ impl Sender { /// } /// } /// ``` - pub async fn send(&mut self, value: T) -> Result<(), SendError> { + pub async fn send(&mut self, value: T) -> Result<(), SendError> { use crate::future::poll_fn; - poll_fn(|cx| self.poll_ready(cx)).await?; - - self.try_send(value).map_err(|_| SendError(())) - } -} - -// ===== impl SendError ===== - -impl fmt::Display for SendError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "channel closed") - } -} - -impl ::std::error::Error for SendError {} - -// ===== impl TrySendError ===== - -impl TrySendError { - /// Get the inner value. - pub fn into_inner(self) -> T { - self.value - } - - /// Did the send fail because the channel has been closed? - pub fn is_closed(&self) -> bool { - if let ErrorKind::Closed = self.kind { - true - } else { - false - } - } - - /// Did the send fail because the channel was at capacity? - pub fn is_full(&self) -> bool { - if let ErrorKind::NoCapacity = self.kind { - true - } else { - false + if let Err(_) = poll_fn(|cx| self.poll_ready(cx)).await { + return Err(SendError(value)); } - } -} - -impl fmt::Display for TrySendError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - let descr = match self.kind { - ErrorKind::Closed => "channel closed", - ErrorKind::NoCapacity => "no available capacity", - }; - write!(fmt, "{}", descr) - } -} -impl ::std::error::Error for TrySendError {} - -impl From<(T, chan::TrySendError)> for TrySendError { - fn from((value, err): (T, chan::TrySendError)) -> TrySendError { - TrySendError { - value, - kind: match err { - chan::TrySendError::Closed => ErrorKind::Closed, - chan::TrySendError::NoPermits => ErrorKind::NoCapacity, - }, + match self.try_send(value) { + Ok(()) => Ok(()), + Err(TrySendError::Full(_)) => unreachable!(), + Err(TrySendError::Closed(value)) => Err(SendError(value)), } } } - -// ===== impl RecvError ===== - -impl fmt::Display for RecvError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "channel closed") - } -} - -impl ::std::error::Error for RecvError {} diff --git a/tokio/src/sync/mpsc/chan.rs b/tokio/src/sync/mpsc/chan.rs index ad0d99a8ffe..03f35339168 100644 --- a/tokio/src/sync/mpsc/chan.rs +++ b/tokio/src/sync/mpsc/chan.rs @@ -1,5 +1,9 @@ -use crate::loom::{cell::CausalCell, future::AtomicWaker, sync::atomic::AtomicUsize, sync::Arc}; -use crate::sync::mpsc::list; +use crate::loom::cell::CausalCell; +use crate::loom::future::AtomicWaker; +use crate::loom::sync::atomic::AtomicUsize; +use crate::loom::sync::Arc; +use crate::sync::mpsc::error::ClosedError; +use crate::sync::mpsc::{error, list}; use std::fmt; use std::process; @@ -43,7 +47,25 @@ where #[derive(Debug, Eq, PartialEq)] pub(crate) enum TrySendError { Closed, - NoPermits, + Full, +} + +impl From<(T, TrySendError)> for error::SendError { + fn from(src: (T, TrySendError)) -> error::SendError { + match src.1 { + TrySendError::Closed => error::SendError(src.0), + TrySendError::Full => unreachable!(), + } + } +} + +impl From<(T, TrySendError)> for error::TrySendError { + fn from(src: (T, TrySendError)) -> error::TrySendError { + match src.1 { + TrySendError::Closed => error::TrySendError::Closed(src.0), + TrySendError::Full => error::TrySendError::Full(src.0), + } + } } pub(crate) trait Semaphore { @@ -59,8 +81,11 @@ pub(crate) trait Semaphore { fn add_permit(&self); - fn poll_acquire(&self, cx: &mut Context<'_>, permit: &mut Self::Permit) - -> Poll>; + fn poll_acquire( + &self, + cx: &mut Context<'_>, + permit: &mut Self::Permit, + ) -> Poll>; fn try_acquire(&self, permit: &mut Self::Permit) -> Result<(), TrySendError>; @@ -161,26 +186,19 @@ where } } - pub(crate) fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + pub(crate) fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { self.inner.semaphore.poll_acquire(cx, &mut self.permit) } /// Send a message and notify the receiver. pub(crate) fn try_send(&mut self, value: T) -> Result<(), (T, TrySendError)> { - if let Err(e) = self.inner.semaphore.try_acquire(&mut self.permit) { - return Err((value, e)); - } - - // Push the value - self.inner.tx.push(value); - - // Notify the rx task - self.inner.rx_waker.wake(); - - // Release the permit - self.inner.semaphore.forget(&mut self.permit); + self.inner.try_send(value, &mut self.permit) + } +} - Ok(()) +impl Tx { + pub(crate) fn send_unbounded(&self, value: T) -> Result<(), (T, TrySendError)> { + self.inner.try_send(value, &mut ()) } } @@ -317,6 +335,28 @@ where // ===== impl Chan ===== +impl Chan +where + S: Semaphore, +{ + fn try_send(&self, value: T, permit: &mut S::Permit) -> Result<(), (T, TrySendError)> { + if let Err(e) = self.semaphore.try_acquire(permit) { + return Err((value, e)); + } + + // Push the value + self.tx.push(value); + + // Notify the rx task + self.rx_waker.wake(); + + // Release the permit + self.semaphore.forget(permit); + + Ok(()) + } +} + impl Drop for Chan { fn drop(&mut self) { use super::block::Read::Value; @@ -339,7 +379,7 @@ impl From for TrySendError { if src.is_closed() { TrySendError::Closed } else if src.is_no_permits() { - TrySendError::NoPermits + TrySendError::Full } else { unreachable!(); } @@ -369,8 +409,14 @@ impl Semaphore for (crate::sync::semaphore::Semaphore, usize) { self.0.available_permits() == self.1 } - fn poll_acquire(&self, cx: &mut Context<'_>, permit: &mut Permit) -> Poll> { - permit.poll_acquire(cx, &self.0).map_err(|_| ()) + fn poll_acquire( + &self, + cx: &mut Context<'_>, + permit: &mut Permit, + ) -> Poll> { + permit + .poll_acquire(cx, &self.0) + .map_err(|_| ClosedError::new()) } fn try_acquire(&self, permit: &mut Permit) -> Result<(), TrySendError> { @@ -412,8 +458,12 @@ impl Semaphore for AtomicUsize { self.load(Acquire) >> 1 == 0 } - fn poll_acquire(&self, _cx: &mut Context<'_>, permit: &mut ()) -> Poll> { - Ready(self.try_acquire(permit).map_err(|_| ())) + fn poll_acquire( + &self, + _cx: &mut Context<'_>, + permit: &mut (), + ) -> Poll> { + Ready(self.try_acquire(permit).map_err(|_| ClosedError::new())) } fn try_acquire(&self, _permit: &mut ()) -> Result<(), TrySendError> { diff --git a/tokio/src/sync/mpsc/error.rs b/tokio/src/sync/mpsc/error.rs new file mode 100644 index 00000000000..6238f8548af --- /dev/null +++ b/tokio/src/sync/mpsc/error.rs @@ -0,0 +1,86 @@ +//! Channel error types + +use std::error::Error; +use std::fmt; + +/// Error returned by the `Sender`. +#[derive(Debug)] +pub struct SendError(pub T); + +impl fmt::Display for SendError { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(fmt, "channel closed") + } +} + +impl ::std::error::Error for SendError {} + +// ===== TrySendError ===== + +/// This enumeration is the list of the possible error outcomes for the +/// [try_send](super::Sender::try_send) method. +#[derive(Debug)] +pub enum TrySendError { + /// The data could not be sent on the channel because the channel is + /// currently full and sending would require blocking. + Full(T), + + /// The receive half of the channel was explicitly closed or has been + /// dropped. + Closed(T), +} + +impl Error for TrySendError {} + +impl fmt::Display for TrySendError { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + fmt, + "{}", + match self { + TrySendError::Full(..) => "no available capacity", + TrySendError::Closed(..) => "channel closed", + } + ) + } +} + +impl From> for TrySendError { + fn from(src: SendError) -> TrySendError { + TrySendError::Closed(src.0) + } +} + +// ===== RecvError ===== + +/// Error returned by `Receiver`. +#[derive(Debug)] +pub struct RecvError(()); + +impl fmt::Display for RecvError { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(fmt, "channel closed") + } +} + +impl Error for RecvError {} + +// ===== ClosedError ===== + +/// Erorr returned by [`Sender::poll_ready`](super::Sender::poll_ready)]. +#[derive(Debug)] +pub struct ClosedError(()); + +impl ClosedError { + pub(crate) fn new() -> ClosedError { + ClosedError(()) + } +} + +impl fmt::Display for ClosedError { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(fmt, "channel closed") + } +} + +impl Error for ClosedError {} diff --git a/tokio/src/sync/mpsc/mod.rs b/tokio/src/sync/mpsc/mod.rs index 3b95b9544d1..7927dde6a0e 100644 --- a/tokio/src/sync/mpsc/mod.rs +++ b/tokio/src/sync/mpsc/mod.rs @@ -46,12 +46,7 @@ pub(super) mod list; mod unbounded; pub use self::unbounded::{unbounded_channel, UnboundedReceiver, UnboundedSender}; -pub mod error { - //! Channel error types - - pub use super::bounded::{RecvError, SendError, TrySendError}; - pub use super::unbounded::{UnboundedRecvError, UnboundedSendError, UnboundedTrySendError}; -} +pub mod error; /// The number of values a block can contain. /// diff --git a/tokio/src/sync/mpsc/unbounded.rs b/tokio/src/sync/mpsc/unbounded.rs index b347dee8797..07a173c213d 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -1,5 +1,6 @@ use crate::loom::sync::atomic::AtomicUsize; use crate::sync::mpsc::chan; +use crate::sync::mpsc::error::SendError; use std::fmt; use std::task::{Context, Poll}; @@ -45,18 +46,6 @@ impl fmt::Debug for UnboundedReceiver { } } -/// Error returned by the `UnboundedSender`. -#[derive(Debug)] -pub struct UnboundedSendError(()); - -/// Returned by `UnboundedSender::try_send` when the channel has been closed. -#[derive(Debug)] -pub struct UnboundedTrySendError(T); - -/// Error returned by `UnboundedReceiver`. -#[derive(Debug)] -pub struct UnboundedRecvError(()); - /// Create an unbounded mpsc channel for communicating between asynchronous /// tasks. /// @@ -84,7 +73,7 @@ impl UnboundedReceiver { UnboundedReceiver { chan } } - #[doc(hidden)] // TODO: remove + #[doc(hidden)] // TODO: doc pub fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { self.chan.recv(cx) } @@ -101,10 +90,10 @@ impl UnboundedReceiver { /// /// #[tokio::main] /// async fn main() { - /// let (mut tx, mut rx) = mpsc::unbounded_channel(); + /// let (tx, mut rx) = mpsc::unbounded_channel(); /// /// tokio::spawn(async move { - /// tx.try_send("hello").unwrap(); + /// tx.send("hello").unwrap(); /// }); /// /// assert_eq!(Some("hello"), rx.recv().await); @@ -119,10 +108,10 @@ impl UnboundedReceiver { /// /// #[tokio::main] /// async fn main() { - /// let (mut tx, mut rx) = mpsc::unbounded_channel(); + /// let (tx, mut rx) = mpsc::unbounded_channel(); /// - /// tx.try_send("hello").unwrap(); - /// tx.try_send("world").unwrap(); + /// tx.send("hello").unwrap(); + /// tx.send("world").unwrap(); /// /// assert_eq!(Some("hello"), rx.recv().await); /// assert_eq!(Some("world"), rx.recv().await); @@ -143,58 +132,23 @@ impl UnboundedReceiver { } } +#[cfg(feature = "stream")] +impl futures_core::Stream for UnboundedReceiver { + type Item = T; + + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_recv(cx) + } +} + impl UnboundedSender { pub(crate) fn new(chan: chan::Tx) -> UnboundedSender { UnboundedSender { chan } } /// Attempts to send a message on this `UnboundedSender` without blocking. - pub fn try_send(&mut self, message: T) -> Result<(), UnboundedTrySendError> { - self.chan.try_send(message)?; + pub fn send(&self, message: T) -> Result<(), SendError> { + self.chan.send_unbounded(message)?; Ok(()) } } - -// ===== impl UnboundedSendError ===== - -impl fmt::Display for UnboundedSendError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "channel closed") - } -} - -impl ::std::error::Error for UnboundedSendError {} - -// ===== impl TrySendError ===== - -impl UnboundedTrySendError { - /// Get the inner value. - pub fn into_inner(self) -> T { - self.0 - } -} - -impl fmt::Display for UnboundedTrySendError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "channel closed") - } -} - -impl ::std::error::Error for UnboundedTrySendError {} - -impl From<(T, chan::TrySendError)> for UnboundedTrySendError { - fn from((value, err): (T, chan::TrySendError)) -> UnboundedTrySendError { - assert_eq!(chan::TrySendError::Closed, err); - UnboundedTrySendError(value) - } -} - -// ===== impl UnboundedRecvError ===== - -impl fmt::Display for UnboundedRecvError { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(fmt, "channel closed") - } -} - -impl ::std::error::Error for UnboundedRecvError {} diff --git a/tokio/src/sync/watch.rs b/tokio/src/sync/watch.rs index a178720b9a3..d8e2cc3596b 100644 --- a/tokio/src/sync/watch.rs +++ b/tokio/src/sync/watch.rs @@ -258,16 +258,26 @@ impl Receiver { impl Receiver { /// Attempts to clone the latest value sent via the channel. - #[allow(clippy::map_clone)] // false positive: https://github.com/rust-lang/rust-clippy/issues/3274 pub async fn recv(&mut self) -> Option { poll_fn(|cx| { let v_ref = ready!(self.poll_recv_ref(cx)); - Poll::Ready(v_ref.map(|v| v.clone())) + Poll::Ready(v_ref.map(|v_ref| (*v_ref).clone())) }) .await } } +#[cfg(feature = "stream")] +impl futures_core::Stream for Receiver { + type Item = T; + + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let v_ref = ready!(self.poll_recv_ref(cx)); + + Poll::Ready(v_ref.map(|v_ref| (*v_ref).clone())) + } +} + impl Clone for Receiver { fn clone(&self) -> Self { let inner = Arc::new(WatchInner::new()); diff --git a/tokio/src/time/interval.rs b/tokio/src/time/interval.rs index 9a0a4e69b8c..f9fa1127b7d 100644 --- a/tokio/src/time/interval.rs +++ b/tokio/src/time/interval.rs @@ -3,7 +3,7 @@ use crate::time::{delay_until, Delay, Duration, Instant}; use std::future::Future; use std::pin::Pin; -use std::task::{self, Poll}; +use std::task::{Context, Poll}; /// Creates new `Interval` that yields with interval of `duration`. The first /// tick completes immediately. @@ -87,7 +87,7 @@ pub struct Interval { impl Interval { #[doc(hidden)] // TODO: document - pub fn poll_tick(&mut self, cx: &mut task::Context<'_>) -> Poll { + pub fn poll_tick(&mut self, cx: &mut Context<'_>) -> Poll { // Wait for the delay to be done ready!(Pin::new(&mut self.delay).poll(cx)); @@ -128,3 +128,12 @@ impl Interval { poll_fn(|cx| self.poll_tick(cx)).await } } + +#[cfg(feature = "stream")] +impl futures_core::Stream for Interval { + type Item = Instant; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(Some(ready!(self.poll_tick(cx)))) + } +} diff --git a/tokio/tests/fs_dir.rs b/tokio/tests/fs_dir.rs index 8f8f2717a46..40e20bdb6b5 100644 --- a/tokio/tests/fs_dir.rs +++ b/tokio/tests/fs_dir.rs @@ -40,7 +40,7 @@ async fn remove() { } #[tokio::test] -async fn read() { +async fn read_inherent() { let base_dir = tempdir().unwrap(); let p = base_dir.path(); @@ -67,3 +67,35 @@ async fn read() { vec!["aa".to_string(), "bb".to_string(), "cc".to_string()] ); } + +#[tokio::test] +async fn read_stream() { + use futures::StreamExt; + + let base_dir = tempdir().unwrap(); + + let p = base_dir.path(); + std::fs::create_dir(p.join("aa")).unwrap(); + std::fs::create_dir(p.join("bb")).unwrap(); + std::fs::create_dir(p.join("cc")).unwrap(); + + let files = Arc::new(Mutex::new(Vec::new())); + + let f = files.clone(); + let p = p.to_path_buf(); + + let mut entries = fs::read_dir(p).await.unwrap(); + + while let Some(res) = entries.next().await { + let e = assert_ok!(res); + let s = e.file_name().to_str().unwrap().to_string(); + f.lock().unwrap().push(s); + } + + let mut files = files.lock().unwrap(); + files.sort(); // because the order is not guaranteed + assert_eq!( + *files, + vec!["aa".to_string(), "bb".to_string(), "cc".to_string()] + ); +} diff --git a/tokio/tests/io_lines.rs b/tokio/tests/io_lines.rs index e479ee98888..83240d62336 100644 --- a/tokio/tests/io_lines.rs +++ b/tokio/tests/io_lines.rs @@ -4,7 +4,7 @@ use tokio::io::AsyncBufReadExt; use tokio_test::assert_ok; #[tokio::test] -async fn lines() { +async fn lines_inherent() { let rd: &[u8] = b"hello\r\nworld\n\n"; let mut st = rd.lines(); @@ -16,3 +16,19 @@ async fn lines() { assert_eq!(b, ""); assert!(assert_ok!(st.next_line().await).is_none()); } + +#[tokio::test] +async fn lines_stream() { + use futures::StreamExt; + + let rd: &[u8] = b"hello\r\nworld\n\n"; + let mut st = rd.lines(); + + let b = assert_ok!(st.next().await.unwrap()); + assert_eq!(b, "hello"); + let b = assert_ok!(st.next().await.unwrap()); + assert_eq!(b, "world"); + let b = assert_ok!(st.next().await.unwrap()); + assert_eq!(b, ""); + assert!(st.next().await.is_none()); +} diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index e8d745e98b4..06b8a1c4f3d 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -138,7 +138,7 @@ rt_test! { tokio::spawn(async move { let msg = assert_ok!(rx.await); assert_eq!(i, msg); - assert_ok!(done_tx.try_send(msg)); + assert_ok!(done_tx.send(msg)); }); tx diff --git a/tokio/tests/sync_errors.rs b/tokio/tests/sync_errors.rs index e68fe08193f..8cc0c0cd99a 100644 --- a/tokio/tests/sync_errors.rs +++ b/tokio/tests/sync_errors.rs @@ -6,11 +6,8 @@ fn is_error() {} fn mpsc_error_bound() { use tokio::sync::mpsc::error; - is_error::(); + is_error::>(); is_error::>(); - is_error::(); - is_error::(); - is_error::>(); } #[test] diff --git a/tokio/tests/sync_mpsc.rs b/tokio/tests/sync_mpsc.rs index 3d931aec89b..040904e4acd 100644 --- a/tokio/tests/sync_mpsc.rs +++ b/tokio/tests/sync_mpsc.rs @@ -1,6 +1,7 @@ #![warn(rust_2018_idioms)] use tokio::sync::mpsc; +use tokio::sync::mpsc::error::TrySendError; use tokio_test::task; use tokio_test::{ assert_err, assert_ok, assert_pending, assert_ready, assert_ready_err, assert_ready_ok, @@ -37,6 +38,22 @@ fn send_recv_with_buffer() { assert!(val.is_none()); } +#[tokio::test] +async fn send_recv_stream_with_buffer() { + use futures::StreamExt; + + let (mut tx, mut rx) = mpsc::channel::(16); + + tokio::spawn(async move { + assert_ok!(tx.send(1).await); + assert_ok!(tx.send(2).await); + }); + + assert_eq!(Some(1), rx.next().await); + assert_eq!(Some(2), rx.next().await); + assert_eq!(None, rx.next().await); +} + #[tokio::test] async fn async_send_recv_with_buffer() { let (mut tx, mut rx) = mpsc::channel(16); @@ -94,11 +111,11 @@ fn buffer_gteq_one() { fn send_recv_unbounded() { let mut t1 = task::spawn(()); - let (mut tx, mut rx) = mpsc::unbounded_channel::(); + let (tx, mut rx) = mpsc::unbounded_channel::(); // Using `try_send` - assert_ok!(tx.try_send(1)); - assert_ok!(tx.try_send(2)); + assert_ok!(tx.send(1)); + assert_ok!(tx.send(2)); let val = assert_ready!(t1.enter(|cx, _| rx.poll_recv(cx))); assert_eq!(val, Some(1)); @@ -114,11 +131,11 @@ fn send_recv_unbounded() { #[tokio::test] async fn async_send_recv_unbounded() { - let (mut tx, mut rx) = mpsc::unbounded_channel(); + let (tx, mut rx) = mpsc::unbounded_channel(); tokio::spawn(async move { - assert_ok!(tx.try_send(1)); - assert_ok!(tx.try_send(2)); + assert_ok!(tx.send(1)); + assert_ok!(tx.send(2)); }); assert_eq!(Some(1), rx.recv().await); @@ -126,6 +143,22 @@ async fn async_send_recv_unbounded() { assert_eq!(None, rx.recv().await); } +#[tokio::test] +async fn send_recv_stream_unbounded() { + use futures::StreamExt; + + let (tx, mut rx) = mpsc::unbounded_channel::(); + + tokio::spawn(async move { + assert_ok!(tx.send(1)); + assert_ok!(tx.send(2)); + }); + + assert_eq!(Some(1), rx.next().await); + assert_eq!(Some(2), rx.next().await); + assert_eq!(None, rx.next().await); +} + #[test] fn no_t_bounds_buffer() { struct NoImpls; @@ -156,7 +189,7 @@ fn no_t_bounds_unbounded() { // same with Receiver println!("{:?}", rx); // and sender should be Clone even though T isn't Clone - assert!(tx.clone().try_send(NoImpls).is_ok()); + assert!(tx.clone().send(NoImpls).is_ok()); let val = assert_ready!(t1.enter(|cx, _| rx.poll_recv(cx))); assert!(val.is_some()); @@ -289,8 +322,10 @@ fn try_send_fail() { tx.try_send("hello").unwrap(); // This should fail - let err = assert_err!(tx.try_send("fail")); - assert!(err.is_full()); + match assert_err!(tx.try_send("fail")) { + TrySendError::Full(..) => {} + _ => panic!(), + } let val = assert_ready!(t1.enter(|cx, _| rx.poll_recv(cx))); assert_eq!(val, Some("hello")); @@ -354,7 +389,10 @@ fn dropping_rx_closes_channel_for_try() { { let err = assert_err!(tx.try_send(msg.clone())); - assert!(err.is_closed()); + match err { + TrySendError::Closed(..) => {} + _ => panic!(), + } } assert_eq!(1, Arc::strong_count(&msg)); diff --git a/tokio/tests/sync_watch.rs b/tokio/tests/sync_watch.rs index 6c24ab01093..7ccad5c2adf 100644 --- a/tokio/tests/sync_watch.rs +++ b/tokio/tests/sync_watch.rs @@ -190,3 +190,40 @@ fn poll_close() { assert!(tx.broadcast("two").is_err()); } + +#[test] +fn stream_impl() { + use futures::StreamExt; + + let (tx, mut rx) = watch::channel("one"); + + { + let mut t = spawn(rx.next()); + let v = assert_ready!(t.poll()).unwrap(); + assert_eq!(v, "one"); + } + + { + let mut t = spawn(rx.next()); + + assert_pending!(t.poll()); + + tx.broadcast("two").unwrap(); + + assert!(t.is_woken()); + + let v = assert_ready!(t.poll()).unwrap(); + assert_eq!(v, "two"); + } + + { + let mut t = spawn(rx.next()); + + assert_pending!(t.poll()); + + drop(tx); + + let res = assert_ready!(t.poll()); + assert!(res.is_none()); + } +} diff --git a/tokio/tests/time_interval.rs b/tokio/tests/time_interval.rs index a5f2910b6de..70709f4a8a8 100644 --- a/tokio/tests/time_interval.rs +++ b/tokio/tests/time_interval.rs @@ -42,6 +42,20 @@ async fn usage() { assert_pending!(poll_next(&mut i)); } +#[tokio::test] +async fn usage_stream() { + use futures::StreamExt; + + let start = Instant::now(); + let mut interval = time::interval(ms(10)); + + for _ in 0..3 { + interval.next().await.unwrap(); + } + + assert!(start.elapsed() > ms(20)); +} + fn poll_next(interval: &mut task::Spawn) -> Poll { interval.enter(|cx, mut interval| interval.poll_tick(cx)) } From b419accbfdeb7f741da815aebbb313d63ea63cf0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 13:38:33 -0800 Subject: [PATCH 06/15] fix tests --- tokio-tls/tests/smoke.rs | 6 ++-- tokio-util/tests/framed.rs | 1 + tokio-util/tests/framed_read.rs | 3 +- tokio-util/tests/length_delimited.rs | 3 +- tokio-util/tests/sync_watch.rs | 44 ---------------------------- tokio-util/tests/udp.rs | 8 ++--- tokio/src/sync/mpsc/bounded.rs | 2 +- tokio/tests/rt_common.rs | 2 +- 8 files changed, 13 insertions(+), 56 deletions(-) delete mode 100644 tokio-util/tests/sync_watch.rs diff --git a/tokio-tls/tests/smoke.rs b/tokio-tls/tests/smoke.rs index 7d66424ba21..64dda6ab0b6 100644 --- a/tokio-tls/tests/smoke.rs +++ b/tokio-tls/tests/smoke.rs @@ -524,7 +524,7 @@ async fn client_to_server() { drop(env_logger::try_init()); // Create a server listening on a port, then figure out what that port is - let srv = t!(TcpListener::bind("127.0.0.1:0").await); + let mut srv = t!(TcpListener::bind("127.0.0.1:0").await); let addr = t!(srv.local_addr()); let (server_cx, client_cx) = contexts(); @@ -559,7 +559,7 @@ async fn server_to_client() { drop(env_logger::try_init()); // Create a server listening on a port, then figure out what that port is - let srv = t!(TcpListener::bind("127.0.0.1:0").await); + let mut srv = t!(TcpListener::bind("127.0.0.1:0").await); let addr = t!(srv.local_addr()); let (server_cx, client_cx) = contexts(); @@ -590,7 +590,7 @@ async fn one_byte_at_a_time() { const AMT: usize = 1024; drop(env_logger::try_init()); - let srv = t!(TcpListener::bind("127.0.0.1:0").await); + let mut srv = t!(TcpListener::bind("127.0.0.1:0").await); let addr = t!(srv.local_addr()); let (server_cx, client_cx) = contexts(); diff --git a/tokio-util/tests/framed.rs b/tokio-util/tests/framed.rs index 7aa40d8814b..9add71a2c41 100644 --- a/tokio-util/tests/framed.rs +++ b/tokio-util/tests/framed.rs @@ -4,6 +4,7 @@ use tokio::prelude::*; use tokio_test::assert_ok; use tokio_util::codec::{Decoder, Encoder, Framed, FramedParts}; +use futures::StreamExt; use bytes::{Buf, BufMut, BytesMut, IntoBuf}; use std::io::{self, Read}; use std::pin::Pin; diff --git a/tokio-util/tests/framed_read.rs b/tokio-util/tests/framed_read.rs index 6636b8804cd..9aa644aaac6 100644 --- a/tokio-util/tests/framed_read.rs +++ b/tokio-util/tests/framed_read.rs @@ -1,11 +1,12 @@ #![warn(rust_2018_idioms)] -use tokio::prelude::*; +use tokio::io::AsyncRead; use tokio_test::assert_ready; use tokio_test::task; use tokio_util::codec::{Decoder, FramedRead}; use bytes::{Buf, BytesMut, IntoBuf}; +use futures::Stream; use std::collections::VecDeque; use std::io; use std::pin::Pin; diff --git a/tokio-util/tests/length_delimited.rs b/tokio-util/tests/length_delimited.rs index b287bb357cb..6c5199167be 100644 --- a/tokio-util/tests/length_delimited.rs +++ b/tokio-util/tests/length_delimited.rs @@ -1,7 +1,6 @@ #![warn(rust_2018_idioms)] use tokio::io::{AsyncRead, AsyncWrite}; -use tokio::prelude::*; use tokio_test::task; use tokio_test::{ assert_err, assert_ok, assert_pending, assert_ready, assert_ready_err, assert_ready_ok, @@ -9,7 +8,7 @@ use tokio_test::{ use tokio_util::codec::*; use bytes::{BufMut, Bytes, BytesMut}; -use futures_util::pin_mut; +use futures::{pin_mut, Sink, Stream}; use std::collections::VecDeque; use std::io; use std::pin::Pin; diff --git a/tokio-util/tests/sync_watch.rs b/tokio-util/tests/sync_watch.rs deleted file mode 100644 index 93757acce4a..00000000000 --- a/tokio-util/tests/sync_watch.rs +++ /dev/null @@ -1,44 +0,0 @@ -#![warn(rust_2018_idioms)] - -use tokio::sync::watch; -use tokio_test::task::spawn; -use tokio_test::{assert_pending, assert_ready}; -use tokio_util::stream::Stream; - -use futures::prelude::*; - -#[test] -fn stream_impl() { - let (tx, rx) = watch::channel("one"); - let mut rx = rx.into_std(); - - { - let mut t = spawn(rx.next()); - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(v, "one"); - } - - { - let mut t = spawn(rx.next()); - - assert_pending!(t.poll()); - - tx.broadcast("two").unwrap(); - - assert!(t.is_woken()); - - let v = assert_ready!(t.poll()).unwrap(); - assert_eq!(v, "two"); - } - - { - let mut t = spawn(rx.next()); - - assert_pending!(t.poll()); - - drop(tx); - - let res = assert_ready!(t.poll()); - assert!(res.is_none()); - } -} diff --git a/tokio-util/tests/udp.rs b/tokio-util/tests/udp.rs index 3aac8eaf738..be05dbc4f6e 100644 --- a/tokio-util/tests/udp.rs +++ b/tokio-util/tests/udp.rs @@ -3,10 +3,10 @@ use tokio_util::codec::{Decoder, Encoder}; use tokio_util::udp::UdpFramed; use bytes::{BufMut, BytesMut}; -use futures_util::future::try_join; -use futures_util::future::FutureExt; -use futures_util::sink::SinkExt; -use futures_util::stream::StreamExt; +use futures::future::try_join; +use futures::future::FutureExt; +use futures::sink::SinkExt; +use futures::stream::StreamExt; use std::io; #[tokio::test] diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 8c84eee4234..523dde75bcd 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -218,7 +218,7 @@ impl Sender { pub async fn send(&mut self, value: T) -> Result<(), SendError> { use crate::future::poll_fn; - if let Err(_) = poll_fn(|cx| self.poll_ready(cx)).await { + if poll_fn(|cx| self.poll_ready(cx)).await.is_err() { return Err(SendError(value)); } diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index 06b8a1c4f3d..2637793ac7f 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -133,7 +133,7 @@ rt_test! { let mut txs = (0..ITER) .map(|i| { let (tx, rx) = oneshot::channel(); - let mut done_tx = done_tx.clone(); + let done_tx = done_tx.clone(); tokio::spawn(async move { let msg = assert_ok!(rx.await); From 9671b4cac0240125b56bc0ce0a3f9a642b1b3a25 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 13:40:28 -0800 Subject: [PATCH 07/15] fmt --- tokio-util/tests/framed.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio-util/tests/framed.rs b/tokio-util/tests/framed.rs index 9add71a2c41..25ddf86a1d8 100644 --- a/tokio-util/tests/framed.rs +++ b/tokio-util/tests/framed.rs @@ -4,8 +4,8 @@ use tokio::prelude::*; use tokio_test::assert_ok; use tokio_util::codec::{Decoder, Encoder, Framed, FramedParts}; -use futures::StreamExt; use bytes::{Buf, BufMut, BytesMut, IntoBuf}; +use futures::StreamExt; use std::io::{self, Read}; use std::pin::Pin; use std::task::{Context, Poll}; From b6791cdb26d11f75ac5860c5b865c24da0267669 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 14:01:35 -0800 Subject: [PATCH 08/15] try to fix ci again --- tokio-util/src/codec/length_delimited.rs | 1 + tokio/src/lib.rs | 2 -- tokio/src/signal/windows.rs | 1 - tokio/src/stream/mod.rs | 10 ---------- 4 files changed, 1 insertion(+), 13 deletions(-) delete mode 100644 tokio/src/stream/mod.rs diff --git a/tokio-util/src/codec/length_delimited.rs b/tokio-util/src/codec/length_delimited.rs index df07466eb35..b36b6aa7f52 100644 --- a/tokio-util/src/codec/length_delimited.rs +++ b/tokio-util/src/codec/length_delimited.rs @@ -42,6 +42,7 @@ //! use tokio::prelude::*; //! use tokio_util::codec::{Framed, LengthDelimitedCodec}; //! +//! use futures::SinkExt; //! use bytes::Bytes; //! //! async fn write_frame(io: T) -> Result<(), Box> diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index 0a651356e84..4be056ecb49 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -116,8 +116,6 @@ pub mod runtime; #[cfg(not(loom))] pub mod signal; -mod stream; - #[cfg(feature = "sync")] pub mod sync; diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index 78795b187e5..10288c86139 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -13,7 +13,6 @@ use crate::sync::mpsc::{channel, Receiver}; use std::convert::TryFrom; use std::io; use std::sync::Once; -use std::task::{Context, Poll}; use winapi::shared::minwindef::*; use winapi::um::consoleapi::SetConsoleCtrlHandler; use winapi::um::wincon::*; diff --git a/tokio/src/stream/mod.rs b/tokio/src/stream/mod.rs deleted file mode 100644 index 27c3d9154a2..00000000000 --- a/tokio/src/stream/mod.rs +++ /dev/null @@ -1,10 +0,0 @@ -use std::pin::Pin; -use std::task::{Context, Poll}; - -/// Asynchronous iteration of values. -#[allow(unreachable_pub)] -pub trait Stream { - type Item; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll>; -} From 97371c6340371af13af9b19e0a23c45c9a7afa9f Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 14:26:37 -0800 Subject: [PATCH 09/15] try to fix ci again --- examples/chat.rs | 16 +++++----------- examples/connect.rs | 10 ++++++---- examples/print_each_packet.rs | 2 +- examples/proxy.rs | 6 +++--- examples/udp-codec.rs | 5 ++--- tokio/src/signal/windows.rs | 2 +- 6 files changed, 18 insertions(+), 23 deletions(-) diff --git a/examples/chat.rs b/examples/chat.rs index e0213afd08a..2553cc5edfe 100644 --- a/examples/chat.rs +++ b/examples/chat.rs @@ -116,18 +116,12 @@ impl Shared { /// Send a `LineCodec` encoded message to every peer, except /// for the sender. - async fn broadcast( - &mut self, - sender: SocketAddr, - message: &str, - ) -> Result<(), mpsc::error::UnboundedSendError> { + async fn broadcast(&mut self, sender: SocketAddr, message: &str) { for peer in self.peers.iter_mut() { if *peer.0 != sender { - peer.1.send(message.into()).await?; + let _ = peer.1.send(message.into()); } } - - Ok(()) } } @@ -218,7 +212,7 @@ async fn process( let mut state = state.lock().await; let msg = format!("{} has joined the chat", username); println!("{}", msg); - state.broadcast(addr, &msg).await?; + state.broadcast(addr, &msg).await; } // Process incoming messages until our stream is exhausted by a disconnect. @@ -230,7 +224,7 @@ async fn process( let mut state = state.lock().await; let msg = format!("{}: {}", username, msg); - state.broadcast(addr, &msg).await?; + state.broadcast(addr, &msg).await; } // A message was received from a peer. Send it to the // current user. @@ -254,7 +248,7 @@ async fn process( let msg = format!("{} has left the chat", username); println!("{}", msg); - state.broadcast(addr, &msg).await?; + state.broadcast(addr, &msg).await; } Ok(()) diff --git a/examples/connect.rs b/examples/connect.rs index 38d81229455..cb003d9de08 100644 --- a/examples/connect.rs +++ b/examples/connect.rs @@ -20,7 +20,7 @@ use tokio::io; use tokio::sync::{mpsc, oneshot}; use tokio_util::codec::{FramedRead, FramedWrite}; -use futures::{SinkExt, Stream, StreamExt}; +use futures::{Stream, StreamExt}; use std::env; use std::error::Error; use std::net::SocketAddr; @@ -69,12 +69,14 @@ async fn run() -> Result<(), Box> { // Temporary work around for stdin blocking the stream fn stdin() -> impl Stream, io::Error>> + Unpin { - let mut stdin = FramedRead::new(io::stdin(), codec::Bytes).map(Ok); + let mut stdin = FramedRead::new(io::stdin(), codec::Bytes); - let (mut tx, rx) = mpsc::unbounded_channel(); + let (tx, rx) = mpsc::unbounded_channel(); tokio::spawn(async move { - tx.send_all(&mut stdin).await.unwrap(); + while let Some(res) = stdin.next().await { + let _ = tx.send(res); + } }); rx diff --git a/examples/print_each_packet.rs b/examples/print_each_packet.rs index 0a27554574c..f056db4abee 100644 --- a/examples/print_each_packet.rs +++ b/examples/print_each_packet.rs @@ -55,9 +55,9 @@ #![warn(rust_2018_idioms)] use tokio::net::TcpListener; -use tokio::prelude::*; use tokio_util::codec::{BytesCodec, Decoder}; +use futures::StreamExt; use std::env; #[tokio::main] diff --git a/examples/proxy.rs b/examples/proxy.rs index 6886a813b5e..4314d1b9551 100644 --- a/examples/proxy.rs +++ b/examples/proxy.rs @@ -22,7 +22,7 @@ #![warn(rust_2018_idioms)] -use futures::{future::try_join, FutureExt, StreamExt}; +use futures::{future::try_join, FutureExt}; use std::{env, error::Error}; use tokio::{ io::AsyncReadExt, @@ -37,9 +37,9 @@ async fn main() -> Result<(), Box> { println!("Listening on: {}", listen_addr); println!("Proxying to: {}", server_addr); - let mut incoming = TcpListener::bind(listen_addr).await?.incoming(); + let mut listener = TcpListener::bind(listen_addr).await?; - while let Some(Ok(inbound)) = incoming.next().await { + while let Ok((inbound, _)) = listener.accept().await { let transfer = transfer(inbound, server_addr.clone()).map(|r| { if let Err(e) = r { println!("Failed to transfer; error={}", e); diff --git a/examples/udp-codec.rs b/examples/udp-codec.rs index baf6488675e..0c9dbf76eb9 100644 --- a/examples/udp-codec.rs +++ b/examples/udp-codec.rs @@ -8,9 +8,8 @@ #![warn(rust_2018_idioms)] -use tokio::future::FutureExt as TokioFutureExt; -use tokio::io; use tokio::net::UdpSocket; +use tokio::{io, time}; use tokio_util::codec::BytesCodec; use tokio_util::udp::UdpFramed; @@ -68,7 +67,7 @@ async fn ping(socket: &mut UdpFramed, b_addr: SocketAddr) -> Result< async fn pong(socket: &mut UdpFramed) -> Result<(), io::Error> { let timeout = Duration::from_millis(200); - while let Ok(Some(Ok((bytes, addr)))) = socket.next().timeout(timeout).await { + while let Ok(Some(Ok((bytes, addr)))) = time::timeout(timeout, socket.next()).await { println!("[b] recv: {}", String::from_utf8_lossy(&bytes)); socket.send((Bytes::from(&b"PONG"[..]), addr)).await?; diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index 10288c86139..fb56efc1dc1 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -163,7 +163,7 @@ mod tests { use super::*; use crate::runtime::Runtime; - use futures_util::stream::StreamExt; + use futures::stream::StreamExt; #[test] fn ctrl_c() { From 39ea1914f82a467a7c13a773afe46b9a9ce32708 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 14:38:59 -0800 Subject: [PATCH 10/15] try to fix windows again --- tokio/src/signal/windows.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index fb56efc1dc1..c0c3c2b394c 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -162,6 +162,7 @@ pub fn ctrl_break() -> io::Result { mod tests { use super::*; use crate::runtime::Runtime; + use tokio_test::assert_ok; use futures::stream::StreamExt; @@ -170,7 +171,7 @@ mod tests { let mut rt = rt(); rt.block_on(async { - let ctrl_c = crate::signal::ctrl_c().expect("failed to create CtrlC"); + let ctrl_c = crate::signal::ctrl_c(); // Windows doesn't have a good programmatic way of sending events // like sending signals on Unix, so we'll stub out the actual OS @@ -179,7 +180,7 @@ mod tests { super::handler(CTRL_C_EVENT); } - let _ = ctrl_c.into_future().await; + assert_ok!(ctrl_c.await); }); } @@ -188,7 +189,7 @@ mod tests { let mut rt = rt(); rt.block_on(async { - let ctrl_break = super::ctrl_break().expect("failed to create CtrlC"); + let ctrl_break = super::ctrl_break(); // Windows doesn't have a good programmatic way of sending events // like sending signals on Unix, so we'll stub out the actual OS @@ -197,7 +198,7 @@ mod tests { super::handler(CTRL_BREAK_EVENT); } - let _ = ctrl_break.into_future().await; + assert_ok!(ctrl_break.await); }); } From 0d80e872cfc6be907fad0f0382d4f9d873dbc2dc Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 14:59:44 -0800 Subject: [PATCH 11/15] try again --- tokio/src/signal/unix.rs | 2 +- tokio/src/signal/windows.rs | 31 +++++++++++++++++++++++-------- 2 files changed, 24 insertions(+), 9 deletions(-) diff --git a/tokio/src/signal/unix.rs b/tokio/src/signal/unix.rs index 17cb2e83578..8551e85cecd 100644 --- a/tokio/src/signal/unix.rs +++ b/tokio/src/signal/unix.rs @@ -392,7 +392,7 @@ pub fn signal(kind: SignalKind) -> io::Result { } impl Signal { - /// TODO: Dox + #[doc(hidden)] // TODO: Dox pub async fn recv(&mut self) -> Option<()> { use crate::future::poll_fn; poll_fn(|cx| self.poll_recv(cx)).await diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index c0c3c2b394c..39b46f3c736 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -13,6 +13,7 @@ use crate::sync::mpsc::{channel, Receiver}; use std::convert::TryFrom; use std::io; use std::sync::Once; +use std::task::{Context, Poll}; use winapi::shared::minwindef::*; use winapi::um::consoleapi::SetConsoleCtrlHandler; use winapi::um::wincon::*; @@ -79,6 +80,10 @@ pub(crate) struct Event { rx: Receiver<()>, } +pub(crate) fn ctrl_c() -> io::Result { + Event::new(CTRL_C_EVENT) +} + impl Event { fn new(signum: DWORD) -> io::Result { global_init()?; @@ -88,13 +93,7 @@ impl Event { Ok(Event { rx }) } -} - -pub(crate) fn ctrl_c() -> io::Result { - Event::new(CTRL_C_EVENT) -} -impl Event { pub(crate) async fn recv(&mut self) -> Option<()> { use crate::future::poll_fn; poll_fn(|cx| self.rx.poll_recv(cx)).await @@ -150,6 +149,22 @@ pub struct CtrlBreak { inner: Event, } +impl CtrlBreak { + #[doc(hidden)] // TODO: document + pub fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { + self.inner.rx.poll_recv(cx) + } +} + +#[cfg(feature = "stream")] +impl futures_core::Stream for CtrlBreak { + type Item = (); + + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_recv(cx) + } +} + /// Creates a new stream which receives "ctrl-break" notifications sent to the /// process. /// @@ -189,7 +204,7 @@ mod tests { let mut rt = rt(); rt.block_on(async { - let ctrl_break = super::ctrl_break(); + let mut ctrl_break = assert_ok!(super::ctrl_break()); // Windows doesn't have a good programmatic way of sending events // like sending signals on Unix, so we'll stub out the actual OS @@ -198,7 +213,7 @@ mod tests { super::handler(CTRL_BREAK_EVENT); } - assert_ok!(ctrl_break.await); + assert_ok!(ctrl_break.next().await.unwrap()); }); } From 53278d584a5c5ee4d54bf0c63895dab2209194ef Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 15:08:17 -0800 Subject: [PATCH 12/15] and again --- tokio/src/signal/windows.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index 39b46f3c736..dc00129a9cf 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -213,7 +213,7 @@ mod tests { super::handler(CTRL_BREAK_EVENT); } - assert_ok!(ctrl_break.next().await.unwrap()); + ctrl_break.next().await.unwrap(); }); } From ab2bbcdd8f4ab1fb406c5edc1a4b2c154d67b0d0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 20:30:12 -0800 Subject: [PATCH 13/15] fix windows (hopefully for real, i ran it locally this time) --- tokio/src/runtime/mod.rs | 8 ++++++++ tokio/src/signal/registry.rs | 4 +++- tokio/src/signal/windows.rs | 12 +++++++----- 3 files changed, 18 insertions(+), 6 deletions(-) diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 615c7074981..686a5c262e0 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -334,6 +334,14 @@ impl Runtime { }) } + /// Enter the runtime context + pub fn enter(&self, f: F) -> R + where + F: FnOnce() -> R, + { + self.handle.enter(f) + } + /// Return a handle to the runtime's spawner. /// /// The returned handle can be used to spawn tasks that run on this runtime. diff --git a/tokio/src/signal/registry.rs b/tokio/src/signal/registry.rs index 0e017965a1d..56429cdb691 100644 --- a/tokio/src/signal/registry.rs +++ b/tokio/src/signal/registry.rs @@ -104,7 +104,9 @@ impl Registry { // has gone away then we can remove that slot. for i in (0..recipients.len()).rev() { match recipients[i].try_send(()) { - Ok(()) => did_notify = true, + Ok(()) => { + did_notify = true + } Err(TrySendError::Closed(..)) => { recipients.swap_remove(i); } diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index dc00129a9cf..19d6d523740 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -177,16 +177,18 @@ pub fn ctrl_break() -> io::Result { mod tests { use super::*; use crate::runtime::Runtime; - use tokio_test::assert_ok; + use tokio_test::{task, assert_ok, assert_ready_ok, assert_pending}; use futures::stream::StreamExt; #[test] fn ctrl_c() { - let mut rt = rt(); + let rt = rt(); - rt.block_on(async { - let ctrl_c = crate::signal::ctrl_c(); + rt.enter(|| { + let mut ctrl_c = task::spawn(crate::signal::ctrl_c()); + + assert_pending!(ctrl_c.poll()); // Windows doesn't have a good programmatic way of sending events // like sending signals on Unix, so we'll stub out the actual OS @@ -195,7 +197,7 @@ mod tests { super::handler(CTRL_C_EVENT); } - assert_ok!(ctrl_c.await); + assert_ready_ok!(ctrl_c.poll()); }); } From ac047a26f51b41eef91ec13dd45976ab9d684f07 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 20:41:36 -0800 Subject: [PATCH 14/15] dox --- tokio/src/signal/ctrl_c.rs | 28 +++++++++++++++++++++------- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/tokio/src/signal/ctrl_c.rs b/tokio/src/signal/ctrl_c.rs index ed393b1dc0d..35ef2393568 100644 --- a/tokio/src/signal/ctrl_c.rs +++ b/tokio/src/signal/ctrl_c.rs @@ -7,14 +7,28 @@ use std::io; /// Completes when a "ctrl-c" notification is sent to the process. /// -/// In general signals are handled very differently across Unix and Windows, but -/// this is somewhat cross platform in terms of how it can be handled. A ctrl-c -/// event to a console process can be represented as a stream for both Windows -/// and Unix. +/// While signals are handled very differently between Unix and Windows, both +/// platforms support receiving a signal on "ctrl-c". This function provides a +/// portable API for receiving this notification. /// -/// Note that there are a number of caveats listening for signals, and you may -/// wish to read up on the documentation in the `unix` or `windows` module to -/// take a peek. +/// Once the returned future is polled, a listener a listener is registered. The +/// future will complete on the first received `ctrl-c` **after** the initial +/// call to either `Future::poll` or `.await`. +/// +/// # Examples +/// +/// ```rust,no_run +/// use tokio::signal; +/// +/// #[tokio::main] +/// async fn main() { +/// println!("waiting for ctrl-c"); +/// +/// signal::ctrl_c().await.expect("failed to listen for event"); +/// +/// println!("received ctrl-c event"); +/// } +/// ``` pub async fn ctrl_c() -> io::Result<()> { os_impl::ctrl_c()?.recv().await; Ok(()) From 5b962cce482f37bbc73fdcf59e0dd022e75f53f7 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 15 Nov 2019 21:50:49 -0800 Subject: [PATCH 15/15] fmt --- tokio/src/signal/registry.rs | 4 +--- tokio/src/signal/windows.rs | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/tokio/src/signal/registry.rs b/tokio/src/signal/registry.rs index 56429cdb691..0e017965a1d 100644 --- a/tokio/src/signal/registry.rs +++ b/tokio/src/signal/registry.rs @@ -104,9 +104,7 @@ impl Registry { // has gone away then we can remove that slot. for i in (0..recipients.len()).rev() { match recipients[i].try_send(()) { - Ok(()) => { - did_notify = true - } + Ok(()) => did_notify = true, Err(TrySendError::Closed(..)) => { recipients.swap_remove(i); } diff --git a/tokio/src/signal/windows.rs b/tokio/src/signal/windows.rs index 19d6d523740..6758566ae05 100644 --- a/tokio/src/signal/windows.rs +++ b/tokio/src/signal/windows.rs @@ -177,7 +177,7 @@ pub fn ctrl_break() -> io::Result { mod tests { use super::*; use crate::runtime::Runtime; - use tokio_test::{task, assert_ok, assert_ready_ok, assert_pending}; + use tokio_test::{assert_ok, assert_pending, assert_ready_ok, task}; use futures::stream::StreamExt;