Skip to content

Commit

Permalink
controller/probe: Reduce lifecycle
Browse files Browse the repository at this point in the history
The probe requires access to members, topics, and partition_leaders.

* Start the probe after all dependencies have been started
* Stop the probe before its dependencies are stopped

Fix redpanda-data#5466

Signed-off-by: Ben Pope <ben@redpanda.com>
  • Loading branch information
BenPope committed Jul 14, 2022
1 parent 9442069 commit 4afbcda
Show file tree
Hide file tree
Showing 3 changed files with 35 additions and 19 deletions.
4 changes: 3 additions & 1 deletion src/v/cluster/controller.cc
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,8 @@ ss::future<> controller::wire_up() {
return _authorizer.start(
[]() { return config::shard_local_cfg().superusers.bind(); });
})
.then([this] { return _tp_state.start(); });
.then([this] { return _tp_state.start(); })
.then([this] { _probe.start(); });
}

ss::future<> controller::start() {
Expand Down Expand Up @@ -377,6 +378,7 @@ ss::future<> controller::stop() {
f = shutdown_input();
}

_probe.stop();
return f.then([this] {
auto stop_leader_balancer = _leader_balancer ? _leader_balancer->stop()
: ss::now();
Expand Down
45 changes: 27 additions & 18 deletions src/v/cluster/controller_probe.cc
Original file line number Diff line number Diff line change
Expand Up @@ -24,24 +24,33 @@
namespace cluster {

controller_probe::controller_probe(controller& c) noexcept
: _controller(c) {
_controller._raft_manager.local().register_leadership_notification(
[this](
raft::group_id group,
model::term_id /*term*/,
std::optional<model::node_id> leader_id) {
// We are only interested in notifications regarding the controller
// group.
if (_controller._raft0->group() != group) {
return;
}

if (leader_id != _controller.self()) {
_public_metrics.reset();
} else {
setup_metrics();
}
});
: _controller(c)
, _leadership_notification_handle{} {}

void controller_probe::start() {
_leadership_notification_handle
= _controller._raft_manager.local().register_leadership_notification(
[this](
raft::group_id group,
model::term_id /*term*/,
std::optional<model::node_id> leader_id) {
// We are only interested in notifications regarding the controller
// group.
if (_controller._raft0->group() != group) {
return;
}

if (leader_id != _controller.self()) {
_public_metrics.reset();
} else {
setup_metrics();
}
});
}

void controller_probe::stop() {
_controller._raft_manager.local().unregister_leadership_notification(
_leadership_notification_handle);
}

void controller_probe::setup_metrics() {
Expand Down
5 changes: 5 additions & 0 deletions src/v/cluster/controller_probe.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#pragma once

#include "cluster/fwd.h"
#include "cluster/types.h"
#include "seastarx.h"

#include <seastar/core/metrics_registration.hh>
Expand All @@ -22,11 +23,15 @@ class controller_probe {
public:
explicit controller_probe(cluster::controller&) noexcept;

void start();
void stop();

void setup_metrics();

private:
cluster::controller& _controller;
std::unique_ptr<ss::metrics::metric_groups> _public_metrics;
cluster::notification_id_type _leadership_notification_handle;
};

} // namespace cluster

0 comments on commit 4afbcda

Please sign in to comment.