Skip to content

Commit

Permalink
c/leader_balancer: schedule timer if leaders_preference changed
Browse files Browse the repository at this point in the history
  • Loading branch information
ztlpn committed Oct 7, 2024
1 parent 11dae97 commit 4abc39b
Show file tree
Hide file tree
Showing 2 changed files with 60 additions and 3 deletions.
58 changes: 56 additions & 2 deletions src/v/cluster/scheduling/leader_balancer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -170,9 +170,53 @@ void leader_balancer::on_maintenance_change(

void leader_balancer::handle_topic_deltas(
const chunked_vector<topic_table_topic_delta>& deltas) {
// TODO: early exit if not leader/disabled

for (const auto& d : deltas) {
if (d.type == topic_table_topic_delta_type::added) {
leader_balancer_types::topic_id_t topic_id{d.creation_revision};

switch (d.type) {
case topic_table_topic_delta_type::removed:
continue;
case topic_table_topic_delta_type::added:
vlog(clusterlog.info, "ADDED1 {}", d.ns_tp);
schedule_timer(throttle_reactivation_delay);
continue;
case topic_table_topic_delta_type::properties_updated:
break;
}

// Schedule a tick if current topic leaders preference differs from
// what we've last seen.

if (!_last_seen_preferences) {
// Last seen map will be built for the first time when the balancer
// will next activate, we don't need to schedule additional ticks
// yet.
continue;
}

const config::leaders_preference* new_lp = nullptr;
auto maybe_md = _topics.get_topic_metadata_ref(d.ns_tp);
if (
maybe_md
&& maybe_md->get()
.get_configuration()
.properties.leaders_preference) {
new_lp = &maybe_md->get()
.get_configuration()
.properties.leaders_preference.value();
}

auto cache_it = _last_seen_preferences->find(topic_id);
if (cache_it != _last_seen_preferences->end()) {
if (!new_lp || *new_lp != cache_it->second) {
vlog(clusterlog.info, "MODIFIED {}", d.ns_tp);
schedule_timer(0s);
}
} else if (new_lp) {
vlog(clusterlog.info, "REMOVED2 {}", d.ns_tp);
schedule_timer(0s);
}
}
}
Expand Down Expand Up @@ -439,6 +483,8 @@ ss::future<ss::stop_iteration> leader_balancer::balance() {
_in_flight_changes.clear();
check_unregister_leadership_change_notification();

_last_seen_preferences = std::nullopt;

auto res = co_await _raft0->linearizable_barrier();
if (!res) {
vlog(
Expand Down Expand Up @@ -720,9 +766,15 @@ leader_balancer_types::muted_groups_t leader_balancer::muted_groups() const {
return res;
}

leader_balancer_types::topic_index leader_balancer::build_topic_index() const {
leader_balancer_types::topic_index leader_balancer::build_topic_index() {
leader_balancer_types::topic_index topic_index;

if (_last_seen_preferences) {
_last_seen_preferences->clear();
} else {
_last_seen_preferences.emplace();
}

// for each ntp in the cluster
for (const auto& topic : _topics.topics_map()) {
auto topic_id = leader_balancer_types::topic_id_t{
Expand All @@ -744,6 +796,8 @@ leader_balancer_types::topic_index leader_balancer::build_topic_index() const {
const auto& preference
= topic.second.get_configuration().properties.leaders_preference;
if (preference.has_value()) {
_last_seen_preferences.value().try_emplace(
topic_id, preference.value());
topic_index.topic2preference.try_emplace(
topic_id, preference.value());
}
Expand Down
5 changes: 4 additions & 1 deletion src/v/cluster/scheduling/leader_balancer.h
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ class leader_balancer {
using group_replicas_t = chunked_hash_map<raft::group_id, replicas_t>;
ss::future<std::optional<group_replicas_t>>
collect_group_replicas_from_health_report(const cluster_health_report&);
leader_balancer_types::topic_index build_topic_index() const;
leader_balancer_types::topic_index build_topic_index();
index_type build_index(std::optional<group_replicas_t>);
absl::flat_hash_set<model::node_id>
collect_muted_nodes(const cluster_health_report&);
Expand Down Expand Up @@ -236,6 +236,9 @@ class leader_balancer {
absl::flat_hash_map<raft::group_id, in_flight_reassignment>
_in_flight_changes;

std::optional<leader_balancer_types::topic_map<config::leaders_preference>>
_last_seen_preferences;

leader_balancer_probe _probe;
};

Expand Down

0 comments on commit 4abc39b

Please sign in to comment.