From 7eee2ecf7d93570ea48062478a3e8239df3e01c5 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 4 Nov 2019 14:22:31 -0800 Subject: [PATCH 01/73] first pass on local task set Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 218 ++++++++++++++++++++++++++++++++++ tokio/src/runtime/mod.rs | 2 + tokio/src/runtime/task/mod.rs | 40 +++++++ 3 files changed, 260 insertions(+) create mode 100644 tokio/src/runtime/local.rs diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs new file mode 100644 index 00000000000..0846cf9563c --- /dev/null +++ b/tokio/src/runtime/local.rs @@ -0,0 +1,218 @@ +//! Groups a set of tasks that execute on the same thread. +use crate::executor::park::{Park, Unpark}; +use crate::executor::task::{self, JoinHandle, Schedule, Task}; + +use std::cell::UnsafeCell; +use std::collections::VecDeque; +use std::fmt; +use std::future::Future; +use std::mem::ManuallyDrop; +use std::ptr::{self, NonNull}; +use std::sync::{Arc, Mutex}; +use std::task::Poll; +use std::time::Duration; + +use pin_project::{pin_project, project}; + +/// A group of tasks which are executed on the same thread. +/// +/// These tasks need not implement `Send`; a local task set provides the +/// capacity to execute `!Send` futures. +#[derive(Debug)] +#[pin_project] +pub struct TaskSet { + scheduler: Scheduler, + #[pin] + future: F, +} + +struct Scheduler { + /// List of all active tasks spawned onto this executor. + /// + /// # Safety + /// + /// Must only be accessed from the primary thread + owned_tasks: UnsafeCell>, + + /// Local run queue. + /// + /// Tasks notified from the current thread are pushed into this queue. + /// + /// # Safety + /// + /// References should not be handed out. Only call `push` / `pop` functions. + /// Only call from the owning thread. + queue: UnsafeCell>>, +} + +thread_local! { + static CURRENT_TASK_SET: Cell>> = Cell::new(None); +} + +/// Returns a local task set for the given future. +/// +/// The provided future may call `spawn_local`. +pub fn task_set(future: F) -> TaskSet +where + F: Future + 'static, + F::Output: 'static, +{ + TaskSet::new(f) +} + +/// Spawns a `!Send` future on the local task set. +/// +/// The spawned future will be run on the same thread that called `spawn_local.` +/// This may only be called from the context of a local task set. +/// +/// # Panics +/// - This function panics if called outside of a local task set. +pub fn spawn_local(future: F) -> JoinHandle +where + F: Future + 'static, + F::Output: 'static, +{ + CURRENT_TASK_SET.with(|current| { + let current = current + .get() + .expect("`local::spawn` called from outside of a local::TaskSet!"); + unsafe { + let scheduler = scheduler.as_ref(); + let (task, handle) = task::joinable_unsend(future); + set.schedule(task); + handle + } + }) +} + +/// Max number of tasks to poll per tick. +const MAX_TASKS_PER_TICK: usize = 61; + +impl TaskSet +where + F: Future + 'static, + F::Output: 'static, +{ + /// Returns a new local task set for the given future. + pub fn new(future: F) -> Self { + Self { + scheduler: Scheduler::new(), + future, + } + } +} + +impl Future for TaskSet { + type Output = F::Output; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let TaskSet { scheduler, future } = self.project(); + scheduler.with(|| { + if let Poll::Ready(v) = future.poll(&mut cx) { + return Poll::Ready(v); + } + + scheduler.tick(local); + + match future.poll(&mut cx) { + Poll::Ready(v) => Poll::Ready(v), + Poll::Pending => { + cx.waker().wake(); + Poll::Pending + } + } + }) + } +} + +// === impl Scheduler === + +impl Schedule for Scheduler { + fn bind(&self, task: &Task) { + assert!(self.is_current()); + unsafe { + (*self.owned_tasks.get()).insert(task); + } + } + + fn release(&self, _: Task) { + unreachable!("tasks should only be completed locally") + } + + fn release_local(&self, task: &Task) { + assert!(self.is_current()); + unsafe { + (*self.owned_tasks.get()).remove(task); + } + } + + fn schedule(&self, task: Task) { + assert!(self.is_current()); + unsafe { + (*self.queue.get()).push_front(task); + } + } +} + +impl Scheduler { + fn new() -> Self { + Self { + owned_tasks: UnsafeCell::new(task::OwnedList::new()), + queue: UnsafeCell::new(VecDeque::with_capacity(64)), + } + } + + fn with(&mut self, f: impl FnOnce() -> F) -> F { + struct Entered<'a> { + current: &'a Cell>>, + } + + impl<'a> Drop for Entered<'a> { + fn drop(&mut self) { + self.current.set(None); + } + } + + CURRENT_TASK_SET.with(|current| { + current.set(Some(NonNull::from(self))); + let _entered = Entered { current }; + f() + }) + } + + fn is_current(&self) -> bool { + CURRENT_TASK_SET + .try_with(|current| { + current + .get() + .iter() + .any(|current| ptr::eq(current.as_ptr(), self as *const _)) + }) + .unwrap_or(false) + } + + fn next_task(&self) -> Option> { + unsafe { (*self.task.get()).pop_front() } + } + + fn tick(&self) { + for _ in 0..MAX_TASKS_PER_TICK { + let task = match self.next_task() { + Some(task) => task, + None => return, + }; + + if let Some(task) = task.run(&mut || Some(self.into())) { + unsafe { + self.schedule_local(task); + } + } + } + } +} + +impl fmt::Debug for Scheduler { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Scheduler { .. }").finish() + } +} diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 9dbc857aafb..ef27ff30187 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -154,6 +154,8 @@ pub use self::global::spawn; mod io; +pub mod local; + mod park; pub use self::park::{Park, Unpark}; diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 7e361e4fb80..ee255d1098b 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -95,6 +95,46 @@ where (task, join) } +/// Create a new `!Send` task without an associated join handle +/// +/// # Safety +/// +/// The returned `Task` will implement `Send`, even though it was constructed +/// from an `!Send` future. Thus, it may not be sent between threads! +pub(super) unsafe fn background_unsend(task: T) -> Task +where + T: Future + 'static, + S: Schedule, +{ + Task { + raw: RawTask::new_background::<_, S>(task), + _p: PhantomData, + } +} + +/// Create a new `!Send` task with an associated join handle +/// +/// # Safety +/// +/// The returned `Task` will implement `Send`, even though it was constructed +/// from an `!Send` future. Thus, it may not be sent between threads! +pub(super) unsafe fn joinable_unsend(task: T) -> (Task, JoinHandle) +where + T: Future + 'static, + S: Schedule, +{ + let raw = RawTask::new_joinable::<_, S>(task); + + let task = Task { + raw, + _p: PhantomData, + }; + + let join = JoinHandle::new(raw); + + (task, join) +} + impl Task { pub(crate) unsafe fn from_raw(ptr: NonNull
) -> Task { Task { From 366ecbda525e508133952cb9d57b4fefc6b28df8 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 4 Nov 2019 15:09:00 -0800 Subject: [PATCH 02/73] it works! Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 180 +++++++++++++++++++++++++++++----- tokio/src/runtime/task/mod.rs | 40 -------- 2 files changed, 155 insertions(+), 65 deletions(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 0846cf9563c..37bfe67bbcf 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -1,29 +1,28 @@ //! Groups a set of tasks that execute on the same thread. -use crate::executor::park::{Park, Unpark}; use crate::executor::task::{self, JoinHandle, Schedule, Task}; -use std::cell::UnsafeCell; +use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; use std::fmt; use std::future::Future; -use std::mem::ManuallyDrop; +use std::marker::PhantomData; +use std::pin::Pin; use std::ptr::{self, NonNull}; -use std::sync::{Arc, Mutex}; -use std::task::Poll; -use std::time::Duration; +use std::task::{Context, Poll}; -use pin_project::{pin_project, project}; +use pin_project::pin_project; /// A group of tasks which are executed on the same thread. /// /// These tasks need not implement `Send`; a local task set provides the /// capacity to execute `!Send` futures. -#[derive(Debug)] #[pin_project] +#[derive(Debug)] pub struct TaskSet { scheduler: Scheduler, #[pin] future: F, + _not_send_or_sync: PhantomData<*const ()>, } struct Scheduler { @@ -57,7 +56,7 @@ where F: Future + 'static, F::Output: 'static, { - TaskSet::new(f) + TaskSet::new(future) } /// Spawns a `!Send` future on the local task set. @@ -66,20 +65,53 @@ where /// This may only be called from the context of a local task set. /// /// # Panics +/// /// - This function panics if called outside of a local task set. +/// +/// # Examples +/// +/// ```rust +/// # use tokio::runtime::Runtime; +/// use std::rc::Rc; +/// use tokio::executor::local; +/// let unsync_data = Rc::new("my unsync data..."); +/// +/// let mut rt = Runtime::new().unwrap(); +/// rt.block_on(local::task_set(async move { +/// let more_unsync_data = unsync_data.clone(); +/// local::spawn_local(async move { +/// println!("{}", more_unsync_data); +/// // ... +/// }).await.unwrap(); +/// })); +/// ``` pub fn spawn_local(future: F) -> JoinHandle where F: Future + 'static, F::Output: 'static, { + /// EXTREMELY UNSAFE type for pretending a task is Send. Don't use this elsewhere. + #[pin_project] + struct Unsend(#[pin] T); + + unsafe impl Send for Unsend {} + + impl Future for Unsend { + type Output = F::Output; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.project(); + this.0.poll(cx) + } + } + CURRENT_TASK_SET.with(|current| { let current = current .get() .expect("`local::spawn` called from outside of a local::TaskSet!"); unsafe { - let scheduler = scheduler.as_ref(); - let (task, handle) = task::joinable_unsend(future); - set.schedule(task); + let (task, handle) = task::joinable(Unsend(future)); + current.as_ref().schedule(task); handle } }) @@ -98,6 +130,7 @@ where Self { scheduler: Scheduler::new(), future, + _not_send_or_sync: PhantomData, } } } @@ -106,18 +139,17 @@ impl Future for TaskSet { type Output = F::Output; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let TaskSet { scheduler, future } = self.project(); - scheduler.with(|| { - if let Poll::Ready(v) = future.poll(&mut cx) { - return Poll::Ready(v); - } + let this = self.project(); + let scheduler = this.scheduler; + let future = this.future; - scheduler.tick(local); + scheduler.with(|| { + scheduler.tick(); - match future.poll(&mut cx) { + match future.poll(cx) { Poll::Ready(v) => Poll::Ready(v), Poll::Pending => { - cx.waker().wake(); + cx.waker().wake_by_ref(); Poll::Pending } } @@ -162,7 +194,7 @@ impl Scheduler { } } - fn with(&mut self, f: impl FnOnce() -> F) -> F { + fn with(&self, f: impl FnOnce() -> F) -> F { struct Entered<'a> { current: &'a Cell>>, } @@ -192,7 +224,7 @@ impl Scheduler { } fn next_task(&self) -> Option> { - unsafe { (*self.task.get()).pop_front() } + unsafe { (*self.queue.get()).pop_front() } } fn tick(&self) { @@ -203,16 +235,114 @@ impl Scheduler { }; if let Some(task) = task.run(&mut || Some(self.into())) { - unsafe { - self.schedule_local(task); - } + self.schedule(task); } } } } +unsafe impl Send for Scheduler {} +unsafe impl Sync for Scheduler {} + impl fmt::Debug for Scheduler { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Scheduler { .. }").finish() } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::runtime; + + #[test] + fn local_current_thread() { + let mut rt = runtime::Builder::new().current_thread().build().unwrap(); + let task_set = task_set(async { + spawn_local(async {}).await.unwrap(); + }); + rt.block_on(task_set); + } + + #[test] + fn local_threadpool() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + let task_set = task_set(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + .await + .unwrap(); + }); + rt.block_on(task_set); + } + + #[test] + fn all_spawn_locals_are_local() { + use futures_util::future; + + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + let task_set = task_set(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + let handles = (0..128) + .map(|_| { + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + }) + .collect::>(); + for result in future::join_all(handles).await { + result.unwrap(); + } + }); + rt.block_on(task_set); + } + + #[test] + fn nested_spawn_local_is_local() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + let task_set = task_set(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }); + rt.block_on(task_set); + } +} diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index ee255d1098b..7e361e4fb80 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -95,46 +95,6 @@ where (task, join) } -/// Create a new `!Send` task without an associated join handle -/// -/// # Safety -/// -/// The returned `Task` will implement `Send`, even though it was constructed -/// from an `!Send` future. Thus, it may not be sent between threads! -pub(super) unsafe fn background_unsend(task: T) -> Task -where - T: Future + 'static, - S: Schedule, -{ - Task { - raw: RawTask::new_background::<_, S>(task), - _p: PhantomData, - } -} - -/// Create a new `!Send` task with an associated join handle -/// -/// # Safety -/// -/// The returned `Task` will implement `Send`, even though it was constructed -/// from an `!Send` future. Thus, it may not be sent between threads! -pub(super) unsafe fn joinable_unsend(task: T) -> (Task, JoinHandle) -where - T: Future + 'static, - S: Schedule, -{ - let raw = RawTask::new_joinable::<_, S>(task); - - let task = Task { - raw, - _p: PhantomData, - }; - - let join = JoinHandle::new(raw); - - (task, join) -} - impl Task { pub(crate) unsafe fn from_raw(ptr: NonNull
) -> Task { Task { From 82857dd2fac659ca5ddf92a25acc2e7b38e65a0d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 4 Nov 2019 16:10:16 -0800 Subject: [PATCH 03/73] change API to work with multiple block ons Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 102 +++++++++++++++++++------------------ 1 file changed, 53 insertions(+), 49 deletions(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 37bfe67bbcf..466458ac1cc 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -8,6 +8,7 @@ use std::future::Future; use std::marker::PhantomData; use std::pin::Pin; use std::ptr::{self, NonNull}; +use std::rc::Rc; use std::task::{Context, Poll}; use pin_project::pin_project; @@ -16,12 +17,9 @@ use pin_project::pin_project; /// /// These tasks need not implement `Send`; a local task set provides the /// capacity to execute `!Send` futures. -#[pin_project] #[derive(Debug)] -pub struct TaskSet { - scheduler: Scheduler, - #[pin] - future: F, +pub struct TaskSet { + scheduler: Rc, _not_send_or_sync: PhantomData<*const ()>, } @@ -44,19 +42,15 @@ struct Scheduler { queue: UnsafeCell>>, } -thread_local! { - static CURRENT_TASK_SET: Cell>> = Cell::new(None); +#[pin_project] +struct LocalFuture { + scheduler: Rc, + #[pin] + future: F, } -/// Returns a local task set for the given future. -/// -/// The provided future may call `spawn_local`. -pub fn task_set(future: F) -> TaskSet -where - F: Future + 'static, - F::Output: 'static, -{ - TaskSet::new(future) +thread_local! { + static CURRENT_TASK_SET: Cell>> = Cell::new(None); } /// Spawns a `!Send` future on the local task set. @@ -90,21 +84,6 @@ where F: Future + 'static, F::Output: 'static, { - /// EXTREMELY UNSAFE type for pretending a task is Send. Don't use this elsewhere. - #[pin_project] - struct Unsend(#[pin] T); - - unsafe impl Send for Unsend {} - - impl Future for Unsend { - type Output = F::Output; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.project(); - this.0.poll(cx) - } - } - CURRENT_TASK_SET.with(|current| { let current = current .get() @@ -116,26 +95,54 @@ where } }) } +/// EXTREMELY UNSAFE type for pretending a task is Send. Don't use this elsewhere. +#[pin_project] +struct Unsend(#[pin] T); + +unsafe impl Send for Unsend {} + +impl Future for Unsend { + type Output = F::Output; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.project(); + this.0.poll(cx) + } +} /// Max number of tasks to poll per tick. const MAX_TASKS_PER_TICK: usize = 61; -impl TaskSet -where - F: Future + 'static, - F::Output: 'static, -{ +impl TaskSet { /// Returns a new local task set for the given future. - pub fn new(future: F) -> Self { + pub fn new() -> Self { Self { - scheduler: Scheduler::new(), - future, + scheduler: Rc::new(Scheduler::new()), _not_send_or_sync: PhantomData, } } + + pub fn spawn_local(&self, future: F) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + let (task, handle) = task::joinable(Unsend(future)); + self.scheduler.schedule(task); + handle + } + + pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output + where + F: Future + 'static, + F::Output: 'static, + { + let scheduler = self.scheduler.clone(); + rt.block_on(LocalFuture { scheduler, future }) + } } -impl Future for TaskSet { +impl Future for LocalFuture { type Output = F::Output; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -258,10 +265,9 @@ mod tests { #[test] fn local_current_thread() { let mut rt = runtime::Builder::new().current_thread().build().unwrap(); - let task_set = task_set(async { + TaskSet::new().block_on(&mut rt, async { spawn_local(async {}).await.unwrap(); }); - rt.block_on(task_set); } #[test] @@ -273,7 +279,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - let task_set = task_set(async { + TaskSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -295,7 +301,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - let task_set = task_set(async { + TaskSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) .map(|_| { @@ -307,8 +313,7 @@ mod tests { for result in future::join_all(handles).await { result.unwrap(); } - }); - rt.block_on(task_set); + }) } #[test] @@ -320,7 +325,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - let task_set = task_set(async { + TaskSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -342,7 +347,6 @@ mod tests { }) .await .unwrap(); - }); - rt.block_on(task_set); + }) } } From edd8ca4f97e452403de4a69188b239b7840ebc95 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 4 Nov 2019 18:06:16 -0800 Subject: [PATCH 04/73] shutdown on drop Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 466458ac1cc..2589c0464b2 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -256,6 +256,19 @@ impl fmt::Debug for Scheduler { fmt.debug_struct("Scheduler { .. }").finish() } } +impl Drop for Scheduler { + fn drop(&mut self) { + // Drain all local tasks + while let Some(task) = self.next_task() { + task.shutdown(); + } + + // Release owned tasks + unsafe { + (*self.tasks.get()).shutdown(); + } + } +} #[cfg(test)] mod tests { @@ -287,7 +300,6 @@ mod tests { .await .unwrap(); }); - rt.block_on(task_set); } #[test] From 2e6cf55fa72c8d0520df4fd4b7d772a29c3b0d65 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 10:59:02 -0800 Subject: [PATCH 05/73] use marker types to indicate whether a task is Send This adds a marker type to the `Task` struct and `Schedule` trait indicating whether or not the task is `Send`, and changes `Task` to only implement `Send` when the `Send` marker is present. Schedulers must indicate whether they are capable of scheduling `Send` or `!Send` tasks. This solution is singnificantly better than the previous one of using an unsafe wrapper to make `!Send` tasks implement `Send`, which was necessary due to the bounds on the `Task` struct. Although the previous solution was correct, since the `!Send` tasks were only ever scheduled locally, the compiler could no longer verify that this was the case. Now, the typesystem actually _helps_ us ensure that schedulers that can only schedule `Send` tasks will never schedule `!Send` tasks. Signed-off-by: Eliza Weisman --- tokio/src/runtime/current_thread/mod.rs | 4 +- tokio/src/runtime/local.rs | 42 +++++---------- tokio/src/runtime/task/core.rs | 23 ++++---- tokio/src/runtime/task/harness.rs | 24 ++++----- tokio/src/runtime/task/list.rs | 16 +++--- tokio/src/runtime/task/mod.rs | 62 +++++++++++++++------ tokio/src/runtime/task/raw.rs | 68 +++++++++++++----------- tokio/src/runtime/task/waker.rs | 44 +++++++-------- tokio/src/runtime/tests/loom_schedule.rs | 4 +- tokio/src/runtime/tests/mock_schedule.rs | 7 ++- tokio/src/runtime/thread_pool/owned.rs | 2 +- tokio/src/runtime/thread_pool/shared.rs | 4 +- 12 files changed, 164 insertions(+), 136 deletions(-) diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index d2cf4581f38..4313a14d180 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -35,7 +35,7 @@ pub(super) struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - owned_tasks: UnsafeCell>, + owned_tasks: UnsafeCell>, /// Local run queue. /// @@ -254,7 +254,7 @@ impl Scheduler { } } -impl Schedule for Scheduler { +impl Schedule for Scheduler { fn bind(&self, task: &Task) { unsafe { (*self.owned_tasks.get()).insert(task); diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 2589c0464b2..f454580249a 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -1,5 +1,5 @@ //! Groups a set of tasks that execute on the same thread. -use crate::executor::task::{self, JoinHandle, Schedule, Task}; +use crate::runtime::task::{self, JoinHandle, Schedule, UnsendMarker, UnsendTask}; use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; @@ -29,7 +29,7 @@ struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - owned_tasks: UnsafeCell>, + tasks: UnsafeCell>, /// Local run queue. /// @@ -39,7 +39,7 @@ struct Scheduler { /// /// References should not be handed out. Only call `push` / `pop` functions. /// Only call from the owning thread. - queue: UnsafeCell>>, + queue: UnsafeCell>>, } #[pin_project] @@ -89,26 +89,12 @@ where .get() .expect("`local::spawn` called from outside of a local::TaskSet!"); unsafe { - let (task, handle) = task::joinable(Unsend(future)); + let (task, handle) = task::joinable_unsend(future); current.as_ref().schedule(task); handle } }) } -/// EXTREMELY UNSAFE type for pretending a task is Send. Don't use this elsewhere. -#[pin_project] -struct Unsend(#[pin] T); - -unsafe impl Send for Unsend {} - -impl Future for Unsend { - type Output = F::Output; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.project(); - this.0.poll(cx) - } -} /// Max number of tasks to poll per tick. const MAX_TASKS_PER_TICK: usize = 61; @@ -127,7 +113,7 @@ impl TaskSet { F: Future + 'static, F::Output: 'static, { - let (task, handle) = task::joinable(Unsend(future)); + let (task, handle) = task::joinable_unsend(future); self.scheduler.schedule(task); handle } @@ -166,26 +152,26 @@ impl Future for LocalFuture { // === impl Scheduler === -impl Schedule for Scheduler { - fn bind(&self, task: &Task) { +impl Schedule for Scheduler { + fn bind(&self, task: &UnsendTask) { assert!(self.is_current()); unsafe { - (*self.owned_tasks.get()).insert(task); + (*self.tasks.get()).insert(task); } } - fn release(&self, _: Task) { + fn release(&self, _: UnsendTask) { unreachable!("tasks should only be completed locally") } - fn release_local(&self, task: &Task) { + fn release_local(&self, task: &UnsendTask) { assert!(self.is_current()); unsafe { - (*self.owned_tasks.get()).remove(task); + (*self.tasks.get()).remove(task); } } - fn schedule(&self, task: Task) { + fn schedule(&self, task: UnsendTask) { assert!(self.is_current()); unsafe { (*self.queue.get()).push_front(task); @@ -196,7 +182,7 @@ impl Schedule for Scheduler { impl Scheduler { fn new() -> Self { Self { - owned_tasks: UnsafeCell::new(task::OwnedList::new()), + tasks: UnsafeCell::new(task::OwnedList::new()), queue: UnsafeCell::new(VecDeque::with_capacity(64)), } } @@ -230,7 +216,7 @@ impl Scheduler { .unwrap_or(false) } - fn next_task(&self) -> Option> { + fn next_task(&self) -> Option> { unsafe { (*self.queue.get()).pop_front() } } diff --git a/tokio/src/runtime/task/core.rs b/tokio/src/runtime/task/core.rs index 957a4a836db..3462eb7adba 100644 --- a/tokio/src/runtime/task/core.rs +++ b/tokio/src/runtime/task/core.rs @@ -7,6 +7,7 @@ use crate::runtime::task::Schedule; use std::cell::UnsafeCell; use std::future::Future; +use std::marker::PhantomData; use std::mem::MaybeUninit; use std::pin::Pin; use std::ptr::{self, NonNull}; @@ -17,12 +18,12 @@ use std::task::{Context, Poll, Waker}; /// It is critical for `Header` to be the first field as the task structure will /// be referenced by both *mut Cell and *mut Header. #[repr(C)] -pub(super) struct Cell { +pub(super) struct Cell { /// Hot task state data pub(super) header: Header, /// Either the future or output, depending on the execution stage. - pub(super) core: Core, + pub(super) core: Core, /// Cold data pub(super) trailer: Trailer, @@ -31,8 +32,9 @@ pub(super) struct Cell { /// The core of the task. /// /// Holds the future or output, depending on the stage of execution. -pub(super) struct Core { +pub(super) struct Core { stage: Stage, + _marker: PhantomData, } /// Crate public as this is also needed by the pool. @@ -74,12 +76,12 @@ enum Stage { Consumed, } -impl Cell { +impl Cell { /// Allocate a new task cell, containing the header, trailer, and core /// structures. - pub(super) fn new(future: T, state: State) -> Box> + pub(super) fn new(future: T, state: State) -> Box> where - S: Schedule, + S: Schedule, { Box::new(Cell { header: Header { @@ -88,11 +90,12 @@ impl Cell { queue_next: UnsafeCell::new(ptr::null()), owned_next: UnsafeCell::new(None), owned_prev: UnsafeCell::new(None), - vtable: raw::vtable::(), + vtable: raw::vtable::(), future_causality: CausalCell::new(()), }, core: Core { stage: Stage::Running(Track::new(future)), + _marker: PhantomData, }, trailer: Trailer { waker: CausalCell::new(MaybeUninit::new(None)), @@ -101,14 +104,14 @@ impl Cell { } } -impl Core { +impl Core { pub(super) fn transition_to_consumed(&mut self) { self.stage = Stage::Consumed } pub(super) fn poll(&mut self, header: &Header) -> Poll where - S: Schedule, + S: Schedule, { let res = { let future = match &mut self.stage { @@ -122,7 +125,7 @@ impl Core { // The waker passed into the `poll` function does not require a ref // count increment. - let waker_ref = waker_ref::(header); + let waker_ref = waker_ref::(header); let mut cx = Context::from_waker(&*waker_ref); future.poll(&mut cx) diff --git a/tokio/src/runtime/task/harness.rs b/tokio/src/runtime/task/harness.rs index 39cda5dd00b..1bdd43333d9 100644 --- a/tokio/src/runtime/task/harness.rs +++ b/tokio/src/runtime/task/harness.rs @@ -11,21 +11,21 @@ use std::ptr::NonNull; use std::task::{Poll, Waker}; /// Typed raw task handle -pub(super) struct Harness { - cell: NonNull>, +pub(super) struct Harness { + cell: NonNull>, _p: PhantomData, } -impl Harness +impl Harness where T: Future, S: 'static, { - pub(super) unsafe fn from_raw(ptr: *mut ()) -> Harness { + pub(super) unsafe fn from_raw(ptr: *mut ()) -> Harness { debug_assert!(!ptr.is_null()); Harness { - cell: NonNull::new_unchecked(ptr as *mut Cell), + cell: NonNull::new_unchecked(ptr as *mut Cell), _p: PhantomData, } } @@ -38,15 +38,15 @@ where unsafe { &self.cell.as_ref().trailer } } - fn core(&mut self) -> &mut Core { + fn core(&mut self) -> &mut Core { unsafe { &mut self.cell.as_mut().core } } } -impl Harness +impl Harness where T: Future, - S: Schedule, + S: Schedule, { /// Poll the inner future. /// @@ -99,12 +99,12 @@ where let res = header.future_causality.with_mut(|_| { panic::catch_unwind(panic::AssertUnwindSafe(|| { - struct Guard<'a, T: Future> { - core: &'a mut Core, + struct Guard<'a, T: Future, M> { + core: &'a mut Core, polled: bool, } - impl Drop for Guard<'_, T> { + impl Drop for Guard<'_, T, M> { fn drop(&mut self) { if !self.polled { self.core.transition_to_consumed(); @@ -552,7 +552,7 @@ where self.trailer().waker.with_deferred(|ptr| ptr.read()) } - unsafe fn to_task(&self) -> Task { + unsafe fn to_task(&self) -> Task { let ptr = self.cell.as_ptr() as *mut Header; Task::from_raw(NonNull::new_unchecked(ptr)) } diff --git a/tokio/src/runtime/task/list.rs b/tokio/src/runtime/task/list.rs index 7ccc5795013..5cb699508a5 100644 --- a/tokio/src/runtime/task/list.rs +++ b/tokio/src/runtime/task/list.rs @@ -4,20 +4,20 @@ use std::fmt; use std::marker::PhantomData; use std::ptr::NonNull; -pub(crate) struct OwnedList { +pub(crate) struct OwnedList { head: Option>, - _p: PhantomData, + _p: PhantomData<(S, M)>, } -impl OwnedList { - pub(crate) fn new() -> OwnedList { +impl OwnedList { + pub(crate) fn new() -> Self { OwnedList { head: None, _p: PhantomData, } } - pub(crate) fn insert(&mut self, task: &Task) { + pub(crate) fn insert(&mut self, task: &Task) { debug_assert!(!self.contains(task)); unsafe { @@ -36,7 +36,7 @@ impl OwnedList { } } - pub(crate) fn remove(&mut self, task: &Task) { + pub(crate) fn remove(&mut self, task: &Task) { debug_assert!(self.head.is_some()); unsafe { @@ -72,7 +72,7 @@ impl OwnedList { } /// Only used by debug assertions - fn contains(&self, task: &Task) -> bool { + fn contains(&self, task: &Task) -> bool { let mut curr = self.head; while let Some(p) = curr { @@ -89,7 +89,7 @@ impl OwnedList { } } -impl fmt::Debug for OwnedList { +impl fmt::Debug for OwnedList { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("OwnedList").finish() } diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 7e361e4fb80..3afc8b419f6 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -38,38 +38,52 @@ use std::ptr::NonNull; use std::{fmt, mem}; /// An owned handle to the task, tracked by ref count -pub(crate) struct Task { +pub(crate) struct Task { raw: RawTask, - _p: PhantomData, + _p: PhantomData<(S, M)>, } -unsafe impl Send for Task {} +/// An owned handle to a `!Send` task, tracked by ref count. +pub(crate) type UnsendTask = Task; + +/// Marker type indicating that a `Task` was constructed from a future that +/// implements `Send`. +#[derive(Debug)] +pub(crate) struct SendMarker {} + +/// Marker type indicating that a `Task` was constructed from a future that +/// does not implement `Send`, and may only be scheduled by a scheduler that is +/// capable of scheduling `!Send` tasks. +#[derive(Debug)] +pub(crate) struct UnsendMarker {} + +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; -pub(crate) trait Schedule: Send + Sync + Sized + 'static { +pub(crate) trait Schedule: Send + Sync + Sized + 'static { /// Bind a task to the executor. /// /// Guaranteed to be called from the thread that called `poll` on the task. - fn bind(&self, task: &Task); + fn bind(&self, task: &Task); /// The task has completed work and is ready to be released. The scheduler /// is free to drop it whenever. - fn release(&self, task: Task); + fn release(&self, task: Task); /// The has been completed by the executor it was bound to. - fn release_local(&self, task: &Task); + fn release_local(&self, task: &Task); /// Schedule the task - fn schedule(&self, task: Task); + fn schedule(&self, task: Task); } /// Create a new task without an associated join handle pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { Task { raw: RawTask::new_background::<_, S>(task), @@ -81,7 +95,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { let raw = RawTask::new_joinable::<_, S>(task); @@ -95,8 +109,26 @@ where (task, join) } -impl Task { - pub(crate) unsafe fn from_raw(ptr: NonNull
) -> Task { +/// Create a new `!Send` task with an associated join handle +pub(crate) fn joinable_unsend(task: T) -> (UnsendTask, JoinHandle) +where + T: Future + 'static, + S: Schedule, +{ + let raw = RawTask::new_joinable_unsend::<_, S>(task); + + let task = Task { + raw, + _p: PhantomData, + }; + + let join = JoinHandle::new(raw); + + (task, join) +} + +impl Task { + pub(crate) unsafe fn from_raw(ptr: NonNull
) -> Task { Task { raw: RawTask::from_raw(ptr), _p: PhantomData, @@ -114,7 +146,7 @@ impl Task { } } -impl Task { +impl, M> Task { /// Returns `self` when the task needs to be immediately re-scheduled pub(crate) fn run(self, mut executor: F) -> Option where @@ -140,13 +172,13 @@ impl Task { } } -impl Drop for Task { +impl Drop for Task { fn drop(&mut self) { self.raw.drop_task(); } } -impl fmt::Debug for Task { +impl fmt::Debug for Task { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Task").finish() } diff --git a/tokio/src/runtime/task/raw.rs b/tokio/src/runtime/task/raw.rs index 51d9d41c274..44739b0f866 100644 --- a/tokio/src/runtime/task/raw.rs +++ b/tokio/src/runtime/task/raw.rs @@ -1,7 +1,7 @@ use crate::loom::alloc::Track; use crate::runtime::task::Cell; use crate::runtime::task::Harness; -use crate::runtime::task::{Header, Schedule}; +use crate::runtime::task::{Header, Schedule, SendMarker, UnsendMarker}; use crate::runtime::task::{Snapshot, State}; use std::future::Future; @@ -42,15 +42,15 @@ pub(super) struct Vtable { } /// Get the vtable for the requested `T` and `S` generics. -pub(super) fn vtable() -> &'static Vtable { +pub(super) fn vtable, M>() -> &'static Vtable { &Vtable { - poll: poll::, - drop_task: drop_task::, - read_output: read_output::, - store_join_waker: store_join_waker::, - swap_join_waker: swap_join_waker::, - drop_join_handle_slow: drop_join_handle_slow::, - cancel: cancel::, + poll: poll::, + drop_task: drop_task::, + read_output: read_output::, + store_join_waker: store_join_waker::, + swap_join_waker: swap_join_waker::, + drop_join_handle_slow: drop_join_handle_slow::, + cancel: cancel::, } } @@ -58,23 +58,31 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S>(task, State::new_background()) + RawTask::new::<_, S, SendMarker>(task, State::new_background()) } pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S>(task, State::new_joinable()) + RawTask::new::<_, S, SendMarker>(task, State::new_joinable()) } - fn new(task: T, state: State) -> RawTask + pub(super) fn new_joinable_unsend(task: T) -> RawTask where - T: Future + Send + 'static, - S: Schedule, + T: Future + 'static, + S: Schedule, + { + RawTask::new::<_, S, UnsendMarker>(task, State::new_joinable()) + } + + fn new(task: T, state: State) -> RawTask + where + T: Future + 'static, + S: Schedule, { let ptr = Box::into_raw(Cell::new::(task, state)); let ptr = unsafe { NonNull::new_unchecked(ptr as *mut Header) }; @@ -150,44 +158,44 @@ impl Clone for RawTask { impl Copy for RawTask {} -unsafe fn poll( +unsafe fn poll, M>( ptr: *mut (), executor: &mut dyn FnMut() -> Option>, ) -> bool { - let harness = Harness::::from_raw(ptr); + let harness = Harness::::from_raw(ptr); harness.poll(executor) } -unsafe fn drop_task(ptr: *mut ()) { - let harness = Harness::::from_raw(ptr); +unsafe fn drop_task, M>(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); harness.drop_task(); } -unsafe fn read_output(ptr: *mut (), dst: *mut (), state: Snapshot) { - let harness = Harness::::from_raw(ptr); +unsafe fn read_output, M>(ptr: *mut (), dst: *mut (), state: Snapshot) { + let harness = Harness::::from_raw(ptr); harness.read_output(dst as *mut Track>, state); } -unsafe fn store_join_waker(ptr: *mut (), waker: &Waker) -> Snapshot { - let harness = Harness::::from_raw(ptr); +unsafe fn store_join_waker, M>(ptr: *mut (), waker: &Waker) -> Snapshot { + let harness = Harness::::from_raw(ptr); harness.store_join_waker(waker) } -unsafe fn swap_join_waker( +unsafe fn swap_join_waker, M>( ptr: *mut (), waker: &Waker, prev: Snapshot, ) -> Snapshot { - let harness = Harness::::from_raw(ptr); + let harness = Harness::::from_raw(ptr); harness.swap_join_waker(waker, prev) } -unsafe fn drop_join_handle_slow(ptr: *mut ()) { - let harness = Harness::::from_raw(ptr); +unsafe fn drop_join_handle_slow, M>(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); harness.drop_join_handle_slow() } -unsafe fn cancel(ptr: *mut (), from_queue: bool) { - let harness = Harness::::from_raw(ptr); +unsafe fn cancel, M>(ptr: *mut (), from_queue: bool) { + let harness = Harness::::from_raw(ptr); harness.cancel(from_queue) } diff --git a/tokio/src/runtime/task/waker.rs b/tokio/src/runtime/task/waker.rs index 00c1e6d46d8..c41b85b602a 100644 --- a/tokio/src/runtime/task/waker.rs +++ b/tokio/src/runtime/task/waker.rs @@ -13,17 +13,17 @@ pub(super) struct WakerRef<'a, S: 'static> { /// Returns a `WakerRef` which avoids having to pre-emptively increase the /// refcount if there is no need to do so. -pub(super) fn waker_ref(meta: &Header) -> WakerRef<'_, S> +pub(super) fn waker_ref(meta: &Header) -> WakerRef<'_, S> where T: Future, - S: Schedule, + S: Schedule, { let ptr = meta as *const _ as *const (); let vtable = &RawWakerVTable::new( - clone_waker::, + clone_waker::, wake_unreachable, - wake_by_local_ref::, + wake_by_local_ref::, noop, ); @@ -43,30 +43,30 @@ impl ops::Deref for WakerRef<'_, S> { } } -unsafe fn clone_waker(ptr: *const ()) -> RawWaker +unsafe fn clone_waker(ptr: *const ()) -> RawWaker where T: Future, - S: Schedule, + S: Schedule, { let meta = ptr as *const Header; (*meta).state.ref_inc(); let vtable = &RawWakerVTable::new( - clone_waker::, - wake_by_val::, - wake_by_ref::, - drop_waker::, + clone_waker::, + wake_by_val::, + wake_by_ref::, + drop_waker::, ); RawWaker::new(ptr, vtable) } -unsafe fn drop_waker(ptr: *const ()) +unsafe fn drop_waker(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.drop_waker(); } @@ -75,32 +75,32 @@ unsafe fn wake_unreachable(_data: *const ()) { unreachable!(); } -unsafe fn wake_by_val(ptr: *const ()) +unsafe fn wake_by_val(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_val(); } // This function can only be called when on the runtime. -unsafe fn wake_by_local_ref(ptr: *const ()) +unsafe fn wake_by_local_ref(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_local_ref(); } // Wake without consuming the waker -unsafe fn wake_by_ref(ptr: *const ()) +unsafe fn wake_by_ref(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_ref(); } diff --git a/tokio/src/runtime/tests/loom_schedule.rs b/tokio/src/runtime/tests/loom_schedule.rs index 1dad6062181..7b19d7bb6b5 100644 --- a/tokio/src/runtime/tests/loom_schedule.rs +++ b/tokio/src/runtime/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::runtime::task::{Schedule, Task}; +use crate::runtime::task::{Schedule, SendMarker, Task}; use loom::sync::Notify; use std::collections::VecDeque; @@ -32,7 +32,7 @@ impl LoomSchedule { } } -impl Schedule for LoomSchedule { +impl Schedule for LoomSchedule { fn bind(&self, _task: &Task) {} fn release(&self, task: Task) { diff --git a/tokio/src/runtime/tests/mock_schedule.rs b/tokio/src/runtime/tests/mock_schedule.rs index ab15c54e237..95b2e03ebe5 100644 --- a/tokio/src/runtime/tests/mock_schedule.rs +++ b/tokio/src/runtime/tests/mock_schedule.rs @@ -1,6 +1,5 @@ #![allow(warnings)] - -use crate::runtime::task::{Header, Schedule, Task}; +use crate::runtime::task::{Header, Schedule, SendMarker, Task}; use std::collections::VecDeque; use std::sync::Mutex; @@ -83,7 +82,7 @@ impl Mock { } } -impl Schedule for Mock { +impl Schedule for Mock { fn bind(&self, task: &Task) { match self.next("bind") { Call::Bind(ptr) => { @@ -120,7 +119,7 @@ impl Drop for Mock { } } -impl Schedule for Noop { +impl Schedule for Noop { fn bind(&self, _task: &Task) {} fn release(&self, _task: Task) {} diff --git a/tokio/src/runtime/thread_pool/owned.rs b/tokio/src/runtime/thread_pool/owned.rs index 04a2f9316e7..daa0b922b54 100644 --- a/tokio/src/runtime/thread_pool/owned.rs +++ b/tokio/src/runtime/thread_pool/owned.rs @@ -31,7 +31,7 @@ pub(super) struct Owned { pub(super) work_queue: queue::Worker>, /// List of tasks owned by the worker - pub(super) owned_tasks: task::OwnedList>, + pub(super) owned_tasks: task::OwnedList, task::SendMarker>, } impl

Owned

diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index e0a1987e223..0835805826e 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,5 +1,5 @@ use crate::runtime::park::Unpark; -use crate::runtime::task::{self, Schedule, Task}; +use crate::runtime::task::{self, Schedule, SendMarker, Task}; use crate::runtime::thread_pool::worker; use std::ptr; @@ -65,7 +65,7 @@ where } } -impl

Schedule for Shared

+impl

Schedule for Shared

where P: Unpark, { From c1e4e4a6fc88aa4cfadde6fe68f9806b8ff6c314 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 11:09:23 -0800 Subject: [PATCH 06/73] add docs Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 30 +++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index f454580249a..521e32cc4ff 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -1,4 +1,4 @@ -//! Groups a set of tasks that execute on the same thread. +//! Utilities for running `!Send` futures on the current thread. use crate::runtime::task::{self, JoinHandle, Schedule, UnsendMarker, UnsendTask}; use std::cell::{Cell, UnsafeCell}; @@ -71,12 +71,16 @@ thread_local! { /// let unsync_data = Rc::new("my unsync data..."); /// /// let mut rt = Runtime::new().unwrap(); -/// rt.block_on(local::task_set(async move { +/// let task_set = local::TaskSet::new(); +/// let local_task = task_set.spawn(async move { /// let more_unsync_data = unsync_data.clone(); /// local::spawn_local(async move { /// println!("{}", more_unsync_data); /// // ... /// }).await.unwrap(); +/// }); +/// rt.block_on(local::task_set(async move { +/// local_task.await.unwrap(); /// })); /// ``` pub fn spawn_local(future: F) -> JoinHandle @@ -108,7 +112,10 @@ impl TaskSet { } } - pub fn spawn_local(&self, future: F) -> JoinHandle + /// Spawns a `!Send` task onto the local task set. + /// + /// This task is guaranteed to be run on the current thread. + pub fn spawn(&self, future: F) -> JoinHandle where F: Future + 'static, F::Output: 'static, @@ -118,6 +125,23 @@ impl TaskSet { handle } + /// Run a future to completion on the provided runtime, driving any local + /// futures spawned on this task set on the current thread. + /// + /// This runs the given future on the runtime, blocking until it is + /// complete, and yielding its resolved result. Any tasks or timers which + /// the future spawns internally will be executed on the runtime. The future + /// may also call [`spawn_local`] to spawn additional local futures on the + /// current thread. + /// + /// This method should not be called from an asynchronous context. + /// + /// # Panics + /// + /// This function panics if the executor is at capacity, if the provided + /// future panics, or if called within an asynchronous execution context. + /// + /// [`spawn_local`]: ../fn.spawn_local.html pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output where F: Future + 'static, From 968eea57764ada18f9f0136c3a3b6e1c803648f0 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 11:19:49 -0800 Subject: [PATCH 07/73] re-export local mod from root, feature flag Signed-off-by: Eliza Weisman --- tokio/Cargo.toml | 2 ++ tokio/src/lib.rs | 7 +++++++ tokio/src/runtime/local.rs | 4 ++-- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 7380b1eac63..1f91a8e7717 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -28,6 +28,7 @@ default = [ "blocking", "fs", "io", + "local", "net-full", "process", "rt-full", @@ -42,6 +43,7 @@ fs = ["blocking", "io-traits"] io-traits = ["bytes", "iovec"] io-util = ["io-traits", "pin-project", "memchr"] io = ["io-traits", "io-util"] +local = ["rt-current-thread"] macros = ["tokio-macros"] net-full = ["tcp", "udp", "uds"] net-driver = ["io-traits", "mio", "blocking", "lazy_static"] diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index b7b99ac6dae..b56797dbc60 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -128,6 +128,13 @@ if_runtime! { #[doc(inline)] pub use crate::runtime::spawn; + #[doc(inline)] + #[cfg(feature = "local")] + pub use crate::local::spawn_local; + + #[cfg(feature = "local")] + pub use crate::runtime::local; + #[cfg(not(test))] // Work around for rust-lang/rust#62127 #[cfg(feature = "macros")] #[doc(inline)] diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 521e32cc4ff..08a56c83afb 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -67,14 +67,14 @@ thread_local! { /// ```rust /// # use tokio::runtime::Runtime; /// use std::rc::Rc; -/// use tokio::executor::local; +/// use tokio::local; /// let unsync_data = Rc::new("my unsync data..."); /// /// let mut rt = Runtime::new().unwrap(); /// let task_set = local::TaskSet::new(); /// let local_task = task_set.spawn(async move { /// let more_unsync_data = unsync_data.clone(); -/// local::spawn_local(async move { +/// tokio::spawn_local(async move { /// println!("{}", more_unsync_data); /// // ... /// }).await.unwrap(); From 983b6b7d2061ff5e64c88c1739b2d76926f8c98d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 11:43:46 -0800 Subject: [PATCH 08/73] placate clippy Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 08a56c83afb..8665d7d17c5 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -152,6 +152,12 @@ impl TaskSet { } } +impl Default for TaskSet { + fn default() -> Self { + Self::new() + } +} + impl Future for LocalFuture { type Output = F::Output; From fb99e6858370ec29ab57d74a9474eb500487e685 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 11:44:35 -0800 Subject: [PATCH 09/73] fix doctest Signed-off-by: Eliza Weisman --- tokio/src/local.rs | 374 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 374 insertions(+) create mode 100644 tokio/src/local.rs diff --git a/tokio/src/local.rs b/tokio/src/local.rs new file mode 100644 index 00000000000..6444104df69 --- /dev/null +++ b/tokio/src/local.rs @@ -0,0 +1,374 @@ +//! Utilities for running `!Send` futures on the current thread. +use crate::executor::task::{self, JoinHandle, Schedule, UnsendMarker, UnsendTask}; + +use std::cell::{Cell, UnsafeCell}; +use std::collections::VecDeque; +use std::fmt; +use std::future::Future; +use std::marker::PhantomData; +use std::pin::Pin; +use std::ptr::{self, NonNull}; +use std::rc::Rc; +use std::task::{Context, Poll}; + +use pin_project::pin_project; + +/// A group of tasks which are executed on the same thread. +/// +/// These tasks need not implement `Send`; a local task set provides the +/// capacity to execute `!Send` futures. +#[derive(Debug)] +pub struct TaskSet { + scheduler: Rc, + _not_send_or_sync: PhantomData<*const ()>, +} + +struct Scheduler { + /// List of all active tasks spawned onto this executor. + /// + /// # Safety + /// + /// Must only be accessed from the primary thread + tasks: UnsafeCell>, + + /// Local run queue. + /// + /// Tasks notified from the current thread are pushed into this queue. + /// + /// # Safety + /// + /// References should not be handed out. Only call `push` / `pop` functions. + /// Only call from the owning thread. + queue: UnsafeCell>>, +} + +#[pin_project] +struct LocalFuture { + scheduler: Rc, + #[pin] + future: F, +} + +thread_local! { + static CURRENT_TASK_SET: Cell>> = Cell::new(None); +} + +/// Spawns a `!Send` future on the local task set. +/// +/// The spawned future will be run on the same thread that called `spawn_local.` +/// This may only be called from the context of a local task set. +/// +/// # Panics +/// +/// - This function panics if called outside of a local task set. +/// +/// # Examples +/// +/// ```rust +/// # use tokio::runtime::Runtime; +/// use std::rc::Rc; +/// use tokio::local; +/// let unsync_data = Rc::new("my unsync data..."); +/// +/// let mut rt = Runtime::new().unwrap(); +/// let task_set = local::TaskSet::new(); +/// let local_task = task_set.spawn(async move { +/// let more_unsync_data = unsync_data.clone(); +/// tokio::spawn_local(async move { +/// println!("{}", more_unsync_data); +/// // ... +/// }).await.unwrap(); +/// }); +/// rt.block_on(local::task_set(async move { +/// local_task.await.unwrap(); +/// })); +/// ``` +pub fn spawn_local(future: F) -> JoinHandle +where + F: Future + 'static, + F::Output: 'static, +{ + CURRENT_TASK_SET.with(|current| { + let current = current + .get() + .expect("`local::spawn` called from outside of a local::TaskSet!"); + unsafe { + let (task, handle) = task::joinable_unsend(future); + current.as_ref().schedule(task); + handle + } + }) +} + +/// Max number of tasks to poll per tick. +const MAX_TASKS_PER_TICK: usize = 61; + +impl TaskSet { + /// Returns a new local task set for the given future. + pub fn new() -> Self { + Self { + scheduler: Rc::new(Scheduler::new()), + _not_send_or_sync: PhantomData, + } + } + + /// Spawns a `!Send` task onto the local task set. + /// + /// This task is guaranteed to be run on the current thread. + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + let (task, handle) = task::joinable_unsend(future); + self.scheduler.schedule(task); + handle + } + + /// Run a future to completion on the provided runtime, driving any local + /// futures spawned on this task set on the current thread. + /// + /// This runs the given future on the runtime, blocking until it is + /// complete, and yielding its resolved result. Any tasks or timers which + /// the future spawns internally will be executed on the runtime. The future + /// may also call [`spawn_local`] to spawn additional local futures on the + /// current thread. + /// + /// This method should not be called from an asynchronous context. + /// + /// # Panics + /// + /// This function panics if the executor is at capacity, if the provided + /// future panics, or if called within an asynchronous execution context. + /// + /// [`spawn_local`]: ../fn.spawn_local.html + pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output + where + F: Future + 'static, + F::Output: 'static, + { + let scheduler = self.scheduler.clone(); + rt.block_on(LocalFuture { scheduler, future }) + } +} + +impl Future for LocalFuture { + type Output = F::Output; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.project(); + let scheduler = this.scheduler; + let future = this.future; + + scheduler.with(|| { + scheduler.tick(); + + match future.poll(cx) { + Poll::Ready(v) => Poll::Ready(v), + Poll::Pending => { + cx.waker().wake_by_ref(); + Poll::Pending + } + } + }) + } +} + +// === impl Scheduler === + +impl Schedule for Scheduler { + fn bind(&self, task: &UnsendTask) { + assert!(self.is_current()); + unsafe { + (*self.tasks.get()).insert(task); + } + } + + fn release(&self, _: UnsendTask) { + unreachable!("tasks should only be completed locally") + } + + fn release_local(&self, task: &UnsendTask) { + assert!(self.is_current()); + unsafe { + (*self.tasks.get()).remove(task); + } + } + + fn schedule(&self, task: UnsendTask) { + assert!(self.is_current()); + unsafe { + (*self.queue.get()).push_front(task); + } + } +} + +impl Scheduler { + fn new() -> Self { + Self { + tasks: UnsafeCell::new(task::OwnedList::new()), + queue: UnsafeCell::new(VecDeque::with_capacity(64)), + } + } + + fn with(&self, f: impl FnOnce() -> F) -> F { + struct Entered<'a> { + current: &'a Cell>>, + } + + impl<'a> Drop for Entered<'a> { + fn drop(&mut self) { + self.current.set(None); + } + } + + CURRENT_TASK_SET.with(|current| { + current.set(Some(NonNull::from(self))); + let _entered = Entered { current }; + f() + }) + } + + fn is_current(&self) -> bool { + CURRENT_TASK_SET + .try_with(|current| { + current + .get() + .iter() + .any(|current| ptr::eq(current.as_ptr(), self as *const _)) + }) + .unwrap_or(false) + } + + fn next_task(&self) -> Option> { + unsafe { (*self.queue.get()).pop_front() } + } + + fn tick(&self) { + for _ in 0..MAX_TASKS_PER_TICK { + let task = match self.next_task() { + Some(task) => task, + None => return, + }; + + if let Some(task) = task.run(&mut || Some(self.into())) { + self.schedule(task); + } + } + } +} + +unsafe impl Send for Scheduler {} +unsafe impl Sync for Scheduler {} + +impl fmt::Debug for Scheduler { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("Scheduler { .. }").finish() + } +} +impl Drop for Scheduler { + fn drop(&mut self) { + // Drain all local tasks + while let Some(task) = self.next_task() { + task.shutdown(); + } + + // Release owned tasks + unsafe { + (*self.tasks.get()).shutdown(); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::runtime; + + #[test] + fn local_current_thread() { + let mut rt = runtime::Builder::new().current_thread().build().unwrap(); + TaskSet::new().block_on(&mut rt, async { + spawn_local(async {}).await.unwrap(); + }); + } + + #[test] + fn local_threadpool() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + .await + .unwrap(); + }); + } + + #[test] + fn all_spawn_locals_are_local() { + use futures_util::future; + + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + let handles = (0..128) + .map(|_| { + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + }) + .collect::>(); + for result in future::join_all(handles).await { + result.unwrap(); + } + }) + } + + #[test] + fn nested_spawn_local_is_local() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + spawn_local(async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }) + .await + .unwrap(); + }) + } +} From d6ff66e98daeaa429632aca69dd2ba96dec1f9f3 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 11:52:34 -0800 Subject: [PATCH 10/73] *actually* fix doctest Signed-off-by: Eliza Weisman --- tokio/src/local.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/local.rs b/tokio/src/local.rs index 6444104df69..c643c513ae8 100644 --- a/tokio/src/local.rs +++ b/tokio/src/local.rs @@ -79,9 +79,9 @@ thread_local! { /// // ... /// }).await.unwrap(); /// }); -/// rt.block_on(local::task_set(async move { +/// task_set.block_on(&mut rt, async move { /// local_task.await.unwrap(); -/// })); +/// }); /// ``` pub fn spawn_local(future: F) -> JoinHandle where From bc93168ae760b88ecf1828289c19f35cc3519d6c Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 13:15:10 -0800 Subject: [PATCH 11/73] fix wrong assertions These assertions asserted that the scheduler was current in all calls to its methods. However, the `TaskSet::spawn_local` method allows scheduling tasks to be run on the `TaskSet` when it is _not_ currently running. Therefore, these assertions are wrong. This test moves the assertion to the _right_ place: when the scheduler actually _runs_ scheduled tasks. Signed-off-by: Eliza Weisman --- tokio/src/local.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tokio/src/local.rs b/tokio/src/local.rs index c643c513ae8..40770ff5d40 100644 --- a/tokio/src/local.rs +++ b/tokio/src/local.rs @@ -178,7 +178,6 @@ impl Future for LocalFuture { impl Schedule for Scheduler { fn bind(&self, task: &UnsendTask) { - assert!(self.is_current()); unsafe { (*self.tasks.get()).insert(task); } @@ -189,14 +188,12 @@ impl Schedule for Scheduler { } fn release_local(&self, task: &UnsendTask) { - assert!(self.is_current()); unsafe { (*self.tasks.get()).remove(task); } } fn schedule(&self, task: UnsendTask) { - assert!(self.is_current()); unsafe { (*self.queue.get()).push_front(task); } @@ -245,6 +242,7 @@ impl Scheduler { } fn tick(&self) { + assert!(self.is_current()); for _ in 0..MAX_TASKS_PER_TICK { let task = match self.next_task() { Some(task) => task, From 17c7f7e72c790f2074211db3ac6266a404dff9ad Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 13:33:24 -0800 Subject: [PATCH 12/73] fix missing pin-project without default features Signed-off-by: Eliza Weisman --- tokio/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 1f91a8e7717..9d20d318920 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -43,7 +43,7 @@ fs = ["blocking", "io-traits"] io-traits = ["bytes", "iovec"] io-util = ["io-traits", "pin-project", "memchr"] io = ["io-traits", "io-util"] -local = ["rt-current-thread"] +local = ["rt-current-thread", "pin-project"] macros = ["tokio-macros"] net-full = ["tcp", "udp", "uds"] net-driver = ["io-traits", "mio", "blocking", "lazy_static"] From 78d41b8376033854e449c3f0a639d32f9237489e Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 14:34:58 -0800 Subject: [PATCH 13/73] fix unused code when `local` is disabled Signed-off-by: Eliza Weisman --- tokio/src/runtime/task/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 3afc8b419f6..a076b69548b 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -44,6 +44,7 @@ pub(crate) struct Task { } /// An owned handle to a `!Send` task, tracked by ref count. +#[cfg(feature = "local")] pub(crate) type UnsendTask = Task; /// Marker type indicating that a `Task` was constructed from a future that @@ -55,6 +56,7 @@ pub(crate) struct SendMarker {} /// does not implement `Send`, and may only be scheduled by a scheduler that is /// capable of scheduling `!Send` tasks. #[derive(Debug)] +#[cfg(feature = "local")] pub(crate) struct UnsendMarker {} unsafe impl Send for Task {} @@ -110,6 +112,7 @@ where } /// Create a new `!Send` task with an associated join handle +#[cfg(feature = "local")] pub(crate) fn joinable_unsend(task: T) -> (UnsendTask, JoinHandle) where T: Future + 'static, From 057f41500c52e47acff0b70d44c804b2d5dedc73 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 14:36:24 -0800 Subject: [PATCH 14/73] add send markers to debug output Signed-off-by: Eliza Weisman --- tokio/src/runtime/task/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index a076b69548b..aaeda7c79cf 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -183,6 +183,8 @@ impl Drop for Task { impl fmt::Debug for Task { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Task").finish() + fmt.debug_struct("Task") + .field("send", format_args!("{}", std::any::type_name::())) + .finish() } } From f00f09998bcd7e1ac788879d14c87d65389f5e89 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 15:06:12 -0800 Subject: [PATCH 15/73] fdsgkhsdfhjkgljkfdg;jklagdsf Signed-off-by: Eliza Weisman --- tokio/src/runtime/task/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index aaeda7c79cf..5312e5a0997 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -184,7 +184,7 @@ impl Drop for Task { impl fmt::Debug for Task { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Task") - .field("send", format_args!("{}", std::any::type_name::())) + .field("send", &format_args!("{}", std::any::type_name::())) .finish() } } From 157e2e2d0b7df7dedcf7c09c09de004a642ef315 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 5 Nov 2019 15:33:25 -0800 Subject: [PATCH 16/73] fix more feature-flaggy sadness Signed-off-by: Eliza Weisman --- tokio/src/runtime/task/raw.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/task/raw.rs b/tokio/src/runtime/task/raw.rs index 44739b0f866..f720b8ff432 100644 --- a/tokio/src/runtime/task/raw.rs +++ b/tokio/src/runtime/task/raw.rs @@ -1,7 +1,9 @@ use crate::loom::alloc::Track; use crate::runtime::task::Cell; use crate::runtime::task::Harness; -use crate::runtime::task::{Header, Schedule, SendMarker, UnsendMarker}; +#[cfg(feature = "local")] +use crate::runtime::task::UnsendMarker; +use crate::runtime::task::{Header, Schedule, SendMarker}; use crate::runtime::task::{Snapshot, State}; use std::future::Future; @@ -71,6 +73,7 @@ impl RawTask { RawTask::new::<_, S, SendMarker>(task, State::new_joinable()) } + #[cfg(feature = "local")] pub(super) fn new_joinable_unsend(task: T) -> RawTask where T: Future + 'static, From 2a88b5f1a3434c849cec90697c11ffdb84f1d8e1 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 10:30:05 -0800 Subject: [PATCH 17/73] rename marker types Signed-off-by: Eliza Weisman --- tokio/src/local.rs | 6 +++--- tokio/src/runtime/current_thread/mod.rs | 4 ++-- tokio/src/runtime/local.rs | 6 +++--- tokio/src/runtime/task/mod.rs | 16 ++++++++-------- tokio/src/runtime/task/raw.rs | 16 ++++++++-------- tokio/src/runtime/tests/loom_schedule.rs | 4 ++-- tokio/src/runtime/tests/mock_schedule.rs | 6 +++--- tokio/src/runtime/thread_pool/owned.rs | 2 +- tokio/src/runtime/thread_pool/shared.rs | 4 ++-- 9 files changed, 32 insertions(+), 32 deletions(-) diff --git a/tokio/src/local.rs b/tokio/src/local.rs index 40770ff5d40..07398a2d51a 100644 --- a/tokio/src/local.rs +++ b/tokio/src/local.rs @@ -1,5 +1,5 @@ //! Utilities for running `!Send` futures on the current thread. -use crate::executor::task::{self, JoinHandle, Schedule, UnsendMarker, UnsendTask}; +use crate::executor::task::{self, JoinHandle, Schedule, UnsendTask, Unsendable}; use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; @@ -29,7 +29,7 @@ struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - tasks: UnsafeCell>, + tasks: UnsafeCell>, /// Local run queue. /// @@ -176,7 +176,7 @@ impl Future for LocalFuture { // === impl Scheduler === -impl Schedule for Scheduler { +impl Schedule for Scheduler { fn bind(&self, task: &UnsendTask) { unsafe { (*self.tasks.get()).insert(task); diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index 4313a14d180..311340f6401 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -35,7 +35,7 @@ pub(super) struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - owned_tasks: UnsafeCell>, + owned_tasks: UnsafeCell>, /// Local run queue. /// @@ -254,7 +254,7 @@ impl Scheduler { } } -impl Schedule for Scheduler { +impl Schedule for Scheduler { fn bind(&self, task: &Task) { unsafe { (*self.owned_tasks.get()).insert(task); diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 8665d7d17c5..351f2ef30ca 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -1,5 +1,5 @@ //! Utilities for running `!Send` futures on the current thread. -use crate::runtime::task::{self, JoinHandle, Schedule, UnsendMarker, UnsendTask}; +use crate::runtime::task::{self, JoinHandle, Schedule, UnsendTask, Unsendable}; use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; @@ -29,7 +29,7 @@ struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - tasks: UnsafeCell>, + tasks: UnsafeCell>, /// Local run queue. /// @@ -182,7 +182,7 @@ impl Future for LocalFuture { // === impl Scheduler === -impl Schedule for Scheduler { +impl Schedule for Scheduler { fn bind(&self, task: &UnsendTask) { assert!(self.is_current()); unsafe { diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 5312e5a0997..194f77b33c5 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -38,28 +38,28 @@ use std::ptr::NonNull; use std::{fmt, mem}; /// An owned handle to the task, tracked by ref count -pub(crate) struct Task { +pub(crate) struct Task { raw: RawTask, _p: PhantomData<(S, M)>, } /// An owned handle to a `!Send` task, tracked by ref count. #[cfg(feature = "local")] -pub(crate) type UnsendTask = Task; +pub(crate) type UnsendTask = Task; /// Marker type indicating that a `Task` was constructed from a future that /// implements `Send`. #[derive(Debug)] -pub(crate) struct SendMarker {} +pub(crate) struct Sendable {} /// Marker type indicating that a `Task` was constructed from a future that /// does not implement `Send`, and may only be scheduled by a scheduler that is /// capable of scheduling `!Send` tasks. #[derive(Debug)] #[cfg(feature = "local")] -pub(crate) struct UnsendMarker {} +pub(crate) struct Unsendable {} -unsafe impl Send for Task {} +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; @@ -85,7 +85,7 @@ pub(crate) trait Schedule: Send + Sync + Sized + 'static { pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { Task { raw: RawTask::new_background::<_, S>(task), @@ -97,7 +97,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { let raw = RawTask::new_joinable::<_, S>(task); @@ -116,7 +116,7 @@ where pub(crate) fn joinable_unsend(task: T) -> (UnsendTask, JoinHandle) where T: Future + 'static, - S: Schedule, + S: Schedule, { let raw = RawTask::new_joinable_unsend::<_, S>(task); diff --git a/tokio/src/runtime/task/raw.rs b/tokio/src/runtime/task/raw.rs index f720b8ff432..e072508ac71 100644 --- a/tokio/src/runtime/task/raw.rs +++ b/tokio/src/runtime/task/raw.rs @@ -2,8 +2,8 @@ use crate::loom::alloc::Track; use crate::runtime::task::Cell; use crate::runtime::task::Harness; #[cfg(feature = "local")] -use crate::runtime::task::UnsendMarker; -use crate::runtime::task::{Header, Schedule, SendMarker}; +use crate::runtime::task::Unsendable; +use crate::runtime::task::{Header, Schedule, Sendable}; use crate::runtime::task::{Snapshot, State}; use std::future::Future; @@ -60,26 +60,26 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S, SendMarker>(task, State::new_background()) + RawTask::new::<_, S, Sendable>(task, State::new_background()) } pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S, SendMarker>(task, State::new_joinable()) + RawTask::new::<_, S, Sendable>(task, State::new_joinable()) } #[cfg(feature = "local")] pub(super) fn new_joinable_unsend(task: T) -> RawTask where T: Future + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S, UnsendMarker>(task, State::new_joinable()) + RawTask::new::<_, S, Unsendable>(task, State::new_joinable()) } fn new(task: T, state: State) -> RawTask diff --git a/tokio/src/runtime/tests/loom_schedule.rs b/tokio/src/runtime/tests/loom_schedule.rs index 7b19d7bb6b5..0bb4c20948c 100644 --- a/tokio/src/runtime/tests/loom_schedule.rs +++ b/tokio/src/runtime/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::runtime::task::{Schedule, SendMarker, Task}; +use crate::runtime::task::{Schedule, Sendable, Task}; use loom::sync::Notify; use std::collections::VecDeque; @@ -32,7 +32,7 @@ impl LoomSchedule { } } -impl Schedule for LoomSchedule { +impl Schedule for LoomSchedule { fn bind(&self, _task: &Task) {} fn release(&self, task: Task) { diff --git a/tokio/src/runtime/tests/mock_schedule.rs b/tokio/src/runtime/tests/mock_schedule.rs index 95b2e03ebe5..a224c54086c 100644 --- a/tokio/src/runtime/tests/mock_schedule.rs +++ b/tokio/src/runtime/tests/mock_schedule.rs @@ -1,5 +1,5 @@ #![allow(warnings)] -use crate::runtime::task::{Header, Schedule, SendMarker, Task}; +use crate::runtime::task::{Header, Schedule, Sendable, Task}; use std::collections::VecDeque; use std::sync::Mutex; @@ -82,7 +82,7 @@ impl Mock { } } -impl Schedule for Mock { +impl Schedule for Mock { fn bind(&self, task: &Task) { match self.next("bind") { Call::Bind(ptr) => { @@ -119,7 +119,7 @@ impl Drop for Mock { } } -impl Schedule for Noop { +impl Schedule for Noop { fn bind(&self, _task: &Task) {} fn release(&self, _task: Task) {} diff --git a/tokio/src/runtime/thread_pool/owned.rs b/tokio/src/runtime/thread_pool/owned.rs index daa0b922b54..8798d4a7532 100644 --- a/tokio/src/runtime/thread_pool/owned.rs +++ b/tokio/src/runtime/thread_pool/owned.rs @@ -31,7 +31,7 @@ pub(super) struct Owned { pub(super) work_queue: queue::Worker>, /// List of tasks owned by the worker - pub(super) owned_tasks: task::OwnedList, task::SendMarker>, + pub(super) owned_tasks: task::OwnedList, task::Sendable>, } impl

Owned

diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index 0835805826e..e9deef9811c 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,5 +1,5 @@ use crate::runtime::park::Unpark; -use crate::runtime::task::{self, Schedule, SendMarker, Task}; +use crate::runtime::task::{self, Schedule, Sendable, Task}; use crate::runtime::thread_pool::worker; use std::ptr; @@ -65,7 +65,7 @@ where } } -impl

Schedule for Shared

+impl

Schedule for Shared

where P: Unpark, { From eef87276e8bc0af162f0c602f8e43c2fe811fc4d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 10:31:19 -0800 Subject: [PATCH 18/73] make markers void structs Signed-off-by: Eliza Weisman --- tokio/src/runtime/task/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 194f77b33c5..faa1debc1cb 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -50,14 +50,14 @@ pub(crate) type UnsendTask = Task; /// Marker type indicating that a `Task` was constructed from a future that /// implements `Send`. #[derive(Debug)] -pub(crate) struct Sendable {} +pub(crate) struct Sendable; /// Marker type indicating that a `Task` was constructed from a future that /// does not implement `Send`, and may only be scheduled by a scheduler that is /// capable of scheduling `!Send` tasks. #[derive(Debug)] #[cfg(feature = "local")] -pub(crate) struct Unsendable {} +pub(crate) struct Unsendable; unsafe impl Send for Task {} From 1c0cea1f92dfa49f477fda7a659e3aa71a0f14ee Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 10:31:45 -0800 Subject: [PATCH 19/73] remove duplicate file Signed-off-by: Eliza Weisman --- tokio/src/local.rs | 372 --------------------------------------------- 1 file changed, 372 deletions(-) delete mode 100644 tokio/src/local.rs diff --git a/tokio/src/local.rs b/tokio/src/local.rs deleted file mode 100644 index 07398a2d51a..00000000000 --- a/tokio/src/local.rs +++ /dev/null @@ -1,372 +0,0 @@ -//! Utilities for running `!Send` futures on the current thread. -use crate::executor::task::{self, JoinHandle, Schedule, UnsendTask, Unsendable}; - -use std::cell::{Cell, UnsafeCell}; -use std::collections::VecDeque; -use std::fmt; -use std::future::Future; -use std::marker::PhantomData; -use std::pin::Pin; -use std::ptr::{self, NonNull}; -use std::rc::Rc; -use std::task::{Context, Poll}; - -use pin_project::pin_project; - -/// A group of tasks which are executed on the same thread. -/// -/// These tasks need not implement `Send`; a local task set provides the -/// capacity to execute `!Send` futures. -#[derive(Debug)] -pub struct TaskSet { - scheduler: Rc, - _not_send_or_sync: PhantomData<*const ()>, -} - -struct Scheduler { - /// List of all active tasks spawned onto this executor. - /// - /// # Safety - /// - /// Must only be accessed from the primary thread - tasks: UnsafeCell>, - - /// Local run queue. - /// - /// Tasks notified from the current thread are pushed into this queue. - /// - /// # Safety - /// - /// References should not be handed out. Only call `push` / `pop` functions. - /// Only call from the owning thread. - queue: UnsafeCell>>, -} - -#[pin_project] -struct LocalFuture { - scheduler: Rc, - #[pin] - future: F, -} - -thread_local! { - static CURRENT_TASK_SET: Cell>> = Cell::new(None); -} - -/// Spawns a `!Send` future on the local task set. -/// -/// The spawned future will be run on the same thread that called `spawn_local.` -/// This may only be called from the context of a local task set. -/// -/// # Panics -/// -/// - This function panics if called outside of a local task set. -/// -/// # Examples -/// -/// ```rust -/// # use tokio::runtime::Runtime; -/// use std::rc::Rc; -/// use tokio::local; -/// let unsync_data = Rc::new("my unsync data..."); -/// -/// let mut rt = Runtime::new().unwrap(); -/// let task_set = local::TaskSet::new(); -/// let local_task = task_set.spawn(async move { -/// let more_unsync_data = unsync_data.clone(); -/// tokio::spawn_local(async move { -/// println!("{}", more_unsync_data); -/// // ... -/// }).await.unwrap(); -/// }); -/// task_set.block_on(&mut rt, async move { -/// local_task.await.unwrap(); -/// }); -/// ``` -pub fn spawn_local(future: F) -> JoinHandle -where - F: Future + 'static, - F::Output: 'static, -{ - CURRENT_TASK_SET.with(|current| { - let current = current - .get() - .expect("`local::spawn` called from outside of a local::TaskSet!"); - unsafe { - let (task, handle) = task::joinable_unsend(future); - current.as_ref().schedule(task); - handle - } - }) -} - -/// Max number of tasks to poll per tick. -const MAX_TASKS_PER_TICK: usize = 61; - -impl TaskSet { - /// Returns a new local task set for the given future. - pub fn new() -> Self { - Self { - scheduler: Rc::new(Scheduler::new()), - _not_send_or_sync: PhantomData, - } - } - - /// Spawns a `!Send` task onto the local task set. - /// - /// This task is guaranteed to be run on the current thread. - pub fn spawn(&self, future: F) -> JoinHandle - where - F: Future + 'static, - F::Output: 'static, - { - let (task, handle) = task::joinable_unsend(future); - self.scheduler.schedule(task); - handle - } - - /// Run a future to completion on the provided runtime, driving any local - /// futures spawned on this task set on the current thread. - /// - /// This runs the given future on the runtime, blocking until it is - /// complete, and yielding its resolved result. Any tasks or timers which - /// the future spawns internally will be executed on the runtime. The future - /// may also call [`spawn_local`] to spawn additional local futures on the - /// current thread. - /// - /// This method should not be called from an asynchronous context. - /// - /// # Panics - /// - /// This function panics if the executor is at capacity, if the provided - /// future panics, or if called within an asynchronous execution context. - /// - /// [`spawn_local`]: ../fn.spawn_local.html - pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output - where - F: Future + 'static, - F::Output: 'static, - { - let scheduler = self.scheduler.clone(); - rt.block_on(LocalFuture { scheduler, future }) - } -} - -impl Future for LocalFuture { - type Output = F::Output; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.project(); - let scheduler = this.scheduler; - let future = this.future; - - scheduler.with(|| { - scheduler.tick(); - - match future.poll(cx) { - Poll::Ready(v) => Poll::Ready(v), - Poll::Pending => { - cx.waker().wake_by_ref(); - Poll::Pending - } - } - }) - } -} - -// === impl Scheduler === - -impl Schedule for Scheduler { - fn bind(&self, task: &UnsendTask) { - unsafe { - (*self.tasks.get()).insert(task); - } - } - - fn release(&self, _: UnsendTask) { - unreachable!("tasks should only be completed locally") - } - - fn release_local(&self, task: &UnsendTask) { - unsafe { - (*self.tasks.get()).remove(task); - } - } - - fn schedule(&self, task: UnsendTask) { - unsafe { - (*self.queue.get()).push_front(task); - } - } -} - -impl Scheduler { - fn new() -> Self { - Self { - tasks: UnsafeCell::new(task::OwnedList::new()), - queue: UnsafeCell::new(VecDeque::with_capacity(64)), - } - } - - fn with(&self, f: impl FnOnce() -> F) -> F { - struct Entered<'a> { - current: &'a Cell>>, - } - - impl<'a> Drop for Entered<'a> { - fn drop(&mut self) { - self.current.set(None); - } - } - - CURRENT_TASK_SET.with(|current| { - current.set(Some(NonNull::from(self))); - let _entered = Entered { current }; - f() - }) - } - - fn is_current(&self) -> bool { - CURRENT_TASK_SET - .try_with(|current| { - current - .get() - .iter() - .any(|current| ptr::eq(current.as_ptr(), self as *const _)) - }) - .unwrap_or(false) - } - - fn next_task(&self) -> Option> { - unsafe { (*self.queue.get()).pop_front() } - } - - fn tick(&self) { - assert!(self.is_current()); - for _ in 0..MAX_TASKS_PER_TICK { - let task = match self.next_task() { - Some(task) => task, - None => return, - }; - - if let Some(task) = task.run(&mut || Some(self.into())) { - self.schedule(task); - } - } - } -} - -unsafe impl Send for Scheduler {} -unsafe impl Sync for Scheduler {} - -impl fmt::Debug for Scheduler { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Scheduler { .. }").finish() - } -} -impl Drop for Scheduler { - fn drop(&mut self) { - // Drain all local tasks - while let Some(task) = self.next_task() { - task.shutdown(); - } - - // Release owned tasks - unsafe { - (*self.tasks.get()).shutdown(); - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::runtime; - - #[test] - fn local_current_thread() { - let mut rt = runtime::Builder::new().current_thread().build().unwrap(); - TaskSet::new().block_on(&mut rt, async { - spawn_local(async {}).await.unwrap(); - }); - } - - #[test] - fn local_threadpool() { - thread_local! { - static ON_RT_THREAD: Cell = Cell::new(false); - } - - ON_RT_THREAD.with(|cell| cell.set(true)); - - let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - }) - .await - .unwrap(); - }); - } - - #[test] - fn all_spawn_locals_are_local() { - use futures_util::future; - - thread_local! { - static ON_RT_THREAD: Cell = Cell::new(false); - } - - ON_RT_THREAD.with(|cell| cell.set(true)); - - let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - let handles = (0..128) - .map(|_| { - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - }) - }) - .collect::>(); - for result in future::join_all(handles).await { - result.unwrap(); - } - }) - } - - #[test] - fn nested_spawn_local_is_local() { - thread_local! { - static ON_RT_THREAD: Cell = Cell::new(false); - } - - ON_RT_THREAD.with(|cell| cell.set(true)); - - let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { - assert!(ON_RT_THREAD.with(|cell| cell.get())); - }) - .await - .unwrap(); - }) - .await - .unwrap(); - }) - .await - .unwrap(); - }) - .await - .unwrap(); - }) - } -} From b6243c6fcfd7d3c6e0e4ab4d3697d2ecaa7063c1 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 11:04:10 -0800 Subject: [PATCH 20/73] fix wrong assertions These assertions asserted that the scheduler was current in all calls to its methods. However, the `TaskSet::spawn_local` method allows scheduling tasks to be run on the `TaskSet` when it is _not_ currently running. Therefore, these assertions are wrong. This commit moves the assertion to the _right_ place: when the scheduler actually _runs_ scheduled tasks. I had previously made this change but I think that it was lost in a rebase. Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 351f2ef30ca..5623b0d9796 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -184,7 +184,6 @@ impl Future for LocalFuture { impl Schedule for Scheduler { fn bind(&self, task: &UnsendTask) { - assert!(self.is_current()); unsafe { (*self.tasks.get()).insert(task); } @@ -195,14 +194,12 @@ impl Schedule for Scheduler { } fn release_local(&self, task: &UnsendTask) { - assert!(self.is_current()); unsafe { (*self.tasks.get()).remove(task); } } fn schedule(&self, task: UnsendTask) { - assert!(self.is_current()); unsafe { (*self.queue.get()).push_front(task); } @@ -251,6 +248,7 @@ impl Scheduler { } fn tick(&self) { + assert!(self.is_current()); for _ in 0..MAX_TASKS_PER_TICK { let task = match self.next_task() { Some(task) => task, From 44fb6c8f6ad3eeb3b82be15651e5061befc7dca0 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 11:21:15 -0800 Subject: [PATCH 21/73] add test that timer exists --- tokio/src/runtime/local.rs | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 5623b0d9796..95bde0442ab 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -316,6 +316,29 @@ mod tests { }); } + #[test] + fn local_threadpool_timer() { + // This test ensures that runtime services like the timer are properly + // set for the local task set. + use std::time::Duration; + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + let join = spawn_local(async move { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + crate::timer::delay_for(Duration::from_millis(10)).await; + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }); + join.await.unwrap(); + }); + } + #[test] fn all_spawn_locals_are_local() { use futures_util::future; From ff052218d100734e9a7f36e9aa0bfc7cc5e4cba3 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 13:11:50 -0800 Subject: [PATCH 22/73] document and test interaction with blocking --- tokio/src/runtime/local.rs | 94 +++++++++++++++++++++++++++++++++++++- 1 file changed, 93 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 95bde0442ab..22bab17a151 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -141,7 +141,50 @@ impl TaskSet { /// This function panics if the executor is at capacity, if the provided /// future panics, or if called within an asynchronous execution context. /// - /// [`spawn_local`]: ../fn.spawn_local.html + /// # Notes + /// + /// Since this function internally calls [`Runtime::block_on`], and drives + /// futures in the local task set inside that call to `block_on`, the local + /// futures may not use [in-place blocking]. If a blocking call needs to be + /// issued from a local task, the [`blocking::run`] API may be used instead. + /// + /// For example, this will panic: + /// ```should_panic + /// use tokio::runtime::{Runtime, local, blocking}; + /// + /// let mut rt = Runtime::new().unwrap(); + /// let local = local::TaskSet::new(); + /// local.block_on(&mut rt, async { + /// let join = tokio::spawn_local(async { + /// let blocking_result = blocking::in_place(|| { + /// // ... + /// }); + /// // ... + /// }); + /// join.await.unwrap(); + /// }) + /// ``` + /// This, however, will not panic: + /// ``` + /// use tokio::runtime::{Runtime, local, blocking}; + /// + /// let mut rt = Runtime::new().unwrap(); + /// let local = local::TaskSet::new(); + /// local.block_on(&mut rt, async { + /// let join = tokio::spawn_local(async { + /// let blocking_result = blocking::run(|| { + /// // ... + /// }).await; + /// // ... + /// }); + /// join.await.unwrap(); + /// }) + /// ``` + /// + /// [`spawn_local`]: fn.spawn_local.html + /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on + /// [in-place blocking]: ../blocking/fn.in_place.html + /// [`blocking::run`]: ../blocking/fn.run.html pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output where F: Future + 'static, @@ -339,6 +382,55 @@ mod tests { }); } + #[test] + // This will panic, since the thread that calls `block_on` cannot use + // in-place blocking inside of `block_on`. + #[should_panic] + fn local_threadpool_blocking_in_place() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + let join = spawn_local(async move { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + runtime::blocking::in_place(|| {}); + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }); + join.await.unwrap(); + }); + } + + #[test] + fn local_threadpool_blocking_run() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Runtime::new().unwrap(); + TaskSet::new().block_on(&mut rt, async { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + let join = spawn_local(async move { + assert!(ON_RT_THREAD.with(|cell| cell.get())); + runtime::blocking::run(|| { + assert!( + !ON_RT_THREAD.with(|cell| cell.get()), + "blocking must not run on the local task set's thread" + ); + }) + .await; + assert!(ON_RT_THREAD.with(|cell| cell.get())); + }); + join.await.unwrap(); + }); + } + #[test] fn all_spawn_locals_are_local() { use futures_util::future; From 45dc55a955eac5dd7c5be8939b0aec19d3285525 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 13:12:06 -0800 Subject: [PATCH 23/73] fix wrong doctest this must've gotten lost in a rebase --- tokio/src/runtime/local.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 22bab17a151..9516e93affc 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -79,7 +79,8 @@ thread_local! { /// // ... /// }).await.unwrap(); /// }); -/// rt.block_on(local::task_set(async move { +/// +/// task_set.block_on(&mut rt, async move { /// local_task.await.unwrap(); /// })); /// ``` From b92ff94845d665f66475e4c84e33a9365c7a463d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 13:18:43 -0800 Subject: [PATCH 24/73] fix missing feature flag this must've also been lost in the rebase --- tokio/src/runtime/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index ef27ff30187..7c1393f1260 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -154,6 +154,7 @@ pub use self::global::spawn; mod io; +#[cfg(feature = "local")] pub mod local; mod park; From e09fb8e188043293565211718cb4f0262264d81d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 13:20:10 -0800 Subject: [PATCH 25/73] agh, doctest --- tokio/src/runtime/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 9516e93affc..b0ed57fb81c 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -82,7 +82,7 @@ thread_local! { /// /// task_set.block_on(&mut rt, async move { /// local_task.await.unwrap(); -/// })); +/// }); /// ``` pub fn spawn_local(future: F) -> JoinHandle where From fec3b91d895072b657818795d1a99f3218524115 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 6 Nov 2019 14:52:49 -0800 Subject: [PATCH 26/73] don't run local tests when running loom tests Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index b0ed57fb81c..f752b8c2402 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -328,7 +328,7 @@ impl Drop for Scheduler { } } -#[cfg(test)] +#[cfg(all(test, not(loom)))] mod tests { use super::*; use crate::runtime; From 478e73f5871170ed5480f8ff0320209f30a565c7 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 7 Nov 2019 11:32:34 -0800 Subject: [PATCH 27/73] rename TaskSet -> TaskGroup Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 50 +++++++++++++++++++------------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index f752b8c2402..e91ea35725c 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -1,4 +1,4 @@ -//! Utilities for running `!Send` futures on the current thread. +//! Runs `!Send` futures on the current thread. use crate::runtime::task::{self, JoinHandle, Schedule, UnsendTask, Unsendable}; use std::cell::{Cell, UnsafeCell}; @@ -15,10 +15,10 @@ use pin_project::pin_project; /// A group of tasks which are executed on the same thread. /// -/// These tasks need not implement `Send`; a local task set provides the +/// These tasks need not implement `Send`; a local task group provides the /// capacity to execute `!Send` futures. #[derive(Debug)] -pub struct TaskSet { +pub struct TaskGroup { scheduler: Rc, _not_send_or_sync: PhantomData<*const ()>, } @@ -53,14 +53,14 @@ thread_local! { static CURRENT_TASK_SET: Cell>> = Cell::new(None); } -/// Spawns a `!Send` future on the local task set. +/// Spawns a `!Send` future on the local task group. /// /// The spawned future will be run on the same thread that called `spawn_local.` -/// This may only be called from the context of a local task set. +/// This may only be called from the context of a local task group. /// /// # Panics /// -/// - This function panics if called outside of a local task set. +/// - This function panics if called outside of a local task group. /// /// # Examples /// @@ -71,7 +71,7 @@ thread_local! { /// let unsync_data = Rc::new("my unsync data..."); /// /// let mut rt = Runtime::new().unwrap(); -/// let task_set = local::TaskSet::new(); +/// let task_set = local::TaskGroup::new(); /// let local_task = task_set.spawn(async move { /// let more_unsync_data = unsync_data.clone(); /// tokio::spawn_local(async move { @@ -92,7 +92,7 @@ where CURRENT_TASK_SET.with(|current| { let current = current .get() - .expect("`local::spawn` called from outside of a local::TaskSet!"); + .expect("`local::spawn` called from outside of a local::TaskGroup!"); unsafe { let (task, handle) = task::joinable_unsend(future); current.as_ref().schedule(task); @@ -104,8 +104,8 @@ where /// Max number of tasks to poll per tick. const MAX_TASKS_PER_TICK: usize = 61; -impl TaskSet { - /// Returns a new local task set for the given future. +impl TaskGroup { + /// Returns a new local task group for the given future. pub fn new() -> Self { Self { scheduler: Rc::new(Scheduler::new()), @@ -113,7 +113,7 @@ impl TaskSet { } } - /// Spawns a `!Send` task onto the local task set. + /// Spawns a `!Send` task onto the local task group. /// /// This task is guaranteed to be run on the current thread. pub fn spawn(&self, future: F) -> JoinHandle @@ -127,7 +127,7 @@ impl TaskSet { } /// Run a future to completion on the provided runtime, driving any local - /// futures spawned on this task set on the current thread. + /// futures spawned on this task group on the current thread. /// /// This runs the given future on the runtime, blocking until it is /// complete, and yielding its resolved result. Any tasks or timers which @@ -145,7 +145,7 @@ impl TaskSet { /// # Notes /// /// Since this function internally calls [`Runtime::block_on`], and drives - /// futures in the local task set inside that call to `block_on`, the local + /// futures in the local task group inside that call to `block_on`, the local /// futures may not use [in-place blocking]. If a blocking call needs to be /// issued from a local task, the [`blocking::run`] API may be used instead. /// @@ -154,7 +154,7 @@ impl TaskSet { /// use tokio::runtime::{Runtime, local, blocking}; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::TaskSet::new(); + /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { /// let join = tokio::spawn_local(async { /// let blocking_result = blocking::in_place(|| { @@ -170,7 +170,7 @@ impl TaskSet { /// use tokio::runtime::{Runtime, local, blocking}; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::TaskSet::new(); + /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { /// let join = tokio::spawn_local(async { /// let blocking_result = blocking::run(|| { @@ -196,7 +196,7 @@ impl TaskSet { } } -impl Default for TaskSet { +impl Default for TaskGroup { fn default() -> Self { Self::new() } @@ -336,7 +336,7 @@ mod tests { #[test] fn local_current_thread() { let mut rt = runtime::Builder::new().current_thread().build().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { spawn_local(async {}).await.unwrap(); }); } @@ -350,7 +350,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -363,7 +363,7 @@ mod tests { #[test] fn local_threadpool_timer() { // This test ensures that runtime services like the timer are properly - // set for the local task set. + // set for the local task group. use std::time::Duration; thread_local! { static ON_RT_THREAD: Cell = Cell::new(false); @@ -372,7 +372,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -395,7 +395,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -415,14 +415,14 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::run(|| { assert!( !ON_RT_THREAD.with(|cell| cell.get()), - "blocking must not run on the local task set's thread" + "blocking must not run on the local task group's thread" ); }) .await; @@ -443,7 +443,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) .map(|_| { @@ -467,7 +467,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskSet::new().block_on(&mut rt, async { + TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); From da87e96705db0f485553004ccd87991723aba26d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 7 Nov 2019 11:46:39 -0800 Subject: [PATCH 28/73] rename `spawn_local` to `local::spawn` --- tokio/src/lib.rs | 14 ++++----- tokio/src/runtime/local.rs | 59 +++++++++++++++++++------------------- tokio/src/runtime/mod.rs | 2 +- 3 files changed, 37 insertions(+), 38 deletions(-) diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index b56797dbc60..b768d5ff6f7 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -123,18 +123,16 @@ pub mod timer; #[cfg(feature = "rt-full")] mod util; -if_runtime! { +#[cfg(feature = "local")] +pub mod local { + //! Runs `!Send` futures on the current thread. + pub use crate::runtime::local::*; +} +if_runtime! { #[doc(inline)] pub use crate::runtime::spawn; - #[doc(inline)] - #[cfg(feature = "local")] - pub use crate::local::spawn_local; - - #[cfg(feature = "local")] - pub use crate::runtime::local; - #[cfg(not(test))] // Work around for rust-lang/rust#62127 #[cfg(feature = "macros")] #[doc(inline)] diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index e91ea35725c..026902e83ec 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -55,7 +55,7 @@ thread_local! { /// Spawns a `!Send` future on the local task group. /// -/// The spawned future will be run on the same thread that called `spawn_local.` +/// The spawned future will be run on the same thread that called `spawn.` /// This may only be called from the context of a local task group. /// /// # Panics @@ -68,23 +68,22 @@ thread_local! { /// # use tokio::runtime::Runtime; /// use std::rc::Rc; /// use tokio::local; -/// let unsync_data = Rc::new("my unsync data..."); +/// +/// let unsend_data = Rc::new("my unsync data..."); /// /// let mut rt = Runtime::new().unwrap(); /// let task_set = local::TaskGroup::new(); -/// let local_task = task_set.spawn(async move { -/// let more_unsync_data = unsync_data.clone(); -/// tokio::spawn_local(async move { -/// println!("{}", more_unsync_data); -/// // ... -/// }).await.unwrap(); -/// }); /// +/// // Run the local task set. /// task_set.block_on(&mut rt, async move { -/// local_task.await.unwrap(); +/// let unsend_data = unsend_data.clone(); +/// local::spawn(async move { +/// println!("{}", unsend_data); +/// // ... +/// }).await.unwrap(); /// }); /// ``` -pub fn spawn_local(future: F) -> JoinHandle +pub fn spawn(future: F) -> JoinHandle where F: Future + 'static, F::Output: 'static, @@ -132,7 +131,7 @@ impl TaskGroup { /// This runs the given future on the runtime, blocking until it is /// complete, and yielding its resolved result. Any tasks or timers which /// the future spawns internally will be executed on the runtime. The future - /// may also call [`spawn_local`] to spawn additional local futures on the + /// may also call [`spawn`] to spawn additional local futures on the /// current thread. /// /// This method should not be called from an asynchronous context. @@ -151,12 +150,13 @@ impl TaskGroup { /// /// For example, this will panic: /// ```should_panic - /// use tokio::runtime::{Runtime, local, blocking}; + /// use tokio::{blocking, local}; + /// use tokio::runtime::Runtime; /// /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { - /// let join = tokio::spawn_local(async { + /// let join = local::spawn(async { /// let blocking_result = blocking::in_place(|| { /// // ... /// }); @@ -167,12 +167,13 @@ impl TaskGroup { /// ``` /// This, however, will not panic: /// ``` - /// use tokio::runtime::{Runtime, local, blocking}; + /// use tokio::{blocking, local}; + /// use tokio::runtime::Runtime; /// /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { - /// let join = tokio::spawn_local(async { + /// let join = local::spawn(async { /// let blocking_result = blocking::run(|| { /// // ... /// }).await; @@ -182,7 +183,7 @@ impl TaskGroup { /// }) /// ``` /// - /// [`spawn_local`]: fn.spawn_local.html + /// [`spawn`]: fn.spawn.html /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on /// [in-place blocking]: ../blocking/fn.in_place.html /// [`blocking::run`]: ../blocking/fn.run.html @@ -337,7 +338,7 @@ mod tests { fn local_current_thread() { let mut rt = runtime::Builder::new().current_thread().build().unwrap(); TaskGroup::new().block_on(&mut rt, async { - spawn_local(async {}).await.unwrap(); + spawn(async {}).await.unwrap(); }); } @@ -352,7 +353,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) .await @@ -374,7 +375,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn_local(async move { + let join = spawn(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); crate::timer::delay_for(Duration::from_millis(10)).await; assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -397,7 +398,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn_local(async move { + let join = spawn(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::in_place(|| {}); assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -417,7 +418,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn_local(async move { + let join = spawn(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::run(|| { assert!( @@ -433,7 +434,7 @@ mod tests { } #[test] - fn all_spawn_locals_are_local() { + fn all_spawns_are_local() { use futures_util::future; thread_local! { @@ -447,7 +448,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) .map(|_| { - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) }) @@ -459,7 +460,7 @@ mod tests { } #[test] - fn nested_spawn_local_is_local() { + fn nested_spawn_is_local() { thread_local! { static ON_RT_THREAD: Cell = Cell::new(false); } @@ -469,13 +470,13 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn_local(async { + spawn(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) .await diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 7c1393f1260..d0aa3c29926 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -155,7 +155,7 @@ pub use self::global::spawn; mod io; #[cfg(feature = "local")] -pub mod local; +pub(super) mod local; mod park; pub use self::park::{Park, Unpark}; From 267f3f3a7f893c39c30013b202840ce30e995d30 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 7 Nov 2019 12:11:09 -0800 Subject: [PATCH 29/73] improve API docs Signed-off-by: Eliza Weisman --- tokio/src/lib.rs | 61 ++++++++++++++++++++++++++++++++++++++ tokio/src/runtime/local.rs | 54 ++++++++++++++++++++++++++------- 2 files changed, 104 insertions(+), 11 deletions(-) diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index b768d5ff6f7..b86a209ef1c 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -126,6 +126,67 @@ mod util; #[cfg(feature = "local")] pub mod local { //! Runs `!Send` futures on the current thread. + //! + //! In some cases, it is necessary to run one or more futures that do not + //! implement [`Send`] and thus are unsafe to send between threads. In these + //! cases, a [local task group] may be used to schedule one or more `!Send` + //! futures to run together on the same thread. + //! + //! For example, the following code will not compile: + //! + //! ```rust,compile_fail + //! # use tokio::runtime::Runtime; + //! use std::rc::Rc; + //! + //! // `Rc` does not implement `Send`, and thus may not be sent between + //! // threads safely. + //! let unsend_data = Rc::new("my unsend data..."); + //! + //! let mut rt = Runtime::new().unwrap(); + //! + //! rt.block_on(async move { + //! let unsend_data = unsend_data.clone(); + //! // Because the `async` block here moves `unsend_data`, the future is `!Send`. + //! // Since `tokio::spawn` requires the spawned future to implement `Send`, this + //! // will not compile. + //! tokio::spawn(async move { + //! println!("{}", unsend_data); + //! // ... + //! }).await.unwrap(); + //! }); + //! ``` + //! In order to spawn `!Send` futures, we can use a local task group to + //! schedule them on the thread calling [`Runtime::block_on`]. When running + //! inside of the local task group, we can use [`local::spawn`], which can + //! spawn `!Send` futures. For example: + //! + //! ```rust + //! # use tokio::runtime::Runtime; + //! use std::rc::Rc; + //! use tokio::local; + //! + //! let unsend_data = Rc::new("my unsend data..."); + //! + //! let mut rt = Runtime::new().unwrap(); + //! // Construct a local task group that can run `!Send` futures. + //! let local_group = local::TaskGroup::new(); + //! + //! // Run the local task group. + //! local_group.block_on(&mut rt, async move { + //! let unsend_data = unsend_data.clone(); + //! // `local::spawn` ensures that the future is spawned on the local + //! // task group. + //! local::spawn(async move { + //! println!("{}", unsend_data); + //! // ... + //! }).await.unwrap(); + //! }); + //! ``` + //! + //! [`Send`]: https://doc.rust-lang.org/std/marker/trait.Send.html + //! [local task group]: struct.TaskGroup.html + //! [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on + //! [`local::spawn`]: fn.spawn.html pub use crate::runtime::local::*; } diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 026902e83ec..19a6ca8511f 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -69,13 +69,13 @@ thread_local! { /// use std::rc::Rc; /// use tokio::local; /// -/// let unsend_data = Rc::new("my unsync data..."); +/// let unsend_data = Rc::new("my unsend data..."); /// /// let mut rt = Runtime::new().unwrap(); -/// let task_set = local::TaskGroup::new(); +/// let local_group = local::TaskGroup::new(); /// -/// // Run the local task set. -/// task_set.block_on(&mut rt, async move { +/// // Run the local task grou[]. +/// local_group.block_on(&mut rt, async move { /// let unsend_data = unsend_data.clone(); /// local::spawn(async move { /// println!("{}", unsend_data); @@ -104,7 +104,7 @@ where const MAX_TASKS_PER_TICK: usize = 61; impl TaskGroup { - /// Returns a new local task group for the given future. + /// Returns a new local task group. pub fn new() -> Self { Self { scheduler: Rc::new(Scheduler::new()), @@ -115,6 +115,38 @@ impl TaskGroup { /// Spawns a `!Send` task onto the local task group. /// /// This task is guaranteed to be run on the current thread. + /// + /// Unlike the free function [`local::spawn`], this method may be used to + /// spawn local tasks when the task group is _not_ running. For example: + /// ```rust + /// # use tokio::runtime::Runtime; + /// use tokio::local; + /// + /// let mut rt = Runtime::new().unwrap(); + /// let local_group = local::TaskGroup::new(); + /// + /// // Spawn a future on the local group. This future will be run when + /// // we call `block_on` to drive the task group. + /// local_group.spawn(async { + /// // ... + /// }); + /// + /// // Run the local task group. + /// local_group.block_on(&mut rt, async move { + /// // ... + /// }); + /// + /// // When `block_on` finishes, we can spawn _more_ futures, which will + /// // run in subsequent calls to `block_on`. + /// local_group.spawn(async { + /// // ... + /// }); + /// + /// local_group.block_on(&mut rt, async move { + /// // ... + /// }); + /// ``` + /// [`local::spawn`]: fn.spawn.html pub fn spawn(&self, future: F) -> JoinHandle where F: Future + 'static, @@ -131,7 +163,7 @@ impl TaskGroup { /// This runs the given future on the runtime, blocking until it is /// complete, and yielding its resolved result. Any tasks or timers which /// the future spawns internally will be executed on the runtime. The future - /// may also call [`spawn`] to spawn additional local futures on the + /// may also call [`local::spawn`] to spawn additional local futures on the /// current thread. /// /// This method should not be called from an asynchronous context. @@ -150,8 +182,8 @@ impl TaskGroup { /// /// For example, this will panic: /// ```should_panic - /// use tokio::{blocking, local}; - /// use tokio::runtime::Runtime; + /// use tokio::runtime::{blocking, Runtime}; + /// use tokio::local; /// /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); @@ -167,8 +199,8 @@ impl TaskGroup { /// ``` /// This, however, will not panic: /// ``` - /// use tokio::{blocking, local}; - /// use tokio::runtime::Runtime; + /// use tokio::runtime::{blocking, Runtime}; + /// use tokio::local; /// /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); @@ -183,7 +215,7 @@ impl TaskGroup { /// }) /// ``` /// - /// [`spawn`]: fn.spawn.html + /// [`local::spawn`]: fn.spawn.html /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on /// [in-place blocking]: ../blocking/fn.in_place.html /// [`blocking::run`]: ../blocking/fn.run.html From 99e05b294fab63479b821f85fa1eee4c3c8886aa Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 7 Nov 2019 12:32:26 -0800 Subject: [PATCH 30/73] track `timer` being renamed to `time` (#1745) Signed-off-by: Eliza Weisman --- tokio/src/runtime/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local.rs b/tokio/src/runtime/local.rs index 19a6ca8511f..a95b16ac4f5 100644 --- a/tokio/src/runtime/local.rs +++ b/tokio/src/runtime/local.rs @@ -409,7 +409,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); - crate::timer::delay_for(Duration::from_millis(10)).await; + crate::time::delay_for(Duration::from_millis(10)).await; assert!(ON_RT_THREAD.with(|cell| cell.get())); }); join.await.unwrap(); From 1b9b25cb9807ae675d97adcaeb78502251f572f7 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:11:18 -0800 Subject: [PATCH 31/73] move local out of runtime and into task Signed-off-by: Eliza Weisman --- tokio/src/{runtime => task}/local.rs | 0 tokio/src/task/mod.rs | 5 +++++ 2 files changed, 5 insertions(+) rename tokio/src/{runtime => task}/local.rs (100%) diff --git a/tokio/src/runtime/local.rs b/tokio/src/task/local.rs similarity index 100% rename from tokio/src/runtime/local.rs rename to tokio/src/task/local.rs diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 415eae19aee..8dce6b2d984 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -15,6 +15,11 @@ mod join; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::join::JoinHandle; +#[cfg(feature = "local")] +mod local; +#[cfg(feature = "local")] +pub use self::local::{spawn, TaskGroup}; + mod list; pub(crate) use self::list::OwnedList; From b8488c7febf22bfa1da1aa03e20a83af6080be96 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:12:42 -0800 Subject: [PATCH 32/73] rename to `spawn_local` Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 50 ++++++++++++++++++++--------------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index e89b1f848f6..1773a26c2cd 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -55,7 +55,7 @@ thread_local! { /// Spawns a `!Send` future on the local task group. /// -/// The spawned future will be run on the same thread that called `spawn.` +/// The spawned future will be run on the same thread that called `spawn_local.` /// This may only be called from the context of a local task group. /// /// # Panics @@ -77,13 +77,13 @@ thread_local! { /// // Run the local task grou[]. /// local_group.block_on(&mut rt, async move { /// let unsend_data = unsend_data.clone(); -/// local::spawn(async move { +/// spawn_local(async move { /// println!("{}", unsend_data); /// // ... /// }).await.unwrap(); /// }); /// ``` -pub fn spawn(future: F) -> JoinHandle +pub fn spawn_local(future: F) -> JoinHandle where F: Future + 'static, F::Output: 'static, @@ -91,7 +91,7 @@ where CURRENT_TASK_SET.with(|current| { let current = current .get() - .expect("`local::spawn` called from outside of a local::TaskGroup!"); + .expect("`spawn_local` called from outside of a local::TaskGroup!"); unsafe { let (task, handle) = task::joinable_unsend(future); current.as_ref().schedule(task); @@ -116,8 +116,8 @@ impl TaskGroup { /// /// This task is guaranteed to be run on the current thread. /// - /// Unlike the free function [`local::spawn`], this method may be used to - /// spawn local tasks when the task group is _not_ running. For example: + /// Unlike the free function [`spawn_local`], this method may be used to + /// spawn_local local tasks when the task group is _not_ running. For example: /// ```rust /// # use tokio::runtime::Runtime; /// use tokio::local; @@ -127,7 +127,7 @@ impl TaskGroup { /// /// // Spawn a future on the local group. This future will be run when /// // we call `block_on` to drive the task group. - /// local_group.spawn(async { + /// local_group.spawn_local(async { /// // ... /// }); /// @@ -136,9 +136,9 @@ impl TaskGroup { /// // ... /// }); /// - /// // When `block_on` finishes, we can spawn _more_ futures, which will + /// // When `block_on` finishes, we can spawn_local _more_ futures, which will /// // run in subsequent calls to `block_on`. - /// local_group.spawn(async { + /// local_group.spawn_local(async { /// // ... /// }); /// @@ -146,8 +146,8 @@ impl TaskGroup { /// // ... /// }); /// ``` - /// [`local::spawn`]: fn.spawn.html - pub fn spawn(&self, future: F) -> JoinHandle + /// [`spawn_local`]: fn.spawn_local.html + pub fn spawn_local(&self, future: F) -> JoinHandle where F: Future + 'static, F::Output: 'static, @@ -163,7 +163,7 @@ impl TaskGroup { /// This runs the given future on the runtime, blocking until it is /// complete, and yielding its resolved result. Any tasks or timers which /// the future spawns internally will be executed on the runtime. The future - /// may also call [`local::spawn`] to spawn additional local futures on the + /// may also call [`spawn_local`] to spawn_local additional local futures on the /// current thread. /// /// This method should not be called from an asynchronous context. @@ -188,7 +188,7 @@ impl TaskGroup { /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { - /// let join = local::spawn(async { + /// let join = spawn_local(async { /// let blocking_result = blocking::in_place(|| { /// // ... /// }); @@ -205,7 +205,7 @@ impl TaskGroup { /// let mut rt = Runtime::new().unwrap(); /// let local = local::TaskGroup::new(); /// local.block_on(&mut rt, async { - /// let join = local::spawn(async { + /// let join = spawn_local(async { /// let blocking_result = blocking::run(|| { /// // ... /// }).await; @@ -215,7 +215,7 @@ impl TaskGroup { /// }) /// ``` /// - /// [`local::spawn`]: fn.spawn.html + /// [`spawn_local`]: fn.spawn_local.html /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on /// [in-place blocking]: ../blocking/fn.in_place.html /// [`blocking::run`]: ../blocking/fn.run.html @@ -370,7 +370,7 @@ mod tests { fn local_current_thread() { let mut rt = runtime::Builder::new().current_thread().build().unwrap(); TaskGroup::new().block_on(&mut rt, async { - spawn(async {}).await.unwrap(); + spawn_local(async {}).await.unwrap(); }); } @@ -385,7 +385,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) .await @@ -407,7 +407,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn(async move { + let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); crate::time::delay_for(Duration::from_millis(10)).await; assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -430,7 +430,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn(async move { + let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::in_place(|| {}); assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -450,7 +450,7 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - let join = spawn(async move { + let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::run(|| { assert!( @@ -480,7 +480,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) .map(|_| { - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) }) @@ -502,13 +502,13 @@ mod tests { let mut rt = runtime::Runtime::new().unwrap(); TaskGroup::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); - spawn(async { + spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); }) .await From 07317afcb44d6657aa090d9b5a9abc58035fbc8d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:14:14 -0800 Subject: [PATCH 33/73] rename `TaskGroup` -> `LocalSet` Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 56 ++++++++++++++++++++--------------------- tokio/src/task/mod.rs | 2 +- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 1773a26c2cd..0c794096b56 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -15,10 +15,10 @@ use pin_project::pin_project; /// A group of tasks which are executed on the same thread. /// -/// These tasks need not implement `Send`; a local task group provides the +/// These tasks need not implement `Send`; a local task set provides the /// capacity to execute `!Send` futures. #[derive(Debug)] -pub struct TaskGroup { +pub struct LocalSet { scheduler: Rc, _not_send_or_sync: PhantomData<*const ()>, } @@ -53,14 +53,14 @@ thread_local! { static CURRENT_TASK_SET: Cell>> = Cell::new(None); } -/// Spawns a `!Send` future on the local task group. +/// Spawns a `!Send` future on the local task set. /// /// The spawned future will be run on the same thread that called `spawn_local.` -/// This may only be called from the context of a local task group. +/// This may only be called from the context of a local task set. /// /// # Panics /// -/// - This function panics if called outside of a local task group. +/// - This function panics if called outside of a local task set. /// /// # Examples /// @@ -72,7 +72,7 @@ thread_local! { /// let unsend_data = Rc::new("my unsend data..."); /// /// let mut rt = Runtime::new().unwrap(); -/// let local_group = local::TaskGroup::new(); +/// let local_group = local::LocalSet::new(); /// /// // Run the local task grou[]. /// local_group.block_on(&mut rt, async move { @@ -91,7 +91,7 @@ where CURRENT_TASK_SET.with(|current| { let current = current .get() - .expect("`spawn_local` called from outside of a local::TaskGroup!"); + .expect("`spawn_local` called from outside of a local::LocalSet!"); unsafe { let (task, handle) = task::joinable_unsend(future); current.as_ref().schedule(task); @@ -103,8 +103,8 @@ where /// Max number of tasks to poll per tick. const MAX_TASKS_PER_TICK: usize = 61; -impl TaskGroup { - /// Returns a new local task group. +impl LocalSet { + /// Returns a new local task set. pub fn new() -> Self { Self { scheduler: Rc::new(Scheduler::new()), @@ -112,26 +112,26 @@ impl TaskGroup { } } - /// Spawns a `!Send` task onto the local task group. + /// Spawns a `!Send` task onto the local task set. /// /// This task is guaranteed to be run on the current thread. /// /// Unlike the free function [`spawn_local`], this method may be used to - /// spawn_local local tasks when the task group is _not_ running. For example: + /// spawn_local local tasks when the task set is _not_ running. For example: /// ```rust /// # use tokio::runtime::Runtime; /// use tokio::local; /// /// let mut rt = Runtime::new().unwrap(); - /// let local_group = local::TaskGroup::new(); + /// let local_group = local::LocalSet::new(); /// /// // Spawn a future on the local group. This future will be run when - /// // we call `block_on` to drive the task group. + /// // we call `block_on` to drive the task set. /// local_group.spawn_local(async { /// // ... /// }); /// - /// // Run the local task group. + /// // Run the local task set. /// local_group.block_on(&mut rt, async move { /// // ... /// }); @@ -158,7 +158,7 @@ impl TaskGroup { } /// Run a future to completion on the provided runtime, driving any local - /// futures spawned on this task group on the current thread. + /// futures spawned on this task set on the current thread. /// /// This runs the given future on the runtime, blocking until it is /// complete, and yielding its resolved result. Any tasks or timers which @@ -176,7 +176,7 @@ impl TaskGroup { /// # Notes /// /// Since this function internally calls [`Runtime::block_on`], and drives - /// futures in the local task group inside that call to `block_on`, the local + /// futures in the local task set inside that call to `block_on`, the local /// futures may not use [in-place blocking]. If a blocking call needs to be /// issued from a local task, the [`blocking::run`] API may be used instead. /// @@ -186,7 +186,7 @@ impl TaskGroup { /// use tokio::local; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::TaskGroup::new(); + /// let local = local::LocalSet::new(); /// local.block_on(&mut rt, async { /// let join = spawn_local(async { /// let blocking_result = blocking::in_place(|| { @@ -203,7 +203,7 @@ impl TaskGroup { /// use tokio::local; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::TaskGroup::new(); + /// let local = local::LocalSet::new(); /// local.block_on(&mut rt, async { /// let join = spawn_local(async { /// let blocking_result = blocking::run(|| { @@ -229,7 +229,7 @@ impl TaskGroup { } } -impl Default for TaskGroup { +impl Default for LocalSet { fn default() -> Self { Self::new() } @@ -369,7 +369,7 @@ mod tests { #[test] fn local_current_thread() { let mut rt = runtime::Builder::new().current_thread().build().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { spawn_local(async {}).await.unwrap(); }); } @@ -383,7 +383,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -396,7 +396,7 @@ mod tests { #[test] fn local_threadpool_timer() { // This test ensures that runtime services like the timer are properly - // set for the local task group. + // set for the local task set. use std::time::Duration; thread_local! { static ON_RT_THREAD: Cell = Cell::new(false); @@ -405,7 +405,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -428,7 +428,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); @@ -448,14 +448,14 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); runtime::blocking::run(|| { assert!( !ON_RT_THREAD.with(|cell| cell.get()), - "blocking must not run on the local task group's thread" + "blocking must not run on the local task set's thread" ); }) .await; @@ -476,7 +476,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) .map(|_| { @@ -500,7 +500,7 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); let mut rt = runtime::Runtime::new().unwrap(); - TaskGroup::new().block_on(&mut rt, async { + LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { assert!(ON_RT_THREAD.with(|cell| cell.get())); diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 8dce6b2d984..96e71481227 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -18,7 +18,7 @@ pub use self::join::JoinHandle; #[cfg(feature = "local")] mod local; #[cfg(feature = "local")] -pub use self::local::{spawn, TaskGroup}; +pub use self::local::{spawn_local, LocalSet}; mod list; pub(crate) use self::list::OwnedList; From 6f7607e4b6f4d589a7948841701dd75ad8425145 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:14:41 -0800 Subject: [PATCH 34/73] remove root reexport Signed-off-by: Eliza Weisman --- tokio/src/lib.rs | 67 ------------------------------------------------ 1 file changed, 67 deletions(-) diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index 79eb646a7b6..3614bb223c0 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -121,73 +121,6 @@ pub mod time; #[cfg(feature = "rt-full")] mod util; -#[cfg(feature = "local")] -pub mod local { - //! Runs `!Send` futures on the current thread. - //! - //! In some cases, it is necessary to run one or more futures that do not - //! implement [`Send`] and thus are unsafe to send between threads. In these - //! cases, a [local task group] may be used to schedule one or more `!Send` - //! futures to run together on the same thread. - //! - //! For example, the following code will not compile: - //! - //! ```rust,compile_fail - //! # use tokio::runtime::Runtime; - //! use std::rc::Rc; - //! - //! // `Rc` does not implement `Send`, and thus may not be sent between - //! // threads safely. - //! let unsend_data = Rc::new("my unsend data..."); - //! - //! let mut rt = Runtime::new().unwrap(); - //! - //! rt.block_on(async move { - //! let unsend_data = unsend_data.clone(); - //! // Because the `async` block here moves `unsend_data`, the future is `!Send`. - //! // Since `tokio::spawn` requires the spawned future to implement `Send`, this - //! // will not compile. - //! tokio::spawn(async move { - //! println!("{}", unsend_data); - //! // ... - //! }).await.unwrap(); - //! }); - //! ``` - //! In order to spawn `!Send` futures, we can use a local task group to - //! schedule them on the thread calling [`Runtime::block_on`]. When running - //! inside of the local task group, we can use [`local::spawn`], which can - //! spawn `!Send` futures. For example: - //! - //! ```rust - //! # use tokio::runtime::Runtime; - //! use std::rc::Rc; - //! use tokio::local; - //! - //! let unsend_data = Rc::new("my unsend data..."); - //! - //! let mut rt = Runtime::new().unwrap(); - //! // Construct a local task group that can run `!Send` futures. - //! let local_group = local::TaskGroup::new(); - //! - //! // Run the local task group. - //! local_group.block_on(&mut rt, async move { - //! let unsend_data = unsend_data.clone(); - //! // `local::spawn` ensures that the future is spawned on the local - //! // task group. - //! local::spawn(async move { - //! println!("{}", unsend_data); - //! // ... - //! }).await.unwrap(); - //! }); - //! ``` - //! - //! [`Send`]: https://doc.rust-lang.org/std/marker/trait.Send.html - //! [local task group]: struct.TaskGroup.html - //! [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on - //! [`local::spawn`]: fn.spawn.html - pub use crate::runtime::local::*; -} - #[doc(inline)] #[cfg(feature = "rt-core")] pub use crate::runtime::spawn; From 2246de23d179b3ed90d4db0caf44539b6d89af2f Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:19:32 -0800 Subject: [PATCH 35/73] remove from runtime Signed-off-by: Eliza Weisman --- tokio/src/runtime/mod.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index c0103334a90..615c7074981 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -156,9 +156,6 @@ pub use self::handle::Handle; mod io; -#[cfg(feature = "local")] -pub(super) mod local; - mod park; pub use self::park::{Park, Unpark}; From c4f6034f308a2ac81e0527eaef186a60761614a8 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 12 Nov 2019 16:25:06 -0800 Subject: [PATCH 36/73] doctest fixup Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 110 ++++++++++++++++++++++++++++++---------- 1 file changed, 84 insertions(+), 26 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 0c794096b56..322413e9547 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -13,10 +13,68 @@ use std::task::{Context, Poll}; use pin_project::pin_project; -/// A group of tasks which are executed on the same thread. +/// A set of tasks which are executed on the same thread. /// -/// These tasks need not implement `Send`; a local task set provides the -/// capacity to execute `!Send` futures. +/// In some cases, it is necessary to run one or more futures that do not +/// implement [`Send`] and thus are unsafe to send between threads. In these +/// cases, a [local task set] may be used to schedule one or more `!Send` +/// futures to run together on the same thread. +/// +/// For example, the following code will not compile: +/// +/// ```rust,compile_fail +/// # use tokio::runtime::Runtime; +/// use std::rc::Rc; +/// +/// // `Rc` does not implement `Send`, and thus may not be sent between +/// // threads safely. +/// let unsend_data = Rc::new("my unsend data..."); +/// +/// let mut rt = Runtime::new().unwrap(); +/// +/// rt.block_on(async move { +/// let unsend_data = unsend_data.clone(); +/// // Because the `async` block here moves `unsend_data`, the future is `!Send`. +/// // Since `tokio::spawn` requires the spawned future to implement `Send`, this +/// // will not compile. +/// tokio::spawn(async move { +/// println!("{}", unsend_data); +/// // ... +/// }).await.unwrap(); +/// }); +/// ``` +/// In order to spawn `!Send` futures, we can use a local task set to +/// schedule them on the thread calling [`Runtime::block_on`]. When running +/// inside of the local task set, we can use [`task::spawn_local`], which can +/// spawn `!Send` futures. For example: +/// +/// ```rust +/// # use tokio::runtime::Runtime; +/// use std::rc::Rc; +/// use tokio::task; +/// +/// let unsend_data = Rc::new("my unsend data..."); +/// +/// let mut rt = Runtime::new().unwrap(); +/// // Construct a local task set that can run `!Send` futures. +/// let local = task::LocalSet::new(); +/// +/// // Run the local task group. +/// local.block_on(&mut rt, async move { +/// let unsend_data = unsend_data.clone(); +/// // `spawn_local` ensures that the future is spawned on the local +/// // task group. +/// task::spawn_local(async move { +/// println!("{}", unsend_data); +/// // ... +/// }).await.unwrap(); +/// }); +/// ``` +/// +/// [`Send`]: https://doc.rust-lang.org/std/marker/trait.Send.html +/// [local task set]: struct.LocalSet.html +/// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on +/// [`task::spawn_local`]: fn.spawn.html #[derive(Debug)] pub struct LocalSet { scheduler: Rc, @@ -67,17 +125,17 @@ thread_local! { /// ```rust /// # use tokio::runtime::Runtime; /// use std::rc::Rc; -/// use tokio::local; +/// use tokio::task; /// /// let unsend_data = Rc::new("my unsend data..."); /// /// let mut rt = Runtime::new().unwrap(); -/// let local_group = local::LocalSet::new(); +/// let local = task::LocalSet::new(); /// -/// // Run the local task grou[]. -/// local_group.block_on(&mut rt, async move { +/// // Run the local task set. +/// local.block_on(&mut rt, async move { /// let unsend_data = unsend_data.clone(); -/// spawn_local(async move { +/// task::spawn_local(async move { /// println!("{}", unsend_data); /// // ... /// }).await.unwrap(); @@ -120,29 +178,29 @@ impl LocalSet { /// spawn_local local tasks when the task set is _not_ running. For example: /// ```rust /// # use tokio::runtime::Runtime; - /// use tokio::local; + /// use tokio::task; /// /// let mut rt = Runtime::new().unwrap(); - /// let local_group = local::LocalSet::new(); + /// let local = task::LocalSet::new(); /// - /// // Spawn a future on the local group. This future will be run when + /// // Spawn a future on the local set. This future will be run when /// // we call `block_on` to drive the task set. - /// local_group.spawn_local(async { + /// local.spawn_local(async { /// // ... /// }); /// /// // Run the local task set. - /// local_group.block_on(&mut rt, async move { + /// local.block_on(&mut rt, async move { /// // ... /// }); /// /// // When `block_on` finishes, we can spawn_local _more_ futures, which will /// // run in subsequent calls to `block_on`. - /// local_group.spawn_local(async { + /// local.spawn_local(async { /// // ... /// }); /// - /// local_group.block_on(&mut rt, async move { + /// local.block_on(&mut rt, async move { /// // ... /// }); /// ``` @@ -178,17 +236,17 @@ impl LocalSet { /// Since this function internally calls [`Runtime::block_on`], and drives /// futures in the local task set inside that call to `block_on`, the local /// futures may not use [in-place blocking]. If a blocking call needs to be - /// issued from a local task, the [`blocking::run`] API may be used instead. + /// issued from a local task, the [`spawn_blocking`] API may be used instead. /// /// For example, this will panic: /// ```should_panic - /// use tokio::runtime::{blocking, Runtime}; - /// use tokio::local; + /// use tokio::runtime::Runtime; + /// use tokio::{task, blocking}; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::LocalSet::new(); + /// let local = task::LocalSet::new(); /// local.block_on(&mut rt, async { - /// let join = spawn_local(async { + /// let join = task::spawn_local(async { /// let blocking_result = blocking::in_place(|| { /// // ... /// }); @@ -199,14 +257,14 @@ impl LocalSet { /// ``` /// This, however, will not panic: /// ``` - /// use tokio::runtime::{blocking, Runtime}; - /// use tokio::local; + /// use tokio::runtime::Runtime; + /// use tokio::{task, blocking}; /// /// let mut rt = Runtime::new().unwrap(); - /// let local = local::LocalSet::new(); + /// let local = task::LocalSet::new(); /// local.block_on(&mut rt, async { - /// let join = spawn_local(async { - /// let blocking_result = blocking::run(|| { + /// let join = task::spawn_local(async { + /// let blocking_result = blocking::spawn_blocking(|| { /// // ... /// }).await; /// // ... @@ -218,7 +276,7 @@ impl LocalSet { /// [`spawn_local`]: fn.spawn_local.html /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on /// [in-place blocking]: ../blocking/fn.in_place.html - /// [`blocking::run`]: ../blocking/fn.run.html + /// [`spawn_blocking`]: ../blocking/fn.spawn_blocking.html pub fn block_on(&self, rt: &mut crate::runtime::Runtime, future: F) -> F::Output where F: Future + 'static, From f4d8b5a12702df7d9c79f7c42af2a27e8bf6cd74 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 10:31:14 -0800 Subject: [PATCH 37/73] simplify Send marker Thanks to @carllerche for the suggestion; this makes the diff much smaller! Signed-off-by: Eliza Weisman --- tokio/src/blocking/schedule.rs | 6 ++- tokio/src/runtime/current_thread/mod.rs | 8 ++-- tokio/src/runtime/thread_pool/owned.rs | 2 +- tokio/src/runtime/thread_pool/shared.rs | 6 ++- tokio/src/task/core.rs | 22 ++++----- tokio/src/task/harness.rs | 24 +++++----- tokio/src/task/list.rs | 14 +++--- tokio/src/task/local.rs | 18 +++---- tokio/src/task/mod.rs | 58 +++++++++------------- tokio/src/task/raw.rs | 64 ++++++++++++------------- tokio/src/task/waker.rs | 44 ++++++++--------- tokio/src/tests/loom_schedule.rs | 6 ++- tokio/src/tests/mock_schedule.rs | 10 ++-- 13 files changed, 138 insertions(+), 144 deletions(-) diff --git a/tokio/src/blocking/schedule.rs b/tokio/src/blocking/schedule.rs index 3c6fdd19535..fe493aa7617 100644 --- a/tokio/src/blocking/schedule.rs +++ b/tokio/src/blocking/schedule.rs @@ -1,11 +1,11 @@ -use crate::task::{Schedule, Sendable, Task}; +use crate::task::{RequiresSend, Schedule, Task}; /// `task::Schedule` implementation that does nothing. This is unique to the /// blocking scheduler as tasks scheduled are not really futures but blocking /// operations. pub(super) struct NoopSchedule; -impl Schedule for NoopSchedule { +impl Schedule for NoopSchedule { fn bind(&self, _task: &Task) {} fn release(&self, _task: Task) {} @@ -16,3 +16,5 @@ impl Schedule for NoopSchedule { unreachable!(); } } + +impl RequiresSend for NoopSchedule {} diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index 3583e2506ac..cd17c577851 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -1,5 +1,5 @@ use crate::runtime::park::{Park, Unpark}; -use crate::task::{self, JoinHandle, Schedule, Task}; +use crate::task::{self, JoinHandle, RequiresSend, Schedule, Task}; use std::cell::UnsafeCell; use std::collections::VecDeque; @@ -35,7 +35,7 @@ pub(super) struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - owned_tasks: UnsafeCell>, + owned_tasks: UnsafeCell>, /// Local run queue. /// @@ -254,7 +254,7 @@ impl Scheduler { } } -impl Schedule for Scheduler { +impl Schedule for Scheduler { fn bind(&self, task: &Task) { unsafe { (*self.owned_tasks.get()).insert(task); @@ -293,6 +293,8 @@ impl Schedule for Scheduler { } } +impl RequiresSend for Scheduler {} + impl

Drop for CurrentThread

where P: Park, diff --git a/tokio/src/runtime/thread_pool/owned.rs b/tokio/src/runtime/thread_pool/owned.rs index b129ba927ff..88284d5ef56 100644 --- a/tokio/src/runtime/thread_pool/owned.rs +++ b/tokio/src/runtime/thread_pool/owned.rs @@ -39,7 +39,7 @@ pub(super) struct Owned { pub(super) work_queue: queue::Worker>, /// List of tasks owned by the worker - pub(super) owned_tasks: task::OwnedList, task::Sendable>, + pub(super) owned_tasks: task::OwnedList>, } impl

Owned

diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index 37cee49b61a..17ca0c0e083 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,6 +1,6 @@ use crate::runtime::park::Unpark; use crate::runtime::thread_pool::slice; -use crate::task::{self, Schedule, Sendable, Task}; +use crate::task::{self, RequiresSend, Schedule, Task}; use std::ptr; @@ -61,7 +61,7 @@ where } } -impl

Schedule for Shared

+impl

Schedule for Shared

where P: Unpark, { @@ -98,3 +98,5 @@ where Self::schedule(self, task); } } + +impl RequiresSend for Shared

{} diff --git a/tokio/src/task/core.rs b/tokio/src/task/core.rs index 5e4bf7dd59b..5805731519e 100644 --- a/tokio/src/task/core.rs +++ b/tokio/src/task/core.rs @@ -18,12 +18,12 @@ use std::task::{Context, Poll, Waker}; /// It is critical for `Header` to be the first field as the task structure will /// be referenced by both *mut Cell and *mut Header. #[repr(C)] -pub(super) struct Cell { +pub(super) struct Cell { /// Hot task state data pub(super) header: Header, /// Either the future or output, depending on the execution stage. - pub(super) core: Core, + pub(super) core: Core, /// Cold data pub(super) trailer: Trailer, @@ -32,9 +32,8 @@ pub(super) struct Cell { /// The core of the task. /// /// Holds the future or output, depending on the stage of execution. -pub(super) struct Core { +pub(super) struct Core { stage: Stage, - _marker: PhantomData, } /// Crate public as this is also needed by the pool. @@ -76,12 +75,12 @@ enum Stage { Consumed, } -impl Cell { +impl Cell { /// Allocate a new task cell, containing the header, trailer, and core /// structures. - pub(super) fn new(future: T, state: State) -> Box> + pub(super) fn new(future: T, state: State) -> Box> where - S: Schedule, + S: Schedule, { Box::new(Cell { header: Header { @@ -90,12 +89,11 @@ impl Cell { queue_next: UnsafeCell::new(ptr::null()), owned_next: UnsafeCell::new(None), owned_prev: UnsafeCell::new(None), - vtable: raw::vtable::(), + vtable: raw::vtable::(), future_causality: CausalCell::new(()), }, core: Core { stage: Stage::Running(Track::new(future)), - _marker: PhantomData, }, trailer: Trailer { waker: CausalCell::new(MaybeUninit::new(None)), @@ -104,14 +102,14 @@ impl Cell { } } -impl Core { +impl Core { pub(super) fn transition_to_consumed(&mut self) { self.stage = Stage::Consumed } pub(super) fn poll(&mut self, header: &Header) -> Poll where - S: Schedule, + S: Schedule, { let res = { let future = match &mut self.stage { @@ -125,7 +123,7 @@ impl Core { // The waker passed into the `poll` function does not require a ref // count increment. - let waker_ref = waker_ref::(header); + let waker_ref = waker_ref::(header); let mut cx = Context::from_waker(&*waker_ref); future.poll(&mut cx) diff --git a/tokio/src/task/harness.rs b/tokio/src/task/harness.rs index 5fbb2b14dd0..39ea2b5da7b 100644 --- a/tokio/src/task/harness.rs +++ b/tokio/src/task/harness.rs @@ -11,21 +11,21 @@ use std::ptr::NonNull; use std::task::{Poll, Waker}; /// Typed raw task handle -pub(super) struct Harness { - cell: NonNull>, +pub(super) struct Harness { + cell: NonNull>, _p: PhantomData, } -impl Harness +impl Harness where T: Future, S: 'static, { - pub(super) unsafe fn from_raw(ptr: *mut ()) -> Harness { + pub(super) unsafe fn from_raw(ptr: *mut ()) -> Harness { debug_assert!(!ptr.is_null()); Harness { - cell: NonNull::new_unchecked(ptr as *mut Cell), + cell: NonNull::new_unchecked(ptr as *mut Cell), _p: PhantomData, } } @@ -38,15 +38,15 @@ where unsafe { &self.cell.as_ref().trailer } } - fn core(&mut self) -> &mut Core { + fn core(&mut self) -> &mut Core { unsafe { &mut self.cell.as_mut().core } } } -impl Harness +impl Harness where T: Future, - S: Schedule, + S: Schedule, { /// Poll the inner future. /// @@ -99,12 +99,12 @@ where let res = header.future_causality.with_mut(|_| { panic::catch_unwind(panic::AssertUnwindSafe(|| { - struct Guard<'a, T: Future, M> { - core: &'a mut Core, + struct Guard<'a, T: Future> { + core: &'a mut Core, polled: bool, } - impl Drop for Guard<'_, T, M> { + impl Drop for Guard<'_, T> { fn drop(&mut self) { if !self.polled { self.core.transition_to_consumed(); @@ -552,7 +552,7 @@ where self.trailer().waker.with_deferred(|ptr| ptr.read()) } - unsafe fn to_task(&self) -> Task { + unsafe fn to_task(&self) -> Task { let ptr = self.cell.as_ptr() as *mut Header; Task::from_raw(NonNull::new_unchecked(ptr)) } diff --git a/tokio/src/task/list.rs b/tokio/src/task/list.rs index c14b0626085..91bcc725a68 100644 --- a/tokio/src/task/list.rs +++ b/tokio/src/task/list.rs @@ -4,12 +4,12 @@ use std::fmt; use std::marker::PhantomData; use std::ptr::NonNull; -pub(crate) struct OwnedList { +pub(crate) struct OwnedList { head: Option>, - _p: PhantomData<(S, M)>, + _p: PhantomData, } -impl OwnedList { +impl OwnedList { pub(crate) fn new() -> Self { OwnedList { head: None, @@ -17,7 +17,7 @@ impl OwnedList { } } - pub(crate) fn insert(&mut self, task: &Task) { + pub(crate) fn insert(&mut self, task: &Task) { debug_assert!(!self.contains(task)); unsafe { @@ -36,7 +36,7 @@ impl OwnedList { } } - pub(crate) fn remove(&mut self, task: &Task) { + pub(crate) fn remove(&mut self, task: &Task) { debug_assert!(self.head.is_some()); unsafe { @@ -72,7 +72,7 @@ impl OwnedList { } /// Only used by debug assertions - fn contains(&self, task: &Task) -> bool { + fn contains(&self, task: &Task) -> bool { let mut curr = self.head; while let Some(p) = curr { @@ -89,7 +89,7 @@ impl OwnedList { } } -impl fmt::Debug for OwnedList { +impl fmt::Debug for OwnedList { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("OwnedList").finish() } diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 322413e9547..6dc41292ccc 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -1,5 +1,5 @@ //! Runs `!Send` futures on the current thread. -use crate::task::{self, JoinHandle, Schedule, UnsendTask, Unsendable}; +use crate::task::{self, JoinHandle, Schedule, Task}; use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; @@ -87,7 +87,7 @@ struct Scheduler { /// # Safety /// /// Must only be accessed from the primary thread - tasks: UnsafeCell>, + tasks: UnsafeCell>, /// Local run queue. /// @@ -97,7 +97,7 @@ struct Scheduler { /// /// References should not be handed out. Only call `push` / `pop` functions. /// Only call from the owning thread. - queue: UnsafeCell>>, + queue: UnsafeCell>>, } #[pin_project] @@ -317,24 +317,24 @@ impl Future for LocalFuture { // === impl Scheduler === -impl Schedule for Scheduler { - fn bind(&self, task: &UnsendTask) { +impl Schedule for Scheduler { + fn bind(&self, task: &Task) { unsafe { (*self.tasks.get()).insert(task); } } - fn release(&self, _: UnsendTask) { + fn release(&self, _: Task) { unreachable!("tasks should only be completed locally") } - fn release_local(&self, task: &UnsendTask) { + fn release_local(&self, task: &Task) { unsafe { (*self.tasks.get()).remove(task); } } - fn schedule(&self, task: UnsendTask) { + fn schedule(&self, task: Task) { unsafe { (*self.queue.get()).push_front(task); } @@ -378,7 +378,7 @@ impl Scheduler { .unwrap_or(false) } - fn next_task(&self) -> Option> { + fn next_task(&self) -> Option> { unsafe { (*self.queue.get()).pop_front() } } diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 96e71481227..4c253c9c3c1 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -47,54 +47,42 @@ use std::ptr::NonNull; use std::{fmt, mem}; /// An owned handle to the task, tracked by ref count -pub(crate) struct Task { +pub(crate) struct Task { raw: RawTask, - _p: PhantomData<(S, M)>, + _p: PhantomData, } -/// An owned handle to a `!Send` task, tracked by ref count. -#[cfg(feature = "local")] -pub(crate) type UnsendTask = Task; - -/// Marker type indicating that a `Task` was constructed from a future that -/// implements `Send`. -#[derive(Debug)] -pub(crate) struct Sendable; - -/// Marker type indicating that a `Task` was constructed from a future that -/// does not implement `Send`, and may only be scheduled by a scheduler that is -/// capable of scheduling `!Send` tasks. -#[derive(Debug)] -#[cfg(feature = "local")] -pub(crate) struct Unsendable; - -unsafe impl Send for Task {} +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; -pub(crate) trait Schedule: Send + Sync + Sized + 'static { +pub(crate) trait Schedule: Send + Sync + Sized + 'static { /// Bind a task to the executor. /// /// Guaranteed to be called from the thread that called `poll` on the task. - fn bind(&self, task: &Task); + fn bind(&self, task: &Task); /// The task has completed work and is ready to be released. The scheduler /// is free to drop it whenever. - fn release(&self, task: Task); + fn release(&self, task: Task); /// The has been completed by the executor it was bound to. - fn release_local(&self, task: &Task); + fn release_local(&self, task: &Task); /// Schedule the task - fn schedule(&self, task: Task); + fn schedule(&self, task: Task); } +/// Marker trait indicating that a scheduler can only schedule tasks which +/// implement `Send`. +pub(crate) trait RequiresSend: Schedule {} + /// Create a new task without an associated join handle pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: Schedule, + S: Schedule + RequiresSend, { Task { raw: RawTask::new_background::<_, S>(task), @@ -106,7 +94,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: Schedule, + S: Schedule + RequiresSend, { let raw = RawTask::new_joinable::<_, S>(task); @@ -122,10 +110,10 @@ where /// Create a new `!Send` task with an associated join handle #[cfg(feature = "local")] -pub(crate) fn joinable_unsend(task: T) -> (UnsendTask, JoinHandle) +pub(crate) fn joinable_unsend(task: T) -> (Task, JoinHandle) where T: Future + 'static, - S: Schedule, + S: Schedule, { let raw = RawTask::new_joinable_unsend::<_, S>(task); @@ -139,8 +127,8 @@ where (task, join) } -impl Task { - pub(crate) unsafe fn from_raw(ptr: NonNull

) -> Task { +impl Task { + pub(crate) unsafe fn from_raw(ptr: NonNull
) -> Task { Task { raw: RawTask::from_raw(ptr), _p: PhantomData, @@ -158,7 +146,7 @@ impl Task { } } -impl, M> Task { +impl Task { /// Returns `self` when the task needs to be immediately re-scheduled pub(crate) fn run(self, mut executor: F) -> Option where @@ -184,16 +172,14 @@ impl, M> Task { } } -impl Drop for Task { +impl Drop for Task { fn drop(&mut self) { self.raw.drop_task(); } } -impl fmt::Debug for Task { +impl fmt::Debug for Task { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("Task") - .field("send", &format_args!("{}", std::any::type_name::())) - .finish() + fmt.pad("Task") } } diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index e0bff5e5fe3..7b2a8e53faf 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -1,9 +1,7 @@ use crate::loom::alloc::Track; use crate::task::Cell; use crate::task::Harness; -#[cfg(feature = "local")] -use crate::task::Unsendable; -use crate::task::{Header, Schedule, Sendable}; +use crate::task::{Header, RequiresSend, Schedule}; use crate::task::{Snapshot, State}; use std::future::Future; @@ -44,15 +42,15 @@ pub(super) struct Vtable { } /// Get the vtable for the requested `T` and `S` generics. -pub(super) fn vtable, M>() -> &'static Vtable { +pub(super) fn vtable() -> &'static Vtable { &Vtable { - poll: poll::, - drop_task: drop_task::, - read_output: read_output::, - store_join_waker: store_join_waker::, - swap_join_waker: swap_join_waker::, - drop_join_handle_slow: drop_join_handle_slow::, - cancel: cancel::, + poll: poll::, + drop_task: drop_task::, + read_output: read_output::, + store_join_waker: store_join_waker::, + swap_join_waker: swap_join_waker::, + drop_join_handle_slow: drop_join_handle_slow::, + cancel: cancel::, } } @@ -60,32 +58,32 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule + RequiresSend, { - RawTask::new::<_, S, Sendable>(task, State::new_background()) + RawTask::new::<_, S>(task, State::new_background()) } pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule, + S: Schedule + RequiresSend, { - RawTask::new::<_, S, Sendable>(task, State::new_joinable()) + RawTask::new::<_, S>(task, State::new_joinable()) } #[cfg(feature = "local")] pub(super) fn new_joinable_unsend(task: T) -> RawTask where T: Future + 'static, - S: Schedule, + S: Schedule, { - RawTask::new::<_, S, Unsendable>(task, State::new_joinable()) + RawTask::new::<_, S>(task, State::new_joinable()) } - fn new(task: T, state: State) -> RawTask + fn new(task: T, state: State) -> RawTask where T: Future + 'static, - S: Schedule, + S: Schedule, { let ptr = Box::into_raw(Cell::new::(task, state)); let ptr = unsafe { NonNull::new_unchecked(ptr as *mut Header) }; @@ -161,44 +159,44 @@ impl Clone for RawTask { impl Copy for RawTask {} -unsafe fn poll, M>( +unsafe fn poll( ptr: *mut (), executor: &mut dyn FnMut() -> Option>, ) -> bool { - let harness = Harness::::from_raw(ptr); + let harness = Harness::::from_raw(ptr); harness.poll(executor) } -unsafe fn drop_task, M>(ptr: *mut ()) { - let harness = Harness::::from_raw(ptr); +unsafe fn drop_task(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); harness.drop_task(); } -unsafe fn read_output, M>(ptr: *mut (), dst: *mut (), state: Snapshot) { - let harness = Harness::::from_raw(ptr); +unsafe fn read_output(ptr: *mut (), dst: *mut (), state: Snapshot) { + let harness = Harness::::from_raw(ptr); harness.read_output(dst as *mut Track>, state); } -unsafe fn store_join_waker, M>(ptr: *mut (), waker: &Waker) -> Snapshot { - let harness = Harness::::from_raw(ptr); +unsafe fn store_join_waker(ptr: *mut (), waker: &Waker) -> Snapshot { + let harness = Harness::::from_raw(ptr); harness.store_join_waker(waker) } -unsafe fn swap_join_waker, M>( +unsafe fn swap_join_waker( ptr: *mut (), waker: &Waker, prev: Snapshot, ) -> Snapshot { - let harness = Harness::::from_raw(ptr); + let harness = Harness::::from_raw(ptr); harness.swap_join_waker(waker, prev) } -unsafe fn drop_join_handle_slow, M>(ptr: *mut ()) { - let harness = Harness::::from_raw(ptr); +unsafe fn drop_join_handle_slow(ptr: *mut ()) { + let harness = Harness::::from_raw(ptr); harness.drop_join_handle_slow() } -unsafe fn cancel, M>(ptr: *mut (), from_queue: bool) { - let harness = Harness::::from_raw(ptr); +unsafe fn cancel(ptr: *mut (), from_queue: bool) { + let harness = Harness::::from_raw(ptr); harness.cancel(from_queue) } diff --git a/tokio/src/task/waker.rs b/tokio/src/task/waker.rs index 387083ed159..e0e1f36ce9f 100644 --- a/tokio/src/task/waker.rs +++ b/tokio/src/task/waker.rs @@ -13,17 +13,17 @@ pub(super) struct WakerRef<'a, S: 'static> { /// Returns a `WakerRef` which avoids having to pre-emptively increase the /// refcount if there is no need to do so. -pub(super) fn waker_ref(meta: &Header) -> WakerRef<'_, S> +pub(super) fn waker_ref(meta: &Header) -> WakerRef<'_, S> where T: Future, - S: Schedule, + S: Schedule, { let ptr = meta as *const _ as *const (); let vtable = &RawWakerVTable::new( - clone_waker::, + clone_waker::, wake_unreachable, - wake_by_local_ref::, + wake_by_local_ref::, noop, ); @@ -43,30 +43,30 @@ impl ops::Deref for WakerRef<'_, S> { } } -unsafe fn clone_waker(ptr: *const ()) -> RawWaker +unsafe fn clone_waker(ptr: *const ()) -> RawWaker where T: Future, - S: Schedule, + S: Schedule, { let meta = ptr as *const Header; (*meta).state.ref_inc(); let vtable = &RawWakerVTable::new( - clone_waker::, - wake_by_val::, - wake_by_ref::, - drop_waker::, + clone_waker::, + wake_by_val::, + wake_by_ref::, + drop_waker::, ); RawWaker::new(ptr, vtable) } -unsafe fn drop_waker(ptr: *const ()) +unsafe fn drop_waker(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.drop_waker(); } @@ -75,32 +75,32 @@ unsafe fn wake_unreachable(_data: *const ()) { unreachable!(); } -unsafe fn wake_by_val(ptr: *const ()) +unsafe fn wake_by_val(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_val(); } // This function can only be called when on the runtime. -unsafe fn wake_by_local_ref(ptr: *const ()) +unsafe fn wake_by_local_ref(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_local_ref(); } // Wake without consuming the waker -unsafe fn wake_by_ref(ptr: *const ()) +unsafe fn wake_by_ref(ptr: *const ()) where T: Future, - S: Schedule, + S: Schedule, { - let harness = Harness::::from_raw(ptr as *mut _); + let harness = Harness::::from_raw(ptr as *mut _); harness.wake_by_ref(); } diff --git a/tokio/src/tests/loom_schedule.rs b/tokio/src/tests/loom_schedule.rs index 54fa6091100..89c8e34deb9 100644 --- a/tokio/src/tests/loom_schedule.rs +++ b/tokio/src/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{Schedule, Sendable, Task}; +use crate::task::{RequiresSend, Schedule, Task}; use loom::sync::Notify; use std::collections::VecDeque; @@ -32,7 +32,7 @@ impl LoomSchedule { } } -impl Schedule for LoomSchedule { +impl Schedule for LoomSchedule { fn bind(&self, _task: &Task) {} fn release(&self, task: Task) { @@ -49,3 +49,5 @@ impl Schedule for LoomSchedule { self.notify.notify(); } } + +impl RequiresSend for LoomSchedule {} diff --git a/tokio/src/tests/mock_schedule.rs b/tokio/src/tests/mock_schedule.rs index ec2441355bf..502c59c445e 100644 --- a/tokio/src/tests/mock_schedule.rs +++ b/tokio/src/tests/mock_schedule.rs @@ -1,5 +1,5 @@ #![allow(warnings)] -use crate::task::{Header, Schedule, Sendable, Task}; +use crate::task::{Header, RequiresSend, Schedule, Task}; use std::collections::VecDeque; use std::sync::Mutex; @@ -82,7 +82,7 @@ impl Mock { } } -impl Schedule for Mock { +impl Schedule for Mock { fn bind(&self, task: &Task) { match self.next("bind") { Call::Bind(ptr) => { @@ -111,6 +111,8 @@ impl Schedule for Mock { } } +impl RequiresSend for Mock {} + impl Drop for Mock { fn drop(&mut self) { if !thread::panicking() { @@ -119,7 +121,7 @@ impl Drop for Mock { } } -impl Schedule for Noop { +impl Schedule for Noop { fn bind(&self, _task: &Task) {} fn release(&self, _task: Task) {} @@ -128,3 +130,5 @@ impl Schedule for Noop { fn schedule(&self, _task: Task) {} } + +impl RequiresSend for Noop {} From 8f21118dd22ac48e61a16964445c216b158d9d78 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 10:32:02 -0800 Subject: [PATCH 38/73] fix tests using the wrong blocking Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 6dc41292ccc..65294b04885 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -422,7 +422,7 @@ impl Drop for Scheduler { #[cfg(all(test, not(loom)))] mod tests { use super::*; - use crate::runtime; + use crate::{blocking, runtime}; #[test] fn local_current_thread() { @@ -490,7 +490,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); - runtime::blocking::in_place(|| {}); + blocking::in_place(|| {}); assert!(ON_RT_THREAD.with(|cell| cell.get())); }); join.await.unwrap(); @@ -510,7 +510,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); - runtime::blocking::run(|| { + blocking::spawn_blocking(|| { assert!( !ON_RT_THREAD.with(|cell| cell.get()), "blocking must not run on the local task set's thread" From cbc1a426e61797ecf9dadeac64b90023002f5c49 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 10:33:56 -0800 Subject: [PATCH 39/73] naming + internal docs improvement Signed-off-by: Eliza Weisman --- tokio/src/blocking/schedule.rs | 4 ++-- tokio/src/runtime/current_thread/mod.rs | 4 ++-- tokio/src/runtime/thread_pool/shared.rs | 4 ++-- tokio/src/task/local.rs | 4 ++-- tokio/src/task/mod.rs | 15 +++++++++------ tokio/src/task/raw.rs | 8 ++++---- tokio/src/tests/loom_schedule.rs | 4 ++-- tokio/src/tests/mock_schedule.rs | 6 +++--- 8 files changed, 26 insertions(+), 23 deletions(-) diff --git a/tokio/src/blocking/schedule.rs b/tokio/src/blocking/schedule.rs index fe493aa7617..01f9ddb1bfb 100644 --- a/tokio/src/blocking/schedule.rs +++ b/tokio/src/blocking/schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{RequiresSend, Schedule, Task}; +use crate::task::{ScheduleSend, Schedule, Task}; /// `task::Schedule` implementation that does nothing. This is unique to the /// blocking scheduler as tasks scheduled are not really futures but blocking @@ -17,4 +17,4 @@ impl Schedule for NoopSchedule { } } -impl RequiresSend for NoopSchedule {} +impl ScheduleSend for NoopSchedule {} diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index cd17c577851..60cac3dffd8 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -1,5 +1,5 @@ use crate::runtime::park::{Park, Unpark}; -use crate::task::{self, JoinHandle, RequiresSend, Schedule, Task}; +use crate::task::{self, JoinHandle, ScheduleSend, Schedule, Task}; use std::cell::UnsafeCell; use std::collections::VecDeque; @@ -293,7 +293,7 @@ impl Schedule for Scheduler { } } -impl RequiresSend for Scheduler {} +impl ScheduleSend for Scheduler {} impl

Drop for CurrentThread

where diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index 17ca0c0e083..f653fb10033 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,6 +1,6 @@ use crate::runtime::park::Unpark; use crate::runtime::thread_pool::slice; -use crate::task::{self, RequiresSend, Schedule, Task}; +use crate::task::{self, Schedule, ScheduleSend, Task}; use std::ptr; @@ -99,4 +99,4 @@ where } } -impl RequiresSend for Shared

{} +impl ScheduleSend for Shared

{} diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 65294b04885..90643ad7eb8 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -151,7 +151,7 @@ where .get() .expect("`spawn_local` called from outside of a local::LocalSet!"); unsafe { - let (task, handle) = task::joinable_unsend(future); + let (task, handle) = task::joinable_local(future); current.as_ref().schedule(task); handle } @@ -210,7 +210,7 @@ impl LocalSet { F: Future + 'static, F::Output: 'static, { - let (task, handle) = task::joinable_unsend(future); + let (task, handle) = task::joinable_local(future); self.scheduler.schedule(task); handle } diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 4c253c9c3c1..1149c726c9b 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -52,7 +52,7 @@ pub(crate) struct Task { _p: PhantomData, } -unsafe impl Send for Task {} +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; @@ -76,13 +76,16 @@ pub(crate) trait Schedule: Send + Sync + Sized + 'static { /// Marker trait indicating that a scheduler can only schedule tasks which /// implement `Send`. -pub(crate) trait RequiresSend: Schedule {} +/// +/// Schedulers that implement this trait may not schedule `!Send` futures. If +/// trait is implemented, the corresponding `Task` type will implement `Send`. +pub(crate) trait ScheduleSend: Schedule {} /// Create a new task without an associated join handle pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: Schedule + RequiresSend, + S: Schedule + ScheduleSend, { Task { raw: RawTask::new_background::<_, S>(task), @@ -94,7 +97,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: Schedule + RequiresSend, + S: Schedule + ScheduleSend, { let raw = RawTask::new_joinable::<_, S>(task); @@ -110,12 +113,12 @@ where /// Create a new `!Send` task with an associated join handle #[cfg(feature = "local")] -pub(crate) fn joinable_unsend(task: T) -> (Task, JoinHandle) +pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) where T: Future + 'static, S: Schedule, { - let raw = RawTask::new_joinable_unsend::<_, S>(task); + let raw = RawTask::new_joinable_local::<_, S>(task); let task = Task { raw, diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 7b2a8e53faf..3e48730331e 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -1,7 +1,7 @@ use crate::loom::alloc::Track; use crate::task::Cell; use crate::task::Harness; -use crate::task::{Header, RequiresSend, Schedule}; +use crate::task::{Header, Schedule, ScheduleSend}; use crate::task::{Snapshot, State}; use std::future::Future; @@ -58,7 +58,7 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule + RequiresSend, + S: Schedule + ScheduleSend, { RawTask::new::<_, S>(task, State::new_background()) } @@ -66,13 +66,13 @@ impl RawTask { pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule + RequiresSend, + S: Schedule + ScheduleSend, { RawTask::new::<_, S>(task, State::new_joinable()) } #[cfg(feature = "local")] - pub(super) fn new_joinable_unsend(task: T) -> RawTask + pub(super) fn new_joinable_local(task: T) -> RawTask where T: Future + 'static, S: Schedule, diff --git a/tokio/src/tests/loom_schedule.rs b/tokio/src/tests/loom_schedule.rs index 89c8e34deb9..5975312c4d8 100644 --- a/tokio/src/tests/loom_schedule.rs +++ b/tokio/src/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{RequiresSend, Schedule, Task}; +use crate::task::{ScheduleSend, Schedule, Task}; use loom::sync::Notify; use std::collections::VecDeque; @@ -50,4 +50,4 @@ impl Schedule for LoomSchedule { } } -impl RequiresSend for LoomSchedule {} +impl ScheduleSend for LoomSchedule {} diff --git a/tokio/src/tests/mock_schedule.rs b/tokio/src/tests/mock_schedule.rs index 502c59c445e..6b0950f4e8e 100644 --- a/tokio/src/tests/mock_schedule.rs +++ b/tokio/src/tests/mock_schedule.rs @@ -1,5 +1,5 @@ #![allow(warnings)] -use crate::task::{Header, RequiresSend, Schedule, Task}; +use crate::task::{Header, ScheduleSend, Schedule, Task}; use std::collections::VecDeque; use std::sync::Mutex; @@ -111,7 +111,7 @@ impl Schedule for Mock { } } -impl RequiresSend for Mock {} +impl ScheduleSend for Mock {} impl Drop for Mock { fn drop(&mut self) { @@ -131,4 +131,4 @@ impl Schedule for Noop { fn schedule(&self, _task: Task) {} } -impl RequiresSend for Noop {} +impl ScheduleSend for Noop {} From 72472ca4cc6005410c8eb0f886743d8586b1f62e Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 10:51:25 -0800 Subject: [PATCH 40/73] set thread local in block_on Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 90643ad7eb8..d4c6ddfdaf7 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -283,7 +283,8 @@ impl LocalSet { F::Output: 'static, { let scheduler = self.scheduler.clone(); - rt.block_on(LocalFuture { scheduler, future }) + self.scheduler + .with(move || rt.block_on(LocalFuture { scheduler, future })) } } @@ -301,17 +302,15 @@ impl Future for LocalFuture { let scheduler = this.scheduler; let future = this.future; - scheduler.with(|| { - scheduler.tick(); + scheduler.tick(); - match future.poll(cx) { - Poll::Ready(v) => Poll::Ready(v), - Poll::Pending => { - cx.waker().wake_by_ref(); - Poll::Pending - } + match future.poll(cx) { + Poll::Ready(v) => Poll::Ready(v), + Poll::Pending => { + cx.waker().wake_by_ref(); + Poll::Pending } - }) + } } } From 927ae9b3e792dd4a57e41daf243cb8e329d70514 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 11:37:37 -0800 Subject: [PATCH 41/73] fix local future spinning instead of parking Signed-off-by: Eliza Weisman --- tokio/src/runtime/mod.rs | 2 +- tokio/src/task/local.rs | 34 +++++++++++++++++++++++++--------- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 615c7074981..f6d14b597f1 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -156,7 +156,7 @@ pub use self::handle::Handle; mod io; -mod park; +pub(crate) mod park; pub use self::park::{Park, Unpark}; mod shell; diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index d4c6ddfdaf7..b51516f2ac6 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -1,4 +1,5 @@ //! Runs `!Send` futures on the current thread. +use crate::runtime::park::{CachedParkThread, Park, Unpark}; use crate::task::{self, JoinHandle, Schedule, Task}; use std::cell::{Cell, UnsafeCell}; @@ -10,6 +11,7 @@ use std::pin::Pin; use std::ptr::{self, NonNull}; use std::rc::Rc; use std::task::{Context, Poll}; +use std::time::Duration; use pin_project::pin_project; @@ -98,6 +100,9 @@ struct Scheduler { /// References should not be handed out. Only call `push` / `pop` functions. /// Only call from the owning thread. queue: UnsafeCell>>, + + park: UnsafeCell, + unpark: Box, } #[pin_project] @@ -300,16 +305,14 @@ impl Future for LocalFuture { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.project(); let scheduler = this.scheduler; - let future = this.future; - - scheduler.tick(); + let mut future = this.future; - match future.poll(cx) { - Poll::Ready(v) => Poll::Ready(v), - Poll::Pending => { - cx.waker().wake_by_ref(); - Poll::Pending + loop { + if let Poll::Ready(output) = future.as_mut().poll(cx) { + return Poll::Ready(output); } + + scheduler.tick(); } } } @@ -337,14 +340,19 @@ impl Schedule for Scheduler { unsafe { (*self.queue.get()).push_front(task); } + self.unpark.unpark(); } } impl Scheduler { fn new() -> Self { + let park = CachedParkThread::new(); + let unpark = park.unpark(); Self { tasks: UnsafeCell::new(task::OwnedList::new()), queue: UnsafeCell::new(VecDeque::with_capacity(64)), + park: UnsafeCell::new(park), + unpark: Box::new(unpark), } } @@ -383,16 +391,24 @@ impl Scheduler { fn tick(&self) { assert!(self.is_current()); + let park = unsafe { &mut (*self.park.get()) }; for _ in 0..MAX_TASKS_PER_TICK { let task = match self.next_task() { Some(task) => task, - None => return, + None => { + park.park().ok().expect("failed to park local task set"); + return; + } }; if let Some(task) = task.run(&mut || Some(self.into())) { self.schedule(task); } } + + park.park_timeout(Duration::from_millis(0)) + .ok() + .expect("failed to park local task set"); } } From 34e3428205c60aa1d3b8257ce7110d536fad44f0 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 11:40:06 -0800 Subject: [PATCH 42/73] local scheduler doesn't need to be Send + Sync it *shouldn't* be, because it's just a pile of UnsafeCells... Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 3 --- tokio/src/task/mod.rs | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index b51516f2ac6..06b8da9a3e6 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -412,9 +412,6 @@ impl Scheduler { } } -unsafe impl Send for Scheduler {} -unsafe impl Sync for Scheduler {} - impl fmt::Debug for Scheduler { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Scheduler { .. }").finish() diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 1149c726c9b..7a0a8dd3b99 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -57,7 +57,7 @@ unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; -pub(crate) trait Schedule: Send + Sync + Sized + 'static { +pub(crate) trait Schedule: Sized + 'static { /// Bind a task to the executor. /// /// Guaranteed to be called from the thread that called `poll` on the task. @@ -79,7 +79,7 @@ pub(crate) trait Schedule: Send + Sync + Sized + 'static { /// /// Schedulers that implement this trait may not schedule `!Send` futures. If /// trait is implemented, the corresponding `Task` type will implement `Send`. -pub(crate) trait ScheduleSend: Schedule {} +pub(crate) trait ScheduleSend: Schedule + Send + Sync {} /// Create a new task without an associated join handle pub(crate) fn background(task: T) -> Task From 0fbe6ab29d56935a5104f4021869dd4dd4fb745d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 12:47:11 -0800 Subject: [PATCH 43/73] add tests for waking a local set from IO Signed-off-by: Eliza Weisman --- tokio/tests/rt_common.rs | 63 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index 73982ced3f8..d0e3df3dd88 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -418,4 +418,67 @@ rt_test! { assert_eq!(buf, b"hello"); tx.send(()).unwrap(); } + + mod local_set { + use tokio::task; + use super::*; + + #[test] + fn block_on_socket() { + let mut rt = rt(); + let local = task::LocalSet::new(); + + local.block_on(&mut rt, async move { + let (tx, rx) = oneshot::channel(); + + let mut listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + let addr = listener.local_addr().unwrap(); + + task::spawn_local(async move { + let _ = listener.accept().await; + tx.send(()).unwrap(); + }); + + TcpStream::connect(&addr).await.unwrap(); + rx.await.unwrap(); + }); + } + + #[test] + fn client_server_block_on() { + let mut rt = rt(); + let (tx, rx) = mpsc::channel(); + + let local = task::LocalSet::new(); + + local.block_on(&mut rt, async move { client_server_local(tx).await }); + + assert_ok!(rx.try_recv()); + assert_err!(rx.try_recv()); + } + + async fn client_server_local(tx: mpsc::Sender<()>) { + let mut server = assert_ok!(TcpListener::bind("127.0.0.1:0").await); + + // Get the assigned address + let addr = assert_ok!(server.local_addr()); + + // Spawn the server + task::spawn_local(async move { + // Accept a socket + let (mut socket, _) = server.accept().await.unwrap(); + + // Write some data + socket.write_all(b"hello").await.unwrap(); + }); + + let mut client = TcpStream::connect(&addr).await.unwrap(); + + let mut buf = vec![]; + client.read_to_end(&mut buf).await.unwrap(); + + assert_eq!(buf, b"hello"); + tx.send(()).unwrap(); + } + } } From 1ad14c751052c3141d35ae1ba8bc467ba01ce799 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 12:49:37 -0800 Subject: [PATCH 44/73] use the rt's waker in block_on this fixes the local set not being woken by IO on the current-thread rt Signed-off-by: Eliza Weisman --- tokio/src/runtime/mod.rs | 2 +- tokio/src/task/local.rs | 46 ++++++++++++++++++---------------------- 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index f6d14b597f1..615c7074981 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -156,7 +156,7 @@ pub use self::handle::Handle; mod io; -pub(crate) mod park; +mod park; pub use self::park::{Park, Unpark}; mod shell; diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 06b8da9a3e6..5f26c4162e5 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -1,5 +1,4 @@ //! Runs `!Send` futures on the current thread. -use crate::runtime::park::{CachedParkThread, Park, Unpark}; use crate::task::{self, JoinHandle, Schedule, Task}; use std::cell::{Cell, UnsafeCell}; @@ -10,8 +9,7 @@ use std::marker::PhantomData; use std::pin::Pin; use std::ptr::{self, NonNull}; use std::rc::Rc; -use std::task::{Context, Poll}; -use std::time::Duration; +use std::task::{Context, Poll, Waker}; use pin_project::pin_project; @@ -101,8 +99,7 @@ struct Scheduler { /// Only call from the owning thread. queue: UnsafeCell>>, - park: UnsafeCell, - unpark: Box, + waker: UnsafeCell>, } #[pin_project] @@ -306,14 +303,16 @@ impl Future for LocalFuture { let this = self.project(); let scheduler = this.scheduler; let mut future = this.future; + unsafe { + (*scheduler.waker.get()) = Some(cx.waker().clone()); + } - loop { - if let Poll::Ready(output) = future.as_mut().poll(cx) { - return Poll::Ready(output); - } - - scheduler.tick(); + if let Poll::Ready(output) = future.as_mut().poll(cx) { + return Poll::Ready(output); } + + scheduler.tick(); + Poll::Pending } } @@ -340,19 +339,19 @@ impl Schedule for Scheduler { unsafe { (*self.queue.get()).push_front(task); } - self.unpark.unpark(); + if let Some(waker) = self.waker() { + // If we are running, wake the local future. + waker.wake_by_ref(); + } } } impl Scheduler { fn new() -> Self { - let park = CachedParkThread::new(); - let unpark = park.unpark(); Self { tasks: UnsafeCell::new(task::OwnedList::new()), queue: UnsafeCell::new(VecDeque::with_capacity(64)), - park: UnsafeCell::new(park), - unpark: Box::new(unpark), + waker: UnsafeCell::new(None), } } @@ -389,26 +388,22 @@ impl Scheduler { unsafe { (*self.queue.get()).pop_front() } } + fn waker(&self) -> Option<&Waker> { + unsafe { (*self.waker.get()).as_ref() } + } + fn tick(&self) { assert!(self.is_current()); - let park = unsafe { &mut (*self.park.get()) }; for _ in 0..MAX_TASKS_PER_TICK { let task = match self.next_task() { Some(task) => task, - None => { - park.park().ok().expect("failed to park local task set"); - return; - } + None => return, }; if let Some(task) = task.run(&mut || Some(self.into())) { self.schedule(task); } } - - park.park_timeout(Duration::from_millis(0)) - .ok() - .expect("failed to park local task set"); } } @@ -417,6 +412,7 @@ impl fmt::Debug for Scheduler { fmt.debug_struct("Scheduler { .. }").finish() } } + impl Drop for Scheduler { fn drop(&mut self) { // Drain all local tasks From ab19812d78eb75814907b7d5f660fbd8b26b8fa6 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 12:56:15 -0800 Subject: [PATCH 45/73] rm phantomdata Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 5f26c4162e5..fc9aab3742b 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -5,7 +5,6 @@ use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; use std::fmt; use std::future::Future; -use std::marker::PhantomData; use std::pin::Pin; use std::ptr::{self, NonNull}; use std::rc::Rc; @@ -78,7 +77,6 @@ use pin_project::pin_project; #[derive(Debug)] pub struct LocalSet { scheduler: Rc, - _not_send_or_sync: PhantomData<*const ()>, } struct Scheduler { @@ -168,7 +166,6 @@ impl LocalSet { pub fn new() -> Self { Self { scheduler: Rc::new(Scheduler::new()), - _not_send_or_sync: PhantomData, } } From 1007cadfbd4bd05842cc5660bd20dc6ca3f7087f Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 13:55:20 -0800 Subject: [PATCH 46/73] fix local queue being pushed to from remote Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 75 +++++++++++++++++++++++++++++------------ 1 file changed, 54 insertions(+), 21 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index fc9aab3742b..5454addd3ef 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -8,6 +8,7 @@ use std::future::Future; use std::pin::Pin; use std::ptr::{self, NonNull}; use std::rc::Rc; +use std::sync::Mutex; use std::task::{Context, Poll, Waker}; use pin_project::pin_project; @@ -87,17 +88,27 @@ struct Scheduler { /// Must only be accessed from the primary thread tasks: UnsafeCell>, - /// Local run queue. + /// Local run local_queue. /// - /// Tasks notified from the current thread are pushed into this queue. + /// Tasks notified from the current thread are pushed into this local_queue. /// /// # Safety /// /// References should not be handed out. Only call `push` / `pop` functions. /// Only call from the owning thread. - queue: UnsafeCell>>, + local_queue: UnsafeCell>>, - waker: UnsafeCell>, + /// Shared state + shared: Mutex, +} + +struct Shared { + waker: Option, + + /// Remote run queue. + /// + /// Tasks notified from another thread are pushed into this queue. + remote_queue: VecDeque>, } #[pin_project] @@ -152,7 +163,7 @@ where .expect("`spawn_local` called from outside of a local::LocalSet!"); unsafe { let (task, handle) = task::joinable_local(future); - current.as_ref().schedule(task); + current.as_ref().schedule_local(task); handle } }) @@ -210,7 +221,11 @@ impl LocalSet { F::Output: 'static, { let (task, handle) = task::joinable_local(future); - self.scheduler.schedule(task); + unsafe { + // This is safe: since `LocalSet` is not Send or Sync, this is + // always being called from the local thread. + self.scheduler.schedule_local(task); + } handle } @@ -300,9 +315,7 @@ impl Future for LocalFuture { let this = self.project(); let scheduler = this.scheduler; let mut future = this.future; - unsafe { - (*scheduler.waker.get()) = Some(cx.waker().clone()); - } + scheduler.shared.lock().unwrap().waker = Some(cx.waker().clone()); if let Poll::Ready(output) = future.as_mut().poll(cx) { return Poll::Ready(output); @@ -333,12 +346,18 @@ impl Schedule for Scheduler { } fn schedule(&self, task: Task) { - unsafe { - (*self.queue.get()).push_front(task); - } - if let Some(waker) = self.waker() { - // If we are running, wake the local future. - waker.wake_by_ref(); + if self.is_current() { + unsafe { + self.schedule_local(task); + } + } else { + let mut lock = self.shared.lock().unwrap(); + + lock.remote_queue.push_back(task); + lock.waker + .as_ref() + .expect("if a local task is notified, the local task set must be running!") + .wake_by_ref(); } } } @@ -347,8 +366,11 @@ impl Scheduler { fn new() -> Self { Self { tasks: UnsafeCell::new(task::OwnedList::new()), - queue: UnsafeCell::new(VecDeque::with_capacity(64)), - waker: UnsafeCell::new(None), + local_queue: UnsafeCell::new(VecDeque::with_capacity(64)), + shared: Mutex::new(Shared { + waker: None, + remote_queue: VecDeque::with_capacity(64), + }), } } @@ -370,6 +392,10 @@ impl Scheduler { }) } + unsafe fn schedule_local(&self, task: Task) { + (*self.local_queue.get()).push_front(task); + } + fn is_current(&self) -> bool { CURRENT_TASK_SET .try_with(|current| { @@ -382,11 +408,15 @@ impl Scheduler { } fn next_task(&self) -> Option> { - unsafe { (*self.queue.get()).pop_front() } + self.next_local_task().or_else(|| self.next_remote_task()) + } + + fn next_local_task(&self) -> Option> { + unsafe { (*self.local_queue.get()).pop_front() } } - fn waker(&self) -> Option<&Waker> { - unsafe { (*self.waker.get()).as_ref() } + fn next_remote_task(&self) -> Option> { + self.shared.lock().unwrap().remote_queue.pop_front() } fn tick(&self) { @@ -398,7 +428,10 @@ impl Scheduler { }; if let Some(task) = task.run(&mut || Some(self.into())) { - self.schedule(task); + unsafe { + // we are on the local thread, so this is okay. + self.schedule_local(task); + } } } } From 5265b410d87a248f0ec147b1cf74e536e3b29dc7 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 13:59:27 -0800 Subject: [PATCH 47/73] remove "local" feature Signed-off-by: Eliza Weisman --- tokio/Cargo.toml | 4 +--- tokio/src/task/mod.rs | 5 ++--- tokio/src/task/raw.rs | 1 - 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 87851147272..7400c1fc38f 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -28,7 +28,6 @@ default = [ "blocking", "fs", "io-util", - "local", "net", "process", "rt-full", @@ -42,7 +41,6 @@ dns = ["blocking"] fs = ["blocking"] io-driver = ["mio", "lazy_static", "sync"] # TODO: get rid of sync io-util = ["pin-project", "memchr"] -local = ["rt-core", "pin-project"] macros = ["tokio-macros"] net = ["dns", "tcp", "udp", "uds"] process = [ @@ -56,7 +54,7 @@ process = [ "winapi/winerror", ] # Includes basic task execution capabilities -rt-core = [] +rt-core = ["pin-project"] rt-full = [ "macros", "num_cpus", diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 7a0a8dd3b99..ed18cbe316c 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -15,9 +15,9 @@ mod join; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::join::JoinHandle; -#[cfg(feature = "local")] +#[cfg(feature = "rt-core")] mod local; -#[cfg(feature = "local")] +#[cfg(feature = "rt-core")] pub use self::local::{spawn_local, LocalSet}; mod list; @@ -112,7 +112,6 @@ where } /// Create a new `!Send` task with an associated join handle -#[cfg(feature = "local")] pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) where T: Future + 'static, diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 3e48730331e..79973f73975 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -71,7 +71,6 @@ impl RawTask { RawTask::new::<_, S>(task, State::new_joinable()) } - #[cfg(feature = "local")] pub(super) fn new_joinable_local(task: T) -> RawTask where T: Future + 'static, From c42992b7cd8559a6f8b72c107e91653bc63a8da2 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:02:33 -0800 Subject: [PATCH 48/73] add additional assertions Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 5454addd3ef..7d95d6dd311 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -330,6 +330,7 @@ impl Future for LocalFuture { impl Schedule for Scheduler { fn bind(&self, task: &Task) { + assert!(self.is_current()); unsafe { (*self.tasks.get()).insert(task); } @@ -340,6 +341,7 @@ impl Schedule for Scheduler { } fn release_local(&self, task: &Task) { + debug_assert!(self.is_current()); unsafe { (*self.tasks.get()).remove(task); } From 0514a5601cbcf8444bd8f52214e04324bdccb463 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:12:01 -0800 Subject: [PATCH 49/73] use AtomicWaker instead of mutex Signed-off-by: Eliza Weisman --- tokio/Cargo.toml | 3 ++- tokio/src/task/local.rs | 36 ++++++++++++++---------------------- tokio/src/task/mod.rs | 4 ++-- 3 files changed, 18 insertions(+), 25 deletions(-) diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 7400c1fc38f..903e0f8f1ef 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -54,11 +54,12 @@ process = [ "winapi/winerror", ] # Includes basic task execution capabilities -rt-core = ["pin-project"] +rt-core = [] rt-full = [ "macros", "num_cpus", "net", + "pin-project", "rt-core", "sync", "time", diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 7d95d6dd311..4415b1c61e6 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -1,4 +1,5 @@ //! Runs `!Send` futures on the current thread. +use crate::sync::AtomicWaker; use crate::task::{self, JoinHandle, Schedule, Task}; use std::cell::{Cell, UnsafeCell}; @@ -9,7 +10,7 @@ use std::pin::Pin; use std::ptr::{self, NonNull}; use std::rc::Rc; use std::sync::Mutex; -use std::task::{Context, Poll, Waker}; +use std::task::{Context, Poll}; use pin_project::pin_project; @@ -90,7 +91,7 @@ struct Scheduler { /// Local run local_queue. /// - /// Tasks notified from the current thread are pushed into this local_queue. + /// Tasks notified from the current thread are pushed into this queue. /// /// # Safety /// @@ -98,17 +99,14 @@ struct Scheduler { /// Only call from the owning thread. local_queue: UnsafeCell>>, - /// Shared state - shared: Mutex, -} - -struct Shared { - waker: Option, - /// Remote run queue. /// /// Tasks notified from another thread are pushed into this queue. - remote_queue: VecDeque>, + remote_queue: Mutex>>, + + /// Used to notify the `LocalFuture` when a task in the local task set is + /// notified. + waker: AtomicWaker, } #[pin_project] @@ -315,7 +313,7 @@ impl Future for LocalFuture { let this = self.project(); let scheduler = this.scheduler; let mut future = this.future; - scheduler.shared.lock().unwrap().waker = Some(cx.waker().clone()); + scheduler.waker.register_by_ref(cx.waker()); if let Poll::Ready(output) = future.as_mut().poll(cx) { return Poll::Ready(output); @@ -353,13 +351,9 @@ impl Schedule for Scheduler { self.schedule_local(task); } } else { - let mut lock = self.shared.lock().unwrap(); + self.remote_queue.lock().unwrap().push_back(task); - lock.remote_queue.push_back(task); - lock.waker - .as_ref() - .expect("if a local task is notified, the local task set must be running!") - .wake_by_ref(); + self.waker.wake(); } } } @@ -369,10 +363,8 @@ impl Scheduler { Self { tasks: UnsafeCell::new(task::OwnedList::new()), local_queue: UnsafeCell::new(VecDeque::with_capacity(64)), - shared: Mutex::new(Shared { - waker: None, - remote_queue: VecDeque::with_capacity(64), - }), + remote_queue: Mutex::new(VecDeque::with_capacity(64)), + waker: AtomicWaker::new(), } } @@ -418,7 +410,7 @@ impl Scheduler { } fn next_remote_task(&self) -> Option> { - self.shared.lock().unwrap().remote_queue.pop_front() + self.remote_queue.lock().unwrap().pop_front() } fn tick(&self) { diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index ed18cbe316c..264a9383a52 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -15,9 +15,9 @@ mod join; #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 pub use self::join::JoinHandle; -#[cfg(feature = "rt-core")] +#[cfg(feature = "rt-full")] mod local; -#[cfg(feature = "rt-core")] +#[cfg(feature = "rt-full")] pub use self::local::{spawn_local, LocalSet}; mod list; From c7b0cb79ee602bcfa02bc9ef70dae1ec0ad25936 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:14:20 -0800 Subject: [PATCH 50/73] avoid double panics Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 4415b1c61e6..a6706231698 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -410,7 +410,15 @@ impl Scheduler { } fn next_remote_task(&self) -> Option> { - self.remote_queue.lock().unwrap().pop_front() + let mut lock = match self.remote_queue.lock() { + // If the lock is poisoned, but the thread is already panicking, + // avoid a double panic. This is necessary since `next_task` (which + // calls `next_remote_task`) can be called in the `Drop` impl. + Err(_) if std::thread::panicking() => return None, + Err(_) => panic!("mutex poisoned"), + Ok(lock) => lock, + }; + lock.pop_front() } fn tick(&self) { From 36efbb43a8ba1281c0cb536ed83b75c81e746fa3 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:15:11 -0800 Subject: [PATCH 51/73] fix warnings Signed-off-by: Eliza Weisman --- tokio/src/task/core.rs | 1 - tokio/src/task/local.rs | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/task/core.rs b/tokio/src/task/core.rs index 5805731519e..67b9bed6e78 100644 --- a/tokio/src/task/core.rs +++ b/tokio/src/task/core.rs @@ -7,7 +7,6 @@ use crate::task::Schedule; use std::cell::UnsafeCell; use std::future::Future; -use std::marker::PhantomData; use std::mem::MaybeUninit; use std::pin::Pin; use std::ptr::{self, NonNull}; diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index a6706231698..ae19588670b 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -556,7 +556,8 @@ mod tests { "blocking must not run on the local task set's thread" ); }) - .await; + .await + .unwrap(); assert!(ON_RT_THREAD.with(|cell| cell.get())); }); join.await.unwrap(); From 25d9bf4927002339b1bcf751163f2bf2604e17ce Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:24:40 -0800 Subject: [PATCH 52/73] rustfmt Signed-off-by: Eliza Weisman --- tokio/src/blocking/schedule.rs | 2 +- tokio/src/runtime/current_thread/mod.rs | 2 +- tokio/src/tests/loom_schedule.rs | 2 +- tokio/src/tests/mock_schedule.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tokio/src/blocking/schedule.rs b/tokio/src/blocking/schedule.rs index 01f9ddb1bfb..c522c484981 100644 --- a/tokio/src/blocking/schedule.rs +++ b/tokio/src/blocking/schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{ScheduleSend, Schedule, Task}; +use crate::task::{Schedule, ScheduleSend, Task}; /// `task::Schedule` implementation that does nothing. This is unique to the /// blocking scheduler as tasks scheduled are not really futures but blocking diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index 60cac3dffd8..6c4b4bb1bd0 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -1,5 +1,5 @@ use crate::runtime::park::{Park, Unpark}; -use crate::task::{self, JoinHandle, ScheduleSend, Schedule, Task}; +use crate::task::{self, JoinHandle, Schedule, ScheduleSend, Task}; use std::cell::UnsafeCell; use std::collections::VecDeque; diff --git a/tokio/src/tests/loom_schedule.rs b/tokio/src/tests/loom_schedule.rs index 5975312c4d8..a6910f728be 100644 --- a/tokio/src/tests/loom_schedule.rs +++ b/tokio/src/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{ScheduleSend, Schedule, Task}; +use crate::task::{Schedule, ScheduleSend, Task}; use loom::sync::Notify; use std::collections::VecDeque; diff --git a/tokio/src/tests/mock_schedule.rs b/tokio/src/tests/mock_schedule.rs index 6b0950f4e8e..8e752e48dab 100644 --- a/tokio/src/tests/mock_schedule.rs +++ b/tokio/src/tests/mock_schedule.rs @@ -1,5 +1,5 @@ #![allow(warnings)] -use crate::task::{Header, ScheduleSend, Schedule, Task}; +use crate::task::{Header, Schedule, ScheduleSend, Task}; use std::collections::VecDeque; use std::sync::Mutex; From 26994c967487776690fcef458e96edbb07e4e81b Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:27:04 -0800 Subject: [PATCH 53/73] placate clippy Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index ae19588670b..53d11533887 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -415,6 +415,10 @@ impl Scheduler { // avoid a double panic. This is necessary since `next_task` (which // calls `next_remote_task`) can be called in the `Drop` impl. Err(_) if std::thread::panicking() => return None, + // there is no semantic information in the `PoisonError`, and it + // doesn't implement `Debug`, but clippy thinks that it's bad to + // match all errors here... + #[allow(clippy::match-wild-err-arm)] Err(_) => panic!("mutex poisoned"), Ok(lock) => lock, }; From 20ccb39e67aa0e057aa8f7a586b85d4a1e5e5752 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:30:32 -0800 Subject: [PATCH 54/73] maybe they were supposed to be underscores??? i just copied the clippy error message... Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 53d11533887..98f49954fab 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -418,7 +418,7 @@ impl Scheduler { // there is no semantic information in the `PoisonError`, and it // doesn't implement `Debug`, but clippy thinks that it's bad to // match all errors here... - #[allow(clippy::match-wild-err-arm)] + #[allow(clippy::match_wild_err_arm)] Err(_) => panic!("mutex poisoned"), Ok(lock) => lock, }; From f28ded767be6157b3470d2147394583fea33388a Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 14:34:05 -0800 Subject: [PATCH 55/73] undo unrelated naming change Signed-off-by: Eliza Weisman --- tokio/src/task/list.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tokio/src/task/list.rs b/tokio/src/task/list.rs index 91bcc725a68..d5dcc9c2e0a 100644 --- a/tokio/src/task/list.rs +++ b/tokio/src/task/list.rs @@ -4,12 +4,12 @@ use std::fmt; use std::marker::PhantomData; use std::ptr::NonNull; -pub(crate) struct OwnedList { +pub(crate) struct OwnedList { head: Option>, - _p: PhantomData, + _p: PhantomData, } -impl OwnedList { +impl OwnedList { pub(crate) fn new() -> Self { OwnedList { head: None, @@ -17,7 +17,7 @@ impl OwnedList { } } - pub(crate) fn insert(&mut self, task: &Task) { + pub(crate) fn insert(&mut self, task: &Task) { debug_assert!(!self.contains(task)); unsafe { @@ -36,7 +36,7 @@ impl OwnedList { } } - pub(crate) fn remove(&mut self, task: &Task) { + pub(crate) fn remove(&mut self, task: &Task) { debug_assert!(self.head.is_some()); unsafe { @@ -72,7 +72,7 @@ impl OwnedList { } /// Only used by debug assertions - fn contains(&self, task: &Task) -> bool { + fn contains(&self, task: &Task) -> bool { let mut curr = self.head; while let Some(p) = curr { @@ -89,7 +89,7 @@ impl OwnedList { } } -impl fmt::Debug for OwnedList { +impl fmt::Debug for OwnedList { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("OwnedList").finish() } From ef29af8880c56eac0dd865d690f43c0933928abd Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 13 Nov 2019 16:23:03 -0800 Subject: [PATCH 56/73] review feedback Signed-off-by: Eliza Weisman --- tokio/src/task/list.rs | 2 +- tokio/src/task/mod.rs | 6 +++--- tokio/src/task/raw.rs | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tokio/src/task/list.rs b/tokio/src/task/list.rs index d5dcc9c2e0a..85ff3dc2287 100644 --- a/tokio/src/task/list.rs +++ b/tokio/src/task/list.rs @@ -10,7 +10,7 @@ pub(crate) struct OwnedList { } impl OwnedList { - pub(crate) fn new() -> Self { + pub(crate) fn new() -> OwnedList { OwnedList { head: None, _p: PhantomData, diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 264a9383a52..ad8653765ad 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -52,7 +52,7 @@ pub(crate) struct Task { _p: PhantomData, } -unsafe impl Send for Task {} +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; @@ -85,7 +85,7 @@ pub(crate) trait ScheduleSend: Schedule + Send + Sync {} pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: Schedule + ScheduleSend, + S: ScheduleSend, { Task { raw: RawTask::new_background::<_, S>(task), @@ -97,7 +97,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: Schedule + ScheduleSend, + S: ScheduleSend, { let raw = RawTask::new_joinable::<_, S>(task); diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 79973f73975..e994300f22f 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -58,7 +58,7 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule + ScheduleSend, + S: ScheduleSend, { RawTask::new::<_, S>(task, State::new_background()) } @@ -66,7 +66,7 @@ impl RawTask { pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: Schedule + ScheduleSend, + S: ScheduleSend, { RawTask::new::<_, S>(task, State::new_joinable()) } From 7a9c84e9c57d393c576569af6ab7225ae0e03073 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 14 Nov 2019 10:12:10 -0800 Subject: [PATCH 57/73] put clippy attribute in the right place Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 98f49954fab..6f6af2da069 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -410,15 +410,15 @@ impl Scheduler { } fn next_remote_task(&self) -> Option> { + // there is no semantic information in the `PoisonError`, and it + // doesn't implement `Debug`, but clippy thinks that it's bad to + // match all errors here... + #[allow(clippy::match_wild_err_arm)] let mut lock = match self.remote_queue.lock() { // If the lock is poisoned, but the thread is already panicking, // avoid a double panic. This is necessary since `next_task` (which // calls `next_remote_task`) can be called in the `Drop` impl. Err(_) if std::thread::panicking() => return None, - // there is no semantic information in the `PoisonError`, and it - // doesn't implement `Debug`, but clippy thinks that it's bad to - // match all errors here... - #[allow(clippy::match_wild_err_arm)] Err(_) => panic!("mutex poisoned"), Ok(lock) => lock, }; From f1211e35c4142a5338f558d4a1e2067c00428b12 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 14 Nov 2019 10:13:14 -0800 Subject: [PATCH 58/73] fix unused code based on feature flags Signed-off-by: Eliza Weisman --- tokio/src/task/mod.rs | 1 + tokio/src/task/raw.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index ad8653765ad..5447a55c2d7 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -112,6 +112,7 @@ where } /// Create a new `!Send` task with an associated join handle +#[cfg(feature = "rt-full")] pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) where T: Future + 'static, diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index e994300f22f..917cc4287fe 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -71,6 +71,7 @@ impl RawTask { RawTask::new::<_, S>(task, State::new_joinable()) } + #[cfg(feature = "rt-full")] pub(super) fn new_joinable_local(task: T) -> RawTask where T: Future + 'static, From 6ff8c4fedfe6f5677305c994d574ec35fe4ec8b5 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 14 Nov 2019 10:17:26 -0800 Subject: [PATCH 59/73] review feedback: rename `ScheduleSend` -> `ScheduleSendOnly` thanks @jonhoo! Signed-off-by: Eliza Weisman --- tokio/src/blocking/schedule.rs | 4 ++-- tokio/src/runtime/current_thread/mod.rs | 4 ++-- tokio/src/runtime/thread_pool/shared.rs | 4 ++-- tokio/src/task/mod.rs | 8 ++++---- tokio/src/task/raw.rs | 6 +++--- tokio/src/tests/loom_schedule.rs | 4 ++-- tokio/src/tests/mock_schedule.rs | 6 +++--- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tokio/src/blocking/schedule.rs b/tokio/src/blocking/schedule.rs index c522c484981..5d2cd5f53c8 100644 --- a/tokio/src/blocking/schedule.rs +++ b/tokio/src/blocking/schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{Schedule, ScheduleSend, Task}; +use crate::task::{Schedule, ScheduleSendOnly, Task}; /// `task::Schedule` implementation that does nothing. This is unique to the /// blocking scheduler as tasks scheduled are not really futures but blocking @@ -17,4 +17,4 @@ impl Schedule for NoopSchedule { } } -impl ScheduleSend for NoopSchedule {} +impl ScheduleSendOnly for NoopSchedule {} diff --git a/tokio/src/runtime/current_thread/mod.rs b/tokio/src/runtime/current_thread/mod.rs index 6c4b4bb1bd0..c926922037a 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/current_thread/mod.rs @@ -1,5 +1,5 @@ use crate::runtime::park::{Park, Unpark}; -use crate::task::{self, JoinHandle, Schedule, ScheduleSend, Task}; +use crate::task::{self, JoinHandle, Schedule, ScheduleSendOnly, Task}; use std::cell::UnsafeCell; use std::collections::VecDeque; @@ -293,7 +293,7 @@ impl Schedule for Scheduler { } } -impl ScheduleSend for Scheduler {} +impl ScheduleSendOnly for Scheduler {} impl

Drop for CurrentThread

where diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index f653fb10033..779d2673531 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,6 +1,6 @@ use crate::runtime::park::Unpark; use crate::runtime::thread_pool::slice; -use crate::task::{self, Schedule, ScheduleSend, Task}; +use crate::task::{self, Schedule, ScheduleSendOnly, Task}; use std::ptr; @@ -99,4 +99,4 @@ where } } -impl ScheduleSend for Shared

{} +impl ScheduleSendOnly for Shared

{} diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 5447a55c2d7..6110c875374 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -52,7 +52,7 @@ pub(crate) struct Task { _p: PhantomData, } -unsafe impl Send for Task {} +unsafe impl Send for Task {} /// Task result sent back pub(crate) type Result = std::result::Result; @@ -79,13 +79,13 @@ pub(crate) trait Schedule: Sized + 'static { /// /// Schedulers that implement this trait may not schedule `!Send` futures. If /// trait is implemented, the corresponding `Task` type will implement `Send`. -pub(crate) trait ScheduleSend: Schedule + Send + Sync {} +pub(crate) trait ScheduleSendOnly: Schedule + Send + Sync {} /// Create a new task without an associated join handle pub(crate) fn background(task: T) -> Task where T: Future + Send + 'static, - S: ScheduleSend, + S: ScheduleSendOnly, { Task { raw: RawTask::new_background::<_, S>(task), @@ -97,7 +97,7 @@ where pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where T: Future + Send + 'static, - S: ScheduleSend, + S: ScheduleSendOnly, { let raw = RawTask::new_joinable::<_, S>(task); diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 917cc4287fe..fe8299efc7d 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -1,7 +1,7 @@ use crate::loom::alloc::Track; use crate::task::Cell; use crate::task::Harness; -use crate::task::{Header, Schedule, ScheduleSend}; +use crate::task::{Header, Schedule, ScheduleSendOnly}; use crate::task::{Snapshot, State}; use std::future::Future; @@ -58,7 +58,7 @@ impl RawTask { pub(super) fn new_background(task: T) -> RawTask where T: Future + Send + 'static, - S: ScheduleSend, + S: ScheduleSendOnly, { RawTask::new::<_, S>(task, State::new_background()) } @@ -66,7 +66,7 @@ impl RawTask { pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, - S: ScheduleSend, + S: ScheduleSendOnly, { RawTask::new::<_, S>(task, State::new_joinable()) } diff --git a/tokio/src/tests/loom_schedule.rs b/tokio/src/tests/loom_schedule.rs index a6910f728be..15ec903b5de 100644 --- a/tokio/src/tests/loom_schedule.rs +++ b/tokio/src/tests/loom_schedule.rs @@ -1,4 +1,4 @@ -use crate::task::{Schedule, ScheduleSend, Task}; +use crate::task::{Schedule, ScheduleSendOnly, Task}; use loom::sync::Notify; use std::collections::VecDeque; @@ -50,4 +50,4 @@ impl Schedule for LoomSchedule { } } -impl ScheduleSend for LoomSchedule {} +impl ScheduleSendOnly for LoomSchedule {} diff --git a/tokio/src/tests/mock_schedule.rs b/tokio/src/tests/mock_schedule.rs index 8e752e48dab..a9941564600 100644 --- a/tokio/src/tests/mock_schedule.rs +++ b/tokio/src/tests/mock_schedule.rs @@ -1,5 +1,5 @@ #![allow(warnings)] -use crate::task::{Header, Schedule, ScheduleSend, Task}; +use crate::task::{Header, Schedule, ScheduleSendOnly, Task}; use std::collections::VecDeque; use std::sync::Mutex; @@ -111,7 +111,7 @@ impl Schedule for Mock { } } -impl ScheduleSend for Mock {} +impl ScheduleSendOnly for Mock {} impl Drop for Mock { fn drop(&mut self) { @@ -131,4 +131,4 @@ impl Schedule for Noop { fn schedule(&self, _task: Task) {} } -impl ScheduleSend for Noop {} +impl ScheduleSendOnly for Noop {} From b21cca79b0ce993b69195d69f8c11c150282b937 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 10:11:53 -0800 Subject: [PATCH 60/73] Merge master + update some things Signed-off-by: Eliza Weisman --- CONTRIBUTING.md | 2 +- azure-pipelines.yml | 1 + examples/chat.rs | 16 +- examples/connect.rs | 10 +- examples/print_each_packet.rs | 2 +- examples/proxy.rs | 6 +- examples/udp-codec.rs | 5 +- tests-integration/tests/process_stdio.rs | 5 +- tokio-macros/Cargo.toml | 2 +- tokio-macros/src/lib.rs | 53 +-- tokio-test/src/io.rs | 8 +- tokio-test/src/lib.rs | 2 +- tokio-test/tests/block_on.rs | 4 +- tokio-tls/tests/smoke.rs | 6 +- tokio-util/Cargo.toml | 3 +- tokio-util/src/codec/framed.rs | 128 +++++--- tokio-util/src/codec/framed_read.rs | 129 +++++--- tokio-util/src/codec/framed_write.rs | 108 ++++--- tokio-util/src/codec/length_delimited.rs | 1 + tokio-util/src/codec/macros.rs | 7 - tokio-util/src/codec/mod.rs | 3 - tokio-util/tests/framed.rs | 1 + tokio-util/tests/framed_read.rs | 3 +- tokio-util/tests/length_delimited.rs | 3 +- tokio-util/tests/udp.rs | 8 +- tokio/Cargo.toml | 41 +-- tokio/src/blocking/mod.rs | 68 +--- tokio/src/blocking/pool.rs | 22 +- tokio/src/fs/file.rs | 5 +- tokio/src/fs/mod.rs | 2 - tokio/src/fs/read_dir.rs | 50 +-- 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 | 82 +++++ tokio/src/io/async_read.rs | 1 - tokio/src/io/async_write.rs | 1 - tokio/src/{fs => io}/blocking.rs | 31 +- tokio/src/io/io/lines.rs | 70 ---- tokio/src/io/io/mod.rs | 51 --- tokio/src/io/io/split.rs | 67 ---- tokio/src/io/mod.rs | 67 ++-- tokio/src/io/split.rs | 1 - tokio/src/io/stderr.rs | 2 +- tokio/src/io/stdin.rs | 2 +- tokio/src/io/stdout.rs | 2 +- .../src/io/{io => util}/async_buf_read_ext.rs | 28 +- 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 | 71 ++-- tokio/src/io/{io => util}/buf_stream.rs | 76 +++-- tokio/src/io/{io => util}/buf_writer.rs | 84 +++-- tokio/src/io/{io => util}/chain.rs | 38 +-- 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 | 113 +++++++ tokio/src/io/util/mod.rs | 75 +++++ 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 | 111 +++++++ tokio/src/io/{io => util}/take.rs | 32 +- tokio/src/io/{io => util}/write.rs | 0 tokio/src/io/{io => util}/write_all.rs | 1 - tokio/src/lib.rs | 80 +++-- tokio/src/loom/std/mod.rs | 8 +- tokio/src/macros/assert.rs | 19 ++ tokio/src/macros/cfg.rs | 217 +++++++++++++ tokio/src/macros/loom.rs | 12 + tokio/src/macros/mod.rs | 17 + tokio/src/macros/ready.rs | 8 + tokio/src/macros/thread_local.rs | 4 + tokio/src/net/addr.rs | 302 ++++++++++-------- tokio/src/net/mod.rs | 40 ++- tokio/src/net/tcp/incoming.rs | 25 +- tokio/src/net/tcp/listener.rs | 11 +- tokio/src/net/tcp/mod.rs | 8 +- 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 | 26 +- tokio/src/net/unix/listener.rs | 9 +- tokio/src/net/unix/stream.rs | 3 +- tokio/src/net/util/poll_evented.rs | 3 +- tokio/src/prelude.rs | 23 +- tokio/src/process/mod.rs | 36 +-- tokio/src/process/unix/mod.rs | 30 +- tokio/src/process/unix/reap.rs | 38 ++- tokio/src/process/windows.rs | 8 +- .../mod.rs => basic_scheduler.rs} | 77 +++-- tokio/src/runtime/blocking.rs | 8 +- tokio/src/runtime/builder.rs | 246 +++++++------- tokio/src/runtime/enter.rs | 96 +++--- tokio/src/runtime/global.rs | 101 ++---- tokio/src/runtime/handle.rs | 124 +++---- tokio/src/runtime/io.rs | 8 +- tokio/src/runtime/mod.rs | 62 ++-- tokio/src/runtime/park/mod.rs | 2 +- tokio/src/runtime/park/thread.rs | 4 +- tokio/src/runtime/thread_pool/mod.rs | 6 +- tokio/src/runtime/thread_pool/slice.rs | 9 - tokio/src/runtime/thread_pool/spawner.rs | 8 +- .../runtime/thread_pool/tests/loom_pool.rs | 24 +- tokio/src/runtime/thread_pool/tests/pool.rs | 2 +- tokio/src/runtime/thread_pool/tests/queue.rs | 2 +- tokio/src/runtime/thread_pool/worker.rs | 38 +-- tokio/src/runtime/time.rs | 8 +- tokio/src/signal/ctrl_c.rs | 61 ++-- tokio/src/signal/mod.rs | 56 +--- tokio/src/signal/registry.rs | 33 +- tokio/src/signal/unix.rs | 32 +- tokio/src/signal/windows.rs | 65 ++-- tokio/src/stream.rs | 78 ----- tokio/src/sync/barrier.rs | 3 +- tokio/src/sync/mod.rs | 55 ++-- tokio/src/sync/mpsc/bounded.rs | 149 ++------- tokio/src/sync/mpsc/chan.rs | 104 ++++-- tokio/src/sync/mpsc/error.rs | 86 +++++ tokio/src/sync/mpsc/list.rs | 7 - tokio/src/sync/mpsc/mod.rs | 9 +- tokio/src/sync/mpsc/unbounded.rs | 102 +----- tokio/src/sync/mutex.rs | 2 +- tokio/src/sync/oneshot.rs | 5 +- tokio/src/sync/semaphore.rs | 108 ++----- tokio/src/sync/task/atomic_waker.rs | 21 +- tokio/src/sync/task/mod.rs | 2 +- .../sync/tests/atomic_waker.rs} | 4 +- tokio/src/sync/tests/loom_atomic_waker.rs | 2 +- tokio/src/sync/tests/loom_list.rs | 4 - 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/tests/mod.rs | 17 +- .../sync/tests/semaphore.rs} | 4 +- tokio/src/sync/watch.rs | 121 ++----- tokio/src/task/blocking.rs | 65 ++++ tokio/src/task/local.rs | 35 +- tokio/src/task/mod.rs | 49 ++- tokio/src/task/raw.rs | 20 +- tokio/src/task/spawn.rs | 53 +++ tokio/src/task/state.rs | 1 + tokio/src/task/tests/loom.rs | 6 +- tokio/src/task/tests/task.rs | 2 +- tokio/src/tests/mod.rs | 22 -- tokio/src/time/clock.rs | 11 +- 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 | 150 +++++---- 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 | 51 ++- tokio/tests/fs_file_mocked.rs | 33 +- tokio/tests/io_lines.rs | 18 +- tokio/tests/net_driver.rs | 2 +- tokio/tests/process_issue_42.rs | 6 +- .../{rt_current_thread.rs => rt_basic.rs} | 2 +- tokio/tests/rt_common.rs | 47 ++- .../{rt_thread_pool.rs => rt_threaded.rs} | 11 +- tokio/tests/signal_ctrl_c.rs | 7 +- tokio/tests/signal_drop_recv.rs | 5 +- tokio/tests/signal_drop_rt.rs | 7 +- tokio/tests/signal_drop_signal.rs | 5 +- tokio/tests/signal_multi_rt.rs | 7 +- tokio/tests/signal_notify_both.rs | 11 +- tokio/tests/signal_twice.rs | 6 +- tokio/tests/signal_usr1.rs | 5 +- tokio/tests/sync_errors.rs | 5 +- tokio/tests/sync_mpsc.rs | 113 +++---- tokio/tests/sync_watch.rs | 157 ++++----- tokio/tests/time_interval.rs | 46 ++- tokio/tests/time_rt.rs | 15 +- tokio/tests/time_throttle.rs | 68 ---- tokio/tests/time_timeout.rs | 71 +--- 188 files changed, 3392 insertions(+), 3016 deletions(-) delete mode 100644 tokio-util/src/codec/macros.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 rename tokio/src/{fs => io}/blocking.rs (93%) delete mode 100644 tokio/src/io/io/lines.rs delete mode 100644 tokio/src/io/io/mod.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 (76%) rename tokio/src/io/{io => util}/buf_stream.rs (62%) rename tokio/src/io/{io => util}/buf_writer.rs (72%) rename tokio/src/io/{io => util}/chain.rs (83%) 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 create mode 100644 tokio/src/io/util/mod.rs 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 (84%) rename tokio/src/io/{io => util}/write.rs (100%) rename tokio/src/io/{io => util}/write_all.rs (98%) create mode 100644 tokio/src/macros/assert.rs create mode 100644 tokio/src/macros/cfg.rs create mode 100644 tokio/src/macros/loom.rs create mode 100644 tokio/src/macros/mod.rs create mode 100644 tokio/src/macros/ready.rs create mode 100644 tokio/src/macros/thread_local.rs rename tokio/src/runtime/{current_thread/mod.rs => basic_scheduler.rs} (83%) delete mode 100644 tokio/src/stream.rs create mode 100644 tokio/src/sync/mpsc/error.rs rename tokio/{tests/sync_atomic_waker.rs => src/sync/tests/atomic_waker.rs} (92%) rename tokio/{tests/sync_semaphore.rs => src/sync/tests/semaphore.rs} (97%) create mode 100644 tokio/src/task/blocking.rs create mode 100644 tokio/src/task/spawn.rs rename tokio/tests/{rt_current_thread.rs => rt_basic.rs} (95%) rename tokio/tests/{rt_thread_pool.rs => rt_threaded.rs} (97%) delete mode 100644 tokio/tests/time_throttle.rs diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 8bd28043c1c..a44be9f6fba 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -12,7 +12,7 @@ use your help. This guide will help you get started. **Do not let this guide intimidate you**. It should be considered a map to help you navigate the process. -You may also get help with contributing in the [dev channel][dev], please join +The [dev channel][dev] is available for any concerns not covered in this guide, please join us! [dev]: https://gitter.im/tokio-rs/dev diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 60d8cf7a65b..ebd27ecab1e 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -37,6 +37,7 @@ jobs: crates: - tokio-macros - tokio-test + - tokio-tls - tokio-util - examples 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/tests-integration/tests/process_stdio.rs b/tests-integration/tests/process_stdio.rs index 7e4f3d3e40b..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}; @@ -47,9 +46,9 @@ 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())) + .unwrap_or_else(|_| Some(String::new())) .expect("failed to read line"); let num_read = data.len(); diff --git a/tokio-macros/Cargo.toml b/tokio-macros/Cargo.toml index 21fe9117a0d..4bf87927ea3 100644 --- a/tokio-macros/Cargo.toml +++ b/tokio-macros/Cargo.toml @@ -29,7 +29,7 @@ quote = "1" syn = { version = "1.0.3", features = ["full"] } [dev-dependencies] -tokio = { version = "=0.2.0-alpha.6", path = "../tokio", default-features = false, features = ["rt-full"] } +tokio = { version = "=0.2.0-alpha.6", path = "../tokio" } [package.metadata.docs.rs] all-features = true diff --git a/tokio-macros/src/lib.rs b/tokio-macros/src/lib.rs index 1cfc14c30fe..62c8d94ba8a 100644 --- a/tokio-macros/src/lib.rs +++ b/tokio-macros/src/lib.rs @@ -18,9 +18,9 @@ extern crate proc_macro; use proc_macro::TokenStream; use quote::quote; -enum RuntimeType { - Single, - Multi, +enum Runtime { + Basic, + Threaded, Auto, } @@ -28,8 +28,8 @@ enum RuntimeType { /// /// ## Options: /// -/// - `current_thread` - Uses the `current_thread` runtime. -/// - `threadpool` - Uses the multi-threaded `threadpool` runtime. Used by default. +/// - `basic_scheduler` - All tasks are executed on the current thread. +/// - `threaded_scheduler` - Uses the multi-threaded scheduler. Used by default. /// /// ## Function arguments: /// @@ -37,18 +37,19 @@ enum RuntimeType { /// /// ## Usage /// -/// ### Select runtime +/// ### Using default /// /// ```rust -/// #[tokio::main(current_thread)] +/// #[tokio::main] /// async fn main() { /// println!("Hello world"); /// } /// ``` -/// ### Using default +/// +/// ### Select runtime /// /// ```rust -/// #[tokio::main] +/// #[tokio::main(basic_scheduler)] /// async fn main() { /// println!("Hello world"); /// } @@ -77,7 +78,7 @@ pub fn main(args: TokenStream, item: TokenStream) -> TokenStream { .into(); } - let mut runtime = RuntimeType::Auto; + let mut runtime = Runtime::Auto; for arg in args { if let syn::NestedMeta::Meta(syn::Meta::Path(path)) = arg { @@ -87,10 +88,10 @@ pub fn main(args: TokenStream, item: TokenStream) -> TokenStream { return syn::Error::new_spanned(path, msg).to_compile_error().into(); } match ident.unwrap().to_string().to_lowercase().as_str() { - "threadpool" => runtime = RuntimeType::Multi, - "current_thread" => runtime = RuntimeType::Single, + "threaded_scheduler" => runtime = Runtime::Threaded, + "basic_scheduler" => runtime = Runtime::Basic, name => { - let msg = format!("Unknown attribute {} is specified; expected `current_thread` or `threadpool`", name); + let msg = format!("Unknown attribute {} is specified; expected `basic_scheduler` or `threaded_scheduler`", name); return syn::Error::new_spanned(path, msg).to_compile_error().into(); } } @@ -98,17 +99,17 @@ pub fn main(args: TokenStream, item: TokenStream) -> TokenStream { } let result = match runtime { - RuntimeType::Multi | RuntimeType::Auto => quote! { + Runtime::Threaded | Runtime::Auto => quote! { #(#attrs)* fn #name(#inputs) #ret { tokio::runtime::Runtime::new().unwrap().block_on(async { #body }) } }, - RuntimeType::Single => quote! { + Runtime::Basic => quote! { #(#attrs)* fn #name(#inputs) #ret { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() .block_on(async { #body }) @@ -123,15 +124,15 @@ pub fn main(args: TokenStream, item: TokenStream) -> TokenStream { /// /// ## Options: /// -/// - `current_thread` - Uses the `current_thread` runtime. Used by default. -/// - `threadpool` - Uses multi-threaded runtime. +/// - `basic_scheduler` - All tasks are executed on the current thread. Used by default. +/// - `threaded_scheduler` - Use multi-threaded scheduler. /// /// ## Usage /// /// ### Select runtime /// /// ```no_run -/// #[tokio::test(threadpool)] +/// #[tokio::test(threaded_scheduler)] /// async fn my_test() { /// assert!(true); /// } @@ -176,7 +177,7 @@ pub fn test(args: TokenStream, item: TokenStream) -> TokenStream { .into(); } - let mut runtime = RuntimeType::Auto; + let mut runtime = Runtime::Auto; for arg in args { if let syn::NestedMeta::Meta(syn::Meta::Path(path)) = arg { @@ -186,10 +187,10 @@ pub fn test(args: TokenStream, item: TokenStream) -> TokenStream { return syn::Error::new_spanned(path, msg).to_compile_error().into(); } match ident.unwrap().to_string().to_lowercase().as_str() { - "threadpool" => runtime = RuntimeType::Multi, - "current_thread" => runtime = RuntimeType::Single, + "threaded_scheduler" => runtime = Runtime::Threaded, + "basic_scheduler" => runtime = Runtime::Basic, name => { - let msg = format!("Unknown attribute {} is specified; expected `current_thread` or `threadpool`", name); + let msg = format!("Unknown attribute {} is specified; expected `basic_scheduler` or `threaded_scheduler`", name); return syn::Error::new_spanned(path, msg).to_compile_error().into(); } } @@ -197,19 +198,19 @@ pub fn test(args: TokenStream, item: TokenStream) -> TokenStream { } let result = match runtime { - RuntimeType::Multi => quote! { + Runtime::Threaded => quote! { #[test] #(#attrs)* fn #name() #ret { tokio::runtime::Runtime::new().unwrap().block_on(async { #body }) } }, - RuntimeType::Single | RuntimeType::Auto => quote! { + Runtime::Basic | Runtime::Auto => quote! { #[test] #(#attrs)* fn #name() #ret { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() .block_on(async { #body }) diff --git a/tokio-test/src/io.rs b/tokio-test/src/io.rs index 1d42dd03f42..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 } } @@ -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/src/lib.rs b/tokio-test/src/lib.rs index e6e9019ef1b..bdd4a9f97c4 100644 --- a/tokio-test/src/lib.rs +++ b/tokio-test/src/lib.rs @@ -27,7 +27,7 @@ pub mod task; pub fn block_on(future: F) -> F::Output { use tokio::runtime; - let mut rt = runtime::Builder::new().current_thread().build().unwrap(); + let mut rt = runtime::Builder::new().basic_scheduler().build().unwrap(); rt.block_on(future) } 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-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/Cargo.toml b/tokio-util/Cargo.toml index 034dcd02ad8..c857c839360 100644 --- a/tokio-util/Cargo.toml +++ b/tokio-util/Cargo.toml @@ -26,12 +26,13 @@ bytes = "0.4.7" futures-core = "0.3.0" futures-sink = "0.3.0" log = "0.4" +pin-project-lite = "0.1.1" [dev-dependencies] 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/codec/framed.rs b/tokio-util/src/codec/framed.rs index c95d643b013..62403d56db0 100644 --- a/tokio-util/src/codec/framed.rs +++ b/tokio-util/src/codec/framed.rs @@ -8,20 +8,52 @@ use tokio::io::{AsyncBufRead, AsyncRead, AsyncWrite}; use bytes::BytesMut; use futures_core::Stream; use futures_sink::Sink; +use pin_project_lite::pin_project; use std::fmt; use std::io::{self, BufRead, Read, Write}; use std::pin::Pin; use std::task::{Context, Poll}; -/// A unified `Stream` and `Sink` interface to an underlying I/O object, using -/// the `Encoder` and `Decoder` traits to encode and decode frames. -/// -/// You can create a `Framed` instance by using the `AsyncRead::framed` adapter. -pub struct Framed { - inner: FramedRead2>>, +pin_project! { + /// A unified `Stream` and `Sink` interface to an underlying I/O object, using + /// the `Encoder` and `Decoder` traits to encode and decode frames. + /// + /// You can create a `Framed` instance by using the `AsyncRead::framed` adapter. + pub struct Framed { + #[pin] + inner: FramedRead2>>, + } +} + +pin_project! { + pub(crate) struct Fuse { + #[pin] + pub(crate) io: T, + pub(crate) codec: U, + } +} + +/// Abstracts over `FramedRead2` being either `FramedRead2>>` or +/// `FramedRead2>` and lets the io and codec parts be extracted in either case. +pub(crate) trait ProjectFuse { + type Io; + type Codec; + + fn project(self: Pin<&mut Self>) -> Fuse, &mut Self::Codec>; } -pub(crate) struct Fuse(pub(crate) T, pub(crate) U); +impl ProjectFuse for Fuse { + type Io = T; + type Codec = U; + + fn project(self: Pin<&mut Self>) -> Fuse, &mut Self::Codec> { + let self_ = self.project(); + Fuse { + io: self_.io, + codec: self_.codec, + } + } +} impl Framed where @@ -47,7 +79,7 @@ where /// break them into separate objects, allowing them to interact more easily. pub fn new(inner: T, codec: U) -> Framed { Framed { - inner: framed_read2(framed_write2(Fuse(inner, codec))), + inner: framed_read2(framed_write2(Fuse { io: inner, codec })), } } } @@ -76,7 +108,13 @@ impl Framed { pub fn from_parts(parts: FramedParts) -> Framed { Framed { inner: framed_read2_with_buffer( - framed_write2_with_buffer(Fuse(parts.io, parts.codec), parts.write_buf), + framed_write2_with_buffer( + Fuse { + io: parts.io, + codec: parts.codec, + }, + parts.write_buf, + ), parts.read_buf, ), } @@ -89,7 +127,7 @@ impl Framed { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_ref(&self) -> &T { - &self.inner.get_ref().get_ref().0 + &self.inner.get_ref().get_ref().io } /// Returns a mutable reference to the underlying I/O stream wrapped by @@ -99,7 +137,7 @@ impl Framed { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_mut(&mut self) -> &mut T { - &mut self.inner.get_mut().get_mut().0 + &mut self.inner.get_mut().get_mut().io } /// Returns a reference to the underlying codec wrapped by @@ -108,7 +146,7 @@ impl Framed { /// Note that care should be taken to not tamper with the underlying codec /// as it may corrupt the stream of frames otherwise being worked with. pub fn codec(&self) -> &U { - &self.inner.get_ref().get_ref().1 + &self.inner.get_ref().get_ref().codec } /// Returns a mutable reference to the underlying codec wrapped by @@ -117,7 +155,7 @@ impl Framed { /// Note that care should be taken to not tamper with the underlying codec /// as it may corrupt the stream of frames otherwise being worked with. pub fn codec_mut(&mut self) -> &mut U { - &mut self.inner.get_mut().get_mut().1 + &mut self.inner.get_mut().get_mut().codec } /// Returns a reference to the read buffer. @@ -131,7 +169,7 @@ impl Framed { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn into_inner(self) -> T { - self.inner.into_inner().into_inner().0 + self.inner.into_inner().into_inner().io } /// Consumes the `Frame`, returning its underlying I/O stream, the buffer @@ -145,8 +183,8 @@ impl Framed { let (inner, write_buf) = inner.into_parts(); FramedParts { - io: inner.0, - codec: inner.1, + io: inner.io, + codec: inner.codec, read_buf, write_buf, _priv: (), @@ -156,38 +194,38 @@ impl Framed { impl Stream for Framed where - T: AsyncRead + Unpin, - U: Decoder + Unpin, + T: AsyncRead, + U: Decoder, { type Item = Result; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().inner).poll_next(cx) + self.project().inner.poll_next(cx) } } impl Sink for Framed where - T: AsyncWrite + Unpin, - U: Encoder + Unpin, + T: AsyncWrite, + U: Encoder, U::Error: From, { type Error = U::Error; fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(Pin::get_mut(self).inner.get_mut()).poll_ready(cx) + self.project().inner.get_pin_mut().poll_ready(cx) } fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { - Pin::new(Pin::get_mut(self).inner.get_mut()).start_send(item) + self.project().inner.get_pin_mut().start_send(item) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(Pin::get_mut(self).inner.get_mut()).poll_flush(cx) + self.project().inner.get_pin_mut().poll_flush(cx) } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(Pin::get_mut(self).inner.get_mut()).poll_close(cx) + self.project().inner.get_pin_mut().poll_close(cx) } } @@ -198,8 +236,8 @@ where { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Framed") - .field("io", &self.inner.get_ref().get_ref().0) - .field("codec", &self.inner.get_ref().get_ref().1) + .field("io", &self.inner.get_ref().get_ref().io) + .field("codec", &self.inner.get_ref().get_ref().codec) .finish() } } @@ -208,23 +246,23 @@ where impl Read for Fuse { fn read(&mut self, dst: &mut [u8]) -> io::Result { - self.0.read(dst) + self.io.read(dst) } } impl BufRead for Fuse { fn fill_buf(&mut self) -> io::Result<&[u8]> { - self.0.fill_buf() + self.io.fill_buf() } fn consume(&mut self, amt: usize) { - self.0.consume(amt) + self.io.consume(amt) } } -impl AsyncRead for Fuse { +impl AsyncRead for Fuse { unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.0.prepare_uninitialized_buffer(buf) + self.io.prepare_uninitialized_buffer(buf) } fn poll_read( @@ -232,45 +270,45 @@ impl AsyncRead for Fuse { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - pin!(self.get_mut().0).poll_read(cx, buf) + self.project().io.poll_read(cx, buf) } } -impl AsyncBufRead for Fuse { +impl AsyncBufRead for Fuse { fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().0).poll_fill_buf(cx) + self.project().io.poll_fill_buf(cx) } fn consume(self: Pin<&mut Self>, amt: usize) { - pin!(self.get_mut().0).consume(amt) + self.project().io.consume(amt) } } impl Write for Fuse { fn write(&mut self, src: &[u8]) -> io::Result { - self.0.write(src) + self.io.write(src) } fn flush(&mut self) -> io::Result<()> { - self.0.flush() + self.io.flush() } } -impl AsyncWrite for Fuse { +impl AsyncWrite for Fuse { fn poll_write( self: Pin<&mut Self>, cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - pin!(self.get_mut().0).poll_write(cx, buf) + self.project().io.poll_write(cx, buf) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().0).poll_flush(cx) + self.project().io.poll_flush(cx) } fn poll_shutdown(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().0).poll_shutdown(cx) + self.project().io.poll_shutdown(cx) } } @@ -279,11 +317,11 @@ impl Decoder for Fuse { type Error = U::Error; fn decode(&mut self, buffer: &mut BytesMut) -> Result, Self::Error> { - self.1.decode(buffer) + self.codec.decode(buffer) } fn decode_eof(&mut self, buffer: &mut BytesMut) -> Result, Self::Error> { - self.1.decode_eof(buffer) + self.codec.decode_eof(buffer) } } @@ -292,7 +330,7 @@ impl Encoder for Fuse { type Error = U::Error; fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { - self.1.encode(item, dst) + self.codec.encode(item, dst) } } diff --git a/tokio-util/src/codec/framed_read.rs b/tokio-util/src/codec/framed_read.rs index e10f2968c6e..bd1f625b0c2 100644 --- a/tokio-util/src/codec/framed_read.rs +++ b/tokio-util/src/codec/framed_read.rs @@ -1,4 +1,4 @@ -use crate::codec::framed::Fuse; +use crate::codec::framed::{Fuse, ProjectFuse}; use crate::codec::Decoder; use tokio::io::AsyncRead; @@ -7,20 +7,27 @@ use bytes::BytesMut; use futures_core::Stream; use futures_sink::Sink; use log::trace; +use pin_project_lite::pin_project; use std::fmt; use std::pin::Pin; use std::task::{Context, Poll}; -/// A `Stream` of messages decoded from an `AsyncRead`. -pub struct FramedRead { - inner: FramedRead2>, +pin_project! { + /// A `Stream` of messages decoded from an `AsyncRead`. + pub struct FramedRead { + #[pin] + inner: FramedRead2>, + } } -pub(crate) struct FramedRead2 { - inner: T, - eof: bool, - is_readable: bool, - buffer: BytesMut, +pin_project! { + pub(crate) struct FramedRead2 { + #[pin] + inner: T, + eof: bool, + is_readable: bool, + buffer: BytesMut, + } } const INITIAL_CAPACITY: usize = 8 * 1024; @@ -35,7 +42,10 @@ where /// Creates a new `FramedRead` with the given `decoder`. pub fn new(inner: T, decoder: D) -> FramedRead { FramedRead { - inner: framed_read2(Fuse(inner, decoder)), + inner: framed_read2(Fuse { + io: inner, + codec: decoder, + }), } } } @@ -48,7 +58,7 @@ impl FramedRead { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_ref(&self) -> &T { - &self.inner.inner.0 + &self.inner.inner.io } /// Returns a mutable reference to the underlying I/O stream wrapped by @@ -58,7 +68,7 @@ impl FramedRead { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_mut(&mut self) -> &mut T { - &mut self.inner.inner.0 + &mut self.inner.inner.io } /// Consumes the `FramedRead`, returning its underlying I/O stream. @@ -67,17 +77,17 @@ impl FramedRead { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn into_inner(self) -> T { - self.inner.inner.0 + self.inner.inner.io } /// Returns a reference to the underlying decoder. pub fn decoder(&self) -> &D { - &self.inner.inner.1 + &self.inner.inner.codec } /// Returns a mutable reference to the underlying decoder. pub fn decoder_mut(&mut self) -> &mut D { - &mut self.inner.inner.1 + &mut self.inner.inner.codec } /// Returns a reference to the read buffer. @@ -88,38 +98,61 @@ impl FramedRead { impl Stream for FramedRead where - T: AsyncRead + Unpin, - D: Decoder + Unpin, + T: AsyncRead, + D: Decoder, { type Item = Result; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().inner).poll_next(cx) + self.project().inner.poll_next(cx) } } // This impl just defers to the underlying T: Sink impl Sink for FramedRead where - T: Sink + Unpin, - D: Unpin, + T: Sink, { type Error = T::Error; fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner.inner.0).poll_ready(cx) + self.project() + .inner + .project() + .inner + .project() + .io + .poll_ready(cx) } fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { - pin!(Pin::get_mut(self).inner.inner.0).start_send(item) + self.project() + .inner + .project() + .inner + .project() + .io + .start_send(item) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner.inner.0).poll_flush(cx) + self.project() + .inner + .project() + .inner + .project() + .io + .poll_flush(cx) } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner.inner.0).poll_close(cx) + self.project() + .inner + .project() + .inner + .project() + .io + .poll_close(cx) } } @@ -130,8 +163,8 @@ where { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FramedRead") - .field("inner", &self.inner.inner.0) - .field("decoder", &self.inner.inner.1) + .field("inner", &self.inner.inner.io) + .field("decoder", &self.inner.inner.codec) .field("eof", &self.inner.eof) .field("is_readable", &self.inner.is_readable) .field("buffer", &self.inner.buffer) @@ -180,6 +213,10 @@ impl FramedRead2 { &mut self.inner } + pub(crate) fn get_pin_mut(self: Pin<&mut Self>) -> Pin<&mut T> { + self.project().inner + } + pub(crate) fn buffer(&self) -> &BytesMut { &self.buffer } @@ -187,49 +224,65 @@ impl FramedRead2 { impl Stream for FramedRead2 where - T: AsyncRead + Decoder + Unpin, + T: ProjectFuse + AsyncRead, + T::Codec: Decoder, { - type Item = Result; + type Item = Result<::Item, ::Error>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let pinned = Pin::get_mut(self); + let mut pinned = self.project(); loop { // Repeatedly call `decode` or `decode_eof` as long as it is // "readable". Readable is defined as not having returned `None`. If // the upstream has returned EOF, and the decoder is no longer // readable, it can be assumed that the decoder will never become // readable again, at which point the stream is terminated. - if pinned.is_readable { - if pinned.eof { - let frame = pinned.inner.decode_eof(&mut pinned.buffer)?; + if *pinned.is_readable { + if *pinned.eof { + let frame = pinned + .inner + .as_mut() + .project() + .codec + .decode_eof(&mut pinned.buffer)?; return Poll::Ready(frame.map(Ok)); } trace!("attempting to decode a frame"); - if let Some(frame) = pinned.inner.decode(&mut pinned.buffer)? { + if let Some(frame) = pinned + .inner + .as_mut() + .project() + .codec + .decode(&mut pinned.buffer)? + { trace!("frame decoded from buffer"); return Poll::Ready(Some(Ok(frame))); } - pinned.is_readable = false; + *pinned.is_readable = false; } - assert!(!pinned.eof); + assert!(!*pinned.eof); // Otherwise, try to read more data and try again. Make sure we've // got room for at least one byte to read to ensure that we don't // get a spurious 0 that looks like EOF pinned.buffer.reserve(1); - let bytect = match pin!(pinned.inner).poll_read_buf(cx, &mut pinned.buffer)? { + let bytect = match pinned + .inner + .as_mut() + .poll_read_buf(cx, &mut pinned.buffer)? + { Poll::Ready(ct) => ct, Poll::Pending => return Poll::Pending, }; if bytect == 0 { - pinned.eof = true; + *pinned.eof = true; } - pinned.is_readable = true; + *pinned.is_readable = true; } } } diff --git a/tokio-util/src/codec/framed_write.rs b/tokio-util/src/codec/framed_write.rs index 3a95612cf16..4296e9930a1 100644 --- a/tokio-util/src/codec/framed_write.rs +++ b/tokio-util/src/codec/framed_write.rs @@ -1,6 +1,6 @@ use crate::codec::decoder::Decoder; use crate::codec::encoder::Encoder; -use crate::codec::framed::Fuse; +use crate::codec::framed::{Fuse, ProjectFuse}; use tokio::io::{AsyncBufRead, AsyncRead, AsyncWrite}; @@ -8,19 +8,26 @@ use bytes::BytesMut; use futures_core::{ready, Stream}; use futures_sink::Sink; use log::trace; +use pin_project_lite::pin_project; use std::fmt; use std::io::{self, BufRead, Read}; use std::pin::Pin; use std::task::{Context, Poll}; -/// A `Sink` of frames encoded to an `AsyncWrite`. -pub struct FramedWrite { - inner: FramedWrite2>, +pin_project! { + /// A `Sink` of frames encoded to an `AsyncWrite`. + pub struct FramedWrite { + #[pin] + inner: FramedWrite2>, + } } -pub(crate) struct FramedWrite2 { - inner: T, - buffer: BytesMut, +pin_project! { + pub(crate) struct FramedWrite2 { + #[pin] + inner: T, + buffer: BytesMut, + } } const INITIAL_CAPACITY: usize = 8 * 1024; @@ -34,7 +41,10 @@ where /// Creates a new `FramedWrite` with the given `encoder`. pub fn new(inner: T, encoder: E) -> FramedWrite { FramedWrite { - inner: framed_write2(Fuse(inner, encoder)), + inner: framed_write2(Fuse { + io: inner, + codec: encoder, + }), } } } @@ -47,7 +57,7 @@ impl FramedWrite { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_ref(&self) -> &T { - &self.inner.inner.0 + &self.inner.inner.io } /// Returns a mutable reference to the underlying I/O stream wrapped by @@ -57,7 +67,7 @@ impl FramedWrite { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn get_mut(&mut self) -> &mut T { - &mut self.inner.inner.0 + &mut self.inner.inner.io } /// Consumes the `FramedWrite`, returning its underlying I/O stream. @@ -66,55 +76,60 @@ impl FramedWrite { /// of data coming in as it may corrupt the stream of frames otherwise /// being worked with. pub fn into_inner(self) -> T { - self.inner.inner.0 + self.inner.inner.io } /// Returns a reference to the underlying decoder. pub fn encoder(&self) -> &E { - &self.inner.inner.1 + &self.inner.inner.codec } /// Returns a mutable reference to the underlying decoder. pub fn encoder_mut(&mut self) -> &mut E { - &mut self.inner.inner.1 + &mut self.inner.inner.codec } } // This impl just defers to the underlying FramedWrite2 impl Sink for FramedWrite where - T: AsyncWrite + Unpin, - E: Encoder + Unpin, + T: AsyncWrite, + E: Encoder, E::Error: From, { type Error = E::Error; fn poll_ready(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner).poll_ready(cx) + self.project().inner.poll_ready(cx) } fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { - pin!(Pin::get_mut(self).inner).start_send(item) + self.project().inner.start_send(item) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner).poll_flush(cx) + self.project().inner.poll_flush(cx) } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(Pin::get_mut(self).inner).poll_close(cx) + self.project().inner.poll_close(cx) } } impl Stream for FramedWrite where - T: Stream + Unpin, - D: Unpin, + T: Stream, { type Item = T::Item; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(Pin::get_mut(self).get_mut()).poll_next(cx) + self.project() + .inner + .project() + .inner + .project() + .io + .poll_next(cx) } } @@ -125,8 +140,8 @@ where { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FramedWrite") - .field("inner", &self.inner.get_ref().0) - .field("encoder", &self.inner.get_ref().1) + .field("inner", &self.inner.get_ref().io) + .field("encoder", &self.inner.get_ref().codec) .field("buffer", &self.inner.buffer) .finish() } @@ -169,9 +184,10 @@ impl FramedWrite2 { impl Sink for FramedWrite2 where - T: AsyncWrite + Encoder + Unpin, + T: ProjectFuse + AsyncWrite, + T::Codec: Encoder, { - type Error = T::Error; + type Error = ::Error; fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // If the buffer is already over 8KiB, then attempt to flush it. If after flushing it's @@ -191,20 +207,24 @@ where } fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { - let pinned = Pin::get_mut(self); - pinned.inner.encode(item, &mut pinned.buffer)?; + let mut pinned = self.project(); + pinned + .inner + .project() + .codec + .encode(item, &mut pinned.buffer)?; Ok(()) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { trace!("flushing framed transport"); - let pinned = Pin::get_mut(self); + let mut pinned = self.project(); while !pinned.buffer.is_empty() { trace!("writing; remaining={}", pinned.buffer.len()); let buf = &pinned.buffer; - let n = ready!(pin!(pinned.inner).poll_write(cx, &buf))?; + let n = ready!(pinned.inner.as_mut().poll_write(cx, &buf))?; if n == 0 { return Poll::Ready(Err(io::Error::new( @@ -220,15 +240,15 @@ where } // Try flushing the underlying IO - ready!(pin!(pinned.inner).poll_flush(cx))?; + ready!(pinned.inner.poll_flush(cx))?; trace!("framed transport flushed"); Poll::Ready(Ok(())) } fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - ready!(pin!(self).poll_flush(cx))?; - ready!(pin!(self.inner).poll_shutdown(cx))?; + ready!(self.as_mut().poll_flush(cx))?; + ready!(self.project().inner.poll_shutdown(cx))?; Poll::Ready(Ok(())) } @@ -263,7 +283,7 @@ impl BufRead for FramedWrite2 { } } -impl AsyncRead for FramedWrite2 { +impl AsyncRead for FramedWrite2 { unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { self.inner.prepare_uninitialized_buffer(buf) } @@ -273,16 +293,28 @@ impl AsyncRead for FramedWrite2 { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - pin!(self.get_mut().inner).poll_read(cx, buf) + self.project().inner.poll_read(cx, buf) } } -impl AsyncBufRead for FramedWrite2 { +impl AsyncBufRead for FramedWrite2 { fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - pin!(self.get_mut().inner).poll_fill_buf(cx) + self.project().inner.poll_fill_buf(cx) } fn consume(self: Pin<&mut Self>, amt: usize) { - pin!(self.get_mut().inner).consume(amt) + self.project().inner.consume(amt) + } +} + +impl ProjectFuse for FramedWrite2 +where + T: ProjectFuse, +{ + type Io = T::Io; + type Codec = T::Codec; + + fn project(self: Pin<&mut Self>) -> Fuse, &mut Self::Codec> { + self.project().inner.project() } } 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-util/src/codec/macros.rs b/tokio-util/src/codec/macros.rs deleted file mode 100644 index 902ada1cc05..00000000000 --- a/tokio-util/src/codec/macros.rs +++ /dev/null @@ -1,7 +0,0 @@ -/// A macro to reduce some of the boilerplate for projecting from -/// `Pin<&mut T>` to `Pin<&mut T.field>` -macro_rules! pin { - ($e:expr) => { - std::pin::Pin::new(&mut $e) - }; -} diff --git a/tokio-util/src/codec/mod.rs b/tokio-util/src/codec/mod.rs index 203efae466d..b162dd3a788 100644 --- a/tokio-util/src/codec/mod.rs +++ b/tokio-util/src/codec/mod.rs @@ -9,9 +9,6 @@ //! [`Sink`]: https://docs.rs/futures-sink/*/futures_sink/trait.Sink.html //! [`Stream`]: https://docs.rs/futures-core/*/futures_core/stream/trait.Stream.html -#[macro_use] -mod macros; - mod bytes_codec; pub use self::bytes_codec::BytesCodec; diff --git a/tokio-util/tests/framed.rs b/tokio-util/tests/framed.rs index 7aa40d8814b..25ddf86a1d8 100644 --- a/tokio-util/tests/framed.rs +++ b/tokio-util/tests/framed.rs @@ -5,6 +5,7 @@ use tokio_test::assert_ok; use tokio_util::codec::{Decoder, Encoder, Framed, FramedParts}; use bytes::{Buf, BufMut, BytesMut, IntoBuf}; +use futures::StreamExt; use std::io::{self, Read}; use std::pin::Pin; use std::task::{Context, Poll}; 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/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/Cargo.toml b/tokio/Cargo.toml index 903e0f8f1ef..9364ea827b3 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -24,27 +24,38 @@ categories = ["asynchronous", "network-programming"] keywords = ["io", "async", "non-blocking", "futures"] [features] -default = [ +default = ["full"] + +# enable everything +full = [ "blocking", + "dns", "fs", + "io-driver", "io-util", + "io-std", + "macros", "net", "process", - "rt-full", + "rt-core", + "rt-threaded", "signal", + "stream", "sync", "time", ] blocking = ["rt-core"] -dns = ["blocking"] -fs = ["blocking"] -io-driver = ["mio", "lazy_static", "sync"] # TODO: get rid of sync -io-util = ["pin-project", "memchr"] +dns = ["rt-core"] +fs = ["rt-core"] +io-driver = ["rt-core", "mio", "lazy_static"] +io-util = ["memchr"] +# stdin, stdout, stderr +io-std = ["rt-core"] macros = ["tokio-macros"] net = ["dns", "tcp", "udp", "uds"] process = [ - "io-util", # TODO: Get rid of + "io-driver", "libc", "mio-named-pipes", "signal", @@ -55,14 +66,9 @@ process = [ ] # Includes basic task execution capabilities rt-core = [] -rt-full = [ - "macros", +rt-threaded = [ "num_cpus", - "net", - "pin-project", "rt-core", - "sync", - "time", ] signal = [ "io-driver", @@ -73,10 +79,11 @@ signal = [ "winapi/consoleapi", "winapi/minwindef", ] +stream = ["futures-core"] sync = ["fnv"] test-util = [] tcp = ["io-driver"] -time = ["rt-core", "sync", "slab"] +time = ["rt-core", "slab"] udp = ["io-driver"] uds = ["io-driver", "mio-uds", "libc"] @@ -85,18 +92,16 @@ 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" +pin-project-lite = "0.1.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 } # Backs `DelayQueue` slab = { version = "0.4.1", optional = true } diff --git a/tokio/src/blocking/mod.rs b/tokio/src/blocking/mod.rs index 1a2b4d11ec3..500055ae94d 100644 --- a/tokio/src/blocking/mod.rs +++ b/tokio/src/blocking/mod.rs @@ -1,65 +1,11 @@ -//! Perform blocking operations from an asynchronous context. - -mod pool; -pub(crate) use self::pool::{BlockingPool, Spawner}; - -mod schedule; -mod task; +#![cfg_attr(not(feature = "blocking"), allow(dead_code, unused_imports))] -use crate::task::JoinHandle; - -/// Run the provided blocking function without blocking the executor. -/// -/// In general, issuing a blocking call or performing a lot of compute in a -/// future without yielding is not okay, as it may prevent the executor from -/// driving other futures forward. If you run a closure through this method, -/// the current executor thread will relegate all its executor duties to another -/// (possibly new) thread, and only then poll the task. Note that this requires -/// additional synchronization. -/// -/// # Examples -/// -/// ``` -/// # async fn docs() { -/// tokio::blocking::in_place(move || { -/// // do some compute-heavy work or call synchronous code -/// }); -/// # } -/// ``` -#[cfg(feature = "rt-full")] -pub fn in_place(f: F) -> R -where - F: FnOnce() -> R, -{ - use crate::runtime::{enter, thread_pool}; +//! Perform blocking operations from an asynchronous context. - enter::exit(|| thread_pool::block_in_place(f)) -} +cfg_blocking_impl! { + mod pool; + pub(crate) use pool::{spawn_blocking, BlockingPool, Spawner}; -/// Run the provided closure on a thread where blocking is acceptable. -/// -/// In general, issuing a blocking call or performing a lot of compute in a future without -/// yielding is not okay, as it may prevent the executor from driving other futures forward. -/// A closure that is run through this method will instead be run on a dedicated thread pool for -/// such blocking tasks without holding up the main futures executor. -/// -/// # Examples -/// -/// ``` -/// # async fn docs() -> Result<(), Box>{ -/// let res = tokio::blocking::spawn_blocking(move || { -/// // do some compute-heavy work or call synchronous code -/// "done computing" -/// }).await?; -/// -/// assert_eq!(res, "done computing"); -/// # Ok(()) -/// # } -/// ``` -pub fn spawn_blocking(f: F) -> JoinHandle -where - F: FnOnce() -> R + Send + 'static, - R: Send + 'static, -{ - pool::spawn(f) + mod schedule; + mod task; } diff --git a/tokio/src/blocking/pool.rs b/tokio/src/blocking/pool.rs index 45161107007..f75af78023c 100644 --- a/tokio/src/blocking/pool.rs +++ b/tokio/src/blocking/pool.rs @@ -53,7 +53,7 @@ const MAX_THREADS: u32 = 1_000; const KEEP_ALIVE: Duration = Duration::from_secs(10); /// Run the provided function on an executor dedicated to blocking operations. -pub(super) fn spawn(func: F) -> JoinHandle +pub(crate) fn spawn_blocking(func: F) -> JoinHandle where F: FnOnce() -> R + Send + 'static, { @@ -116,16 +116,20 @@ impl fmt::Debug for BlockingPool { // ===== impl Spawner ===== -impl Spawner { - #[cfg(feature = "rt-full")] - pub(crate) fn spawn_background(&self, func: F) - where - F: FnOnce() + Send + 'static, - { - let task = task::background(BlockingTask::new(func)); - self.schedule(task); + +cfg_rt_threaded! { + impl Spawner { + pub(crate) fn spawn_background(&self, func: F) + where + F: FnOnce() + Send + 'static, + { + let task = task::background(BlockingTask::new(func)); + self.schedule(task); + } } +} +impl Spawner { /// Set the blocking pool for the duration of the closure /// /// If a blocking pool is already set, it will be restored when the closure diff --git a/tokio/src/fs/file.rs b/tokio/src/fs/file.rs index 3f18831e56c..af7be5863c9 100644 --- a/tokio/src/fs/file.rs +++ b/tokio/src/fs/file.rs @@ -3,11 +3,10 @@ //! [`File`]: file/struct.File.html use self::State::*; -use crate::fs::blocking::Buf; use crate::fs::{asyncify, sys}; +use crate::io::blocking::Buf; 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/mod.rs b/tokio/src/fs/mod.rs index 9108116abbe..937242803bf 100644 --- a/tokio/src/fs/mod.rs +++ b/tokio/src/fs/mod.rs @@ -22,8 +22,6 @@ //! //! [`AsyncRead`]: https://docs.rs/tokio-io/0.1/tokio_io/trait.AsyncRead.html -pub(crate) mod blocking; - mod create_dir; pub use self::create_dir::create_dir; diff --git a/tokio/src/fs/read_dir.rs b/tokio/src/fs/read_dir.rs index 9492a2f4c69..219c7b35435 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 { + /// 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 + } - 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); } @@ -77,6 +84,19 @@ impl Stream for 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 @@ -100,13 +120,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 +151,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 +180,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 +215,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..5bd80dc89a2 --- /dev/null +++ b/tokio/src/future/try_join.rs @@ -0,0 +1,82 @@ +use crate::future::{maybe_done, MaybeDone}; + +use pin_project_lite::pin_project; +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), + } +} + +pin_project! { + pub(crate) struct TryJoin3 + where + F1: Future, + F2: Future, + F3: Future, + { + #[pin] + future1: MaybeDone, + #[pin] + future2: MaybeDone, + #[pin] + 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; + + let mut me = self.project(); + + if me.future1.as_mut().poll(cx).is_pending() { + all_done = false; + } else if me.future1.as_mut().output_mut().unwrap().is_err() { + return Poll::Ready(Err(me.future1.take_output().unwrap().err().unwrap())); + } + + if me.future2.as_mut().poll(cx).is_pending() { + all_done = false; + } else if me.future2.as_mut().output_mut().unwrap().is_err() { + return Poll::Ready(Err(me.future2.take_output().unwrap().err().unwrap())); + } + + if me.future3.as_mut().poll(cx).is_pending() { + all_done = false; + } else if me.future3.as_mut().output_mut().unwrap().is_err() { + return Poll::Ready(Err(me.future3.take_output().unwrap().err().unwrap())); + } + + if all_done { + Poll::Ready(Ok(( + me.future1.take_output().unwrap().ok().unwrap(), + me.future2.take_output().unwrap().ok().unwrap(), + 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/fs/blocking.rs b/tokio/src/io/blocking.rs similarity index 93% rename from tokio/src/fs/blocking.rs rename to tokio/src/io/blocking.rs index 695358a3900..87b251b76c8 100644 --- a/tokio/src/fs/blocking.rs +++ b/tokio/src/io/blocking.rs @@ -1,7 +1,6 @@ -use crate::fs::sys; +use crate::io::sys; use crate::io::{AsyncRead, AsyncWrite}; -use futures_core::ready; use std::cmp; use std::future::Future; use std::io; @@ -35,12 +34,14 @@ enum State { Busy(sys::Blocking<(io::Result, Buf, T)>), } -impl Blocking { - pub(crate) fn new(inner: T) -> Blocking { - Blocking { - inner: Some(inner), - state: State::Idle(Some(Buf::with_capacity(0))), - need_flush: false, +cfg_io_std! { + impl Blocking { + pub(crate) fn new(inner: T) -> Blocking { + Blocking { + inner: Some(inner), + state: State::Idle(Some(Buf::with_capacity(0))), + need_flush: false, + } } } } @@ -264,11 +265,15 @@ impl Buf { self.buf.clear(); res } +} - pub(crate) fn discard_read(&mut self) -> i64 { - let ret = -(self.bytes().len() as i64); - self.pos = 0; - self.buf.truncate(0); - ret +cfg_fs! { + impl Buf { + pub(crate) fn discard_read(&mut self) -> i64 { + let ret = -(self.bytes().len() as i64); + self.pos = 0; + self.buf.truncate(0); + ret + } } } 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/mod.rs b/tokio/src/io/io/mod.rs deleted file mode 100644 index 07ad0e7bcc4..00000000000 --- a/tokio/src/io/io/mod.rs +++ /dev/null @@ -1,51 +0,0 @@ -mod async_buf_read_ext; -mod async_read_ext; -mod async_write_ext; -mod buf_reader; -mod buf_stream; -mod buf_writer; -mod chain; -mod copy; -mod empty; -mod flush; -mod 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}; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::repeat::{repeat, Repeat}; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::sink::{sink, Sink}; -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::take::Take; - -// 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/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..df1888ceaa7 100644 --- a/tokio/src/io/mod.rs +++ b/tokio/src/io/mod.rs @@ -36,6 +36,10 @@ //! [`ErrorKind`]: enum.ErrorKind.html //! [`Result`]: type.Result.html +cfg_io_blocking! { + pub(crate) mod blocking; +} + mod async_buf_read; pub use self::async_buf_read::AsyncBufRead; @@ -45,38 +49,43 @@ 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; -#[cfg(feature = "io-util")] -pub use self::io::{ - copy, empty, repeat, sink, AsyncBufReadExt, AsyncReadExt, AsyncWriteExt, BufReader, BufStream, - BufWriter, Copy, Empty, Repeat, Sink, Take, -}; +cfg_io_std! { + mod stderr; + pub use stderr::{stderr, Stderr}; + + mod stdin; + pub use stdin::{stdin, Stdin}; -#[cfg(feature = "io-util")] -pub mod split; -#[cfg(feature = "io-util")] -pub use self::split::split; + mod stdout; + pub use stdout::{stdout, Stdout}; +} -// TODO: These should not be guarded by `fs` +cfg_io_util! { + pub mod split; + pub use split::split; -#[cfg(feature = "fs")] -mod stderr; -#[cfg(feature = "fs")] -pub use self::stderr::{stderr, Stderr}; + pub(crate) mod util; + pub use util::{ + copy, empty, repeat, sink, AsyncBufReadExt, AsyncReadExt, AsyncWriteExt, BufReader, BufStream, + BufWriter, Copy, Empty, Lines, Repeat, Sink, Split, Take, + }; -#[cfg(feature = "fs")] -mod stdin; -#[cfg(feature = "fs")] -pub use self::stdin::{stdin, Stdin}; + // Re-export io::Error so that users don't have to deal with conflicts when + // `use`ing `tokio::io` and `std::io`. + pub use std::io::{Error, ErrorKind, Result}; +} -#[cfg(feature = "fs")] -mod stdout; -#[cfg(feature = "fs")] -pub use self::stdout::{stdout, Stdout}; +cfg_not_io_util! { + cfg_process! { + pub(crate) mod util; + } +} -// Re-export io::Error so that users don't have to deal -// with conflicts when `use`ing `tokio::io` and `std::io`. -#[cfg(feature = "io-util")] -pub use std::io::{Error, ErrorKind, Result}; +cfg_io_blocking! { + /// Types in this module can be mocked out in tests. + mod sys { + // TODO: don't rename + pub(crate) use crate::blocking::spawn_blocking as run; + pub(crate) use crate::task::JoinHandle as Blocking; + } +} 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/stderr.rs b/tokio/src/io/stderr.rs index 2b9c4d71572..9e3f2e9b695 100644 --- a/tokio/src/io/stderr.rs +++ b/tokio/src/io/stderr.rs @@ -1,4 +1,4 @@ -use crate::fs::blocking::Blocking; +use crate::io::blocking::Blocking; use crate::io::AsyncWrite; use std::io; diff --git a/tokio/src/io/stdin.rs b/tokio/src/io/stdin.rs index f85bbddc052..58b1908261c 100644 --- a/tokio/src/io/stdin.rs +++ b/tokio/src/io/stdin.rs @@ -1,4 +1,4 @@ -use crate::fs::blocking::Blocking; +use crate::io::blocking::Blocking; use crate::io::AsyncRead; use std::io; diff --git a/tokio/src/io/stdout.rs b/tokio/src/io/stdout.rs index ea96dd33e4f..c06c7d3866f 100644 --- a/tokio/src/io/stdout.rs +++ b/tokio/src/io/stdout.rs @@ -1,4 +1,4 @@ -use crate::fs::blocking::Blocking; +use crate::io::blocking::Blocking; use crate::io::AsyncWrite; 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..7567d72ed9b 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) } 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 76% rename from tokio/src/io/io/buf_reader.rs rename to tokio/src/io/util/buf_reader.rs index c9698e1672e..46caa1fa7f0 100644 --- a/tokio/src/io/io/buf_reader.rs +++ b/tokio/src/io/util/buf_reader.rs @@ -1,36 +1,36 @@ -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 pin_project_lite::pin_project; use std::io::{self, Read}; use std::pin::Pin; use std::task::{Context, Poll}; use std::{cmp, fmt}; -/// The `BufReader` struct adds buffering to any reader. -/// -/// It can be excessively inefficient to work directly with a [`AsyncRead`] -/// instance. A `BufReader` performs large, infrequent reads on the underlying -/// [`AsyncRead`] and maintains an in-memory buffer of the results. -/// -/// `BufReader` can improve the speed of programs that make *small* and -/// *repeated* read calls to the same file or network socket. It does not -/// help when reading very large amounts at once, or reading just one or a few -/// times. It also provides no advantage when reading from a source that is -/// already in memory, like a `Vec`. -/// -/// When the `BufReader` is dropped, the contents of its buffer will be -/// discarded. Creating multiple instances of a `BufReader` on the same -/// stream can cause data loss. -// TODO: Examples -#[pin_project] -pub struct BufReader { - #[pin] - pub(super) inner: R, - pub(super) buf: Box<[u8]>, - pub(super) pos: usize, - pub(super) cap: usize, +pin_project! { + /// The `BufReader` struct adds buffering to any reader. + /// + /// It can be excessively inefficient to work directly with a [`AsyncRead`] + /// instance. A `BufReader` performs large, infrequent reads on the underlying + /// [`AsyncRead`] and maintains an in-memory buffer of the results. + /// + /// `BufReader` can improve the speed of programs that make *small* and + /// *repeated* read calls to the same file or network socket. It does not + /// help when reading very large amounts at once, or reading just one or a few + /// times. It also provides no advantage when reading from a source that is + /// already in memory, like a `Vec`. + /// + /// When the `BufReader` is dropped, the contents of its buffer will be + /// discarded. Creating multiple instances of a `BufReader` on the same + /// stream can cause data loss. + // TODO: Examples + pub struct BufReader { + #[pin] + pub(super) inner: R, + pub(super) buf: Box<[u8]>, + pub(super) pos: usize, + pub(super) cap: usize, + } } impl BufReader { @@ -126,26 +126,19 @@ impl AsyncRead for BufReader { } impl AsyncBufRead for BufReader { - #[project] fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let BufReader { - inner, - buf, - cap, - pos, - } = self.project(); + let me = self.project(); // If we've reached the end of our internal buffer then we need to fetch // some more data from the underlying reader. // Branch using `>=` instead of the more correct `==` // to tell the compiler that the pos..cap slice is always valid. - if *pos >= *cap { - debug_assert!(*pos == *cap); - *cap = ready!(inner.poll_read(cx, buf))?; - *pos = 0; + if *me.pos >= *me.cap { + debug_assert!(*me.pos == *me.cap); + *me.cap = ready!(me.inner.poll_read(cx, me.buf))?; + *me.pos = 0; } - Poll::Ready(Ok(&buf[*pos..*cap])) + Poll::Ready(Ok(&me.buf[*me.pos..*me.cap])) } fn consume(self: Pin<&mut Self>, amt: usize) { diff --git a/tokio/src/io/io/buf_stream.rs b/tokio/src/io/util/buf_stream.rs similarity index 62% rename from tokio/src/io/io/buf_stream.rs rename to tokio/src/io/util/buf_stream.rs index 51b6bbe512e..7ff1d3f4a37 100644 --- a/tokio/src/io/io/buf_stream.rs +++ b/tokio/src/io/util/buf_stream.rs @@ -1,64 +1,70 @@ -use crate::io::io::{BufReader, BufWriter}; +use crate::io::util::{BufReader, BufWriter}; use crate::io::{AsyncBufRead, AsyncRead, AsyncWrite}; -use pin_project::pin_project; +use pin_project_lite::pin_project; use std::io::{self}; use std::{ pin::Pin, task::{Context, Poll}, }; -/// Wraps a type that is [`AsyncWrite`] and [`AsyncRead`], and buffers its input and output. -/// -/// It can be excessively inefficient to work directly with something that implements [`AsyncWrite`] -/// and [`AsyncRead`]. For example, every `write`, however small, has to traverse the syscall -/// interface, and similarly, every read has to do the same. The [`BufWriter`] and [`BufReader`] -/// types aid with these problems respectively, but do so in only one direction. `BufStream` wraps -/// one in the other so that both directions are buffered. See their documentation for details. -#[pin_project] -#[derive(Debug)] -pub struct BufStream(#[pin] BufReader>); +pin_project! { + /// Wraps a type that is [`AsyncWrite`] and [`AsyncRead`], and buffers its input and output. + /// + /// It can be excessively inefficient to work directly with something that implements [`AsyncWrite`] + /// and [`AsyncRead`]. For example, every `write`, however small, has to traverse the syscall + /// interface, and similarly, every read has to do the same. The [`BufWriter`] and [`BufReader`] + /// types aid with these problems respectively, but do so in only one direction. `BufStream` wraps + /// one in the other so that both directions are buffered. See their documentation for details. + #[derive(Debug)] + pub struct BufStream { + #[pin] + inner: BufReader>, + } +} impl BufStream { /// Wrap a type in both [`BufWriter`] and [`BufReader`]. /// /// See the documentation for those types and [`BufStream`] for details. pub fn new(stream: RW) -> BufStream { - BufStream(BufReader::new(BufWriter::new(stream))) + BufStream { + inner: BufReader::new(BufWriter::new(stream)), + } } /// Gets a reference to the underlying I/O object. /// /// It is inadvisable to directly read from the underlying I/O object. pub fn get_ref(&self) -> &RW { - self.0.get_ref().get_ref() + self.inner.get_ref().get_ref() } /// Gets a mutable reference to the underlying I/O object. /// /// It is inadvisable to directly read from the underlying I/O object. pub fn get_mut(&mut self) -> &mut RW { - self.0.get_mut().get_mut() + self.inner.get_mut().get_mut() } /// Gets a pinned mutable reference to the underlying I/O object. /// /// It is inadvisable to directly read from the underlying I/O object. pub fn get_pin_mut(self: Pin<&mut Self>) -> Pin<&mut RW> { - self.project().0.get_pin_mut().get_pin_mut() + self.project().inner.get_pin_mut().get_pin_mut() } /// Consumes this `BufStream`, returning the underlying I/O object. /// /// Note that any leftover data in the internal buffer is lost. pub fn into_inner(self) -> RW { - self.0.into_inner().into_inner() + self.inner.into_inner().into_inner() } } impl From>> for BufStream { fn from(b: BufReader>) -> Self { - BufStream(b) + BufStream { inner: b } } } @@ -77,16 +83,18 @@ impl From>> for BufStream { written, } = b; - BufStream(BufReader { - inner: BufWriter { - inner, - buf: wbuf, - written, + BufStream { + inner: BufReader { + inner: BufWriter { + inner, + buf: wbuf, + written, + }, + buf: rbuf, + pos, + cap, }, - buf: rbuf, - pos, - cap, - }) + } } } @@ -96,15 +104,15 @@ impl AsyncWrite for BufStream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - self.project().0.poll_write(cx, buf) + self.project().inner.poll_write(cx, buf) } fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.project().0.poll_flush(cx) + self.project().inner.poll_flush(cx) } fn poll_shutdown(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.project().0.poll_shutdown(cx) + self.project().inner.poll_shutdown(cx) } } @@ -114,22 +122,22 @@ impl AsyncRead for BufStream { cx: &mut Context<'_>, buf: &mut [u8], ) -> Poll> { - self.project().0.poll_read(cx, buf) + self.project().inner.poll_read(cx, buf) } // we can't skip unconditionally because of the large buffer case in read. unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.0.prepare_uninitialized_buffer(buf) + self.inner.prepare_uninitialized_buffer(buf) } } impl AsyncBufRead for BufStream { fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.project().0.poll_fill_buf(cx) + self.project().inner.poll_fill_buf(cx) } fn consume(self: Pin<&mut Self>, amt: usize) { - self.project().0.consume(amt) + self.project().inner.consume(amt) } } diff --git a/tokio/src/io/io/buf_writer.rs b/tokio/src/io/util/buf_writer.rs similarity index 72% rename from tokio/src/io/io/buf_writer.rs rename to tokio/src/io/util/buf_writer.rs index 7415c8788d6..5c98bd82efa 100644 --- a/tokio/src/io/io/buf_writer.rs +++ b/tokio/src/io/util/buf_writer.rs @@ -1,40 +1,40 @@ -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 pin_project_lite::pin_project; use std::fmt; use std::io::{self, Write}; use std::pin::Pin; use std::task::{Context, Poll}; -/// Wraps a writer and buffers its output. -/// -/// It can be excessively inefficient to work directly with something that -/// implements [`AsyncWrite`]. A `BufWriter` keeps an in-memory buffer of data and -/// writes it to an underlying writer in large, infrequent batches. -/// -/// `BufWriter` can improve the speed of programs that make *small* and -/// *repeated* write calls to the same file or network socket. It does not -/// help when writing very large amounts at once, or writing just one or a few -/// times. It also provides no advantage when writing to a destination that is -/// in memory, like a `Vec`. -/// -/// When the `BufWriter` is dropped, the contents of its buffer will be -/// discarded. Creating multiple instances of a `BufWriter` on the same -/// stream can cause data loss. If you need to write out the contents of its -/// buffer, you must manually call flush before the writer is dropped. -/// -/// [`AsyncWrite`]: AsyncWrite -/// [`flush`]: super::AsyncWriteExt::flush -/// -// TODO: Examples -#[pin_project] -pub struct BufWriter { - #[pin] - pub(super) inner: W, - pub(super) buf: Vec, - pub(super) written: usize, +pin_project! { + /// Wraps a writer and buffers its output. + /// + /// It can be excessively inefficient to work directly with something that + /// implements [`AsyncWrite`]. A `BufWriter` keeps an in-memory buffer of data and + /// writes it to an underlying writer in large, infrequent batches. + /// + /// `BufWriter` can improve the speed of programs that make *small* and + /// *repeated* write calls to the same file or network socket. It does not + /// help when writing very large amounts at once, or writing just one or a few + /// times. It also provides no advantage when writing to a destination that is + /// in memory, like a `Vec`. + /// + /// When the `BufWriter` is dropped, the contents of its buffer will be + /// discarded. Creating multiple instances of a `BufWriter` on the same + /// stream can cause data loss. If you need to write out the contents of its + /// buffer, you must manually call flush before the writer is dropped. + /// + /// [`AsyncWrite`]: AsyncWrite + /// [`flush`]: super::AsyncWriteExt::flush + /// + // TODO: Examples + pub struct BufWriter { + #[pin] + pub(super) inner: W, + pub(super) buf: Vec, + pub(super) written: usize, + } } impl BufWriter { @@ -53,19 +53,13 @@ impl BufWriter { } } - #[project] fn flush_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let BufWriter { - mut inner, - buf, - written, - } = self.project(); - - let len = buf.len(); + let mut me = self.project(); + + let len = me.buf.len(); let mut ret = Ok(()); - while *written < len { - match ready!(inner.as_mut().poll_write(cx, &buf[*written..])) { + while *me.written < len { + match ready!(me.inner.as_mut().poll_write(cx, &me.buf[*me.written..])) { Ok(0) => { ret = Err(io::Error::new( io::ErrorKind::WriteZero, @@ -73,17 +67,17 @@ impl BufWriter { )); break; } - Ok(n) => *written += n, + Ok(n) => *me.written += n, Err(e) => { ret = Err(e); break; } } } - if *written > 0 { - buf.drain(..*written); + if *me.written > 0 { + me.buf.drain(..*me.written); } - *written = 0; + *me.written = 0; Poll::Ready(ret) } diff --git a/tokio/src/io/io/chain.rs b/tokio/src/io/util/chain.rs similarity index 83% rename from tokio/src/io/io/chain.rs rename to tokio/src/io/util/chain.rs index 3f997ff55aa..2b3531e78ba 100644 --- a/tokio/src/io/io/chain.rs +++ b/tokio/src/io/util/chain.rs @@ -1,21 +1,21 @@ use crate::io::{AsyncBufRead, AsyncRead}; -use futures_core::ready; -use pin_project::{pin_project, project}; +use pin_project_lite::pin_project; use std::fmt; use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -/// Stream for the [`chain`](super::AsyncReadExt::chain) method. -#[pin_project] -#[must_use = "streams do nothing unless polled"] -pub struct Chain { - #[pin] - first: T, - #[pin] - second: U, - done_first: bool, +pin_project! { + /// Stream for the [`chain`](super::AsyncReadExt::chain) method. + #[must_use = "streams do nothing unless polled"] + pub struct Chain { + #[pin] + first: T, + #[pin] + second: U, + done_first: bool, + } } pub(super) fn chain(first: T, second: U) -> Chain @@ -105,24 +105,18 @@ where T: AsyncBufRead, U: AsyncBufRead, { - #[project] fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let Chain { - first, - second, - done_first, - } = self.project(); + let me = self.project(); - if !*done_first { - match ready!(first.poll_fill_buf(cx)?) { + if !*me.done_first { + match ready!(me.first.poll_fill_buf(cx)?) { buf if buf.is_empty() => { - *done_first = true; + *me.done_first = true; } buf => return Poll::Ready(Ok(buf)), } } - second.poll_fill_buf(cx) + me.second.poll_fill_buf(cx) } fn consume(self: Pin<&mut Self>, amt: usize) { 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..96c0cc27a95 --- /dev/null +++ b/tokio/src/io/util/lines.rs @@ -0,0 +1,113 @@ +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}; + +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 +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| Pin::new(&mut *self).poll_next_line(cx)).await + } +} + +impl Lines +where + R: AsyncBufRead, +{ + #[doc(hidden)] + pub fn poll_next_line( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { + let me = self.project(); + + 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 me.buf.ends_with('\n') { + me.buf.pop(); + + if me.buf.ends_with('\r') { + me.buf.pop(); + } + } + + 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)), + }) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn assert_unpin() { + crate::is_unpin::>(); + } +} diff --git a/tokio/src/io/util/mod.rs b/tokio/src/io/util/mod.rs new file mode 100644 index 00000000000..24451ef6a85 --- /dev/null +++ b/tokio/src/io/util/mod.rs @@ -0,0 +1,75 @@ +#![allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 + +cfg_io_util! { + mod async_buf_read_ext; + pub use async_buf_read_ext::AsyncBufReadExt; + + mod async_read_ext; + pub use async_read_ext::AsyncReadExt; + + mod async_write_ext; + pub use async_write_ext::AsyncWriteExt; + + mod buf_reader; + pub use buf_reader::BufReader; + + mod buf_stream; + pub use buf_stream::BufStream; + + mod buf_writer; + pub use buf_writer::BufWriter; + + mod chain; + + mod copy; + pub use copy::{copy, Copy}; + + mod empty; + pub use empty::{empty, Empty}; + + mod flush; + + mod lines; + pub use lines::Lines; + + mod read; + mod read_exact; + mod read_line; + + mod read_to_end; + cfg_process! { + pub(crate) use read_to_end::read_to_end; + } + + mod read_to_string; + mod read_until; + + mod repeat; + pub use repeat::{repeat, Repeat}; + + mod shutdown; + + mod sink; + pub use sink::{sink, Sink}; + + mod split; + pub use split::Split; + + mod take; + pub use 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; +} + +cfg_not_io_util! { + cfg_process! { + mod read_to_end; + // Used by process + pub(crate) use read_to_end::read_to_end; + } +} 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..be66f70c3f8 --- /dev/null +++ b/tokio/src/io/util/split.rs @@ -0,0 +1,111 @@ +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}; + +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 +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| Pin::new(&mut *self).poll_next_segment(cx)).await + } +} + +impl Split +where + R: AsyncBufRead, +{ + #[doc(hidden)] + pub fn poll_next_segment( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>>> { + let me = self.project(); + + let n = ready!(read_until_internal( + me.reader, cx, *me.delim, me.buf, me.read, + ))?; + + if n == 0 && me.buf.is_empty() { + return Poll::Ready(Ok(None)); + } + + if me.buf.last() == Some(me.delim) { + me.buf.pop(); + } + + 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)), + }) + } +} + +#[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 84% rename from tokio/src/io/io/take.rs rename to tokio/src/io/util/take.rs index 255f835302f..86403853f98 100644 --- a/tokio/src/io/io/take.rs +++ b/tokio/src/io/util/take.rs @@ -1,20 +1,20 @@ use crate::io::{AsyncBufRead, AsyncRead}; -use futures_core::ready; -use pin_project::{pin_project, project}; +use pin_project_lite::pin_project; use std::pin::Pin; use std::task::{Context, Poll}; use std::{cmp, io}; -/// Stream for the [`take`](super::AsyncReadExt::take) method. -#[pin_project] -#[derive(Debug)] -#[must_use = "streams do nothing unless you `.await` or poll them"] -pub struct Take { - #[pin] - inner: R, - // Add '_' to avoid conflicts with `limit` method. - limit_: u64, +pin_project! { + /// Stream for the [`take`](super::AsyncReadExt::take) method. + #[derive(Debug)] + #[must_use = "streams do nothing unless you `.await` or poll them"] + pub struct Take { + #[pin] + inner: R, + // Add '_' to avoid conflicts with `limit` method. + limit_: u64, + } } pub(super) fn take(inner: R, limit: u64) -> Take { @@ -96,18 +96,16 @@ impl AsyncRead for Take { } impl AsyncBufRead for Take { - #[project] fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - #[project] - let Take { inner, limit_ } = self.project(); + let me = self.project(); // Don't call into inner reader at all at EOF because it may still block - if *limit_ == 0 { + if *me.limit_ == 0 { return Poll::Ready(Ok(&[])); } - let buf = ready!(inner.poll_fill_buf(cx)?); - let cap = cmp::min(buf.len() as u64, *limit_) as usize; + let buf = ready!(me.inner.poll_fill_buf(cx)?); + let cap = cmp::min(buf.len() as u64, *me.limit_) as usize; Poll::Ready(Ok(&buf[..cap])) } 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..d82d58d8a5b 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -70,70 +70,68 @@ //! } //! ``` -#[cfg(all(loom, test))] -macro_rules! thread_local { - ($($tts:tt)+) => { loom::thread_local!{ $($tts)+ } } -} - -// At the top due to macros -#[cfg(test)] +// macros used internally #[macro_use] -mod tests; +mod macros; -#[cfg(feature = "blocking")] -pub mod blocking; +// Blocking task implementation +pub(crate) mod blocking; -#[cfg(feature = "fs")] -pub mod fs; +cfg_fs! { + pub mod fs; +} -pub mod future; +mod future; pub mod io; -#[cfg(feature = "io-driver")] pub mod net; mod loom; pub mod prelude; -#[cfg(feature = "process")] -#[cfg(not(loom))] -pub mod process; +cfg_process! { + pub mod process; +} pub mod runtime; -#[cfg(feature = "signal")] -#[cfg(not(loom))] -pub mod signal; - -pub mod stream; - -#[cfg(feature = "sync")] -pub mod sync; +cfg_signal! { + pub mod signal; +} -#[cfg(feature = "rt-core")] -pub mod task; +cfg_sync! { + pub mod sync; +} +cfg_not_sync! { + mod sync; +} -#[cfg(feature = "time")] -pub mod time; +cfg_rt_core! { + pub mod task; + pub use task::spawn; +} -#[cfg(feature = "rt-full")] -mod util; +cfg_time! { + pub mod time; +} -#[doc(inline)] -#[cfg(feature = "rt-core")] -pub use crate::runtime::spawn; +cfg_rt_threaded! { + mod util; +} -#[cfg(not(test))] // Work around for rust-lang/rust#62127 -#[cfg(feature = "macros")] -#[doc(inline)] -pub use tokio_macros::main; +cfg_macros! { + #[cfg(not(test))] // Work around for rust-lang/rust#62127 + pub use tokio_macros::main; + pub use tokio_macros::test; +} -#[cfg(feature = "macros")] -#[doc(inline)] -pub use tokio_macros::test; +// Tests +#[cfg(test)] +mod tests; +// TODO: rm #[cfg(feature = "io-util")] #[cfg(test)] fn is_unpin() {} diff --git a/tokio/src/loom/std/mod.rs b/tokio/src/loom/std/mod.rs index c5bd6039fb3..2c5b7eaa3e9 100644 --- a/tokio/src/loom/std/mod.rs +++ b/tokio/src/loom/std/mod.rs @@ -1,4 +1,4 @@ -#![cfg_attr(not(feature = "rt-full"), allow(unused_imports, dead_code))] +#![cfg_attr(not(feature = "full"), allow(unused_imports, dead_code))] mod atomic_u32; mod atomic_u64; @@ -11,7 +11,7 @@ pub(crate) mod cell { pub(crate) use super::causal_cell::{CausalCell, CausalCheck}; } -#[cfg(feature = "sync")] +#[cfg(any(feature = "sync", feature = "io-driver"))] pub(crate) mod future { pub(crate) use crate::sync::AtomicWaker; } @@ -51,12 +51,12 @@ pub(crate) mod sync { } pub(crate) mod sys { - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] pub(crate) fn num_cpus() -> usize { usize::max(1, num_cpus::get_physical()) } - #[cfg(not(feature = "rt-full"))] + #[cfg(not(feature = "rt-threaded"))] pub(crate) fn num_cpus() -> usize { 1 } diff --git a/tokio/src/macros/assert.rs b/tokio/src/macros/assert.rs new file mode 100644 index 00000000000..fd6601b445e --- /dev/null +++ b/tokio/src/macros/assert.rs @@ -0,0 +1,19 @@ +/// Assert option is some +macro_rules! assert_some { + ($e:expr) => {{ + match $e { + Some(v) => v, + _ => panic!("expected some, was none"), + } + }}; +} + +/// Assert option is none +macro_rules! assert_none { + ($e:expr) => {{ + match $e { + Some(v) => panic!("expected none, was {:?}", v), + _ => {} + } + }}; +} diff --git a/tokio/src/macros/cfg.rs b/tokio/src/macros/cfg.rs new file mode 100644 index 00000000000..5e84a3ac192 --- /dev/null +++ b/tokio/src/macros/cfg.rs @@ -0,0 +1,217 @@ +#![allow(unused_macros)] + +macro_rules! cfg_atomic_waker { + ($($item:item)*) => { + $( #[cfg(any(feature = "io-driver", feature = "time"))] $item )* + } +} + +macro_rules! cfg_blocking { + ($($item:item)*) => { + $( #[cfg(feature = "blocking")] $item )* + } +} + +/// Enable blocking API internals +macro_rules! cfg_blocking_impl { + ($($item:item)*) => { + $( + #[cfg(any( + feature = "blocking", + feature = "fs", + feature = "dns", + feature = "io-std", + feature = "rt-threaded", + ))] + $item + )* + } +} + +/// Enable blocking API internals +macro_rules! cfg_not_blocking_impl { + ($($item:item)*) => { + $( + #[cfg(not(any( + feature = "blocking", + feature = "fs", + feature = "dns", + feature = "io-std", + feature = "rt-threaded", + )))] + $item + )* + } +} + +macro_rules! cfg_dns { + ($($item:item)*) => { + $( #[cfg(feature = "dns")] $item )* + } +} + +macro_rules! cfg_fs { + ($($item:item)*) => { $( #[cfg(feature = "fs")] $item )* } +} + +macro_rules! cfg_io_blocking { + ($($item:item)*) => { + $( #[cfg(any(feature = "io-std", feature = "fs"))] $item )* + } +} + +macro_rules! cfg_io_driver { + ($($item:item)*) => { + $( #[cfg(feature = "io-driver")] $item )* + } +} + +macro_rules! cfg_not_io_driver { + ($($item:item)*) => { + $( #[cfg(not(feature = "io-driver"))] $item )* + } +} + +macro_rules! cfg_io_std { + ($($item:item)*) => { + $( #[cfg(feature = "io-std")] $item )* + } +} + +macro_rules! cfg_io_util { + ($($item:item)*) => { + $( #[cfg(feature = "io-util")] $item )* + } +} + +macro_rules! cfg_not_io_util { + ($($item:item)*) => { + $( #[cfg(not(feature = "io-util"))] $item )* + } +} + +macro_rules! cfg_loom { + ($($item:item)*) => { + $( #[cfg(loom)] $item )* + } +} + +macro_rules! cfg_not_loom { + ($($item:item)*) => { + $( #[cfg(not(loom))] $item )* + } +} + +macro_rules! cfg_macros { + ($($item:item)*) => { + $( + #[cfg(feature = "macros")] + #[doc(inline)] + $item + )* + } +} + +macro_rules! cfg_process { + ($($item:item)*) => { + $( + #[cfg(feature = "process")] + #[cfg(not(loom))] + $item + )* + } +} + +macro_rules! cfg_signal { + ($($item:item)*) => { + $( + #[cfg(feature = "signal")] + #[cfg(not(loom))] + $item + )* + } +} + +macro_rules! cfg_stream { + ($($item:item)*) => { + $( #[cfg(feature = "stream")] $item )* + } +} + +macro_rules! cfg_sync { + ($($item:item)*) => { + $( #[cfg(feature = "sync")] $item )* + } +} + +macro_rules! cfg_not_sync { + ($($item:item)*) => { + $( #[cfg(not(feature = "sync"))] $item )* + } +} + +macro_rules! cfg_rt_core { + ($($item:item)*) => { + $( #[cfg(feature = "rt-core")] $item )* + } +} + +macro_rules! cfg_not_rt_core { + ($($item:item)*) => { + $( #[cfg(not(feature = "rt-core"))] $item )* + } +} + +macro_rules! cfg_rt_threaded { + ($($item:item)*) => { + $( #[cfg(feature = "rt-threaded")] $item )* + } +} + +macro_rules! cfg_not_rt_threaded { + ($($item:item)*) => { + $( #[cfg(not(feature = "rt-threaded"))] $item )* + } +} + +macro_rules! cfg_tcp { + ($($item:item)*) => { + $( #[cfg(feature = "tcp")] $item )* + } +} + +macro_rules! cfg_test_util { + ($($item:item)*) => { + $( #[cfg(feature = "test-util")] $item )* + } +} + +macro_rules! cfg_not_test_util { + ($($item:item)*) => { + $( #[cfg(not(feature = "test-util"))] $item )* + } +} + +macro_rules! cfg_time { + ($($item:item)*) => { + $( #[cfg(feature = "time")] $item )* + } +} + +macro_rules! cfg_not_time { + ($($item:item)*) => { + $( #[cfg(not(feature = "time"))] $item )* + } +} + +macro_rules! cfg_udp { + ($($item:item)*) => { + $( #[cfg(feature = "udp")] $item )* + } +} + +macro_rules! cfg_uds { + ($($item:item)*) => { + $( #[cfg(all(unix, feature = "uds"))] $item )* + } +} diff --git a/tokio/src/macros/loom.rs b/tokio/src/macros/loom.rs new file mode 100644 index 00000000000..d57d9fb0f7b --- /dev/null +++ b/tokio/src/macros/loom.rs @@ -0,0 +1,12 @@ +macro_rules! if_loom { + ($($t:tt)*) => {{ + #[cfg(loom)] + const LOOM: bool = true; + #[cfg(not(loom))] + const LOOM: bool = false; + + if LOOM { + $($t)* + } + }} +} diff --git a/tokio/src/macros/mod.rs b/tokio/src/macros/mod.rs new file mode 100644 index 00000000000..9136e594449 --- /dev/null +++ b/tokio/src/macros/mod.rs @@ -0,0 +1,17 @@ +#![cfg_attr(not(feature = "full"), allow(unused_macros))] + +#[macro_use] +#[cfg(test)] +mod assert; + +#[macro_use] +mod cfg; + +#[macro_use] +mod loom; + +#[macro_use] +mod ready; + +#[macro_use] +mod thread_local; diff --git a/tokio/src/macros/ready.rs b/tokio/src/macros/ready.rs new file mode 100644 index 00000000000..1f48623b801 --- /dev/null +++ b/tokio/src/macros/ready.rs @@ -0,0 +1,8 @@ +macro_rules! ready { + ($e:expr $(,)?) => { + match $e { + std::task::Poll::Ready(t) => t, + std::task::Poll::Pending => return std::task::Poll::Pending, + } + }; +} diff --git a/tokio/src/macros/thread_local.rs b/tokio/src/macros/thread_local.rs new file mode 100644 index 00000000000..d848947350d --- /dev/null +++ b/tokio/src/macros/thread_local.rs @@ -0,0 +1,4 @@ +#[cfg(all(loom, test))] +macro_rules! thread_local { + ($($tts:tt)+) => { loom::thread_local!{ $($tts)+ } } +} diff --git a/tokio/src/net/addr.rs b/tokio/src/net/addr.rs index 9b3cc2bcb5c..108557e4d36 100644 --- a/tokio/src/net/addr.rs +++ b/tokio/src/net/addr.rs @@ -1,8 +1,7 @@ -use futures_util::future; +use crate::future; + use std::io; -use std::net::{IpAddr, SocketAddr}; -#[cfg(feature = "dns")] -use std::net::{Ipv4Addr, Ipv6Addr}; +use std::net::{IpAddr, Ipv4Addr, Ipv6Addr, SocketAddr, SocketAddrV4, SocketAddrV6}; /// Convert or resolve without blocking to one or more `SocketAddr` values. /// @@ -17,6 +16,22 @@ pub trait ToSocketAddrs: sealed::ToSocketAddrsPriv {} type ReadyFuture = future::Ready>; +// ===== impl &impl ToSocketAddrs ===== + +impl ToSocketAddrs for &T {} + +impl sealed::ToSocketAddrsPriv for &T +where + T: sealed::ToSocketAddrsPriv + ?Sized, +{ + type Iter = T::Iter; + type Future = T::Future; + + fn to_socket_addrs(&self) -> Self::Future { + (**self).to_socket_addrs() + } +} + // ===== impl SocketAddr ===== impl ToSocketAddrs for SocketAddr {} @@ -27,122 +42,155 @@ 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) } } -// ===== impl str ===== +// ===== impl SocketAddrV4 ===== -#[cfg(feature = "dns")] -impl ToSocketAddrs for str {} +impl ToSocketAddrs for SocketAddrV4 {} -#[cfg(feature = "dns")] -impl sealed::ToSocketAddrsPriv for str { - type Iter = sealed::OneOrMore; - type Future = sealed::MaybeReady; +impl sealed::ToSocketAddrsPriv for SocketAddrV4 { + type Iter = std::option::IntoIter; + type Future = ReadyFuture; fn to_socket_addrs(&self) -> Self::Future { - use crate::blocking; - use sealed::MaybeReady; + SocketAddr::V4(*self).to_socket_addrs() + } +} - // First check if the input parses as a socket address - let res: Result = self.parse(); +// ===== impl SocketAddrV6 ===== - if let Ok(addr) = res { - return MaybeReady::Ready(Some(addr)); - } +impl ToSocketAddrs for SocketAddrV6 {} - // Run DNS lookup on the blocking pool - let s = self.to_owned(); +impl sealed::ToSocketAddrsPriv for SocketAddrV6 { + type Iter = std::option::IntoIter; + type Future = ReadyFuture; - MaybeReady::Blocking(blocking::spawn_blocking(move || { - std::net::ToSocketAddrs::to_socket_addrs(&s) - })) + fn to_socket_addrs(&self) -> Self::Future { + SocketAddr::V6(*self).to_socket_addrs() } } -// ===== impl (&str, u16) ===== +// ===== impl (IpAddr, u16) ===== -#[cfg(feature = "dns")] -impl ToSocketAddrs for (&'_ str, u16) {} +impl ToSocketAddrs for (IpAddr, u16) {} -#[cfg(feature = "dns")] -impl sealed::ToSocketAddrsPriv for (&'_ str, u16) { - type Iter = sealed::OneOrMore; - type Future = sealed::MaybeReady; +impl sealed::ToSocketAddrsPriv for (IpAddr, u16) { + type Iter = std::option::IntoIter; + type Future = ReadyFuture; fn to_socket_addrs(&self) -> Self::Future { - use crate::blocking; - use sealed::MaybeReady; - use std::net::{SocketAddrV4, SocketAddrV6}; - - let (host, port) = *self; - - // try to parse the host as a regular IP address first - if let Ok(addr) = host.parse::() { - let addr = SocketAddrV4::new(addr, port); - let addr = SocketAddr::V4(addr); - - return MaybeReady::Ready(Some(addr)); - } + let iter = Some(SocketAddr::from(*self)).into_iter(); + future::ok(iter) + } +} - if let Ok(addr) = host.parse::() { - let addr = SocketAddrV6::new(addr, port, 0, 0); - let addr = SocketAddr::V6(addr); +// ===== impl (Ipv4Addr, u16) ===== - return MaybeReady::Ready(Some(addr)); - } +impl ToSocketAddrs for (Ipv4Addr, u16) {} - let host = host.to_owned(); +impl sealed::ToSocketAddrsPriv for (Ipv4Addr, u16) { + type Iter = std::option::IntoIter; + type Future = ReadyFuture; - MaybeReady::Blocking(blocking::spawn_blocking(move || { - std::net::ToSocketAddrs::to_socket_addrs(&(&host[..], port)) - })) + fn to_socket_addrs(&self) -> Self::Future { + let (ip, port) = *self; + SocketAddrV4::new(ip, port).to_socket_addrs() } } -// ===== impl (IpAddr, u16) ===== +// ===== impl (Ipv6Addr, u16) ===== -impl ToSocketAddrs for (IpAddr, u16) {} +impl ToSocketAddrs for (Ipv6Addr, u16) {} -impl sealed::ToSocketAddrsPriv for (IpAddr, u16) { +impl sealed::ToSocketAddrsPriv for (Ipv6Addr, u16) { type Iter = std::option::IntoIter; type Future = ReadyFuture; fn to_socket_addrs(&self) -> Self::Future { - let iter = Some(SocketAddr::from(*self)).into_iter(); - future::ready(Ok(iter)) + let (ip, port) = *self; + SocketAddrV6::new(ip, port, 0, 0).to_socket_addrs() } } -// ===== impl String ===== +cfg_dns! { + // ===== impl str ===== -#[cfg(feature = "dns")] -impl ToSocketAddrs for String {} + impl ToSocketAddrs for str {} -#[cfg(feature = "dns")] -impl sealed::ToSocketAddrsPriv for String { - type Iter = ::Iter; - type Future = ::Future; + impl sealed::ToSocketAddrsPriv for str { + type Iter = sealed::OneOrMore; + type Future = sealed::MaybeReady; - fn to_socket_addrs(&self) -> Self::Future { - (&self[..]).to_socket_addrs() + fn to_socket_addrs(&self) -> Self::Future { + use crate::blocking; + use sealed::MaybeReady; + + // First check if the input parses as a socket address + let res: Result = self.parse(); + + if let Ok(addr) = res { + return MaybeReady::Ready(Some(addr)); + } + + // Run DNS lookup on the blocking pool + let s = self.to_owned(); + + MaybeReady::Blocking(blocking::spawn_blocking(move || { + std::net::ToSocketAddrs::to_socket_addrs(&s) + })) + } } -} -// ===== impl &'_ impl ToSocketAddrs ===== + // ===== impl (&str, u16) ===== -impl ToSocketAddrs for &'_ T {} + impl ToSocketAddrs for (&str, u16) {} -impl sealed::ToSocketAddrsPriv for &'_ T -where - T: sealed::ToSocketAddrsPriv + ?Sized, -{ - type Iter = T::Iter; - type Future = T::Future; + impl sealed::ToSocketAddrsPriv for (&str, u16) { + type Iter = sealed::OneOrMore; + type Future = sealed::MaybeReady; - fn to_socket_addrs(&self) -> Self::Future { - (**self).to_socket_addrs() + fn to_socket_addrs(&self) -> Self::Future { + use crate::blocking; + use sealed::MaybeReady; + + let (host, port) = *self; + + // try to parse the host as a regular IP address first + if let Ok(addr) = host.parse::() { + let addr = SocketAddrV4::new(addr, port); + let addr = SocketAddr::V4(addr); + + return MaybeReady::Ready(Some(addr)); + } + + if let Ok(addr) = host.parse::() { + let addr = SocketAddrV6::new(addr, port, 0, 0); + let addr = SocketAddr::V6(addr); + + return MaybeReady::Ready(Some(addr)); + } + + let host = host.to_owned(); + + MaybeReady::Blocking(blocking::spawn_blocking(move || { + std::net::ToSocketAddrs::to_socket_addrs(&(&host[..], port)) + })) + } + } + + // ===== impl String ===== + + impl ToSocketAddrs for String {} + + impl sealed::ToSocketAddrsPriv for String { + type Iter = ::Iter; + type Future = ::Future; + + fn to_socket_addrs(&self) -> Self::Future { + (&self[..]).to_socket_addrs() + } } } @@ -151,20 +199,18 @@ pub(crate) mod sealed { //! part of the `ToSocketAddrs` public API. The details will change over //! time. - #[cfg(feature = "dns")] - use crate::task::JoinHandle; - use std::future::Future; use std::io; use std::net::SocketAddr; - #[cfg(feature = "dns")] - use std::option; - #[cfg(feature = "dns")] - use std::pin::Pin; - #[cfg(feature = "dns")] - use std::task::{Context, Poll}; - #[cfg(feature = "dns")] - use std::vec; + + cfg_dns! { + use crate::task::JoinHandle; + + use std::option; + use std::pin::Pin; + use std::task::{Context, Poll}; + use std::vec; + } #[doc(hidden)] pub trait ToSocketAddrsPriv { @@ -174,58 +220,54 @@ pub(crate) mod sealed { fn to_socket_addrs(&self) -> Self::Future; } - #[doc(hidden)] - #[derive(Debug)] - #[cfg(feature = "dns")] - pub enum MaybeReady { - Ready(Option), - Blocking(JoinHandle>>), - } - - #[doc(hidden)] - #[derive(Debug)] - #[cfg(feature = "dns")] - pub enum OneOrMore { - One(option::IntoIter), - More(vec::IntoIter), - } + cfg_dns! { + #[doc(hidden)] + #[derive(Debug)] + pub enum MaybeReady { + Ready(Option), + Blocking(JoinHandle>>), + } - #[cfg(feature = "dns")] - impl Future for MaybeReady { - type Output = io::Result; + #[doc(hidden)] + #[derive(Debug)] + pub enum OneOrMore { + One(option::IntoIter), + More(vec::IntoIter), + } - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - use futures_core::ready; + impl Future for MaybeReady { + type Output = io::Result; - match *self { - MaybeReady::Ready(ref mut i) => { - let iter = OneOrMore::One(i.take().into_iter()); - Poll::Ready(Ok(iter)) - } - MaybeReady::Blocking(ref mut rx) => { - let res = ready!(Pin::new(rx).poll(cx))?.map(OneOrMore::More); + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match *self { + MaybeReady::Ready(ref mut i) => { + let iter = OneOrMore::One(i.take().into_iter()); + Poll::Ready(Ok(iter)) + } + MaybeReady::Blocking(ref mut rx) => { + let res = ready!(Pin::new(rx).poll(cx))?.map(OneOrMore::More); - Poll::Ready(res) + Poll::Ready(res) + } } } } - } - #[cfg(feature = "dns")] - impl Iterator for OneOrMore { - type Item = SocketAddr; + impl Iterator for OneOrMore { + type Item = SocketAddr; - fn next(&mut self) -> Option { - match self { - OneOrMore::One(i) => i.next(), - OneOrMore::More(i) => i.next(), + fn next(&mut self) -> Option { + match self { + OneOrMore::One(i) => i.next(), + OneOrMore::More(i) => i.next(), + } } - } - fn size_hint(&self) -> (usize, Option) { - match self { - OneOrMore::One(i) => i.size_hint(), - OneOrMore::More(i) => i.size_hint(), + fn size_hint(&self) -> (usize, Option) { + match self { + OneOrMore::One(i) => i.size_hint(), + OneOrMore::More(i) => i.size_hint(), + } } } } diff --git a/tokio/src/net/mod.rs b/tokio/src/net/mod.rs index 2ebf773fe21..50021e37b9c 100644 --- a/tokio/src/net/mod.rs +++ b/tokio/src/net/mod.rs @@ -24,24 +24,22 @@ mod addr; pub use addr::ToSocketAddrs; -pub mod driver; - -pub mod util; - -#[cfg(feature = "tcp")] -pub mod tcp; - -#[cfg(feature = "tcp")] -pub use self::tcp::{TcpListener, TcpStream}; - -#[cfg(feature = "udp")] -pub mod udp; - -#[cfg(feature = "udp")] -pub use self::udp::UdpSocket; - -#[cfg(all(unix, feature = "uds"))] -pub mod unix; - -#[cfg(all(unix, feature = "uds"))] -pub use self::unix::{UnixDatagram, UnixListener, UnixStream}; +cfg_io_driver! { + pub mod driver; + pub mod util; +} + +cfg_tcp! { + pub mod tcp; + pub use tcp::{TcpListener, TcpStream}; +} + +cfg_udp! { + pub mod udp; + pub use udp::UdpSocket; +} + +cfg_uds! { + pub mod unix; + pub use unix::{UnixDatagram, UnixListener, UnixStream}; +} diff --git a/tokio/src/net/tcp/incoming.rs b/tokio/src/net/tcp/incoming.rs index 0339615ae2e..3033aefa33c 100644 --- a/tokio/src/net/tcp/incoming.rs +++ b/tokio/src/net/tcp/incoming.rs @@ -1,8 +1,5 @@ -use crate::net::tcp::TcpListener; -use crate::net::tcp::TcpStream; +use crate::net::tcp::{TcpListener, TcpStream}; -use futures_core::ready; -use futures_core::stream::Stream; use std::io; use std::pin::Pin; use std::task::{Context, Poll}; @@ -11,17 +8,27 @@ use std::task::{Context, Poll}; /// stream of sockets received from a listener. #[must_use = "streams do nothing unless polled"] #[derive(Debug)] -pub struct Incoming { - inner: TcpListener, +pub struct Incoming<'a> { + inner: &'a mut TcpListener, } -impl Incoming { - pub(crate) fn new(listener: TcpListener) -> Incoming { +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)) + } } -impl Stream for Incoming { +#[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> { diff --git a/tokio/src/net/tcp/listener.rs b/tokio/src/net/tcp/listener.rs index b4bf4fcb110..c0e7f1b3478 100644 --- a/tokio/src/net/tcp/listener.rs +++ b/tokio/src/net/tcp/listener.rs @@ -1,9 +1,8 @@ +use crate::future::poll_fn; use crate::net::tcp::{Incoming, 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 @@ -126,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> { @@ -242,7 +239,7 @@ impl TcpListener { /// 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 { + pub fn incoming(&mut self) -> Incoming<'_> { Incoming::new(self) } diff --git a/tokio/src/net/tcp/mod.rs b/tokio/src/net/tcp/mod.rs index f80eb833874..e3acf54217e 100644 --- a/tokio/src/net/tcp/mod.rs +++ b/tokio/src/net/tcp/mod.rs @@ -14,15 +14,13 @@ //! [`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; +mod incoming; +pub use self::incoming::Incoming; + pub mod split; mod stream; 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 index a66f21da0f0..dbe964a8d29 100644 --- a/tokio/src/net/unix/incoming.rs +++ b/tokio/src/net/unix/incoming.rs @@ -1,7 +1,5 @@ -use super::{UnixListener, UnixStream}; +use crate::net::unix::{UnixListener, UnixStream}; -use futures_core::ready; -use futures_core::stream::Stream; use std::io; use std::pin::Pin; use std::task::{Context, Poll}; @@ -9,21 +7,31 @@ use std::task::{Context, Poll}; /// Stream of listeners #[derive(Debug)] #[must_use = "streams do nothing unless polled"] -pub struct Incoming { - inner: UnixListener, +pub struct Incoming<'a> { + inner: &'a mut UnixListener, } -impl Incoming { - pub(crate) fn new(listener: UnixListener) -> Incoming { +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)) + } } -impl Stream for Incoming { +#[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!(Pin::new(&mut self.inner).poll_accept(cx))?; + 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 3cf8eff3235..c7de68b1338 100644 --- a/tokio/src/net/unix/listener.rs +++ b/tokio/src/net/unix/listener.rs @@ -1,8 +1,7 @@ -use crate::net::unix::UnixStream; +use crate::future::poll_fn; +use crate::net::unix::{Incoming, 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; @@ -90,8 +89,8 @@ impl UnixListener { /// /// 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) + pub fn incoming(&mut self) -> Incoming<'_> { + Incoming::new(self) } } 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..6e62c519d28 100644 --- a/tokio/src/prelude.rs +++ b/tokio/src/prelude.rs @@ -11,22 +11,9 @@ //! //! 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)] -pub use crate::io::{AsyncBufReadExt as _, AsyncReadExt as _, AsyncWriteExt as _}; + +cfg_io_util! { + #[doc(no_inline)] + pub use crate::io::{AsyncBufReadExt as _, AsyncReadExt as _, AsyncWriteExt as _}; +} diff --git a/tokio/src/process/mod.rs b/tokio/src/process/mod.rs index e777da52502..ee43a9fb067 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(()) @@ -117,11 +116,9 @@ mod imp; mod kill; -use crate::io::{AsyncRead, AsyncReadExt, AsyncWrite}; +use crate::io::{AsyncRead, 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,10 +766,12 @@ 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 { - AsyncReadExt::read_to_end(&mut io, &mut vec).await?; + crate::io::util::read_to_end(&mut io, &mut vec).await?; } Ok(vec) } @@ -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/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/runtime/current_thread/mod.rs b/tokio/src/runtime/basic_scheduler.rs similarity index 83% rename from tokio/src/runtime/current_thread/mod.rs rename to tokio/src/runtime/basic_scheduler.rs index c926922037a..abcb6a20eb8 100644 --- a/tokio/src/runtime/current_thread/mod.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -12,24 +12,24 @@ use std::time::Duration; /// Executes tasks on the current thread #[derive(Debug)] -pub(crate) struct CurrentThread

+pub(crate) struct BasicScheduler

where P: Park, { /// Scheduler component - scheduler: Arc, + scheduler: Arc, /// Local state - local: Local

, + local: LocalState

, } #[derive(Debug, Clone)] pub(crate) struct Spawner { - scheduler: Arc, + scheduler: Arc, } /// The scheduler component. -pub(super) struct Scheduler { +pub(super) struct SchedulerPriv { /// List of all active tasks spawned onto this executor. /// /// # Safety @@ -45,7 +45,7 @@ pub(super) struct Scheduler { /// /// References should not be handed out. Only call `push` / `pop` functions. /// Only call from the owning thread. - local_queue: UnsafeCell>>, + local_queue: UnsafeCell>>, /// Remote run queue. /// @@ -59,12 +59,12 @@ pub(super) struct Scheduler { unpark: Box, } -unsafe impl Send for Scheduler {} -unsafe impl Sync for Scheduler {} +unsafe impl Send for SchedulerPriv {} +unsafe impl Sync for SchedulerPriv {} /// Local state #[derive(Debug)] -struct Local

{ +struct LocalState

{ /// Current tick tick: u8, @@ -75,7 +75,7 @@ struct Local

{ #[derive(Debug)] struct RemoteQueue { /// FIFO list of tasks - queue: VecDeque>, + queue: VecDeque>, /// `true` when a task can be pushed into the queue, false otherwise. open: bool, @@ -87,15 +87,15 @@ const MAX_TASKS_PER_TICK: usize = 61; /// How often to check the remote queue first const CHECK_REMOTE_INTERVAL: u8 = 13; -impl

CurrentThread

+impl

BasicScheduler

where P: Park, { - pub(crate) fn new(park: P) -> CurrentThread

{ + pub(crate) fn new(park: P) -> BasicScheduler

{ let unpark = park.unpark(); - CurrentThread { - scheduler: Arc::new(Scheduler { + BasicScheduler { + scheduler: Arc::new(SchedulerPriv { owned_tasks: UnsafeCell::new(task::OwnedList::new()), local_queue: UnsafeCell::new(VecDeque::with_capacity(64)), remote_queue: Mutex::new(RemoteQueue { @@ -105,7 +105,7 @@ where pending_drop: task::TransferStack::new(), unpark: Box::new(unpark), }), - local: Local { tick: 0, park }, + local: LocalState { tick: 0, park }, } } @@ -138,11 +138,11 @@ where let local = &mut self.local; let scheduler = &*self.scheduler; - runtime::global::with_current_thread(scheduler, || { + runtime::global::with_basic_scheduler(scheduler, || { let mut _enter = runtime::enter(); let raw_waker = RawWaker::new( - scheduler as *const Scheduler as *const (), + scheduler as *const SchedulerPriv as *const (), &RawWakerVTable::new(sched_clone_waker, sched_noop, sched_wake_by_ref, sched_noop), ); @@ -179,10 +179,19 @@ impl Spawner { self.scheduler.schedule(task); handle } + + /// Enter the executor context + pub(crate) fn enter(&self, f: F) -> R + where + F: FnOnce() -> R, + { + use crate::runtime::global; + global::with_basic_scheduler(&*self.scheduler, f) + } } -impl Scheduler { - fn tick(&self, local: &mut Local) { +impl SchedulerPriv { + fn tick(&self, local: &mut LocalState) { for _ in 0..MAX_TASKS_PER_TICK { // Get the current tick let tick = local.tick; @@ -223,14 +232,16 @@ impl Scheduler { /// # Safety /// - /// Must be called from the same thread that holds the `CurrentThread` + /// Must be called from the same thread that holds the `BasicScheduler` /// value. - pub(super) unsafe fn spawn_background(&self, future: F) + pub(super) unsafe fn spawn(&self, future: F) -> JoinHandle where - F: Future + Send + 'static, + F: Future + Send + 'static, + F::Output: Send + 'static, { - let task = task::background(future); + let (task, handle) = task::joinable(future); self.schedule_local(task); + handle } unsafe fn schedule_local(&self, task: Task) { @@ -254,7 +265,7 @@ impl Scheduler { } } -impl Schedule for Scheduler { +impl Schedule for SchedulerPriv { fn bind(&self, task: &Task) { unsafe { (*self.owned_tasks.get()).insert(task); @@ -274,7 +285,7 @@ impl Schedule for Scheduler { fn schedule(&self, task: Task) { use crate::runtime::global; - if global::current_thread_is_current(self) { + if global::basic_scheduler_is_current(self) { unsafe { self.schedule_local(task) }; } else { let mut lock = self.remote_queue.lock().unwrap(); @@ -293,9 +304,9 @@ impl Schedule for Scheduler { } } -impl ScheduleSendOnly for Scheduler {} +impl ScheduleSendOnly for SchedulerPriv {} -impl

Drop for CurrentThread

+impl

Drop for BasicScheduler

where P: Park, { @@ -330,36 +341,36 @@ where } } -impl fmt::Debug for Scheduler { +impl fmt::Debug for SchedulerPriv { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Scheduler").finish() } } unsafe fn sched_clone_waker(ptr: *const ()) -> RawWaker { - let s1 = ManuallyDrop::new(Arc::from_raw(ptr as *const Scheduler)); + let s1 = ManuallyDrop::new(Arc::from_raw(ptr as *const SchedulerPriv)); #[allow(clippy::redundant_clone)] let s2 = s1.clone(); RawWaker::new( - &**s2 as *const Scheduler as *const (), + &**s2 as *const SchedulerPriv as *const (), &RawWakerVTable::new(sched_clone_waker, sched_wake, sched_wake_by_ref, sched_drop), ) } unsafe fn sched_wake(ptr: *const ()) { - let scheduler = Arc::from_raw(ptr as *const Scheduler); + let scheduler = Arc::from_raw(ptr as *const SchedulerPriv); scheduler.unpark.unpark(); } unsafe fn sched_wake_by_ref(ptr: *const ()) { - let scheduler = ManuallyDrop::new(Arc::from_raw(ptr as *const Scheduler)); + let scheduler = ManuallyDrop::new(Arc::from_raw(ptr as *const SchedulerPriv)); scheduler.unpark.unpark(); } unsafe fn sched_drop(ptr: *const ()) { - let _ = Arc::from_raw(ptr as *const Scheduler); + let _ = Arc::from_raw(ptr as *const SchedulerPriv); } unsafe fn sched_noop(_ptr: *const ()) { diff --git a/tokio/src/runtime/blocking.rs b/tokio/src/runtime/blocking.rs index 39d103bd5c1..8408e78b7ee 100644 --- a/tokio/src/runtime/blocking.rs +++ b/tokio/src/runtime/blocking.rs @@ -3,10 +3,7 @@ //! shells. This isolates the complexity of dealing with conditional //! compilation. -pub(crate) use self::variant::*; - -#[cfg(feature = "blocking")] -mod variant { +cfg_blocking_impl! { pub(crate) use crate::blocking::BlockingPool; pub(crate) use crate::blocking::Spawner; @@ -17,8 +14,7 @@ mod variant { } } -#[cfg(not(feature = "blocking"))] -mod variant { +cfg_not_blocking_impl! { use crate::runtime::Builder; #[derive(Debug, Clone)] diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index 66c9e166c31..793146f700c 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -61,8 +61,8 @@ pub struct Builder { enum Kind { Shell, #[cfg(feature = "rt-core")] - CurrentThread, - #[cfg(feature = "rt-full")] + Basic, + #[cfg(feature = "rt-threaded")] ThreadPool, } @@ -121,19 +121,19 @@ impl Builder { self } - /// Use only the current thread for executing tasks. + /// Use a simpler scheduler that runs all tasks on the current-thread. /// /// The executor and all necessary drivers will all be run on the current /// thread during `block_on` calls. #[cfg(feature = "rt-core")] - pub fn current_thread(&mut self) -> &mut Self { - self.kind = Kind::CurrentThread; + pub fn basic_scheduler(&mut self) -> &mut Self { + self.kind = Kind::Basic; self } - /// Use a thread-pool for executing tasks. - #[cfg(feature = "rt-full")] - pub fn thread_pool(&mut self) -> &mut Self { + /// Use a multi-threaded scheduler for executing tasks. + #[cfg(feature = "rt-threaded")] + pub fn threaded_scheduler(&mut self) -> &mut Self { self.kind = Kind::ThreadPool; self } @@ -252,15 +252,15 @@ impl Builder { /// ``` pub fn build(&mut self) -> io::Result { match self.kind { - Kind::Shell => self.build_shell(), + Kind::Shell => self.build_shell_runtime(), #[cfg(feature = "rt-core")] - Kind::CurrentThread => self.build_current_thread(), - #[cfg(feature = "rt-full")] - Kind::ThreadPool => self.build_threadpool(), + Kind::Basic => self.build_basic_runtime(), + #[cfg(feature = "rt-threaded")] + Kind::ThreadPool => self.build_threaded_runtime(), } } - fn build_shell(&mut self) -> io::Result { + fn build_shell_runtime(&mut self) -> io::Result { use crate::runtime::Kind; let clock = time::create_clock(); @@ -287,121 +287,127 @@ impl Builder { blocking_pool, }) } +} - #[cfg(feature = "rt-core")] - fn build_current_thread(&mut self) -> io::Result { - use crate::runtime::{CurrentThread, Kind}; - - let clock = time::create_clock(); - - // Create I/O driver - let (io_driver, handle) = io::create_driver()?; - let io_handles = vec![handle]; - - let (driver, handle) = time::create_driver(io_driver, clock.clone()); - let time_handles = vec![handle]; - - // And now put a single-threaded scheduler on top of the timer. When - // there are no futures ready to do something, it'll let the timer or - // the reactor to generate some new stimuli for the futures to continue - // in their life. - let scheduler = CurrentThread::new(driver); - let spawner = scheduler.spawner(); - - // Blocking pool - let blocking_pool = blocking::create_blocking_pool(self); - let blocking_spawner = blocking_pool.spawner().clone(); - - Ok(Runtime { - kind: Kind::CurrentThread(scheduler), - handle: Handle { - kind: handle::Kind::CurrentThread(spawner), - io_handles, - time_handles, - clock, - blocking_spawner, - }, - blocking_pool, - }) - } - - #[cfg(feature = "rt-full")] - fn build_threadpool(&mut self) -> io::Result { - use crate::runtime::{Kind, ThreadPool}; - use std::sync::Mutex; - - let clock = time::create_clock(); +cfg_rt_core! { + impl Builder { + fn build_basic_runtime(&mut self) -> io::Result { + use crate::runtime::{BasicScheduler, Kind}; - let mut io_handles = Vec::new(); - let mut time_handles = Vec::new(); - let mut drivers = Vec::new(); + let clock = time::create_clock(); - for _ in 0..self.num_threads { - // Create I/O driver and handle + // Create I/O driver let (io_driver, handle) = io::create_driver()?; - io_handles.push(handle); - - // Create a new timer. - let (time_driver, handle) = time::create_driver(io_driver, clock.clone()); - time_handles.push(handle); - drivers.push(Mutex::new(Some(time_driver))); + let io_handles = vec![handle]; + + let (driver, handle) = time::create_driver(io_driver, clock.clone()); + let time_handles = vec![handle]; + + // And now put a single-threaded scheduler on top of the timer. When + // there are no futures ready to do something, it'll let the timer or + // the reactor to generate some new stimuli for the futures to continue + // in their life. + let scheduler = BasicScheduler::new(driver); + let spawner = scheduler.spawner(); + + // Blocking pool + let blocking_pool = blocking::create_blocking_pool(self); + let blocking_spawner = blocking_pool.spawner().clone(); + + Ok(Runtime { + kind: Kind::Basic(scheduler), + handle: Handle { + kind: handle::Kind::Basic(spawner), + io_handles, + time_handles, + clock, + blocking_spawner, + }, + blocking_pool, + }) } + } +} - // Create the blocking pool - let blocking_pool = blocking::create_blocking_pool(self); - let blocking_spawner = blocking_pool.spawner().clone(); - - let scheduler = { - let clock = clock.clone(); - let io_handles = io_handles.clone(); - let time_handles = time_handles.clone(); - - let after_start = self.after_start.clone(); - let before_stop = self.before_stop.clone(); - - let around_worker = Arc::new(Box::new(move |index, next: &mut dyn FnMut()| { - // Configure the I/O driver - let _io = io::set_default(&io_handles[index]); - - // Configure time - time::with_default(&time_handles[index], &clock, || { - // Call the start callback - if let Some(after_start) = after_start.as_ref() { - after_start(); - } - - // Run the worker - next(); - - // Call the after call back - if let Some(before_stop) = before_stop.as_ref() { - before_stop(); - } +cfg_rt_threaded! { + impl Builder { + fn build_threaded_runtime(&mut self) -> io::Result { + use crate::runtime::{Kind, ThreadPool}; + use std::sync::Mutex; + + let clock = time::create_clock(); + + let mut io_handles = Vec::new(); + let mut time_handles = Vec::new(); + let mut drivers = Vec::new(); + + for _ in 0..self.num_threads { + // Create I/O driver and handle + let (io_driver, handle) = io::create_driver()?; + io_handles.push(handle); + + // Create a new timer. + let (time_driver, handle) = time::create_driver(io_driver, clock.clone()); + time_handles.push(handle); + drivers.push(Mutex::new(Some(time_driver))); + } + + // Create the blocking pool + let blocking_pool = blocking::create_blocking_pool(self); + let blocking_spawner = blocking_pool.spawner().clone(); + + let scheduler = { + let clock = clock.clone(); + let io_handles = io_handles.clone(); + let time_handles = time_handles.clone(); + + let after_start = self.after_start.clone(); + let before_stop = self.before_stop.clone(); + + let around_worker = Arc::new(Box::new(move |index, next: &mut dyn FnMut()| { + // Configure the I/O driver + let _io = io::set_default(&io_handles[index]); + + // Configure time + time::with_default(&time_handles[index], &clock, || { + // Call the start callback + if let Some(after_start) = after_start.as_ref() { + after_start(); + } + + // Run the worker + next(); + + // Call the after call back + if let Some(before_stop) = before_stop.as_ref() { + before_stop(); + } + }) }) + as Box); + + ThreadPool::new( + self.num_threads, + blocking_pool.spawner().clone(), + around_worker, + move |index| drivers[index].lock().unwrap().take().unwrap(), + ) + }; + + let spawner = scheduler.spawner().clone(); + + Ok(Runtime { + kind: Kind::ThreadPool(scheduler), + handle: Handle { + kind: handle::Kind::ThreadPool(spawner), + io_handles, + time_handles, + clock, + blocking_spawner, + }, + blocking_pool, }) - as Box); - - ThreadPool::new( - self.num_threads, - blocking_pool.spawner().clone(), - around_worker, - move |index| drivers[index].lock().unwrap().take().unwrap(), - ) - }; - - let spawner = scheduler.spawner().clone(); - - Ok(Runtime { - kind: Kind::ThreadPool(scheduler), - handle: Handle { - kind: handle::Kind::ThreadPool(spawner), - io_handles, - time_handles, - clock, - blocking_spawner, - }, - blocking_pool, - }) + } } } diff --git a/tokio/src/runtime/enter.rs b/tokio/src/runtime/enter.rs index 9caea0cf922..a0925cb494d 100644 --- a/tokio/src/runtime/enter.rs +++ b/tokio/src/runtime/enter.rs @@ -1,7 +1,5 @@ use std::cell::{Cell, RefCell}; use std::fmt; -#[cfg(feature = "rt-full")] -use std::future::Future; use std::marker::PhantomData; thread_local!(static ENTERED: Cell = Cell::new(false)); @@ -48,58 +46,62 @@ pub(crate) fn try_enter() -> Option { // // This is hidden for a reason. Do not use without fully understanding // executors. Misuing can easily cause your program to deadlock. -#[cfg(feature = "rt-full")] -pub(crate) fn exit R, R>(f: F) -> R { - // Reset in case the closure panics - struct Reset; - impl Drop for Reset { - fn drop(&mut self) { - ENTERED.with(|c| { - c.set(true); - }); +cfg_rt_threaded! { + #[cfg(feature = "blocking")] + pub(crate) fn exit R, R>(f: F) -> R { + // Reset in case the closure panics + struct Reset; + impl Drop for Reset { + fn drop(&mut self) { + ENTERED.with(|c| { + c.set(true); + }); + } } - } - ENTERED.with(|c| { - debug_assert!(c.get()); - c.set(false); - }); + ENTERED.with(|c| { + debug_assert!(c.get()); + c.set(false); + }); - let reset = Reset; - let ret = f(); - ::std::mem::forget(reset); + let reset = Reset; + let ret = f(); + ::std::mem::forget(reset); - ENTERED.with(|c| { - assert!(!c.get(), "closure claimed permanent executor"); - c.set(true); - }); + ENTERED.with(|c| { + assert!(!c.get(), "closure claimed permanent executor"); + c.set(true); + }); - ret -} + ret + } -impl Enter { - /// Blocks the thread on the specified future, returning the value with - /// which that future completes. - #[cfg(feature = "rt-full")] - pub(crate) fn block_on(&mut self, mut f: F) -> F::Output { - use crate::runtime::park::{CachedParkThread, Park}; - use std::pin::Pin; - use std::task::Context; - use std::task::Poll::Ready; - - let mut park = CachedParkThread::new(); - let waker = park.unpark().into_waker(); - let mut cx = Context::from_waker(&waker); - - // `block_on` takes ownership of `f`. Once it is pinned here, the original `f` binding can - // no longer be accessed, making the pinning safe. - let mut f = unsafe { Pin::new_unchecked(&mut f) }; - - loop { - if let Ready(v) = f.as_mut().poll(&mut cx) { - return v; + impl Enter { + /// Blocks the thread on the specified future, returning the value with + /// which that future completes. + pub(crate) fn block_on(&mut self, mut f: F) -> F::Output + where + F: std::future::Future, + { + use crate::runtime::park::{CachedParkThread, Park}; + use std::pin::Pin; + use std::task::Context; + use std::task::Poll::Ready; + + let mut park = CachedParkThread::new(); + let waker = park.unpark().into_waker(); + let mut cx = Context::from_waker(&waker); + + // `block_on` takes ownership of `f`. Once it is pinned here, the original `f` binding can + // no longer be accessed, making the pinning safe. + let mut f = unsafe { Pin::new_unchecked(&mut f) }; + + loop { + if let Ready(v) = f.as_mut().poll(&mut cx) { + return v; + } + park.park().unwrap(); } - park.park().unwrap(); } } } diff --git a/tokio/src/runtime/global.rs b/tokio/src/runtime/global.rs index f1cb8d1b848..d8b7ed9b936 100644 --- a/tokio/src/runtime/global.rs +++ b/tokio/src/runtime/global.rs @@ -1,7 +1,5 @@ -use crate::runtime::current_thread; - -#[cfg(feature = "rt-full")] -use crate::runtime::thread_pool; +use crate::runtime::basic_scheduler; +use crate::task::JoinHandle; use std::cell::Cell; use std::future::Future; @@ -11,11 +9,11 @@ enum State { // default executor not defined Empty, - // Current-thread executor - CurrentThread(*const current_thread::Scheduler), + // Basic scheduler (runs on the current-thread) + Basic(*const basic_scheduler::SchedulerPriv), // default executor is a thread pool instance. - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] ThreadPool(*const thread_pool::Spawner), } @@ -27,64 +25,23 @@ thread_local! { // ===== global spawn fns ===== /// Spawns a future on the default executor. -/// -/// In order for a future to do work, it must be spawned on an executor. The -/// `spawn` function is the easiest way to do this. It spawns a future on the -/// [default executor] for the current execution context (tracked using a -/// thread-local variable). -/// -/// The default executor is **usually** a thread pool. -/// -/// # Examples -/// -/// In this example, a server is started and `spawn` is used to start a new task -/// that processes each received connection. -/// -/// ``` -/// use tokio::net::TcpListener; -/// -/// # async fn process(_t: T) {} -/// # async fn dox() -> Result<(), Box> { -/// let mut listener = TcpListener::bind("127.0.0.1:8080").await?; -/// -/// loop { -/// let (socket, _) = listener.accept().await?; -/// -/// tokio::spawn(async move { -/// // Process each socket concurrently. -/// process(socket).await -/// }); -/// } -/// # } -/// ``` -/// -/// [default executor]: struct.DefaultExecutor.html -/// -/// # Panics -/// -/// This function will panic if the default executor is not set or if spawning -/// onto the default executor returns an error. To avoid the panic, use -/// [`DefaultExecutor`]. -/// -/// [`DefaultExecutor`]: struct.DefaultExecutor.html -pub fn spawn(future: T) +pub(crate) fn spawn(future: T) -> JoinHandle where - T: Future + Send + 'static, + T: Future + Send + 'static, + T::Output: Send + 'static, { EXECUTOR.with(|current_executor| match current_executor.get() { - #[cfg(feature = "rt-full")] - State::ThreadPool(threadpool_ptr) => { - let thread_pool = unsafe { &*threadpool_ptr }; - thread_pool.spawn_background(future); + #[cfg(feature = "rt-threaded")] + State::ThreadPool(thread_pool_ptr) => { + let thread_pool = unsafe { &*thread_pool_ptr }; + thread_pool.spawn(future) } - State::CurrentThread(current_thread_ptr) => { - let current_thread = unsafe { &*current_thread_ptr }; + State::Basic(basic_scheduler_ptr) => { + let basic_scheduler = unsafe { &*basic_scheduler_ptr }; - // Safety: The `CurrentThread` value set the thread-local (same + // Safety: The `BasicScheduler` value set the thread-local (same // thread). - unsafe { - current_thread.spawn_background(future); - } + unsafe { basic_scheduler.spawn(future) } } State::Empty => { // Explicit drop of `future` silences the warning that `future` is @@ -95,29 +52,35 @@ where }) } -pub(super) fn with_current_thread(current_thread: ¤t_thread::Scheduler, f: F) -> R +pub(super) fn with_basic_scheduler( + basic_scheduler: &basic_scheduler::SchedulerPriv, + f: F, +) -> R where F: FnOnce() -> R, { with_state( - State::CurrentThread(current_thread as *const current_thread::Scheduler), + State::Basic(basic_scheduler as *const basic_scheduler::SchedulerPriv), f, ) } -pub(super) fn current_thread_is_current(current_thread: ¤t_thread::Scheduler) -> bool { +pub(super) fn basic_scheduler_is_current(basic_scheduler: &basic_scheduler::SchedulerPriv) -> bool { EXECUTOR.with(|current_executor| match current_executor.get() { - State::CurrentThread(ptr) => ptr == current_thread as *const _, + State::Basic(ptr) => ptr == basic_scheduler as *const _, _ => false, }) } -#[cfg(feature = "rt-full")] -pub(super) fn with_thread_pool(thread_pool: &thread_pool::Spawner, f: F) -> R -where - F: FnOnce() -> R, -{ - with_state(State::ThreadPool(thread_pool as *const _), f) +cfg_rt_threaded! { + use crate::runtime::thread_pool; + + pub(super) fn with_thread_pool(thread_pool: &thread_pool::Spawner, f: F) -> R + where + F: FnOnce() -> R, + { + with_state(State::ThreadPool(thread_pool as *const _), f) + } } fn with_state(state: State, f: F) -> R diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index c5f2b6715ca..f8119e65577 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -1,13 +1,15 @@ -#[cfg(feature = "rt-core")] -use crate::runtime::current_thread; -#[cfg(feature = "rt-full")] -use crate::runtime::thread_pool; use crate::runtime::{blocking, io, time}; -#[cfg(feature = "rt-core")] -use crate::task::JoinHandle; -#[cfg(feature = "rt-core")] -use std::future::Future; +cfg_rt_core! { + use crate::runtime::basic_scheduler; + use crate::task::JoinHandle; + + use std::future::Future; +} + +cfg_rt_threaded! { + use crate::runtime::thread_pool; +} /// Handle to the runtime #[derive(Debug, Clone)] @@ -30,57 +32,12 @@ pub struct Handle { pub(super) enum Kind { Shell, #[cfg(feature = "rt-core")] - CurrentThread(current_thread::Spawner), - #[cfg(feature = "rt-full")] + Basic(basic_scheduler::Spawner), + #[cfg(feature = "rt-threaded")] ThreadPool(thread_pool::Spawner), } impl Handle { - /// Spawn a future onto the Tokio runtime. - /// - /// This spawns the given future onto the runtime's executor, usually a - /// thread pool. The thread pool is then responsible for polling the future - /// until it completes. - /// - /// See [module level][mod] documentation for more details. - /// - /// [mod]: index.html - /// - /// # Examples - /// - /// ``` - /// use tokio::runtime::Runtime; - /// - /// # fn dox() { - /// // Create the runtime - /// let rt = Runtime::new().unwrap(); - /// let handle = rt.handle(); - /// - /// // Spawn a future onto the runtime - /// handle.spawn(async { - /// println!("now running on a worker thread"); - /// }); - /// # } - /// ``` - /// - /// # Panics - /// - /// This function panics if the spawn fails. Failure occurs if the executor - /// is currently at capacity and is unable to spawn a new future. - #[cfg(feature = "rt-core")] - pub fn spawn(&self, future: F) -> JoinHandle - where - F: Future + Send + 'static, - { - match &self.kind { - Kind::Shell => panic!("spawning not enabled for runtime"), - #[cfg(feature = "rt-core")] - Kind::CurrentThread(spawner) => spawner.spawn(future), - #[cfg(feature = "rt-full")] - Kind::ThreadPool(spawner) => spawner.spawn(future), - } - } - /// Enter the runtime context pub fn enter(&self, f: F) -> R where @@ -89,7 +46,62 @@ impl Handle { self.blocking_spawner.enter(|| { let _io = io::set_default(&self.io_handles[0]); - time::with_default(&self.time_handles[0], &self.clock, f) + time::with_default(&self.time_handles[0], &self.clock, || match &self.kind { + Kind::Shell => f(), + #[cfg(feature = "rt-core")] + Kind::Basic(spawner) => spawner.enter(f), + #[cfg(feature = "rt-threaded")] + Kind::ThreadPool(spawner) => spawner.enter(f), + }) }) } } + +cfg_rt_core! { + impl Handle { + /// Spawn a future onto the Tokio runtime. + /// + /// This spawns the given future onto the runtime's executor, usually a + /// thread pool. The thread pool is then responsible for polling the future + /// until it completes. + /// + /// See [module level][mod] documentation for more details. + /// + /// [mod]: index.html + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Runtime; + /// + /// # fn dox() { + /// // Create the runtime + /// let rt = Runtime::new().unwrap(); + /// let handle = rt.handle(); + /// + /// // Spawn a future onto the runtime + /// handle.spawn(async { + /// println!("now running on a worker thread"); + /// }); + /// # } + /// ``` + /// + /// # Panics + /// + /// This function panics if the spawn fails. Failure occurs if the executor + /// is currently at capacity and is unable to spawn a new future. + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + match &self.kind { + Kind::Shell => panic!("spawning not enabled for runtime"), + #[cfg(feature = "rt-core")] + Kind::Basic(spawner) => spawner.spawn(future), + #[cfg(feature = "rt-threaded")] + Kind::ThreadPool(spawner) => spawner.spawn(future), + } + } + } +} diff --git a/tokio/src/runtime/io.rs b/tokio/src/runtime/io.rs index 35009a67f31..564e51a6520 100644 --- a/tokio/src/runtime/io.rs +++ b/tokio/src/runtime/io.rs @@ -3,13 +3,10 @@ //! shells. This isolates the complexity of dealing with conditional //! compilation. -pub(crate) use self::variant::*; - /// Re-exported for convenience. pub(crate) use std::io::Result; -#[cfg(feature = "io-driver")] -mod variant { +cfg_io_driver! { use crate::net::driver; use std::io; @@ -38,8 +35,7 @@ mod variant { } } -#[cfg(not(feature = "io-driver"))] -mod variant { +cfg_not_io_driver! { use crate::runtime::park::ParkThread; use std::io; diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 615c7074981..a022f955f7e 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -132,24 +132,24 @@ #[macro_use] mod tests; +cfg_rt_core! { + mod basic_scheduler; + use basic_scheduler::BasicScheduler; +} + mod blocking; use blocking::BlockingPool; mod builder; pub use self::builder::Builder; -#[cfg(feature = "rt-core")] -mod current_thread; -#[cfg(feature = "rt-core")] -use self::current_thread::CurrentThread; - pub(crate) mod enter; use self::enter::enter; -#[cfg(feature = "rt-core")] -mod global; -#[cfg(feature = "rt-core")] -pub use self::global::spawn; +cfg_rt_core! { + mod global; + pub(crate) use global::spawn; +} mod handle; pub use self::handle::Handle; @@ -164,13 +164,14 @@ use self::shell::Shell; mod time; -#[cfg(feature = "rt-full")] -pub(crate) mod thread_pool; -#[cfg(feature = "rt-full")] -use self::thread_pool::ThreadPool; +cfg_rt_threaded! { + pub(crate) mod thread_pool; + use self::thread_pool::ThreadPool; +} -#[cfg(feature = "rt-core")] -use crate::task::JoinHandle; +cfg_rt_core! { + use crate::task::JoinHandle; +} use std::future::Future; @@ -220,10 +221,10 @@ enum Kind { /// Execute all tasks on the current-thread. #[cfg(feature = "rt-core")] - CurrentThread(CurrentThread), + Basic(BasicScheduler), /// Execute tasks across multiple threads. - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] ThreadPool(ThreadPool), } @@ -254,11 +255,11 @@ impl Runtime { /// /// [mod]: index.html pub fn new() -> io::Result { - #[cfg(feature = "rt-full")] - let ret = Builder::new().thread_pool().build(); + #[cfg(feature = "rt-threaded")] + let ret = Builder::new().threaded_scheduler().build(); - #[cfg(all(not(feature = "rt-full"), feature = "rt-core"))] - let ret = Builder::new().current_thread().build(); + #[cfg(all(not(feature = "rt-threaded"), feature = "rt-core"))] + let ret = Builder::new().basic_scheduler().build(); #[cfg(not(feature = "rt-core"))] let ret = Builder::new().build(); @@ -299,13 +300,14 @@ impl Runtime { #[cfg(feature = "rt-core")] pub fn spawn(&self, future: F) -> JoinHandle where - F: Future + Send + 'static, + F: Future + Send + 'static, + F::Output: Send + 'static, { match &self.kind { Kind::Shell(_) => panic!("task execution disabled"), - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] Kind::ThreadPool(exec) => exec.spawn(future), - Kind::CurrentThread(exec) => exec.spawn(future), + Kind::Basic(exec) => exec.spawn(future), } } @@ -328,12 +330,20 @@ impl Runtime { self.handle.enter(|| match kind { Kind::Shell(exec) => exec.block_on(future), #[cfg(feature = "rt-core")] - Kind::CurrentThread(exec) => exec.block_on(future), - #[cfg(feature = "rt-full")] + Kind::Basic(exec) => exec.block_on(future), + #[cfg(feature = "rt-threaded")] Kind::ThreadPool(exec) => exec.block_on(future), }) } + /// 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/runtime/park/mod.rs b/tokio/src/runtime/park/mod.rs index 1221283235c..bae96d9dd3b 100644 --- a/tokio/src/runtime/park/mod.rs +++ b/tokio/src/runtime/park/mod.rs @@ -45,7 +45,7 @@ //! [mio]: https://docs.rs/mio/0.6/mio/struct.Poll.html mod thread; -#[cfg(feature = "rt-full")] +#[cfg(feature = "rt-threaded")] pub(crate) use self::thread::CachedParkThread; #[cfg(not(feature = "io-driver"))] pub(crate) use self::thread::ParkThread; diff --git a/tokio/src/runtime/park/thread.rs b/tokio/src/runtime/park/thread.rs index 7d9bf46519a..71bd5b92af5 100644 --- a/tokio/src/runtime/park/thread.rs +++ b/tokio/src/runtime/park/thread.rs @@ -168,7 +168,7 @@ impl CachedParkThread { /// /// This type cannot be moved to other threads, so it should be created on /// the thread that the caller intends to park. - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] pub(crate) fn new() -> CachedParkThread { CachedParkThread { _anchor: PhantomData, @@ -217,7 +217,7 @@ impl Unpark for UnparkThread { } } -#[cfg(feature = "rt-full")] +#[cfg(feature = "rt-threaded")] mod waker { use super::{Inner, UnparkThread}; use crate::loom::sync::Arc; diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 599ce548026..4b23c3b90ed 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -22,7 +22,9 @@ mod shutdown; mod worker; -pub(crate) use worker::block_in_place; +cfg_blocking! { + pub(crate) use worker::block_in_place; +} /// Unit tests #[cfg(test)] @@ -117,7 +119,7 @@ impl ThreadPool { where F: Future, { - crate::runtime::global::with_thread_pool(self.spawner(), || { + self.spawner.enter(|| { let mut enter = crate::runtime::enter(); enter.block_on(future) }) diff --git a/tokio/src/runtime/thread_pool/slice.rs b/tokio/src/runtime/thread_pool/slice.rs index 1a0bd381779..4b3ef996104 100644 --- a/tokio/src/runtime/thread_pool/slice.rs +++ b/tokio/src/runtime/thread_pool/slice.rs @@ -95,15 +95,6 @@ where } } - pub(crate) fn spawn_background(&self, future: F) - where - F: Future + Send + 'static, - F::Output: Send + 'static, - { - let task = task::background(future); - self.schedule(task); - } - pub(crate) fn schedule(&self, task: Task>) { current::get(|current_worker| match current_worker.as_member(self) { Some(worker) => { diff --git a/tokio/src/runtime/thread_pool/spawner.rs b/tokio/src/runtime/thread_pool/spawner.rs index b7031c43042..4773ea9ada6 100644 --- a/tokio/src/runtime/thread_pool/spawner.rs +++ b/tokio/src/runtime/thread_pool/spawner.rs @@ -37,12 +37,12 @@ impl Spawner { self.workers.spawn_typed(future) } - /// Spawn a task in the background - pub(crate) fn spawn_background(&self, future: F) + /// Enter the executor context + pub(crate) fn enter(&self, f: F) -> R where - F: Future + Send + 'static, + F: FnOnce() -> R, { - self.workers.spawn_background(future); + crate::runtime::global::with_thread_pool(self, f) } /// Reference to the worker set. Used by `ThreadPool` to initiate shutdown. diff --git a/tokio/src/runtime/thread_pool/tests/loom_pool.rs b/tokio/src/runtime/thread_pool/tests/loom_pool.rs index 065d515e590..b982e24ec6d 100644 --- a/tokio/src/runtime/thread_pool/tests/loom_pool.rs +++ b/tokio/src/runtime/thread_pool/tests/loom_pool.rs @@ -50,7 +50,7 @@ fn only_blocking() { let (block_tx, block_rx) = oneshot::channel(); pool.spawn(async move { - crate::blocking::in_place(move || { + crate::task::block_in_place(move || { block_tx.send(()); }) }); @@ -72,7 +72,7 @@ fn blocking_and_regular() { let done_tx = Arc::new(Mutex::new(Some(done_tx))); pool.spawn(async move { - crate::blocking::in_place(move || { + crate::task::block_in_place(move || { block_tx.send(()); }) }); @@ -150,18 +150,22 @@ fn pool_shutdown() { #[test] fn complete_block_on_under_load() { + use futures::FutureExt; + loom::model(|| { let pool = mk_pool(2); - pool.block_on(async { - // Spin hard - crate::spawn(async { - for _ in 0..2 { - yield_once().await; - } - }); + pool.block_on({ + futures::future::lazy(|_| ()).then(|_| { + // Spin hard + crate::spawn(async { + for _ in 0..2 { + yield_once().await; + } + }); - gated2(true).await + gated2(true) + }) }); }); } 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/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 2de2101e2c9..cf6b66d84b1 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -15,24 +15,26 @@ thread_local! { static ON_BLOCK: Cell> = Cell::new(None) } -pub(crate) fn block_in_place(f: F) -> R -where - F: FnOnce() -> R, -{ - // Make the current worker give away its Worker to another thread so that we can safely block - // this one without preventing progress on other futures the worker owns. - ON_BLOCK.with(|ob| { - let allow_blocking = ob - .get() - .expect("can only call blocking when on Tokio runtime"); - - // This is safe, because ON_BLOCK was set from an &mut dyn FnMut in the worker that wraps - // the worker's operation, and is unset just prior to when the FnMut is dropped. - let allow_blocking = unsafe { &*allow_blocking }; - - allow_blocking(); - f() - }) +cfg_blocking! { + pub(crate) fn block_in_place(f: F) -> R + where + F: FnOnce() -> R, + { + // Make the current worker give away its Worker to another thread so that we can safely block + // this one without preventing progress on other futures the worker owns. + ON_BLOCK.with(|ob| { + let allow_blocking = ob + .get() + .expect("can only call blocking when on Tokio runtime"); + + // This is safe, because ON_BLOCK was set from an &mut dyn FnMut in the worker that wraps + // the worker's operation, and is unset just prior to when the FnMut is dropped. + let allow_blocking = unsafe { &*allow_blocking }; + + allow_blocking(); + f() + }) + } } pub(crate) struct Worker { diff --git a/tokio/src/runtime/time.rs b/tokio/src/runtime/time.rs index c04c2c966ad..b72971f869d 100644 --- a/tokio/src/runtime/time.rs +++ b/tokio/src/runtime/time.rs @@ -3,10 +3,7 @@ //! shells. This isolates the complexity of dealing with conditional //! compilation. -pub(crate) use self::variant::*; - -#[cfg(feature = "time")] -mod variant { +cfg_time! { use crate::runtime::io; use crate::time::{self, driver}; @@ -35,8 +32,7 @@ mod variant { } } -#[cfg(not(feature = "time"))] -mod variant { +cfg_not_time! { use crate::runtime::io; pub(crate) type Clock = (); diff --git a/tokio/src/signal/ctrl_c.rs b/tokio/src/signal/ctrl_c.rs index f9dd467924c..35ef2393568 100644 --- a/tokio/src/signal/ctrl_c.rs +++ b/tokio/src/signal/ctrl_c.rs @@ -1,46 +1,35 @@ #[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 -/// 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`. /// -/// 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. +/// # Examples /// -/// 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) - } +/// ```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(()) } 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..e4bfe758f75 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(_) => {} } } }); @@ -180,7 +178,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 +219,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 +274,7 @@ mod tests { }); let _ = fire.send(()); - let results: Vec<()> = third_rx.collect().await; + let results = collect(third_rx).await; assert_eq!(1, results.len()); }); @@ -309,6 +304,16 @@ mod tests { } fn rt() -> Runtime { - runtime::Builder::new().current_thread().build().unwrap() + runtime::Builder::new().basic_scheduler().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..8551e85cecd 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 { + #[doc(hidden)] // 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/signal/windows.rs b/tokio/src/signal/windows.rs index 96e585ba588..725518d3ea7 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::*; @@ -82,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()?; @@ -91,17 +93,10 @@ impl Event { Ok(Event { rx }) } -} - -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) + pub(crate) async fn recv(&mut self) -> Option<()> { + use crate::future::poll_fn; + poll_fn(|cx| self.rx.poll_recv(cx)).await } } @@ -109,6 +104,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 { @@ -153,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. /// @@ -161,29 +173,22 @@ 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::*; use crate::runtime::Runtime; + use tokio_test::{assert_ok, assert_pending, assert_ready_ok, task}; - use futures_util::stream::StreamExt; + 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().expect("failed to create CtrlC"); + 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 @@ -192,7 +197,7 @@ mod tests { super::handler(CTRL_C_EVENT); } - let _ = ctrl_c.into_future().await; + assert_ready_ok!(ctrl_c.poll()); }); } @@ -201,7 +206,7 @@ mod tests { let mut rt = rt(); rt.block_on(async { - let ctrl_break = super::ctrl_break().expect("failed to create CtrlC"); + 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 @@ -210,13 +215,13 @@ mod tests { super::handler(CTRL_BREAK_EVENT); } - let _ = ctrl_break.into_future().await; + ctrl_break.next().await.unwrap(); }); } fn rt() -> Runtime { crate::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() } 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/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/mod.rs b/tokio/src/sync/mod.rs index 25f58ffdd09..40566c0c0a6 100644 --- a/tokio/src/sync/mod.rs +++ b/tokio/src/sync/mod.rs @@ -13,43 +13,40 @@ //! - [watch](watch/index.html), a single-producer, multi-consumer channel that //! only stores the **most recently** sent value. -macro_rules! debug { - ($($t:tt)*) => { - if false { - println!($($t)*); - } - } -} +cfg_sync! { + mod barrier; + pub use barrier::{Barrier, BarrierWaitResult}; -macro_rules! if_loom { - ($($t:tt)*) => {{ - #[cfg(loom)] - const LOOM: bool = true; - #[cfg(not(loom))] - const LOOM: bool = false; - - if LOOM { - $($t)* - } - }} -} + pub mod mpsc; -mod barrier; -pub use barrier::{Barrier, BarrierWaitResult}; + mod mutex; + pub use mutex::{Mutex, MutexGuard}; -pub mod mpsc; + pub mod oneshot; -mod mutex; -pub use mutex::{Mutex, MutexGuard}; + pub(crate) mod semaphore; -pub mod oneshot; + mod task; + pub(crate) use task::AtomicWaker; -pub mod semaphore; + pub mod watch; +} -mod task; -pub use task::AtomicWaker; +cfg_not_sync! { + cfg_atomic_waker! { + mod task; + pub(crate) use task::AtomicWaker; + } -pub mod watch; + cfg_rt_threaded! { + pub(crate) mod oneshot; + } + + cfg_signal! { + pub(crate) mod mpsc; + pub(crate) mod semaphore; + } +} /// Unit tests #[cfg(test)] diff --git a/tokio/src/sync/mpsc/bounded.rs b/tokio/src/sync/mpsc/bounded.rs index 787dd5077c4..d635e1380c8 100644 --- a/tokio/src/sync/mpsc/bounded.rs +++ b/tokio/src/sync/mpsc/bounded.rs @@ -1,8 +1,8 @@ use crate::sync::mpsc::chan; +use crate::sync::mpsc::error::{ClosedError, SendError, TrySendError}; use crate::sync::semaphore; use std::fmt; -use std::pin::Pin; use std::task::{Context, Poll}; /// Send values to the associated `Receiver`. @@ -44,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. /// @@ -161,12 +140,12 @@ 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 } - #[doc(hidden)] // TODO: remove + #[doc(hidden)] // TODO: document pub fn poll_recv(&mut self, cx: &mut Context<'_>) -> Poll> { self.chan.recv(cx) } @@ -180,11 +159,15 @@ impl Receiver { } } -impl futures_core::Stream for Receiver { - type Item = T; +impl Unpin for Receiver {} - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.get_mut().poll_recv(cx) +cfg_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) + } } } @@ -193,9 +176,9 @@ impl 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 @@ -233,105 +216,17 @@ impl Sender { /// } /// } /// ``` - pub async fn send(&mut self, value: T) -> Result<(), SendError> { - use futures_util::future::poll_fn; - - poll_fn(|cx| self.poll_ready(cx)).await?; - - self.try_send(value).map_err(|_| SendError(())) - } -} + pub async fn send(&mut self, value: T) -> Result<(), SendError> { + use crate::future::poll_fn; -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 { - 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 + if poll_fn(|cx| self.poll_ready(cx)).await.is_err() { + return Err(SendError(value)); } - } - /// 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 + match self.try_send(value) { + Ok(()) => Ok(()), + Err(TrySendError::Full(_)) => unreachable!(), + Err(TrySendError::Closed(value)) => 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, - }, - } - } -} - -// ===== 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..4030e380198 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 ()) } } @@ -281,12 +299,6 @@ where // second time here. try_recv!(); - debug!( - "recv; rx_closed = {:?}; is_idle = {:?}", - rx_fields.rx_closed, - self.inner.semaphore.is_idle() - ); - if rx_fields.rx_closed && self.inner.semaphore.is_idle() { Ready(None) } else { @@ -317,6 +329,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 +373,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 +403,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 +452,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/list.rs b/tokio/src/sync/mpsc/list.rs index eecc4da3339..dc9564032cc 100644 --- a/tokio/src/sync/mpsc/list.rs +++ b/tokio/src/sync/mpsc/list.rs @@ -169,7 +169,6 @@ impl Tx { } pub(crate) unsafe fn reclaim_block(&self, mut block: NonNull>) { - debug!("+ reclaim_block({:p})", block); // The block has been removed from the linked list and ownership // is reclaimed. // @@ -206,7 +205,6 @@ impl Tx { } if !reused { - debug!(" + block freed {:p}", block); let _ = Box::from_raw(block.as_ptr()); } } @@ -226,7 +224,6 @@ impl Rx { pub(crate) fn pop(&mut self, tx: &Tx) -> Option> { // Advance `head`, if needed if !self.try_advancing_head() { - debug!("+ !self.try_advancing_head() -> false"); return None; } @@ -276,8 +273,6 @@ impl Rx { } fn reclaim_blocks(&mut self, tx: &Tx) { - debug!("+ reclaim_blocks()"); - while self.free_head != self.head { unsafe { // Get a handle to the block that will be freed and update @@ -316,7 +311,6 @@ impl Rx { /// Effectively `Drop` all the blocks. Should only be called once, when /// the list is dropping. pub(super) unsafe fn free_blocks(&mut self) { - debug!("+ free_blocks()"); debug_assert_ne!(self.free_head, NonNull::dangling()); let mut cur = Some(self.free_head); @@ -331,7 +325,6 @@ impl Rx { while let Some(block) = cur { cur = block.as_ref().load_next(Relaxed); - debug!(" + free: block = {:p}", block); drop(Box::from_raw(block.as_ptr())); } } diff --git a/tokio/src/sync/mpsc/mod.rs b/tokio/src/sync/mpsc/mod.rs index 3b95b9544d1..60ae60cdef6 100644 --- a/tokio/src/sync/mpsc/mod.rs +++ b/tokio/src/sync/mpsc/mod.rs @@ -1,3 +1,5 @@ +#![cfg_attr(not(feature = "sync"), allow(dead_code, unreachable_pub))] + //! A multi-producer, single-consumer queue for sending values across //! asynchronous tasks. //! @@ -46,12 +48,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 4eb750efbe7..07a173c213d 100644 --- a/tokio/src/sync/mpsc/unbounded.rs +++ b/tokio/src/sync/mpsc/unbounded.rs @@ -1,11 +1,10 @@ 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}; -use std::pin::Pin; - /// Send values to the associated `UnboundedReceiver`. /// /// Instances are created by the @@ -47,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. /// @@ -86,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) } @@ -103,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); @@ -121,17 +108,17 @@ 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); /// } /// ``` 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,11 +132,12 @@ impl UnboundedReceiver { } } +#[cfg(feature = "stream")] 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) + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_recv(cx) } } @@ -159,72 +147,8 @@ impl UnboundedSender { } /// 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 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 { - 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/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..ed3801c8126 100644 --- a/tokio/src/sync/oneshot.rs +++ b/tokio/src/sync/oneshot.rs @@ -1,10 +1,11 @@ +#![cfg_attr(not(feature = "sync"), allow(dead_code, unreachable_pub))] + //! A channel for sending a single message between asynchronous tasks. 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 +226,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/semaphore.rs b/tokio/src/sync/semaphore.rs index b4a093f8549..dab73a099b8 100644 --- a/tokio/src/sync/semaphore.rs +++ b/tokio/src/sync/semaphore.rs @@ -1,3 +1,5 @@ +#![cfg_attr(not(feature = "sync"), allow(dead_code, unreachable_pub))] + //! Thread-safe, asynchronous counting semaphore. //! //! A `Semaphore` instance holds a set of permits. Permits are used to @@ -24,7 +26,7 @@ use std::task::{Context, Poll}; use std::usize; /// Futures-aware semaphore. -pub struct Semaphore { +pub(crate) struct Semaphore { /// Tracks both the waiter queue tail pointer and the number of remaining /// permits. state: AtomicUsize, @@ -51,18 +53,18 @@ pub struct Semaphore { /// is the user's responsibility to ensure that `Permit::release` is called /// before dropping the permit. #[derive(Debug)] -pub struct Permit { +pub(crate) struct Permit { waiter: Option>, state: PermitState, } /// Error returned by `Permit::poll_acquire`. #[derive(Debug)] -pub struct AcquireError(()); +pub(crate) struct AcquireError(()); /// Error returned by `Permit::try_acquire`. #[derive(Debug)] -pub struct TryAcquireError { +pub(crate) struct TryAcquireError { kind: ErrorKind, } @@ -150,7 +152,7 @@ impl Semaphore { /// # Panics /// /// Panics if `permits` is zero. - pub fn new(permits: usize) -> Semaphore { + pub(crate) fn new(permits: usize) -> Semaphore { let stub = Box::new(WaiterNode::new()); let ptr = NonNull::new(&*stub as *const _ as *mut _).unwrap(); @@ -168,7 +170,7 @@ impl Semaphore { } /// Returns the current number of available permits - pub fn available_permits(&self) -> usize { + pub(crate) fn available_permits(&self) -> usize { let curr = SemState::load(&self.state, Acquire); curr.available_permits() } @@ -181,8 +183,6 @@ impl Semaphore { // Load the current state let mut curr = SemState::load(&self.state, Acquire); - debug!(" + poll_permit; sem-state = {:?}", curr); - // Tracks a *mut WaiterNode representing an Arc clone. // // This avoids having to bump the ref count unless required. @@ -210,8 +210,6 @@ impl Semaphore { } if !next.acquire_permit(&self.stub) { - debug!(" + poll_permit -- no permits"); - debug_assert!(curr.waiter().is_some()); if maybe_strong.is_none() { @@ -223,10 +221,7 @@ impl Semaphore { waiter.register(cx); - debug!(" + poll_permit -- to_queued_waiting"); - if !waiter.to_queued_waiting() { - debug!(" + poll_permit; waiter already queued"); // The node is alrady queued, there is no further work // to do. return Pending; @@ -243,14 +238,11 @@ impl Semaphore { next.set_waiter(maybe_strong.unwrap()); } - debug!(" + poll_permit -- pre-CAS; next = {:?}", next); - debug_assert_ne!(curr.0, 0); debug_assert_ne!(next.0, 0); match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => { - debug!(" + poll_permit -- CAS ok"); match curr.waiter() { Some(prev_waiter) => { let waiter = maybe_strong.unwrap(); @@ -260,13 +252,9 @@ impl Semaphore { prev_waiter.as_ref().next.store(waiter.as_ptr(), Release); } - debug!(" + poll_permit -- waiter pushed"); - return Pending; } None => { - debug!(" + poll_permit -- permit acquired"); - undo_strong!(); return Ready(Ok(())); @@ -282,15 +270,11 @@ impl Semaphore { /// Close the semaphore. This prevents the semaphore from issuing new /// permits and notifies all pending waiters. - pub fn close(&self) { - debug!("+ Semaphore::close"); - + pub(crate) fn close(&self) { // Acquire the `rx_lock`, setting the "closed" flag on the lock. let prev = self.rx_lock.fetch_or(1, AcqRel); - debug!(" + close -- rx_lock.fetch_add(1)"); if prev != 0 { - debug!("+ close -- locked; prev = {}", prev); // Another thread has the lock and will be responsible for notifying // pending waiters. return; @@ -300,9 +284,7 @@ impl Semaphore { } /// Add `n` new permits to the semaphore. - pub fn add_permits(&self, n: usize) { - debug!(" + add_permits; n = {}", n); - + pub(crate) fn add_permits(&self, n: usize) { if n == 0 { return; } @@ -310,10 +292,8 @@ impl Semaphore { // TODO: Handle overflow. A panic is not sufficient, the process must // abort. let prev = self.rx_lock.fetch_add(n << 1, AcqRel); - debug!(" + add_permits; rx_lock.fetch_add(n << 1); n = {}", n); if prev != 0 { - debug!(" + add_permits -- locked; prev = {}", prev); // Another thread has the lock and will be responsible for notifying // pending waiters. return; @@ -324,11 +304,6 @@ impl Semaphore { fn add_permits_locked(&self, mut rem: usize, mut closed: bool) { while rem > 0 || closed { - debug!( - " + add_permits_locked -- iter; rem = {}; closed = {:?}", - rem, closed - ); - if closed { SemState::fetch_set_closed(&self.state, AcqRel); } @@ -340,28 +315,16 @@ impl Semaphore { let actual = if closed { let actual = self.rx_lock.fetch_sub(n | 1, AcqRel); - debug!( - " + add_permits_locked; rx_lock.fetch_sub(n | 1); n = {}; actual={}", - n, actual - ); - closed = false; actual } else { let actual = self.rx_lock.fetch_sub(n, AcqRel); - debug!( - " + add_permits_locked; rx_lock.fetch_sub(n); n = {}; actual={}", - n, actual - ); - closed = actual & 1 == 1; actual }; rem = (actual >> 1) - rem; } - - debug!(" + add_permits; done"); } /// Release a specific amount of permits to the semaphore @@ -377,11 +340,8 @@ impl Semaphore { } }; - debug!(" + release_n -- notify"); - if waiter.notify(closed) { n = n.saturating_sub(1); - debug!(" + release_n -- dec"); } } } @@ -392,8 +352,6 @@ impl Semaphore { /// there are no more waiters to pop, `rem` is used to set the available /// permits. fn pop(&self, rem: usize, closed: bool) -> Option> { - debug!(" + pop; rem = {}", rem); - 'outer: loop { unsafe { let mut head = self.head.with(|head| *head); @@ -402,8 +360,6 @@ impl Semaphore { let stub = self.stub(); if head == stub { - debug!(" + pop; head == stub"); - let next = match NonNull::new(next_ptr) { Some(next) => next, None => { @@ -429,7 +385,6 @@ impl Semaphore { loop { if curr.has_waiter(&self.stub) { // Inconsistent - debug!(" + pop; inconsistent 1"); thread::yield_now(); continue 'outer; } @@ -456,8 +411,6 @@ impl Semaphore { } }; - debug!(" + pop; got next waiter"); - self.head.with_mut(|head| *head = next); head = next; next_ptr = next.as_ref().next.load(Acquire); @@ -476,7 +429,6 @@ impl Semaphore { if tail != head { // Inconsistent - debug!(" + pop; inconsistent 2"); thread::yield_now(); continue 'outer; } @@ -492,7 +444,6 @@ impl Semaphore { } // Inconsistent state, loop - debug!(" + pop; inconsistent 3"); thread::yield_now(); } } @@ -549,16 +500,7 @@ impl Permit { /// Create a new `Permit`. /// /// The permit begins in the "unacquired" state. - /// - /// # Examples - /// - /// ``` - /// use tokio::sync::semaphore::Permit; - /// - /// let permit = Permit::new(); - /// assert!(!permit.is_acquired()); - /// ``` - pub fn new() -> Permit { + pub(crate) fn new() -> Permit { Permit { waiter: None, state: PermitState::Idle, @@ -566,13 +508,13 @@ impl Permit { } /// Returns true if the permit has been acquired - pub fn is_acquired(&self) -> bool { + pub(crate) fn is_acquired(&self) -> bool { self.state == PermitState::Acquired } /// Try to acquire the permit. If no permits are available, the current task /// is notified once a new permit becomes available. - pub fn poll_acquire( + pub(crate) fn poll_acquire( &mut self, cx: &mut Context<'_>, semaphore: &Semaphore, @@ -607,7 +549,7 @@ impl Permit { } /// Try to acquire the permit. - pub fn try_acquire(&mut self, semaphore: &Semaphore) -> Result<(), TryAcquireError> { + pub(crate) fn try_acquire(&mut self, semaphore: &Semaphore) -> Result<(), TryAcquireError> { match self.state { PermitState::Idle => {} PermitState::Waiting => { @@ -635,7 +577,7 @@ impl Permit { } /// Release a permit back to the semaphore - pub fn release(&mut self, semaphore: &Semaphore) { + pub(crate) fn release(&mut self, semaphore: &Semaphore) { if self.forget2() { semaphore.add_permits(1); } @@ -648,7 +590,7 @@ impl Permit { /// /// Repeatedly calling `forget` without associated calls to `add_permit` /// will result in the semaphore losing all permits. - pub fn forget(&mut self) { + pub(crate) fn forget(&mut self) { self.forget2(); } @@ -711,7 +653,7 @@ impl TryAcquireError { } /// Returns true if the error was caused by a closed semaphore. - pub fn is_closed(&self) -> bool { + pub(crate) fn is_closed(&self) -> bool { match self.kind { ErrorKind::Closed => true, _ => false, @@ -720,7 +662,7 @@ impl TryAcquireError { /// Returns true if the error was caused by calling `try_acquire` on a /// semaphore with no available permits. - pub fn is_no_permits(&self) -> bool { + pub(crate) fn is_no_permits(&self) -> bool { match self.kind { ErrorKind::NoPermits => true, _ => false, @@ -857,14 +799,10 @@ impl WaiterNode { match next.compare_exchange(&self.state, curr, AcqRel, Acquire) { Ok(_) => match curr { QueuedWaiting => { - debug!(" + notify -- task notified"); self.waker.wake(); return true; } - other => { - debug!(" + notify -- not notified; state = {:?}", other); - return false; - } + _ => return false, }, Err(actual) => curr = actual, } @@ -1021,7 +959,6 @@ impl SemState { /// Load the state from an AtomicUsize. fn load(cell: &AtomicUsize, ordering: Ordering) -> SemState { let value = cell.load(ordering); - debug!(" + SemState::load; value = {}", value); SemState(value) } @@ -1044,13 +981,6 @@ impl SemState { let res = cell.compare_exchange(prev.to_usize(), self.to_usize(), success, failure); - debug!( - " + SemState::compare_exchange; prev = {}; next = {}; result = {:?}", - prev.to_usize(), - self.to_usize(), - res - ); - res.map(SemState).map_err(SemState) } diff --git a/tokio/src/sync/task/atomic_waker.rs b/tokio/src/sync/task/atomic_waker.rs index 88d083d6f94..49a0ac04845 100644 --- a/tokio/src/sync/task/atomic_waker.rs +++ b/tokio/src/sync/task/atomic_waker.rs @@ -20,7 +20,7 @@ use std::task::Waker; /// /// A single `AtomicWaker` may be reused for any number of calls to `register` or /// `wake`. -pub struct AtomicWaker { +pub(crate) struct AtomicWaker { state: AtomicUsize, waker: CausalCell>, } @@ -132,7 +132,7 @@ const WAKING: usize = 0b10; impl AtomicWaker { /// Create an `AtomicWaker` - pub fn new() -> AtomicWaker { + pub(crate) fn new() -> AtomicWaker { AtomicWaker { state: AtomicUsize::new(WAITING), waker: CausalCell::new(None), @@ -142,7 +142,8 @@ impl AtomicWaker { /// Registers the current waker to be notified on calls to `wake`. /// /// This is the same as calling `register_task` with `task::current()`. - pub fn register(&self, waker: Waker) { + #[cfg(feature = "io-driver")] + pub(crate) fn register(&self, waker: Waker) { self.do_register(waker); } @@ -161,7 +162,7 @@ impl AtomicWaker { /// idea. Concurrent calls to `register` will attempt to register different /// tasks to be woken. One of the callers will win and have its task set, /// but there is no guarantee as to which caller will succeed. - pub fn register_by_ref(&self, waker: &Waker) { + pub(crate) fn register_by_ref(&self, waker: &Waker) { self.do_register(waker); } @@ -169,10 +170,8 @@ impl AtomicWaker { where W: WakerRef, { - debug!(" + register_task"); match self.state.compare_and_swap(WAITING, REGISTERING, Acquire) { WAITING => { - debug!(" + WAITING"); unsafe { // Locked acquired, update the waker cell self.waker.with_mut(|t| *t = Some(waker.into_waker())); @@ -213,7 +212,6 @@ impl AtomicWaker { } } WAKING => { - debug!(" + WAKING"); // Currently in the process of waking the task, i.e., // `wake` is currently being called on the old waker. // So, we call wake on the new waker. @@ -238,8 +236,7 @@ impl AtomicWaker { /// Wakes the task that last called `register`. /// /// If `register` has not been called yet, then this does nothing. - pub fn wake(&self) { - debug!(" + wake"); + pub(crate) fn wake(&self) { if let Some(waker) = self.take_waker() { waker.wake(); } @@ -247,25 +244,21 @@ impl AtomicWaker { /// Attempts to take the `Waker` value out of the `AtomicWaker` with the /// intention that the caller will wake the task later. - pub fn take_waker(&self) -> Option { - debug!(" + take_waker"); + pub(crate) fn take_waker(&self) -> Option { // AcqRel ordering is used in order to acquire the value of the `waker` // cell as well as to establish a `release` ordering with whatever // memory the `AtomicWaker` is associated with. match self.state.fetch_or(WAKING, AcqRel) { WAITING => { - debug!(" + WAITING"); // The waking lock has been acquired. let waker = unsafe { self.waker.with_mut(|t| (*t).take()) }; // Release the lock self.state.fetch_and(!WAKING, Release); - debug!(" + Done taking"); waker } state => { - debug!(" + state = {:?}", state); // There is a concurrent thread currently updating the // associated waker. // diff --git a/tokio/src/sync/task/mod.rs b/tokio/src/sync/task/mod.rs index 446ff712655..a6bc6ed06ea 100644 --- a/tokio/src/sync/task/mod.rs +++ b/tokio/src/sync/task/mod.rs @@ -1,4 +1,4 @@ //! Thread-safe task notification primitives. mod atomic_waker; -pub use self::atomic_waker::AtomicWaker; +pub(crate) use self::atomic_waker::AtomicWaker; diff --git a/tokio/tests/sync_atomic_waker.rs b/tokio/src/sync/tests/atomic_waker.rs similarity index 92% rename from tokio/tests/sync_atomic_waker.rs rename to tokio/src/sync/tests/atomic_waker.rs index 8e725526d7a..c832d62e9ae 100644 --- a/tokio/tests/sync_atomic_waker.rs +++ b/tokio/src/sync/tests/atomic_waker.rs @@ -1,6 +1,4 @@ -#![warn(rust_2018_idioms)] - -use tokio::sync::AtomicWaker; +use crate::sync::AtomicWaker; use tokio_test::task; use std::task::Waker; 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_list.rs b/tokio/src/sync/tests/loom_list.rs index 4f7746d58f0..4067f865ce4 100644 --- a/tokio/src/sync/tests/loom_list.rs +++ b/tokio/src/sync/tests/loom_list.rs @@ -21,17 +21,14 @@ fn smoke() { for i in 0..NUM_MSG { tx.push((th, i)); } - debug!(" + tx thread done"); }); } let mut next = vec![0; NUM_TX]; loop { - debug!(" + rx.pop()"); match rx.pop(&tx) { Some(Value((th, v))) => { - debug!(" + pop() -> Some(Value({}))", v); assert_eq!(v, next[th]); next[th] += 1; @@ -43,7 +40,6 @@ fn smoke() { panic!(); } None => { - debug!(" + pop() -> None"); thread::yield_now(); } } 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/tests/mod.rs b/tokio/src/sync/tests/mod.rs index 8e627cb84b1..06d18e9a969 100644 --- a/tokio/src/sync/tests/mod.rs +++ b/tokio/src/sync/tests/mod.rs @@ -1,7 +1,12 @@ -#![cfg(loom)] +cfg_not_loom! { + mod atomic_waker; + mod semaphore; +} -mod loom_atomic_waker; -mod loom_list; -mod loom_mpsc; -mod loom_oneshot; -mod loom_semaphore; +cfg_loom! { + mod loom_atomic_waker; + mod loom_list; + mod loom_mpsc; + mod loom_oneshot; + mod loom_semaphore; +} diff --git a/tokio/tests/sync_semaphore.rs b/tokio/src/sync/tests/semaphore.rs similarity index 97% rename from tokio/tests/sync_semaphore.rs rename to tokio/src/sync/tests/semaphore.rs index 93660532dfd..86dd7da5891 100644 --- a/tokio/tests/sync_semaphore.rs +++ b/tokio/src/sync/tests/semaphore.rs @@ -1,6 +1,4 @@ -#![warn(rust_2018_idioms)] - -use tokio::sync::semaphore::{Permit, Semaphore}; +use crate::sync::semaphore::{Permit, Semaphore}; use tokio_test::task; use tokio_test::{assert_pending, assert_ready_err, assert_ready_ok}; diff --git a/tokio/src/sync/watch.rs b/tokio/src/sync/watch.rs index 928c2c46410..36bab7bc604 100644 --- a/tokio/src/sync/watch.rs +++ b/tokio/src/sync/watch.rs @@ -8,11 +8,11 @@ //! //! [`channel`] returns a [`Sender`] / [`Receiver`] pair. These are //! the producer and sender halves of the channel. The channel is -//! created with an initial value. [`Receiver::get_ref`] will always +//! created with an initial value. [`Receiver::recv`] will always //! be ready upon creation and will yield either this initial value or //! the latest value that has been sent by `Sender`. //! -//! Calls to [`Receiver::get_ref`] will always yield the latest value. +//! Calls to [`Receiver::recv`] will always yield the latest value. //! //! # Examples //! @@ -49,22 +49,17 @@ //! [`Receiver`]: struct.Receiver.html //! [`channel`]: fn.channel.html //! [`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). /// @@ -182,7 +177,7 @@ const CLOSED: usize = 1; /// /// [`Sender`]: struct.Sender.html /// [`Receiver`]: struct.Receiver.html -pub fn channel(init: T) -> (Sender, Receiver) { +pub fn channel(init: T) -> (Sender, Receiver) { const INIT_ID: u64 = 0; let inner = Arc::new(WatchInner::new()); @@ -228,84 +223,57 @@ impl Receiver { /// use tokio::sync::watch; /// /// let (_, rx) = watch::channel("hello"); - /// assert_eq!(*rx.get_ref(), "hello"); + /// assert_eq!(*rx.borrow(), "hello"); /// ``` - pub fn get_ref(&self) -> Ref<'_, T> { + pub fn borrow(&self) -> Ref<'_, T> { let inner = self.shared.value.read().unwrap(); 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()) + poll_fn(|cx| { + let v_ref = ready!(self.poll_recv_ref(cx)); + 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(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - use std::future::Future; + fn poll_next(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let v_ref = ready!(self.poll_recv_ref(cx)); - let fut = self.get_mut().recv(); - pin_mut!(fut); - - let item = ready!(fut.poll(cx)); - Ready(item.map(|v_ref| v_ref)) + Poll::Ready(v_ref.map(|v_ref| (*v_ref).clone())) } } @@ -394,27 +362,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/blocking.rs b/tokio/src/task/blocking.rs new file mode 100644 index 00000000000..4445bb729c1 --- /dev/null +++ b/tokio/src/task/blocking.rs @@ -0,0 +1,65 @@ +use crate::blocking; +use crate::task::JoinHandle; + +cfg_rt_threaded! { + /// Run the provided blocking function without blocking the executor. + /// + /// In general, issuing a blocking call or performing a lot of compute in a + /// future without yielding is not okay, as it may prevent the executor from + /// driving other futures forward. If you run a closure through this method, + /// the current executor thread will relegate all its executor duties to another + /// (possibly new) thread, and only then poll the task. Note that this requires + /// additional synchronization. + /// + /// # Examples + /// + /// ``` + /// use tokio::task; + /// + /// # async fn docs() { + /// task::block_in_place(move || { + /// // do some compute-heavy work or call synchronous code + /// }); + /// # } + /// ``` + pub fn block_in_place(f: F) -> R + where + F: FnOnce() -> R, + { + use crate::runtime::{enter, thread_pool}; + + enter::exit(|| thread_pool::block_in_place(f)) + } +} + +cfg_blocking! { + /// Run the provided closure on a thread where blocking is acceptable. + /// + /// In general, issuing a blocking call or performing a lot of compute in a future without + /// yielding is not okay, as it may prevent the executor from driving other futures forward. + /// A closure that is run through this method will instead be run on a dedicated thread pool for + /// such blocking tasks without holding up the main futures executor. + /// + /// # Examples + /// + /// ``` + /// use tokio::task; + /// + /// # async fn docs() -> Result<(), Box>{ + /// let res = task::spawn_blocking(move || { + /// // do some compute-heavy work or call synchronous code + /// "done computing" + /// }).await?; + /// + /// assert_eq!(res, "done computing"); + /// # Ok(()) + /// # } + /// ``` + pub fn spawn_blocking(f: F) -> JoinHandle + where + F: FnOnce() -> R + Send + 'static, + R: Send + 'static, + { + blocking::spawn_blocking(f) + } +} diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 6f6af2da069..cc7cb7cc8c4 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -12,7 +12,7 @@ use std::rc::Rc; use std::sync::Mutex; use std::task::{Context, Poll}; -use pin_project::pin_project; +use pin_project_lite::pin_project; /// A set of tasks which are executed on the same thread. /// @@ -109,11 +109,12 @@ struct Scheduler { waker: AtomicWaker, } -#[pin_project] -struct LocalFuture { - scheduler: Rc, - #[pin] - future: F, +pin_project! { + struct LocalFuture { + scheduler: Rc, + #[pin] + future: F, + } } thread_local! { @@ -253,13 +254,13 @@ impl LocalSet { /// For example, this will panic: /// ```should_panic /// use tokio::runtime::Runtime; - /// use tokio::{task, blocking}; + /// use tokio::task; /// /// let mut rt = Runtime::new().unwrap(); /// let local = task::LocalSet::new(); /// local.block_on(&mut rt, async { /// let join = task::spawn_local(async { - /// let blocking_result = blocking::in_place(|| { + /// let blocking_result = task::block_in_place(|| { /// // ... /// }); /// // ... @@ -270,13 +271,13 @@ impl LocalSet { /// This, however, will not panic: /// ``` /// use tokio::runtime::Runtime; - /// use tokio::{task, blocking}; + /// use tokio::task; /// /// let mut rt = Runtime::new().unwrap(); /// let local = task::LocalSet::new(); /// local.block_on(&mut rt, async { /// let join = task::spawn_local(async { - /// let blocking_result = blocking::spawn_blocking(|| { + /// let blocking_result = task::spawn_blocking(|| { /// // ... /// }).await; /// // ... @@ -466,11 +467,11 @@ impl Drop for Scheduler { #[cfg(all(test, not(loom)))] mod tests { use super::*; - use crate::{blocking, runtime}; + use crate::{runtime, task}; #[test] fn local_current_thread() { - let mut rt = runtime::Builder::new().current_thread().build().unwrap(); + let mut rt = runtime::Builder::new().basic_scheduler().build().unwrap(); LocalSet::new().block_on(&mut rt, async { spawn_local(async {}).await.unwrap(); }); @@ -534,7 +535,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); - blocking::in_place(|| {}); + task::block_in_place(|| {}); assert!(ON_RT_THREAD.with(|cell| cell.get())); }); join.await.unwrap(); @@ -554,7 +555,7 @@ mod tests { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { assert!(ON_RT_THREAD.with(|cell| cell.get())); - blocking::spawn_blocking(|| { + task::spawn_blocking(|| { assert!( !ON_RT_THREAD.with(|cell| cell.get()), "blocking must not run on the local task set's thread" @@ -570,8 +571,6 @@ mod tests { #[test] fn all_spawns_are_local() { - use futures_util::future; - thread_local! { static ON_RT_THREAD: Cell = Cell::new(false); } @@ -588,8 +587,8 @@ mod tests { }) }) .collect::>(); - for result in future::join_all(handles).await { - result.unwrap(); + for handle in handles { + handle.await.unwrap(); } }) } diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 6110c875374..10ce68097f4 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -1,5 +1,14 @@ //! Asynchronous green-threads. +cfg_blocking! { + mod blocking; + pub use blocking::spawn_blocking; + + cfg_rt_threaded! { + pub use blocking::block_in_place; + } +} + mod core; use self::core::Cell; pub(crate) use self::core::Header; @@ -10,14 +19,15 @@ pub use self::error::JoinError; mod harness; use self::harness::Harness; -mod join; -#[cfg(feature = "rt-core")] -#[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 -pub use self::join::JoinHandle; +cfg_rt_core! { + mod join; + #[allow(unreachable_pub)] // https://github.com/rust-lang/rust/issues/57411 + pub use self::join::JoinHandle; +} -#[cfg(feature = "rt-full")] +#[cfg(feature = "rt-threaded")] mod local; -#[cfg(feature = "rt-full")] +#[cfg(feature = "rt-threaded")] pub use self::local::{spawn_local, LocalSet}; mod list; @@ -26,6 +36,11 @@ pub(crate) use self::list::OwnedList; mod raw; use self::raw::RawTask; +cfg_rt_core! { + mod spawn; + pub use spawn::spawn; +} + mod stack; pub(crate) use self::stack::TransferStack; @@ -81,15 +96,17 @@ pub(crate) trait Schedule: Sized + 'static { /// trait is implemented, the corresponding `Task` type will implement `Send`. pub(crate) trait ScheduleSendOnly: Schedule + Send + Sync {} -/// Create a new task without an associated join handle -pub(crate) fn background(task: T) -> Task -where - T: Future + Send + 'static, - S: ScheduleSendOnly, -{ - Task { - raw: RawTask::new_background::<_, S>(task), - _p: PhantomData, +cfg_rt_threaded! { + /// Create a new task without an associated join handle + pub(crate) fn background(task: T) -> Task + where + T: Future + Send + 'static, + S: ScheduleSendOnly, + { + Task { + raw: RawTask::new_background::<_, S>(task), + _p: PhantomData, + } } } @@ -112,7 +129,7 @@ where } /// Create a new `!Send` task with an associated join handle -#[cfg(feature = "rt-full")] +#[cfg(feature = "rt-threaded")] pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) where T: Future + 'static, diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index fe8299efc7d..c319e232b71 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -54,15 +54,19 @@ pub(super) fn vtable() -> &'static Vtable { } } -impl RawTask { - pub(super) fn new_background(task: T) -> RawTask - where - T: Future + Send + 'static, - S: ScheduleSendOnly, - { - RawTask::new::<_, S>(task, State::new_background()) +cfg_rt_threaded! { + impl RawTask { + pub(super) fn new_background(task: T) -> RawTask + where + T: Future + Send + 'static, + S: ScheduleSendOnly, + { + RawTask::new::<_, S>(task, State::new_background()) + } } +} +impl RawTask { pub(super) fn new_joinable(task: T) -> RawTask where T: Future + Send + 'static, @@ -71,7 +75,7 @@ impl RawTask { RawTask::new::<_, S>(task, State::new_joinable()) } - #[cfg(feature = "rt-full")] + #[cfg(feature = "rt-threaded")] pub(super) fn new_joinable_local(task: T) -> RawTask where T: Future + 'static, diff --git a/tokio/src/task/spawn.rs b/tokio/src/task/spawn.rs new file mode 100644 index 00000000000..6fdff651a59 --- /dev/null +++ b/tokio/src/task/spawn.rs @@ -0,0 +1,53 @@ +use crate::runtime; +use crate::task::JoinHandle; + +use std::future::Future; + +/// Spawns a new asynchronous task, returning a +/// [`JoinHandle`](super::JoinHandle)] for it. +/// +/// Spawning a task enables the task to execute concurrently to other tasks. The +/// spawned task may execute on the current thread, or it may be sent to a +/// different thread to be executed. The specifics depend on the current +/// [`Runtime`](crate::runtime::Runtime) configuration. +/// +/// # Examples +/// +/// In this example, a server is started and `spawn` is used to start a new task +/// that processes each received connection. +/// +/// ```no_run +/// use tokio::net::{TcpListener, TcpStream}; +/// +/// use std::io; +/// +/// async fn process(socket: TcpStream) { +/// // ... +/// # drop(socket); +/// } +/// +/// #[tokio::main] +/// async fn main() -> io::Result<()> { +/// let mut listener = TcpListener::bind("127.0.0.1:8080").await?; +/// +/// loop { +/// let (socket, _) = listener.accept().await?; +/// +/// tokio::spawn(async move { +/// // Process each socket concurrently. +/// process(socket).await +/// }); +/// } +/// } +/// ``` +/// +/// # Panics +/// +/// Panics if called from **outside** of the Tokio runtime. +pub fn spawn(task: T) -> JoinHandle +where + T: Future + Send + 'static, + T::Output: Send + 'static, +{ + runtime::spawn(task) +} diff --git a/tokio/src/task/state.rs b/tokio/src/task/state.rs index 3adfea91ea9..a9284ac9eed 100644 --- a/tokio/src/task/state.rs +++ b/tokio/src/task/state.rs @@ -58,6 +58,7 @@ const INITIAL_STATE: usize = NOTIFIED; /// unambiguous modification order. impl State { /// Starts with a ref count of 1 + #[cfg(feature = "rt-threaded")] pub(super) fn new_background() -> State { State { val: AtomicUsize::new(INITIAL_STATE), 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/tests/mod.rs b/tokio/src/tests/mod.rs index 0f934f57da5..b326561b63e 100644 --- a/tokio/src/tests/mod.rs +++ b/tokio/src/tests/mod.rs @@ -1,25 +1,3 @@ -#[macro_export] -/// Assert option is some -macro_rules! assert_some { - ($e:expr) => {{ - match $e { - Some(v) => v, - _ => panic!("expected some, was none"), - } - }}; -} - -#[macro_export] -/// Assert option is none -macro_rules! assert_none { - ($e:expr) => {{ - match $e { - Some(v) => panic!("expected none, was {:?}", v), - _ => {} - } - }}; -} - #[cfg(not(loom))] pub(crate) mod backoff; diff --git a/tokio/src/time/clock.rs b/tokio/src/time/clock.rs index 982c79aae42..e44f4925b97 100644 --- a/tokio/src/time/clock.rs +++ b/tokio/src/time/clock.rs @@ -4,13 +4,7 @@ //! `test-util` feature flag is enabled, the values returned for `now()` are //! 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, Clock}; - -#[cfg(not(feature = "test-util"))] -mod variant { +cfg_not_test_util! { use crate::time::Instant; #[derive(Debug, Clone)] @@ -38,8 +32,7 @@ mod variant { } } -#[cfg(feature = "test-util")] -mod variant { +cfg_test_util! { use crate::time::{Duration, Instant}; use std::cell::Cell; 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..f9fa1127b7d 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}; +use std::task::{Context, 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 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,32 @@ 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 + pub async fn tick(&mut self) -> Instant { + poll_fn(|cx| self.poll_tick(cx)).await } } +#[cfg(feature = "stream")] 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) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(Some(ready!(self.poll_tick(cx)))) } } 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..40e20bdb6b5 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; @@ -42,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(); @@ -55,15 +53,44 @@ 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 + assert_eq!( + *files, + 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 diff --git a/tokio/tests/fs_file_mocked.rs b/tokio/tests/fs_file_mocked.rs index 4697814cee6..12463cfc34a 100644 --- a/tokio/tests/fs_file_mocked.rs +++ b/tokio/tests/fs_file_mocked.rs @@ -1,5 +1,26 @@ #![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, + } + }; +} + +#[macro_export] +macro_rules! cfg_fs { + ($($item:item)*) => { $($item)* } +} + +#[macro_export] +macro_rules! cfg_io_std { + ($($item:item)*) => { $($item)* } +} + +use futures::future; + // Load source #[allow(warnings)] #[path = "../src/fs/file.rs"] @@ -7,7 +28,7 @@ mod file; use file::File; #[allow(warnings)] -#[path = "../src/fs/blocking.rs"] +#[path = "../src/io/blocking.rs"] mod blocking; // Load mocked types @@ -18,9 +39,16 @@ mod support { pub(crate) use support::mock_pool as pool; // Place them where the source expects them -pub(crate) mod fs { +pub(crate) mod io { + pub(crate) use tokio::io::*; + pub(crate) use crate::blocking; + pub(crate) mod sys { + pub(crate) use crate::support::mock_pool::{run, Blocking}; + } +} +pub(crate) mod fs { pub(crate) mod sys { pub(crate) use crate::support::mock_file::File; pub(crate) use crate::support::mock_pool::{run, Blocking}; @@ -30,7 +58,6 @@ pub(crate) mod fs { } use fs::sys; -use tokio::io; use tokio::prelude::*; use tokio_test::{assert_pending, assert_ready, assert_ready_err, assert_ready_ok, task}; diff --git a/tokio/tests/io_lines.rs b/tokio/tests/io_lines.rs index e85fbff7a7d..83240d62336 100644 --- a/tokio/tests/io_lines.rs +++ b/tokio/tests/io_lines.rs @@ -3,10 +3,24 @@ use tokio::io::AsyncBufReadExt; use tokio_test::assert_ok; -use futures_util::StreamExt; +#[tokio::test] +async fn lines_inherent() { + let rd: &[u8] = b"hello\r\nworld\n\n"; + let mut st = rd.lines(); + + let b = assert_ok!(st.next_line().await).unwrap(); + assert_eq!(b, "hello"); + let b = assert_ok!(st.next_line().await).unwrap(); + assert_eq!(b, "world"); + let b = assert_ok!(st.next_line().await).unwrap(); + assert_eq!(b, ""); + assert!(assert_ok!(st.next_line().await).is_none()); +} #[tokio::test] -async fn lines() { +async fn lines_stream() { + use futures::StreamExt; + let rd: &[u8] = b"hello\r\nworld\n\n"; let mut st = rd.lines(); 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..5571c199a37 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; @@ -18,7 +18,7 @@ fn run_test() { let finished_clone = finished.clone(); thread::spawn(move || { - let mut rt = runtime::Builder::new().current_thread().build().unwrap(); + let mut rt = runtime::Builder::new().basic_scheduler().build().unwrap(); let mut futures = FuturesOrdered::new(); rt.block_on(async { diff --git a/tokio/tests/rt_current_thread.rs b/tokio/tests/rt_basic.rs similarity index 95% rename from tokio/tests/rt_current_thread.rs rename to tokio/tests/rt_basic.rs index b233deee221..039bb22f1e7 100644 --- a/tokio/tests/rt_current_thread.rs +++ b/tokio/tests/rt_basic.rs @@ -28,7 +28,7 @@ fn spawned_task_does_not_progress_without_block_on() { fn rt() -> Runtime { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() } diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index d0e3df3dd88..f588415822e 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -4,12 +4,12 @@ macro_rules! rt_test { ($($t:tt)*) => { - mod current_thread { + mod basic_scheduler { $($t)* fn rt() -> Runtime { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() } @@ -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}; @@ -80,7 +80,7 @@ rt_test! { } #[test] - fn spawn_one() { + fn spawn_one_bg() { let mut rt = rt(); let out = rt.block_on(async { @@ -96,6 +96,29 @@ rt_test! { assert_eq!(out, "ZOMG"); } + #[test] + fn spawn_one_join() { + let mut rt = rt(); + + let out = rt.block_on(async { + let (tx, rx) = oneshot::channel(); + + let handle = tokio::spawn(async move { + tx.send("ZOMG").unwrap(); + "DONE" + }); + + let msg = assert_ok!(rx.await); + + let out = assert_ok!(handle.await); + assert_eq!(out, "DONE"); + + msg + }); + + assert_eq!(out, "ZOMG"); + } + #[test] fn spawn_two() { let mut rt = rt(); @@ -133,12 +156,12 @@ 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); assert_eq!(i, msg); - assert_ok!(done_tx.try_send(msg)); + assert_ok!(done_tx.send(msg)); }); tx @@ -180,7 +203,7 @@ rt_test! { tokio::spawn(poll_fn(move |_| { assert_eq!(2, Arc::strong_count(&cnt)); - Poll::Pending + Poll::<()>::Pending })); }); @@ -395,6 +418,16 @@ rt_test! { .await } + #[test] + fn enter_and_spawn() { + let mut rt = rt(); + let handle = rt.enter(|| { + tokio::spawn(async {}) + }); + + assert_ok!(rt.block_on(handle)); + } + async fn client_server(tx: mpsc::Sender<()>) { let mut server = assert_ok!(TcpListener::bind("127.0.0.1:0").await); diff --git a/tokio/tests/rt_thread_pool.rs b/tokio/tests/rt_threaded.rs similarity index 97% rename from tokio/tests/rt_thread_pool.rs rename to tokio/tests/rt_threaded.rs index d290d75a948..8be6d036a09 100644 --- a/tokio/tests/rt_thread_pool.rs +++ b/tokio/tests/rt_threaded.rs @@ -16,7 +16,10 @@ use std::task::{Context, Poll}; #[test] fn single_thread() { // No panic when starting a runtime w/ a single thread - let _ = runtime::Builder::new().thread_pool().num_threads(1).build(); + let _ = runtime::Builder::new() + .threaded_scheduler() + .num_threads(1) + .build(); } #[test] @@ -185,7 +188,7 @@ fn drop_threadpool_drops_futures() { let b = num_dec.clone(); let rt = runtime::Builder::new() - .thread_pool() + .threaded_scheduler() .after_start(move || { a.fetch_add(1, Relaxed); }) @@ -224,7 +227,7 @@ fn after_start_and_before_stop_is_called() { let after_inner = after_start.clone(); let before_inner = before_stop.clone(); let mut rt = tokio::runtime::Builder::new() - .thread_pool() + .threaded_scheduler() .after_start(move || { after_inner.clone().fetch_add(1, Ordering::Relaxed); }) @@ -265,7 +268,7 @@ fn blocking() { for _ in 0..4 { let block = block.clone(); rt.spawn(async move { - tokio::blocking::in_place(move || { + tokio::task::block_in_place(move || { block.wait(); block.wait(); }) 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..0cb7d48225c 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,12 +32,12 @@ 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 { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() } 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..8020c593680 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 }); }) }) @@ -48,7 +47,7 @@ fn multi_loop() { fn rt() -> Runtime { tokio::runtime::Builder::new() - .current_thread() + .basic_scheduler() .build() .unwrap() } 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_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 f724c564685..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, @@ -38,47 +39,33 @@ fn send_recv_with_buffer() { } #[tokio::test] -async fn async_send_recv_with_buffer() { - let (mut tx, mut rx) = mpsc::channel(16); +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.recv().await); - assert_eq!(Some(2), rx.recv().await); - assert_eq!(None, rx.recv().await); + assert_eq!(Some(1), rx.next().await); + assert_eq!(Some(2), rx.next().await); + assert_eq!(None, rx.next().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)); +#[tokio::test] +async fn async_send_recv_with_buffer() { + let (mut tx, mut rx) = mpsc::channel(16); - assert_ready_ok!(tx.as_mut().poll_flush(cx)); - assert_ready_ok!(tx.as_mut().poll_close(cx)); + tokio::spawn(async move { + assert_ok!(tx.send(1).await); + assert_ok!(tx.send(2).await); }); - 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()); - }); + assert_eq!(Some(1), rx.recv().await); + assert_eq!(Some(2), rx.recv().await); + assert_eq!(None, rx.recv().await); } #[test] @@ -124,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)); @@ -144,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); @@ -156,41 +143,20 @@ 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)); +#[tokio::test] +async fn send_recv_stream_unbounded() { + use futures::StreamExt; - assert_ready_ok!(tx.as_mut().poll_ready(cx)); - assert_ok!(tx.as_mut().start_send(2)); + let (tx, mut rx) = mpsc::unbounded_channel::(); - assert_ready_ok!(tx.as_mut().poll_flush(cx)); - assert_ready_ok!(tx.as_mut().poll_close(cx)); + tokio::spawn(async move { + assert_ok!(tx.send(1)); + assert_ok!(tx.send(2)); }); - 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()); - }); + assert_eq!(Some(1), rx.next().await); + assert_eq!(Some(2), rx.next().await); + assert_eq!(None, rx.next().await); } #[test] @@ -223,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()); @@ -356,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")); @@ -421,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 4d73bc81ff6..7ccad5c2adf 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()); } @@ -262,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 c884ca8e9f5..70709f4a8a8 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,44 @@ 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)); +} + +#[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)) } fn ms(n: u64) -> Duration { diff --git a/tokio/tests/time_rt.rs b/tokio/tests/time_rt.rs index ecce72d6f0e..235d1960574 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(); @@ -25,16 +24,16 @@ fn timer_with_threaded_runtime() { } #[test] -fn timer_with_current_thread_runtime() { +fn timer_with_basic_scheduler() { use tokio::runtime::Builder; - let mut rt = Builder::new().current_thread().build().unwrap(); + let mut rt = Builder::new().basic_scheduler().build().unwrap(); let (tx, rx) = mpsc::channel(); 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 17d520a5fca62c4fe817b78d88eb7266f2ed4d85 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 10:19:26 -0800 Subject: [PATCH 61/73] Avoid starving the remote queue Copied from `basic_scheduler` Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index cc7cb7cc8c4..b59ffb190eb 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -99,6 +99,8 @@ struct Scheduler { /// Only call from the owning thread. local_queue: UnsafeCell>>, + tick: Cell, + /// Remote run queue. /// /// Tasks notified from another thread are pushed into this queue. @@ -171,6 +173,9 @@ where /// Max number of tasks to poll per tick. const MAX_TASKS_PER_TICK: usize = 61; +/// How often to check the remote queue first +const CHECK_REMOTE_INTERVAL: u8 = 13; + impl LocalSet { /// Returns a new local task set. pub fn new() -> Self { @@ -364,6 +369,7 @@ impl Scheduler { Self { tasks: UnsafeCell::new(task::OwnedList::new()), local_queue: UnsafeCell::new(VecDeque::with_capacity(64)), + tick: Cell::new(0), remote_queue: Mutex::new(VecDeque::with_capacity(64)), waker: AtomicWaker::new(), } @@ -402,8 +408,12 @@ impl Scheduler { .unwrap_or(false) } - fn next_task(&self) -> Option> { - self.next_local_task().or_else(|| self.next_remote_task()) + fn next_task(&self, tick: u8) -> Option> { + if 0 == tick % CHECK_REMOTE_INTERVAL { + self.next_remote_task().or_else(|| self.next_local_task()) + } else { + self.next_local_task().or_else(|| self.next_remote_task()) + } } fn next_local_task(&self) -> Option> { @@ -429,7 +439,9 @@ impl Scheduler { fn tick(&self) { assert!(self.is_current()); for _ in 0..MAX_TASKS_PER_TICK { - let task = match self.next_task() { + let tick = self.tick.get().wrapping_add(1); + self.tick.set(tick); + let task = match self.next_task(tick) { Some(task) => task, None => return, }; From cfa280832592994a54da68ea8cb43812449a2dee Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 10:25:32 -0800 Subject: [PATCH 62/73] allow tasks to be cancelled remotely Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 26 ++++++++++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index b59ffb190eb..961e7fefdc3 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -1,6 +1,6 @@ //! Runs `!Send` futures on the current thread. use crate::sync::AtomicWaker; -use crate::task::{self, JoinHandle, Schedule, Task}; +use crate::task::{self, JoinHandle, Schedule, Task, TransferStack}; use std::cell::{Cell, UnsafeCell}; use std::collections::VecDeque; @@ -106,6 +106,9 @@ struct Scheduler { /// Tasks notified from another thread are pushed into this queue. remote_queue: Mutex>>, + /// Tasks pending drop + pending_drop: TransferStack, + /// Used to notify the `LocalFuture` when a task in the local task set is /// notified. waker: AtomicWaker, @@ -370,6 +373,7 @@ impl Scheduler { tasks: UnsafeCell::new(task::OwnedList::new()), local_queue: UnsafeCell::new(VecDeque::with_capacity(64)), tick: Cell::new(0), + pending_drop: TransferStack::new(), remote_queue: Mutex::new(VecDeque::with_capacity(64)), waker: AtomicWaker::new(), } @@ -454,6 +458,15 @@ impl Scheduler { } } } + + fn drain_pending_drop(&self) { + for task in self.pending_drop.drain() { + unsafe { + (*self.tasks.get()).remove(&task); + } + drop(task); + } + } } impl fmt::Debug for Scheduler { @@ -465,7 +478,7 @@ impl fmt::Debug for Scheduler { impl Drop for Scheduler { fn drop(&mut self) { // Drain all local tasks - while let Some(task) = self.next_task() { + while let Some(task) = self.next_local_task() { task.shutdown(); } @@ -473,6 +486,15 @@ impl Drop for Scheduler { unsafe { (*self.tasks.get()).shutdown(); } + + self.drain_pending_drop(); + + // Wait until all tasks have been released. + // XXX: this is a busy loop, but we don't really have any way to park + // the thread here? + while unsafe { !(*self.tasks.get()).is_empty() } { + self.drain_pending_drop(); + } } } From 6f43e7e2c4b6200fc50ca48ad3e73ea3a1ebd453 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 11:14:04 -0800 Subject: [PATCH 63/73] update tests to use schedulers that actually do stuff --- tokio/src/task/local.rs | 25 ++++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 961e7fefdc3..f6548e7e5d2 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -541,7 +541,10 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); - let mut rt = runtime::Runtime::new().unwrap(); + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { @@ -564,7 +567,10 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); - let mut rt = runtime::Runtime::new().unwrap(); + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { @@ -584,7 +590,10 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); - let mut rt = runtime::Runtime::new().unwrap(); + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let join = spawn_local(async move { @@ -611,7 +620,10 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); - let mut rt = runtime::Runtime::new().unwrap(); + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); let handles = (0..128) @@ -635,7 +647,10 @@ mod tests { ON_RT_THREAD.with(|cell| cell.set(true)); - let mut rt = runtime::Runtime::new().unwrap(); + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); LocalSet::new().block_on(&mut rt, async { assert!(ON_RT_THREAD.with(|cell| cell.get())); spawn_local(async { From f1c82c55e96bfa1a4b079a66a56a7a1fb4a1ab28 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 11:22:09 -0800 Subject: [PATCH 64/73] fix accidental hang in test --- tokio/src/task/local.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index f6548e7e5d2..303a59e2ff4 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -614,6 +614,7 @@ mod tests { #[test] fn all_spawns_are_local() { + use futures::future; thread_local! { static ON_RT_THREAD: Cell = Cell::new(false); } @@ -633,8 +634,8 @@ mod tests { }) }) .collect::>(); - for handle in handles { - handle.await.unwrap(); + for joined in future::join_all(handles).await { + joined.unwrap(); } }) } From b919c958da87e04eb3548589512a67693d265881 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 18 Nov 2019 11:22:48 -0800 Subject: [PATCH 65/73] add test for sending a local future's JoinHandle --- tokio/src/task/local.rs | 42 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 303a59e2ff4..c7333891cdc 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -676,4 +676,46 @@ mod tests { .unwrap(); }) } + #[test] + fn join_local_future_elsewhere() { + thread_local! { + static ON_RT_THREAD: Cell = Cell::new(false); + } + + ON_RT_THREAD.with(|cell| cell.set(true)); + + let mut rt = runtime::Builder::new() + .threaded_scheduler() + .build() + .unwrap(); + let local = LocalSet::new(); + local.block_on(&mut rt, async move { + let (tx, rx) = crate::sync::oneshot::channel(); + let join = spawn_local(async move { + println!("hello world running..."); + assert!( + ON_RT_THREAD.with(|cell| cell.get()), + "local task must run on local thread, no matter where it is awaited" + ); + rx.await.unwrap(); + + println!("hello world task done"); + "hello world" + }); + let join2 = task::spawn(async move { + assert!( + !ON_RT_THREAD.with(|cell| cell.get()), + "spawned task should be on a worker" + ); + + tx.send(()).expect("task shouldn't have ended yet"); + println!("waking up hello world..."); + + join.await.expect("task should complete successfully"); + + println!("hello world task joined"); + }); + join2.await.unwrap() + }); + } } From fe93b4584fd84b81173f2f48ff57fcaa9e02c952 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 20 Nov 2019 22:24:08 -0800 Subject: [PATCH 66/73] fix yielding Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index c7333891cdc..a5a01b22f62 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -398,7 +398,7 @@ impl Scheduler { } unsafe fn schedule_local(&self, task: Task) { - (*self.local_queue.get()).push_front(task); + (*self.local_queue.get()).push_back(task); } fn is_current(&self) -> bool { From 40374f724b0721b6c9d022cae72941e3a7f21203 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 14:17:22 -0800 Subject: [PATCH 67/73] fix bad merge Signed-off-by: Eliza Weisman --- tokio/src/io/poll_evented.rs | 2 +- tokio/src/io/util/async_buf_read_ext.rs | 220 +++++----- tokio/src/io/util/async_read_ext.rs | 389 ++++++++++++++---- tokio/src/io/util/async_write_ext.rs | 224 ++++++++-- tokio/src/io/util/copy.rs | 130 +++--- tokio/src/io/util/empty.rs | 70 ++-- tokio/src/io/util/repeat.rs | 72 ++-- tokio/src/io/util/sink.rs | 66 +-- tokio/src/runtime/blocking/pool.rs | 5 +- tokio/src/runtime/enter.rs | 27 +- tokio/src/runtime/thread_pool/shared.rs | 4 +- .../runtime/thread_pool/tests/loom_pool.rs | 2 +- tokio/src/task/local.rs | 3 + tokio/src/task/mod.rs | 52 +-- tokio/src/task/raw.rs | 12 +- tokio/src/task/state.rs | 8 - tokio/tests/fs_file_mocked.rs | 21 - 17 files changed, 840 insertions(+), 467 deletions(-) diff --git a/tokio/src/io/poll_evented.rs b/tokio/src/io/poll_evented.rs index 1fdb146ba5a..bbd60259d31 100644 --- a/tokio/src/io/poll_evented.rs +++ b/tokio/src/io/poll_evented.rs @@ -1,5 +1,5 @@ -use crate::io::driver::platform; use crate::io::{AsyncRead, AsyncWrite, Registration}; +use crate::io::driver::{platform}; use mio::event::Evented; use std::fmt; diff --git a/tokio/src/io/util/async_buf_read_ext.rs b/tokio/src/io/util/async_buf_read_ext.rs index 7567d72ed9b..86d09dd33b6 100644 --- a/tokio/src/io/util/async_buf_read_ext.rs +++ b/tokio/src/io/util/async_buf_read_ext.rs @@ -4,118 +4,120 @@ 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. -pub trait AsyncBufReadExt: AsyncBufRead { - /// Creates a future which will read all the bytes associated with this I/O - /// object into `buf` until the delimiter `byte` or EOF is reached. - /// This method is the async equivalent to [`BufRead::read_until`](std::io::BufRead::read_until). - /// - /// This function will read bytes from the underlying stream until the - /// delimiter or EOF is found. Once found, all bytes up to, and including, - /// the delimiter (if found) will be appended to `buf`. - /// - /// The returned future will resolve to the number of bytes read once the read - /// operation is completed. - /// - /// In the case of an error the buffer and the object will be discarded, with - /// the error yielded. - fn read_until<'a>(&'a mut self, byte: u8, buf: &'a mut Vec) -> ReadUntil<'a, Self> - where - Self: Unpin, - { - read_until(self, byte, buf) - } +cfg_io_util! { + /// An extension trait which adds utility methods to `AsyncBufRead` types. + pub trait AsyncBufReadExt: AsyncBufRead { + /// Creates a future which will read all the bytes associated with this I/O + /// object into `buf` until the delimiter `byte` or EOF is reached. + /// This method is the async equivalent to [`BufRead::read_until`](std::io::BufRead::read_until). + /// + /// This function will read bytes from the underlying stream until the + /// delimiter or EOF is found. Once found, all bytes up to, and including, + /// the delimiter (if found) will be appended to `buf`. + /// + /// The returned future will resolve to the number of bytes read once the read + /// operation is completed. + /// + /// In the case of an error the buffer and the object will be discarded, with + /// the error yielded. + fn read_until<'a>(&'a mut self, byte: u8, buf: &'a mut Vec) -> ReadUntil<'a, Self> + where + Self: Unpin, + { + read_until(self, byte, buf) + } - /// Creates a future which will read all the bytes associated with this I/O - /// object into `buf` until a newline (the 0xA byte) or EOF is reached, - /// This method is the async equivalent to [`BufRead::read_line`](std::io::BufRead::read_line). - /// - /// This function will read bytes from the underlying stream until the - /// newline delimiter (the 0xA byte) or EOF is found. Once found, all bytes - /// up to, and including, the delimiter (if found) will be appended to - /// `buf`. - /// - /// The returned future will resolve to the number of bytes read once the read - /// operation is completed. - /// - /// In the case of an error the buffer and the object will be discarded, with - /// the error yielded. - /// - /// # Errors - /// - /// This function has the same error semantics as [`read_until`] and will - /// also return an error if the read bytes are not valid UTF-8. If an I/O - /// error is encountered then `buf` may contain some bytes already read in - /// the event that all data read so far was valid UTF-8. - /// - /// [`read_until`]: AsyncBufReadExt::read_until - fn read_line<'a>(&'a mut self, buf: &'a mut String) -> ReadLine<'a, Self> - where - Self: Unpin, - { - read_line(self, buf) - } + /// Creates a future which will read all the bytes associated with this I/O + /// object into `buf` until a newline (the 0xA byte) or EOF is reached, + /// This method is the async equivalent to [`BufRead::read_line`](std::io::BufRead::read_line). + /// + /// This function will read bytes from the underlying stream until the + /// newline delimiter (the 0xA byte) or EOF is found. Once found, all bytes + /// up to, and including, the delimiter (if found) will be appended to + /// `buf`. + /// + /// The returned future will resolve to the number of bytes read once the read + /// operation is completed. + /// + /// In the case of an error the buffer and the object will be discarded, with + /// the error yielded. + /// + /// # Errors + /// + /// This function has the same error semantics as [`read_until`] and will + /// also return an error if the read bytes are not valid UTF-8. If an I/O + /// error is encountered then `buf` may contain some bytes already read in + /// the event that all data read so far was valid UTF-8. + /// + /// [`read_until`]: AsyncBufReadExt::read_until + fn read_line<'a>(&'a mut self, buf: &'a mut String) -> ReadLine<'a, Self> + where + Self: Unpin, + { + read_line(self, buf) + } - /// Returns a stream of the contents of this reader split on the byte - /// `byte`. - /// - /// This method is the asynchronous equivalent to - /// [`BufRead::split`](std::io::BufRead::split). - /// - /// The stream returned from this function will yield instances of - /// [`io::Result`]`<`[`Vec`]`>`. Each vector returned will *not* have - /// the delimiter byte at the end. - /// - /// [`io::Result`]: std::io::Result - /// [`Vec`]: std::vec::Vec - /// - /// # Errors - /// - /// 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 + Unpin, - { - split(self, byte) - } + /// Returns a stream of the contents of this reader split on the byte + /// `byte`. + /// + /// This method is the asynchronous equivalent to + /// [`BufRead::split`](std::io::BufRead::split). + /// + /// The stream returned from this function will yield instances of + /// [`io::Result`]`<`[`Vec`]`>`. Each vector returned will *not* have + /// the delimiter byte at the end. + /// + /// [`io::Result`]: std::io::Result + /// [`Vec`]: std::vec::Vec + /// + /// # Errors + /// + /// 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 + Unpin, + { + split(self, byte) + } - /// Returns a stream over the lines of this reader. - /// This method is the async equivalent to [`BufRead::lines`](std::io::BufRead::lines). - /// - /// The stream returned from this function will yield instances of - /// [`io::Result`]`<`[`String`]`>`. Each string returned will *not* have a newline - /// byte (the 0xA byte) or CRLF (0xD, 0xA bytes) at the end. - /// - /// [`io::Result`]: std::io::Result - /// [`String`]: String - /// - /// # Errors - /// - /// Each line of the stream has the same error semantics as [`AsyncBufReadExt::read_line`]. - /// - /// [`AsyncBufReadExt::read_line`]: AsyncBufReadExt::read_line - fn lines(self) -> Lines - where - Self: Sized, - { - lines(self) + /// Returns a stream over the lines of this reader. + /// This method is the async equivalent to [`BufRead::lines`](std::io::BufRead::lines). + /// + /// The stream returned from this function will yield instances of + /// [`io::Result`]`<`[`String`]`>`. Each string returned will *not* have a newline + /// byte (the 0xA byte) or CRLF (0xD, 0xA bytes) at the end. + /// + /// [`io::Result`]: std::io::Result + /// [`String`]: String + /// + /// # Errors + /// + /// Each line of the stream has the same error semantics as [`AsyncBufReadExt::read_line`]. + /// + /// [`AsyncBufReadExt::read_line`]: AsyncBufReadExt::read_line + fn lines(self) -> Lines + where + Self: Sized, + { + lines(self) + } } } diff --git a/tokio/src/io/util/async_read_ext.rs b/tokio/src/io/util/async_read_ext.rs index 0b7cdbf1394..fe1e646eee2 100644 --- a/tokio/src/io/util/async_read_ext.rs +++ b/tokio/src/io/util/async_read_ext.rs @@ -1,91 +1,334 @@ 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}; +use crate::io::AsyncRead; -/// An extension trait which adds utility methods to `AsyncRead` types. -pub trait AsyncReadExt: AsyncRead { - /// Creates an adaptor which will chain this stream with another. +cfg_io_util! { + /// Read bytes from a source. /// - /// The returned `AsyncRead` instance will first read all bytes from this object - /// until EOF is encountered. Afterwards the output is equivalent to the - /// output of `next`. - fn chain(self, next: R) -> Chain - where - Self: Sized, - R: AsyncRead, - { - chain(self, next) - } - - /// Copy all data from `self` into the provided `AsyncWrite`. + /// Implemented as an extention trait, adding utility methods to all + /// [`AsyncRead`] types. Callers will tend to import this trait instead of + /// [`AsyncRead`]. /// - /// The returned future will copy all the bytes read from `reader` into the - /// `writer` specified. This future will only complete once the `reader` - /// has hit EOF and all bytes have been written to and flushed from the - /// `writer` provided. + /// As a convenience, this trait may be imported using the [`prelude`]: /// - /// On success the number of bytes is returned and the `reader` and `writer` - /// are consumed. On error the error is returned and the I/O objects are - /// consumed as well. - fn copy<'a, W>(&'a mut self, dst: &'a mut W) -> Copy<'a, Self, W> - where - Self: Unpin, - W: AsyncWrite + Unpin + ?Sized, - { - copy(self, dst) - } - - /// Read data into the provided buffer. + /// ```no_run + /// use tokio::fs::File; + /// use tokio::prelude::*; /// - /// The returned future will resolve to the number of bytes read once the - /// read operation is completed. - fn read<'a>(&'a mut self, dst: &'a mut [u8]) -> Read<'a, Self> - where - Self: Unpin, - { - read(self, dst) - } + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut f = File::open("foo.txt").await?; + /// let mut buffer = [0; 10]; + /// + /// // The `read` method is defined by this trait. + /// let n = f.read(&mut buffer[..]).await?; + /// + /// Ok(()) + /// } + /// ``` + /// + /// See [module][crate::io] documentation for more details. + /// + /// [`AsyncRead`]: AsyncRead + /// [`prelude`]: crate::prelude + pub trait AsyncReadExt: AsyncRead { + /// Create a new `AsyncRead` instance that chains this stream with + /// `next`. + /// + /// The returned `AsyncRead` instance will first read all bytes from this object + /// until EOF is encountered. Afterwards the output is equivalent to the + /// output of `next`. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `AsyncRead`: + /// + /// ```no_run + /// use tokio::fs::File; + /// use tokio::io::{self, AsyncReadExt}; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let f1 = File::open("foo.txt").await?; + /// let f2 = File::open("bar.txt").await?; + /// + /// let mut handle = f1.chain(f2); + /// let mut buffer = String::new(); + /// + /// // read the value into a String. We could use any AsyncRead + /// // method here, this is just one example. + /// handle.read_to_string(&mut buffer).await?; + /// Ok(()) + /// } + /// ``` + fn chain(self, next: R) -> Chain + where + Self: Sized, + R: AsyncRead, + { + chain(self, next) + } - /// Read exactly the amount of data needed to fill the provided buffer. - fn read_exact<'a>(&'a mut self, dst: &'a mut [u8]) -> ReadExact<'a, Self> - where - Self: Unpin, - { - read_exact(self, dst) - } + /// Pull some bytes from this source into the specified buffer, + /// returning how many bytes were read. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn read(&mut self, buf: &mut [u8]) -> io::Result; + /// ``` + /// + /// This function does not provide any guarantees about whether it + /// completes immediately or asynchronously + /// + /// If the return value of this method is `Ok(n)`, then it must be + /// guaranteed that `0 <= n <= buf.len()`. A nonzero `n` value indicates + /// that the buffer `buf` has been filled in with `n` bytes of data from + /// this source. If `n` is `0`, then it can indicate one of two + /// scenarios: + /// + /// 1. This reader has reached its "end of file" and will likely no longer + /// be able to produce bytes. Note that this does not mean that the + /// reader will *always* no longer be able to produce bytes. + /// 2. The buffer specified was 0 bytes in length. + /// + /// No guarantees are provided about the contents of `buf` when this + /// function is called, implementations cannot rely on any property of the + /// contents of `buf` being true. It is recommended that *implementations* + /// only write data to `buf` instead of reading its contents. + /// + /// Correspondingly, however, *callers* of this method may not assume + /// any guarantees about how the implementation uses `buf`. It is + /// possible that the code that's supposed to write to the buffer might + /// also read from it. It is your responsibility to make sure that `buf` + /// is initialized before calling `read`. + /// + /// # Errors + /// + /// If this function encounters any form of I/O or other error, an error + /// variant will be returned. If an error is returned then it must be + /// guaranteed that no bytes were read. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `Read`: + /// + /// ```no_run + /// use tokio::fs::File; + /// use tokio::io::{self, AsyncReadExt}; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut f = File::open("foo.txt").await?; + /// let mut buffer = [0; 10]; + /// + /// // read up to 10 bytes + /// let n = f.read(&mut buffer[..]).await?; + /// + /// println!("The bytes: {:?}", &buffer[..n]); + /// Ok(()) + /// } + /// ``` + fn read<'a>(&'a mut self, buf: &'a mut [u8]) -> Read<'a, Self> + where + Self: Unpin, + { + read(self, buf) + } - /// Read all bytes until EOF in this source, placing them into `dst`. - /// - /// On success the total number of bytes read is returned. - fn read_to_end<'a>(&'a mut self, dst: &'a mut Vec) -> ReadToEnd<'a, Self> - where - Self: Unpin, - { - read_to_end(self, dst) - } + /// Read the exact number of bytes required to fill `buf`. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn read_exact(&mut self, buf: &mut [u8]) -> io::Result; + /// ``` + /// + /// This function reads as many bytes as necessary to completely fill + /// the specified buffer `buf`. + /// + /// No guarantees are provided about the contents of `buf` when this + /// function is called, implementations cannot rely on any property of + /// the contents of `buf` being true. It is recommended that + /// implementations only write data to `buf` instead of reading its + /// contents. + /// + /// # Errors + /// + /// If the operation encounters an "end of file" before completely + /// filling the buffer, it returns an error of the kind + /// [`ErrorKind::UnexpectedEof`]. The contents of `buf` are unspecified + /// in this case. + /// + /// If any other read error is encountered then the operation + /// immediately returns. The contents of `buf` are unspecified in this + /// case. + /// + /// If this operation returns an error, it is unspecified how many bytes + /// it has read, but it will never read more than would be necessary to + /// completely fill the buffer. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `Read`: + /// + /// ```no_run + /// use tokio::fs::File; + /// use tokio::io::{self, AsyncReadExt}; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut f = File::open("foo.txt").await?; + /// let mut buffer = [0; 10]; + /// + /// // read exactly 10 bytes + /// f.read_exact(&mut buffer).await?; + /// Ok(()) + /// } + /// ``` + /// + /// [`ErrorKind::UnexpectedEof`]: std::io::ErrorKind::UnexpectedEof + fn read_exact<'a>(&'a mut self, buf: &'a mut [u8]) -> ReadExact<'a, Self> + where + Self: Unpin, + { + read_exact(self, buf) + } - /// Read all bytes until EOF in this source, placing them into `dst`. - /// - /// On success the total number of bytes read is returned. - fn read_to_string<'a>(&'a mut self, dst: &'a mut String) -> ReadToString<'a, Self> - where - Self: Unpin, - { - read_to_string(self, dst) - } + /// Read all bytes until EOF in this source, placing them into `buf`. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn read_to_end(&mut self, buf: &mut Vec) -> io::Result; + /// ``` + /// + /// All bytes read from this source will be appended to the specified + /// buffer `buf`. This function will continuously call [`read()`] to + /// append more data to `buf` until [`read()`][read] returns `Ok(0)`. + /// + /// If successful, the total number of bytes read is returned. + /// + /// # Errors + /// + /// If a read error is encountered then the `read_to_end` operation + /// immediately completes. Any bytes which have already been read will + /// be appended to `buf`. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `Read`: + /// + /// ```no_run + /// use tokio::io::{self, AsyncReadExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut f = File::open("foo.txt").await?; + /// let mut buffer = Vec::new(); + /// + /// // read the whole file + /// f.read_to_end(&mut buffer).await?; + /// Ok(()) + /// } + /// ``` + /// + /// (See also the [`tokio::fs::read`] convenience function for reading from a + /// file.) + /// + /// [`tokio::fs::read`]: crate::fs::read::read + fn read_to_end<'a>(&'a mut self, buf: &'a mut Vec) -> ReadToEnd<'a, Self> + where + Self: Unpin, + { + read_to_end(self, buf) + } + + /// Read all bytes until EOF in this source, appending them to `buf`. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn read_to_string(&mut self, buf: &mut String) -> io::Result; + /// ``` + /// + /// If successful, the number of bytes which were read and appended to + /// `buf` is returned. + /// + /// # Errors + /// + /// If the data in this stream is *not* valid UTF-8 then an error is + /// returned and `buf` is unchanged. + /// + /// See [`read_to_end`][AsyncReadExt::read_to_end] for other error semantics. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `Read`: + /// + /// ```no_run + /// use tokio::io::{self, AsyncReadExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut f = File::open("foo.txt").await?; + /// let mut buffer = String::new(); + /// + /// f.read_to_string(&mut buffer).await?; + /// Ok(()) + /// } + /// ``` + /// + /// (See also the [`crate::fs::read_to_string`] convenience function for + /// reading from a file.) + /// + /// [`crate::fs::read_to_string`]: crate::fs::read_to_string::read_to_string + fn read_to_string<'a>(&'a mut self, dst: &'a mut String) -> ReadToString<'a, Self> + where + Self: Unpin, + { + read_to_string(self, dst) + } - /// Creates an AsyncRead adapter which will read at most `limit` bytes - /// from the underlying reader. - fn take(self, limit: u64) -> Take - where - Self: Sized, - { - take(self, limit) + /// Creates an adaptor which reads at most `limit` bytes from it. + /// + /// This function returns a new instance of `AsyncRead` which will read + /// at most `limit` bytes, after which it will always return EOF + /// (`Ok(0)`). Any read errors will not count towards the number of + /// bytes read and future calls to [`read()`][read] may succeed. + /// + /// # Examples + /// + /// [`File`][crate::fs::File]s implement `Read`: + /// + /// ```no_run + /// use tokio::io::{self, AsyncReadExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let f = File::open("foo.txt").await?; + /// let mut buffer = [0; 5]; + /// + /// // read at most five bytes + /// let mut handle = f.take(5); + /// + /// handle.read(&mut buffer).await?; + /// Ok(()) + /// } + /// ``` + fn take(self, limit: u64) -> Take + where + Self: Sized, + { + take(self, limit) + } } } diff --git a/tokio/src/io/util/async_write_ext.rs b/tokio/src/io/util/async_write_ext.rs index 9e40e2594aa..82de2576e77 100644 --- a/tokio/src/io/util/async_write_ext.rs +++ b/tokio/src/io/util/async_write_ext.rs @@ -4,38 +4,204 @@ 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. -pub trait AsyncWriteExt: AsyncWrite { - /// Write a buffer into this writter, returning how many bytes were written. - fn write<'a>(&'a mut self, src: &'a [u8]) -> Write<'a, Self> - where - Self: Unpin, - { - write(self, src) - } +cfg_io_util! { + /// Write bytes to a sink. + /// + /// Implemented as an extention trait, adding utility methods to all + /// [`AsyncWrite`] types. Callers will tend to import this trait instead of + /// [`AsyncWrite`]. + /// + /// As a convenience, this trait may be imported using the [`prelude`]: + /// + /// ```no_run + /// use tokio::prelude::*; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let data = b"some bytes"; + /// + /// let mut pos = 0; + /// let mut buffer = File::create("foo.txt").await?; + /// + /// while pos < data.len() { + /// let bytes_written = buffer.write(&data[pos..]).await?; + /// pos += bytes_written; + /// } + /// + /// Ok(()) + /// } + /// ``` + /// + /// See [module][crate::io] documentation for more details. + /// + /// [`AsyncWrite`]: AsyncWrite + /// [`prelude`]: crate::prelude + pub trait AsyncWriteExt: AsyncWrite { + /// Write a buffer into this writer, returning how many bytes were + /// written. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn write(&mut self, buf: &[u8]) -> io::Result; + /// ``` + /// + /// This function will attempt to write the entire contents of `buf`, but + /// the entire write may not succeed, or the write may also generate an + /// error. A call to `write` represents *at most one* attempt to write to + /// any wrapped object. + /// + /// If the return value is `Ok(n)` then it must be guaranteed that `n <= + /// buf.len()`. A return value of `0` typically means that the + /// underlying object is no longer able to accept bytes and will likely + /// not be able to in the future as well, or that the buffer provided is + /// empty. + /// + /// # Errors + /// + /// Each call to `write` may generate an I/O error indicating that the + /// operation could not be completed. If an error is returned then no bytes + /// in the buffer were written to this writer. + /// + /// It is **not** considered an error if the entire buffer could not be + /// written to this writer. + /// + /// # Examples + /// + /// ```no_run + /// use tokio::io::{self, AsyncWriteExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut buffer = File::create("foo.txt").await?; + /// + /// // Writes some prefix of the byte string, not necessarily all of it. + /// buffer.write(b"some bytes").await?; + /// Ok(()) + /// } + /// ``` + fn write<'a>(&'a mut self, src: &'a [u8]) -> Write<'a, Self> + where + Self: Unpin, + { + write(self, src) + } - /// Attempt to write an entire buffer into this writter. - fn write_all<'a>(&'a mut self, src: &'a [u8]) -> WriteAll<'a, Self> - where - Self: Unpin, - { - write_all(self, src) - } + /// Attempts to write an entire buffer into this writer. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn write_all(&mut self, buf: &[u8]) -> io::Result<()>; + /// ``` + /// + /// This method will continuously call [`write`] until there is no more data + /// to be written. This method will not return until the entire buffer + /// has been successfully written or such an error occurs. The first + /// error generated from this method will be returned. + /// + /// # Errors + /// + /// This function will return the first error that [`write`] returns. + /// + /// # Examples + /// + /// ```no_run + /// use tokio::io::{self, AsyncWriteExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let mut buffer = File::create("foo.txt").await?; + /// + /// buffer.write_all(b"some bytes").await?; + /// Ok(()) + /// } + /// ``` + /// + /// [`write`]: AsyncWriteExt::write + fn write_all<'a>(&'a mut self, src: &'a [u8]) -> WriteAll<'a, Self> + where + Self: Unpin, + { + write_all(self, src) + } - /// Flush the contents of this writer. - fn flush(&mut self) -> Flush<'_, Self> - where - Self: Unpin, - { - flush(self) - } + /// Flush this output stream, ensuring that all intermediately buffered + /// contents reach their destination. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn flush(&mut self) -> io::Result<()>; + /// ``` + /// + /// # Errors + /// + /// It is considered an error if not all bytes could be written due to + /// I/O errors or EOF being reached. + /// + /// # Examples + /// + /// ```no_run + /// use tokio::io::{self, BufWriter, AsyncWriteExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let f = File::create("foo.txt").await?; + /// let mut buffer = BufWriter::new(f); + /// + /// buffer.write_all(b"some bytes").await?; + /// buffer.flush().await?; + /// Ok(()) + /// } + /// ``` + fn flush(&mut self) -> Flush<'_, Self> + where + Self: Unpin, + { + flush(self) + } - /// Shutdown this writer. - fn shutdown(&mut self) -> Shutdown<'_, Self> - where - Self: Unpin, - { - shutdown(self) + /// Shuts down the output stream, ensuring that the value can be dropped + /// cleanly. + /// + /// Equivalent to: + /// + /// ```ignore + /// async fn shutdown(&mut self) -> io::Result<()>; + /// ``` + /// + /// Similar to [`flush`], all intermediately buffered is written to the + /// underlying stream. Once the operation completes, the caller should + /// no longer attempt to write to the stream. For example, the + /// `TcpStream` implementation will issue a `shutdown(Write)` sys call. + /// + /// # Examples + /// + /// ```no_run + /// use tokio::io::{self, BufWriter, AsyncWriteExt}; + /// use tokio::fs::File; + /// + /// #[tokio::main] + /// async fn main() -> io::Result<()> { + /// let f = File::create("foo.txt").await?; + /// let mut buffer = BufWriter::new(f); + /// + /// buffer.write_all(b"some bytes").await?; + /// buffer.shutdown().await?; + /// Ok(()) + /// } + /// ``` + fn shutdown(&mut self) -> Shutdown<'_, Self> + where + Self: Unpin, + { + shutdown(self) + } } } diff --git a/tokio/src/io/util/copy.rs b/tokio/src/io/util/copy.rs index 65dfa9587b7..312ff3605b0 100644 --- a/tokio/src/io/util/copy.rs +++ b/tokio/src/io/util/copy.rs @@ -5,71 +5,73 @@ use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -/// A future that asynchronously copies the entire contents of a reader into a -/// writer. -/// -/// This struct is generally created by calling [`copy`][copy]. Please -/// see the documentation of `copy()` for more details. -/// -/// [copy]: fn.copy.html -#[derive(Debug)] -#[must_use = "futures do nothing unless you `.await` or poll them"] -pub struct Copy<'a, R: ?Sized, W: ?Sized> { - reader: &'a mut R, - read_done: bool, - writer: &'a mut W, - pos: usize, - cap: usize, - amt: u64, - buf: Box<[u8]>, -} +cfg_io_util! { + /// A future that asynchronously copies the entire contents of a reader into a + /// writer. + /// + /// This struct is generally created by calling [`copy`][copy]. Please + /// see the documentation of `copy()` for more details. + /// + /// [copy]: fn.copy.html + #[derive(Debug)] + #[must_use = "futures do nothing unless you `.await` or poll them"] + pub struct Copy<'a, R: ?Sized, W: ?Sized> { + reader: &'a mut R, + read_done: bool, + writer: &'a mut W, + pos: usize, + cap: usize, + amt: u64, + buf: Box<[u8]>, + } -/// Asynchronously copies the entire contents of a reader into a writer. -/// -/// This function returns a future that will continuously read data from -/// `reader` and then write it into `writer` in a streaming fashion until -/// `reader` returns EOF. -/// -/// On success, the total number of bytes that were copied from -/// `reader` to `writer` is returned. -/// -/// This is an asynchronous version of [`std::io::copy`][std]. -/// -/// # Errors -/// -/// The returned future will finish with an error will return an error -/// immediately if any call to `poll_read` or `poll_write` returns an error. -/// -/// # Examples -/// -/// ``` -/// use tokio::io; -/// -/// # async fn dox() -> std::io::Result<()> { -/// let mut reader: &[u8] = b"hello"; -/// let mut writer: Vec = vec![]; -/// -/// io::copy(&mut reader, &mut writer).await?; -/// -/// assert_eq!(&b"hello"[..], &writer[..]); -/// # Ok(()) -/// # } -/// ``` -/// -/// [std]: https://doc.rust-lang.org/std/io/fn.copy.html -pub fn copy<'a, R, W>(reader: &'a mut R, writer: &'a mut W) -> Copy<'a, R, W> -where - R: AsyncRead + Unpin + ?Sized, - W: AsyncWrite + Unpin + ?Sized, -{ - Copy { - reader, - read_done: false, - writer, - amt: 0, - pos: 0, - cap: 0, - buf: Box::new([0; 2048]), + /// Asynchronously copies the entire contents of a reader into a writer. + /// + /// This function returns a future that will continuously read data from + /// `reader` and then write it into `writer` in a streaming fashion until + /// `reader` returns EOF. + /// + /// On success, the total number of bytes that were copied from `reader` to + /// `writer` is returned. + /// + /// This is an asynchronous version of [`std::io::copy`][std]. + /// + /// # Errors + /// + /// The returned future will finish with an error will return an error + /// immediately if any call to `poll_read` or `poll_write` returns an error. + /// + /// # Examples + /// + /// ``` + /// use tokio::io; + /// + /// # async fn dox() -> std::io::Result<()> { + /// let mut reader: &[u8] = b"hello"; + /// let mut writer: Vec = vec![]; + /// + /// io::copy(&mut reader, &mut writer).await?; + /// + /// assert_eq!(&b"hello"[..], &writer[..]); + /// # Ok(()) + /// # } + /// ``` + /// + /// [std]: https://doc.rust-lang.org/std/io/fn.copy.html + pub fn copy<'a, R, W>(reader: &'a mut R, writer: &'a mut W) -> Copy<'a, R, W> + where + R: AsyncRead + Unpin + ?Sized, + W: AsyncWrite + Unpin + ?Sized, + { + Copy { + reader, + read_done: false, + writer, + amt: 0, + pos: 0, + cap: 0, + buf: Box::new([0; 2048]), + } } } diff --git a/tokio/src/io/util/empty.rs b/tokio/src/io/util/empty.rs index 3868eb6d08b..054d8e6023d 100644 --- a/tokio/src/io/util/empty.rs +++ b/tokio/src/io/util/empty.rs @@ -5,41 +5,43 @@ use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -// An async reader which is always at EOF. -/// -/// This struct is generally created by calling [`empty`]. Please see -/// the documentation of [`empty()`][`empty`] for more details. -/// -/// This is an asynchronous version of [`std::io::empty`][std]. -/// -/// [`empty`]: fn.empty.html -/// [std]: https://doc.rust-lang.org/std/io/struct.Empty.html -pub struct Empty { - _p: (), -} +cfg_io_util! { + // An async reader which is always at EOF. + /// + /// This struct is generally created by calling [`empty`]. Please see + /// the documentation of [`empty()`][`empty`] for more details. + /// + /// This is an asynchronous version of [`std::io::empty`][std]. + /// + /// [`empty`]: fn.empty.html + /// [std]: https://doc.rust-lang.org/std/io/struct.Empty.html + pub struct Empty { + _p: (), + } -/// Creates a new empty async reader. -/// -/// All reads from the returned reader will return `Poll::Ready(Ok(0))`. -/// -/// This is an asynchronous version of [`std::io::empty`][std]. -/// -/// # Examples -/// -/// A slightly sad example of not reading anything into a buffer: -/// -/// ```rust -/// # use tokio::io::{self, AsyncReadExt}; -/// # async fn dox() { -/// let mut buffer = String::new(); -/// io::empty().read_to_string(&mut buffer).await.unwrap(); -/// assert!(buffer.is_empty()); -/// # } -/// ``` -/// -/// [std]: https://doc.rust-lang.org/std/io/fn.empty.html -pub fn empty() -> Empty { - Empty { _p: () } + /// Creates a new empty async reader. + /// + /// All reads from the returned reader will return `Poll::Ready(Ok(0))`. + /// + /// This is an asynchronous version of [`std::io::empty`][std]. + /// + /// # Examples + /// + /// A slightly sad example of not reading anything into a buffer: + /// + /// ```rust + /// # use tokio::io::{self, AsyncReadExt}; + /// # async fn dox() { + /// let mut buffer = String::new(); + /// io::empty().read_to_string(&mut buffer).await.unwrap(); + /// assert!(buffer.is_empty()); + /// # } + /// ``` + /// + /// [std]: https://doc.rust-lang.org/std/io/fn.empty.html + pub fn empty() -> Empty { + Empty { _p: () } + } } impl AsyncRead for Empty { diff --git a/tokio/src/io/util/repeat.rs b/tokio/src/io/util/repeat.rs index 22e62a9377d..a74a153b5d4 100644 --- a/tokio/src/io/util/repeat.rs +++ b/tokio/src/io/util/repeat.rs @@ -4,42 +4,44 @@ use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -/// An async reader which yields one byte over and over and over and over and -/// over and... -/// -/// This struct is generally created by calling [`repeat`][repeat]. Please -/// see the documentation of `repeat()` for more details. -/// -/// This is an asynchronous version of [`std::io::Repeat`][std]. -/// -/// [repeat]: fn.repeat.html -/// [std]: https://doc.rust-lang.org/std/io/struct.Repeat.html -#[derive(Debug)] -pub struct Repeat { - byte: u8, -} +cfg_io_util! { + /// An async reader which yields one byte over and over and over and over and + /// over and... + /// + /// This struct is generally created by calling [`repeat`][repeat]. Please + /// see the documentation of `repeat()` for more details. + /// + /// This is an asynchronous version of [`std::io::Repeat`][std]. + /// + /// [repeat]: fn.repeat.html + /// [std]: https://doc.rust-lang.org/std/io/struct.Repeat.html + #[derive(Debug)] + pub struct Repeat { + byte: u8, + } -/// Creates an instance of an async reader that infinitely repeats one byte. -/// -/// All reads from this reader will succeed by filling the specified buffer with -/// the given byte. -/// -/// This is an asynchronous version of [`std::io::repeat`][std]. -/// -/// # Examples -/// -/// ``` -/// # use tokio::io::{self, AsyncReadExt}; -/// # async fn dox() { -/// let mut buffer = [0; 3]; -/// io::repeat(0b101).read_exact(&mut buffer).await.unwrap(); -/// assert_eq!(buffer, [0b101, 0b101, 0b101]); -/// # } -/// ``` -/// -/// [std]: https://doc.rust-lang.org/std/io/fn.repeat.html -pub fn repeat(byte: u8) -> Repeat { - Repeat { byte } + /// Creates an instance of an async reader that infinitely repeats one byte. + /// + /// All reads from this reader will succeed by filling the specified buffer with + /// the given byte. + /// + /// This is an asynchronous version of [`std::io::repeat`][std]. + /// + /// # Examples + /// + /// ``` + /// # use tokio::io::{self, AsyncReadExt}; + /// # async fn dox() { + /// let mut buffer = [0; 3]; + /// io::repeat(0b101).read_exact(&mut buffer).await.unwrap(); + /// assert_eq!(buffer, [0b101, 0b101, 0b101]); + /// # } + /// ``` + /// + /// [std]: https://doc.rust-lang.org/std/io/fn.repeat.html + pub fn repeat(byte: u8) -> Repeat { + Repeat { byte } + } } impl AsyncRead for Repeat { diff --git a/tokio/src/io/util/sink.rs b/tokio/src/io/util/sink.rs index 483636589c7..74802f979e7 100644 --- a/tokio/src/io/util/sink.rs +++ b/tokio/src/io/util/sink.rs @@ -5,39 +5,41 @@ use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -/// An async writer which will move data into the void. -/// -/// This struct is generally created by calling [`sink`][sink]. Please -/// see the documentation of `sink()` for more details. -/// -/// This is an asynchronous version of `std::io::Sink`. -/// -/// [sink]: fn.sink.html -pub struct Sink { - _p: (), -} +cfg_io_util! { + /// An async writer which will move data into the void. + /// + /// This struct is generally created by calling [`sink`][sink]. Please + /// see the documentation of `sink()` for more details. + /// + /// This is an asynchronous version of `std::io::Sink`. + /// + /// [sink]: fn.sink.html + pub struct Sink { + _p: (), + } -/// Creates an instance of an async writer which will successfully consume all -/// data. -/// -/// All calls to `poll_write` on the returned instance will return -/// `Poll::Ready(Ok(buf.len()))` and the contents of the buffer will not be -/// inspected. -/// -/// This is an asynchronous version of `std::io::sink`. -/// -/// # Examples -/// -/// ```rust -/// # use tokio::io::{self, AsyncWriteExt}; -/// # async fn dox() { -/// let buffer = vec![1, 2, 3, 5, 8]; -/// let num_bytes = io::sink().write(&buffer).await.unwrap(); -/// assert_eq!(num_bytes, 5); -/// # } -/// ``` -pub fn sink() -> Sink { - Sink { _p: () } + /// Creates an instance of an async writer which will successfully consume all + /// data. + /// + /// All calls to `poll_write` on the returned instance will return + /// `Poll::Ready(Ok(buf.len()))` and the contents of the buffer will not be + /// inspected. + /// + /// This is an asynchronous version of `std::io::sink`. + /// + /// # Examples + /// + /// ```rust + /// # use tokio::io::{self, AsyncWriteExt}; + /// # async fn dox() { + /// let buffer = vec![1, 2, 3, 5, 8]; + /// let num_bytes = io::sink().write(&buffer).await.unwrap(); + /// assert_eq!(num_bytes, 5); + /// # } + /// ``` + pub fn sink() -> Sink { + Sink { _p: () } + } } impl AsyncWrite for Sink { diff --git a/tokio/src/runtime/blocking/pool.rs b/tokio/src/runtime/blocking/pool.rs index 60244f22189..052d361ad55 100644 --- a/tokio/src/runtime/blocking/pool.rs +++ b/tokio/src/runtime/blocking/pool.rs @@ -2,10 +2,10 @@ use crate::loom::sync::{Arc, Condvar, Mutex}; use crate::loom::thread; -use crate::runtime::blocking::schedule::NoopSchedule; +use crate::runtime::{self, io, time, Builder, Callback}; use crate::runtime::blocking::shutdown; +use crate::runtime::blocking::schedule::NoopSchedule; use crate::runtime::blocking::task::BlockingTask; -use crate::runtime::{self, io, time, Builder, Callback}; use crate::task::{self, JoinHandle}; use std::cell::Cell; @@ -53,6 +53,7 @@ struct Inner { /// Source of `Instant::now()` clock: time::Clock, + } struct Shared { diff --git a/tokio/src/runtime/enter.rs b/tokio/src/runtime/enter.rs index 904f3e180b4..1995da63c27 100644 --- a/tokio/src/runtime/enter.rs +++ b/tokio/src/runtime/enter.rs @@ -56,23 +56,24 @@ pub(crate) fn exit R, R>(f: F) -> R { c.set(true); }); } + } - ENTERED.with(|c| { - debug_assert!(c.get()); - c.set(false); - }); + ENTERED.with(|c| { + debug_assert!(c.get()); + c.set(false); + }); - let reset = Reset; - let ret = f(); - ::std::mem::forget(reset); + let reset = Reset; + let ret = f(); + ::std::mem::forget(reset); - ENTERED.with(|c| { - assert!(!c.get(), "closure claimed permanent executor"); - c.set(true); - }); + ENTERED.with(|c| { + assert!(!c.get(), "closure claimed permanent executor"); + c.set(true); + }); - ret - } + ret +} cfg_blocking_impl! { impl Enter { diff --git a/tokio/src/runtime/thread_pool/shared.rs b/tokio/src/runtime/thread_pool/shared.rs index 01d687c01af..f61bafaba80 100644 --- a/tokio/src/runtime/thread_pool/shared.rs +++ b/tokio/src/runtime/thread_pool/shared.rs @@ -1,6 +1,6 @@ use crate::park::Unpark; -use crate::runtime::Unparker; use crate::runtime::thread_pool::slice; +use crate::runtime::Unparker; use crate::task::{self, Schedule, ScheduleSendOnly, Task}; use std::ptr; @@ -91,4 +91,4 @@ impl Schedule for Shared { } } -impl ScheduleSendOnly for Shared

{} +impl ScheduleSendOnly for Shared {} diff --git a/tokio/src/runtime/thread_pool/tests/loom_pool.rs b/tokio/src/runtime/thread_pool/tests/loom_pool.rs index 78da2fca88e..81e292d6e36 100644 --- a/tokio/src/runtime/thread_pool/tests/loom_pool.rs +++ b/tokio/src/runtime/thread_pool/tests/loom_pool.rs @@ -1,5 +1,5 @@ -use crate::runtime::tests::loom_oneshot as oneshot; use crate::runtime::{self, Runtime}; +use crate::runtime::tests::loom_oneshot as oneshot; use crate::spawn; use loom::sync::atomic::{AtomicBool, AtomicUsize}; diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index a5a01b22f62..f218e7cf052 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -543,6 +543,7 @@ mod tests { let mut rt = runtime::Builder::new() .threaded_scheduler() + .enable_all() .build() .unwrap(); LocalSet::new().block_on(&mut rt, async { @@ -569,6 +570,7 @@ mod tests { let mut rt = runtime::Builder::new() .threaded_scheduler() + .enable_all() .build() .unwrap(); LocalSet::new().block_on(&mut rt, async { @@ -592,6 +594,7 @@ mod tests { let mut rt = runtime::Builder::new() .threaded_scheduler() + .enable_all() .build() .unwrap(); LocalSet::new().block_on(&mut rt, async { diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index 6cda8bd59a4..bae2328cf3c 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -216,15 +216,6 @@ cfg_blocking! { } } -cfg_blocking! { - mod blocking; - pub use blocking::spawn_blocking; - - cfg_rt_threaded! { - pub use blocking::block_in_place; - } -} - mod core; use self::core::Cell; pub(crate) use self::core::Header; @@ -312,20 +303,6 @@ pub(crate) trait Schedule: Sized + 'static { /// trait is implemented, the corresponding `Task` type will implement `Send`. pub(crate) trait ScheduleSendOnly: Schedule + Send + Sync {} -cfg_rt_threaded! { - /// Create a new task without an associated join handle - pub(crate) fn background(task: T) -> Task - where - T: Future + Send + 'static, - S: ScheduleSendOnly, - { - Task { - raw: RawTask::new_background::<_, S>(task), - _p: PhantomData, - } - } -} - /// Create a new task with an associated join handle pub(crate) fn joinable(task: T) -> (Task, JoinHandle) where @@ -344,23 +321,24 @@ where (task, join) } -/// Create a new `!Send` task with an associated join handle -#[cfg(feature = "rt-threaded")] -pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) -where - T: Future + 'static, - S: Schedule, -{ - let raw = RawTask::new_joinable_local::<_, S>(task); +cfg_rt_threaded! { + /// Create a new `!Send` task with an associated join handle + pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) + where + T: Future + 'static, + S: Schedule, + { + let raw = RawTask::new_joinable_local::<_, S>(task); - let task = Task { - raw, - _p: PhantomData, - }; + let task = Task { + raw, + _p: PhantomData, + }; - let join = JoinHandle::new(raw); + let join = JoinHandle::new(raw); - (task, join) + (task, join) + } } impl Task { diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 8268635b572..3d65901833c 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -56,12 +56,12 @@ pub(super) fn vtable() -> &'static Vtable { cfg_rt_threaded! { impl RawTask { - pub(super) fn new_background(task: T) -> RawTask + pub(super) fn new_joinable_local(task: T) -> RawTask where - T: Future + Send + 'static, - S: ScheduleSendOnly, + T: Future + 'static, + S: Schedule, { - RawTask::new::<_, S>(task, State::new_background()) + RawTask::new::<_, S>(task, State::new_joinable()) } } } @@ -69,8 +69,8 @@ cfg_rt_threaded! { impl RawTask { pub(super) fn new_joinable(task: T) -> RawTask where - T: Future + 'static, - S: Schedule, + T: Future + Send + 'static, + S: ScheduleSendOnly, { RawTask::new::<_, S>(task, State::new_joinable()) } diff --git a/tokio/src/task/state.rs b/tokio/src/task/state.rs index a9284ac9eed..dadfe5401db 100644 --- a/tokio/src/task/state.rs +++ b/tokio/src/task/state.rs @@ -57,14 +57,6 @@ const INITIAL_STATE: usize = NOTIFIED; /// All transitions are performed via RMW operations. This establishes an /// unambiguous modification order. impl State { - /// Starts with a ref count of 1 - #[cfg(feature = "rt-threaded")] - pub(super) fn new_background() -> State { - State { - val: AtomicUsize::new(INITIAL_STATE), - } - } - /// Starts with a ref count of 2 pub(super) fn new_joinable() -> State { State { diff --git a/tokio/tests/fs_file_mocked.rs b/tokio/tests/fs_file_mocked.rs index 8a866a575d6..0c5722404ea 100644 --- a/tokio/tests/fs_file_mocked.rs +++ b/tokio/tests/fs_file_mocked.rs @@ -22,27 +22,6 @@ macro_rules! cfg_io_std { use futures::future; -macro_rules! ready { - ($e:expr $(,)?) => { - match $e { - std::task::Poll::Ready(t) => t, - std::task::Poll::Pending => return std::task::Poll::Pending, - } - }; -} - -#[macro_export] -macro_rules! cfg_fs { - ($($item:item)*) => { $($item)* } -} - -#[macro_export] -macro_rules! cfg_io_std { - ($($item:item)*) => { $($item)* } -} - -use futures::future; - // Load source #[allow(warnings)] #[path = "../src/fs/file.rs"] From 72609982f941950cbadc0f88c68c9e20ce13c27d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 14:19:13 -0800 Subject: [PATCH 68/73] fix bad merge 2: the fixening Signed-off-by: Eliza Weisman --- tokio/src/runtime/thread_pool/slice.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/thread_pool/slice.rs b/tokio/src/runtime/thread_pool/slice.rs index 05380329c10..aa521a15e52 100644 --- a/tokio/src/runtime/thread_pool/slice.rs +++ b/tokio/src/runtime/thread_pool/slice.rs @@ -4,8 +4,8 @@ use crate::loom::rand::seed; use crate::park::Park; -use crate::runtime::thread_pool::{current, queue, Idle, Owned, Shared}; use crate::runtime::Parker; +use crate::runtime::thread_pool::{current, queue, Idle, Owned, Shared}; use crate::task::{self, JoinHandle, Task}; use crate::util::{CachePadded, FastRand}; From e40facad9104078d9a2963055b56f45a9a89a0c1 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 14:21:03 -0800 Subject: [PATCH 69/73] more cleanup Signed-off-by: Eliza Weisman --- tokio/src/task/mod.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index bae2328cf3c..c1fb8cb336a 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -232,10 +232,10 @@ cfg_rt_core! { pub use self::join::JoinHandle; } -#[cfg(feature = "rt-threaded")] -mod local; -#[cfg(feature = "rt-threaded")] -pub use self::local::{spawn_local, LocalSet}; +cfg_rt_threaded! { + mod local; + pub use self::local::{spawn_local, LocalSet}; +} mod list; pub(crate) use self::list::OwnedList; @@ -394,6 +394,6 @@ impl Drop for Task { impl fmt::Debug for Task { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.pad("Task") + fmt.debug_struct("Task").finish() } } From 9c805b680c604f7e8f5b4c09878772a6b0b8277a Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 14:28:07 -0800 Subject: [PATCH 70/73] rm empty file committed accidentally Signed-off-by: Eliza Weisman --- tokio-util/src/task.rs | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tokio-util/src/task.rs diff --git a/tokio-util/src/task.rs b/tokio-util/src/task.rs deleted file mode 100644 index e69de29bb2d..00000000000 From 04b018980a4a947630b13dc30c3b27bda523a5f7 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 15:30:17 -0800 Subject: [PATCH 71/73] move to rt-util feature Signed-off-by: Eliza Weisman --- tokio/Cargo.toml | 2 + tokio/src/macros/cfg.rs | 21 ++++ tokio/src/sync/mod.rs | 2 +- tokio/src/task/local.rs | 226 ++++++++++++++++++++-------------------- tokio/src/task/mod.rs | 6 +- tokio/src/task/raw.rs | 2 +- 6 files changed, 142 insertions(+), 117 deletions(-) diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index ba6d1d7da27..eab9d8ddab4 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -39,6 +39,7 @@ full = [ "net", "process", "rt-core", + "rt-util", "rt-threaded", "signal", "stream", @@ -67,6 +68,7 @@ process = [ ] # Includes basic task execution capabilities rt-core = [] +rt-util = [] rt-threaded = [ "num_cpus", "rt-core", diff --git a/tokio/src/macros/cfg.rs b/tokio/src/macros/cfg.rs index 959eed22e82..d703269805f 100644 --- a/tokio/src/macros/cfg.rs +++ b/tokio/src/macros/cfg.rs @@ -52,6 +52,17 @@ macro_rules! cfg_not_blocking_impl { } } +/// Enable internal `AtomicWaker` impl +macro_rules! cfg_atomic_waker_impl { + ($($item:item)*) => { + $( + #[cfg(any(feature = "io-driver", feature = "time", feature = "rt-util"))] + #[cfg(not(loom))] + $item + )* + } +} + macro_rules! cfg_dns { ($($item:item)*) => { $( @@ -220,6 +231,16 @@ macro_rules! cfg_rt_threaded { } } +macro_rules! cfg_rt_util { + ($($item:item)*) => { + $( + #[cfg(feature = "rt-util")] + #[cfg_attr(docsrs, doc(cfg(feature = "rt-util")))] + $item + )* + } +} + macro_rules! cfg_not_rt_threaded { ($($item:item)*) => { $( #[cfg(not(feature = "rt-threaded"))] $item )* diff --git a/tokio/src/sync/mod.rs b/tokio/src/sync/mod.rs index 0db703bbe15..943f8f1122b 100644 --- a/tokio/src/sync/mod.rs +++ b/tokio/src/sync/mod.rs @@ -35,7 +35,7 @@ cfg_sync! { } cfg_not_sync! { - cfg_resource_drivers! { + cfg_atomic_waker_impl! { mod task; pub(crate) use task::AtomicWaker; } diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index f218e7cf052..0c658ad1fce 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -13,74 +13,74 @@ use std::sync::Mutex; use std::task::{Context, Poll}; use pin_project_lite::pin_project; - -/// A set of tasks which are executed on the same thread. -/// -/// In some cases, it is necessary to run one or more futures that do not -/// implement [`Send`] and thus are unsafe to send between threads. In these -/// cases, a [local task set] may be used to schedule one or more `!Send` -/// futures to run together on the same thread. -/// -/// For example, the following code will not compile: -/// -/// ```rust,compile_fail -/// # use tokio::runtime::Runtime; -/// use std::rc::Rc; -/// -/// // `Rc` does not implement `Send`, and thus may not be sent between -/// // threads safely. -/// let unsend_data = Rc::new("my unsend data..."); -/// -/// let mut rt = Runtime::new().unwrap(); -/// -/// rt.block_on(async move { -/// let unsend_data = unsend_data.clone(); -/// // Because the `async` block here moves `unsend_data`, the future is `!Send`. -/// // Since `tokio::spawn` requires the spawned future to implement `Send`, this -/// // will not compile. -/// tokio::spawn(async move { -/// println!("{}", unsend_data); -/// // ... -/// }).await.unwrap(); -/// }); -/// ``` -/// In order to spawn `!Send` futures, we can use a local task set to -/// schedule them on the thread calling [`Runtime::block_on`]. When running -/// inside of the local task set, we can use [`task::spawn_local`], which can -/// spawn `!Send` futures. For example: -/// -/// ```rust -/// # use tokio::runtime::Runtime; -/// use std::rc::Rc; -/// use tokio::task; -/// -/// let unsend_data = Rc::new("my unsend data..."); -/// -/// let mut rt = Runtime::new().unwrap(); -/// // Construct a local task set that can run `!Send` futures. -/// let local = task::LocalSet::new(); -/// -/// // Run the local task group. -/// local.block_on(&mut rt, async move { -/// let unsend_data = unsend_data.clone(); -/// // `spawn_local` ensures that the future is spawned on the local -/// // task group. -/// task::spawn_local(async move { -/// println!("{}", unsend_data); -/// // ... -/// }).await.unwrap(); -/// }); -/// ``` -/// -/// [`Send`]: https://doc.rust-lang.org/std/marker/trait.Send.html -/// [local task set]: struct.LocalSet.html -/// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on -/// [`task::spawn_local`]: fn.spawn.html -#[derive(Debug)] -pub struct LocalSet { - scheduler: Rc, +cfg_rt_util! { + /// A set of tasks which are executed on the same thread. + /// + /// In some cases, it is necessary to run one or more futures that do not + /// implement [`Send`] and thus are unsafe to send between threads. In these + /// cases, a [local task set] may be used to schedule one or more `!Send` + /// futures to run together on the same thread. + /// + /// For example, the following code will not compile: + /// + /// ```rust,compile_fail + /// # use tokio::runtime::Runtime; + /// use std::rc::Rc; + /// + /// // `Rc` does not implement `Send`, and thus may not be sent between + /// // threads safely. + /// let unsend_data = Rc::new("my unsend data..."); + /// + /// let mut rt = Runtime::new().unwrap(); + /// + /// rt.block_on(async move { + /// let unsend_data = unsend_data.clone(); + /// // Because the `async` block here moves `unsend_data`, the future is `!Send`. + /// // Since `tokio::spawn` requires the spawned future to implement `Send`, this + /// // will not compile. + /// tokio::spawn(async move { + /// println!("{}", unsend_data); + /// // ... + /// }).await.unwrap(); + /// }); + /// ``` + /// In order to spawn `!Send` futures, we can use a local task set to + /// schedule them on the thread calling [`Runtime::block_on`]. When running + /// inside of the local task set, we can use [`task::spawn_local`], which can + /// spawn `!Send` futures. For example: + /// + /// ```rust + /// # use tokio::runtime::Runtime; + /// use std::rc::Rc; + /// use tokio::task; + /// + /// let unsend_data = Rc::new("my unsend data..."); + /// + /// let mut rt = Runtime::new().unwrap(); + /// // Construct a local task set that can run `!Send` futures. + /// let local = task::LocalSet::new(); + /// + /// // Run the local task group. + /// local.block_on(&mut rt, async move { + /// let unsend_data = unsend_data.clone(); + /// // `spawn_local` ensures that the future is spawned on the local + /// // task group. + /// task::spawn_local(async move { + /// println!("{}", unsend_data); + /// // ... + /// }).await.unwrap(); + /// }); + /// ``` + /// + /// [`Send`]: https://doc.rust-lang.org/std/marker/trait.Send.html + /// [local task set]: struct.LocalSet.html + /// [`Runtime::block_on`]: ../struct.Runtime.html#method.block_on + /// [`task::spawn_local`]: fn.spawn.html + #[derive(Debug)] + pub struct LocalSet { + scheduler: Rc, + } } - struct Scheduler { /// List of all active tasks spawned onto this executor. /// @@ -126,51 +126,53 @@ thread_local! { static CURRENT_TASK_SET: Cell>> = Cell::new(None); } -/// Spawns a `!Send` future on the local task set. -/// -/// The spawned future will be run on the same thread that called `spawn_local.` -/// This may only be called from the context of a local task set. -/// -/// # Panics -/// -/// - This function panics if called outside of a local task set. -/// -/// # Examples -/// -/// ```rust -/// # use tokio::runtime::Runtime; -/// use std::rc::Rc; -/// use tokio::task; -/// -/// let unsend_data = Rc::new("my unsend data..."); -/// -/// let mut rt = Runtime::new().unwrap(); -/// let local = task::LocalSet::new(); -/// -/// // Run the local task set. -/// local.block_on(&mut rt, async move { -/// let unsend_data = unsend_data.clone(); -/// task::spawn_local(async move { -/// println!("{}", unsend_data); -/// // ... -/// }).await.unwrap(); -/// }); -/// ``` -pub fn spawn_local(future: F) -> JoinHandle -where - F: Future + 'static, - F::Output: 'static, -{ - CURRENT_TASK_SET.with(|current| { - let current = current - .get() - .expect("`spawn_local` called from outside of a local::LocalSet!"); - unsafe { - let (task, handle) = task::joinable_local(future); - current.as_ref().schedule_local(task); - handle - } - }) +cfg_rt_util! { + /// Spawns a `!Send` future on the local task set. + /// + /// The spawned future will be run on the same thread that called `spawn_local.` + /// This may only be called from the context of a local task set. + /// + /// # Panics + /// + /// - This function panics if called outside of a local task set. + /// + /// # Examples + /// + /// ```rust + /// # use tokio::runtime::Runtime; + /// use std::rc::Rc; + /// use tokio::task; + /// + /// let unsend_data = Rc::new("my unsend data..."); + /// + /// let mut rt = Runtime::new().unwrap(); + /// let local = task::LocalSet::new(); + /// + /// // Run the local task set. + /// local.block_on(&mut rt, async move { + /// let unsend_data = unsend_data.clone(); + /// task::spawn_local(async move { + /// println!("{}", unsend_data); + /// // ... + /// }).await.unwrap(); + /// }); + /// ``` + pub fn spawn_local(future: F) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + CURRENT_TASK_SET.with(|current| { + let current = current + .get() + .expect("`spawn_local` called from outside of a local::LocalSet!"); + unsafe { + let (task, handle) = task::joinable_local(future); + current.as_ref().schedule_local(task); + handle + } + }) + } } /// Max number of tasks to poll per tick. diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index c1fb8cb336a..f84659e0f99 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -232,9 +232,9 @@ cfg_rt_core! { pub use self::join::JoinHandle; } -cfg_rt_threaded! { +cfg_rt_util! { mod local; - pub use self::local::{spawn_local, LocalSet}; + pub use local::{spawn_local, LocalSet}; } mod list; @@ -321,7 +321,7 @@ where (task, join) } -cfg_rt_threaded! { +cfg_rt_util! { /// Create a new `!Send` task with an associated join handle pub(crate) fn joinable_local(task: T) -> (Task, JoinHandle) where diff --git a/tokio/src/task/raw.rs b/tokio/src/task/raw.rs index 3d65901833c..93bb79c8288 100644 --- a/tokio/src/task/raw.rs +++ b/tokio/src/task/raw.rs @@ -54,7 +54,7 @@ pub(super) fn vtable() -> &'static Vtable { } } -cfg_rt_threaded! { +cfg_rt_util! { impl RawTask { pub(super) fn new_joinable_local(task: T) -> RawTask where From 05285372889d2ced838fa3d9c7f2aef34790abdb Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 15:57:34 -0800 Subject: [PATCH 72/73] panic if nested Signed-off-by: Eliza Weisman --- tokio/src/task/local.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index 0c658ad1fce..cb62bd5edf0 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -393,7 +393,8 @@ impl Scheduler { } CURRENT_TASK_SET.with(|current| { - current.set(Some(NonNull::from(self))); + let prev = current.replace(Some(NonNull::from(self))); + assert!(prev.is_none(), "nested call to local::Scheduler::with"); let _entered = Entered { current }; f() }) From 3418bfb76183fb912b198419bb73461f6ad8ee16 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 26 Nov 2019 16:47:20 -0800 Subject: [PATCH 73/73] fix AtomicWaker being enabled when not used Signed-off-by: Eliza Weisman --- tokio/src/macros/cfg.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tokio/src/macros/cfg.rs b/tokio/src/macros/cfg.rs index d703269805f..4aee3b7f349 100644 --- a/tokio/src/macros/cfg.rs +++ b/tokio/src/macros/cfg.rs @@ -56,7 +56,11 @@ macro_rules! cfg_not_blocking_impl { macro_rules! cfg_atomic_waker_impl { ($($item:item)*) => { $( - #[cfg(any(feature = "io-driver", feature = "time", feature = "rt-util"))] + #[cfg(any( + feature = "io-driver", + feature = "time", + all(feature = "rt-core", feature = "rt-util") + ))] #[cfg(not(loom))] $item )*