Skip to content

Commit

Permalink
r/prevote_stm: fail prevote if term changed
Browse files Browse the repository at this point in the history
If a candidate receives a reply with term which is different than
requested then the vote should not be treated as success. Additionally
the term of candidate should be updated.

Signed-off-by: Michał Maślanka <michal@redpanda.com>
  • Loading branch information
mmaslankaprv committed May 22, 2024
1 parent 0d7f850 commit a6db30d
Show file tree
Hide file tree
Showing 2 changed files with 42 additions and 14 deletions.
53 changes: 39 additions & 14 deletions src/v/raft/prevote_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#include "rpc/types.h"
#include "ssx/semaphore.h"

#include <seastar/core/future.hh>
#include <seastar/util/bool_class.hh>

#include <chrono>
Expand Down Expand Up @@ -81,6 +82,18 @@ prevote_stm::process_reply(vnode n, ss::future<result<vote_reply>> f) {
auto r = f.get0();
if (r.has_value()) {
auto v = r.value();
if (v.term != _req.term) {
vlog(
_ctxlog.trace,
"prevote ack: node {} has a higher term {}",
n,
v.term);
voter_reply->second._is_failed = true;
voter_reply->second._is_pending = false;
if (v.term > _req.term) {
_term_update = v.term;
}
}
_ptr->maybe_update_node_reply_timestamp(n);
if (v.log_ok) {
vlog(
Expand Down Expand Up @@ -182,20 +195,32 @@ ss::future<bool> prevote_stm::do_prevote() {
[this](vnode id) { ssx::background = dispatch_prevote(id); });

// process results
return process_replies().then([this]() {
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: {}",
_ptr->_node_priority_override.value());
_success = false;
}
return _success;
});
return process_replies()
.then([this]() {
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: {}",
_ptr->_node_priority_override.value());
_success = false;
}
})
.then([this] {
if (_term_update) {
return update_term().then([this] { return _success; });
}
return ss::make_ready_future<bool>(_success);
});
}

ss::future<> prevote_stm::update_term() {
auto u = co_await _ptr->_op_lock.get_units();
_ptr->_term = std::max(_ptr->term(), _term_update.value());
_ptr->_vstate = consensus::vote_state::candidate;
}

ss::future<> prevote_stm::process_replies() {
Expand Down
3 changes: 3 additions & 0 deletions src/v/raft/prevote_stm.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

#pragma once

#include "model/fundamental.h"
#include "outcome.h"
#include "raft/logger.h"
#include "raft/types.h"
Expand Down Expand Up @@ -51,11 +52,13 @@ class prevote_stm {
ss::future<result<vote_reply>> do_dispatch_prevote(vnode);
ss::future<> process_reply(vnode n, ss::future<result<vote_reply>> f);
ss::future<> process_replies();
ss::future<> update_term();
// args
consensus* _ptr;
// make sure to always make a copy; never move() this struct
vote_request _req;
bool _success = false;
std::optional<model::term_id> _term_update;
// for sequentiality/progress
ssx::semaphore _sem;
std::optional<raft::group_configuration> _config;
Expand Down

0 comments on commit a6db30d

Please sign in to comment.