Skip to content

Commit f821bda

Browse files
committed
Merge 'Address non-monotonicity of steal time and other issues' from Travis Downs
This series primarily addresses the problem that on systems with low amounts of steal, steal time appears negative (i.e., the cumulative steal time counter goes down from sample to sample). This wrong on the face of it and also causes serious problems as a metric in prometheus since the counter contract (monotonic increase) is violated. This causes spurious "counter reset" detection in prometheus and hence bogus very large or very small steal time results in `rate` (or similar) queries. This is addressed in two ways: - We make the sleep time calculation more accurate, which is the underlying reason for negative steal which reduces the error (and so "negativeness") of steal by a couple orders of magnitude. After this change steal time is often 0 when rounded to the nearest ms where it wasn't before. - Because the reduction above still does not prevent small negative steal completely, we change the implementation of the metric to essentially cap steal from below 0 in periods where steal was negative. The individual changes have further details. I am open to splitting commits that may be less popular or require more discussion into a different PR if it makes sense. Closes #2390 * https://github.com/scylladb/seastar: Make total_steal_time() monotonic. Remove account_idle reactor: add better sleep time accounting reactor: add cpu and awake time reactor metrics Zero-init total sleep time
2 parents 14a59f3 + 8efeeb0 commit f821bda

File tree

3 files changed

+71
-16
lines changed

3 files changed

+71
-16
lines changed

include/seastar/core/reactor.hh

+13-3
Original file line numberDiff line numberDiff line change
@@ -302,8 +302,17 @@ private:
302302
const bool _reuseport;
303303
circular_buffer<double> _loads;
304304
double _load = 0;
305+
// Next two fields are required to enforce the monotonicity of total_steal_time()
306+
// see that method for details.
307+
308+
// Last measured accumulated steal time, i.e., the simple difference of accumulated
309+
// awake time and consumed thread CPU time.
310+
sched_clock::duration _last_true_steal{0};
311+
// Accumulated steal time forced to be monotinic by rejecting any updates that would
312+
// decrease it. See total_steal_time() for details.
313+
sched_clock::duration _last_mono_steal{0};
305314
sched_clock::duration _total_idle{0};
306-
sched_clock::duration _total_sleep;
315+
sched_clock::duration _total_sleep{0};
307316
sched_clock::time_point _start_time = now();
308317
output_stream<char>::batch_flush_list_t _flush_batching;
309318
std::atomic<bool> _sleeping alignas(seastar::cache_line_size){0};
@@ -382,7 +391,6 @@ private:
382391
task_queue* pop_active_task_queue(sched_clock::time_point now);
383392
void insert_activating_task_queues();
384393
void account_runtime(task_queue& tq, sched_clock::duration runtime);
385-
void account_idle(sched_clock::duration idletime);
386394
void allocate_scheduling_group_specific_data(scheduling_group sg, unsigned long key_id);
387395
future<> rename_scheduling_group_specific_data(scheduling_group sg);
388396
future<> init_scheduling_group(scheduling_group sg, sstring name, sstring shortname, float shares);
@@ -549,10 +557,12 @@ public:
549557
[[deprecated("Use this_shard_id")]]
550558
shard_id cpu_id() const;
551559

552-
void sleep();
560+
void try_sleep();
553561

554562
steady_clock_type::duration total_idle_time();
555563
steady_clock_type::duration total_busy_time();
564+
steady_clock_type::duration total_awake_time() const;
565+
std::chrono::nanoseconds total_cpu_time() const;
556566
std::chrono::nanoseconds total_steal_time();
557567

558568
const io_stats& get_io_stats() const { return _io_stats; }

src/core/reactor.cc

+49-13
Original file line numberDiff line numberDiff line change
@@ -1006,11 +1006,6 @@ reactor::account_runtime(task_queue& tq, sched_clock::duration runtime) {
10061006
tq._runtime += runtime;
10071007
}
10081008

