From 7c71ee10614c7a21210d29213cc637fccb509ad0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 28 Dec 2021 15:07:05 -0800 Subject: [PATCH 01/23] rt: expand on runtime metrics. --- tokio/src/runtime/basic_scheduler.rs | 4 + tokio/src/runtime/queue.rs | 23 ++++- tokio/src/runtime/stats/mock.rs | 26 ++++- tokio/src/runtime/stats/mod.rs | 2 +- tokio/src/runtime/stats/stats.rs | 127 +++++++++++++++++++++++- tokio/src/runtime/thread_pool/worker.rs | 25 +++-- 6 files changed, 183 insertions(+), 24 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index f70fa656925..168a8042e6c 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -411,10 +411,14 @@ impl Schedule for Arc { // If `None`, the runtime is shutting down, so there is no need // to schedule the task. if let Some(core) = core.as_mut() { + core.stats.inc_local_schedule_count(); core.tasks.push_back(task); } } _ => { + // Track that a task was scheduled from **outside** of the runtime. + self.stats.inc_remote_schedule_count(); + // If the queue is None, then the runtime has shut down. We // don't need to do anything with the notification in that case. let mut guard = self.queue.lock(); diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index a88dffcaf5d..63049483928 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -3,7 +3,7 @@ use crate::loom::cell::UnsafeCell; use crate::loom::sync::atomic::{AtomicU16, AtomicU32}; use crate::loom::sync::Arc; -use crate::runtime::stats::WorkerStatsBatcher; +use crate::runtime::stats::{WorkerStats, WorkerStatsBatcher}; use crate::runtime::task::{self, Inject}; use std::mem::MaybeUninit; @@ -102,7 +102,12 @@ impl Local { } /// Pushes a task to the back of the local queue, skipping the LIFO slot. - pub(super) fn push_back(&mut self, mut task: task::Notified, inject: &Inject) { + pub(super) fn push_back( + &mut self, + mut task: task::Notified, + inject: &Inject, + stats: &mut WorkerStatsBatcher, + ) { let tail = loop { let head = self.inner.head.load(Acquire); let (steal, real) = unpack(head); @@ -117,11 +122,12 @@ impl Local { // Concurrently stealing, this will free up capacity, so only // push the task onto the inject queue inject.push(task); + stats.incr_overflow_count(1); return; } else { // Push the current task and half of the queue into the // inject queue. - match self.push_overflow(task, real, tail, inject) { + match self.push_overflow(task, real, tail, inject, stats) { Ok(_) => return, // Lost the race, try again Err(v) => { @@ -163,6 +169,7 @@ impl Local { head: u16, tail: u16, inject: &Inject, + stats: &mut WorkerStatsBatcher, ) -> Result<(), task::Notified> { /// How many elements are we taking from the local queue. /// @@ -246,6 +253,9 @@ impl Local { }; inject.push_batch(batch_iter.chain(std::iter::once(task))); + // Add 1 to factor in the task currently being scheduled. + stats.incr_overflow_count(NUM_TASKS_TAKEN + 1); + Ok(()) } @@ -300,7 +310,8 @@ impl Steal { pub(super) fn steal_into( &self, dst: &mut Local, - stats: &mut WorkerStatsBatcher, + dst_stats: &mut WorkerStatsBatcher, + src_stats: &WorkerStats, ) -> Option> { // Safety: the caller is the only thread that mutates `dst.tail` and // holds a mutable reference. @@ -320,13 +331,15 @@ impl Steal { // Steal the tasks into `dst`'s buffer. This does not yet expose the // tasks in `dst`. let mut n = self.steal_into2(dst, dst_tail); - stats.incr_steal_count(n); if n == 0 { // No tasks were stolen return None; } + dst_stats.incr_steal_count(n); + src_stats.incr_stolen_count(n); + // We are returning a task here n -= 1; diff --git a/tokio/src/runtime/stats/mock.rs b/tokio/src/runtime/stats/mock.rs index 3bda8bffde6..1bc1e35ab94 100644 --- a/tokio/src/runtime/stats/mock.rs +++ b/tokio/src/runtime/stats/mock.rs @@ -6,22 +6,38 @@ impl RuntimeStats { pub(crate) fn new(_worker_threads: usize) -> Self { Self {} } + + /// Increment the number of tasks scheduled externally + pub(crate) fn inc_remote_schedule_count(&self) {} + + pub(crate) fn worker(&self, _index: usize) -> &WorkerStats { + &WorkerStats {} + } } +pub(crate) struct WorkerStats {} + pub(crate) struct WorkerStatsBatcher {} +impl WorkerStats { + pub(crate) fn incr_stolen_count(&self, _n: u16) {} +} + impl WorkerStatsBatcher { pub(crate) fn new(_my_index: usize) -> Self { Self {} } pub(crate) fn submit(&mut self, _to: &RuntimeStats) {} - pub(crate) fn about_to_park(&mut self) {} pub(crate) fn returned_from_park(&mut self) {} - - #[cfg(feature = "rt-multi-thread")] - pub(crate) fn incr_steal_count(&mut self, _by: u16) {} - pub(crate) fn incr_poll_count(&mut self) {} + pub(crate) fn inc_local_schedule_count(&mut self) {} +} + +cfg_rt_multi_thread! { + impl WorkerStatsBatcher { + pub(crate) fn incr_steal_count(&mut self, _by: u16) {} + pub(crate) fn incr_overflow_count(&mut self, _by: u16) {} + } } diff --git a/tokio/src/runtime/stats/mod.rs b/tokio/src/runtime/stats/mod.rs index 355e400602d..cc3fd47897f 100644 --- a/tokio/src/runtime/stats/mod.rs +++ b/tokio/src/runtime/stats/mod.rs @@ -19,5 +19,5 @@ cfg_not_stats! { #[path = "mock.rs"] mod stats; - pub(crate) use self::stats::{RuntimeStats, WorkerStatsBatcher}; + pub(crate) use self::stats::{RuntimeStats, WorkerStats, WorkerStatsBatcher}; } diff --git a/tokio/src/runtime/stats/stats.rs b/tokio/src/runtime/stats/stats.rs index 375786300e7..fdb012e72b7 100644 --- a/tokio/src/runtime/stats/stats.rs +++ b/tokio/src/runtime/stats/stats.rs @@ -13,6 +13,9 @@ use std::time::{Duration, Instant}; /// [unstable]: crate#unstable-features #[derive(Debug)] pub struct RuntimeStats { + /// Number of tasks that are scheduled from outside the runtime. + remote_schedule_count: AtomicU64, + /// Tracks per-worker performance counters workers: Box<[WorkerStats]>, } @@ -26,10 +29,29 @@ pub struct RuntimeStats { #[derive(Debug)] #[repr(align(128))] pub struct WorkerStats { + /// Number of times the worker parked. park_count: AtomicU64, + + /// Number of times the worker woke then parked again without doing work. + noop_count: AtomicU64, + + /// Number of times the worker attempted to steal. steal_count: AtomicU64, + + /// Number of tasks the worker polled. poll_count: AtomicU64, + + /// Number of tasks stolen from the current worker. + stolen_count: AtomicU64, + + /// Amount of time the worker spent doing work vs. parking. busy_duration_total: AtomicU64, + + /// Number of tasks scheduled for execution on the worker's local queue. + local_schedule_count: AtomicU64, + + /// Number of tasks moved from the local queue to the global queue to free space. + overflow_count: AtomicU64, } impl RuntimeStats { @@ -38,21 +60,43 @@ impl RuntimeStats { for _ in 0..worker_threads { workers.push(WorkerStats { park_count: AtomicU64::new(0), + noop_count: AtomicU64::new(0), steal_count: AtomicU64::new(0), poll_count: AtomicU64::new(0), + stolen_count: AtomicU64::new(0), + overflow_count: AtomicU64::new(0), busy_duration_total: AtomicU64::new(0), + local_schedule_count: AtomicU64::new(0), }); } Self { + remote_schedule_count: AtomicU64::new(0), workers: workers.into_boxed_slice(), } } + /// Returns the number of tasks scheduled from **outside** of the runtime. + /// + /// Tasks scheduled from outside of the runtime go via the runtime's + /// injection queue, which is usually is slower. + pub fn remote_schedule_count(&self) -> u64 { + self.remote_schedule_count.load(Relaxed) + } + /// Returns a slice containing the worker stats for each worker thread. pub fn workers(&self) -> impl Iterator { self.workers.iter() } + + /// Increment the number of tasks scheduled externally + pub(crate) fn inc_remote_schedule_count(&self) { + self.remote_schedule_count.fetch_add(1, Relaxed); + } + + pub(crate) fn worker(&self, index: usize) -> &WorkerStats { + &self.workers[index] + } } impl WorkerStats { @@ -61,12 +105,24 @@ impl WorkerStats { self.park_count.load(Relaxed) } + /// Returns the number of times this worker unparked but performed no work. + /// + /// This is the false-positive wake count. + pub fn noop_count(&self) -> u64 { + self.noop_count.load(Relaxed) + } + /// Returns the number of tasks this worker has stolen from other worker /// threads. pub fn steal_count(&self) -> u64 { self.steal_count.load(Relaxed) } + /// Returns the number of tasks that were stolen from this worker. + pub fn stolen_count(&self) -> u64 { + self.stolen_count.load(Relaxed) + } + /// Returns the number of times this worker has polled a task. pub fn poll_count(&self) -> u64 { self.poll_count.load(Relaxed) @@ -76,13 +132,50 @@ impl WorkerStats { pub fn total_busy_duration(&self) -> Duration { Duration::from_nanos(self.busy_duration_total.load(Relaxed)) } + + /// TODO + pub fn local_schedule_count(&self) -> u64 { + self.local_schedule_count.load(Relaxed) + } + + /// Returns the number of tasks moved from this worker's local queue to the + /// remote queue. + pub fn overflow_count(&self) -> u64 { + self.overflow_count.load(Relaxed) + } + + pub(crate) fn incr_stolen_count(&self, n: u16) { + self.stolen_count.fetch_add(n as _, Relaxed); + } } pub(crate) struct WorkerStatsBatcher { + /// Identifies the worker within the runtime. my_index: usize, + + /// Number of times the worker parked park_count: u64, + + /// Number of times the worker woke w/o doing work. + noop_count: u64, + + /// Number of times stolen steal_count: u64, + + /// Number of tasks that were polled by the worker. poll_count: u64, + + /// Number of tasks polled when the worker entered park. This is used to + /// track the noop count. + poll_count_on_last_park: u64, + + /// Number of tasks that were scheduled locally on this worker. + local_schedule_count: u64, + + /// Number of tasks moved to the global queue to make space in the local + /// queue + overflow_count: u64, + /// The total busy duration in nanoseconds. busy_duration_total: u64, last_resume_time: Instant, @@ -93,8 +186,12 @@ impl WorkerStatsBatcher { Self { my_index, park_count: 0, + noop_count: 0, steal_count: 0, poll_count: 0, + poll_count_on_last_park: 0, + local_schedule_count: 0, + overflow_count: 0, busy_duration_total: 0, last_resume_time: Instant::now(), } @@ -103,17 +200,30 @@ impl WorkerStatsBatcher { let worker = &to.workers[self.my_index]; worker.park_count.store(self.park_count, Relaxed); + worker.noop_count.store(self.noop_count, Relaxed); worker.steal_count.store(self.steal_count, Relaxed); worker.poll_count.store(self.poll_count, Relaxed); worker .busy_duration_total .store(self.busy_duration_total, Relaxed); + + worker + .local_schedule_count + .store(self.local_schedule_count, Relaxed); + worker.overflow_count.store(self.overflow_count, Relaxed); } + /// The worker is about to park. pub(crate) fn about_to_park(&mut self) { self.park_count += 1; + if self.poll_count_on_last_park == self.poll_count { + self.noop_count += 1; + } else { + self.poll_count_on_last_park = self.poll_count; + } + let busy_duration = self.last_resume_time.elapsed(); let busy_duration = u64::try_from(busy_duration.as_nanos()).unwrap_or(u64::MAX); self.busy_duration_total += busy_duration; @@ -123,12 +233,23 @@ impl WorkerStatsBatcher { self.last_resume_time = Instant::now(); } - #[cfg(feature = "rt-multi-thread")] - pub(crate) fn incr_steal_count(&mut self, by: u16) { - self.steal_count += u64::from(by); + pub(crate) fn inc_local_schedule_count(&mut self) { + self.local_schedule_count += 1; } pub(crate) fn incr_poll_count(&mut self) { self.poll_count += 1; } } + +cfg_rt_multi_thread! { + impl WorkerStatsBatcher { + pub(crate) fn incr_steal_count(&mut self, by: u16) { + self.steal_count += by as u64; + } + + pub(crate) fn incr_overflow_count(&mut self, by: u16) { + self.overflow_count += by as u64; + } + } +} diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 27d0d5e7d32..b1da35c71e3 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -445,7 +445,8 @@ impl Context { } else { // Not enough budget left to run the LIFO task, push it to // the back of the queue and return. - core.run_queue.push_back(task, self.worker.inject()); + core.run_queue + .push_back(task, self.worker.inject(), &mut core.stats); return Ok(core); } } @@ -472,7 +473,9 @@ impl Context { if core.transition_to_parked(&self.worker) { while !core.is_shutdown { + core.stats.about_to_park(); core = self.park_timeout(core, None); + core.stats.returned_from_park(); // Run regularly scheduled maintenance core.maintenance(&self.worker); @@ -493,8 +496,6 @@ impl Context { // Take the parker out of core let mut park = core.park.take().expect("park missing"); - core.stats.about_to_park(); - // Store `core` in context *self.core.borrow_mut() = Some(core); @@ -516,8 +517,6 @@ impl Context { self.worker.shared.notify_parked(); } - core.stats.returned_from_park(); - core } } @@ -559,9 +558,11 @@ impl Core { } let target = &worker.shared.remotes[i]; - if let Some(task) = target - .steal - .steal_into(&mut self.run_queue, &mut self.stats) + let target_stats = worker.shared.stats.worker(i); + if let Some(task) = + target + .steal + .steal_into(&mut self.run_queue, &mut self.stats, target_stats) { return Some(task); } @@ -726,12 +727,15 @@ impl Shared { } fn schedule_local(&self, core: &mut Core, task: Notified, is_yield: bool) { + core.stats.inc_local_schedule_count(); + // Spawning from the worker thread. If scheduling a "yield" then the // task must always be pushed to the back of the queue, enabling other // tasks to be executed. If **not** a yield, then there is more // flexibility and the task may go to the front of the queue. let should_notify = if is_yield { - core.run_queue.push_back(task, &self.inject); + core.run_queue + .push_back(task, &self.inject, &mut core.stats); true } else { // Push to the LIFO slot @@ -739,7 +743,8 @@ impl Shared { let ret = prev.is_some(); if let Some(prev) = prev { - core.run_queue.push_back(prev, &self.inject); + core.run_queue + .push_back(prev, &self.inject, &mut core.stats); } core.lifo_slot = Some(task); From 82d4467d01df24ab7f8ab9702567c9003a154cc1 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 12 Jan 2022 14:59:20 -0800 Subject: [PATCH 02/23] rename stats -> metrics --- tokio/Cargo.toml | 6 +- tokio/src/lib.rs | 1 - tokio/src/macros/cfg.rs | 12 +- tokio/src/runtime/basic_scheduler.rs | 32 +-- tokio/src/runtime/handle.rs | 8 +- tokio/src/runtime/metrics/batch.rs | 111 ++++++++ tokio/src/runtime/{stats => metrics}/mock.rs | 26 +- tokio/src/runtime/metrics/mod.rs | 28 ++ tokio/src/runtime/metrics/runtime.rs | 64 +++++ tokio/src/runtime/metrics/worker.rs | 88 +++++++ tokio/src/runtime/mod.rs | 13 +- tokio/src/runtime/queue.rs | 20 +- tokio/src/runtime/spawner.rs | 17 +- tokio/src/runtime/stats/mod.rs | 23 -- tokio/src/runtime/stats/stats.rs | 255 ------------------- tokio/src/runtime/tests/queue.rs | 6 +- tokio/src/runtime/thread_pool/mod.rs | 7 +- tokio/src/runtime/thread_pool/worker.rs | 43 ++-- 18 files changed, 392 insertions(+), 368 deletions(-) create mode 100644 tokio/src/runtime/metrics/batch.rs rename tokio/src/runtime/{stats => metrics}/mock.rs (62%) create mode 100644 tokio/src/runtime/metrics/mod.rs create mode 100644 tokio/src/runtime/metrics/runtime.rs create mode 100644 tokio/src/runtime/metrics/worker.rs delete mode 100644 tokio/src/runtime/stats/mod.rs delete mode 100644 tokio/src/runtime/stats/stats.rs diff --git a/tokio/Cargo.toml b/tokio/Cargo.toml index 2a88b766c0a..5aa5c399d27 100644 --- a/tokio/Cargo.toml +++ b/tokio/Cargo.toml @@ -46,7 +46,6 @@ io-util = ["memchr", "bytes"] # stdin, stdout, stderr io-std = [] macros = ["tokio-macros"] -stats = [] net = [ "libc", "mio/os-poll", @@ -85,6 +84,11 @@ sync = [] test-util = ["rt", "sync", "time"] time = [] +# Technically, removing this is a breaking change even though it only ever did +# anything with the unstable flag on. It is probably safe to get rid of it after +# a few releases. +stats = [] + [dependencies] tokio-macros = { version = "1.7.0", path = "../tokio-macros", optional = true } diff --git a/tokio/src/lib.rs b/tokio/src/lib.rs index 01878524e13..35295d837a6 100644 --- a/tokio/src/lib.rs +++ b/tokio/src/lib.rs @@ -346,7 +346,6 @@ //! `RUSTFLAGS="--cfg tokio_unstable"`. //! //! - `tracing`: Enables tracing events. -//! - `stats`: Enables runtime stats collection. ([RFC](https://github.com/tokio-rs/tokio/pull/3845)) //! //! [feature flags]: https://doc.rust-lang.org/cargo/reference/manifest.html#the-features-section diff --git a/tokio/src/macros/cfg.rs b/tokio/src/macros/cfg.rs index 9fa30ca27d6..3b8fd3a7890 100644 --- a/tokio/src/macros/cfg.rs +++ b/tokio/src/macros/cfg.rs @@ -174,20 +174,22 @@ macro_rules! cfg_macros { } } -macro_rules! cfg_stats { +macro_rules! cfg_metrics { ($($item:item)*) => { $( - #[cfg(all(tokio_unstable, feature = "stats"))] - #[cfg_attr(docsrs, doc(cfg(all(tokio_unstable, feature = "stats"))))] + // For now, metrics is always enabled. When stabilized, it might + // have a dedicated feature flag. + #[cfg(tokio_unstable)] + #[cfg_attr(docsrs, doc(cfg(tokio_unstable)))] $item )* } } -macro_rules! cfg_not_stats { +macro_rules! cfg_not_metrics { ($($item:item)*) => { $( - #[cfg(not(all(tokio_unstable, feature = "stats")))] + #[cfg(not(tokio_unstable))] $item )* } diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 168a8042e6c..3b4509fdb74 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -4,7 +4,7 @@ use crate::loom::sync::Mutex; use crate::park::{Park, Unpark}; use crate::runtime::context::EnterGuard; use crate::runtime::driver::Driver; -use crate::runtime::stats::{RuntimeStats, WorkerStatsBatcher}; +use crate::runtime::{RuntimeMetrics, MetricsBatch}; use crate::runtime::task::{self, JoinHandle, OwnedTasks, Schedule, Task}; use crate::runtime::Callback; use crate::sync::notify::Notify; @@ -56,8 +56,8 @@ struct Core { /// The driver is removed before starting to park the thread driver: Option, - /// Stats batcher - stats: WorkerStatsBatcher, + /// Metrics batch + metrics: MetricsBatch, } #[derive(Clone)] @@ -98,8 +98,8 @@ struct Shared { /// Callback for a worker unparking itself after_unpark: Option, - /// Keeps track of various runtime stats. - stats: RuntimeStats, + /// Keeps track of various runtime metrics. + metrics: RuntimeMetrics, } /// Thread-local context. @@ -143,7 +143,7 @@ impl BasicScheduler { woken: AtomicBool::new(false), before_park, after_unpark, - stats: RuntimeStats::new(1), + metrics: RuntimeMetrics::new(1), }), }; @@ -152,7 +152,7 @@ impl BasicScheduler { spawner: spawner.clone(), tick: 0, driver: Some(driver), - stats: WorkerStatsBatcher::new(0), + metrics: MetricsBatch::new(0), }))); BasicScheduler { @@ -223,7 +223,7 @@ impl Context { /// Execute the closure with the given scheduler core stored in the /// thread-local context. fn run_task(&self, mut core: Box, f: impl FnOnce() -> R) -> (Box, R) { - core.stats.incr_poll_count(); + core.metrics.incr_poll_count(); self.enter(core, || crate::coop::budget(f)) } @@ -244,15 +244,15 @@ impl Context { // instead of parking the thread if core.tasks.is_empty() { // Park until the thread is signaled - core.stats.about_to_park(); - core.stats.submit(&core.spawner.shared.stats); + core.metrics.about_to_park(); + core.metrics.submit(&core.spawner.shared.metrics); let (c, _) = self.enter(core, || { driver.park().expect("failed to park"); }); core = c; - core.stats.returned_from_park(); + core.metrics.returned_from_park(); } if let Some(f) = &self.spawner.shared.after_unpark { @@ -271,7 +271,7 @@ impl Context { fn park_yield(&self, mut core: Box) -> Box { let mut driver = core.driver.take().expect("driver missing"); - core.stats.submit(&core.spawner.shared.stats); + core.metrics.submit(&core.spawner.shared.metrics); let (mut core, _) = self.enter(core, || { driver .park_timeout(Duration::from_millis(0)) @@ -366,8 +366,8 @@ impl Spawner { handle } - pub(crate) fn stats(&self) -> &RuntimeStats { - &self.shared.stats + pub(crate) fn metrics(&self) -> &RuntimeMetrics { + &self.shared.metrics } fn pop(&self) -> Option { @@ -411,13 +411,13 @@ impl Schedule for Arc { // If `None`, the runtime is shutting down, so there is no need // to schedule the task. if let Some(core) = core.as_mut() { - core.stats.inc_local_schedule_count(); + core.metrics.inc_local_schedule_count(); core.tasks.push_back(task); } } _ => { // Track that a task was scheduled from **outside** of the runtime. - self.stats.inc_remote_schedule_count(); + self.metrics.inc_remote_schedule_count(); // If the queue is None, then the runtime has shut down. We // don't need to do anything with the notification in that case. diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index 612205cccfa..7d8ffb1c4f7 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -1,6 +1,6 @@ use crate::runtime::blocking::{BlockingTask, NoopSchedule}; use crate::runtime::task::{self, JoinHandle}; -use crate::runtime::{blocking, context, driver, Spawner}; +use crate::runtime::{blocking, context, driver, Spawner, RuntimeMetrics}; use crate::util::error::{CONTEXT_MISSING_ERROR, THREAD_LOCAL_DESTROYED_ERROR}; use std::future::Future; @@ -126,11 +126,11 @@ impl Handle { context::try_current() } - cfg_stats! { + cfg_metrics! { /// Returns a view that lets you get information about how the runtime /// is performing. - pub fn stats(&self) -> &crate::runtime::stats::RuntimeStats { - self.spawner.stats() + pub fn metrics(&self) -> &RuntimeMetrics { + self.spawner.metrics() } } diff --git a/tokio/src/runtime/metrics/batch.rs b/tokio/src/runtime/metrics/batch.rs new file mode 100644 index 00000000000..31005c35c61 --- /dev/null +++ b/tokio/src/runtime/metrics/batch.rs @@ -0,0 +1,111 @@ +use crate::runtime::RuntimeMetrics; + +use std::convert::TryFrom; +use std::sync::atomic::Ordering::Relaxed; +use std::time::Instant; + +pub(crate) struct MetricsBatch { + /// Identifies the worker within the runtime. + my_index: usize, + + /// Number of times the worker parked + park_count: u64, + + /// Number of times the worker woke w/o doing work. + noop_count: u64, + + /// Number of times stolen + steal_count: u64, + + /// Number of tasks that were polled by the worker. + poll_count: u64, + + /// Number of tasks polled when the worker entered park. This is used to + /// track the noop count. + poll_count_on_last_park: u64, + + /// Number of tasks that were scheduled locally on this worker. + local_schedule_count: u64, + + /// Number of tasks moved to the global queue to make space in the local + /// queue + overflow_count: u64, + + /// The total busy duration in nanoseconds. + busy_duration_total: u64, + last_resume_time: Instant, +} + +impl MetricsBatch { + pub(crate) fn new(my_index: usize) -> MetricsBatch { + MetricsBatch { + my_index, + park_count: 0, + noop_count: 0, + steal_count: 0, + poll_count: 0, + poll_count_on_last_park: 0, + local_schedule_count: 0, + overflow_count: 0, + busy_duration_total: 0, + last_resume_time: Instant::now(), + } + } + + pub(crate) fn submit(&mut self, to: &RuntimeMetrics) { + let worker = to.worker(self.my_index); + + worker.park_count.store(self.park_count, Relaxed); + worker.noop_count.store(self.noop_count, Relaxed); + worker.steal_count.store(self.steal_count, Relaxed); + worker.poll_count.store(self.poll_count, Relaxed); + + worker + .busy_duration_total + .store(self.busy_duration_total, Relaxed); + + worker + .local_schedule_count + .store(self.local_schedule_count, Relaxed); + worker.overflow_count.store(self.overflow_count, Relaxed); + } + + /// The worker is about to park. + pub(crate) fn about_to_park(&mut self) { + self.park_count += 1; + + if self.poll_count_on_last_park == self.poll_count { + self.noop_count += 1; + } else { + self.poll_count_on_last_park = self.poll_count; + } + + let busy_duration = self.last_resume_time.elapsed(); + let busy_duration = u64::try_from(busy_duration.as_nanos()).unwrap_or(u64::MAX); + self.busy_duration_total += busy_duration; + } + + pub(crate) fn returned_from_park(&mut self) { + self.last_resume_time = Instant::now(); + } + + pub(crate) fn inc_local_schedule_count(&mut self) { + self.local_schedule_count += 1; + } + + pub(crate) fn incr_poll_count(&mut self) { + self.poll_count += 1; + } +} + +cfg_rt_multi_thread! { + impl MetricsBatch { + pub(crate) fn incr_steal_count(&mut self, by: u16) { + self.steal_count += by as u64; + } + + pub(crate) fn incr_overflow_count(&mut self, by: u16) { + self.overflow_count += by as u64; + } + } +} \ No newline at end of file diff --git a/tokio/src/runtime/stats/mock.rs b/tokio/src/runtime/metrics/mock.rs similarity index 62% rename from tokio/src/runtime/stats/mock.rs rename to tokio/src/runtime/metrics/mock.rs index 1bc1e35ab94..40acf28061e 100644 --- a/tokio/src/runtime/stats/mock.rs +++ b/tokio/src/runtime/metrics/mock.rs @@ -1,8 +1,12 @@ -//! This file contains mocks of the types in src/runtime/stats/stats.rs +//! This file contains mocks of the types in src/runtime/metrics -pub(crate) struct RuntimeStats {} +pub(crate) struct RuntimeMetrics {} -impl RuntimeStats { +pub(crate) struct WorkerMetrics {} + +pub(crate) struct MetricsBatch {} + +impl RuntimeMetrics { pub(crate) fn new(_worker_threads: usize) -> Self { Self {} } @@ -10,25 +14,21 @@ impl RuntimeStats { /// Increment the number of tasks scheduled externally pub(crate) fn inc_remote_schedule_count(&self) {} - pub(crate) fn worker(&self, _index: usize) -> &WorkerStats { - &WorkerStats {} + pub(crate) fn worker(&self, _index: usize) -> &WorkerMetrics { + &WorkerMetrics {} } } -pub(crate) struct WorkerStats {} - -pub(crate) struct WorkerStatsBatcher {} - -impl WorkerStats { +impl WorkerMetrics { pub(crate) fn incr_stolen_count(&self, _n: u16) {} } -impl WorkerStatsBatcher { +impl MetricsBatch { pub(crate) fn new(_my_index: usize) -> Self { Self {} } - pub(crate) fn submit(&mut self, _to: &RuntimeStats) {} + pub(crate) fn submit(&mut self, _to: &RuntimeMetrics) {} pub(crate) fn about_to_park(&mut self) {} pub(crate) fn returned_from_park(&mut self) {} pub(crate) fn incr_poll_count(&mut self) {} @@ -36,7 +36,7 @@ impl WorkerStatsBatcher { } cfg_rt_multi_thread! { - impl WorkerStatsBatcher { + impl MetricsBatch { pub(crate) fn incr_steal_count(&mut self, _by: u16) {} pub(crate) fn incr_overflow_count(&mut self, _by: u16) {} } diff --git a/tokio/src/runtime/metrics/mod.rs b/tokio/src/runtime/metrics/mod.rs new file mode 100644 index 00000000000..31c3a7dcd99 --- /dev/null +++ b/tokio/src/runtime/metrics/mod.rs @@ -0,0 +1,28 @@ +//! This module contains information need to view information about how the +//! runtime is performing. +//! +//! **Note**: This is an [unstable API][unstable]. The public API of types in +//! this module may break in 1.x releases. See [the documentation on unstable +//! features][unstable] for details. +//! +//! [unstable]: crate#unstable-features +#![allow(clippy::module_inception)] + +cfg_metrics! { + mod batch; + pub(crate) use batch::MetricsBatch; + + mod runtime; + #[allow(unreachable_pub)] // rust-lang/rust#57411 + pub use runtime::RuntimeMetrics; + + mod worker; + #[allow(unreachable_pub)] // rust-lang/rust#57411 + pub use worker::WorkerMetrics; +} + +cfg_not_metrics! { + mod mock; + + pub(crate) use mock::{RuntimeMetrics, WorkerMetrics, MetricsBatch}; +} diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs new file mode 100644 index 00000000000..b1b37f2675d --- /dev/null +++ b/tokio/src/runtime/metrics/runtime.rs @@ -0,0 +1,64 @@ +use crate::loom::sync::atomic::{AtomicU64, Ordering::Relaxed}; +use crate::runtime::metrics::WorkerMetrics; + +/// Retrieves metrics from the Tokio runtime. +/// +/// **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. +/// +/// [unstable]: crate#unstable-features +#[derive(Debug)] +pub struct RuntimeMetrics { + /// Number of tasks that are scheduled from outside the runtime. + remote_schedule_count: AtomicU64, + + /// Tracks per-worker metrics + workers: Box<[WorkerMetrics]>, +} + +impl RuntimeMetrics { + pub(crate) fn new(worker_threads: usize) -> RuntimeMetrics { + let mut workers = Vec::with_capacity(worker_threads); + + for _ in 0..worker_threads { + workers.push(WorkerMetrics { + park_count: AtomicU64::new(0), + noop_count: AtomicU64::new(0), + steal_count: AtomicU64::new(0), + poll_count: AtomicU64::new(0), + stolen_count: AtomicU64::new(0), + overflow_count: AtomicU64::new(0), + busy_duration_total: AtomicU64::new(0), + local_schedule_count: AtomicU64::new(0), + }); + } + + RuntimeMetrics { + remote_schedule_count: AtomicU64::new(0), + workers: workers.into_boxed_slice(), + } + } + + /// Returns the number of tasks scheduled from **outside** of the runtime. + /// + /// Tasks scheduled from outside of the runtime go via the runtime's + /// injection queue, which is usually is slower. + pub fn remote_schedule_count(&self) -> u64 { + self.remote_schedule_count.load(Relaxed) + } + + /// Returns a slice containing the metrics for each worker thread. + pub fn workers(&self) -> &[WorkerMetrics] { + &self.workers + } + + /// Increment the number of tasks scheduled externally + pub(crate) fn inc_remote_schedule_count(&self) { + self.remote_schedule_count.fetch_add(1, Relaxed); + } + + pub(crate) fn worker(&self, index: usize) -> &WorkerMetrics { + &self.workers[index] + } +} diff --git a/tokio/src/runtime/metrics/worker.rs b/tokio/src/runtime/metrics/worker.rs new file mode 100644 index 00000000000..78ed46a1a68 --- /dev/null +++ b/tokio/src/runtime/metrics/worker.rs @@ -0,0 +1,88 @@ +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering::Relaxed; +use std::time::Duration; + +/// Retreive runtime worker metrics. +/// +/// **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. +/// +/// [unstable]: crate#unstable-features +#[derive(Debug)] +#[repr(align(128))] +pub struct WorkerMetrics { + /// Number of times the worker parked. + pub(super) park_count: AtomicU64, + + /// Number of times the worker woke then parked again without doing work. + pub(super) noop_count: AtomicU64, + + /// Number of times the worker attempted to steal. + pub(super) steal_count: AtomicU64, + + /// Number of tasks the worker polled. + pub(super) poll_count: AtomicU64, + + /// Number of tasks stolen from the current worker. + pub(super) stolen_count: AtomicU64, + + /// Amount of time the worker spent doing work vs. parking. + pub(super) busy_duration_total: AtomicU64, + + /// Number of tasks scheduled for execution on the worker's local queue. + pub(super) local_schedule_count: AtomicU64, + + /// Number of tasks moved from the local queue to the global queue to free space. + pub(super) overflow_count: AtomicU64, +} + +impl WorkerMetrics { + /// Returns the total number of times this worker thread has parked. + pub fn park_count(&self) -> u64 { + self.park_count.load(Relaxed) + } + + /// Returns the number of times this worker unparked but performed no work. + /// + /// This is the false-positive wake count. + pub fn noop_count(&self) -> u64 { + self.noop_count.load(Relaxed) + } + + /// Returns the number of tasks this worker has stolen from other worker + /// threads. + pub fn steal_count(&self) -> u64 { + self.steal_count.load(Relaxed) + } + + /// Returns the number of tasks that were stolen from this worker. + pub fn stolen_count(&self) -> u64 { + self.stolen_count.load(Relaxed) + } + + /// Returns the number of times this worker has polled a task. + pub fn poll_count(&self) -> u64 { + self.poll_count.load(Relaxed) + } + + /// Returns the total amount of time this worker has been busy for. + pub fn total_busy_duration(&self) -> Duration { + Duration::from_nanos(self.busy_duration_total.load(Relaxed)) + } + + /// TODO + pub fn local_schedule_count(&self) -> u64 { + self.local_schedule_count.load(Relaxed) + } + + /// Returns the number of tasks moved from this worker's local queue to the + /// remote queue. + pub fn overflow_count(&self) -> u64 { + self.overflow_count.load(Relaxed) + } + + pub(crate) fn incr_stolen_count(&self, n: u16) { + self.stolen_count.fetch_add(n as _, Relaxed); + } +} \ No newline at end of file diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index e77c5e3a0f8..bc94bf0cb3c 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -181,11 +181,16 @@ pub(crate) mod enter; pub(crate) mod task; -cfg_stats! { - pub mod stats; +cfg_metrics! { + mod metrics; + pub use metrics::{RuntimeMetrics, WorkerMetrics}; + + pub(crate) use metrics::MetricsBatch; } -cfg_not_stats! { - pub(crate) mod stats; + +cfg_not_metrics! { + pub(crate) mod metrics; + pub(crate) use metrics::{RuntimeMetrics, WorkerMetrics, MetricsBatch}; } cfg_rt! { diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index 63049483928..c1cb15ebfdb 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -3,7 +3,7 @@ use crate::loom::cell::UnsafeCell; use crate::loom::sync::atomic::{AtomicU16, AtomicU32}; use crate::loom::sync::Arc; -use crate::runtime::stats::{WorkerStats, WorkerStatsBatcher}; +use crate::runtime::{WorkerMetrics, MetricsBatch}; use crate::runtime::task::{self, Inject}; use std::mem::MaybeUninit; @@ -106,7 +106,7 @@ impl Local { &mut self, mut task: task::Notified, inject: &Inject, - stats: &mut WorkerStatsBatcher, + metrics: &mut MetricsBatch, ) { let tail = loop { let head = self.inner.head.load(Acquire); @@ -122,12 +122,12 @@ impl Local { // Concurrently stealing, this will free up capacity, so only // push the task onto the inject queue inject.push(task); - stats.incr_overflow_count(1); + metrics.incr_overflow_count(1); return; } else { // Push the current task and half of the queue into the // inject queue. - match self.push_overflow(task, real, tail, inject, stats) { + match self.push_overflow(task, real, tail, inject, metrics) { Ok(_) => return, // Lost the race, try again Err(v) => { @@ -169,7 +169,7 @@ impl Local { head: u16, tail: u16, inject: &Inject, - stats: &mut WorkerStatsBatcher, + metrics: &mut MetricsBatch, ) -> Result<(), task::Notified> { /// How many elements are we taking from the local queue. /// @@ -254,7 +254,7 @@ impl Local { inject.push_batch(batch_iter.chain(std::iter::once(task))); // Add 1 to factor in the task currently being scheduled. - stats.incr_overflow_count(NUM_TASKS_TAKEN + 1); + metrics.incr_overflow_count(NUM_TASKS_TAKEN + 1); Ok(()) } @@ -310,8 +310,8 @@ impl Steal { pub(super) fn steal_into( &self, dst: &mut Local, - dst_stats: &mut WorkerStatsBatcher, - src_stats: &WorkerStats, + dst_metrics: &mut MetricsBatch, + src_metrics: &WorkerMetrics, ) -> Option> { // Safety: the caller is the only thread that mutates `dst.tail` and // holds a mutable reference. @@ -337,8 +337,8 @@ impl Steal { return None; } - dst_stats.incr_steal_count(n); - src_stats.incr_stolen_count(n); + dst_metrics.incr_steal_count(n); + src_metrics.incr_stolen_count(n); // We are returning a task here n -= 1; diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index 9a3d465aef9..90ba400479c 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -1,6 +1,6 @@ use crate::future::Future; +use crate::runtime::RuntimeMetrics; use crate::runtime::basic_scheduler; -use crate::runtime::stats::RuntimeStats; use crate::task::JoinHandle; cfg_rt_multi_thread! { @@ -35,13 +35,16 @@ impl Spawner { Spawner::ThreadPool(spawner) => spawner.spawn(future), } } +} - #[cfg_attr(not(all(tokio_unstable, feature = "stats")), allow(dead_code))] - pub(crate) fn stats(&self) -> &RuntimeStats { - match self { - Spawner::Basic(spawner) => spawner.stats(), - #[cfg(feature = "rt-multi-thread")] - Spawner::ThreadPool(spawner) => spawner.stats(), +cfg_metrics! { + impl Spawner { + pub(crate) fn metrics(&self) -> &RuntimeMetrics { + match self { + Spawner::Basic(spawner) => spawner.metrics(), + #[cfg(feature = "rt-multi-thread")] + Spawner::ThreadPool(spawner) => spawner.metrics(), + } } } } diff --git a/tokio/src/runtime/stats/mod.rs b/tokio/src/runtime/stats/mod.rs deleted file mode 100644 index cc3fd47897f..00000000000 --- a/tokio/src/runtime/stats/mod.rs +++ /dev/null @@ -1,23 +0,0 @@ -//! This module contains information need to view information about how the -//! runtime is performing. -//! -//! **Note**: This is an [unstable API][unstable]. The public API of types in -//! this module may break in 1.x releases. See [the documentation on unstable -//! features][unstable] for details. -//! -//! [unstable]: crate#unstable-features -#![allow(clippy::module_inception)] - -cfg_stats! { - mod stats; - - pub use self::stats::{RuntimeStats, WorkerStats}; - pub(crate) use self::stats::WorkerStatsBatcher; -} - -cfg_not_stats! { - #[path = "mock.rs"] - mod stats; - - pub(crate) use self::stats::{RuntimeStats, WorkerStats, WorkerStatsBatcher}; -} diff --git a/tokio/src/runtime/stats/stats.rs b/tokio/src/runtime/stats/stats.rs deleted file mode 100644 index fdb012e72b7..00000000000 --- a/tokio/src/runtime/stats/stats.rs +++ /dev/null @@ -1,255 +0,0 @@ -//! This file contains the types necessary to collect various types of stats. -use crate::loom::sync::atomic::{AtomicU64, Ordering::Relaxed}; - -use std::convert::TryFrom; -use std::time::{Duration, Instant}; - -/// This type contains methods to retrieve stats from a Tokio runtime. -/// -/// **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. -/// -/// [unstable]: crate#unstable-features -#[derive(Debug)] -pub struct RuntimeStats { - /// Number of tasks that are scheduled from outside the runtime. - remote_schedule_count: AtomicU64, - /// Tracks per-worker performance counters - workers: Box<[WorkerStats]>, -} - -/// This type contains methods to retrieve stats from a worker thread on a Tokio runtime. -/// -/// **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. -/// -/// [unstable]: crate#unstable-features -#[derive(Debug)] -#[repr(align(128))] -pub struct WorkerStats { - /// Number of times the worker parked. - park_count: AtomicU64, - - /// Number of times the worker woke then parked again without doing work. - noop_count: AtomicU64, - - /// Number of times the worker attempted to steal. - steal_count: AtomicU64, - - /// Number of tasks the worker polled. - poll_count: AtomicU64, - - /// Number of tasks stolen from the current worker. - stolen_count: AtomicU64, - - /// Amount of time the worker spent doing work vs. parking. - busy_duration_total: AtomicU64, - - /// Number of tasks scheduled for execution on the worker's local queue. - local_schedule_count: AtomicU64, - - /// Number of tasks moved from the local queue to the global queue to free space. - overflow_count: AtomicU64, -} - -impl RuntimeStats { - pub(crate) fn new(worker_threads: usize) -> Self { - let mut workers = Vec::with_capacity(worker_threads); - for _ in 0..worker_threads { - workers.push(WorkerStats { - park_count: AtomicU64::new(0), - noop_count: AtomicU64::new(0), - steal_count: AtomicU64::new(0), - poll_count: AtomicU64::new(0), - stolen_count: AtomicU64::new(0), - overflow_count: AtomicU64::new(0), - busy_duration_total: AtomicU64::new(0), - local_schedule_count: AtomicU64::new(0), - }); - } - - Self { - remote_schedule_count: AtomicU64::new(0), - workers: workers.into_boxed_slice(), - } - } - - /// Returns the number of tasks scheduled from **outside** of the runtime. - /// - /// Tasks scheduled from outside of the runtime go via the runtime's - /// injection queue, which is usually is slower. - pub fn remote_schedule_count(&self) -> u64 { - self.remote_schedule_count.load(Relaxed) - } - - /// Returns a slice containing the worker stats for each worker thread. - pub fn workers(&self) -> impl Iterator { - self.workers.iter() - } - - /// Increment the number of tasks scheduled externally - pub(crate) fn inc_remote_schedule_count(&self) { - self.remote_schedule_count.fetch_add(1, Relaxed); - } - - pub(crate) fn worker(&self, index: usize) -> &WorkerStats { - &self.workers[index] - } -} - -impl WorkerStats { - /// Returns the total number of times this worker thread has parked. - pub fn park_count(&self) -> u64 { - self.park_count.load(Relaxed) - } - - /// Returns the number of times this worker unparked but performed no work. - /// - /// This is the false-positive wake count. - pub fn noop_count(&self) -> u64 { - self.noop_count.load(Relaxed) - } - - /// Returns the number of tasks this worker has stolen from other worker - /// threads. - pub fn steal_count(&self) -> u64 { - self.steal_count.load(Relaxed) - } - - /// Returns the number of tasks that were stolen from this worker. - pub fn stolen_count(&self) -> u64 { - self.stolen_count.load(Relaxed) - } - - /// Returns the number of times this worker has polled a task. - pub fn poll_count(&self) -> u64 { - self.poll_count.load(Relaxed) - } - - /// Returns the total amount of time this worker has been busy for. - pub fn total_busy_duration(&self) -> Duration { - Duration::from_nanos(self.busy_duration_total.load(Relaxed)) - } - - /// TODO - pub fn local_schedule_count(&self) -> u64 { - self.local_schedule_count.load(Relaxed) - } - - /// Returns the number of tasks moved from this worker's local queue to the - /// remote queue. - pub fn overflow_count(&self) -> u64 { - self.overflow_count.load(Relaxed) - } - - pub(crate) fn incr_stolen_count(&self, n: u16) { - self.stolen_count.fetch_add(n as _, Relaxed); - } -} - -pub(crate) struct WorkerStatsBatcher { - /// Identifies the worker within the runtime. - my_index: usize, - - /// Number of times the worker parked - park_count: u64, - - /// Number of times the worker woke w/o doing work. - noop_count: u64, - - /// Number of times stolen - steal_count: u64, - - /// Number of tasks that were polled by the worker. - poll_count: u64, - - /// Number of tasks polled when the worker entered park. This is used to - /// track the noop count. - poll_count_on_last_park: u64, - - /// Number of tasks that were scheduled locally on this worker. - local_schedule_count: u64, - - /// Number of tasks moved to the global queue to make space in the local - /// queue - overflow_count: u64, - - /// The total busy duration in nanoseconds. - busy_duration_total: u64, - last_resume_time: Instant, -} - -impl WorkerStatsBatcher { - pub(crate) fn new(my_index: usize) -> Self { - Self { - my_index, - park_count: 0, - noop_count: 0, - steal_count: 0, - poll_count: 0, - poll_count_on_last_park: 0, - local_schedule_count: 0, - overflow_count: 0, - busy_duration_total: 0, - last_resume_time: Instant::now(), - } - } - pub(crate) fn submit(&mut self, to: &RuntimeStats) { - let worker = &to.workers[self.my_index]; - - worker.park_count.store(self.park_count, Relaxed); - worker.noop_count.store(self.noop_count, Relaxed); - worker.steal_count.store(self.steal_count, Relaxed); - worker.poll_count.store(self.poll_count, Relaxed); - - worker - .busy_duration_total - .store(self.busy_duration_total, Relaxed); - - worker - .local_schedule_count - .store(self.local_schedule_count, Relaxed); - worker.overflow_count.store(self.overflow_count, Relaxed); - } - - /// The worker is about to park. - pub(crate) fn about_to_park(&mut self) { - self.park_count += 1; - - if self.poll_count_on_last_park == self.poll_count { - self.noop_count += 1; - } else { - self.poll_count_on_last_park = self.poll_count; - } - - let busy_duration = self.last_resume_time.elapsed(); - let busy_duration = u64::try_from(busy_duration.as_nanos()).unwrap_or(u64::MAX); - self.busy_duration_total += busy_duration; - } - - pub(crate) fn returned_from_park(&mut self) { - self.last_resume_time = Instant::now(); - } - - pub(crate) fn inc_local_schedule_count(&mut self) { - self.local_schedule_count += 1; - } - - pub(crate) fn incr_poll_count(&mut self) { - self.poll_count += 1; - } -} - -cfg_rt_multi_thread! { - impl WorkerStatsBatcher { - pub(crate) fn incr_steal_count(&mut self, by: u16) { - self.steal_count += by as u64; - } - - pub(crate) fn incr_overflow_count(&mut self, by: u16) { - self.overflow_count += by as u64; - } - } -} diff --git a/tokio/src/runtime/tests/queue.rs b/tokio/src/runtime/tests/queue.rs index 47f1b01d6a6..51a90416724 100644 --- a/tokio/src/runtime/tests/queue.rs +++ b/tokio/src/runtime/tests/queue.rs @@ -1,5 +1,5 @@ use crate::runtime::queue; -use crate::runtime::stats::WorkerStatsBatcher; +use crate::runtime::MetricsBatch; use crate::runtime::task::{self, Inject, Schedule, Task}; use std::thread; @@ -45,7 +45,7 @@ fn overflow() { #[test] fn steal_batch() { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(0); let (steal1, mut local1) = queue::local(); let (_, mut local2) = queue::local(); @@ -56,7 +56,7 @@ fn steal_batch() { local1.push_back(task, &inject); } - assert!(steal1.steal_into(&mut local2, &mut stats).is_some()); + assert!(steal1.steal_into(&mut local2, &mut metrics).is_some()); for _ in 0..1 { assert!(local2.pop().is_some()); diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 3e1ce448215..5348c19093d 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -9,9 +9,8 @@ pub(crate) use worker::Launch; pub(crate) use worker::block_in_place; use crate::loom::sync::Arc; -use crate::runtime::stats::RuntimeStats; use crate::runtime::task::JoinHandle; -use crate::runtime::{Callback, Parker}; +use crate::runtime::{Callback, Parker, RuntimeMetrics}; use std::fmt; use std::future::Future; @@ -103,8 +102,8 @@ impl Spawner { self.shared.close(); } - pub(crate) fn stats(&self) -> &RuntimeStats { - self.shared.stats() + pub(crate) fn metrics(&self) -> &RuntimeMetrics { + self.shared.metrics() } } diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index b1da35c71e3..923e1d38bc1 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -64,10 +64,9 @@ use crate::park::{Park, Unpark}; use crate::runtime; use crate::runtime::enter::EnterContext; use crate::runtime::park::{Parker, Unparker}; -use crate::runtime::stats::{RuntimeStats, WorkerStatsBatcher}; use crate::runtime::task::{Inject, JoinHandle, OwnedTasks}; use crate::runtime::thread_pool::Idle; -use crate::runtime::{queue, task, Callback}; +use crate::runtime::{queue, task, Callback, RuntimeMetrics, MetricsBatch}; use crate::util::atomic_cell::AtomicCell; use crate::util::FastRand; @@ -114,8 +113,8 @@ struct Core { /// borrow checker happy. park: Option, - /// Batching stats so they can be submitted to RuntimeStats. - stats: WorkerStatsBatcher, + /// Batching metrics so they can be submitted to RuntimeMetrics. + metrics: MetricsBatch, /// Fast random number generator. rand: FastRand, @@ -148,8 +147,8 @@ pub(super) struct Shared { /// Callback for a worker unparking itself after_unpark: Option, - /// Collects stats from the runtime. - stats: RuntimeStats, + /// Collects metrics from the runtime. + metrics: RuntimeMetrics, } /// Used to communicate with a worker from other threads. @@ -210,7 +209,7 @@ pub(super) fn create( is_searching: false, is_shutdown: false, park: Some(park), - stats: WorkerStatsBatcher::new(i), + metrics: MetricsBatch::new(i), rand: FastRand::new(seed()), })); @@ -225,7 +224,7 @@ pub(super) fn create( shutdown_cores: Mutex::new(vec![]), before_park, after_unpark, - stats: RuntimeStats::new(size), + metrics: RuntimeMetrics::new(size), }); let mut launch = Launch(vec![]); @@ -413,7 +412,7 @@ impl Context { core.transition_from_searching(&self.worker); // Make the core available to the runtime context - core.stats.incr_poll_count(); + core.metrics.incr_poll_count(); *self.core.borrow_mut() = Some(core); // Run the task @@ -438,7 +437,7 @@ impl Context { if coop::has_budget_remaining() { // Run the LIFO task, then loop - core.stats.incr_poll_count(); + core.metrics.incr_poll_count(); *self.core.borrow_mut() = Some(core); let task = self.worker.shared.owned.assert_owner(task); task.run(); @@ -446,7 +445,7 @@ impl Context { // Not enough budget left to run the LIFO task, push it to // the back of the queue and return. core.run_queue - .push_back(task, self.worker.inject(), &mut core.stats); + .push_back(task, self.worker.inject(), &mut core.metrics); return Ok(core); } } @@ -473,9 +472,9 @@ impl Context { if core.transition_to_parked(&self.worker) { while !core.is_shutdown { - core.stats.about_to_park(); + core.metrics.about_to_park(); core = self.park_timeout(core, None); - core.stats.returned_from_park(); + core.metrics.returned_from_park(); // Run regularly scheduled maintenance core.maintenance(&self.worker); @@ -558,11 +557,11 @@ impl Core { } let target = &worker.shared.remotes[i]; - let target_stats = worker.shared.stats.worker(i); + let target_metrics = worker.shared.metrics.worker(i); if let Some(task) = target .steal - .steal_into(&mut self.run_queue, &mut self.stats, target_stats) + .steal_into(&mut self.run_queue, &mut self.metrics, target_metrics) { return Some(task); } @@ -638,7 +637,7 @@ impl Core { /// Runs maintenance work such as checking the pool's state. fn maintenance(&mut self, worker: &Worker) { - self.stats.submit(&worker.shared.stats); + self.metrics.submit(&worker.shared.metrics); if !self.is_shutdown { // Check if the scheduler has been shutdown @@ -652,7 +651,7 @@ impl Core { // Signal to all tasks to shut down. worker.shared.owned.close_and_shutdown_all(); - self.stats.submit(&worker.shared.stats); + self.metrics.submit(&worker.shared.metrics); } /// Shuts down the core. @@ -703,8 +702,8 @@ impl Shared { handle } - pub(crate) fn stats(&self) -> &RuntimeStats { - &self.stats + pub(crate) fn metrics(&self) -> &RuntimeMetrics { + &self.metrics } pub(super) fn schedule(&self, task: Notified, is_yield: bool) { @@ -727,7 +726,7 @@ impl Shared { } fn schedule_local(&self, core: &mut Core, task: Notified, is_yield: bool) { - core.stats.inc_local_schedule_count(); + core.metrics.inc_local_schedule_count(); // Spawning from the worker thread. If scheduling a "yield" then the // task must always be pushed to the back of the queue, enabling other @@ -735,7 +734,7 @@ impl Shared { // flexibility and the task may go to the front of the queue. let should_notify = if is_yield { core.run_queue - .push_back(task, &self.inject, &mut core.stats); + .push_back(task, &self.inject, &mut core.metrics); true } else { // Push to the LIFO slot @@ -744,7 +743,7 @@ impl Shared { if let Some(prev) = prev { core.run_queue - .push_back(prev, &self.inject, &mut core.stats); + .push_back(prev, &self.inject, &mut core.metrics); } core.lifo_slot = Some(task); From 5895905ab5a6751ab59c991249e8737fdeb5fab9 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 12 Jan 2022 20:03:23 -0800 Subject: [PATCH 03/23] RuntimeMetrics -> SchedulerMetrics --- tokio/src/runtime/basic_scheduler.rs | 8 ++++---- tokio/src/runtime/handle.rs | 4 ++-- tokio/src/runtime/metrics/batch.rs | 4 ++-- tokio/src/runtime/metrics/mock.rs | 4 ++-- tokio/src/runtime/metrics/mod.rs | 6 +++--- .../src/runtime/metrics/{runtime.rs => scheduler.rs} | 12 ++++++------ tokio/src/runtime/mod.rs | 4 ++-- tokio/src/runtime/spawner.rs | 4 ++-- tokio/src/runtime/thread_pool/mod.rs | 4 ++-- tokio/src/runtime/thread_pool/worker.rs | 8 ++++---- 10 files changed, 29 insertions(+), 29 deletions(-) rename tokio/src/runtime/metrics/{runtime.rs => scheduler.rs} (88%) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 3b4509fdb74..0ed140b7729 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -4,7 +4,7 @@ use crate::loom::sync::Mutex; use crate::park::{Park, Unpark}; use crate::runtime::context::EnterGuard; use crate::runtime::driver::Driver; -use crate::runtime::{RuntimeMetrics, MetricsBatch}; +use crate::runtime::{SchedulerMetrics, MetricsBatch}; use crate::runtime::task::{self, JoinHandle, OwnedTasks, Schedule, Task}; use crate::runtime::Callback; use crate::sync::notify::Notify; @@ -99,7 +99,7 @@ struct Shared { after_unpark: Option, /// Keeps track of various runtime metrics. - metrics: RuntimeMetrics, + metrics: SchedulerMetrics, } /// Thread-local context. @@ -143,7 +143,7 @@ impl BasicScheduler { woken: AtomicBool::new(false), before_park, after_unpark, - metrics: RuntimeMetrics::new(1), + metrics: SchedulerMetrics::new(1), }), }; @@ -366,7 +366,7 @@ impl Spawner { handle } - pub(crate) fn metrics(&self) -> &RuntimeMetrics { + pub(crate) fn metrics(&self) -> &SchedulerMetrics { &self.shared.metrics } diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index 7d8ffb1c4f7..cdbc1185d2c 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -1,6 +1,6 @@ use crate::runtime::blocking::{BlockingTask, NoopSchedule}; use crate::runtime::task::{self, JoinHandle}; -use crate::runtime::{blocking, context, driver, Spawner, RuntimeMetrics}; +use crate::runtime::{blocking, context, driver, Spawner, SchedulerMetrics}; use crate::util::error::{CONTEXT_MISSING_ERROR, THREAD_LOCAL_DESTROYED_ERROR}; use std::future::Future; @@ -129,7 +129,7 @@ impl Handle { cfg_metrics! { /// Returns a view that lets you get information about how the runtime /// is performing. - pub fn metrics(&self) -> &RuntimeMetrics { + pub fn metrics(&self) -> &SchedulerMetrics { self.spawner.metrics() } } diff --git a/tokio/src/runtime/metrics/batch.rs b/tokio/src/runtime/metrics/batch.rs index 31005c35c61..84e89a1dc19 100644 --- a/tokio/src/runtime/metrics/batch.rs +++ b/tokio/src/runtime/metrics/batch.rs @@ -1,4 +1,4 @@ -use crate::runtime::RuntimeMetrics; +use crate::runtime::SchedulerMetrics; use std::convert::TryFrom; use std::sync::atomic::Ordering::Relaxed; @@ -52,7 +52,7 @@ impl MetricsBatch { } } - pub(crate) fn submit(&mut self, to: &RuntimeMetrics) { + pub(crate) fn submit(&mut self, to: &SchedulerMetrics) { let worker = to.worker(self.my_index); worker.park_count.store(self.park_count, Relaxed); diff --git a/tokio/src/runtime/metrics/mock.rs b/tokio/src/runtime/metrics/mock.rs index 40acf28061e..bc3abfb5a79 100644 --- a/tokio/src/runtime/metrics/mock.rs +++ b/tokio/src/runtime/metrics/mock.rs @@ -1,12 +1,12 @@ //! This file contains mocks of the types in src/runtime/metrics -pub(crate) struct RuntimeMetrics {} +pub(crate) struct SchedulerMetrics {} pub(crate) struct WorkerMetrics {} pub(crate) struct MetricsBatch {} -impl RuntimeMetrics { +impl SchedulerMetrics { pub(crate) fn new(_worker_threads: usize) -> Self { Self {} } diff --git a/tokio/src/runtime/metrics/mod.rs b/tokio/src/runtime/metrics/mod.rs index 31c3a7dcd99..11840f1510d 100644 --- a/tokio/src/runtime/metrics/mod.rs +++ b/tokio/src/runtime/metrics/mod.rs @@ -12,9 +12,9 @@ cfg_metrics! { mod batch; pub(crate) use batch::MetricsBatch; - mod runtime; + mod scheduler; #[allow(unreachable_pub)] // rust-lang/rust#57411 - pub use runtime::RuntimeMetrics; + pub use scheduler::SchedulerMetrics; mod worker; #[allow(unreachable_pub)] // rust-lang/rust#57411 @@ -24,5 +24,5 @@ cfg_metrics! { cfg_not_metrics! { mod mock; - pub(crate) use mock::{RuntimeMetrics, WorkerMetrics, MetricsBatch}; + pub(crate) use mock::{SchedulerMetrics, WorkerMetrics, MetricsBatch}; } diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/scheduler.rs similarity index 88% rename from tokio/src/runtime/metrics/runtime.rs rename to tokio/src/runtime/metrics/scheduler.rs index b1b37f2675d..8ff3110d0a0 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/scheduler.rs @@ -9,7 +9,7 @@ use crate::runtime::metrics::WorkerMetrics; /// /// [unstable]: crate#unstable-features #[derive(Debug)] -pub struct RuntimeMetrics { +pub struct SchedulerMetrics { /// Number of tasks that are scheduled from outside the runtime. remote_schedule_count: AtomicU64, @@ -17,11 +17,11 @@ pub struct RuntimeMetrics { workers: Box<[WorkerMetrics]>, } -impl RuntimeMetrics { - pub(crate) fn new(worker_threads: usize) -> RuntimeMetrics { - let mut workers = Vec::with_capacity(worker_threads); +impl SchedulerMetrics { + pub(crate) fn new(num_workers: usize) -> SchedulerMetrics { + let mut workers = Vec::with_capacity(num_workers); - for _ in 0..worker_threads { + for _ in 0..num_workers { workers.push(WorkerMetrics { park_count: AtomicU64::new(0), noop_count: AtomicU64::new(0), @@ -34,7 +34,7 @@ impl RuntimeMetrics { }); } - RuntimeMetrics { + SchedulerMetrics { remote_schedule_count: AtomicU64::new(0), workers: workers.into_boxed_slice(), } diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index bc94bf0cb3c..6f6e826bd4e 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -183,14 +183,14 @@ pub(crate) mod task; cfg_metrics! { mod metrics; - pub use metrics::{RuntimeMetrics, WorkerMetrics}; + pub use metrics::{SchedulerMetrics, WorkerMetrics}; pub(crate) use metrics::MetricsBatch; } cfg_not_metrics! { pub(crate) mod metrics; - pub(crate) use metrics::{RuntimeMetrics, WorkerMetrics, MetricsBatch}; + pub(crate) use metrics::{SchedulerMetrics, WorkerMetrics, MetricsBatch}; } cfg_rt! { diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index 90ba400479c..ab895eb0d38 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -1,5 +1,5 @@ use crate::future::Future; -use crate::runtime::RuntimeMetrics; +use crate::runtime::SchedulerMetrics; use crate::runtime::basic_scheduler; use crate::task::JoinHandle; @@ -39,7 +39,7 @@ impl Spawner { cfg_metrics! { impl Spawner { - pub(crate) fn metrics(&self) -> &RuntimeMetrics { + pub(crate) fn metrics(&self) -> &SchedulerMetrics { match self { Spawner::Basic(spawner) => spawner.metrics(), #[cfg(feature = "rt-multi-thread")] diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 5348c19093d..44ffe0c7bea 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -10,7 +10,7 @@ pub(crate) use worker::block_in_place; use crate::loom::sync::Arc; use crate::runtime::task::JoinHandle; -use crate::runtime::{Callback, Parker, RuntimeMetrics}; +use crate::runtime::{Callback, Parker, SchedulerMetrics}; use std::fmt; use std::future::Future; @@ -102,7 +102,7 @@ impl Spawner { self.shared.close(); } - pub(crate) fn metrics(&self) -> &RuntimeMetrics { + pub(crate) fn metrics(&self) -> &SchedulerMetrics { self.shared.metrics() } } diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 923e1d38bc1..d82a145e1a6 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -66,7 +66,7 @@ use crate::runtime::enter::EnterContext; use crate::runtime::park::{Parker, Unparker}; use crate::runtime::task::{Inject, JoinHandle, OwnedTasks}; use crate::runtime::thread_pool::Idle; -use crate::runtime::{queue, task, Callback, RuntimeMetrics, MetricsBatch}; +use crate::runtime::{queue, task, Callback, SchedulerMetrics, MetricsBatch}; use crate::util::atomic_cell::AtomicCell; use crate::util::FastRand; @@ -148,7 +148,7 @@ pub(super) struct Shared { after_unpark: Option, /// Collects metrics from the runtime. - metrics: RuntimeMetrics, + metrics: SchedulerMetrics, } /// Used to communicate with a worker from other threads. @@ -224,7 +224,7 @@ pub(super) fn create( shutdown_cores: Mutex::new(vec![]), before_park, after_unpark, - metrics: RuntimeMetrics::new(size), + metrics: SchedulerMetrics::new(size), }); let mut launch = Launch(vec![]); @@ -702,7 +702,7 @@ impl Shared { handle } - pub(crate) fn metrics(&self) -> &RuntimeMetrics { + pub(crate) fn metrics(&self) -> &SchedulerMetrics { &self.metrics } From 5bff8440312a14e0c680586cf88dfdb24b239b22 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 12 Jan 2022 21:53:53 -0800 Subject: [PATCH 04/23] refactor metrics --- tokio/src/runtime/basic_scheduler.rs | 35 ++++++++---- tokio/src/runtime/handle.rs | 6 +- tokio/src/runtime/metrics/batch.rs | 14 ++--- tokio/src/runtime/metrics/mod.rs | 10 ++-- tokio/src/runtime/metrics/runtime.rs | 74 +++++++++++++++++++++++++ tokio/src/runtime/metrics/scheduler.rs | 43 +------------- tokio/src/runtime/metrics/worker.rs | 58 +++++-------------- tokio/src/runtime/mod.rs | 4 +- tokio/src/runtime/queue.rs | 2 +- tokio/src/runtime/spawner.rs | 16 ++++-- tokio/src/runtime/tests/queue.rs | 2 +- tokio/src/runtime/thread_pool/mod.rs | 14 ++++- tokio/src/runtime/thread_pool/worker.rs | 27 +++++---- 13 files changed, 169 insertions(+), 136 deletions(-) create mode 100644 tokio/src/runtime/metrics/runtime.rs diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 0ed140b7729..7a37126e21e 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -4,9 +4,9 @@ use crate::loom::sync::Mutex; use crate::park::{Park, Unpark}; use crate::runtime::context::EnterGuard; use crate::runtime::driver::Driver; -use crate::runtime::{SchedulerMetrics, MetricsBatch}; use crate::runtime::task::{self, JoinHandle, OwnedTasks, Schedule, Task}; use crate::runtime::Callback; +use crate::runtime::{MetricsBatch, SchedulerMetrics, WorkerMetrics}; use crate::sync::notify::Notify; use crate::util::atomic_cell::AtomicCell; use crate::util::{waker_ref, Wake, WakerRef}; @@ -99,7 +99,10 @@ struct Shared { after_unpark: Option, /// Keeps track of various runtime metrics. - metrics: SchedulerMetrics, + scheduler_metrics: SchedulerMetrics, + + /// This scheduler only has one worker + worker_metrics: WorkerMetrics, } /// Thread-local context. @@ -143,7 +146,8 @@ impl BasicScheduler { woken: AtomicBool::new(false), before_park, after_unpark, - metrics: SchedulerMetrics::new(1), + scheduler_metrics: SchedulerMetrics::new(), + worker_metrics: WorkerMetrics::new(), }), }; @@ -152,7 +156,7 @@ impl BasicScheduler { spawner: spawner.clone(), tick: 0, driver: Some(driver), - metrics: MetricsBatch::new(0), + metrics: MetricsBatch::new(), }))); BasicScheduler { @@ -245,7 +249,7 @@ impl Context { if core.tasks.is_empty() { // Park until the thread is signaled core.metrics.about_to_park(); - core.metrics.submit(&core.spawner.shared.metrics); + core.metrics.submit(&core.spawner.shared.worker_metrics); let (c, _) = self.enter(core, || { driver.park().expect("failed to park"); @@ -271,7 +275,7 @@ impl Context { fn park_yield(&self, mut core: Box) -> Box { let mut driver = core.driver.take().expect("driver missing"); - core.metrics.submit(&core.spawner.shared.metrics); + core.metrics.submit(&core.spawner.shared.worker_metrics); let (mut core, _) = self.enter(core, || { driver .park_timeout(Duration::from_millis(0)) @@ -366,10 +370,6 @@ impl Spawner { handle } - pub(crate) fn metrics(&self) -> &SchedulerMetrics { - &self.shared.metrics - } - fn pop(&self) -> Option { match self.shared.queue.lock().as_mut() { Some(queue) => queue.pop_front(), @@ -390,6 +390,19 @@ impl Spawner { } } +cfg_metrics! { + impl Spawner { + pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { + &self.shared.scheduler_metrics + } + + pub(crate) fn worker_metrics(&self, worker: usize) -> &WorkerMetrics { + assert_eq!(1, worker); + &self.shared.worker_metrics + } + } +} + impl fmt::Debug for Spawner { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Spawner").finish() @@ -417,7 +430,7 @@ impl Schedule for Arc { } _ => { // Track that a task was scheduled from **outside** of the runtime. - self.metrics.inc_remote_schedule_count(); + self.scheduler_metrics.inc_remote_schedule_count(); // If the queue is None, then the runtime has shut down. We // don't need to do anything with the notification in that case. diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index cdbc1185d2c..44e9e510cac 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -1,6 +1,6 @@ use crate::runtime::blocking::{BlockingTask, NoopSchedule}; use crate::runtime::task::{self, JoinHandle}; -use crate::runtime::{blocking, context, driver, Spawner, SchedulerMetrics}; +use crate::runtime::{blocking, context, driver, RuntimeMetrics, Spawner}; use crate::util::error::{CONTEXT_MISSING_ERROR, THREAD_LOCAL_DESTROYED_ERROR}; use std::future::Future; @@ -129,8 +129,8 @@ impl Handle { cfg_metrics! { /// Returns a view that lets you get information about how the runtime /// is performing. - pub fn metrics(&self) -> &SchedulerMetrics { - self.spawner.metrics() + pub fn metrics(&self) -> RuntimeMetrics { + RuntimeMetrics::new(self.clone()) } } diff --git a/tokio/src/runtime/metrics/batch.rs b/tokio/src/runtime/metrics/batch.rs index 84e89a1dc19..e99a4d70390 100644 --- a/tokio/src/runtime/metrics/batch.rs +++ b/tokio/src/runtime/metrics/batch.rs @@ -1,13 +1,10 @@ -use crate::runtime::SchedulerMetrics; +use crate::runtime::WorkerMetrics; use std::convert::TryFrom; use std::sync::atomic::Ordering::Relaxed; use std::time::Instant; pub(crate) struct MetricsBatch { - /// Identifies the worker within the runtime. - my_index: usize, - /// Number of times the worker parked park_count: u64, @@ -37,9 +34,8 @@ pub(crate) struct MetricsBatch { } impl MetricsBatch { - pub(crate) fn new(my_index: usize) -> MetricsBatch { + pub(crate) fn new() -> MetricsBatch { MetricsBatch { - my_index, park_count: 0, noop_count: 0, steal_count: 0, @@ -52,9 +48,7 @@ impl MetricsBatch { } } - pub(crate) fn submit(&mut self, to: &SchedulerMetrics) { - let worker = to.worker(self.my_index); - + pub(crate) fn submit(&mut self, worker: &WorkerMetrics) { worker.park_count.store(self.park_count, Relaxed); worker.noop_count.store(self.noop_count, Relaxed); worker.steal_count.store(self.steal_count, Relaxed); @@ -108,4 +102,4 @@ cfg_rt_multi_thread! { self.overflow_count += by as u64; } } -} \ No newline at end of file +} diff --git a/tokio/src/runtime/metrics/mod.rs b/tokio/src/runtime/metrics/mod.rs index 11840f1510d..ca643a59047 100644 --- a/tokio/src/runtime/metrics/mod.rs +++ b/tokio/src/runtime/metrics/mod.rs @@ -12,13 +12,15 @@ cfg_metrics! { mod batch; pub(crate) use batch::MetricsBatch; - mod scheduler; + mod runtime; #[allow(unreachable_pub)] // rust-lang/rust#57411 - pub use scheduler::SchedulerMetrics; + pub use runtime::RuntimeMetrics; + + mod scheduler; + pub(crate) use scheduler::SchedulerMetrics; mod worker; - #[allow(unreachable_pub)] // rust-lang/rust#57411 - pub use worker::WorkerMetrics; + pub(crate) use worker::WorkerMetrics; } cfg_not_metrics! { diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs new file mode 100644 index 00000000000..f6fc84d41eb --- /dev/null +++ b/tokio/src/runtime/metrics/runtime.rs @@ -0,0 +1,74 @@ +use crate::runtime::Handle; + +use std::sync::atomic::Ordering::Relaxed; +use std::time::Duration; + +/// TODO: dox +#[derive(Debug)] +pub struct RuntimeMetrics { + handle: Handle, +} + +impl RuntimeMetrics { + pub(crate) fn new(handle: Handle) -> RuntimeMetrics { + RuntimeMetrics { handle } + } + + /// Returns the number of worker threads used by the runtime. + pub fn num_workers(&self) -> usize { + todo!(); + } + + /// Returns the number of tasks scheduled from **outside** of the runtime. + /// + /// Tasks scheduled from outside of the runtime go via the runtime's + /// injection queue, which is usually is slower. + pub fn remote_schedule_count(&self) -> u64 { + self.handle.spawner.scheduler_metrics().remote_schedule_count.load(Relaxed) + } + + /// Returns the total number of times this worker thread has parked. + pub fn worker_park_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).park_count.load(Relaxed) + } + + /// Returns the number of times this worker unparked but performed no work. + /// + /// This is the false-positive wake count. + pub fn worker_noop_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).noop_count.load(Relaxed) + } + + /// Returns the number of tasks this worker has stolen from other worker + /// threads. + pub fn worker_steal_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).steal_count.load(Relaxed) + } + + /// Returns the number of tasks that were stolen from this worker. + pub fn worker_stolen_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).stolen_count.load(Relaxed) + } + + /// Returns the number of times this worker has polled a task. + pub fn worker_poll_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).poll_count.load(Relaxed) + } + + /// Returns the total amount of time this worker has been busy for. + pub fn worker_total_busy_duration(&self, worker: usize) -> Duration { + let nanos = self.handle.spawner.worker_metrics(worker).busy_duration_total.load(Relaxed); + Duration::from_nanos(nanos) + } + + /// TODO + pub fn worker_local_schedule_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).local_schedule_count.load(Relaxed) + } + + /// Returns the number of tasks moved from this worker's local queue to the + /// remote queue. + pub fn worker_overflow_count(&self, worker: usize) -> u64 { + self.handle.spawner.worker_metrics(worker).overflow_count.load(Relaxed) + } +} \ No newline at end of file diff --git a/tokio/src/runtime/metrics/scheduler.rs b/tokio/src/runtime/metrics/scheduler.rs index 8ff3110d0a0..d1ba3b64420 100644 --- a/tokio/src/runtime/metrics/scheduler.rs +++ b/tokio/src/runtime/metrics/scheduler.rs @@ -1,5 +1,4 @@ use crate::loom::sync::atomic::{AtomicU64, Ordering::Relaxed}; -use crate::runtime::metrics::WorkerMetrics; /// Retrieves metrics from the Tokio runtime. /// @@ -9,56 +8,20 @@ use crate::runtime::metrics::WorkerMetrics; /// /// [unstable]: crate#unstable-features #[derive(Debug)] -pub struct SchedulerMetrics { +pub(crate) struct SchedulerMetrics { /// Number of tasks that are scheduled from outside the runtime. - remote_schedule_count: AtomicU64, - - /// Tracks per-worker metrics - workers: Box<[WorkerMetrics]>, + pub(super) remote_schedule_count: AtomicU64, } impl SchedulerMetrics { - pub(crate) fn new(num_workers: usize) -> SchedulerMetrics { - let mut workers = Vec::with_capacity(num_workers); - - for _ in 0..num_workers { - workers.push(WorkerMetrics { - park_count: AtomicU64::new(0), - noop_count: AtomicU64::new(0), - steal_count: AtomicU64::new(0), - poll_count: AtomicU64::new(0), - stolen_count: AtomicU64::new(0), - overflow_count: AtomicU64::new(0), - busy_duration_total: AtomicU64::new(0), - local_schedule_count: AtomicU64::new(0), - }); - } - + pub(crate) fn new() -> SchedulerMetrics { SchedulerMetrics { remote_schedule_count: AtomicU64::new(0), - workers: workers.into_boxed_slice(), } } - /// Returns the number of tasks scheduled from **outside** of the runtime. - /// - /// Tasks scheduled from outside of the runtime go via the runtime's - /// injection queue, which is usually is slower. - pub fn remote_schedule_count(&self) -> u64 { - self.remote_schedule_count.load(Relaxed) - } - - /// Returns a slice containing the metrics for each worker thread. - pub fn workers(&self) -> &[WorkerMetrics] { - &self.workers - } - /// Increment the number of tasks scheduled externally pub(crate) fn inc_remote_schedule_count(&self) { self.remote_schedule_count.fetch_add(1, Relaxed); } - - pub(crate) fn worker(&self, index: usize) -> &WorkerMetrics { - &self.workers[index] - } } diff --git a/tokio/src/runtime/metrics/worker.rs b/tokio/src/runtime/metrics/worker.rs index 78ed46a1a68..395e4e88724 100644 --- a/tokio/src/runtime/metrics/worker.rs +++ b/tokio/src/runtime/metrics/worker.rs @@ -1,6 +1,5 @@ use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering::Relaxed; -use std::time::Duration; /// Retreive runtime worker metrics. /// @@ -11,7 +10,7 @@ use std::time::Duration; /// [unstable]: crate#unstable-features #[derive(Debug)] #[repr(align(128))] -pub struct WorkerMetrics { +pub(crate) struct WorkerMetrics { /// Number of times the worker parked. pub(super) park_count: AtomicU64, @@ -38,51 +37,20 @@ pub struct WorkerMetrics { } impl WorkerMetrics { - /// Returns the total number of times this worker thread has parked. - pub fn park_count(&self) -> u64 { - self.park_count.load(Relaxed) - } - - /// Returns the number of times this worker unparked but performed no work. - /// - /// This is the false-positive wake count. - pub fn noop_count(&self) -> u64 { - self.noop_count.load(Relaxed) - } - - /// Returns the number of tasks this worker has stolen from other worker - /// threads. - pub fn steal_count(&self) -> u64 { - self.steal_count.load(Relaxed) - } - - /// Returns the number of tasks that were stolen from this worker. - pub fn stolen_count(&self) -> u64 { - self.stolen_count.load(Relaxed) - } - - /// Returns the number of times this worker has polled a task. - pub fn poll_count(&self) -> u64 { - self.poll_count.load(Relaxed) - } - - /// Returns the total amount of time this worker has been busy for. - pub fn total_busy_duration(&self) -> Duration { - Duration::from_nanos(self.busy_duration_total.load(Relaxed)) - } - - /// TODO - pub fn local_schedule_count(&self) -> u64 { - self.local_schedule_count.load(Relaxed) - } - - /// Returns the number of tasks moved from this worker's local queue to the - /// remote queue. - pub fn overflow_count(&self) -> u64 { - self.overflow_count.load(Relaxed) + pub(crate) fn new() -> WorkerMetrics { + WorkerMetrics { + park_count: AtomicU64::new(0), + noop_count: AtomicU64::new(0), + steal_count: AtomicU64::new(0), + poll_count: AtomicU64::new(0), + stolen_count: AtomicU64::new(0), + overflow_count: AtomicU64::new(0), + busy_duration_total: AtomicU64::new(0), + local_schedule_count: AtomicU64::new(0), + } } pub(crate) fn incr_stolen_count(&self, n: u16) { self.stolen_count.fetch_add(n as _, Relaxed); } -} \ No newline at end of file +} diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 6f6e826bd4e..035350f5350 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -183,9 +183,9 @@ pub(crate) mod task; cfg_metrics! { mod metrics; - pub use metrics::{SchedulerMetrics, WorkerMetrics}; + pub use metrics::RuntimeMetrics; - pub(crate) use metrics::MetricsBatch; + pub(crate) use metrics::{MetricsBatch, SchedulerMetrics, WorkerMetrics}; } cfg_not_metrics! { diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index c1cb15ebfdb..cc25c6b1c97 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -3,8 +3,8 @@ use crate::loom::cell::UnsafeCell; use crate::loom::sync::atomic::{AtomicU16, AtomicU32}; use crate::loom::sync::Arc; -use crate::runtime::{WorkerMetrics, MetricsBatch}; use crate::runtime::task::{self, Inject}; +use crate::runtime::{MetricsBatch, WorkerMetrics}; use std::mem::MaybeUninit; use std::ptr; diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index ab895eb0d38..03e1a602d91 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -1,6 +1,6 @@ use crate::future::Future; -use crate::runtime::SchedulerMetrics; use crate::runtime::basic_scheduler; +use crate::runtime::{SchedulerMetrics, WorkerMetrics}; use crate::task::JoinHandle; cfg_rt_multi_thread! { @@ -39,11 +39,19 @@ impl Spawner { cfg_metrics! { impl Spawner { - pub(crate) fn metrics(&self) -> &SchedulerMetrics { + pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { match self { - Spawner::Basic(spawner) => spawner.metrics(), + Spawner::Basic(spawner) => spawner.scheduler_metrics(), #[cfg(feature = "rt-multi-thread")] - Spawner::ThreadPool(spawner) => spawner.metrics(), + Spawner::ThreadPool(spawner) => spawner.scheduler_metrics(), + } + } + + pub(crate) fn worker_metrics(&self, worker: usize) -> &WorkerMetrics { + match self { + Spawner::Basic(spawner) => spawner.worker_metrics(worker), + #[cfg(feature = "rt-multi-thread")] + Spawner::ThreadPool(spawner) => spawner.worker_metrics(worker), } } } diff --git a/tokio/src/runtime/tests/queue.rs b/tokio/src/runtime/tests/queue.rs index 51a90416724..c6729e7dd7f 100644 --- a/tokio/src/runtime/tests/queue.rs +++ b/tokio/src/runtime/tests/queue.rs @@ -1,6 +1,6 @@ use crate::runtime::queue; -use crate::runtime::MetricsBatch; use crate::runtime::task::{self, Inject, Schedule, Task}; +use crate::runtime::MetricsBatch; use std::thread; use std::time::Duration; diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 44ffe0c7bea..551f7a1bac0 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -10,7 +10,7 @@ pub(crate) use worker::block_in_place; use crate::loom::sync::Arc; use crate::runtime::task::JoinHandle; -use crate::runtime::{Callback, Parker, SchedulerMetrics}; +use crate::runtime::{Callback, Parker, SchedulerMetrics, WorkerMetrics}; use std::fmt; use std::future::Future; @@ -101,9 +101,17 @@ impl Spawner { pub(crate) fn shutdown(&mut self) { self.shared.close(); } +} + +cfg_metrics! { + impl Spawner { + pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { + &self.shared.scheduler_metrics + } - pub(crate) fn metrics(&self) -> &SchedulerMetrics { - self.shared.metrics() + pub(crate) fn worker_metrics(&self, worker: usize) -> &WorkerMetrics { + &self.shared.worker_metrics[worker] + } } } diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index d82a145e1a6..5495eebef01 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -66,7 +66,7 @@ use crate::runtime::enter::EnterContext; use crate::runtime::park::{Parker, Unparker}; use crate::runtime::task::{Inject, JoinHandle, OwnedTasks}; use crate::runtime::thread_pool::Idle; -use crate::runtime::{queue, task, Callback, SchedulerMetrics, MetricsBatch}; +use crate::runtime::{queue, task, Callback, MetricsBatch, SchedulerMetrics, WorkerMetrics}; use crate::util::atomic_cell::AtomicCell; use crate::util::FastRand; @@ -148,7 +148,9 @@ pub(super) struct Shared { after_unpark: Option, /// Collects metrics from the runtime. - metrics: SchedulerMetrics, + pub(super) scheduler_metrics: SchedulerMetrics, + + pub(super) worker_metrics: Box<[WorkerMetrics]>, } /// Used to communicate with a worker from other threads. @@ -194,9 +196,10 @@ pub(super) fn create( ) -> (Arc, Launch) { let mut cores = vec![]; let mut remotes = vec![]; + let mut worker_metrics = vec![]; // Create the local queues - for i in 0..size { + for _ in 0..size { let (steal, run_queue) = queue::local(); let park = park.clone(); @@ -209,11 +212,12 @@ pub(super) fn create( is_searching: false, is_shutdown: false, park: Some(park), - metrics: MetricsBatch::new(i), + metrics: MetricsBatch::new(), rand: FastRand::new(seed()), })); remotes.push(Remote { steal, unpark }); + worker_metrics.push(WorkerMetrics::new()); } let shared = Arc::new(Shared { @@ -224,7 +228,8 @@ pub(super) fn create( shutdown_cores: Mutex::new(vec![]), before_park, after_unpark, - metrics: SchedulerMetrics::new(size), + scheduler_metrics: SchedulerMetrics::new(), + worker_metrics: worker_metrics.into_boxed_slice(), }); let mut launch = Launch(vec![]); @@ -557,7 +562,7 @@ impl Core { } let target = &worker.shared.remotes[i]; - let target_metrics = worker.shared.metrics.worker(i); + let target_metrics = &worker.shared.worker_metrics[i]; if let Some(task) = target .steal @@ -637,7 +642,8 @@ impl Core { /// Runs maintenance work such as checking the pool's state. fn maintenance(&mut self, worker: &Worker) { - self.metrics.submit(&worker.shared.metrics); + self.metrics + .submit(&worker.shared.worker_metrics[worker.index]); if !self.is_shutdown { // Check if the scheduler has been shutdown @@ -651,7 +657,8 @@ impl Core { // Signal to all tasks to shut down. worker.shared.owned.close_and_shutdown_all(); - self.metrics.submit(&worker.shared.metrics); + self.metrics + .submit(&worker.shared.worker_metrics[worker.index]); } /// Shuts down the core. @@ -702,10 +709,6 @@ impl Shared { handle } - pub(crate) fn metrics(&self) -> &SchedulerMetrics { - &self.metrics - } - pub(super) fn schedule(&self, task: Notified, is_yield: bool) { CURRENT.with(|maybe_cx| { if let Some(cx) = maybe_cx { From ea4a303f0676f1de22c425f04e7651bc1435a4d8 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 13 Jan 2022 13:26:52 -0800 Subject: [PATCH 05/23] instrument queue depth --- tokio/src/runtime/basic_scheduler.rs | 129 ++++++++++++++---------- tokio/src/runtime/metrics/mock.rs | 1 + tokio/src/runtime/metrics/runtime.rs | 62 ++++++++++-- tokio/src/runtime/metrics/worker.rs | 15 ++- tokio/src/runtime/queue.rs | 18 +++- tokio/src/runtime/spawner.rs | 8 ++ tokio/src/runtime/thread_pool/mod.rs | 4 + tokio/src/runtime/thread_pool/worker.rs | 8 ++ 8 files changed, 178 insertions(+), 67 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 7a37126e21e..65485845fd4 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -223,6 +223,81 @@ impl BasicScheduler { } } +impl Drop for BasicScheduler { + fn drop(&mut self) { + // Avoid a double panic if we are currently panicking and + // the lock may be poisoned. + + let core = match self.take_core() { + Some(core) => core, + None if std::thread::panicking() => return, + None => panic!("Oh no! We never placed the Core back, this is a bug!"), + }; + + core.enter(|mut core, context| { + // Drain the OwnedTasks collection. This call also closes the + // collection, ensuring that no tasks are ever pushed after this + // call returns. + context.spawner.shared.owned.close_and_shutdown_all(); + + // Drain local queue + // We already shut down every task, so we just need to drop the task. + while let Some(task) = core.pop_task() { + drop(task); + } + + // Drain remote queue and set it to None + let remote_queue = core.spawner.shared.queue.lock().take(); + + // Using `Option::take` to replace the shared queue with `None`. + // We already shut down every task, so we just need to drop the task. + if let Some(remote_queue) = remote_queue { + for entry in remote_queue { + match entry { + RemoteMsg::Schedule(task) => { + drop(task); + } + } + } + } + + assert!(context.spawner.shared.owned.is_empty()); + + (core, ()) + }); + } +} + +impl fmt::Debug for BasicScheduler { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.debug_struct("BasicScheduler").finish() + } +} + +// ===== impl Core ===== + +impl Core { + fn pop_task(&mut self) -> Option>> { + let ret = self.tasks.pop_front(); + self.spawner + .shared + .worker_metrics + .set_queue_depth(self.tasks.len()); + ret + } + + fn push_task(&mut self, task: task::Notified>) { + self.tasks.push_back(task); + self.metrics.inc_local_schedule_count(); + self.spawner + .shared + .worker_metrics + .set_queue_depth(self.tasks.len()); + } +} + +// ===== impl Context ===== + impl Context { /// Execute the closure with the given scheduler core stored in the /// thread-local context. @@ -301,57 +376,6 @@ impl Context { } } -impl Drop for BasicScheduler { - fn drop(&mut self) { - // Avoid a double panic if we are currently panicking and - // the lock may be poisoned. - - let core = match self.take_core() { - Some(core) => core, - None if std::thread::panicking() => return, - None => panic!("Oh no! We never placed the Core back, this is a bug!"), - }; - - core.enter(|mut core, context| { - // Drain the OwnedTasks collection. This call also closes the - // collection, ensuring that no tasks are ever pushed after this - // call returns. - context.spawner.shared.owned.close_and_shutdown_all(); - - // Drain local queue - // We already shut down every task, so we just need to drop the task. - while let Some(task) = core.tasks.pop_front() { - drop(task); - } - - // Drain remote queue and set it to None - let remote_queue = core.spawner.shared.queue.lock().take(); - - // Using `Option::take` to replace the shared queue with `None`. - // We already shut down every task, so we just need to drop the task. - if let Some(remote_queue) = remote_queue { - for entry in remote_queue { - match entry { - RemoteMsg::Schedule(task) => { - drop(task); - } - } - } - } - - assert!(context.spawner.shared.owned.is_empty()); - - (core, ()) - }); - } -} - -impl fmt::Debug for BasicScheduler { - fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt.debug_struct("BasicScheduler").finish() - } -} - // ===== impl Spawner ===== impl Spawner { @@ -424,8 +448,7 @@ impl Schedule for Arc { // If `None`, the runtime is shutting down, so there is no need // to schedule the task. if let Some(core) = core.as_mut() { - core.metrics.inc_local_schedule_count(); - core.tasks.push_back(task); + core.push_task(task); } } _ => { diff --git a/tokio/src/runtime/metrics/mock.rs b/tokio/src/runtime/metrics/mock.rs index bc3abfb5a79..bfeb621142e 100644 --- a/tokio/src/runtime/metrics/mock.rs +++ b/tokio/src/runtime/metrics/mock.rs @@ -21,6 +21,7 @@ impl SchedulerMetrics { impl WorkerMetrics { pub(crate) fn incr_stolen_count(&self, _n: u16) {} + pub(crate) fn set_queue_depth(&self, len: usize) {} } impl MetricsBatch { diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index f6fc84d41eb..17921b7909c 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -24,51 +24,93 @@ impl RuntimeMetrics { /// Tasks scheduled from outside of the runtime go via the runtime's /// injection queue, which is usually is slower. pub fn remote_schedule_count(&self) -> u64 { - self.handle.spawner.scheduler_metrics().remote_schedule_count.load(Relaxed) + self.handle + .spawner + .scheduler_metrics() + .remote_schedule_count + .load(Relaxed) } /// Returns the total number of times this worker thread has parked. pub fn worker_park_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).park_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .park_count + .load(Relaxed) } /// Returns the number of times this worker unparked but performed no work. /// /// This is the false-positive wake count. pub fn worker_noop_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).noop_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .noop_count + .load(Relaxed) } /// Returns the number of tasks this worker has stolen from other worker /// threads. pub fn worker_steal_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).steal_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .steal_count + .load(Relaxed) } /// Returns the number of tasks that were stolen from this worker. pub fn worker_stolen_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).stolen_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .stolen_count + .load(Relaxed) } /// Returns the number of times this worker has polled a task. pub fn worker_poll_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).poll_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .poll_count + .load(Relaxed) } /// Returns the total amount of time this worker has been busy for. pub fn worker_total_busy_duration(&self, worker: usize) -> Duration { - let nanos = self.handle.spawner.worker_metrics(worker).busy_duration_total.load(Relaxed); + let nanos = self + .handle + .spawner + .worker_metrics(worker) + .busy_duration_total + .load(Relaxed); Duration::from_nanos(nanos) } /// TODO pub fn worker_local_schedule_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).local_schedule_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .local_schedule_count + .load(Relaxed) } /// Returns the number of tasks moved from this worker's local queue to the /// remote queue. pub fn worker_overflow_count(&self, worker: usize) -> u64 { - self.handle.spawner.worker_metrics(worker).overflow_count.load(Relaxed) + self.handle + .spawner + .worker_metrics(worker) + .overflow_count + .load(Relaxed) } -} \ No newline at end of file + + /// Returns the number of tasks currently in the worker's local queue + pub fn worker_local_queue_depth(&self, worker: usize) -> usize { + self.handle.spawner.worker_local_queue_depth(worker) + } +} diff --git a/tokio/src/runtime/metrics/worker.rs b/tokio/src/runtime/metrics/worker.rs index 395e4e88724..8cd69e089dc 100644 --- a/tokio/src/runtime/metrics/worker.rs +++ b/tokio/src/runtime/metrics/worker.rs @@ -1,5 +1,5 @@ -use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering::Relaxed; +use std::sync::atomic::{AtomicU64, AtomicUsize}; /// Retreive runtime worker metrics. /// @@ -34,6 +34,10 @@ pub(crate) struct WorkerMetrics { /// Number of tasks moved from the local queue to the global queue to free space. pub(super) overflow_count: AtomicU64, + + /// Number of tasks currently in the local queue. Used only by the + /// current-thread scheduler. + pub(super) queue_depth: AtomicUsize, } impl WorkerMetrics { @@ -47,10 +51,19 @@ impl WorkerMetrics { overflow_count: AtomicU64::new(0), busy_duration_total: AtomicU64::new(0), local_schedule_count: AtomicU64::new(0), + queue_depth: AtomicUsize::new(0), } } pub(crate) fn incr_stolen_count(&self, n: u16) { self.stolen_count.fetch_add(n as _, Relaxed); } + + pub(crate) fn queue_depth(&self) -> usize { + self.queue_depth.load(Relaxed) + } + + pub(crate) fn set_queue_depth(&self, len: usize) { + self.queue_depth.store(len, Relaxed); + } } diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index cc25c6b1c97..e7e6724d28f 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -303,7 +303,7 @@ impl Local { impl Steal { pub(super) fn is_empty(&self) -> bool { - self.0.is_empty() + self.0.len() == 0 } /// Steals half the tasks from self and place them into `dst`. @@ -459,6 +459,14 @@ impl Steal { } } +cfg_metrics! { + impl Steal { + pub(crate) fn len(&self) -> usize { + self.0.len() as _ + } + } +} + impl Clone for Steal { fn clone(&self) -> Steal { Steal(self.0.clone()) @@ -474,11 +482,15 @@ impl Drop for Local { } impl Inner { - fn is_empty(&self) -> bool { + fn len(&self) -> u16 { let (_, head) = unpack(self.head.load(Acquire)); let tail = self.tail.load(Acquire); - head == tail + tail.wrapping_sub(head) + } + + fn is_empty(&self) -> bool { + self.len() == 0 } } diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index 03e1a602d91..b5e38a37aca 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -54,5 +54,13 @@ cfg_metrics! { Spawner::ThreadPool(spawner) => spawner.worker_metrics(worker), } } + + pub(crate) fn worker_local_queue_depth(&self, worker: usize) -> usize { + match self { + Spawner::Basic(spawner) => spawner.worker_metrics(worker).queue_depth(), + #[cfg(feature = "rt-multi-thread")] + Spawner::ThreadPool(spawner) => spawner.worker_local_queue_depth(worker), + } + } } } diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 551f7a1bac0..39413bfc3a5 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -112,6 +112,10 @@ cfg_metrics! { pub(crate) fn worker_metrics(&self, worker: usize) -> &WorkerMetrics { &self.shared.worker_metrics[worker] } + + pub(crate) fn worker_local_queue_depth(&self, worker: usize) -> usize { + self.shared.worker_local_queue_depth(worker) + } } } diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 5495eebef01..ed5f0796c2b 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -831,3 +831,11 @@ impl Shared { std::ptr::eq(self, other) } } + +cfg_metrics! { + impl Shared { + pub(super) fn worker_local_queue_depth(&self, worker: usize) -> usize { + self.remotes[worker].steal.len() + } + } +} From fc8f28a49c09c56def103e0ab9b65e092562420f Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 13 Jan 2022 13:32:46 -0800 Subject: [PATCH 06/23] remove stolen count --- tokio/src/runtime/metrics/runtime.rs | 9 --------- tokio/src/runtime/metrics/worker.rs | 8 -------- tokio/src/runtime/queue.rs | 4 +--- tokio/src/runtime/thread_pool/worker.rs | 3 +-- 4 files changed, 2 insertions(+), 22 deletions(-) diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index 17921b7909c..27d1f9a652a 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -61,15 +61,6 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the number of tasks that were stolen from this worker. - pub fn worker_stolen_count(&self, worker: usize) -> u64 { - self.handle - .spawner - .worker_metrics(worker) - .stolen_count - .load(Relaxed) - } - /// Returns the number of times this worker has polled a task. pub fn worker_poll_count(&self, worker: usize) -> u64 { self.handle diff --git a/tokio/src/runtime/metrics/worker.rs b/tokio/src/runtime/metrics/worker.rs index 8cd69e089dc..3f54f31594d 100644 --- a/tokio/src/runtime/metrics/worker.rs +++ b/tokio/src/runtime/metrics/worker.rs @@ -23,9 +23,6 @@ pub(crate) struct WorkerMetrics { /// Number of tasks the worker polled. pub(super) poll_count: AtomicU64, - /// Number of tasks stolen from the current worker. - pub(super) stolen_count: AtomicU64, - /// Amount of time the worker spent doing work vs. parking. pub(super) busy_duration_total: AtomicU64, @@ -47,7 +44,6 @@ impl WorkerMetrics { noop_count: AtomicU64::new(0), steal_count: AtomicU64::new(0), poll_count: AtomicU64::new(0), - stolen_count: AtomicU64::new(0), overflow_count: AtomicU64::new(0), busy_duration_total: AtomicU64::new(0), local_schedule_count: AtomicU64::new(0), @@ -55,10 +51,6 @@ impl WorkerMetrics { } } - pub(crate) fn incr_stolen_count(&self, n: u16) { - self.stolen_count.fetch_add(n as _, Relaxed); - } - pub(crate) fn queue_depth(&self) -> usize { self.queue_depth.load(Relaxed) } diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index e7e6724d28f..25ccda37796 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -4,7 +4,7 @@ use crate::loom::cell::UnsafeCell; use crate::loom::sync::atomic::{AtomicU16, AtomicU32}; use crate::loom::sync::Arc; use crate::runtime::task::{self, Inject}; -use crate::runtime::{MetricsBatch, WorkerMetrics}; +use crate::runtime::MetricsBatch; use std::mem::MaybeUninit; use std::ptr; @@ -311,7 +311,6 @@ impl Steal { &self, dst: &mut Local, dst_metrics: &mut MetricsBatch, - src_metrics: &WorkerMetrics, ) -> Option> { // Safety: the caller is the only thread that mutates `dst.tail` and // holds a mutable reference. @@ -338,7 +337,6 @@ impl Steal { } dst_metrics.incr_steal_count(n); - src_metrics.incr_stolen_count(n); // We are returning a task here n -= 1; diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index ed5f0796c2b..1e757ddd8a7 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -562,11 +562,10 @@ impl Core { } let target = &worker.shared.remotes[i]; - let target_metrics = &worker.shared.worker_metrics[i]; if let Some(task) = target .steal - .steal_into(&mut self.run_queue, &mut self.metrics, target_metrics) + .steal_into(&mut self.run_queue, &mut self.metrics) { return Some(task); } From b60a16a4ed8ea96200c408cbd9a69932f1653874 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 13 Jan 2022 14:34:17 -0800 Subject: [PATCH 07/23] fix build --- tokio/src/runtime/handle.rs | 22 +++++++++++++--------- tokio/src/runtime/metrics/mock.rs | 17 ++++++++--------- tokio/src/runtime/metrics/runtime.rs | 2 +- tokio/src/runtime/spawner.rs | 3 ++- tokio/src/runtime/thread_pool/mod.rs | 4 +++- tokio/src/runtime/thread_pool/worker.rs | 8 ++++---- 6 files changed, 31 insertions(+), 25 deletions(-) diff --git a/tokio/src/runtime/handle.rs b/tokio/src/runtime/handle.rs index 44e9e510cac..3481a2552f3 100644 --- a/tokio/src/runtime/handle.rs +++ b/tokio/src/runtime/handle.rs @@ -1,6 +1,6 @@ use crate::runtime::blocking::{BlockingTask, NoopSchedule}; use crate::runtime::task::{self, JoinHandle}; -use crate::runtime::{blocking, context, driver, RuntimeMetrics, Spawner}; +use crate::runtime::{blocking, context, driver, Spawner}; use crate::util::error::{CONTEXT_MISSING_ERROR, THREAD_LOCAL_DESTROYED_ERROR}; use std::future::Future; @@ -126,14 +126,6 @@ impl Handle { context::try_current() } - cfg_metrics! { - /// Returns a view that lets you get information about how the runtime - /// is performing. - pub fn metrics(&self) -> RuntimeMetrics { - RuntimeMetrics::new(self.clone()) - } - } - /// Spawns a future onto the Tokio runtime. /// /// This spawns the given future onto the runtime's executor, usually a @@ -327,6 +319,18 @@ impl Handle { } } +cfg_metrics! { + use crate::runtime::RuntimeMetrics; + + impl Handle { + /// Returns a view that lets you get information about how the runtime + /// is performing. + pub fn metrics(&self) -> RuntimeMetrics { + RuntimeMetrics::new(self.clone()) + } + } +} + /// Error returned by `try_current` when no Runtime has been started #[derive(Debug)] pub struct TryCurrentError { diff --git a/tokio/src/runtime/metrics/mock.rs b/tokio/src/runtime/metrics/mock.rs index bfeb621142e..ef75e6379ab 100644 --- a/tokio/src/runtime/metrics/mock.rs +++ b/tokio/src/runtime/metrics/mock.rs @@ -7,29 +7,28 @@ pub(crate) struct WorkerMetrics {} pub(crate) struct MetricsBatch {} impl SchedulerMetrics { - pub(crate) fn new(_worker_threads: usize) -> Self { + pub(crate) fn new() -> Self { Self {} } /// Increment the number of tasks scheduled externally pub(crate) fn inc_remote_schedule_count(&self) {} - - pub(crate) fn worker(&self, _index: usize) -> &WorkerMetrics { - &WorkerMetrics {} - } } impl WorkerMetrics { - pub(crate) fn incr_stolen_count(&self, _n: u16) {} - pub(crate) fn set_queue_depth(&self, len: usize) {} + pub(crate) fn new() -> Self { + Self {} + } + + pub(crate) fn set_queue_depth(&self, _len: usize) {} } impl MetricsBatch { - pub(crate) fn new(_my_index: usize) -> Self { + pub(crate) fn new() -> Self { Self {} } - pub(crate) fn submit(&mut self, _to: &RuntimeMetrics) {} + pub(crate) fn submit(&mut self, _to: &WorkerMetrics) {} pub(crate) fn about_to_park(&mut self) {} pub(crate) fn returned_from_park(&mut self) {} pub(crate) fn incr_poll_count(&mut self) {} diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index 27d1f9a652a..ee265e25f09 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -4,7 +4,7 @@ use std::sync::atomic::Ordering::Relaxed; use std::time::Duration; /// TODO: dox -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct RuntimeMetrics { handle: Handle, } diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index b5e38a37aca..3956a01a135 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -1,6 +1,5 @@ use crate::future::Future; use crate::runtime::basic_scheduler; -use crate::runtime::{SchedulerMetrics, WorkerMetrics}; use crate::task::JoinHandle; cfg_rt_multi_thread! { @@ -38,6 +37,8 @@ impl Spawner { } cfg_metrics! { + use crate::runtime::{SchedulerMetrics, WorkerMetrics}; + impl Spawner { pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { match self { diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 39413bfc3a5..386953ebe7e 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -10,7 +10,7 @@ pub(crate) use worker::block_in_place; use crate::loom::sync::Arc; use crate::runtime::task::JoinHandle; -use crate::runtime::{Callback, Parker, SchedulerMetrics, WorkerMetrics}; +use crate::runtime::{Callback, Parker}; use std::fmt; use std::future::Future; @@ -104,6 +104,8 @@ impl Spawner { } cfg_metrics! { + use crate::runtime::{SchedulerMetrics, WorkerMetrics}; + impl Spawner { pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { &self.shared.scheduler_metrics diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 1e757ddd8a7..62fe8d89bac 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -562,10 +562,9 @@ impl Core { } let target = &worker.shared.remotes[i]; - if let Some(task) = - target - .steal - .steal_into(&mut self.run_queue, &mut self.metrics) + if let Some(task) = target + .steal + .steal_into(&mut self.run_queue, &mut self.metrics) { return Some(task); } @@ -723,6 +722,7 @@ impl Shared { // Otherwise, use the inject queue. self.inject.push(task); + self.scheduler_metrics.inc_remote_schedule_count(); self.notify_parked(); }) } From b0098f07dcc67d91164fcfc7042661edfc75a712 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 13 Jan 2022 15:07:25 -0800 Subject: [PATCH 08/23] instrument remote queue --- tokio/src/runtime/basic_scheduler.rs | 11 ++++++++++- tokio/src/runtime/metrics/runtime.rs | 7 ++++++- tokio/src/runtime/spawner.rs | 16 ++++++++++++++++ tokio/src/runtime/thread_pool/mod.rs | 8 ++++++++ tokio/src/runtime/thread_pool/worker.rs | 4 ++++ 5 files changed, 44 insertions(+), 2 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 65485845fd4..8381f74f593 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -420,8 +420,17 @@ cfg_metrics! { &self.shared.scheduler_metrics } + pub(crate) fn remote_queue_depth(&self) -> usize { + // TODO: avoid having to lock. The multi-threaded injection queue + // could probably be used here. + self.shared.queue.lock() + .as_ref() + .map(|queue| queue.len()) + .unwrap_or(0) + } + pub(crate) fn worker_metrics(&self, worker: usize) -> &WorkerMetrics { - assert_eq!(1, worker); + assert_eq!(0, worker); &self.shared.worker_metrics } } diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index ee265e25f09..6c5afd093ef 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -16,7 +16,7 @@ impl RuntimeMetrics { /// Returns the number of worker threads used by the runtime. pub fn num_workers(&self) -> usize { - todo!(); + self.handle.spawner.num_workers() } /// Returns the number of tasks scheduled from **outside** of the runtime. @@ -100,6 +100,11 @@ impl RuntimeMetrics { .load(Relaxed) } + /// TODO + pub fn remote_queue_depth(&self) -> usize { + self.handle.spawner.remote_queue_depth() + } + /// Returns the number of tasks currently in the worker's local queue pub fn worker_local_queue_depth(&self, worker: usize) -> usize { self.handle.spawner.worker_local_queue_depth(worker) diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index 3956a01a135..82337b4a415 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -40,6 +40,14 @@ cfg_metrics! { use crate::runtime::{SchedulerMetrics, WorkerMetrics}; impl Spawner { + pub(crate) fn num_workers(&self) -> usize { + match self { + Spawner::Basic(_) => 1, + #[cfg(feature = "rt-multi-thread")] + Spawner::ThreadPool(spawner) => spawner.num_workers(), + } + } + pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { match self { Spawner::Basic(spawner) => spawner.scheduler_metrics(), @@ -56,6 +64,14 @@ cfg_metrics! { } } + pub(crate) fn remote_queue_depth(&self) -> usize { + match self { + Spawner::Basic(spawner) => spawner.remote_queue_depth(), + #[cfg(feature = "rt-multi-thread")] + Spawner::ThreadPool(spawner) => spawner.remote_queue_depth(), + } + } + pub(crate) fn worker_local_queue_depth(&self, worker: usize) -> usize { match self { Spawner::Basic(spawner) => spawner.worker_metrics(worker).queue_depth(), diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 386953ebe7e..2fc83a99013 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -107,6 +107,10 @@ cfg_metrics! { use crate::runtime::{SchedulerMetrics, WorkerMetrics}; impl Spawner { + pub(crate) fn num_workers(&self) -> usize { + self.shared.worker_metrics.len() + } + pub(crate) fn scheduler_metrics(&self) -> &SchedulerMetrics { &self.shared.scheduler_metrics } @@ -115,6 +119,10 @@ cfg_metrics! { &self.shared.worker_metrics[worker] } + pub(crate) fn remote_queue_depth(&self) -> usize { + self.shared.remote_queue_depth() + } + pub(crate) fn worker_local_queue_depth(&self, worker: usize) -> usize { self.shared.worker_local_queue_depth(worker) } diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 62fe8d89bac..6c10fe996c1 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -833,6 +833,10 @@ impl Shared { cfg_metrics! { impl Shared { + pub(super) fn remote_queue_depth(&self) -> usize { + self.inject.len() + } + pub(super) fn worker_local_queue_depth(&self, worker: usize) -> usize { self.remotes[worker].steal.len() } From 852406bced8249d1273f5bd5d99cf413baf2d587 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 14 Jan 2022 12:02:37 -0800 Subject: [PATCH 09/23] fix queue tests --- tokio/src/runtime/tests/queue.rs | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/tokio/src/runtime/tests/queue.rs b/tokio/src/runtime/tests/queue.rs index c6729e7dd7f..e0744b97430 100644 --- a/tokio/src/runtime/tests/queue.rs +++ b/tokio/src/runtime/tests/queue.rs @@ -9,10 +9,11 @@ use std::time::Duration; fn fits_256() { let (_, mut local) = queue::local(); let inject = Inject::new(); + let mut metrics = MetricsBatch::new(); for _ in 0..256 { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } assert!(inject.pop().is_none()); @@ -24,10 +25,11 @@ fn fits_256() { fn overflow() { let (_, mut local) = queue::local(); let inject = Inject::new(); + let mut metrics = MetricsBatch::new(); for _ in 0..257 { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } let mut n = 0; @@ -45,7 +47,7 @@ fn overflow() { #[test] fn steal_batch() { - let mut metrics = MetricsBatch::new(0); + let mut metrics = MetricsBatch::new(); let (steal1, mut local1) = queue::local(); let (_, mut local2) = queue::local(); @@ -53,7 +55,7 @@ fn steal_batch() { for _ in 0..4 { let (task, _) = super::unowned(async {}); - local1.push_back(task, &inject); + local1.push_back(task, &inject, &mut metrics); } assert!(steal1.steal_into(&mut local2, &mut metrics).is_some()); @@ -79,12 +81,14 @@ fn stress1() { const NUM_PUSH: usize = 500; const NUM_POP: usize = 250; + let mut metrics = MetricsBatch::new(); + for _ in 0..NUM_ITER { let (steal, mut local) = queue::local(); let inject = Inject::new(); let th = thread::spawn(move || { - let mut stats = WorkerStatsBatcher::new(0); + let mut stats = MetricsBatch::new(); let (_, mut local) = queue::local(); let mut n = 0; @@ -108,7 +112,7 @@ fn stress1() { for _ in 0..NUM_LOCAL { for _ in 0..NUM_PUSH { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } for _ in 0..NUM_POP { @@ -136,12 +140,14 @@ fn stress2() { const NUM_TASKS: usize = 1_000_000; const NUM_STEAL: usize = 1_000; + let mut metrics = MetricsBatch::new(); + for _ in 0..NUM_ITER { let (steal, mut local) = queue::local(); let inject = Inject::new(); let th = thread::spawn(move || { - let mut stats = WorkerStatsBatcher::new(0); + let mut stats = MetricsBatch::new(); let (_, mut local) = queue::local(); let mut n = 0; @@ -164,7 +170,7 @@ fn stress2() { for i in 0..NUM_TASKS { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); if i % 128 == 0 && local.pop().is_some() { num_pop += 1; From 7ed15d1e0cae49187026a8b1352d0232b5d436a1 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 14 Jan 2022 13:15:42 -0800 Subject: [PATCH 10/23] fix loom tests --- tokio/src/runtime/tests/loom_queue.rs | 40 ++++++++++++++------------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/tokio/src/runtime/tests/loom_queue.rs b/tokio/src/runtime/tests/loom_queue.rs index 2cbb0a18b53..b5f78d7ebe7 100644 --- a/tokio/src/runtime/tests/loom_queue.rs +++ b/tokio/src/runtime/tests/loom_queue.rs @@ -1,7 +1,6 @@ use crate::runtime::blocking::NoopSchedule; -use crate::runtime::queue; -use crate::runtime::stats::WorkerStatsBatcher; use crate::runtime::task::Inject; +use crate::runtime::{queue, MetricsBatch}; use loom::thread; @@ -10,14 +9,15 @@ fn basic() { loom::model(|| { let (steal, mut local) = queue::local(); let inject = Inject::new(); + let mut metrics = MetricsBatch::new(); let th = thread::spawn(move || { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(); let (_, mut local) = queue::local(); let mut n = 0; for _ in 0..3 { - if steal.steal_into(&mut local, &mut stats).is_some() { + if steal.steal_into(&mut local, &mut metrics).is_some() { n += 1; } @@ -34,7 +34,7 @@ fn basic() { for _ in 0..2 { for _ in 0..2 { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } if local.pop().is_some() { @@ -43,7 +43,7 @@ fn basic() { // Push another task let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); while local.pop().is_some() { n += 1; @@ -65,13 +65,14 @@ fn steal_overflow() { loom::model(|| { let (steal, mut local) = queue::local(); let inject = Inject::new(); + let mut metrics = MetricsBatch::new(); let th = thread::spawn(move || { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(); let (_, mut local) = queue::local(); let mut n = 0; - if steal.steal_into(&mut local, &mut stats).is_some() { + if steal.steal_into(&mut local, &mut metrics).is_some() { n += 1; } @@ -86,7 +87,7 @@ fn steal_overflow() { // push a task, pop a task let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); if local.pop().is_some() { n += 1; @@ -94,7 +95,7 @@ fn steal_overflow() { for _ in 0..6 { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } n += th.join().unwrap(); @@ -116,10 +117,10 @@ fn multi_stealer() { const NUM_TASKS: usize = 5; fn steal_tasks(steal: queue::Steal) -> usize { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(); let (_, mut local) = queue::local(); - if steal.steal_into(&mut local, &mut stats).is_none() { + if steal.steal_into(&mut local, &mut metrics).is_none() { return 0; } @@ -135,11 +136,12 @@ fn multi_stealer() { loom::model(|| { let (steal, mut local) = queue::local(); let inject = Inject::new(); + let mut metrics = MetricsBatch::new(); // Push work for _ in 0..NUM_TASKS { let (task, _) = super::unowned(async {}); - local.push_back(task, &inject); + local.push_back(task, &inject, &mut metrics); } let th1 = { @@ -169,7 +171,7 @@ fn multi_stealer() { #[test] fn chained_steal() { loom::model(|| { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(); let (s1, mut l1) = queue::local(); let (s2, mut l2) = queue::local(); let inject = Inject::new(); @@ -177,17 +179,17 @@ fn chained_steal() { // Load up some tasks for _ in 0..4 { let (task, _) = super::unowned(async {}); - l1.push_back(task, &inject); + l1.push_back(task, &inject, &mut metrics); let (task, _) = super::unowned(async {}); - l2.push_back(task, &inject); + l2.push_back(task, &inject, &mut metrics); } // Spawn a task to steal from **our** queue let th = thread::spawn(move || { - let mut stats = WorkerStatsBatcher::new(0); + let mut metrics = MetricsBatch::new(); let (_, mut local) = queue::local(); - s1.steal_into(&mut local, &mut stats); + s1.steal_into(&mut local, &mut metrics); while local.pop().is_some() {} }); @@ -195,7 +197,7 @@ fn chained_steal() { // Drain our tasks, then attempt to steal while l1.pop().is_some() {} - s2.steal_into(&mut l1, &mut stats); + s2.steal_into(&mut l1, &mut metrics); th.join().unwrap(); From 66cf82afc06ef5595b46121dcfee6e1e5a66cf36 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 14 Jan 2022 13:54:08 -0800 Subject: [PATCH 11/23] test queue metrics --- tokio/src/runtime/metrics/batch.rs | 4 ++-- tokio/src/runtime/metrics/mock.rs | 2 +- tokio/src/runtime/metrics/worker.rs | 16 ++++++------- tokio/src/runtime/queue.rs | 3 +-- tokio/src/runtime/tests/queue.rs | 36 +++++++++++++++++++++++++++-- 5 files changed, 46 insertions(+), 15 deletions(-) diff --git a/tokio/src/runtime/metrics/batch.rs b/tokio/src/runtime/metrics/batch.rs index e99a4d70390..5c43988f22f 100644 --- a/tokio/src/runtime/metrics/batch.rs +++ b/tokio/src/runtime/metrics/batch.rs @@ -98,8 +98,8 @@ cfg_rt_multi_thread! { self.steal_count += by as u64; } - pub(crate) fn incr_overflow_count(&mut self, by: u16) { - self.overflow_count += by as u64; + pub(crate) fn incr_overflow_count(&mut self) { + self.overflow_count += 1; } } } diff --git a/tokio/src/runtime/metrics/mock.rs b/tokio/src/runtime/metrics/mock.rs index ef75e6379ab..6b9cf704f42 100644 --- a/tokio/src/runtime/metrics/mock.rs +++ b/tokio/src/runtime/metrics/mock.rs @@ -38,6 +38,6 @@ impl MetricsBatch { cfg_rt_multi_thread! { impl MetricsBatch { pub(crate) fn incr_steal_count(&mut self, _by: u16) {} - pub(crate) fn incr_overflow_count(&mut self, _by: u16) {} + pub(crate) fn incr_overflow_count(&mut self) {} } } diff --git a/tokio/src/runtime/metrics/worker.rs b/tokio/src/runtime/metrics/worker.rs index 3f54f31594d..ec59ce04730 100644 --- a/tokio/src/runtime/metrics/worker.rs +++ b/tokio/src/runtime/metrics/worker.rs @@ -12,29 +12,29 @@ use std::sync::atomic::{AtomicU64, AtomicUsize}; #[repr(align(128))] pub(crate) struct WorkerMetrics { /// Number of times the worker parked. - pub(super) park_count: AtomicU64, + pub(crate) park_count: AtomicU64, /// Number of times the worker woke then parked again without doing work. - pub(super) noop_count: AtomicU64, + pub(crate) noop_count: AtomicU64, /// Number of times the worker attempted to steal. - pub(super) steal_count: AtomicU64, + pub(crate) steal_count: AtomicU64, /// Number of tasks the worker polled. - pub(super) poll_count: AtomicU64, + pub(crate) poll_count: AtomicU64, /// Amount of time the worker spent doing work vs. parking. - pub(super) busy_duration_total: AtomicU64, + pub(crate) busy_duration_total: AtomicU64, /// Number of tasks scheduled for execution on the worker's local queue. - pub(super) local_schedule_count: AtomicU64, + pub(crate) local_schedule_count: AtomicU64, /// Number of tasks moved from the local queue to the global queue to free space. - pub(super) overflow_count: AtomicU64, + pub(crate) overflow_count: AtomicU64, /// Number of tasks currently in the local queue. Used only by the /// current-thread scheduler. - pub(super) queue_depth: AtomicUsize, + pub(crate) queue_depth: AtomicUsize, } impl WorkerMetrics { diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index 25ccda37796..5b8a56c9308 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -122,7 +122,6 @@ impl Local { // Concurrently stealing, this will free up capacity, so only // push the task onto the inject queue inject.push(task); - metrics.incr_overflow_count(1); return; } else { // Push the current task and half of the queue into the @@ -254,7 +253,7 @@ impl Local { inject.push_batch(batch_iter.chain(std::iter::once(task))); // Add 1 to factor in the task currently being scheduled. - metrics.incr_overflow_count(NUM_TASKS_TAKEN + 1); + metrics.incr_overflow_count(); Ok(()) } diff --git a/tokio/src/runtime/tests/queue.rs b/tokio/src/runtime/tests/queue.rs index e0744b97430..2de17b9f4c4 100644 --- a/tokio/src/runtime/tests/queue.rs +++ b/tokio/src/runtime/tests/queue.rs @@ -5,6 +5,22 @@ use crate::runtime::MetricsBatch; use std::thread; use std::time::Duration; +#[allow(unused)] +macro_rules! assert_metrics { + ($metrics:ident, $field:ident == $v:expr) => {{ + use crate::runtime::WorkerMetrics; + use std::sync::atomic::Ordering::Relaxed; + + let worker = WorkerMetrics::new(); + $metrics.submit(&worker); + + let expect = $v; + let actual = worker.$field.load(Relaxed); + + assert!(actual == expect, "expect = {}; actual = {}", expect, actual) + }}; +} + #[test] fn fits_256() { let (_, mut local) = queue::local(); @@ -16,6 +32,10 @@ fn fits_256() { local.push_back(task, &inject, &mut metrics); } + cfg_metrics! { + assert_metrics!(metrics, overflow_count == 0); + } + assert!(inject.pop().is_none()); while local.pop().is_some() {} @@ -32,6 +52,10 @@ fn overflow() { local.push_back(task, &inject, &mut metrics); } + cfg_metrics! { + assert_metrics!(metrics, overflow_count == 1); + } + let mut n = 0; while inject.pop().is_some() { @@ -60,6 +84,10 @@ fn steal_batch() { assert!(steal1.steal_into(&mut local2, &mut metrics).is_some()); + cfg_metrics! { + assert_metrics!(metrics, steal_count == 2); + } + for _ in 0..1 { assert!(local2.pop().is_some()); } @@ -88,12 +116,12 @@ fn stress1() { let inject = Inject::new(); let th = thread::spawn(move || { - let mut stats = MetricsBatch::new(); + let mut metrics = MetricsBatch::new(); let (_, mut local) = queue::local(); let mut n = 0; for _ in 0..NUM_STEAL { - if steal.steal_into(&mut local, &mut stats).is_some() { + if steal.steal_into(&mut local, &mut metrics).is_some() { n += 1; } @@ -104,6 +132,10 @@ fn stress1() { thread::yield_now(); } + cfg_metrics! { + assert_metrics!(metrics, steal_count == n as _); + } + n }); From b62f72481b3d38e96c0142403e3426dc0d8ff864 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 11:26:28 -0800 Subject: [PATCH 12/23] more work --- tokio/src/runtime/basic_scheduler.rs | 3 + tokio/src/runtime/mod.rs | 9 +++ tokio/tests/rt_basic.rs | 76 +++++++++++++++++++++ tokio/tests/rt_common.rs | 53 +++++++++++++++ tokio/tests/rt_metrics.rs | 98 ++++++++++++++++++++++++++++ tokio/tests/rt_threaded.rs | 41 ++++++++++++ 6 files changed, 280 insertions(+) create mode 100644 tokio/tests/rt_metrics.rs diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 8381f74f593..1659f2eecda 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -263,6 +263,9 @@ impl Drop for BasicScheduler { assert!(context.spawner.shared.owned.is_empty()); + // Submit metrics + core.metrics.submit(&core.spawner.shared.worker_metrics); + (core, ()) }); } diff --git a/tokio/src/runtime/mod.rs b/tokio/src/runtime/mod.rs index 035350f5350..c8d97e1b19a 100644 --- a/tokio/src/runtime/mod.rs +++ b/tokio/src/runtime/mod.rs @@ -602,4 +602,13 @@ cfg_rt! { } } } + + cfg_metrics! { + impl Runtime { + /// TODO + pub fn metrics(&self) -> RuntimeMetrics { + self.handle.metrics() + } + } + } } diff --git a/tokio/tests/rt_basic.rs b/tokio/tests/rt_basic.rs index 149b3bfaad7..cc6ac677280 100644 --- a/tokio/tests/rt_basic.rs +++ b/tokio/tests/rt_basic.rs @@ -16,6 +16,15 @@ mod support { pub(crate) mod mpsc_stream; } +macro_rules! cfg_metrics { + ($($t:tt)*) => { + #[cfg(tokio_unstable)] + { + $( $t )* + } + } +} + #[test] fn spawned_task_does_not_progress_without_block_on() { let (tx, mut rx) = oneshot::channel(); @@ -197,6 +206,73 @@ fn wake_in_drop_after_panic() { }); } +#[test] +fn spawn_two() { + let rt = rt(); + + let out = rt.block_on(async { + let (tx, rx) = oneshot::channel(); + + tokio::spawn(async move { + tokio::spawn(async move { + tx.send("ZOMG").unwrap(); + }); + }); + + assert_ok!(rx.await) + }); + + assert_eq!(out, "ZOMG"); + + cfg_metrics! { + let metrics = rt.metrics(); + drop(rt); + assert_eq!(0, metrics.remote_schedule_count()); + + let mut local = 0; + for i in 0..metrics.num_workers() { + local += metrics.worker_local_schedule_count(i); + } + + assert_eq!(2, local); + } +} + +#[test] +fn spawn_remote() { + let rt = rt(); + + let out = rt.block_on(async { + let (tx, rx) = oneshot::channel(); + + let handle = tokio::spawn(async move { + std::thread::spawn(move || { + std::thread::sleep(Duration::from_millis(10)); + tx.send("ZOMG").unwrap(); + }); + + rx.await.unwrap() + }); + + handle.await.unwrap() + }); + + assert_eq!(out, "ZOMG"); + + cfg_metrics! { + let metrics = rt.metrics(); + drop(rt); + assert_eq!(1, metrics.remote_schedule_count()); + + let mut local = 0; + for i in 0..metrics.num_workers() { + local += metrics.worker_local_schedule_count(i); + } + + assert_eq!(1, local); + } +} + #[test] #[should_panic( expected = "A Tokio 1.x context was found, but timers are disabled. Call `enable_time` on the runtime builder to enable timers." diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index cb1d0f66152..7e15a363327 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -46,6 +46,15 @@ macro_rules! rt_test { } } +macro_rules! cfg_metrics { + ($($t:tt)*) => { + #[cfg(tokio_unstable)] + { + $( $t )* + } + } +} + #[test] fn send_sync_bound() { use tokio::runtime::Runtime; @@ -99,6 +108,35 @@ rt_test! { }); assert_eq!(out, "ZOMG"); + + cfg_metrics! {{ + let metrics = rt.metrics(); + + let mut n = 0; + + for i in 0..metrics.num_workers() { + super::wait_for(|| metrics.worker_park_count(i) > 0); + let park_count = metrics.worker_park_count(i); + let noop_count = metrics.worker_noop_count(i); + + if park_count == 0 { + continue; + } + + n += 1; + + assert!(park_count >= noop_count); + assert!(noop_count > 0); + + assert_eq!(0, metrics.worker_steal_count(i)); + assert_eq!(0, metrics.worker_local_schedule_count(i)); + assert_eq!(0, metrics.worker_overflow_count(i)); + assert_eq!(0, metrics.remote_queue_depth()); + assert_eq!(0, metrics.worker_local_queue_depth(i)); + } + + assert!(n > 0); + }} } #[test] @@ -1107,3 +1145,18 @@ rt_test! { }); } } + +#[cfg_attr(not(tokio_unstable), allow(foo))] +#[track_caller] +fn wait_for(mut f: impl FnMut() -> bool) { + use std::thread; + use std::time::Duration; + + for i in 1..18 { + if f() { + return; + } + + thread::sleep(Duration::from_micros(1 << i)); + } +} diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs new file mode 100644 index 00000000000..48d1373b90c --- /dev/null +++ b/tokio/tests/rt_metrics.rs @@ -0,0 +1,98 @@ +#![warn(rust_2018_idioms)] +#![cfg(all(feature = "full", tokio_unstable))] + +use tokio::runtime::Runtime; +use tokio::time::{self, Duration}; + +#[test] +fn num_workers() { + let rt = basic(); + assert_eq!(1, rt.metrics().num_workers()); + + let rt = threaded(); + assert_eq!(2, rt.metrics().num_workers()); +} + +#[test] +fn remote_schedule_count() { + use std::thread; + + let rt = basic(); + let handle = rt.handle().clone(); + let task = thread::spawn(move || { + handle.spawn(async { + // DO nothing + }) + }).join().unwrap(); + + rt.block_on(task).unwrap(); + + assert_eq!(1, rt.metrics().remote_schedule_count()); + + let rt = threaded(); + let handle = rt.handle().clone(); + let task = thread::spawn(move || { + handle.spawn(async { + // DO nothing + }) + }).join().unwrap(); + + rt.block_on(task).unwrap(); + + assert_eq!(1, rt.metrics().remote_schedule_count()); +} + +#[test] +fn worker_park_count() { + let rt = basic(); + let metrics = rt.metrics(); + rt.block_on(async { + time::sleep(Duration::from_millis(1)).await; + }); + drop(rt); + assert_eq!(2, metrics.worker_park_count(0)); + + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async { + time::sleep(Duration::from_millis(1)).await; + }); + drop(rt); + assert!(1 <= metrics.worker_park_count(0)); + assert_eq!(1, metrics.worker_park_count(1)); +} + +#[test] +fn worker_noop_count() { + let rt = basic(); + let metrics = rt.metrics(); + rt.block_on(async { + time::sleep(Duration::from_millis(1)).await; + }); + drop(rt); + assert_eq!(1, metrics.worker_noop_count(0)); + + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async { + time::sleep(Duration::from_millis(1)).await; + }); + drop(rt); + assert!(1 <= metrics.worker_noop_count(0)); + assert_eq!(1, metrics.worker_noop_count(1)); +} + +fn basic() -> Runtime { + tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .unwrap() +} + +fn threaded() -> Runtime { + tokio::runtime::Builder::new_multi_thread() + .worker_threads(2) + .enable_all() + .build() + .unwrap() +} \ No newline at end of file diff --git a/tokio/tests/rt_threaded.rs b/tokio/tests/rt_threaded.rs index 5f047a7962d..b2f84fd33ff 100644 --- a/tokio/tests/rt_threaded.rs +++ b/tokio/tests/rt_threaded.rs @@ -15,6 +15,15 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::{mpsc, Arc, Mutex}; use std::task::{Context, Poll, Waker}; +macro_rules! cfg_metrics { + ($($t:tt)*) => { + #[cfg(tokio_unstable)] + { + $( $t )* + } + } +} + #[test] fn single_thread() { // No panic when starting a runtime w/ a single thread @@ -55,6 +64,38 @@ fn many_oneshot_futures() { } } +#[test] +fn spawn_two() { + let rt = rt(); + + let out = rt.block_on(async { + let (tx, rx) = oneshot::channel(); + + tokio::spawn(async move { + tokio::spawn(async move { + tx.send("ZOMG").unwrap(); + }); + }); + + assert_ok!(rx.await) + }); + + assert_eq!(out, "ZOMG"); + + cfg_metrics! { + let metrics = rt.metrics(); + drop(rt); + assert_eq!(1, metrics.remote_schedule_count()); + + let mut local = 0; + for i in 0..metrics.num_workers() { + local += metrics.worker_local_schedule_count(i); + } + + assert_eq!(1, local); + } +} + #[test] fn many_multishot_futures() { const CHAIN: usize = 200; From 2bae9e84d473c3e17ff128d2c8c9a4f5a0f11d72 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 14:41:57 -0800 Subject: [PATCH 13/23] more tests --- tokio/src/runtime/basic_scheduler.rs | 4 +- tokio/tests/rt_common.rs | 1 - tokio/tests/rt_metrics.rs | 163 +++++++++++++++++++++++++-- 3 files changed, 157 insertions(+), 11 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 1659f2eecda..7447261d629 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -512,7 +512,9 @@ impl CoreGuard<'_> { 'outer: loop { if core.spawner.reset_woken() { - let (c, res) = context.run_task(core, || future.as_mut().poll(&mut cx)); + let (c, res) = context.enter(core, || { + crate::coop::budget(|| future.as_mut().poll(&mut cx)) + }); core = c; diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index 7e15a363327..67f1b46d08d 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -126,7 +126,6 @@ rt_test! { n += 1; assert!(park_count >= noop_count); - assert!(noop_count > 0); assert_eq!(0, metrics.worker_steal_count(i)); assert_eq!(0, metrics.worker_local_schedule_count(i)); diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index 48d1373b90c..441392030fb 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -23,7 +23,9 @@ fn remote_schedule_count() { handle.spawn(async { // DO nothing }) - }).join().unwrap(); + }) + .join() + .unwrap(); rt.block_on(task).unwrap(); @@ -35,11 +37,13 @@ fn remote_schedule_count() { handle.spawn(async { // DO nothing }) - }).join().unwrap(); + }) + .join() + .unwrap(); rt.block_on(task).unwrap(); - assert_eq!(1, rt.metrics().remote_schedule_count()); + assert_eq!(1, rt.metrics().remote_schedule_count()); } #[test] @@ -58,19 +62,22 @@ fn worker_park_count() { time::sleep(Duration::from_millis(1)).await; }); drop(rt); - assert!(1 <= metrics.worker_park_count(0)); - assert_eq!(1, metrics.worker_park_count(1)); + assert!(1 <= metrics.worker_park_count(0)); + assert!(1 <= metrics.worker_park_count(1)); } #[test] fn worker_noop_count() { + // There isn't really a great way to generate no-op parks as they happen as + // false-positive events under concurrency. + let rt = basic(); let metrics = rt.metrics(); rt.block_on(async { time::sleep(Duration::from_millis(1)).await; }); drop(rt); - assert_eq!(1, metrics.worker_noop_count(0)); + assert_eq!(2, metrics.worker_noop_count(0)); let rt = threaded(); let metrics = rt.metrics(); @@ -78,10 +85,148 @@ fn worker_noop_count() { time::sleep(Duration::from_millis(1)).await; }); drop(rt); - assert!(1 <= metrics.worker_noop_count(0)); - assert_eq!(1, metrics.worker_noop_count(1)); + assert!(1 <= metrics.worker_noop_count(0)); + assert!(1 <= metrics.worker_noop_count(1)); +} + +#[test] +fn worker_steal_count() { + // This metric only applies to the multi-threaded runtime. + // + // We use a blocking channel to backup one worker thread. + use std::sync::mpsc::channel; + + let rt = threaded(); + let metrics = rt.metrics(); + + rt.block_on(async { + let (tx, rx) = channel(); + + // Move to the runtime. + tokio::spawn(async move { + // Spawn the task that sends to the channel + tokio::spawn(async move { + tx.send(()).unwrap(); + }); + + // Spawn a task that bumps the previous task out of the "next + // scheduled" slot. + tokio::spawn(async {}); + + // Blocking receive on the channe. + rx.recv().unwrap(); + }) + .await + .unwrap(); + }); + + drop(rt); + + let n: u64 = (0..metrics.num_workers()) + .map(|i| metrics.worker_steal_count(i)) + .sum(); + + assert_eq!(1, n); } +#[test] +fn worker_poll_count() { + const N: u64 = 5; + + let rt = basic(); + let metrics = rt.metrics(); + rt.block_on(async { + for _ in 0..N { + tokio::spawn(async {}).await.unwrap(); + } + }); + drop(rt); + assert_eq!(N, metrics.worker_poll_count(0)); + + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async { + for _ in 0..N { + tokio::spawn(async {}).await.unwrap(); + } + }); + drop(rt); + // Account for the `block_on` task + let n = (0..metrics.num_workers()) + .map(|i| metrics.worker_poll_count(i)) + .sum(); + + assert_eq!(N, n); +} + +#[test] +fn worker_total_busy_duration() { + const N: usize = 5; + + let zero = Duration::from_millis(0); + + let rt = basic(); + let metrics = rt.metrics(); + + rt.block_on(async { + for _ in 0..N { + tokio::spawn(async { + tokio::task::yield_now().await; + }) + .await + .unwrap(); + } + }); + + drop(rt); + + assert!(zero < metrics.worker_total_busy_duration(0)); + + let rt = threaded(); + let metrics = rt.metrics(); + + rt.block_on(async { + for _ in 0..N { + tokio::spawn(async { + tokio::task::yield_now().await; + }) + .await + .unwrap(); + } + }); + + drop(rt); + + for i in 0..metrics.num_workers() { + assert!(zero < metrics.worker_total_busy_duration(i)); + } +} + +#[test] +fn worker_local_schedule_count() { + let rt = basic(); + let metrics = rt.metrics(); + rt.block_on(async { + tokio::spawn(async {}).await.unwrap(); + }); + drop(rt); + + assert_eq!(1, metrics.worker_local_schedule_count(0)); + assert_eq!(0, metrics.remote_schedule_count()); +} + +#[test] +#[ignore] +fn worker_overflow_count() {} + +#[test] +#[ignore] +fn remote_queue_depth() {} + +#[test] +#[ignore] +fn worker_local_queue_depth() {} + fn basic() -> Runtime { tokio::runtime::Builder::new_current_thread() .enable_all() @@ -95,4 +240,4 @@ fn threaded() -> Runtime { .enable_all() .build() .unwrap() -} \ No newline at end of file +} From 24ebd656eae35bac83f73675a7a3c44ae664a790 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 20:29:45 -0800 Subject: [PATCH 14/23] more tests --- tokio/tests/rt_metrics.rs | 94 +++++++++++++++++++++++++++++++++++++-- 1 file changed, 90 insertions(+), 4 deletions(-) diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index 441392030fb..207b0d7fc02 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -213,15 +213,101 @@ fn worker_local_schedule_count() { assert_eq!(1, metrics.worker_local_schedule_count(0)); assert_eq!(0, metrics.remote_schedule_count()); + + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async { + // Move to the runtime + tokio::spawn(async { + tokio::spawn(async {}).await.unwrap(); + }).await.unwrap(); + }); + drop(rt); + + let n: u64 = (0..metrics.num_workers()) + .map(|i| metrics.worker_local_schedule_count(i)) + .sum(); + + assert_eq!(2, n); + assert_eq!(1, metrics.remote_schedule_count()); } #[test] -#[ignore] -fn worker_overflow_count() {} +fn worker_overflow_count() { + // Only applies to the threaded worker + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async { + // Move to the runtime + tokio::spawn(async { + let (tx1, rx1) = std::sync::mpsc::channel(); + let (tx2, rx2) = std::sync::mpsc::channel(); + + // First, we need to block the other worker until all tasks have + // been spawned. + tokio::spawn(async move { + tx1.send(()).unwrap(); + rx2.recv().unwrap(); + }); + + // Bump the next-run spawn + tokio::spawn(async {}); + + rx1.recv().unwrap(); + + // Spawn many tasks + for _ in 0..300 { + tokio::spawn(async {}); + } + + tx2.send(()).unwrap(); + }).await.unwrap(); + }); + drop(rt); + + let n: u64 = (0..metrics.num_workers()) + .map(|i| metrics.worker_overflow_count(i)) + .sum(); + + assert_eq!(1, n); +} #[test] -#[ignore] -fn remote_queue_depth() {} +fn remote_queue_depth() { + use std::thread; + + let rt = basic(); + let handle = rt.handle().clone(); + let metrics = rt.metrics(); + + thread::spawn(move || { + handle.spawn(async {}); + }).join().unwrap(); + + assert_eq!(1, metrics.remote_queue_depth()); + + let rt = threaded(); + let handle = rt.handle().clone(); + let metrics = rt.metrics(); + + // First we need to block the runtime workers + let (tx1, rx1) = std::sync::mpsc::channel(); + let (tx2, rx2) = std::sync::mpsc::channel(); + + rt.spawn(async move { rx1.recv().unwrap() }); + rt.spawn(async move { rx2.recv().unwrap() }); + + thread::spawn(move || { + handle.spawn(async {}); + }).join().unwrap(); + + let n = metrics.remote_queue_depth(); + assert!(1 <= n, "{}", n); + assert!(3 >= n, "{}", n); + + tx1.send(()).unwrap(); + tx2.send(()).unwrap(); +} #[test] #[ignore] From 5f58cf6ab4fd14c26d18aab0647acb98da16f322 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 20:36:35 -0800 Subject: [PATCH 15/23] tests --- tokio/tests/rt_metrics.rs | 68 +++++++++++++++++++++++++++++++++++---- 1 file changed, 62 insertions(+), 6 deletions(-) diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index 207b0d7fc02..bdba096313b 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -220,7 +220,9 @@ fn worker_local_schedule_count() { // Move to the runtime tokio::spawn(async { tokio::spawn(async {}).await.unwrap(); - }).await.unwrap(); + }) + .await + .unwrap(); }); drop(rt); @@ -261,7 +263,9 @@ fn worker_overflow_count() { } tx2.send(()).unwrap(); - }).await.unwrap(); + }) + .await + .unwrap(); }); drop(rt); @@ -282,7 +286,9 @@ fn remote_queue_depth() { thread::spawn(move || { handle.spawn(async {}); - }).join().unwrap(); + }) + .join() + .unwrap(); assert_eq!(1, metrics.remote_queue_depth()); @@ -299,7 +305,9 @@ fn remote_queue_depth() { thread::spawn(move || { handle.spawn(async {}); - }).join().unwrap(); + }) + .join() + .unwrap(); let n = metrics.remote_queue_depth(); assert!(1 <= n, "{}", n); @@ -310,8 +318,56 @@ fn remote_queue_depth() { } #[test] -#[ignore] -fn worker_local_queue_depth() {} +fn worker_local_queue_depth() { + const N: usize = 100; + + let rt = basic(); + let metrics = rt.metrics(); + rt.block_on(async { + for _ in 0..N { + tokio::spawn(async {}); + } + + assert_eq!(N, metrics.worker_local_queue_depth(0)); + }); + + let rt = threaded(); + let metrics = rt.metrics(); + rt.block_on(async move { + // Move to the runtime + tokio::spawn(async move { + let (tx1, rx1) = std::sync::mpsc::channel(); + let (tx2, rx2) = std::sync::mpsc::channel(); + + // First, we need to block the other worker until all tasks have + // been spawned. + tokio::spawn(async move { + tx1.send(()).unwrap(); + rx2.recv().unwrap(); + }); + + // Bump the next-run spawn + tokio::spawn(async {}); + + rx1.recv().unwrap(); + + // Spawn some tasks + for _ in 0..100 { + tokio::spawn(async {}); + } + + let n: usize = (0..metrics.num_workers()) + .map(|i| metrics.worker_local_queue_depth(i)) + .sum(); + + assert_eq!(n, N); + + tx2.send(()).unwrap(); + }) + .await + .unwrap(); + }); +} fn basic() -> Runtime { tokio::runtime::Builder::new_current_thread() From f1d0dcf63b725d686c06c3a6c70e184451cff0f4 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 21:21:38 -0800 Subject: [PATCH 16/23] try fixing tests --- tokio/tests/rt_common.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index 67f1b46d08d..b134c569a15 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -1145,7 +1145,7 @@ rt_test! { } } -#[cfg_attr(not(tokio_unstable), allow(foo))] +#[cfg_attr(not(tokio_unstable), allow(dead_code))] #[track_caller] fn wait_for(mut f: impl FnMut() -> bool) { use std::thread; From 0192ace8762f391bcea79655d6465b5dc4695cdb Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 21:29:23 -0800 Subject: [PATCH 17/23] remove some adhoc tests --- tokio/tests/rt_common.rs | 52 ---------------------------------------- 1 file changed, 52 deletions(-) diff --git a/tokio/tests/rt_common.rs b/tokio/tests/rt_common.rs index b134c569a15..cb1d0f66152 100644 --- a/tokio/tests/rt_common.rs +++ b/tokio/tests/rt_common.rs @@ -46,15 +46,6 @@ macro_rules! rt_test { } } -macro_rules! cfg_metrics { - ($($t:tt)*) => { - #[cfg(tokio_unstable)] - { - $( $t )* - } - } -} - #[test] fn send_sync_bound() { use tokio::runtime::Runtime; @@ -108,34 +99,6 @@ rt_test! { }); assert_eq!(out, "ZOMG"); - - cfg_metrics! {{ - let metrics = rt.metrics(); - - let mut n = 0; - - for i in 0..metrics.num_workers() { - super::wait_for(|| metrics.worker_park_count(i) > 0); - let park_count = metrics.worker_park_count(i); - let noop_count = metrics.worker_noop_count(i); - - if park_count == 0 { - continue; - } - - n += 1; - - assert!(park_count >= noop_count); - - assert_eq!(0, metrics.worker_steal_count(i)); - assert_eq!(0, metrics.worker_local_schedule_count(i)); - assert_eq!(0, metrics.worker_overflow_count(i)); - assert_eq!(0, metrics.remote_queue_depth()); - assert_eq!(0, metrics.worker_local_queue_depth(i)); - } - - assert!(n > 0); - }} } #[test] @@ -1144,18 +1107,3 @@ rt_test! { }); } } - -#[cfg_attr(not(tokio_unstable), allow(dead_code))] -#[track_caller] -fn wait_for(mut f: impl FnMut() -> bool) { - use std::thread; - use std::time::Duration; - - for i in 1..18 { - if f() { - return; - } - - thread::sleep(Duration::from_micros(1 << i)); - } -} From 6ea5437b85401924a6750e203c578a55c0ec95b8 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 21:35:41 -0800 Subject: [PATCH 18/23] make test more robust --- tokio/tests/rt_metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index bdba096313b..41d8b824342 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -54,7 +54,7 @@ fn worker_park_count() { time::sleep(Duration::from_millis(1)).await; }); drop(rt); - assert_eq!(2, metrics.worker_park_count(0)); + assert!(2 <= metrics.worker_park_count(0)); let rt = threaded(); let metrics = rt.metrics(); From 61f1621fb58a6a3bb2488988bd085c3d7789d550 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 18 Jan 2022 21:43:27 -0800 Subject: [PATCH 19/23] make tests more robust --- tokio/tests/rt_metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index 41d8b824342..95eb6d91ba6 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -77,7 +77,7 @@ fn worker_noop_count() { time::sleep(Duration::from_millis(1)).await; }); drop(rt); - assert_eq!(2, metrics.worker_noop_count(0)); + assert!(2 <= metrics.worker_noop_count(0)); let rt = threaded(); let metrics = rt.metrics(); From 43f58f3fe587a43cff3dfe5a59bfeae5dca2482e Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 19 Jan 2022 15:53:04 -0800 Subject: [PATCH 20/23] write some docs --- tokio/src/runtime/metrics/runtime.rs | 366 +++++++++++++++++++++++++-- 1 file changed, 351 insertions(+), 15 deletions(-) diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index 6c5afd093ef..f07ed803460 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -3,7 +3,12 @@ use crate::runtime::Handle; use std::sync::atomic::Ordering::Relaxed; use std::time::Duration; -/// TODO: dox +/// Handle to the runtime's metrics. +/// +/// This handle is internally reference-counted and can be freely cloned. A +/// `RuntimeMetrics` handle is obtained using the [`Runtime::metrics`] method. +/// +/// [`Runtime::metrics`]: crate::runtime::Runtime::metrics() #[derive(Clone, Debug)] pub struct RuntimeMetrics { handle: Handle, @@ -15,14 +20,52 @@ impl RuntimeMetrics { } /// Returns the number of worker threads used by the runtime. + /// + /// The number of workers is set by configuring `worker_threads` on + /// `runtime::Builder`. When using the `current_thread` runtime, the return + /// value is always `1`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.num_workers(); + /// println!("Runtime is using {} workers", n); + /// } + /// ``` pub fn num_workers(&self) -> usize { self.handle.spawner.num_workers() } /// Returns the number of tasks scheduled from **outside** of the runtime. /// - /// Tasks scheduled from outside of the runtime go via the runtime's - /// injection queue, which is usually is slower. + /// The remote schedule count starts at zero when the runtime is created and + /// increases by one each time a task is woken from **outside** of the + /// runtime. This usually means that a task is spawned or notified from a + /// non-runtime thread and must be queued using the Runtime's injection + /// queue, which tends to be slower. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.remote_schedule_count(); + /// println!("{} tasks were scheduled from outside the runtime", n); + /// } + /// ``` pub fn remote_schedule_count(&self) -> u64 { self.handle .spawner @@ -31,7 +74,41 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the total number of times this worker thread has parked. + /// Returns the total number of times the given worker thread has parked. + /// + /// The worker park count starts at zero when the runtime is created and + /// increases by one each time the worker parks the thread waiting for new + /// inbound events to process. This usually means the worker has processed + /// all pending work and is currently idle. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_park_count(0); + /// println!("worker 0 parked {} times", n); + /// } + /// ``` pub fn worker_park_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -40,9 +117,41 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the number of times this worker unparked but performed no work. + /// Returns the number of times the given worker thread unparked but + /// performed no work before parking again. + /// + /// The worker no-op count starts at zero when the runtime is created and + /// increases by one each time the worker unparks the thread but finds no + /// new work and goes back to sleep. This indicates a false-positive wake up. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. /// - /// This is the false-positive wake count. + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_noop_count(0); + /// println!("worker 0 had {} no-op unparks", n); + /// } + /// ``` pub fn worker_noop_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -51,8 +160,43 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the number of tasks this worker has stolen from other worker - /// threads. + /// Returns the number of times the given worker thread stole tasks from + /// another worker thread. + /// + /// This metric only applies to the **multi-threaded** runtime. + /// + /// The worker steal count starts at zero when the runtime is created and + /// increases by one each time the worker has processed its scheduled queue + /// and successfully steals more pending tasks from another worker. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_noop_count(0); + /// println!("worker 0 had {} no-op unparks", n); + /// } + /// ``` pub fn worker_steal_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -61,7 +205,39 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the number of times this worker has polled a task. + /// Returns the number of tasks the given worker thread has polled. + /// + /// The worker steal count starts at zero when the runtime is created and + /// increases by one each time the worker polls a scheduled task. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_poll_count(0); + /// println!("worker 0 has polled {} tasks", n); + /// } + /// ``` pub fn worker_poll_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -70,7 +246,42 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the total amount of time this worker has been busy for. + /// Returns the amount of time the given worker thread has been busy. + /// + /// The worker busy duration starts at zero when the runtime is created and + /// increases whenever the worker is spending time processing work. Using + /// this value can indicate the load of the given worker. If a lot of time + /// is spent busy, then the worker is under load and will check for inbound + /// events less often. + /// + /// The timer is monotonically increasing. It is never decremented or reset + /// to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_poll_count(0); + /// println!("worker 0 has polled {} tasks", n); + /// } + /// ``` pub fn worker_total_busy_duration(&self, worker: usize) -> Duration { let nanos = self .handle @@ -81,7 +292,43 @@ impl RuntimeMetrics { Duration::from_nanos(nanos) } - /// TODO + /// Returns the number of tasks scheduled from **within** the runtime on the + /// given worker's local queue. + /// + /// The local schedule count starts at zero when the runtime is created and + /// increases by one each time a task is woken from **inside** of the + /// runtime on the given worker. This usually means that a task is spawned + /// or notified from within a runtime thread and will be queued on the + /// worker-local queue. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_local_schedule_count(0); + /// println!("{} tasks were scheduled on the worker's local queue", n); + /// } + /// ``` pub fn worker_local_schedule_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -90,8 +337,44 @@ impl RuntimeMetrics { .load(Relaxed) } - /// Returns the number of tasks moved from this worker's local queue to the - /// remote queue. + /// Returns the number of times the given worker thread saturated its local + /// queue. + /// + /// This metric only applies to the **multi-threaded** scheduler. + /// + /// The worker steal count starts at zero when the runtime is created and + /// increases by one each time the worker attempts to schedule a task + /// locally, but its local queue is full. When this happens, half of the + /// local queue is moved to the injection queue. + /// + /// The counter is monotonically increasing. It is never decremented or + /// reset to zero. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_overflow_count(0); + /// println!("worker 0 has overflowed its queue {} times", n); + /// } + /// ``` pub fn worker_overflow_count(&self, worker: usize) -> u64 { self.handle .spawner @@ -100,12 +383,65 @@ impl RuntimeMetrics { .load(Relaxed) } - /// TODO + /// Returns the number of tasks currently scheduled in the runtime's + /// injection queue. + /// + /// Tasks that are spanwed or notified from a non-runtime thread are + /// scheduled using the runtime's injection queue. This metric returns the + /// **current** number of tasks pending in the injection queue. As such, the + /// returned value may increase or decrease as new tasks are scheduled and + /// processed. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.remote_queue_depth(); + /// println!("{} tasks currently pending in the runtime's injection queue", n); + /// } + /// ``` pub fn remote_queue_depth(&self) -> usize { self.handle.spawner.remote_queue_depth() } - /// Returns the number of tasks currently in the worker's local queue + /// Returns the number of tasks currently scheduled in the given worker's + /// local queue. + /// + /// Tasks that are spawned or notified from within a runtime thread are + /// scheduled using that worker's local queue. This metric returns the + /// **current** number of tasks pending in the worker's local queue. As + /// such, the returned value may increase or decrease as new tasks are scheduled and processed. + /// + /// # Arguments + /// + /// `worker` is the index of the worker being queried. The given value must + /// be between 0 and `num_workers()`. The index uniquely identifies a single + /// worker and will continue to indentify the worker throughout the lifetime + /// of the runtime instance. + /// + /// # Panics + /// + /// The method panics when `worker` represents an invalid worker, i.e. is + /// greater than or equal to `num_workers()`. + /// + /// # Examples + /// + /// ``` + /// use tokio::runtime::Handle; + /// + /// #[tokio::main] + /// async fn main() { + /// let metrics = Handle::current().metrics(); + /// + /// let n = metrics.worker_local_queue_depth(0); + /// println!("{} tasks currently pending in worker 0's local queue", n); + /// } + /// ``` pub fn worker_local_queue_depth(&self, worker: usize) -> usize { self.handle.spawner.worker_local_queue_depth(worker) } From 47ef235bcdc71de3019b32af9b185f854d3bccfc Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 21 Jan 2022 13:01:26 -0800 Subject: [PATCH 21/23] Apply suggestions from code review Co-authored-by: Alice Ryhl Co-authored-by: Lucio Franco --- tokio/src/runtime/basic_scheduler.rs | 2 +- tokio/src/runtime/metrics/batch.rs | 4 ++-- tokio/src/runtime/metrics/runtime.rs | 9 +++++---- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 7447261d629..4fa1671107b 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -101,7 +101,7 @@ struct Shared { /// Keeps track of various runtime metrics. scheduler_metrics: SchedulerMetrics, - /// This scheduler only has one worker + /// This scheduler only has one worker. worker_metrics: WorkerMetrics, } diff --git a/tokio/src/runtime/metrics/batch.rs b/tokio/src/runtime/metrics/batch.rs index 5c43988f22f..f1c3fa6b747 100644 --- a/tokio/src/runtime/metrics/batch.rs +++ b/tokio/src/runtime/metrics/batch.rs @@ -5,13 +5,13 @@ use std::sync::atomic::Ordering::Relaxed; use std::time::Instant; pub(crate) struct MetricsBatch { - /// Number of times the worker parked + /// Number of times the worker parked. park_count: u64, /// Number of times the worker woke w/o doing work. noop_count: u64, - /// Number of times stolen + /// Number of times stolen. steal_count: u64, /// Number of tasks that were polled by the worker. diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index f07ed803460..61dd053d663 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -163,7 +163,7 @@ impl RuntimeMetrics { /// Returns the number of times the given worker thread stole tasks from /// another worker thread. /// - /// This metric only applies to the **multi-threaded** runtime. + /// This metric only applies to the **multi-threaded** runtime and will always return `0` when using the current thread runtime. /// /// The worker steal count starts at zero when the runtime is created and /// increases by one each time the worker has processed its scheduled queue @@ -194,7 +194,7 @@ impl RuntimeMetrics { /// let metrics = Handle::current().metrics(); /// /// let n = metrics.worker_noop_count(0); - /// println!("worker 0 had {} no-op unparks", n); + /// println!("worker 0 has stolen tasks {} times", n); /// } /// ``` pub fn worker_steal_count(&self, worker: usize) -> u64 { @@ -207,7 +207,7 @@ impl RuntimeMetrics { /// Returns the number of tasks the given worker thread has polled. /// - /// The worker steal count starts at zero when the runtime is created and + /// The worker poll count starts at zero when the runtime is created and /// increases by one each time the worker polls a scheduled task. /// /// The counter is monotonically increasing. It is never decremented or @@ -415,7 +415,8 @@ impl RuntimeMetrics { /// Tasks that are spawned or notified from within a runtime thread are /// scheduled using that worker's local queue. This metric returns the /// **current** number of tasks pending in the worker's local queue. As - /// such, the returned value may increase or decrease as new tasks are scheduled and processed. + /// such, the returned value may increase or decrease as new tasks are + /// scheduled and processed. /// /// # Arguments /// From db9e1390989619689ce229f52b27a5db8f00e109 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 21 Jan 2022 13:25:32 -0800 Subject: [PATCH 22/23] rename -> injection_queue_depth --- tokio/src/runtime/basic_scheduler.rs | 2 +- tokio/src/runtime/metrics/runtime.rs | 6 +++--- tokio/src/runtime/spawner.rs | 6 +++--- tokio/src/runtime/thread_pool/mod.rs | 4 ++-- tokio/src/runtime/thread_pool/worker.rs | 2 +- tokio/tests/rt_metrics.rs | 6 +++--- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tokio/src/runtime/basic_scheduler.rs b/tokio/src/runtime/basic_scheduler.rs index 4fa1671107b..00193329137 100644 --- a/tokio/src/runtime/basic_scheduler.rs +++ b/tokio/src/runtime/basic_scheduler.rs @@ -423,7 +423,7 @@ cfg_metrics! { &self.shared.scheduler_metrics } - pub(crate) fn remote_queue_depth(&self) -> usize { + pub(crate) fn injection_queue_depth(&self) -> usize { // TODO: avoid having to lock. The multi-threaded injection queue // could probably be used here. self.shared.queue.lock() diff --git a/tokio/src/runtime/metrics/runtime.rs b/tokio/src/runtime/metrics/runtime.rs index 61dd053d663..0f8055907f5 100644 --- a/tokio/src/runtime/metrics/runtime.rs +++ b/tokio/src/runtime/metrics/runtime.rs @@ -401,12 +401,12 @@ impl RuntimeMetrics { /// async fn main() { /// let metrics = Handle::current().metrics(); /// - /// let n = metrics.remote_queue_depth(); + /// let n = metrics.injection_queue_depth(); /// println!("{} tasks currently pending in the runtime's injection queue", n); /// } /// ``` - pub fn remote_queue_depth(&self) -> usize { - self.handle.spawner.remote_queue_depth() + pub fn injection_queue_depth(&self) -> usize { + self.handle.spawner.injection_queue_depth() } /// Returns the number of tasks currently scheduled in the given worker's diff --git a/tokio/src/runtime/spawner.rs b/tokio/src/runtime/spawner.rs index 82337b4a415..d81a806cb59 100644 --- a/tokio/src/runtime/spawner.rs +++ b/tokio/src/runtime/spawner.rs @@ -64,11 +64,11 @@ cfg_metrics! { } } - pub(crate) fn remote_queue_depth(&self) -> usize { + pub(crate) fn injection_queue_depth(&self) -> usize { match self { - Spawner::Basic(spawner) => spawner.remote_queue_depth(), + Spawner::Basic(spawner) => spawner.injection_queue_depth(), #[cfg(feature = "rt-multi-thread")] - Spawner::ThreadPool(spawner) => spawner.remote_queue_depth(), + Spawner::ThreadPool(spawner) => spawner.injection_queue_depth(), } } diff --git a/tokio/src/runtime/thread_pool/mod.rs b/tokio/src/runtime/thread_pool/mod.rs index 2fc83a99013..d3f46517cb0 100644 --- a/tokio/src/runtime/thread_pool/mod.rs +++ b/tokio/src/runtime/thread_pool/mod.rs @@ -119,8 +119,8 @@ cfg_metrics! { &self.shared.worker_metrics[worker] } - pub(crate) fn remote_queue_depth(&self) -> usize { - self.shared.remote_queue_depth() + pub(crate) fn injection_queue_depth(&self) -> usize { + self.shared.injection_queue_depth() } pub(crate) fn worker_local_queue_depth(&self, worker: usize) -> usize { diff --git a/tokio/src/runtime/thread_pool/worker.rs b/tokio/src/runtime/thread_pool/worker.rs index 6c10fe996c1..60706cad356 100644 --- a/tokio/src/runtime/thread_pool/worker.rs +++ b/tokio/src/runtime/thread_pool/worker.rs @@ -833,7 +833,7 @@ impl Shared { cfg_metrics! { impl Shared { - pub(super) fn remote_queue_depth(&self) -> usize { + pub(super) fn injection_queue_depth(&self) -> usize { self.inject.len() } diff --git a/tokio/tests/rt_metrics.rs b/tokio/tests/rt_metrics.rs index 95eb6d91ba6..0a26b80285d 100644 --- a/tokio/tests/rt_metrics.rs +++ b/tokio/tests/rt_metrics.rs @@ -277,7 +277,7 @@ fn worker_overflow_count() { } #[test] -fn remote_queue_depth() { +fn injection_queue_depth() { use std::thread; let rt = basic(); @@ -290,7 +290,7 @@ fn remote_queue_depth() { .join() .unwrap(); - assert_eq!(1, metrics.remote_queue_depth()); + assert_eq!(1, metrics.injection_queue_depth()); let rt = threaded(); let handle = rt.handle().clone(); @@ -309,7 +309,7 @@ fn remote_queue_depth() { .join() .unwrap(); - let n = metrics.remote_queue_depth(); + let n = metrics.injection_queue_depth(); assert!(1 <= n, "{}", n); assert!(3 >= n, "{}", n); From 3fe79b34037e721bb27ebc4b2354fd74047c531d Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Fri, 21 Jan 2022 13:28:52 -0800 Subject: [PATCH 23/23] revert unnecessary change --- tokio/src/runtime/queue.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tokio/src/runtime/queue.rs b/tokio/src/runtime/queue.rs index 5b8a56c9308..ad9085a6545 100644 --- a/tokio/src/runtime/queue.rs +++ b/tokio/src/runtime/queue.rs @@ -302,7 +302,7 @@ impl Local { impl Steal { pub(super) fn is_empty(&self) -> bool { - self.0.len() == 0 + self.0.is_empty() } /// Steals half the tasks from self and place them into `dst`.