diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 780a7c23a56c..d3e7cc20b5e3 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -58,7 +58,6 @@ #include "redpanda/admin_server.h" #include "resource_mgmt/io_priority.h" #include "rpc/simple_protocol.h" -#include "ssx/metrics.h" #include "storage/backlog_controller.h" #include "storage/chunk_cache.h" #include "storage/compaction_controller.h" @@ -294,8 +293,11 @@ void application::initialize( } _scheduling_groups.create_groups().get(); - _deferred.emplace_back( - [this] { _scheduling_groups.destroy_groups().get(); }); + _scheduling_groups_probe.wire_up(_scheduling_groups); + _deferred.emplace_back([this] { + _scheduling_groups_probe.clear(); + _scheduling_groups.destroy_groups().get(); + }); if (proxy_cfg) { _proxy_config.emplace(*proxy_cfg); @@ -314,23 +316,69 @@ void application::initialize( } void application::setup_metrics() { - if (!config::shard_local_cfg().disable_public_metrics()) { - seastar::metrics::replicate_metric_families( - seastar::metrics::default_handle(), - {{"scheduler_runtime_ms", ssx::metrics::public_metrics_handle}, - {"io_queue_total_read_ops", ssx::metrics::public_metrics_handle}, - {"io_queue_total_write_ops", ssx::metrics::public_metrics_handle}, - {"memory_allocated_memory", ssx::metrics::public_metrics_handle}, - {"memory_free_memory", ssx::metrics::public_metrics_handle}}) - .get(); - } + setup_internal_metrics(); + setup_public_metrics(); +} - if (config::shard_local_cfg().disable_metrics()) { +void application::setup_public_metrics() { + namespace sm = ss::metrics; + + if (config::shard_local_cfg().disable_public_metrics()) { return; } + seastar::metrics::replicate_metric_families( + seastar::metrics::default_handle(), + {{"io_queue_total_read_ops", ssx::metrics::public_metrics_handle}, + {"io_queue_total_write_ops", ssx::metrics::public_metrics_handle}, + {"memory_allocated_memory", ssx::metrics::public_metrics_handle}, + {"memory_free_memory", ssx::metrics::public_metrics_handle}}) + .get(); + + _public_metrics.start().get(); + + _public_metrics + .invoke_on( + ss::this_shard_id(), + [](auto& public_metrics) { + public_metrics.groups.add_group( + "application", + {sm::make_gauge( + "uptime_seconds_total", + [] { + return std::chrono::duration(ss::engine().uptime()) + .count(); + }, + sm::description("Redpanda uptime in seconds")) + .aggregate({sm::shard_label})}); + }) + .get(); + + _public_metrics.invoke_on_all([](auto& public_metrics) { + public_metrics.groups.add_group( + "cpu", + {sm::make_gauge( + "busy_seconds_total", + [] { + return std::chrono::duration( + ss::engine().total_busy_time()) + .count(); + }, + sm::description("Total CPU busy time in seconds"))}); + }).get(); + + _deferred.emplace_back([this] { + _public_metrics.stop().get(); + }); +} + +void application::setup_internal_metrics() { namespace sm = ss::metrics; + if (config::shard_local_cfg().disable_metrics()) { + return; + } + // build info auto version_label = sm::label("version"); auto revision_label = sm::label("revision"); diff --git a/src/v/redpanda/application.h b/src/v/redpanda/application.h index c88bc13e916d..01a9bb602140 100644 --- a/src/v/redpanda/application.h +++ b/src/v/redpanda/application.h @@ -31,9 +31,11 @@ #include "redpanda/admin_server.h" #include "resource_mgmt/cpu_scheduling.h" #include "resource_mgmt/memory_groups.h" +#include "resource_mgmt/scheduling_groups_probe.h" #include "resource_mgmt/smp_groups.h" #include "rpc/fwd.h" #include "seastarx.h" +#include "ssx/metrics.h" #include "storage/fwd.h" #include "v8_engine/fwd.h" @@ -148,6 +150,8 @@ class application { } void setup_metrics(); + void setup_public_metrics(); + void setup_internal_metrics(); std::unique_ptr _app; bool _redpanda_enabled{true}; cluster::config_manager::preload_result _config_preload; @@ -157,6 +161,7 @@ class application { _schema_reg_config; std::optional _schema_reg_client_config; scheduling_groups _scheduling_groups; + scheduling_groups_probe _scheduling_groups_probe; ss::logger _log; ss::sharded _connection_cache; @@ -174,6 +179,7 @@ class application { ss::sharded _archival_upload_controller; ss::metrics::metric_groups _metrics; + ss::sharded _public_metrics; std::unique_ptr _rm_group_proxy; // run these first on destruction deferred_actions _deferred; diff --git a/src/v/resource_mgmt/cpu_scheduling.h b/src/v/resource_mgmt/cpu_scheduling.h index 0d07761e068f..796ef761449b 100644 --- a/src/v/resource_mgmt/cpu_scheduling.h +++ b/src/v/resource_mgmt/cpu_scheduling.h @@ -65,7 +65,24 @@ class scheduling_groups final { } ss::scheduling_group archival_upload() { return _archival_upload; } + std::vector> + all_scheduling_groups() const { + return { + std::cref(_default), + std::cref(_admin), + std::cref(_raft), + std::cref(_kafka), + std::cref(_cluster), + std::cref(_coproc), + std::cref(_cache_background_reclaim), + std::cref(_compaction), + std::cref(_raft_learner_recovery), + std::cref(_archival_upload)}; + } + private: + ss::scheduling_group _default{ + seastar::default_scheduling_group()}; // created and managed by seastar ss::scheduling_group _admin; ss::scheduling_group _raft; ss::scheduling_group _kafka; diff --git a/src/v/resource_mgmt/scheduling_groups_probe.h b/src/v/resource_mgmt/scheduling_groups_probe.h new file mode 100644 index 000000000000..013d5a1b851c --- /dev/null +++ b/src/v/resource_mgmt/scheduling_groups_probe.h @@ -0,0 +1,52 @@ +/* + * Copyright 2022 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 + +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 (const auto& group_ref : groups) { + _public_metrics.add_group( + prometheus_sanitize::metrics_name("scheduler"), + {seastar::metrics::make_counter( + "runtime_seconds_total", + [group_ref] { + auto runtime_duration = group_ref.get().get_stats().runtime; + return std::chrono::duration(runtime_duration).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}; +}; diff --git a/src/v/ssx/metrics.h b/src/v/ssx/metrics.h index 4f3afd9ae8db..b1c65a5fafe2 100644 --- a/src/v/ssx/metrics.h +++ b/src/v/ssx/metrics.h @@ -13,6 +13,7 @@ #include "utils/hdr_hist.h" +#include #include namespace ssx::metrics { @@ -40,4 +41,12 @@ inline ss::metrics::label make_namespaced_label(const seastar::sstring& name) { return ss::metrics::label(ssx::sformat("{}_{}", label_namespace, name)); } +struct public_metrics_group { + ss::metrics::metric_groups groups{public_metrics_handle}; + ss::future<> stop() { + groups.clear(); + return ss::make_ready_future<>(); + } +}; + } // namespace ssx::metrics