Skip to content

Commit

Permalink
cpu_scheduling: add scheduling_groups probe
Browse files Browse the repository at this point in the history
This patch introduces a probe that queries each scheduling group
for its usage stats and publishes metrics based on that.

The following new metric is introduced:
Name: redpanda_scheduler_runtime_seconds
Description: Accumulated runtime of task queue associated with this
scheduling group
Labels:
  - redpanda_scheduling_group
  - shard
  • Loading branch information
Vlad Lazar committed Aug 8, 2022
1 parent b06d762 commit 2309fce
Showing 1 changed file with 51 additions and 0 deletions.
51 changes: 51 additions & 0 deletions src/v/resource_mgmt/scheduling_groups_probe.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* Copyright 2020 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#pragma once

#include "cluster/partition_leaders_table.h"
#include "config/configuration.h"
#include "prometheus/prometheus_sanitize.h"
#include "resource_mgmt/cpu_scheduling.h"
#include "ssx/metrics.h"

#include <seastar/core/metrics.hh>

class scheduling_groups_probe {
public:
void wire_up(const scheduling_groups& scheduling_groups) {
if (config::shard_local_cfg().disable_public_metrics()) {
return;
}

auto groups = scheduling_groups.all_scheduling_groups();
for (auto&& group_ref : groups) {
_public_metrics.add_group(
prometheus_sanitize::metrics_name("scheduler"),
{seastar::metrics::make_gauge(
"runtime_seconds",
[group_ref] {
return group_ref.get().get_stats().runtime.count();
},
seastar::metrics::description(
"Accumulated runtime of task queue associated with this "
"scheduling group"),
{ssx::metrics::make_namespaced_label("scheduling_group")(
group_ref.get().name())})});
}
}

void clear() { _public_metrics.clear(); }

private:
seastar::metrics::metric_groups _public_metrics{
ssx::metrics::public_metrics_handle};
};

0 comments on commit 2309fce

Please sign in to comment.