@@ -605,7 +605,16 @@ handle_leader({command, Cmd}, #{cfg := #cfg{id = Self,
605605 Effects = append_error_reply (Cmd , Reason , Effects0 ),
606606 {leader , State , Effects };
607607 {ok , Idx , Term , State0 , Effects00 } ->
608- {State , _ , Effects0 } = make_pipelined_rpc_effects (State0 , Effects00 ),
608+ % % if the command is a noop command we should force it to
609+ % % be pipelined to all followers
610+ Force = case Cmd of
611+ {noop , _ , _ } ->
612+ true ;
613+ _ ->
614+ false
615+ end ,
616+ {State , _ , Effects0 } =
617+ make_pipelined_rpc_effects (State0 , Effects00 , Force ),
609618 % check if a reply is required.
610619 Effects = after_log_append_reply (Cmd , Idx , Term , Effects0 ),
611620 {leader , State , Effects }
@@ -934,11 +943,9 @@ handle_candidate(#request_vote_result{term = Term, vote_granted = true},
934943 [LogId , Term , NewVotes ]),
935944 case required_quorum (Nodes ) of
936945 NewVotes ->
937- {State1 , Effects } = make_all_rpcs (initialise_peers (State0 )),
938- State = State1 #{leader_id => Id },
939- PostElectionEffects = post_election_effects (State ),
940- {leader , maps :without ([votes ], State ),
941- PostElectionEffects ++ Effects };
946+ State = initialise_peers (State0 #{leader_id => Id }),
947+ Effects = post_election_effects (State ),
948+ {leader , maps :without ([votes ], State ), Effects };
942949 _ ->
943950 {candidate , State0 #{votes => NewVotes }, []}
944951 end ;
@@ -1982,14 +1989,17 @@ evaluate_commit_index_follower(State, Effects) ->
19821989 % % when no leader is known
19831990 {follower , State , Effects }.
19841991
1992+ make_pipelined_rpc_effects (State , Effects ) ->
1993+ make_pipelined_rpc_effects (State , Effects , false ).
1994+
19851995make_pipelined_rpc_effects (#{cfg := # cfg {id = Id ,
19861996 max_append_entries_rpc_batch_size =
19871997 MaxBatchSize ,
19881998 max_pipeline_count = MaxPipelineCount },
19891999 commit_index := CommitIndex ,
19902000 log := Log ,
19912001 cluster := Cluster } = State0 ,
1992- Effects0 ) ->
2002+ Effects0 , Force ) ->
19932003 NextLogIdx = ra_log :next_index (Log ),
19942004 % % TODO: refactor this please, why does make_rpc_effect need to take the
19952005 % % full state
@@ -2006,7 +2016,8 @@ make_pipelined_rpc_effects(#{cfg := #cfg{id = Id,
20062016 % check if the match index isn't too far behind the
20072017 % next index
20082018 NumInFlight = NextIdx - MatchIdx - 1 ,
2009- case NumInFlight < MaxPipelineCount of
2019+ case NumInFlight < MaxPipelineCount orelse
2020+ Force of
20102021 true ->
20112022 % % use the last list of entries as a cache
20122023 % % for the next to potentially avoid additional reads
@@ -2686,7 +2697,8 @@ append_self(Self, Nodes) ->
26862697initialise_peers (State = #{log := Log , cluster := Cluster0 }) ->
26872698 NextIdx = ra_log :next_index (Log ),
26882699 Cluster = maps :map (fun (_ , Peer0 ) ->
2689- Peer1 = maps :with ([voter_status ], Peer0 ),
2700+ Peer1 = maps :with ([voter_status ,
2701+ machine_version ], Peer0 ),
26902702 Peer2 = Peer1 #{next_index => NextIdx },
26912703 new_peer_with (Peer2 )
26922704 end , Cluster0 ),
@@ -3461,10 +3473,9 @@ after_log_append_reply(Cmd, Idx, Term, Effects0) ->
34613473 Effects0
34623474 end .
34633475
3464- post_election_effects (
3465- #{cfg := # cfg {effective_machine_version = EffectiveMacVer ,
3466- machine = Mac ,
3467- system_config = SystemConfig }} = State ) ->
3476+ post_election_effects (#{cfg := # cfg {effective_machine_version = EffectiveMacVer ,
3477+ machine = Mac ,
3478+ system_config = SystemConfig }} = State ) ->
34683479 Peers = peers (State ),
34693480 PeerIds = maps :keys (Peers ),
34703481
@@ -3510,8 +3521,8 @@ info_rpc_effects(#{cfg := #cfg{id = Id}, cluster := Cluster} = State) ->
35103521 end , [], Cluster ),
35113522 InfoRpcEffects .
35123523
3513- info_rpc_effects_for_peer (
3514- #{ cluster : = Cluster , current_term := CurTerm } = State , PeerId ) ->
3524+ info_rpc_effects_for_peer (#{ cluster : = Cluster ,
3525+ current_term := CurTerm } = State , PeerId ) ->
35153526 % % We determine if we need to ask (for the fist time or again) the info
35163527 % % from a peer.
35173528 SendRpc = case Cluster of
@@ -3571,9 +3582,8 @@ ra_server_info(State, Keys) ->
35713582 end , Info0 ),
35723583 Info1 .
35733584
3574- handle_info_reply (
3575- #{cluster := Cluster } = State ,
3576- # info_reply {from = PeerId , keys = Keys , info = Info }) ->
3585+ handle_info_reply (#{cluster := Cluster } = State ,
3586+ # info_reply {from = PeerId , keys = Keys , info = Info }) ->
35773587 PeerState0 = maps :get (PeerId , Cluster ),
35783588 PeerState1 = lists :foldl (fun (Key , PS ) ->
35793589 case Info of
0 commit comments