1009-
void
1010-
reactor::account_idle(sched_clock::duration runtime) {
1011-
// anything to do here?
1012-
}
1013-
10141009
struct reactor::task_queue::indirect_compare {
10151010
bool operator()(const task_queue* tq1, const task_queue* tq2) const {
10161011
return tq1->_vruntime < tq2->_vruntime;
@@ -2515,8 +2510,14 @@ void reactor::register_metrics() {
25152510
sm::make_gauge("utilization", [this] { return (1-_load) * 100; }, sm::description("CPU utilization")),
25162511
sm::make_counter("cpu_busy_ms", [this] () -> int64_t { return total_busy_time() / 1ms; },
25172512
sm::description("Total cpu busy time in milliseconds")),
2513+
sm::make_counter("sleep_time_ms_total", [this] () -> int64_t { return _total_sleep / 1ms; },
2514+
sm::description("Total reactor sleep time (wall clock)")),
2515+
sm::make_counter("awake_time_ms_total", [this] () -> int64_t { return total_awake_time() / 1ms; },
2516+
sm::description("Total reactor awake time (wall_clock)")),
2517+
sm::make_counter("cpu_used_time_ms", [this] () -> int64_t { return total_cpu_time() / 1ms; },
2518+
sm::description("Total reactor thread CPU time (from CLOCK_THREAD_CPUTIME)")),
25182519
sm::make_counter("cpu_steal_time_ms", [this] () -> int64_t { return total_steal_time() / 1ms; },
2519-
sm::description("Total steal time, the time in which some other process was running while Seastar was not trying to run (not sleeping)."
2520+
sm::description("Total steal time, the time in which something else was running while the reactor was runnable (not sleeping)."
25202521
"Because this is in userspace, some time that could be legitimally thought as steal time is not accounted as such. For example, if we are sleeping and can wake up but the kernel hasn't woken us up yet.")),
25212522
// total_operations value:DERIVE:0:U
25222523
sm::make_counter("aio_reads", _io_stats.aio_reads, sm::description("Total aio-reads operations")),
@@ -3255,7 +3256,6 @@ int reactor::do_run() {
32553256
if (check_for_work()) {
32563257
if (idle) {
32573258
_total_idle += idle_end - idle_start;
3258-
account_idle(idle_end - idle_start);
32593259
idle_start = idle_end;
32603260
idle = false;
32613261
}
@@ -3281,13 +3281,11 @@ int reactor::do_run() {
32813281
// Turn off the task quota timer to avoid spurious wakeups
32823282
struct itimerspec zero_itimerspec = {};
32833283
_task_quota_timer.timerfd_settime(0, zero_itimerspec);
3284-
auto start_sleep = now();
32853284
_cpu_stall_detector->start_sleep();
3286-
sleep();
3285+
try_sleep();
32873286
_cpu_stall_detector->end_sleep();
32883287
// We may have slept for a while, so freshen idle_end
32893288
idle_end = now();
3290-
_total_sleep += idle_end - start_sleep;
32913289
_task_quota_timer.timerfd_settime(0, task_quote_itimerspec);
32923290
}
32933291
} else {
@@ -3305,8 +3303,9 @@ int reactor::do_run() {
33053303
return _return;
33063304
}
33073305

3306+
33083307
void
3309-
reactor::sleep() {
3308+
reactor::try_sleep() {
33103309
for (auto i = _pollers.begin(); i != _pollers.end(); ++i) {
33113310
auto ok = (*i)->try_enter_interrupt_mode();
33123311
if (!ok) {
@@ -4795,6 +4794,14 @@ steady_clock_type::duration reactor::total_busy_time() {
47954794
return now() - _start_time - _total_idle;
47964795
}
47974796

4797+
steady_clock_type::duration reactor::total_awake_time() const {
4798+
return now() - _start_time - _total_sleep;
4799+
}
4800+
4801+
std::chrono::nanoseconds reactor::total_cpu_time() const {
4802+
return thread_cputime_clock::now().time_since_epoch();
4803+
}
4804+
47984805
std::chrono::nanoseconds reactor::total_steal_time() {
47994806
// Steal time: this mimics the concept some Hypervisors have about Steal time.
48004807
// That is the time in which a VM has something to run, but is not running because some other
@@ -4808,9 +4815,38 @@ std::chrono::nanoseconds reactor::total_steal_time() {
48084815
// process is ready to run but the kernel hasn't scheduled us yet, that would be technically
48094816
// steal time but we have no ways to account it.
48104817
//
4818+
// Furthermore, not all steal is from other processes: time used by the syscall thread and any
4819+
// alien threads will show up as steal as well as any time spent in a system call that
4820+
// unexpectedly blocked (since CPU time won't tick up when that occurs).
4821+
//
48114822
// But what we have here should be good enough and at least has a well defined meaning.
4812-
return std::chrono::duration_cast<std::chrono::nanoseconds>(now() - _start_time - _total_sleep) -
4813-
std::chrono::duration_cast<std::chrono::nanoseconds>(thread_cputime_clock::now().time_since_epoch());
4823+
//
4824+
// Because we calculate sleep time with timestamps around polling methods that may sleep, like
4825+
// io_getevents, we systematically over-count sleep time, since there is CPU usage within the
4826+
// period timed as sleep, before and after an actual sleep occurs (and no sleep may occur at all,
4827+
// e.g., if there are events immediately available). Over-counting sleep means we under-count the
4828+
// wall-clock awake time, and so if there is no "true" steal, we will generally have a small
4829+
// *negative* steal time, because we under-count awake wall clock time while thread CPU time does
4830+
// not have a corresponding error.
4831+
//
4832+
// Becuase we claim "steal" is a counter, we must ensure that it never deceases, because PromQL
4833+
// functions which use counters will produce non-sensical results if they do. Therefore we clamp
4834+
// the output such that it never decreases.
4835+
//
4836+
// Finally, we don't just clamp difference of awake and CPU time since proces start at 0, but
4837+
// take the last value we returned from this function and then calculate the incremental steal
4838+
// time since that measurement, clamped to 0. This means that as soon as steal time becomes
4839+
// positive, it will be reflected in the measurement, rather than needing to "consume" all the
4840+
// accumulated negative steal time before positive steal times start showing up.
4841+
4842+
4843+
auto true_steal = total_awake_time() - total_cpu_time();
4844+
auto mono_steal = _last_mono_steal + std::max(true_steal - _last_true_steal, 0ns);
4845+
4846+
_last_true_steal = true_steal;
4847+
_last_mono_steal = mono_steal;
4848+
4849+
return mono_steal;
48144850
}
48154851

48164852
static std::atomic<unsigned long> s_used_scheduling_group_ids_bitmap{3}; // 0=main, 1=atexit

src/core/reactor_backend.cc

+9
Original file line numberDiff line numberDiff line change
@@ -491,7 +491,12 @@ bool reactor_backend_aio::await_events(int timeout, const sigset_t* active_sigma
491491
bool did_work = false;
492492
int r;
493493
do {
494+
const bool may_sleep = !tsp || (tsp->tv_nsec + tsp->tv_sec > 0);
495+
const auto before_getevents = may_sleep ? sched_clock::now() : sched_clock::time_point{};
494496
r = io_pgetevents(_polling_io.io_context, 1, batch_size, batch, tsp, active_sigmask);
497+
if (may_sleep) {
498+
_r._total_sleep += sched_clock::now() - before_getevents;
499+
}
495500
if (r == -1 && errno == EINTR) {
496501
return true;
497502
}
@@ -841,7 +846,9 @@ reactor_backend_epoll::wait_and_process(int timeout, const sigset_t* active_sigm
841846
}
842847
});
843848
std::array<epoll_event, 128> eevt;
849+
const auto before_pwait = sched_clock::now();
844850
int nr = ::epoll_pwait(_epollfd.get(), eevt.data(), eevt.size(), timeout, active_sigmask);
851+
_r._total_sleep += sched_clock::now() - before_pwait;
845852
if (nr == -1 && errno == EINTR) {
846853
return false; // gdb can cause this
847854
}
@@ -1468,7 +1475,9 @@ class reactor_backend_uring final : public reactor_backend {
14681475
}
14691476
struct ::io_uring_cqe* cqe = nullptr;
14701477
sigset_t sigs = *active_sigmask; // io_uring_wait_cqes() wants non-const
1478+
const auto before_wait_cqes = sched_clock::now();
14711479
auto r = ::io_uring_wait_cqes(&_uring, &cqe, 1, nullptr, &sigs);
1480+
_r._total_sleep += sched_clock::now() - before_wait_cqes;
14721481
if (__builtin_expect(r < 0, false)) {
14731482
switch (-r) {
14741483
case EINTR:

0 commit comments

Comments
 (0)