Skip to content

Commit ddf5f52

Browse files
committed
no-leaderless-notification
Signed-off-by: Michał Maślanka <[email protected]>
1 parent 76406d1 commit ddf5f52

File tree

4 files changed

+17
-16
lines changed

4 files changed

+17
-16
lines changed

src/v/raft/consensus.cc

-4
Original file line numberDiff line numberDiff line change
@@ -237,7 +237,6 @@ void consensus::maybe_step_down() {
237237
do_step_down("heartbeats_majority");
238238
if (_leader_id) {
239239
_leader_id = std::nullopt;
240-
trigger_leadership_notification();
241240
}
242241
}
243242
}
@@ -1881,7 +1880,6 @@ ss::future<vote_reply> consensus::do_vote(vote_request r) {
18811880
do_step_down("candidate_term_greater");
18821881
if (_leader_id) {
18831882
_leader_id = std::nullopt;
1884-
trigger_leadership_notification();
18851883
}
18861884

18871885
// do not grant vote if log isn't ok
@@ -3134,7 +3132,6 @@ ss::future<> consensus::maybe_commit_configuration(ssx::semaphore_units u) {
31343132
co_await transfer_and_stepdown("no_longer_member");
31353133
if (_leader_id) {
31363134
_leader_id = std::nullopt;
3137-
trigger_leadership_notification();
31383135
}
31393136
}
31403137
}
@@ -3664,7 +3661,6 @@ consensus::do_transfer_leadership(transfer_leadership_request req) {
36643661
do_step_down("leadership_transfer");
36653662
if (_leader_id) {
36663663
_leader_id = std::nullopt;
3667-
trigger_leadership_notification();
36683664
}
36693665

36703666
return make_error_code(errc::success);

src/v/raft/consensus.h

-1
Original file line numberDiff line numberDiff line change
@@ -359,7 +359,6 @@ class consensus {
359359
do_step_down(fmt::format("external_stepdown - {}", ctx));
360360
if (_leader_id) {
361361
_leader_id = std::nullopt;
362-
trigger_leadership_notification();
363362
}
364363
});
365364
}

src/v/raft/tests/basic_raft_fixture_test.cc

+17-10
Original file line numberDiff line numberDiff line change
@@ -770,15 +770,15 @@ TEST_F_CORO(raft_fixture, leadership_transfer_delay) {
770770
auto leader_id = get_leader().value();
771771
auto& leader_node = node(leader_id);
772772
auto current_term = leader_node.raft()->term();
773+
clock_t::time_point start = clock_t::now();
773774
auto r = co_await leader_node.raft()->transfer_leadership(
774775
transfer_leadership_request{.group = leader_node.raft()->group()});
775776
ASSERT_TRUE_CORO(r.success);
776777
// here we wait for all the replicas to notify about the leadership changes,
777-
// each replica will notify two times, one when there is no leader, second
778-
// time when the leader is elected. We have 4 replicas so in total we expect
779-
// 8 notifications to be fired.
778+
// each replica will notify one time only when the leader is elected. We
779+
// have 4 replicas so in total we expect 4 notifications to be fired.
780780
co_await tests::cooperative_spin_wait_with_timeout(
781-
10s, [&] { return events.size() >= 8; });
781+
10s, [&] { return events.size() >= 4; });
782782

783783
// calculate the time needed to transfer leadership, in our case it is the
784784
// time between first notification reporting no leader and first reporting
@@ -789,8 +789,7 @@ TEST_F_CORO(raft_fixture, leadership_transfer_delay) {
789789
&& ev.status.term > current_term;
790790
});
791791

792-
auto transfer_time = new_leader_reported_ev->timestamp
793-
- events.begin()->timestamp;
792+
auto transfer_time = new_leader_reported_ev->timestamp - start;
794793
vlog(
795794
tstlog.info,
796795
"leadership_transfer - new leader reported after: {} ms",
@@ -806,10 +805,19 @@ TEST_F_CORO(raft_fixture, leadership_transfer_delay) {
806805
co_await new_leader_node.raft()->replace_configuration(
807806
std::vector<vnode>{new_nodes.begin(), new_nodes.end()},
808807
model::revision_id(2));
809-
// analogically to the previous case we wait for 6 notifications as
808+
// wait for no leader to be reported
809+
co_await tests::cooperative_spin_wait_with_timeout(10s, [&] {
810+
if (new_leader_node.raft()->get_leader_id() == std::nullopt) {
811+
return true;
812+
}
813+
return new_leader_node.raft()->term() > current_term;
814+
});
815+
816+
start = clock_t::now();
817+
// analogically to the previous case we wait for 3 notifications as
810818
// currently the group has only 3 replicas
811819
co_await tests::cooperative_spin_wait_with_timeout(
812-
10s, [&] { return events.size() >= 6; });
820+
10s, [&] { return events.size() >= 3; });
813821

814822
auto leader_reported_after_reconfiguration = std::find_if(
815823
events.begin(), events.end(), [&](leadership_changed_event& ev) {
@@ -818,7 +826,7 @@ TEST_F_CORO(raft_fixture, leadership_transfer_delay) {
818826
});
819827

820828
auto election_time = leader_reported_after_reconfiguration->timestamp
821-
- events.begin()->timestamp;
829+
- start;
822830
vlog(
823831
tstlog.info,
824832
"reconfiguration - new leader reported after: {} ms",
@@ -834,7 +842,6 @@ TEST_F_CORO(raft_fixture, leadership_transfer_delay) {
834842
* time needed for leadership transfer
835843
*/
836844
ASSERT_LE_CORO(election_time * 1.0, transfer_time * tolerance_multiplier);
837-
ASSERT_GE_CORO(election_time * 1.0, transfer_time / tolerance_multiplier);
838845
}
839846

840847
TEST_F_CORO(raft_fixture, test_no_stepdown_on_append_entries_timeout) {

src/v/raft/vote_stm.cc

-1
Original file line numberDiff line numberDiff line change
@@ -140,7 +140,6 @@ ss::future<election_success> vote_stm::vote(bool leadership_transfer) {
140140
// only trigger notification when we had a leader previously
141141
if (_ptr->_leader_id) {
142142
_ptr->_leader_id = std::nullopt;
143-
_ptr->trigger_leadership_notification();
144143
}
145144

146145
if (_prevote && leadership_transfer) {

0 commit comments

Comments
 (0)