Skip to content

Commit

Permalink
r/consensus: prefer availability over voter priority
Browse files Browse the repository at this point in the history
Redpanda Raft implementation exposes an API allowing to override a voter
priority. This is used by the drain manager when a node is in
maintenance mode. In current implementation when the only voter is in
maintenance mode the Raft group is not able to elect a leader as the
reported priority it to low (the priority override in maintenance is set to 0).

Fixed Raft implementation to make sure that it prioritize an
availability over the user priority preference. If a node is the only
voter the priority override is ignored.

Fixes: redpanda-data/cloudv2#6174

Signed-off-by: Michal Maslanka <[email protected]>
  • Loading branch information
mmaslankaprv committed Jun 16, 2023
1 parent e13a04d commit 1b21f83
Show file tree
Hide file tree
Showing 3 changed files with 26 additions and 11 deletions.
26 changes: 18 additions & 8 deletions src/v/raft/consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -883,25 +883,35 @@ void consensus::dispatch_vote(bool leadership_transfer) {
// lower our required priority for next time.
_target_priority = next_target_priority();

const auto& latest_config = _configuration_manager.get_latest();
// skip sending vote request if current node is not a voter in current
// configuration
if (!_configuration_manager.get_latest().is_allowed_to_request_votes(
_self)) {
if (!latest_config.is_allowed_to_request_votes(_self)) {
arm_vote_timeout();
return;
}

// if priority is to low, skip dispatching votes, do not take priority into
// account when we transfer leadership
if (current_priority_to_low && !leadership_transfer) {
const bool is_only_voter = latest_config.unique_voter_count() == 1
&& latest_config.is_voter(_self);
if (!is_only_voter) {
vlog(
_ctxlog.trace,
"current node priority {} is lower than target {} (next vote {})",
self_priority,
cur_target_priority,
_target_priority);
arm_vote_timeout();
return;
}
vlog(
_ctxlog.trace,
"current node priority {} is lower than target {} (next vote {})",
_ctxlog.info,
"current node priority {} is lower than target {}, however the node "
"is the only voter, continue with dispatching vote",
self_priority,
cur_target_priority,
_target_priority);
arm_vote_timeout();
return;
cur_target_priority);
}
// background, acquire lock, transition state
ssx::background
Expand Down
6 changes: 5 additions & 1 deletion src/v/raft/prevote_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,11 @@ ss::future<bool> prevote_stm::do_prevote() {

// process results
return process_replies().then([this]() {
if (_success && _ptr->_node_priority_override == zero_voter_priority) {
const auto only_voter = _config->unique_voter_count() == 1
&& _config->is_voter(_ptr->self());
if (
_success && !only_voter
&& _ptr->_node_priority_override == zero_voter_priority) {
vlog(
_ctxlog.debug,
"Ignoring successful pre-vote. Node priority too low: {}",
Expand Down
5 changes: 3 additions & 2 deletions src/v/raft/vote_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -235,8 +235,9 @@ ss::future<> vote_stm::update_vote_state(ssx::semaphore_units u) {
_ptr->_vstate = consensus::vote_state::follower;
co_return;
}

if (_ptr->_node_priority_override == zero_voter_priority) {
const auto only_voter = _config->unique_voter_count() == 1
&& _config->is_voter(_ptr->self());
if (!only_voter && _ptr->_node_priority_override == zero_voter_priority) {
vlog(
_ctxlog.debug,
"Ignoring successful vote. Node priority too low: {}",
Expand Down

0 comments on commit 1b21f83

Please sign in to comment.