Skip to content

Commit

Permalink
Merge pull request #11667 from mmaslankaprv/vbotbuildovich/backport-1…
Browse files Browse the repository at this point in the history
…0800-v22.3.x-720

[v22.3.x] Fixed Raft voter priority override with single replica topics
  • Loading branch information
mmaslankaprv authored Jul 24, 2023
2 parents 52b8606 + b8a5e51 commit ad6fae3
Show file tree
Hide file tree
Showing 5 changed files with 56 additions and 12 deletions.
26 changes: 18 additions & 8 deletions src/v/raft/consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -856,25 +856,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
2 changes: 1 addition & 1 deletion tests/rptest/tests/cluster_features_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -330,7 +330,7 @@ def test_upgrade(self):
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
wait_until(lambda: CURRENT_LOGICAL_VERSION == self.admin.get_features(
)['cluster_version'],
timeout_sec=5,
timeout_sec=10,
backoff_sec=1)


Expand Down
29 changes: 29 additions & 0 deletions tests/rptest/tests/maintenance_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
# by the Apache License, Version 2.0

import random
from rptest.clients.default import DefaultClient

from rptest.services.admin import Admin
from rptest.tests.redpanda_test import RedpandaTest
Expand Down Expand Up @@ -251,3 +252,31 @@ def test_exclusive_maintenance(self, use_rpk):
raise
else:
raise Exception("Expected maintenance enable to fail")

@cluster(num_nodes=3)
@matrix(use_rpk=[True, False])
def test_maintenance_with_single_replicas(self, use_rpk):
self._use_rpk = use_rpk
single_replica_topic = TopicSpec(partition_count=18,
replication_factor=1)
DefaultClient(self.redpanda).create_topic(single_replica_topic)

target = random.choice(self.redpanda.nodes)

self._enable_maintenance(target)
self.redpanda.restart_nodes(target)
rpk = RpkTool(self.redpanda)

def all_partitions_have_leaders():
partitions = list(
rpk.describe_topic(single_replica_topic.name, tolerant=True))
for p in partitions:
self.logger.info(f"DBG: {p.high_watermark}")
return len(
partitions) == single_replica_topic.partition_count and all(
[p.high_watermark is not None for p in partitions])

wait_until(all_partitions_have_leaders,
30,
backoff_sec=1,
err_msg="Error waiting for all partitions to have leaders")

0 comments on commit ad6fae3

Please sign in to comment.