Skip to content

Commit 4684ca6

Browse files
author
Alex Valiushko
committed
ignore pre_vote from non-voters
1 parent d7e3030 commit 4684ca6

File tree

2 files changed

+50
-24
lines changed

2 files changed

+50
-24
lines changed

src/ra_server.erl

Lines changed: 35 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -777,7 +777,7 @@ handle_candidate(#request_vote_result{term = Term, vote_granted = true},
777777
NewVotes = Votes + 1,
778778
?DEBUG("~s: vote granted for term ~b votes ~b",
779779
[LogId, Term, NewVotes]),
780-
case need_acks(Nodes) of
780+
case trunc(maps:size(Nodes) / 2) + 1 of
781781
NewVotes ->
782782
{State1, Effects} = make_all_rpcs(initialise_peers(State0)),
783783
Noop = {noop, #{ts => erlang:system_time(millisecond)},
@@ -923,7 +923,7 @@ handle_pre_vote(#pre_vote_result{term = Term, vote_granted = true,
923923
[LogId, Token, Term, Votes + 1]),
924924
NewVotes = Votes + 1,
925925
State = update_term(Term, State0),
926-
case need_acks(Nodes) of
926+
case trunc(maps:size(Nodes) / 2) + 1 of
927927
NewVotes ->
928928
call_for_election(candidate, State);
929929
_ ->
@@ -1362,6 +1362,8 @@ overview(#{cfg := #cfg{effective_machine_module = MacMod} = Cfg,
13621362
log := Log,
13631363
machine_state := MacState,
13641364
aux_state := Aux,
1365+
leader_id := Leader,
1366+
cluster := Cluster,
13651367
queries_waiting_heartbeats := Queries
13661368
} = State) ->
13671369
NumQueries = queue:len(Queries),
@@ -1375,9 +1377,22 @@ overview(#{cfg := #cfg{effective_machine_module = MacMod} = Cfg,
13751377
cluster_index_term,
13761378
query_index
13771379
], State),
1378-
O = maps:merge(O0, cfg_to_map(Cfg)),
1380+
O1 = maps:merge(O0, cfg_to_map(Cfg)),
13791381
LogOverview = ra_log:overview(Log),
13801382
MacOverview = ra_machine:overview(MacMod, MacState),
1383+
case id(State) of
1384+
Leader ->
1385+
O = O1#{
1386+
voters => maps:fold(fun(Id, Peer, Voters) ->
1387+
case is_voter(Peer) of
1388+
true -> [Id | Voters];
1389+
false -> Voters
1390+
end
1391+
end, [], Cluster)
1392+
};
1393+
_ ->
1394+
O = O1
1395+
end,
13811396
O#{log => LogOverview,
13821397
aux => Aux,
13831398
machine => MacOverview,
@@ -2034,7 +2049,8 @@ process_pre_vote(FsmState, #pre_vote_rpc{term = Term, candidate_id = Cand,
20342049
when Term >= CurTerm ->
20352050
State = update_term(Term, State0),
20362051
LastIdxTerm = last_idx_term(State),
2037-
case is_voter(Cand, Cluster) andalso
2052+
CandidateIsVoter = is_voter(Cand, Cluster, LLIdx),
2053+
case CandidateIsVoter andalso
20382054
is_candidate_log_up_to_date(LLIdx, LLTerm, LastIdxTerm) of
20392055
true when Version > ?RA_PROTO_VERSION->
20402056
?DEBUG("~s: declining pre-vote for ~w for protocol version ~b",
@@ -2065,7 +2081,7 @@ process_pre_vote(FsmState, #pre_vote_rpc{term = Term, candidate_id = Cand,
20652081
[log_id(State0), Cand, Term, {LLIdx, LLTerm}, LastIdxTerm,
20662082
get_voter_status(Cand, Cluster)]),
20672083
case FsmState of
2068-
follower ->
2084+
follower when CandidateIsVoter ->
20692085
{FsmState, State, [start_election_timeout]};
20702086
_ ->
20712087
{FsmState, State,
@@ -2620,29 +2636,28 @@ get_voter_status(_) ->
26202636
yes.
26212637

26222638

2623-
is_voter(Id, Cluster) ->
2624-
case maps:get(Id, Cluster) of
2639+
is_voter(Id, Cluster, Idx) ->
2640+
case maps:get(Id, Cluster, undefined) of
26252641
undefined -> false;
2626-
Peer -> is_voter(Peer)
2642+
Peer -> is_voter(Peer, Idx)
26272643
end.
26282644

2629-
is_voter(#{voter := {matching, Target}, match_index := MI})
2645+
is_voter(#{} = Peer) ->
2646+
is_voter(Peer, undefined).
2647+
2648+
is_voter(#{voter := {matching, Target}, match_index := MI}, undefined)
26302649
when MI >= Target ->
26312650
true;
2632-
is_voter(#{voter := {matching, _}}) ->
2651+
is_voter(#{voter := {matching, _}}, undefined) ->
26332652
false;
2634-
is_voter(_Peer) ->
2653+
is_voter(#{voter := {matching, Target}}, Idx)
2654+
when Idx >= Target ->
2655+
true;
2656+
is_voter(#{voter := {matching, _}}, _Idx) ->
2657+
false;
2658+
is_voter(#{}, _) ->
26352659
true.
26362660

2637-
need_acks(Cluster) ->
2638-
NumVoters = maps:fold(fun(_, Peer, Count) ->
2639-
case is_voter(Peer) of
2640-
true -> Count + 1;
2641-
false -> Count
2642-
end
2643-
end, 0, Cluster),
2644-
trunc(NumVoters / 2) + 1.
2645-
26462661
-spec agreed_commit(list()) -> ra_index().
26472662
agreed_commit(Indexes) ->
26482663
SortedIdxs = lists:sort(fun erlang:'>'/2, Indexes),

test/ra_server_SUITE.erl

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -884,7 +884,7 @@ append_entries_nonvoter(_Config) ->
884884
N3 => new_peer_with(#{next_index => 2, match_index => 1}),
885885
N4 => new_peer_with(#{next_index => 2, match_index => 1,
886886
voter => {matching, 3}})},
887-
State0 = (base_state(3, ?FUNCTION_NAME))#{
887+
State0 = (base_state(4, ?FUNCTION_NAME))#{
888888
commit_index => 1,
889889
last_applied => 1,
890890
cluster => Cluster,
@@ -956,9 +956,13 @@ follower_request_vote(_Config) ->
956956
ok.
957957

958958
follower_pre_vote(_Config) ->
959-
State = base_state(3, ?FUNCTION_NAME),
960-
Term = 5,
961-
Token = make_ref(),
959+
Term = 5, Token = make_ref(), N1 = ?N1, N2 = ?N2, N3 = ?N3,
960+
Cluster = #{N1 => new_peer_with(#{next_index => 4, match_index => 3}),
961+
N2 => new_peer_with(#{next_index => 4, match_index => 3,
962+
voter => {matching, 3}}),
963+
N3 => new_peer_with(#{next_index => 4, match_index => 3,
964+
voter => {matching, 4}})},
965+
State = (base_state(3, ?FUNCTION_NAME))#{cluster => Cluster},
962966
Msg = #pre_vote_rpc{candidate_id = ?N2, term = Term, last_log_index = 3,
963967
machine_version = 0,
964968
token = Token, last_log_term = 5},
@@ -968,6 +972,13 @@ follower_pre_vote(_Config) ->
968972
vote_granted = true}}]} =
969973
ra_server:handle_follower(Msg, State),
970974

975+
% disallow pre votes from non-voters
976+
{follower, _,
977+
[{reply, #pre_vote_result{term = Term, token = Token,
978+
vote_granted = false}}]} =
979+
ra_server:handle_follower(Msg#pre_vote_rpc{candidate_id = N3},
980+
State),
981+
971982
% disallow pre votes from higher protocol version
972983
?assertMatch(
973984
{follower, _,

0 commit comments

Comments
 (0)