From 5887974206bf621a3b04e581f041367b4700cd6a Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 11:58:20 -0500 Subject: [PATCH 01/16] rebase and rewrite and fix issues raised by @hawkw fix clippy warnings fix clippy warnings and address the issues raised by @Darksonn fix more clippy warnings Update tokio/src/runtime/task/core.rs Co-authored-by: Eliza Weisman Update tokio/src/runtime/task/mod.rs Co-authored-by: Eliza Weisman fix nits raised by @hawkw --- tokio/src/runtime/context.rs | 11 ++ tokio/src/runtime/task/abort.rs | 2 +- tokio/src/runtime/task/core.rs | 23 +++ tokio/src/runtime/task/error.rs | 2 +- tokio/src/runtime/task/harness.rs | 7 +- tokio/src/runtime/task/join.rs | 4 +- tokio/src/runtime/task/mod.rs | 54 +++++- tokio/src/task/mod.rs | 2 +- tokio/tests/task_id.rs | 271 ++++++++++++++++++++++++++++++ 9 files changed, 361 insertions(+), 15 deletions(-) create mode 100644 tokio/tests/task_id.rs diff --git a/tokio/src/runtime/context.rs b/tokio/src/runtime/context.rs index 6de657481e0..e4e9d3875ce 100644 --- a/tokio/src/runtime/context.rs +++ b/tokio/src/runtime/context.rs @@ -1,4 +1,5 @@ use crate::runtime::coop; +use crate::runtime::task::Id; use std::cell::Cell; @@ -17,6 +18,7 @@ struct Context { /// Handle to the runtime scheduler running on the current thread. #[cfg(feature = "rt")] handle: RefCell>, + current_task_id: Cell>, /// Tracks if the current thread is currently driving a runtime. /// Note, that if this is set to "entered", the current scheduler @@ -41,6 +43,7 @@ tokio_thread_local! { /// accessing drivers, etc... #[cfg(feature = "rt")] handle: RefCell::new(None), + current_task_id: Cell::new(None), /// Tracks if the current thread is currently driving a runtime. /// Note, that if this is set to "entered", the current scheduler @@ -107,6 +110,14 @@ cfg_rt! { pub(crate) struct DisallowBlockInPlaceGuard(bool); + pub(crate) fn set_current_task_id(id: Option) -> Option { + CONTEXT.try_with(|ctx| ctx.current_task_id.replace(id)).unwrap_or(None) + } + + pub(crate) fn current_task_id() -> Option { + CONTEXT.try_with(|ctx| ctx.current_task_id.get()).unwrap_or(None) + } + pub(crate) fn try_current() -> Result { match CONTEXT.try_with(|ctx| ctx.handle.borrow().clone()) { Ok(Some(handle)) => Ok(handle), diff --git a/tokio/src/runtime/task/abort.rs b/tokio/src/runtime/task/abort.rs index c34e2bb9a02..bfdf53c5105 100644 --- a/tokio/src/runtime/task/abort.rs +++ b/tokio/src/runtime/task/abort.rs @@ -67,7 +67,7 @@ impl AbortHandle { #[cfg(tokio_unstable)] #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] pub fn id(&self) -> super::Id { - self.id.clone() + self.id } } diff --git a/tokio/src/runtime/task/core.rs b/tokio/src/runtime/task/core.rs index 3e07d7c97fd..c4a7c6c72e3 100644 --- a/tokio/src/runtime/task/core.rs +++ b/tokio/src/runtime/task/core.rs @@ -11,6 +11,7 @@ use crate::future::Future; use crate::loom::cell::UnsafeCell; +use crate::runtime::context; use crate::runtime::task::raw::{self, Vtable}; use crate::runtime::task::state::State; use crate::runtime::task::{Id, Schedule}; @@ -157,6 +158,26 @@ impl CoreStage { } } +/// Set and clear the task id in the context when the future is executed or +/// dropped, or when the output produced by the future is dropped. +pub(crate) struct TaskIdGuard { + parent_task_id: Option, +} + +impl TaskIdGuard { + fn enter(id: Id) -> Self { + TaskIdGuard { + parent_task_id: context::set_current_task_id(Some(id)), + } + } +} + +impl Drop for TaskIdGuard { + fn drop(&mut self) { + context::set_current_task_id(self.parent_task_id); + } +} + impl Core { /// Polls the future. /// @@ -183,6 +204,7 @@ impl Core { // Safety: The caller ensures the future is pinned. let future = unsafe { Pin::new_unchecked(future) }; + let _guard = TaskIdGuard::enter(self.task_id); future.poll(&mut cx) }) }; @@ -236,6 +258,7 @@ impl Core { } unsafe fn set_stage(&self, stage: Stage) { + let _guard = TaskIdGuard::enter(self.task_id); self.stage.stage.with_mut(|ptr| *ptr = stage) } } diff --git a/tokio/src/runtime/task/error.rs b/tokio/src/runtime/task/error.rs index 7cf602abd33..f7ead77b7cc 100644 --- a/tokio/src/runtime/task/error.rs +++ b/tokio/src/runtime/task/error.rs @@ -128,7 +128,7 @@ impl JoinError { #[cfg(tokio_unstable)] #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] pub fn id(&self) -> Id { - self.id.clone() + self.id } } diff --git a/tokio/src/runtime/task/harness.rs b/tokio/src/runtime/task/harness.rs index 545b01b7ff7..085aebe92ac 100644 --- a/tokio/src/runtime/task/harness.rs +++ b/tokio/src/runtime/task/harness.rs @@ -454,13 +454,12 @@ fn cancel_task(core: &Core) { core.drop_future_or_output(); })); - let id = core.task_id.clone(); match res { Ok(()) => { - core.store_output(Err(JoinError::cancelled(id))); + core.store_output(Err(JoinError::cancelled(core.task_id))); } Err(panic) => { - core.store_output(Err(JoinError::panic(id, panic))); + core.store_output(Err(JoinError::panic(core.task_id, panic))); } } } @@ -492,7 +491,7 @@ fn poll_future(core: &Core, cx: Context<'_>) -> Po Ok(Poll::Ready(output)) => Ok(output), Err(panic) => { core.scheduler.unhandled_panic(); - Err(JoinError::panic(core.task_id.clone(), panic)) + Err(JoinError::panic(core.task_id, panic)) } }; diff --git a/tokio/src/runtime/task/join.rs b/tokio/src/runtime/task/join.rs index 31f6a6f8748..21ef2f1ba8b 100644 --- a/tokio/src/runtime/task/join.rs +++ b/tokio/src/runtime/task/join.rs @@ -267,7 +267,7 @@ impl JoinHandle { raw.ref_inc(); raw }); - super::AbortHandle::new(raw, self.id.clone()) + super::AbortHandle::new(raw, self.id) } /// Returns a [task ID] that uniquely identifies this task relative to other @@ -282,7 +282,7 @@ impl JoinHandle { #[cfg(tokio_unstable)] #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] pub fn id(&self) -> super::Id { - self.id.clone() + self.id } } diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 3d5b1cbf373..40752f3965b 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -139,6 +139,8 @@ // unstable. This should be removed once `JoinSet` is stabilized. #![cfg_attr(not(tokio_unstable), allow(dead_code))] +use crate::runtime::context; + mod core; use self::core::Cell; use self::core::Header; @@ -190,9 +192,13 @@ use std::{fmt, mem}; /// # Notes /// /// - Task IDs are unique relative to other *currently running* tasks. When a -/// task completes, the same ID may be used for another task. -/// - Task IDs are *not* sequential, and do not indicate the order in which -/// tasks are spawned, what runtime a task is spawned on, or any other data. +/// task completes, the same ID may be used for another task. - Task IDs are +/// *not* sequential, and do not indicate the order in which tasks are spawned, +/// what runtime a task is spawned on, or any other data. - The task ID of the +/// currently running task can be obtained from inside the task via the +/// [`task::try_id()`](crate::task::try_id()) and +/// [`task::id()`](crate::task::id()) functions and from outside the task via +/// the [`JoinHandle::id()`](create::task::JoinHandle::id()`) function. /// /// **Note**: This is an [unstable API][unstable]. The public API of this type /// may break in 1.x releases. See [the documentation on unstable @@ -201,10 +207,46 @@ use std::{fmt, mem}; /// [unstable]: crate#unstable-features #[cfg_attr(docsrs, doc(cfg(all(feature = "rt", tokio_unstable))))] #[cfg_attr(not(tokio_unstable), allow(unreachable_pub))] -// TODO(eliza): there's almost certainly no reason not to make this `Copy` as well... -#[derive(Clone, Debug, Hash, Eq, PartialEq)] +#[derive(Clone, Copy, Debug, Hash, Eq, PartialEq)] pub struct Id(u64); +/// Returns the [`Id`] of the currently running task. +/// +/// # Panics +/// +/// This function panics if called from outside a task. Please note that calls +/// to `block_on` do not have task IDs, so the method will panic if called from +/// within a call to `block_on`. For a version of this function that doesn't +/// panic, see [`task::try_id()`](crate::runtime::task::try_id()). +/// **Note**: This is an [unstable API][unstable]. The public API of this type +/// may break in 1.x releases. See [the documentation on unstable +/// features][unstable] for details. +/// +/// [task ID]: crate::task::Id +#[cfg_attr(not(tokio_unstable), allow(unreachable_pub))] +#[track_caller] +pub fn id() -> Id { + context::current_task_id().expect("Can't get a task id when not inside a task") +} + +/// Returns the [`Id`] of the currently running task, or `None` if called outside +/// of a task. +/// +/// This function is similar to [`task::id()`](crate::runtime::task::id()), except +/// that it returns `None` rather than panicking if called outside of a task +/// context. +/// +/// **Note**: This is an [unstable API][unstable]. The public API of this type +/// may break in 1.x releases. See [the documentation on unstable +/// features][unstable] for details. +/// +/// [task ID]: crate::task::Id +#[cfg_attr(not(tokio_unstable), allow(unreachable_pub))] +#[track_caller] +pub fn try_id() -> Option { + context::current_task_id() +} + /// An owned handle to the task, tracked by ref count. #[repr(transparent)] pub(crate) struct Task { @@ -284,7 +326,7 @@ cfg_rt! { T: Future + 'static, T::Output: 'static, { - let raw = RawTask::new::(task, scheduler, id.clone()); + let raw = RawTask::new::(task, scheduler, id); let task = Task { raw, _p: PhantomData, diff --git a/tokio/src/task/mod.rs b/tokio/src/task/mod.rs index f1683f7e07f..9b753701854 100644 --- a/tokio/src/task/mod.rs +++ b/tokio/src/task/mod.rs @@ -318,7 +318,7 @@ cfg_rt! { pub mod join_set; cfg_unstable! { - pub use crate::runtime::task::Id; + pub use crate::runtime::task::{Id, id, try_id}; } cfg_trace! { diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs new file mode 100644 index 00000000000..601b4c2ddcf --- /dev/null +++ b/tokio/tests/task_id.rs @@ -0,0 +1,271 @@ +#![warn(rust_2018_idioms)] +#![allow(clippy::declare_interior_mutable_const)] +#![cfg(all(feature = "full", not(tokio_wasi)))] + +use std::error::Error; +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; +use tokio::runtime::Runtime; +use tokio::sync::oneshot; +use tokio::task::{self, LocalSet}; + +mod support { + pub mod panic; +} +use support::panic::test_panic; + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_spawn() { + tokio::spawn(async { println!("task id: {}", task::id()) }) + .await + .unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_spawn_blocking() { + task::spawn_blocking(|| println!("task id: {}", task::id())) + .await + .unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_collision_current_thread() { + let handle1 = tokio::spawn(async { task::id() }); + let handle2 = tokio::spawn(async { task::id() }); + + let (id1, id2) = tokio::join!(handle1, handle2); + assert_ne!(id1.unwrap(), id2.unwrap()); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_id_collision_multi_thread() { + let handle1 = tokio::spawn(async { task::id() }); + let handle2 = tokio::spawn(async { task::id() }); + + let (id1, id2) = tokio::join!(handle1, handle2); + assert_ne!(id1.unwrap(), id2.unwrap()); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_ids_match_current_thread() { + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(async { + let id = rx.await.unwrap(); + assert_eq!(id, task::id()); + }); + tx.send(handle.id()).unwrap(); + handle.await.unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_ids_match_multi_thread() { + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(async { + let id = rx.await.unwrap(); + assert_eq!(id, task::id()); + }); + tx.send(handle.id()).unwrap(); + handle.await.unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_id_future_destructor_completion() { + struct MyFuture; + + impl Future for MyFuture { + type Output = (); + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<()> { + Poll::Ready(()) + } + } + + impl Drop for MyFuture { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + tokio::spawn(MyFuture).await.unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_id_future_destructor_abort() { + struct MyFuture; + + impl Future for MyFuture { + type Output = (); + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<()> { + Poll::Pending + } + } + impl Drop for MyFuture { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + tokio::spawn(MyFuture).abort(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_output_destructor_handle_dropped_before_completion() { + struct MyOutput; + + impl Drop for MyOutput { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + struct MyFuture { + tx: Option>, + } + + impl Future for MyFuture { + type Output = MyOutput; + + fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { + let _ = self.tx.take().unwrap().send(()); + Poll::Ready(MyOutput) + } + } + + impl Drop for MyFuture { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(MyFuture { tx: Some(tx) }); + drop(handle); + rx.await.unwrap(); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_output_destructor_handle_dropped_after_completion() { + struct MyOutput; + + impl Drop for MyOutput { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + struct MyFuture { + tx: Option>, + } + + impl Future for MyFuture { + type Output = MyOutput; + + fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { + let _ = self.tx.take().unwrap().send(()); + Poll::Ready(MyOutput) + } + } + + impl Drop for MyFuture { + fn drop(&mut self) { + println!("task id: {}", task::id()); + } + } + + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(MyFuture { tx: Some(tx) }); + rx.await.unwrap(); + drop(handle); +} + +#[cfg(tokio_unstable)] +#[test] +fn task_try_id_outside_task() { + assert_eq!(None, task::try_id()); +} + +#[cfg(tokio_unstable)] +#[test] +fn task_try_id_inside_block_on() { + let rt = Runtime::new().unwrap(); + rt.block_on(async { + assert_eq!(None, task::try_id()); + }); +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_spawn_local() { + LocalSet::new() + .run_until(async { + task::spawn_local(async { println!("task id: {}", task::id()) }) + .await + .unwrap(); + }) + .await +} + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "current_thread")] +async fn task_id_nested_spawn_local() { + LocalSet::new() + .run_until(async { + task::spawn_local(async { + let outer_id = task::id(); + LocalSet::new() + .run_until(async { + task::spawn_local(async move { + assert_ne!(outer_id, task::id()); + }) + .await + .unwrap(); + }) + .await; + assert_eq!(outer_id, task::id()); + }) + .await + .unwrap(); + }) + .await; +} + +#[cfg(tokio_unstable)] +#[test] +fn task_id_outside_task_panic_caller() -> Result<(), Box> { + let panic_location_file = test_panic(|| { + let _ = task::id(); + }); + + // The panic location should be in this file + assert_eq!(&panic_location_file.unwrap(), file!()); + + Ok(()) +} + +#[cfg(tokio_unstable)] +#[test] +fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { + let panic_location_file = test_panic(|| { + let rt = Runtime::new().unwrap(); + rt.block_on(async { + task::id(); + }); + }); + + // The panic location should be in this file + assert_eq!(&panic_location_file.unwrap(), file!()); + + Ok(()) +} From ed1a9fccf3e8c807043c493dddba92b81d90038f Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 16:39:28 -0500 Subject: [PATCH 02/16] fix documentation issues --- tokio/src/runtime/task/mod.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 40752f3965b..0ca349631f2 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -198,7 +198,7 @@ use std::{fmt, mem}; /// currently running task can be obtained from inside the task via the /// [`task::try_id()`](crate::task::try_id()) and /// [`task::id()`](crate::task::id()) functions and from outside the task via -/// the [`JoinHandle::id()`](create::task::JoinHandle::id()`) function. +/// the [`JoinHandle::id()`](crate::task::JoinHandle::id()) function. /// /// **Note**: This is an [unstable API][unstable]. The public API of this type /// may break in 1.x releases. See [the documentation on unstable @@ -218,11 +218,13 @@ pub struct Id(u64); /// to `block_on` do not have task IDs, so the method will panic if called from /// within a call to `block_on`. For a version of this function that doesn't /// panic, see [`task::try_id()`](crate::runtime::task::try_id()). +/// /// **Note**: This is an [unstable API][unstable]. The public API of this type /// may break in 1.x releases. See [the documentation on unstable /// features][unstable] for details. /// /// [task ID]: crate::task::Id +/// [unstable]: crate#unstable-features #[cfg_attr(not(tokio_unstable), allow(unreachable_pub))] #[track_caller] pub fn id() -> Id { @@ -241,6 +243,7 @@ pub fn id() -> Id { /// features][unstable] for details. /// /// [task ID]: crate::task::Id +/// [unstable]: crate#unstable-features #[cfg_attr(not(tokio_unstable), allow(unreachable_pub))] #[track_caller] pub fn try_id() -> Option { From 1f40bd4ea857d74e305713adf552ba72775040d3 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 17:07:01 -0500 Subject: [PATCH 03/16] fix build failure with features disabled --- tokio/src/runtime/context.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/src/runtime/context.rs b/tokio/src/runtime/context.rs index e4e9d3875ce..d224cac171b 100644 --- a/tokio/src/runtime/context.rs +++ b/tokio/src/runtime/context.rs @@ -18,6 +18,7 @@ struct Context { /// Handle to the runtime scheduler running on the current thread. #[cfg(feature = "rt")] handle: RefCell>, + #[cfg(feature = "rt")] current_task_id: Cell>, /// Tracks if the current thread is currently driving a runtime. From 01305af22101be1fffa224ce54d829534d10f997 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 17:47:50 -0500 Subject: [PATCH 04/16] added the last test --- tokio/tests/task_id.rs | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 601b4c2ddcf..c0dcb61ebf2 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -6,7 +6,7 @@ use std::error::Error; use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; -use tokio::runtime::Runtime; +use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot; use tokio::task::{self, LocalSet}; @@ -269,3 +269,21 @@ fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { Ok(()) } + +#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_id_block_in_place_block_on_spawn() { + task::spawn(async { + let id1 = task::id(); + + task::block_in_place(|| { + let rt = Builder::new_current_thread().build().unwrap(); + rt.block_on(rt.spawn(async {})).unwrap(); + }); + + let id2 = task::id(); + assert_eq!(id1, id2); + }) + .await + .unwrap(); +} From 87dfea0285a966f52b77f815c09500ff3e55e4ab Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 17:56:29 -0500 Subject: [PATCH 05/16] fix clippy error --- tokio-util/src/task/join_map.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tokio-util/src/task/join_map.rs b/tokio-util/src/task/join_map.rs index d5b41e437f5..c6bf5bc241a 100644 --- a/tokio-util/src/task/join_map.rs +++ b/tokio-util/src/task/join_map.rs @@ -363,10 +363,7 @@ where fn insert(&mut self, key: K, abort: AbortHandle) { let hash = self.hash(&key); let id = abort.id(); - let map_key = Key { - id: id.clone(), - key, - }; + let map_key = Key { id, key }; // Insert the new key into the map of tasks by keys. let entry = self From 1f93f173dccc25832e546a5b3d2c3df73571499d Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 18:18:26 -0500 Subject: [PATCH 06/16] trying brute-force approach to clippy failure fixing --- tokio/tests/task_id.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index c0dcb61ebf2..2aad4107cb0 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -2,17 +2,26 @@ #![allow(clippy::declare_interior_mutable_const)] #![cfg(all(feature = "full", not(tokio_wasi)))] +#[cfg(tokio_unstable)] use std::error::Error; +#[cfg(tokio_unstable)] use std::future::Future; +#[cfg(tokio_unstable)] use std::pin::Pin; +#[cfg(tokio_unstable)] use std::task::{Context, Poll}; +#[cfg(tokio_unstable)] use tokio::runtime::{Builder, Runtime}; +#[cfg(tokio_unstable)] use tokio::sync::oneshot; +#[cfg(tokio_unstable)] use tokio::task::{self, LocalSet}; +#[cfg(tokio_unstable)] mod support { pub mod panic; } +#[cfg(tokio_unstable)] use support::panic::test_panic; #[cfg(tokio_unstable)] From f6add69b105f9d8b63bd39bacbb35e899e51c4a5 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 18:21:32 -0500 Subject: [PATCH 07/16] fix conditional build failure --- tokio/src/runtime/context.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/src/runtime/context.rs b/tokio/src/runtime/context.rs index d224cac171b..293bc472536 100644 --- a/tokio/src/runtime/context.rs +++ b/tokio/src/runtime/context.rs @@ -44,6 +44,7 @@ tokio_thread_local! { /// accessing drivers, etc... #[cfg(feature = "rt")] handle: RefCell::new(None), + #[cfg(feature = "rt")] current_task_id: Cell::new(None), /// Tracks if the current thread is currently driving a runtime. From 7dfa98d13f8b64dd2ab8a5a749615f6c9870d4e3 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Thu, 10 Nov 2022 20:58:29 -0500 Subject: [PATCH 08/16] one final build fix --- tokio/src/runtime/context.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tokio/src/runtime/context.rs b/tokio/src/runtime/context.rs index 293bc472536..ef9b05b4a26 100644 --- a/tokio/src/runtime/context.rs +++ b/tokio/src/runtime/context.rs @@ -1,4 +1,6 @@ use crate::runtime::coop; + +#[cfg(feature = "rt")] use crate::runtime::task::Id; use std::cell::Cell; From 77b87e3a55bc793e0c110a0398b67fcf1d10dafa Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 11:57:17 -0500 Subject: [PATCH 09/16] address issues raised by @Darksonn --- tokio/src/runtime/context.rs | 4 +- tokio/src/runtime/task/mod.rs | 14 ++-- tokio/tests/task_id.rs | 149 ++++++++++++++++------------------ 3 files changed, 77 insertions(+), 90 deletions(-) diff --git a/tokio/src/runtime/context.rs b/tokio/src/runtime/context.rs index ef9b05b4a26..c3ef6469926 100644 --- a/tokio/src/runtime/context.rs +++ b/tokio/src/runtime/context.rs @@ -1,8 +1,5 @@ use crate::runtime::coop; -#[cfg(feature = "rt")] -use crate::runtime::task::Id; - use std::cell::Cell; #[cfg(any(feature = "rt", feature = "macros"))] @@ -10,6 +7,7 @@ use crate::util::rand::{FastRand, RngSeed}; cfg_rt! { use crate::runtime::scheduler; + use crate::runtime::task::Id; use std::cell::RefCell; use std::marker::PhantomData; diff --git a/tokio/src/runtime/task/mod.rs b/tokio/src/runtime/task/mod.rs index 0ca349631f2..c31b110a476 100644 --- a/tokio/src/runtime/task/mod.rs +++ b/tokio/src/runtime/task/mod.rs @@ -192,13 +192,13 @@ use std::{fmt, mem}; /// # Notes /// /// - Task IDs are unique relative to other *currently running* tasks. When a -/// task completes, the same ID may be used for another task. - Task IDs are -/// *not* sequential, and do not indicate the order in which tasks are spawned, -/// what runtime a task is spawned on, or any other data. - The task ID of the -/// currently running task can be obtained from inside the task via the -/// [`task::try_id()`](crate::task::try_id()) and -/// [`task::id()`](crate::task::id()) functions and from outside the task via -/// the [`JoinHandle::id()`](crate::task::JoinHandle::id()) function. +/// task completes, the same ID may be used for another task. +/// - Task IDs are *not* sequential, and do not indicate the order in which +/// tasks are spawned, what runtime a task is spawned on, or any other data. +/// - The task ID of the currently running task can be obtained from inside the +/// task via the [`task::try_id()`](crate::task::try_id()) and +/// [`task::id()`](crate::task::id()) functions and from outside the task via +/// the [`JoinHandle::id()`](crate::task::JoinHandle::id()) function. /// /// **Note**: This is an [unstable API][unstable]. The public API of this type /// may break in 1.x releases. See [the documentation on unstable diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 2aad4107cb0..04a223e98de 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -1,30 +1,20 @@ #![warn(rust_2018_idioms)] #![allow(clippy::declare_interior_mutable_const)] -#![cfg(all(feature = "full", not(tokio_wasi)))] +#![cfg(all(feature = "full", tokio_unstable, not(tokio_wasi)))] -#[cfg(tokio_unstable)] use std::error::Error; -#[cfg(tokio_unstable)] use std::future::Future; -#[cfg(tokio_unstable)] use std::pin::Pin; -#[cfg(tokio_unstable)] use std::task::{Context, Poll}; -#[cfg(tokio_unstable)] use tokio::runtime::{Builder, Runtime}; -#[cfg(tokio_unstable)] use tokio::sync::oneshot; -#[cfg(tokio_unstable)] -use tokio::task::{self, LocalSet}; +use tokio::task::{self, Id, LocalSet}; -#[cfg(tokio_unstable)] mod support { pub mod panic; } -#[cfg(tokio_unstable)] use support::panic::test_panic; -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn() { tokio::spawn(async { println!("task id: {}", task::id()) }) @@ -32,7 +22,6 @@ async fn task_id_spawn() { .unwrap(); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn_blocking() { task::spawn_blocking(|| println!("task id: {}", task::id())) @@ -40,7 +29,6 @@ async fn task_id_spawn_blocking() { .unwrap(); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_collision_current_thread() { let handle1 = tokio::spawn(async { task::id() }); @@ -50,7 +38,6 @@ async fn task_id_collision_current_thread() { assert_ne!(id1.unwrap(), id2.unwrap()); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "multi_thread")] async fn task_id_collision_multi_thread() { let handle1 = tokio::spawn(async { task::id() }); @@ -60,7 +47,6 @@ async fn task_id_collision_multi_thread() { assert_ne!(id1.unwrap(), id2.unwrap()); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_ids_match_current_thread() { let (tx, rx) = oneshot::channel(); @@ -72,7 +58,6 @@ async fn task_ids_match_current_thread() { handle.await.unwrap(); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "multi_thread")] async fn task_ids_match_multi_thread() { let (tx, rx) = oneshot::channel(); @@ -84,10 +69,11 @@ async fn task_ids_match_multi_thread() { handle.await.unwrap(); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_completion() { - struct MyFuture; + struct MyFuture { + tx: Option>, + } impl Future for MyFuture { type Output = (); @@ -99,17 +85,22 @@ async fn task_id_future_destructor_completion() { impl Drop for MyFuture { fn drop(&mut self) { - println!("task id: {}", task::id()); + let _ = self.tx.take().unwrap().send(task::id()); } } - tokio::spawn(MyFuture).await.unwrap(); + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(MyFuture { tx: Some(tx) }); + let id = handle.id(); + handle.await.unwrap(); + assert_eq!(rx.await.unwrap(), id); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_abort() { - struct MyFuture; + struct MyFuture { + tx: Option>, + } impl Future for MyFuture { type Output = (); @@ -120,92 +111,93 @@ async fn task_id_future_destructor_abort() { } impl Drop for MyFuture { fn drop(&mut self) { - println!("task id: {}", task::id()); + let _ = self.tx.take().unwrap().send(task::id()); } } - tokio::spawn(MyFuture).abort(); + let (tx, rx) = oneshot::channel(); + let handle = tokio::spawn(MyFuture { tx: Some(tx) }); + let id = handle.id(); + handle.abort(); + assert_eq!(rx.await.unwrap(), id); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_output_destructor_handle_dropped_before_completion() { - struct MyOutput; + struct MyOutput { + tx: Option>, + } impl Drop for MyOutput { fn drop(&mut self) { - println!("task id: {}", task::id()); + let _ = self.tx.take().unwrap().send(task::id()); } } struct MyFuture { - tx: Option>, + tx: Option>, } impl Future for MyFuture { type Output = MyOutput; fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { - let _ = self.tx.take().unwrap().send(()); - Poll::Ready(MyOutput) - } - } - - impl Drop for MyFuture { - fn drop(&mut self) { - println!("task id: {}", task::id()); + Poll::Ready(MyOutput { tx: self.tx.take() }) } } let (tx, rx) = oneshot::channel(); let handle = tokio::spawn(MyFuture { tx: Some(tx) }); + let id = handle.id(); drop(handle); - rx.await.unwrap(); + assert_eq!(rx.await.unwrap(), id); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_output_destructor_handle_dropped_after_completion() { - struct MyOutput; + struct MyOutput { + tx: Option>, + } impl Drop for MyOutput { fn drop(&mut self) { - println!("task id: {}", task::id()); + let _ = self.tx.take().unwrap().send(task::id()); } } struct MyFuture { - tx: Option>, + tx_output: Option>, + tx_future: Option>, } impl Future for MyFuture { type Output = MyOutput; fn poll(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll { - let _ = self.tx.take().unwrap().send(()); - Poll::Ready(MyOutput) - } - } - - impl Drop for MyFuture { - fn drop(&mut self) { - println!("task id: {}", task::id()); + let _ = self.tx_future.take().unwrap().send(()); + Poll::Ready(MyOutput { + tx: self.tx_output.take(), + }) } } - let (tx, rx) = oneshot::channel(); - let handle = tokio::spawn(MyFuture { tx: Some(tx) }); - rx.await.unwrap(); + let (tx_output, rx_output) = oneshot::channel(); + let (tx_future, rx_future) = oneshot::channel(); + let handle = tokio::spawn(MyFuture { + tx_output: Some(tx_output), + tx_future: Some(tx_future), + }); + let id = handle.id(); + rx_future.await.unwrap(); drop(handle); + assert_eq!(rx_output.await.unwrap(), id); } -#[cfg(tokio_unstable)] #[test] fn task_try_id_outside_task() { assert_eq!(None, task::try_id()); } -#[cfg(tokio_unstable)] #[test] fn task_try_id_inside_block_on() { let rt = Runtime::new().unwrap(); @@ -214,7 +206,6 @@ fn task_try_id_inside_block_on() { }); } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn_local() { LocalSet::new() @@ -226,23 +217,22 @@ async fn task_id_spawn_local() { .await } -#[cfg(tokio_unstable)] #[tokio::test(flavor = "current_thread")] async fn task_id_nested_spawn_local() { LocalSet::new() .run_until(async { task::spawn_local(async { - let outer_id = task::id(); + let parent_id = task::id(); LocalSet::new() .run_until(async { task::spawn_local(async move { - assert_ne!(outer_id, task::id()); + assert_ne!(parent_id, task::id()); }) .await .unwrap(); }) .await; - assert_eq!(outer_id, task::id()); + assert_eq!(parent_id, task::id()); }) .await .unwrap(); @@ -250,7 +240,25 @@ async fn task_id_nested_spawn_local() { .await; } -#[cfg(tokio_unstable)] +#[tokio::test(flavor = "multi_thread")] +async fn task_id_block_in_place_block_on_spawn() { + task::spawn(async { + let parent_id = task::id(); + + task::block_in_place(move || { + let rt = Builder::new_current_thread().build().unwrap(); + rt.block_on(rt.spawn(async move { + assert_ne!(parent_id, task::id()); + })) + .unwrap(); + }); + + assert_eq!(parent_id, task::id()); + }) + .await + .unwrap(); +} + #[test] fn task_id_outside_task_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -263,7 +271,6 @@ fn task_id_outside_task_panic_caller() -> Result<(), Box> { Ok(()) } -#[cfg(tokio_unstable)] #[test] fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -278,21 +285,3 @@ fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { Ok(()) } - -#[cfg(tokio_unstable)] -#[tokio::test(flavor = "multi_thread")] -async fn task_id_block_in_place_block_on_spawn() { - task::spawn(async { - let id1 = task::id(); - - task::block_in_place(|| { - let rt = Builder::new_current_thread().build().unwrap(); - rt.block_on(rt.spawn(async {})).unwrap(); - }); - - let id2 = task::id(); - assert_eq!(id1, id2); - }) - .await - .unwrap(); -} From 33129d842a977031a08108bb3c78348f163a6448 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 13:55:14 -0500 Subject: [PATCH 10/16] more fixes --- tokio/tests/task_id.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 04a223e98de..8ed72bf16b3 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -1,6 +1,6 @@ #![warn(rust_2018_idioms)] #![allow(clippy::declare_interior_mutable_const)] -#![cfg(all(feature = "full", tokio_unstable, not(tokio_wasi)))] +#![cfg(all(feature = "full", tokio_unstable))] use std::error::Error; use std::future::Future; @@ -119,6 +119,7 @@ async fn task_id_future_destructor_abort() { let handle = tokio::spawn(MyFuture { tx: Some(tx) }); let id = handle.id(); handle.abort(); + assert!(handle.await.unwrap_err().is_cancelled()); assert_eq!(rx.await.unwrap(), id); } @@ -146,10 +147,11 @@ async fn task_id_output_destructor_handle_dropped_before_completion() { } } - let (tx, rx) = oneshot::channel(); + let (tx, mut rx) = oneshot::channel(); let handle = tokio::spawn(MyFuture { tx: Some(tx) }); let id = handle.id(); drop(handle); + assert!(rx.try_recv().is_err()); assert_eq!(rx.await.unwrap(), id); } @@ -181,7 +183,7 @@ async fn task_id_output_destructor_handle_dropped_after_completion() { } } - let (tx_output, rx_output) = oneshot::channel(); + let (tx_output, mut rx_output) = oneshot::channel(); let (tx_future, rx_future) = oneshot::channel(); let handle = tokio::spawn(MyFuture { tx_output: Some(tx_output), @@ -189,6 +191,7 @@ async fn task_id_output_destructor_handle_dropped_after_completion() { }); let id = handle.id(); rx_future.await.unwrap(); + assert!(rx_output.try_recv().is_err()); drop(handle); assert_eq!(rx_output.await.unwrap(), id); } @@ -259,6 +262,7 @@ async fn task_id_block_in_place_block_on_spawn() { .unwrap(); } +#[cfg(not(tokio_wasi))] #[test] fn task_id_outside_task_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -271,6 +275,7 @@ fn task_id_outside_task_panic_caller() -> Result<(), Box> { Ok(()) } +#[cfg(not(tokio_wasi))] #[test] fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { From 0d7be89462fbe25b812bd23004bccdf3e285445c Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 14:29:10 -0500 Subject: [PATCH 11/16] make wasm happy --- tokio/tests/task_id.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 8ed72bf16b3..686cbccfe02 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -2,6 +2,7 @@ #![allow(clippy::declare_interior_mutable_const)] #![cfg(all(feature = "full", tokio_unstable))] +#[cfg_attr(tokio_wasm, ignore)] use std::error::Error; use std::future::Future; use std::pin::Pin; @@ -13,6 +14,7 @@ use tokio::task::{self, Id, LocalSet}; mod support { pub mod panic; } +#[cfg_attr(tokio_wasm, ignore)] use support::panic::test_panic; #[tokio::test(flavor = "current_thread")] @@ -22,6 +24,7 @@ async fn task_id_spawn() { .unwrap(); } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn_blocking() { task::spawn_blocking(|| println!("task id: {}", task::id())) @@ -38,6 +41,7 @@ async fn task_id_collision_current_thread() { assert_ne!(id1.unwrap(), id2.unwrap()); } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_collision_multi_thread() { let handle1 = tokio::spawn(async { task::id() }); @@ -58,6 +62,7 @@ async fn task_ids_match_current_thread() { handle.await.unwrap(); } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_ids_match_multi_thread() { let (tx, rx) = oneshot::channel(); @@ -69,6 +74,7 @@ async fn task_ids_match_multi_thread() { handle.await.unwrap(); } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_completion() { struct MyFuture { @@ -96,6 +102,7 @@ async fn task_id_future_destructor_completion() { assert_eq!(rx.await.unwrap(), id); } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_abort() { struct MyFuture { @@ -243,6 +250,7 @@ async fn task_id_nested_spawn_local() { .await; } +#[cfg_attr(tokio_wasm, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_block_in_place_block_on_spawn() { task::spawn(async { @@ -262,7 +270,7 @@ async fn task_id_block_in_place_block_on_spawn() { .unwrap(); } -#[cfg(not(tokio_wasi))] +#[cfg_attr(tokio_wasm, ignore)] #[test] fn task_id_outside_task_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -275,7 +283,7 @@ fn task_id_outside_task_panic_caller() -> Result<(), Box> { Ok(()) } -#[cfg(not(tokio_wasi))] +#[cfg_attr(tokio_wasm, ignore)] #[test] fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { From 287cb618fd46592e0884f51295ed9fecb4aaa81a Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 14:38:20 -0500 Subject: [PATCH 12/16] wasm -> wasi --- tokio/tests/task_id.rs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 686cbccfe02..8f62faae1dc 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -2,7 +2,7 @@ #![allow(clippy::declare_interior_mutable_const)] #![cfg(all(feature = "full", tokio_unstable))] -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] use std::error::Error; use std::future::Future; use std::pin::Pin; @@ -14,7 +14,7 @@ use tokio::task::{self, Id, LocalSet}; mod support { pub mod panic; } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] use support::panic::test_panic; #[tokio::test(flavor = "current_thread")] @@ -24,7 +24,7 @@ async fn task_id_spawn() { .unwrap(); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn_blocking() { task::spawn_blocking(|| println!("task id: {}", task::id())) @@ -41,7 +41,7 @@ async fn task_id_collision_current_thread() { assert_ne!(id1.unwrap(), id2.unwrap()); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_collision_multi_thread() { let handle1 = tokio::spawn(async { task::id() }); @@ -62,7 +62,7 @@ async fn task_ids_match_current_thread() { handle.await.unwrap(); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_ids_match_multi_thread() { let (tx, rx) = oneshot::channel(); @@ -74,7 +74,7 @@ async fn task_ids_match_multi_thread() { handle.await.unwrap(); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_completion() { struct MyFuture { @@ -102,7 +102,7 @@ async fn task_id_future_destructor_completion() { assert_eq!(rx.await.unwrap(), id); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_abort() { struct MyFuture { @@ -250,7 +250,7 @@ async fn task_id_nested_spawn_local() { .await; } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[tokio::test(flavor = "multi_thread")] async fn task_id_block_in_place_block_on_spawn() { task::spawn(async { @@ -270,7 +270,7 @@ async fn task_id_block_in_place_block_on_spawn() { .unwrap(); } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[test] fn task_id_outside_task_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -283,7 +283,7 @@ fn task_id_outside_task_panic_caller() -> Result<(), Box> { Ok(()) } -#[cfg_attr(tokio_wasm, ignore)] +#[cfg_attr(tokio_wasi, ignore)] #[test] fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { From d8c85b134bf34cb2feae8b3dd20b9c80c80f13f0 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 14:50:58 -0500 Subject: [PATCH 13/16] try #[cfg(not(tokio_wasi))] instead of #[cfg_attr(tokio_wasi, ignore)] --- tokio/tests/task_id.rs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 8f62faae1dc..0be6b12e9a6 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -2,7 +2,7 @@ #![allow(clippy::declare_interior_mutable_const)] #![cfg(all(feature = "full", tokio_unstable))] -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] use std::error::Error; use std::future::Future; use std::pin::Pin; @@ -14,7 +14,7 @@ use tokio::task::{self, Id, LocalSet}; mod support { pub mod panic; } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] use support::panic::test_panic; #[tokio::test(flavor = "current_thread")] @@ -24,7 +24,7 @@ async fn task_id_spawn() { .unwrap(); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "current_thread")] async fn task_id_spawn_blocking() { task::spawn_blocking(|| println!("task id: {}", task::id())) @@ -41,7 +41,7 @@ async fn task_id_collision_current_thread() { assert_ne!(id1.unwrap(), id2.unwrap()); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "multi_thread")] async fn task_id_collision_multi_thread() { let handle1 = tokio::spawn(async { task::id() }); @@ -62,7 +62,7 @@ async fn task_ids_match_current_thread() { handle.await.unwrap(); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "multi_thread")] async fn task_ids_match_multi_thread() { let (tx, rx) = oneshot::channel(); @@ -74,7 +74,7 @@ async fn task_ids_match_multi_thread() { handle.await.unwrap(); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_completion() { struct MyFuture { @@ -102,7 +102,7 @@ async fn task_id_future_destructor_completion() { assert_eq!(rx.await.unwrap(), id); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "multi_thread")] async fn task_id_future_destructor_abort() { struct MyFuture { @@ -250,7 +250,7 @@ async fn task_id_nested_spawn_local() { .await; } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[tokio::test(flavor = "multi_thread")] async fn task_id_block_in_place_block_on_spawn() { task::spawn(async { @@ -270,7 +270,7 @@ async fn task_id_block_in_place_block_on_spawn() { .unwrap(); } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[test] fn task_id_outside_task_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { @@ -283,7 +283,7 @@ fn task_id_outside_task_panic_caller() -> Result<(), Box> { Ok(()) } -#[cfg_attr(tokio_wasi, ignore)] +#[cfg(not(tokio_wasi))] #[test] fn task_id_inside_block_on_panic_caller() -> Result<(), Box> { let panic_location_file = test_panic(|| { From 76cb324c3d4ca93ff45f11873d0c6101572326f5 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 14:55:46 -0500 Subject: [PATCH 14/16] one final fix --- tokio/tests/task_id.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index 0be6b12e9a6..befe272a58c 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -208,6 +208,7 @@ fn task_try_id_outside_task() { assert_eq!(None, task::try_id()); } +#[cfg(not(tokio_wasi))] #[test] fn task_try_id_inside_block_on() { let rt = Runtime::new().unwrap(); From 942106f4acc2375cdd38e237abf8498c85c0b815 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 15:02:32 -0500 Subject: [PATCH 15/16] final final --- tokio/tests/task_id.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index befe272a58c..fe6fe930f74 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -7,6 +7,7 @@ use std::error::Error; use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; +#[cfg(not(tokio_wasi))] use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot; use tokio::task::{self, Id, LocalSet}; From 01d84b05f36264d8ba3d2c1544cbceb6015cfee9 Mon Sep 17 00:00:00 2001 From: Abutalib Aghayev Date: Sat, 12 Nov 2022 16:27:06 -0500 Subject: [PATCH 16/16] definitely final --- tokio/tests/task_id.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/tests/task_id.rs b/tokio/tests/task_id.rs index fe6fe930f74..d7b7c0cd812 100644 --- a/tokio/tests/task_id.rs +++ b/tokio/tests/task_id.rs @@ -12,6 +12,7 @@ use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot; use tokio::task::{self, Id, LocalSet}; +#[cfg(not(tokio_wasi))] mod support { pub mod panic; }