From 5b87bc6c8ad1b1a011ab8afe036c534b4835269f Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 17:52:29 +0100 Subject: [PATCH 01/32] rt: add LocalRuntime This change adds LocalRuntime, a new unstable runtime type which cannot be transferred across thread boundaries and supports spawn_local when called from the thread which owns the runtime. The initial set of docs for this are iffy. Documentation is absent right now at the module level, with the docs for the LocalRuntime struct itself being somewhat duplicative of those for the `Runtime` type. This can probably be addressed later as stabilization nears. This API has a few interesting implementation details: - because it was considered beneficial to reuse the same Handle as the normal runtime, it is possible to call spawn_local from a runtime context while on a different thread from the one which drives the runtime and owns it. This forces us to check the thread ID before attempting a local spawn. - An empty LocalOptions struct is passed into the build_local method in order to build the runtime. This will eventually have stuff in it like hooks. Relates to #6739. --- tokio/src/runtime/builder.rs | 72 +++- tokio/src/runtime/handle.rs | 28 +- tokio/src/runtime/local_runtime/mod.rs | 7 + tokio/src/runtime/local_runtime/options.rs | 9 + tokio/src/runtime/local_runtime/runtime.rs | 375 ++++++++++++++++++ tokio/src/runtime/mod.rs | 3 + .../runtime/scheduler/current_thread/mod.rs | 33 ++ tokio/src/runtime/scheduler/mod.rs | 42 ++ tokio/src/runtime/task/list.rs | 20 + tokio/src/task/local.rs | 44 +- 10 files changed, 618 insertions(+), 15 deletions(-) create mode 100644 tokio/src/runtime/local_runtime/mod.rs create mode 100644 tokio/src/runtime/local_runtime/options.rs create mode 100644 tokio/src/runtime/local_runtime/runtime.rs diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index b5bf35d69b4..e6c3bea6a86 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -3,11 +3,16 @@ use crate::runtime::handle::Handle; #[cfg(tokio_unstable)] use crate::runtime::TaskMeta; -use crate::runtime::{blocking, driver, Callback, HistogramBuilder, Runtime, TaskCallback}; +use crate::runtime::{ + blocking, driver, Callback, HistogramBuilder, LocalOptions, LocalRuntime, Runtime, TaskCallback, +}; use crate::util::rand::{RngSeed, RngSeedGenerator}; +use crate::runtime::blocking::BlockingPool; +use crate::runtime::scheduler::CurrentThread; use std::fmt; use std::io; +use std::thread::ThreadId; use std::time::Duration; /// Builds Tokio Runtime with custom configuration values. @@ -800,6 +805,29 @@ impl Builder { } } + /// Creates the configured `LocalRuntime`. + /// + /// The returned `LocalRuntime` instance is ready to spawn tasks. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Builder; + /// + /// let rt = Builder::new_current_thread().build_local(&mut Default::default()).unwrap(); + /// + /// rt.block_on(async { + /// println!("Hello from the Tokio runtime"); + /// }); + /// ``` + #[allow(unused_variables)] + pub fn build_local(&mut self, options: &mut LocalOptions) -> io::Result { + match &self.kind { + Kind::CurrentThread => self.build_current_thread_local_runtime(), + _ => panic!("Only current_thread is supported when building a local runtime"), + } + } + fn get_cfg(&self, workers: usize) -> driver::Cfg { driver::Cfg { enable_pause_time: match self.kind { @@ -1191,8 +1219,39 @@ impl Builder { } fn build_current_thread_runtime(&mut self) -> io::Result { - use crate::runtime::scheduler::{self, CurrentThread}; - use crate::runtime::{runtime::Scheduler, Config}; + use crate::runtime::runtime::Scheduler; + + let (scheduler, handle, blocking_pool) = + self.build_current_thread_runtime_components(None)?; + + Ok(Runtime::from_parts( + Scheduler::CurrentThread(scheduler), + handle, + blocking_pool, + )) + } + + fn build_current_thread_local_runtime(&mut self) -> io::Result { + use crate::runtime::local_runtime::LocalRuntimeScheduler; + + let tid = std::thread::current().id(); + + let (scheduler, handle, blocking_pool) = + self.build_current_thread_runtime_components(Some(tid))?; + + Ok(LocalRuntime::from_parts( + LocalRuntimeScheduler::CurrentThread(scheduler), + handle, + blocking_pool, + )) + } + + fn build_current_thread_runtime_components( + &mut self, + local_tid: Option, + ) -> io::Result<(CurrentThread, Handle, BlockingPool)> { + use crate::runtime::scheduler; + use crate::runtime::Config; let (driver, driver_handle) = driver::Driver::new(self.get_cfg(1))?; @@ -1227,17 +1286,14 @@ impl Builder { seed_generator: seed_generator_1, metrics_poll_count_histogram: self.metrics_poll_count_histogram_builder(), }, + local_tid, ); let handle = Handle { inner: scheduler::Handle::CurrentThread(handle), }; - Ok(Runtime::from_parts( - Scheduler::CurrentThread(scheduler), - handle, - blocking_pool, - )) + Ok((scheduler, handle, blocking_pool)) } fn metrics_poll_count_histogram_builder(&self) -> Option { diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index 9026e8773a0..802b73ee30c 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -250,8 +250,8 @@ impl Handle { /// # Panics /// /// This function panics if the provided future panics, if called within an - /// asynchronous execution context, or if a timer future is executed on a - /// runtime that has been shut down. + /// asynchronous execution context, or if a timer future is executed on a runtime that has been + /// shut down. /// /// # Examples /// @@ -348,6 +348,30 @@ impl Handle { self.inner.spawn(future, id) } + #[track_caller] + pub(crate) unsafe fn spawn_local_named( + &self, + future: F, + _name: Option<&str>, + ) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + let id = crate::runtime::task::Id::next(); + #[cfg(all( + tokio_unstable, + tokio_taskdump, + feature = "rt", + target_os = "linux", + any(target_arch = "aarch64", target_arch = "x86", target_arch = "x86_64") + ))] + let future = super::task::trace::Trace::root(future); + #[cfg(all(tokio_unstable, feature = "tracing"))] + let future = crate::util::trace::task(future, "task", _name, id.as_u64()); + self.inner.spawn_local(future, id) + } + /// Returns the flavor of the current `Runtime`. /// /// # Examples diff --git a/tokio/src/runtime/local_runtime/mod.rs b/tokio/src/runtime/local_runtime/mod.rs new file mode 100644 index 00000000000..1ea7693f292 --- /dev/null +++ b/tokio/src/runtime/local_runtime/mod.rs @@ -0,0 +1,7 @@ +mod runtime; + +mod options; + +pub use options::LocalOptions; +pub use runtime::LocalRuntime; +pub(super) use runtime::LocalRuntimeScheduler; diff --git a/tokio/src/runtime/local_runtime/options.rs b/tokio/src/runtime/local_runtime/options.rs new file mode 100644 index 00000000000..1ff0d59b2cd --- /dev/null +++ b/tokio/src/runtime/local_runtime/options.rs @@ -0,0 +1,9 @@ +/// LocalRuntime-only config options +/// +/// Currently, there are no such options, but in the future, things like `!Send + !Sync` hooks may +/// be added. +#[derive(Default, Debug)] +#[non_exhaustive] +pub struct LocalOptions { + // todo add local hooks at a later point +} diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs new file mode 100644 index 00000000000..277d225b95c --- /dev/null +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -0,0 +1,375 @@ +#![allow(irrefutable_let_patterns)] + +use crate::runtime::blocking::BlockingPool; +use crate::runtime::scheduler::CurrentThread; +use crate::runtime::{context, Builder, EnterGuard, Handle, BOX_FUTURE_THRESHOLD}; +use crate::task::JoinHandle; + +use std::future::Future; +use std::marker::PhantomData; +use std::time::Duration; + +/// A local Tokio runtime. +/// +/// This runtime is identical to a current_thread [runtime], save for not being `!Send + !Sync`, +/// and supporting spawn_local. +/// +/// For more general information on how to use runtimes, see the [module] docs. +/// +/// [runtime]: crate::runtime::Runtime +/// [module]: crate::runtime +#[derive(Debug)] +#[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] +pub struct LocalRuntime { + /// Task scheduler + scheduler: LocalRuntimeScheduler, + + /// Handle to runtime, also contains driver handles + handle: Handle, + + /// Blocking pool handle, used to signal shutdown + blocking_pool: BlockingPool, + + /// Marker used to make this !Send and !Sync. + _phantom: PhantomData<*mut u8>, +} + +/// The runtime scheduler is always a current_thread scheduler right now. +#[derive(Debug)] +pub(crate) enum LocalRuntimeScheduler { + /// Execute all tasks on the current-thread. + CurrentThread(CurrentThread), +} + +impl LocalRuntime { + pub(crate) fn from_parts( + scheduler: LocalRuntimeScheduler, + handle: Handle, + blocking_pool: BlockingPool, + ) -> LocalRuntime { + LocalRuntime { + scheduler, + handle, + blocking_pool, + _phantom: Default::default(), + } + } + + /// Creates a new local runtime instance with default configuration values. + /// + /// This results in the scheduler, I/O driver, and time driver being + /// initialized. + /// + /// When a more complex configuration is necessary, the [runtime builder] may be used. + /// + /// See [module level][mod] documentation for more details. + /// + /// # Examples + /// + /// Creating a new `LocalRuntime` with default configuration values. + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// + /// let rt = LocalRuntime::new() + /// .unwrap(); + /// + /// // Use the runtime... + /// ``` + /// + /// [mod]: crate::runtime + /// [runtime builder]: crate::runtime::Builder + pub fn new() -> std::io::Result { + Builder::new_current_thread() + .enable_all() + .build_local(&mut Default::default()) + } + + /// Returns a handle to the runtime's spawner. + /// + /// The returned handle can be used to spawn tasks that run on this runtime, and can + /// be cloned to allow moving the `Handle` to other threads. + /// + /// Local tasks cannot be spawned on this handle. + /// + /// Calling [`Handle::block_on`] on a handle to a `LocalRuntime` is error-prone. + /// Refer to the documentation of [`Handle::block_on`] for more. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// + /// let rt = LocalRuntime::new() + /// .unwrap(); + /// + /// let handle = rt.handle(); + /// + /// // Use the handle... + /// ``` + pub fn handle(&self) -> &Handle { + &self.handle + } + + /// Spawns a future onto the LocalRuntime. + /// + /// See the documentation for the equivalent method on [Runtime] for more information + /// + /// [Runtime]: crate::runtime::Runtime::spawn + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// + /// # fn dox() { + /// // Create the runtime + /// let rt = LocalRuntime::new().unwrap(); + /// + /// // Spawn a future onto the runtime + /// rt.spawn(async { + /// println!("now running on a worker thread"); + /// }); + /// # } + /// ``` + #[track_caller] + pub fn spawn(&self, future: F) -> JoinHandle + where + F: Future + Send + 'static, + F::Output: Send + 'static, + { + if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { + self.handle.spawn_named(Box::pin(future), None) + } else { + self.handle.spawn_named(future, None) + } + } + + /// Spawns a task which isn't `!Send + Sync` on the runtime. + #[track_caller] + pub fn spawn_local(&self, future: F) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + // safety: spawn_local can only be called from LocalRuntime, which this is + unsafe { + if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { + self.handle.spawn_local_named(Box::pin(future), None) + } else { + self.handle.spawn_local_named(future, None) + } + } + } + + /// Runs the provided function on an executor dedicated to blocking operations. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// + /// # fn dox() { + /// // Create the runtime + /// let rt = LocalRuntime::new().unwrap(); + /// + /// // Spawn a blocking function onto the runtime + /// rt.spawn_blocking(|| { + /// println!("now running on a worker thread"); + /// }); + /// # } + /// ``` + #[track_caller] + pub fn spawn_blocking(&self, func: F) -> JoinHandle + where + F: FnOnce() -> R + Send + 'static, + R: Send + 'static, + { + self.handle.spawn_blocking(func) + } + + /// Runs a future to completion on the Tokio runtime. This is the + /// runtime's entry point. + /// + /// See the documentation for the equivalent method on [Runtime] for more information. + /// + /// [Runtime]: crate::runtime::Runtime::block_on + /// + /// # Examples + /// + /// ```no_run + /// use tokio::runtime::LocalRuntime; + /// + /// // Create the runtime + /// let rt = LocalRuntime::new().unwrap(); + /// + /// // Execute the future, blocking the current thread until completion + /// rt.block_on(async { + /// println!("hello"); + /// }); + /// ``` + #[track_caller] + pub fn block_on(&self, future: F) -> F::Output { + if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { + self.block_on_inner(Box::pin(future)) + } else { + self.block_on_inner(future) + } + } + + #[track_caller] + fn block_on_inner(&self, future: F) -> F::Output { + #[cfg(all( + tokio_unstable, + tokio_taskdump, + feature = "rt", + target_os = "linux", + any(target_arch = "aarch64", target_arch = "x86", target_arch = "x86_64") + ))] + let future = super::task::trace::Trace::root(future); + + #[cfg(all(tokio_unstable, feature = "tracing"))] + let future = crate::util::trace::task( + future, + "block_on", + None, + crate::runtime::task::Id::next().as_u64(), + ); + + let _enter = self.enter(); + + if let LocalRuntimeScheduler::CurrentThread(exec) = &self.scheduler { + exec.block_on(&self.handle.inner, future) + } else { + unreachable!("LocalRuntime only supports current_thread") + } + } + + /// Enters the runtime context. + /// + /// This allows you to construct types that must have an executor + /// available on creation such as [`Sleep`] or [`TcpStream`]. It will + /// also allow you to call methods such as [`tokio::spawn`]. + /// + /// [`Sleep`]: struct@crate::time::Sleep + /// [`TcpStream`]: struct@crate::net::TcpStream + /// [`tokio::spawn`]: fn@crate::spawn + /// + /// # Example + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// use tokio::task::JoinHandle; + /// + /// fn function_that_spawns(msg: String) -> JoinHandle<()> { + /// // Had we not used `rt.enter` below, this would panic. + /// tokio::spawn(async move { + /// println!("{}", msg); + /// }) + /// } + /// + /// fn main() { + /// let rt = LocalRuntime::new().unwrap(); + /// + /// let s = "Hello World!".to_string(); + /// + /// // By entering the context, we tie `tokio::spawn` to this executor. + /// let _guard = rt.enter(); + /// let handle = function_that_spawns(s); + /// + /// // Wait for the task before we end the test. + /// rt.block_on(handle).unwrap(); + /// } + /// ``` + pub fn enter(&self) -> EnterGuard<'_> { + self.handle.enter() + } + + /// Shuts down the runtime, waiting for at most `duration` for all spawned + /// work to stop. + /// + /// See the [struct level documentation](LocalRuntime#shutdown) for more details. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// use tokio::task; + /// + /// use std::thread; + /// use std::time::Duration; + /// + /// fn main() { + /// let runtime = LocalRuntime::new().unwrap(); + /// + /// runtime.block_on(async move { + /// task::spawn_blocking(move || { + /// thread::sleep(Duration::from_secs(10_000)); + /// }); + /// }); + /// + /// runtime.shutdown_timeout(Duration::from_millis(100)); + /// } + /// ``` + pub fn shutdown_timeout(mut self, duration: Duration) { + // Wakeup and shutdown all the worker threads + self.handle.inner.shutdown(); + self.blocking_pool.shutdown(Some(duration)); + } + + /// Shuts down the runtime, without waiting for any spawned work to stop. + /// + /// This can be useful if you want to drop a runtime from within another runtime. + /// Normally, dropping a runtime will block indefinitely for spawned blocking tasks + /// to complete, which would normally not be permitted within an asynchronous context. + /// By calling `shutdown_background()`, you can drop the runtime from such a context. + /// + /// Note however, that because we do not wait for any blocking tasks to complete, this + /// may result in a resource leak (in that any blocking tasks are still running until they + /// return. + /// + /// See the [struct level documentation](LocalRuntime#shutdown) for more details. + /// + /// This function is equivalent to calling `shutdown_timeout(Duration::from_nanos(0))`. + /// + /// ``` + /// use tokio::runtime::LocalRuntime; + /// + /// fn main() { + /// let runtime = LocalRuntime::new().unwrap(); + /// + /// runtime.block_on(async move { + /// let inner_runtime = LocalRuntime::new().unwrap(); + /// // ... + /// inner_runtime.shutdown_background(); + /// }); + /// } + /// ``` + pub fn shutdown_background(self) { + self.shutdown_timeout(Duration::from_nanos(0)); + } + + /// Returns a view that lets you get information about how the runtime + /// is performing. + pub fn metrics(&self) -> crate::runtime::RuntimeMetrics { + self.handle.metrics() + } +} + +#[allow(clippy::single_match)] // there are comments in the error branch, so we don't want if-let +impl Drop for LocalRuntime { + fn drop(&mut self) { + if let LocalRuntimeScheduler::CurrentThread(current_thread) = &mut self.scheduler { + // This ensures that tasks spawned on the current-thread + // runtime are dropped inside the runtime's context. + let _guard = context::try_set_current(&self.handle.inner); + current_thread.shutdown(&self.handle.inner); + } else { + unreachable!("LocalRuntime only supports current-thread") + } + } +} + +impl std::panic::UnwindSafe for LocalRuntime {} + +impl std::panic::RefUnwindSafe for LocalRuntime {} diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 3f2467f6dbc..c8efbe2f1cd 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -372,6 +372,9 @@ cfg_rt! { pub use self::builder::UnhandledPanic; pub use crate::util::rand::RngSeed; + + mod local_runtime; + pub use local_runtime::{LocalRuntime, LocalOptions}; } cfg_taskdump! { diff --git a/tokio/src/runtime/scheduler/current_thread/mod.rs b/tokio/src/runtime/scheduler/current_thread/mod.rs index 9959dff8e46..f64fa449b29 100644 --- a/tokio/src/runtime/scheduler/current_thread/mod.rs +++ b/tokio/src/runtime/scheduler/current_thread/mod.rs @@ -18,6 +18,7 @@ use std::future::{poll_fn, Future}; use std::sync::atomic::Ordering::{AcqRel, Release}; use std::task::Poll::{Pending, Ready}; use std::task::Waker; +use std::thread::ThreadId; use std::time::Duration; use std::{fmt, thread}; @@ -47,6 +48,9 @@ pub(crate) struct Handle { /// User-supplied hooks to invoke for things pub(crate) task_hooks: TaskHooks, + + /// If this is a LocalRuntime, flags the owning thread ID. + pub(crate) local_tid: Option, } /// Data required for executing the scheduler. The struct is passed around to @@ -127,6 +131,7 @@ impl CurrentThread { blocking_spawner: blocking::Spawner, seed_generator: RngSeedGenerator, config: Config, + local_tid: Option, ) -> (CurrentThread, Arc) { let worker_metrics = WorkerMetrics::from_config(&config); worker_metrics.set_thread_id(thread::current().id()); @@ -152,6 +157,7 @@ impl CurrentThread { driver: driver_handle, blocking_spawner, seed_generator, + local_tid, }); let core = AtomicCell::new(Some(Box::new(Core { @@ -458,6 +464,33 @@ impl Handle { handle } + /// Spawn a task which isn't safe to send across thread boundaries onto the runtime. + /// + /// # Safety + pub(crate) unsafe fn spawn_local( + me: &Arc, + future: F, + id: crate::runtime::task::Id, + ) -> JoinHandle + where + F: crate::future::Future + 'static, + F::Output: 'static, + { + let (handle, notified) = me.shared.owned.bind_local(future, me.clone(), id); + + me.task_hooks.spawn(&TaskMeta { + #[cfg(tokio_unstable)] + id, + _phantom: Default::default(), + }); + + if let Some(notified) = notified { + me.schedule(notified); + } + + handle + } + /// Capture a snapshot of this runtime's state. #[cfg(all( tokio_unstable, diff --git a/tokio/src/runtime/scheduler/mod.rs b/tokio/src/runtime/scheduler/mod.rs index ada8efbad63..749d85525e5 100644 --- a/tokio/src/runtime/scheduler/mod.rs +++ b/tokio/src/runtime/scheduler/mod.rs @@ -113,6 +113,31 @@ cfg_rt! { match_flavor!(self, Handle(h) => &h.blocking_spawner) } + pub(crate) fn is_local(&self) -> bool { + match self { + Handle::CurrentThread(h) => h.local_tid.is_some(), + + #[cfg(feature = "rt-multi-thread")] + Handle::MultiThread(_) => false, + + #[cfg(all(tokio_unstable, feature = "rt-multi-thread"))] + Handle::MultiThreadAlt(_) => false, + } + } + + /// Returns true if this is a local runtime and the runtime is owned by the current thread. + pub(crate) fn can_spawn_local_on_local_runtime(&self) -> bool { + match self { + Handle::CurrentThread(h) => h.local_tid.map(|x| std::thread::current().id() == x).unwrap_or(false), + + #[cfg(feature = "rt-multi-thread")] + Handle::MultiThread(_) => false, + + #[cfg(all(tokio_unstable, feature = "rt-multi-thread"))] + Handle::MultiThreadAlt(_) => false, + } + } + pub(crate) fn spawn(&self, future: F, id: Id) -> JoinHandle where F: Future + Send + 'static, @@ -129,6 +154,23 @@ cfg_rt! { } } + /// Spawn a local task + /// + /// # Safety + /// This should only be called in LocalRuntime if the runtime has been verified to be owned + /// by the current thread. + pub(crate) unsafe fn spawn_local(&self, future: F, id: Id) -> JoinHandle + where + F: Future + 'static, + F::Output: 'static, + { + if let Handle::CurrentThread(h) = self { + current_thread::Handle::spawn_local(h, future, id) + } else { + panic!("Only current_thread and LocalSet have spawn_local internals implemented") + } + } + pub(crate) fn shutdown(&self) { match *self { Handle::CurrentThread(_) => {}, diff --git a/tokio/src/runtime/task/list.rs b/tokio/src/runtime/task/list.rs index 988d422836d..273ab60fb8c 100644 --- a/tokio/src/runtime/task/list.rs +++ b/tokio/src/runtime/task/list.rs @@ -102,6 +102,26 @@ impl OwnedTasks { (join, notified) } + /// Bind a task that isn't safe to transfer across thread boundaries. + /// + /// # Safety + /// Only use this in LocalRuntime where the task cannot move + pub(crate) unsafe fn bind_local( + &self, + task: T, + scheduler: S, + id: super::Id, + ) -> (JoinHandle, Option>) + where + S: Schedule, + T: Future + 'static, + T::Output: 'static, + { + let (task, notified, join) = super::new_task(task, scheduler, id); + let notified = unsafe { self.bind_inner(task, notified) }; + (join, notified) + } + /// The part of `bind` that's the same for every type of future. unsafe fn bind_inner(&self, task: Task, notified: Notified) -> Option> where diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index d5341937893..dd785003e3d 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -322,7 +322,7 @@ impl<'a> Drop for LocalDataEnterGuard<'a> { } cfg_rt! { - /// Spawns a `!Send` future on the current [`LocalSet`]. + /// Spawns a `!Send` future on the current [`LocalSet`] or [`LocalRuntime`]. /// /// The spawned future will run on the same thread that called `spawn_local`. /// @@ -362,6 +362,7 @@ cfg_rt! { /// ``` /// /// [`LocalSet`]: struct@crate::task::LocalSet + /// [`LocalRuntime`]: struct@crate::runtime::LocalRuntime /// [`tokio::spawn`]: fn@crate::task::spawn #[track_caller] pub fn spawn_local(future: F) -> JoinHandle @@ -383,10 +384,43 @@ cfg_rt! { where F: Future + 'static, F::Output: 'static { - match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { - None => panic!("`spawn_local` called from outside of a `task::LocalSet`"), - Some(cx) => cx.spawn(future, meta) - } + use crate::runtime::{context, task}; + + let res = context::with_current(|handle| { + Some(if handle.is_local() { + if !handle.can_spawn_local_on_local_runtime() { + return None; + } + #[cfg(all( + tokio_unstable, + tokio_taskdump, + feature = "rt", + target_os = "linux", + any( + target_arch = "aarch64", + target_arch = "x86", + target_arch = "x86_64" + ) + ))] + let future = task::trace::Trace::root(future); + let id = task::Id::next(); + let task = crate::util::trace::task(future, "task", name, id.as_u64()); + + // safety: we have verified that this is a LocalRuntime owned by the current thread + unsafe { handle.spawn_local(task, id) } + } else { + match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { + None => panic!("`spawn_local` called from outside of a `task::LocalSet` or LocalRuntime"), + Some(cx) => cx.spawn(future, meta) + } + }) + }); + + match res { + Ok(None) => panic!("Local tasks can only be spawned on a LocalRuntime from the thread the runtime was created on"), + Ok(Some(join_handle)) => join_handle, + Err(e) => panic!("{}", e), + } } } From 1ba8bc07e99573eaaeb96dbb611b77c339dae1f7 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 18:01:42 +0100 Subject: [PATCH 02/32] fix oversite with tokio_unstable gates leading to compile failure --- tokio/src/runtime/builder.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index e6c3bea6a86..9d00deb4418 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -1,11 +1,9 @@ #![cfg_attr(loom, allow(unused_imports))] use crate::runtime::handle::Handle; +use crate::runtime::{blocking, driver, Callback, HistogramBuilder, Runtime, TaskCallback}; #[cfg(tokio_unstable)] -use crate::runtime::TaskMeta; -use crate::runtime::{ - blocking, driver, Callback, HistogramBuilder, LocalOptions, LocalRuntime, Runtime, TaskCallback, -}; +use crate::runtime::{LocalOptions, LocalRuntime, TaskMeta}; use crate::util::rand::{RngSeed, RngSeedGenerator}; use crate::runtime::blocking::BlockingPool; @@ -821,6 +819,8 @@ impl Builder { /// }); /// ``` #[allow(unused_variables)] + #[cfg(tokio_unstable)] + #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] pub fn build_local(&mut self, options: &mut LocalOptions) -> io::Result { match &self.kind { Kind::CurrentThread => self.build_current_thread_local_runtime(), @@ -1231,6 +1231,7 @@ impl Builder { )) } + #[cfg(tokio_unstable)] fn build_current_thread_local_runtime(&mut self) -> io::Result { use crate::runtime::local_runtime::LocalRuntimeScheduler; From e9ca7b845376ab0b129958a40b42b4cd174cea4c Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 18:05:27 +0100 Subject: [PATCH 03/32] appease clippy --- tokio/src/runtime/handle.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index 802b73ee30c..ae8f5969312 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -349,6 +349,7 @@ impl Handle { } #[track_caller] + #[allow(dead_code)] pub(crate) unsafe fn spawn_local_named( &self, future: F, From 439c0840cfcd8c233dd6b06c13c53db5de199a1e Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 18:31:26 +0100 Subject: [PATCH 04/32] fix paths --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 277d225b95c..524ae4274ce 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -226,7 +226,7 @@ impl LocalRuntime { target_os = "linux", any(target_arch = "aarch64", target_arch = "x86", target_arch = "x86_64") ))] - let future = super::task::trace::Trace::root(future); + let future = crate::runtime::task::trace::Trace::root(future); #[cfg(all(tokio_unstable, feature = "tracing"))] let future = crate::util::trace::task( From 20e7bdac959de0761ab8b8988307032de6478b95 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 19:12:39 +0100 Subject: [PATCH 05/32] fix localset spawn --- tokio/src/task/local.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index dd785003e3d..a8c096aed49 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -386,11 +386,16 @@ cfg_rt! { { use crate::runtime::{context, task}; + let mut future = Some(future); + let res = context::with_current(|handle| { Some(if handle.is_local() { if !handle.can_spawn_local_on_local_runtime() { return None; } + + let future = future.take().unwrap(); + #[cfg(all( tokio_unstable, tokio_taskdump, @@ -411,7 +416,7 @@ cfg_rt! { } else { match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { None => panic!("`spawn_local` called from outside of a `task::LocalSet` or LocalRuntime"), - Some(cx) => cx.spawn(future, meta) + Some(cx) => cx.spawn(future.take().unwrap(), meta) } }) }); @@ -419,7 +424,10 @@ cfg_rt! { match res { Ok(None) => panic!("Local tasks can only be spawned on a LocalRuntime from the thread the runtime was created on"), Ok(Some(join_handle)) => join_handle, - Err(e) => panic!("{}", e), + Err(_) => match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { + None => panic!("`spawn_local` called from outside of a `task::LocalSet` or LocalRuntime"), + Some(cx) => cx.spawn(future.unwrap(), name) + } } } } From 5aa073b015bd483bf822dd4cf4a0555a9f3db4e7 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 19:17:18 +0100 Subject: [PATCH 06/32] no rustc, i actually want it this way to prevent this from accidentally getting broken in the future --- tokio/src/runtime/scheduler/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tokio/src/runtime/scheduler/mod.rs b/tokio/src/runtime/scheduler/mod.rs index 749d85525e5..f33c0caf148 100644 --- a/tokio/src/runtime/scheduler/mod.rs +++ b/tokio/src/runtime/scheduler/mod.rs @@ -159,6 +159,7 @@ cfg_rt! { /// # Safety /// This should only be called in LocalRuntime if the runtime has been verified to be owned /// by the current thread. + #[allow(irrefutable_let_patterns)] pub(crate) unsafe fn spawn_local(&self, future: F, id: Id) -> JoinHandle where F: Future + 'static, From 169578f45a1947370988cf5891e5ff06393340a3 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 19:27:57 +0100 Subject: [PATCH 07/32] useless lints --- tokio/src/runtime/builder.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index 9d00deb4418..6126b51d0b7 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -818,7 +818,7 @@ impl Builder { /// println!("Hello from the Tokio runtime"); /// }); /// ``` - #[allow(unused_variables)] + #[allow(unused_variables, unreachable_patterns)] #[cfg(tokio_unstable)] #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] pub fn build_local(&mut self, options: &mut LocalOptions) -> io::Result { From 2faea81dd03973729eb4dbd75e7d6fec5baa2008 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 19:38:32 +0100 Subject: [PATCH 08/32] i dislike having spellcheck --- tokio/src/runtime/local_runtime/options.rs | 2 +- tokio/src/runtime/local_runtime/runtime.rs | 4 ++-- tokio/src/runtime/scheduler/current_thread/mod.rs | 2 +- tokio/src/runtime/scheduler/mod.rs | 2 +- tokio/src/runtime/task/list.rs | 2 +- tokio/src/task/local.rs | 2 +- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tokio/src/runtime/local_runtime/options.rs b/tokio/src/runtime/local_runtime/options.rs index 1ff0d59b2cd..268a4658fab 100644 --- a/tokio/src/runtime/local_runtime/options.rs +++ b/tokio/src/runtime/local_runtime/options.rs @@ -1,4 +1,4 @@ -/// LocalRuntime-only config options +/// `LocalRuntime`-only config options /// /// Currently, there are no such options, but in the future, things like `!Send + !Sync` hooks may /// be added. diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 524ae4274ce..8a4dc75ab3b 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -111,7 +111,7 @@ impl LocalRuntime { &self.handle } - /// Spawns a future onto the LocalRuntime. + /// Spawns a future onto the `LocalRuntime`. /// /// See the documentation for the equivalent method on [Runtime] for more information /// @@ -152,7 +152,7 @@ impl LocalRuntime { F: Future + 'static, F::Output: 'static, { - // safety: spawn_local can only be called from LocalRuntime, which this is + // safety: spawn_local can only be called from `LocalRuntime`, which this is unsafe { if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { self.handle.spawn_local_named(Box::pin(future), None) diff --git a/tokio/src/runtime/scheduler/current_thread/mod.rs b/tokio/src/runtime/scheduler/current_thread/mod.rs index f64fa449b29..8dbbfa142ab 100644 --- a/tokio/src/runtime/scheduler/current_thread/mod.rs +++ b/tokio/src/runtime/scheduler/current_thread/mod.rs @@ -49,7 +49,7 @@ pub(crate) struct Handle { /// User-supplied hooks to invoke for things pub(crate) task_hooks: TaskHooks, - /// If this is a LocalRuntime, flags the owning thread ID. + /// If this is a `LocalRuntime`, flags the owning thread ID. pub(crate) local_tid: Option, } diff --git a/tokio/src/runtime/scheduler/mod.rs b/tokio/src/runtime/scheduler/mod.rs index f33c0caf148..e0a1b20b5bc 100644 --- a/tokio/src/runtime/scheduler/mod.rs +++ b/tokio/src/runtime/scheduler/mod.rs @@ -157,7 +157,7 @@ cfg_rt! { /// Spawn a local task /// /// # Safety - /// This should only be called in LocalRuntime if the runtime has been verified to be owned + /// This should only be called in `LocalRuntime` if the runtime has been verified to be owned /// by the current thread. #[allow(irrefutable_let_patterns)] pub(crate) unsafe fn spawn_local(&self, future: F, id: Id) -> JoinHandle diff --git a/tokio/src/runtime/task/list.rs b/tokio/src/runtime/task/list.rs index 273ab60fb8c..54bfc01aafb 100644 --- a/tokio/src/runtime/task/list.rs +++ b/tokio/src/runtime/task/list.rs @@ -105,7 +105,7 @@ impl OwnedTasks { /// Bind a task that isn't safe to transfer across thread boundaries. /// /// # Safety - /// Only use this in LocalRuntime where the task cannot move + /// Only use this in `LocalRuntime` where the task cannot move pub(crate) unsafe fn bind_local( &self, task: T, diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index a8c096aed49..e30d21a88b4 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -411,7 +411,7 @@ cfg_rt! { let id = task::Id::next(); let task = crate::util::trace::task(future, "task", name, id.as_u64()); - // safety: we have verified that this is a LocalRuntime owned by the current thread + // safety: we have verified that this is a `LocalRuntime` owned by the current thread unsafe { handle.spawn_local(task, id) } } else { match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { From d7fa83e0a128bca1231866af847b27ede4af4d99 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 30 Aug 2024 19:45:36 +0100 Subject: [PATCH 09/32] i dislike having spellcheck even more --- tokio/src/runtime/local_runtime/runtime.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 8a4dc75ab3b..b7610768dbb 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -11,8 +11,8 @@ use std::time::Duration; /// A local Tokio runtime. /// -/// This runtime is identical to a current_thread [runtime], save for not being `!Send + !Sync`, -/// and supporting spawn_local. +/// This runtime is identical to a `current_thread` [runtime], save for not being `!Send + !Sync`, +/// and supporting `spawn_local`. /// /// For more general information on how to use runtimes, see the [module] docs. /// @@ -34,7 +34,7 @@ pub struct LocalRuntime { _phantom: PhantomData<*mut u8>, } -/// The runtime scheduler is always a current_thread scheduler right now. +/// The runtime scheduler is always a `current_thread` scheduler right now. #[derive(Debug)] pub(crate) enum LocalRuntimeScheduler { /// Execute all tasks on the current-thread. From 29d72ba809fa9f8518561150070dc2fe49a7efb5 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Tue, 10 Sep 2024 09:50:43 -0500 Subject: [PATCH 10/32] fix typo Co-authored-by: Yotam Ofek --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index b7610768dbb..a82d17b33e3 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -145,7 +145,7 @@ impl LocalRuntime { } } - /// Spawns a task which isn't `!Send + Sync` on the runtime. + /// Spawns a task which isn't `Send + Sync` on the runtime. #[track_caller] pub fn spawn_local(&self, future: F) -> JoinHandle where From 0dcfcd98c7615437572b5665a80eebb153f29620 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Tue, 10 Sep 2024 10:12:20 -0500 Subject: [PATCH 11/32] get rid of spawn, improve docs for spawn_local --- tokio/src/runtime/local_runtime/runtime.rs | 23 +++++----------------- 1 file changed, 5 insertions(+), 18 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index a82d17b33e3..2470947fda1 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -111,11 +111,12 @@ impl LocalRuntime { &self.handle } - /// Spawns a future onto the `LocalRuntime`. + /// Spawns a task which isn't `Send + Sync` on the runtime. /// - /// See the documentation for the equivalent method on [Runtime] for more information + /// This is analogous to the [spawn] method on the standard [Runtime]. /// - /// [Runtime]: crate::runtime::Runtime::spawn + /// [spawn]: crate::runtime::Runtime::spawn + /// [Runtime]: crate::runtime::Runtime /// /// # Examples /// @@ -127,26 +128,12 @@ impl LocalRuntime { /// let rt = LocalRuntime::new().unwrap(); /// /// // Spawn a future onto the runtime - /// rt.spawn(async { + /// rt.spawn_local(async { /// println!("now running on a worker thread"); /// }); /// # } /// ``` #[track_caller] - pub fn spawn(&self, future: F) -> JoinHandle - where - F: Future + Send + 'static, - F::Output: Send + 'static, - { - if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { - self.handle.spawn_named(Box::pin(future), None) - } else { - self.handle.spawn_named(future, None) - } - } - - /// Spawns a task which isn't `Send + Sync` on the runtime. - #[track_caller] pub fn spawn_local(&self, future: F) -> JoinHandle where F: Future + 'static, From 662e37e1545e75e12038aedd8cbebf19ef8b761e Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Mon, 16 Sep 2024 11:36:37 -0500 Subject: [PATCH 12/32] Update tokio/src/runtime/local_runtime/runtime.rs Co-authored-by: Alice Ryhl --- tokio/src/runtime/local_runtime/runtime.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 2470947fda1..6f413f90dfd 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -111,9 +111,9 @@ impl LocalRuntime { &self.handle } - /// Spawns a task which isn't `Send + Sync` on the runtime. + /// Spawns a task on the runtime. /// - /// This is analogous to the [spawn] method on the standard [Runtime]. + /// This is analogous to the [`spawn`] method on the standard [`Runtime`], but works even if the task is not threads safe. /// /// [spawn]: crate::runtime::Runtime::spawn /// [Runtime]: crate::runtime::Runtime From 17a3862fcaa420261f25f68385a66517e6eeef5a Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Mon, 16 Sep 2024 11:38:43 -0500 Subject: [PATCH 13/32] Update tokio/src/runtime/local_runtime/runtime.rs Co-authored-by: Alice Ryhl --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 6f413f90dfd..23a07ffc255 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -90,7 +90,7 @@ impl LocalRuntime { /// The returned handle can be used to spawn tasks that run on this runtime, and can /// be cloned to allow moving the `Handle` to other threads. /// - /// Local tasks cannot be spawned on this handle. + /// As the handle can be sent to other threads, it can only be used to spawn tasks that are `Send`. /// /// Calling [`Handle::block_on`] on a handle to a `LocalRuntime` is error-prone. /// Refer to the documentation of [`Handle::block_on`] for more. From 11715ac99baf7bcfbf8c3b5575be2248ba8cc6cf Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 14:55:32 -0500 Subject: [PATCH 14/32] take LocalOptions by reference in build_local --- tokio/src/runtime/builder.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index 6126b51d0b7..8519f29661e 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -821,7 +821,7 @@ impl Builder { #[allow(unused_variables, unreachable_patterns)] #[cfg(tokio_unstable)] #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] - pub fn build_local(&mut self, options: &mut LocalOptions) -> io::Result { + pub fn build_local(&mut self, options: &LocalOptions) -> io::Result { match &self.kind { Kind::CurrentThread => self.build_current_thread_local_runtime(), _ => panic!("Only current_thread is supported when building a local runtime"), From 23396465cf2476c9dd782c9beccebb723a81c904 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 14:55:50 -0500 Subject: [PATCH 15/32] make LocalOptions !Send + !Sync --- tokio/src/runtime/local_runtime/options.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/options.rs b/tokio/src/runtime/local_runtime/options.rs index 268a4658fab..ed25d9ccd44 100644 --- a/tokio/src/runtime/local_runtime/options.rs +++ b/tokio/src/runtime/local_runtime/options.rs @@ -1,3 +1,5 @@ +use std::marker::PhantomData; + /// `LocalRuntime`-only config options /// /// Currently, there are no such options, but in the future, things like `!Send + !Sync` hooks may @@ -5,5 +7,6 @@ #[derive(Default, Debug)] #[non_exhaustive] pub struct LocalOptions { - // todo add local hooks at a later point + /// Marker used to make this !Send and !Sync. + _phantom: PhantomData<*mut u8>, } From 073a04924d46b27ae7c12ddb0571594c89220664 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 14:59:16 -0500 Subject: [PATCH 16/32] fix safety docs --- tokio/src/runtime/scheduler/current_thread/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tokio/src/runtime/scheduler/current_thread/mod.rs b/tokio/src/runtime/scheduler/current_thread/mod.rs index 8dbbfa142ab..d68ba6c27fd 100644 --- a/tokio/src/runtime/scheduler/current_thread/mod.rs +++ b/tokio/src/runtime/scheduler/current_thread/mod.rs @@ -467,6 +467,9 @@ impl Handle { /// Spawn a task which isn't safe to send across thread boundaries onto the runtime. /// /// # Safety + /// This should only be used when this is a LocalRuntime or in another case where the runtime + /// provably cannot be driven from or moved to different threads from the one on which the task + /// is spawned. pub(crate) unsafe fn spawn_local( me: &Arc, future: F, From fdb49762750ea7e1494498b328d8f3fdef39bb83 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:02:52 -0500 Subject: [PATCH 17/32] cleanup spawn_blocking docs --- tokio/src/runtime/local_runtime/runtime.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 23a07ffc255..073d428b8ad 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -149,7 +149,13 @@ impl LocalRuntime { } } - /// Runs the provided function on an executor dedicated to blocking operations. + /// Runs the provided function on a thread from a dedicated blocking thread pool. + /// + /// This function _will_ be run on another thread. + /// + /// See the documentation in the non-local runtime for more information. + /// + /// [Runtime]: crate::runtime::Runtime::spawn_blocking /// /// # Examples /// From db4d554768898e57dd36c14e8ab5ab2910c387f2 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:06:22 -0500 Subject: [PATCH 18/32] cleanup runtime docs --- tokio/src/runtime/local_runtime/runtime.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 073d428b8ad..e4d90e4d108 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -11,8 +11,11 @@ use std::time::Duration; /// A local Tokio runtime. /// -/// This runtime is identical to a `current_thread` [runtime], save for not being `!Send + !Sync`, -/// and supporting `spawn_local`. +/// This runtime is capable of driving tasks which are not `Send + Sync` without the use of a +/// `LocalSet`, and thus supports `spawn_local` without the need for a LocalSet context. +/// +/// This runtime is incompatible with LocalSet. You should not attempt to drive a LocalSet within a +/// LocalRuntime. /// /// For more general information on how to use runtimes, see the [module] docs. /// From 0e6b6a491899a682c0f3c5c09ceca9f0c5c18cc2 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:08:16 -0500 Subject: [PATCH 19/32] cleanup runtime docs --- tokio/src/runtime/local_runtime/runtime.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index e4d90e4d108..547c1dd0665 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -14,9 +14,14 @@ use std::time::Duration; /// This runtime is capable of driving tasks which are not `Send + Sync` without the use of a /// `LocalSet`, and thus supports `spawn_local` without the need for a LocalSet context. /// +/// This runtime cannot be moved between threads or driven from different threads. +/// /// This runtime is incompatible with LocalSet. You should not attempt to drive a LocalSet within a /// LocalRuntime. /// +/// Currently, this runtime supports one flavor, which is internally identical to current_thread, +/// save for the aforementioned differences related to spawn_local. +/// /// For more general information on how to use runtimes, see the [module] docs. /// /// [runtime]: crate::runtime::Runtime From 30721a55bed02eadfcac7fe89c1a4965ad6499bb Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:13:17 -0500 Subject: [PATCH 20/32] hopefully this fixes docs --- tokio/src/runtime/local_runtime/runtime.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 547c1dd0665..abead5ed0a7 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -123,8 +123,8 @@ impl LocalRuntime { /// /// This is analogous to the [`spawn`] method on the standard [`Runtime`], but works even if the task is not threads safe. /// - /// [spawn]: crate::runtime::Runtime::spawn - /// [Runtime]: crate::runtime::Runtime + /// [`spawn`]: crate::runtime::Runtime::spawn + /// [`Runtime`]: crate::runtime::Runtime /// /// # Examples /// From 8c4599351e1e1237a39df317e73e8b6f72bb7f5a Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:32:29 -0500 Subject: [PATCH 21/32] add tests --- tokio/tests/rt_local.rs | 100 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100644 tokio/tests/rt_local.rs diff --git a/tokio/tests/rt_local.rs b/tokio/tests/rt_local.rs new file mode 100644 index 00000000000..1f14f5444d3 --- /dev/null +++ b/tokio/tests/rt_local.rs @@ -0,0 +1,100 @@ +#![allow(unknown_lints, unexpected_cfgs)] +#![warn(rust_2018_idioms)] +#![cfg(all(feature = "full", tokio_unstable))] + +use tokio::runtime::LocalOptions; +use tokio::task::spawn_local; + +#[test] +fn test_spawn_local_in_runtime() { + let rt = rt(); + + let res = rt.block_on(async move { + let (tx, rx) = tokio::sync::oneshot::channel(); + + spawn_local(async { + tokio::task::yield_now().await; + tx.send(5).unwrap(); + }); + + rx.await.unwrap() + }); + + assert_eq!(res, 5); +} + +#[test] +fn test_spawn_from_handle() { + let rt = rt(); + + let (tx, rx) = tokio::sync::oneshot::channel(); + + rt.handle().spawn(async { + tokio::task::yield_now().await; + tx.send(5).unwrap(); + }); + + let res = rt.block_on(async move { rx.await.unwrap() }); + + assert_eq!(res, 5); +} + +#[test] +fn test_spawn_local_on_runtime_object() { + let rt = rt(); + + let (tx, rx) = tokio::sync::oneshot::channel(); + + rt.spawn_local(async { + tokio::task::yield_now().await; + tx.send(5).unwrap(); + }); + + let res = rt.block_on(async move { rx.await.unwrap() }); + + assert_eq!(res, 5); +} + +#[test] +fn test_spawn_local_from_guard() { + let rt = rt(); + + let (tx, rx) = tokio::sync::oneshot::channel(); + + let _guard = rt.enter(); + + spawn_local(async { + tokio::task::yield_now().await; + tx.send(5).unwrap(); + }); + + let res = rt.block_on(async move { rx.await.unwrap() }); + + assert_eq!(res, 5); +} + +#[test] +#[should_panic] +fn test_spawn_local_from_guard_other_thread() { + let (tx, rx) = std::sync::mpsc::channel(); + + std::thread::spawn(move || { + let rt = rt(); + let handle = rt.handle().clone(); + + tx.send(handle).unwrap(); + }); + + let handle = rx.recv().unwrap(); + + let _guard = handle.enter(); + + spawn_local(async {}); +} + +fn rt() -> tokio::runtime::LocalRuntime { + tokio::runtime::Builder::new_current_thread() + .enable_all() + .build_local(&LocalOptions::default()) + .unwrap() +} From 40e69edd3e7ed00b22a1c04c06249532e2240111 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:34:27 -0500 Subject: [PATCH 22/32] "spelling" --- tokio/src/runtime/local_runtime/runtime.rs | 8 ++++---- tokio/src/runtime/scheduler/current_thread/mod.rs | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index abead5ed0a7..fa60a401506 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -12,15 +12,15 @@ use std::time::Duration; /// A local Tokio runtime. /// /// This runtime is capable of driving tasks which are not `Send + Sync` without the use of a -/// `LocalSet`, and thus supports `spawn_local` without the need for a LocalSet context. +/// `LocalSet`, and thus supports `spawn_local` without the need for a `LocalSet` context. /// /// This runtime cannot be moved between threads or driven from different threads. /// -/// This runtime is incompatible with LocalSet. You should not attempt to drive a LocalSet within a -/// LocalRuntime. +/// This runtime is incompatible with LocalSet. You should not attempt to drive a `LocalSet` within a +/// `LocalRuntime`. /// /// Currently, this runtime supports one flavor, which is internally identical to current_thread, -/// save for the aforementioned differences related to spawn_local. +/// save for the aforementioned differences related to `spawn_local`. /// /// For more general information on how to use runtimes, see the [module] docs. /// diff --git a/tokio/src/runtime/scheduler/current_thread/mod.rs b/tokio/src/runtime/scheduler/current_thread/mod.rs index d68ba6c27fd..2ec63c67b03 100644 --- a/tokio/src/runtime/scheduler/current_thread/mod.rs +++ b/tokio/src/runtime/scheduler/current_thread/mod.rs @@ -467,7 +467,7 @@ impl Handle { /// Spawn a task which isn't safe to send across thread boundaries onto the runtime. /// /// # Safety - /// This should only be used when this is a LocalRuntime or in another case where the runtime + /// This should only be used when this is a `LocalRuntime` or in another case where the runtime /// provably cannot be driven from or moved to different threads from the one on which the task /// is spawned. pub(crate) unsafe fn spawn_local( From 4b58bdb5540ef902c6acf1daca00c3a1445e2045 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:40:37 -0500 Subject: [PATCH 23/32] i am going to become the joker --- tokio/src/runtime/local_runtime/runtime.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index fa60a401506..d63eb8bdf12 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -16,10 +16,10 @@ use std::time::Duration; /// /// This runtime cannot be moved between threads or driven from different threads. /// -/// This runtime is incompatible with LocalSet. You should not attempt to drive a `LocalSet` within a +/// This runtime is incompatible with `LocalSet`. You should not attempt to drive a `LocalSet` within a /// `LocalRuntime`. /// -/// Currently, this runtime supports one flavor, which is internally identical to current_thread, +/// Currently, this runtime supports one flavor, which is internally identical to `current_thread`, /// save for the aforementioned differences related to `spawn_local`. /// /// For more general information on how to use runtimes, see the [module] docs. From 49798313be0df68f948e6082fbdc83625298c87a Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Wed, 2 Oct 2024 15:45:01 -0500 Subject: [PATCH 24/32] hah, spellcheck doesnt even catch my actual grammar issues --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index d63eb8bdf12..ddd8671da1a 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -121,7 +121,7 @@ impl LocalRuntime { /// Spawns a task on the runtime. /// - /// This is analogous to the [`spawn`] method on the standard [`Runtime`], but works even if the task is not threads safe. + /// This is analogous to the [`spawn`] method on the standard [`Runtime`], but works even if the task is not thread-safe. /// /// [`spawn`]: crate::runtime::Runtime::spawn /// [`Runtime`]: crate::runtime::Runtime From 6fc68fb5b14001cd6c42f5571ccda9a92e3659c4 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 10:34:05 -0500 Subject: [PATCH 25/32] Update tokio/src/runtime/local_runtime/runtime.rs Co-authored-by: Alice Ryhl --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index ddd8671da1a..060d6ee2b2a 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -90,7 +90,7 @@ impl LocalRuntime { pub fn new() -> std::io::Result { Builder::new_current_thread() .enable_all() - .build_local(&mut Default::default()) + .build_local(&Default::default()) } /// Returns a handle to the runtime's spawner. From 8e414e67d4ee86b922e11def252b26fd84b79564 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 10:34:30 -0500 Subject: [PATCH 26/32] Update tokio/src/runtime/local_runtime/runtime.rs Co-authored-by: Alice Ryhl --- tokio/src/runtime/local_runtime/runtime.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 060d6ee2b2a..ab9d1c1a121 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -192,7 +192,7 @@ impl LocalRuntime { /// Runs a future to completion on the Tokio runtime. This is the /// runtime's entry point. /// - /// See the documentation for the equivalent method on [Runtime] for more information. + /// See the documentation for [the equivalent method on Runtime] for more information. /// /// [Runtime]: crate::runtime::Runtime::block_on /// From 20b4c92125c1442d586b056ac39cfb910cb95f1b Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 10:43:42 -0500 Subject: [PATCH 27/32] add panics section to build_local --- tokio/src/runtime/builder.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tokio/src/runtime/builder.rs b/tokio/src/runtime/builder.rs index 8519f29661e..4d35120b1f9 100644 --- a/tokio/src/runtime/builder.rs +++ b/tokio/src/runtime/builder.rs @@ -807,6 +807,12 @@ impl Builder { /// /// The returned `LocalRuntime` instance is ready to spawn tasks. /// + /// # Panics + /// This will panic if `current_thread` is not the selected runtime flavor. + /// All other runtime flavors are unsupported by [`LocalRuntime`]. + /// + /// [`LocalRuntime`]: [crate::runtime::LocalRuntime] + /// /// # Examples /// /// ``` From bf5e5f9b127e1420b7de31d3c61091494706d881 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 10:49:50 -0500 Subject: [PATCH 28/32] send big futures to the heap in release mode --- tokio/src/runtime/local_runtime/runtime.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index ab9d1c1a121..eb5b39cddde 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -149,7 +149,7 @@ impl LocalRuntime { { // safety: spawn_local can only be called from `LocalRuntime`, which this is unsafe { - if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { + if std::mem::size_of::() > BOX_FUTURE_THRESHOLD { self.handle.spawn_local_named(Box::pin(future), None) } else { self.handle.spawn_local_named(future, None) @@ -211,7 +211,7 @@ impl LocalRuntime { /// ``` #[track_caller] pub fn block_on(&self, future: F) -> F::Output { - if cfg!(debug_assertions) && std::mem::size_of::() > BOX_FUTURE_THRESHOLD { + if std::mem::size_of::() > BOX_FUTURE_THRESHOLD { self.block_on_inner(Box::pin(future)) } else { self.block_on_inner(future) From 85f2c36f9f734eb456d5d2602d6827d5d4bdb4eb Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 10:54:47 -0500 Subject: [PATCH 29/32] document that handle.enter() allows spawn_local with LocalRuntime --- tokio/src/runtime/local_runtime/runtime.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index eb5b39cddde..c174749631b 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -252,9 +252,15 @@ impl LocalRuntime { /// available on creation such as [`Sleep`] or [`TcpStream`]. It will /// also allow you to call methods such as [`tokio::spawn`]. /// + /// If this is a handle to a [`LocalRuntime`], and this function is being invoked from the same + /// thread that the runtime was created on, you will also be able to call + /// [`tokio::task::spawn_local`]. + /// /// [`Sleep`]: struct@crate::time::Sleep /// [`TcpStream`]: struct@crate::net::TcpStream /// [`tokio::spawn`]: fn@crate::spawn + /// [`LocalRuntime`]: struct@crate::runtime::LocalRuntime + /// [`tokio::task::spawn_local`]: fn@crate::task::spawn_local /// /// # Example /// From 017adc985889fd217b003a621f56288c1f2e2067 Mon Sep 17 00:00:00 2001 From: Noah Kennedy Date: Fri, 4 Oct 2024 11:06:28 -0500 Subject: [PATCH 30/32] document that only blocking tasks leak --- tokio/src/runtime/local_runtime/runtime.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index c174749631b..9908c856709 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -295,6 +295,9 @@ impl LocalRuntime { /// Shuts down the runtime, waiting for at most `duration` for all spawned /// work to stop. /// + /// Note that `spawn_blocking` tasks, and only `spawn_blocking` tasks, can get left behind if + /// the timeout expires. + /// /// See the [struct level documentation](LocalRuntime#shutdown) for more details. /// /// # Examples @@ -333,7 +336,7 @@ impl LocalRuntime { /// /// Note however, that because we do not wait for any blocking tasks to complete, this /// may result in a resource leak (in that any blocking tasks are still running until they - /// return. + /// return. No other tasks will leak. /// /// See the [struct level documentation](LocalRuntime#shutdown) for more details. /// From 613dcd6342a56d8f2dac4acc2e07bc4782f810fa Mon Sep 17 00:00:00 2001 From: noah Date: Sat, 12 Oct 2024 09:51:40 -0500 Subject: [PATCH 31/32] update with tracing task size stuff --- tokio/src/runtime/handle.rs | 4 ++-- tokio/src/runtime/local_runtime/runtime.rs | 20 ++++++++++++++------ tokio/src/task/local.rs | 4 ++-- tokio/src/util/trace.rs | 1 + 4 files changed, 19 insertions(+), 10 deletions(-) diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index ae8f5969312..752640d75bd 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -353,7 +353,7 @@ impl Handle { pub(crate) unsafe fn spawn_local_named( &self, future: F, - _name: Option<&str>, + _meta: SpawnMeta<'_>, ) -> JoinHandle where F: Future + 'static, @@ -369,7 +369,7 @@ impl Handle { ))] let future = super::task::trace::Trace::root(future); #[cfg(all(tokio_unstable, feature = "tracing"))] - let future = crate::util::trace::task(future, "task", _name, id.as_u64()); + let future = crate::util::trace::task(future, "task", _meta, id.as_u64()); self.inner.spawn_local(future, id) } diff --git a/tokio/src/runtime/local_runtime/runtime.rs b/tokio/src/runtime/local_runtime/runtime.rs index 9908c856709..0f2b944e4eb 100644 --- a/tokio/src/runtime/local_runtime/runtime.rs +++ b/tokio/src/runtime/local_runtime/runtime.rs @@ -5,8 +5,10 @@ use crate::runtime::scheduler::CurrentThread; use crate::runtime::{context, Builder, EnterGuard, Handle, BOX_FUTURE_THRESHOLD}; use crate::task::JoinHandle; +use crate::util::trace::SpawnMeta; use std::future::Future; use std::marker::PhantomData; +use std::mem; use std::time::Duration; /// A local Tokio runtime. @@ -147,12 +149,15 @@ impl LocalRuntime { F: Future + 'static, F::Output: 'static, { + let fut_size = std::mem::size_of::(); + let meta = SpawnMeta::new_unnamed(fut_size); + // safety: spawn_local can only be called from `LocalRuntime`, which this is unsafe { if std::mem::size_of::() > BOX_FUTURE_THRESHOLD { - self.handle.spawn_local_named(Box::pin(future), None) + self.handle.spawn_local_named(Box::pin(future), meta) } else { - self.handle.spawn_local_named(future, None) + self.handle.spawn_local_named(future, meta) } } } @@ -211,15 +216,18 @@ impl LocalRuntime { /// ``` #[track_caller] pub fn block_on(&self, future: F) -> F::Output { + let fut_size = mem::size_of::(); + let meta = SpawnMeta::new_unnamed(fut_size); + if std::mem::size_of::() > BOX_FUTURE_THRESHOLD { - self.block_on_inner(Box::pin(future)) + self.block_on_inner(Box::pin(future), meta) } else { - self.block_on_inner(future) + self.block_on_inner(future, meta) } } #[track_caller] - fn block_on_inner(&self, future: F) -> F::Output { + fn block_on_inner(&self, future: F, _meta: SpawnMeta<'_>) -> F::Output { #[cfg(all( tokio_unstable, tokio_taskdump, @@ -233,7 +241,7 @@ impl LocalRuntime { let future = crate::util::trace::task( future, "block_on", - None, + _meta, crate::runtime::task::Id::next().as_u64(), ); diff --git a/tokio/src/task/local.rs b/tokio/src/task/local.rs index e30d21a88b4..edd02acbac0 100644 --- a/tokio/src/task/local.rs +++ b/tokio/src/task/local.rs @@ -409,7 +409,7 @@ cfg_rt! { ))] let future = task::trace::Trace::root(future); let id = task::Id::next(); - let task = crate::util::trace::task(future, "task", name, id.as_u64()); + let task = crate::util::trace::task(future, "task", meta, id.as_u64()); // safety: we have verified that this is a `LocalRuntime` owned by the current thread unsafe { handle.spawn_local(task, id) } @@ -426,7 +426,7 @@ cfg_rt! { Ok(Some(join_handle)) => join_handle, Err(_) => match CURRENT.with(|LocalData { ctx, .. }| ctx.get()) { None => panic!("`spawn_local` called from outside of a `task::LocalSet` or LocalRuntime"), - Some(cx) => cx.spawn(future.unwrap(), name) + Some(cx) => cx.spawn(future.unwrap(), meta) } } } diff --git a/tokio/src/util/trace.rs b/tokio/src/util/trace.rs index 97006df474e..b6eadba2205 100644 --- a/tokio/src/util/trace.rs +++ b/tokio/src/util/trace.rs @@ -1,6 +1,7 @@ cfg_rt! { use std::marker::PhantomData; + #[derive(Copy, Clone)] pub(crate) struct SpawnMeta<'a> { /// The name of the task #[cfg(all(tokio_unstable, feature = "tracing"))] From 92cafd608548ae1744bda08a05df229eab472225 Mon Sep 17 00:00:00 2001 From: noah Date: Sat, 12 Oct 2024 09:54:52 -0500 Subject: [PATCH 32/32] we stabilized id! --- tokio/src/runtime/scheduler/current_thread/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/tokio/src/runtime/scheduler/current_thread/mod.rs b/tokio/src/runtime/scheduler/current_thread/mod.rs index 2ec63c67b03..c66635e7bd6 100644 --- a/tokio/src/runtime/scheduler/current_thread/mod.rs +++ b/tokio/src/runtime/scheduler/current_thread/mod.rs @@ -482,7 +482,6 @@ impl Handle { let (handle, notified) = me.shared.owned.bind_local(future, me.clone(), id); me.task_hooks.spawn(&TaskMeta { - #[cfg(tokio_unstable)] id, _phantom: Default::default(), });