You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by rn...@apache.org on 2022/09/01 16:05:58 UTC

[couchdb] 03/09: don't track matchIndex/nextIndex in non-leaders, pointless

This is an automated email from the ASF dual-hosted git repository.

rnewson pushed a commit to branch raft_storemodule
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit eeead2a86521372346516e6df2e720fb559903ef
Author: Robert Newson <rn...@apache.org>
AuthorDate: Mon Jul 4 23:23:36 2022 +0100

    don't track matchIndex/nextIndex in non-leaders, pointless
---
 src/couch/src/couch_raft.erl | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/src/couch/src/couch_raft.erl b/src/couch/src/couch_raft.erl
index fda19cc22..c580346fc 100644
--- a/src/couch/src/couch_raft.erl
+++ b/src/couch/src/couch_raft.erl
@@ -49,14 +49,10 @@ start_link(Name, StoreModule, StoreState) ->
     gen_statem:start_link({local, Name}, ?MODULE, new(Name, StoreModule, StoreState), []).
 
 new(Name, StoreModule, StoreState) ->
-    #{cohort := Cohort} = StoreState,
-    Peers = peers(Cohort),
     maps:merge(#{
         name => Name,
         store_module => StoreModule,
         votesGranted => #{},
-        nextIndex => maps:from_list([{Peer, 1} || Peer <- Peers]),
-        matchIndex => maps:from_list([{Peer, 0} || Peer <- Peers]),
         froms => #{}
     }, StoreState).
 
@@ -81,7 +77,8 @@ handle_event(enter, _OldState, follower, Data) ->
     #{term := Term, froms := Froms} = Data,
     couch_log:notice("~p became follower in term ~B", [node(), Term]),
     Replies = [{reply, From, {error, deposed}} || From <- maps:values(Froms)],
-    persist({keep_state, Data#{votedFor => undefined, froms => #{}}, [restart_election_timeout() | Replies]});
+    persist({keep_state, maps:without([nextIndex, matchIndex], Data#{votedFor => undefined, froms => #{}}),
+        [restart_election_timeout() | Replies]});
 
 handle_event(enter, _OldState, candidate, Data) ->
     #{term := Term} = Data,
@@ -234,7 +231,7 @@ handle_event(cast, #{type := 'AppendEntriesResponse', term := PastTerm}, _State,
     couch_log:notice("~p ignored AppendEntriesResponse from past term ~B", [node(), PastTerm]),
     keep_state_and_data;
 
-handle_event(cast, #{type := 'AppendEntriesResponse', term := Term} = Msg, _State, #{term := Term} = Data) ->
+handle_event(cast, #{type := 'AppendEntriesResponse', term := Term} = Msg, leader, #{term := Term} = Data) ->
     #{success := MSuccess, matchIndex := MMatchIndex, source := MSource} = Msg,
     #{nextIndex := NextIndex, matchIndex := MatchIndex} = Data,
     couch_log:debug("~p received AppendEntriesResponse from ~p in current term ~B (Success:~p)", [node(), MSource, Term, MSuccess]),
@@ -251,6 +248,9 @@ handle_event(cast, #{type := 'AppendEntriesResponse', term := Term} = Msg, _Stat
             }}
     end;
 
+handle_event(cast, #{type := 'AppendEntriesResponse'}, _State, _Data) ->
+    keep_state_and_data;
+
 handle_event({call, From}, #{type := 'ClientRequest'} = Msg, leader, Data) ->
     #{value := Value} = Msg,
     #{term := Term, store_module := StoreModule, froms := Froms} = Data,