From 987cfd23c51d8d776c76f0fc41c91bb336a515ea Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 8 Nov 2022 16:21:38 -0500 Subject: [PATCH 01/87] prevote initial implementation --- commands.go | 4 +++ raft.go | 85 +++++++++++++++++++++++++++++++++++++++++----------- raft_test.go | 1 - 3 files changed, 71 insertions(+), 19 deletions(-) diff --git a/commands.go b/commands.go index 2ddd418d2..399b43207 100644 --- a/commands.go +++ b/commands.go @@ -89,6 +89,7 @@ type RequestVoteRequest struct { // transfer. It is required for leadership transfer to work, because servers // wouldn't vote otherwise if they are aware of an existing leader. LeadershipTransfer bool + PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -110,6 +111,9 @@ type RequestVoteResponse struct { // Is the vote granted. Granted bool + + // Is it a preVote response + PreVote bool } // GetRPCHeader - See WithRPCHeader. diff --git a/raft.go b/raft.go index 5ad23a04b..374b9912e 100644 --- a/raft.go +++ b/raft.go @@ -284,7 +284,7 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - voteCh := r.electSelf() + prevoteCh := r.electSelf(true) // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -298,9 +298,10 @@ func (r *Raft) runCandidate() { // Tally the votes, need a simple majority grantedVotes := 0 + preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - + var voteCh <-chan *voteResult for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -309,6 +310,43 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.processRPC(rpc) + case vote := <-prevoteCh: + r.mainThreadSaturation.working() + r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + // Check if the term is greater than ours, bail + if vote.Term > r.getCurrentTerm() && !vote.PreVote { + r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } + + // Check if the vote is granted + if vote.Granted { + if !vote.PreVote { + grantedVotes++ + r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + } else { + preVoteGrantedVotes++ + r.logger.Debug("prevote granted", "from", vote.voterID, "term", vote.Term, "tally", preVoteGrantedVotes) + } + } + + // Check if we've become the leader + if grantedVotes >= votesNeeded { + r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) + r.setState(Leader) + r.setLeader(r.localAddr, r.localID) + //r.setCurrentTerm(term) + return + } + // Check if we've become the leader + if preVoteGrantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + preVoteGrantedVotes = 0 + grantedVotes = 0 + voteCh = r.electSelf(false) + } case vote := <-voteCh: r.mainThreadSaturation.working() // Check if the term is greater than ours, bail @@ -332,7 +370,6 @@ func (r *Raft) runCandidate() { r.setLeader(r.localAddr, r.localID) return } - case c := <-r.configurationChangeCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader @@ -1534,6 +1571,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, + PreVote: req.PreVote, } var rpcErr error defer func() { @@ -1589,8 +1627,10 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - r.setState(Follower) - r.setCurrentTerm(req.Term) + if !req.PreVote { + r.setState(Follower) + r.setCurrentTerm(req.Term) + } resp.Term = req.Term } @@ -1621,7 +1661,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { // Check if we've voted in this election before if lastVoteTerm == req.Term && lastVoteCandBytes != nil { r.logger.Info("duplicate requestVote for same term", "term", req.Term) - if bytes.Compare(lastVoteCandBytes, candidateBytes) == 0 { + if bytes.Equal(lastVoteCandBytes, candidateBytes) { r.logger.Warn("duplicate requestVote from", "candidate", candidate) resp.Granted = true } @@ -1647,14 +1687,14 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if err := r.persistVote(req.Term, candidateBytes); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return + if !req.PreVote { + if err := r.persistVote(req.Term, candidateBytes); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return + } } - resp.Granted = true r.setLastContact() - return } // installSnapshot is invoked when we get a InstallSnapshot RPC call. @@ -1814,23 +1854,26 @@ type voteResult struct { // ourself. This has the side affecting of incrementing the current term. The // response channel returned is used to wait for all the responses (including a // vote for ourself). This must only be called from the main thread. -func (r *Raft) electSelf() <-chan *voteResult { +func (r *Raft) electSelf(preVote bool) <-chan *voteResult { // Create a response channel respCh := make(chan *voteResult, len(r.configurations.latest.Servers)) // Increment the term - r.setCurrentTerm(r.getCurrentTerm() + 1) - + newTerm := r.getCurrentTerm() + 1 + if !preVote { + r.setCurrentTerm(newTerm) + } // Construct the request lastIdx, lastTerm := r.getLastEntry() req := &RequestVoteRequest{ RPCHeader: r.getRPCHeader(), - Term: r.getCurrentTerm(), + Term: newTerm, // this is needed for retro compatibility, before RPCHeader.Addr was added Candidate: r.trans.EncodePeer(r.localID, r.localAddr), LastLogIndex: lastIdx, LastLogTerm: lastTerm, LeadershipTransfer: r.candidateFromLeadershipTransfer, + PreVote: preVote, } // Construct a function to ask for a vote @@ -1846,7 +1889,9 @@ func (r *Raft) electSelf() <-chan *voteResult { "term", req.Term) resp.Term = req.Term resp.Granted = false + resp.PreVote = req.PreVote } + r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1857,9 +1902,11 @@ func (r *Raft) electSelf() <-chan *voteResult { if server.ID == r.localID { r.logger.Debug("voting for self", "term", req.Term, "id", r.localID) // Persist a vote for ourselves - if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return nil + if !preVote { + if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return nil + } } // Include our own vote respCh <- &voteResult{ @@ -1867,6 +1914,7 @@ func (r *Raft) electSelf() <-chan *voteResult { RPCHeader: r.getRPCHeader(), Term: req.Term, Granted: true, + PreVote: req.PreVote, }, voterID: r.localID, } @@ -1893,6 +1941,7 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { + r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) diff --git a/raft_test.go b/raft_test.go index 84234e7f1..653eb9167 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2830,7 +2830,6 @@ func TestRaft_VoteWithNoIDNoAddr(t *testing.T) { var resp RequestVoteResponse followerT := c.trans[c.IndexOf(followers[1])] c.Partition([]ServerAddress{leader.localAddr}) - time.Sleep(c.propagateTimeout) // wait for the remaining follower to trigger an election waitForState(follower, Candidate) From f1ed619ff061ec3ad858be62e5bd05eba4cb5a42 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 14:30:30 -0500 Subject: [PATCH 02/87] add config and relevant tests --- api.go | 4 ++ config.go | 127 ++++++++++++++++++++++++++++----------------------- raft.go | 21 ++++++--- raft_test.go | 56 +++++++++++++++++++++++ 4 files changed, 145 insertions(+), 63 deletions(-) diff --git a/api.go b/api.go index eda33c3f0..9af4a1ff2 100644 --- a/api.go +++ b/api.go @@ -210,6 +210,9 @@ type Raft struct { // mainThreadSaturation measures the saturation of the main raft goroutine. mainThreadSaturation *saturationMetric + + // preVote control if the pre-vote feature is activated + preVote bool } // BootstrapCluster initializes a server's storage with the given cluster @@ -557,6 +560,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna leaderNotifyCh: make(chan struct{}, 1), followerNotifyCh: make(chan struct{}, 1), mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), + preVote: conf.PreVote, } r.conf.Store(*conf) diff --git a/config.go b/config.go index 8df4ae74e..fdc72b09a 100644 --- a/config.go +++ b/config.go @@ -25,67 +25,74 @@ import ( // The version details are complicated, but here's a summary of what's required // to get from a version 0 cluster to version 3: // -// 1. In version N of your app that starts using the new Raft library with -// versioning, set ProtocolVersion to 1. -// 2. Make version N+1 of your app require version N as a prerequisite (all -// servers must be upgraded). For version N+1 of your app set ProtocolVersion -// to 2. -// 3. Similarly, make version N+2 of your app require version N+1 as a -// prerequisite. For version N+2 of your app, set ProtocolVersion to 3. +// 1. In version N of your app that starts using the new Raft library with +// versioning, set ProtocolVersion to 1. +// 2. Make version N+1 of your app require version N as a prerequisite (all +// servers must be upgraded). For version N+1 of your app set ProtocolVersion +// to 2. +// 3. Similarly, make version N+2 of your app require version N+1 as a +// prerequisite. For version N+2 of your app, set ProtocolVersion to 3. // // During this upgrade, older cluster members will still have Server IDs equal // to their network addresses. To upgrade an older member and give it an ID, it // needs to leave the cluster and re-enter: // -// 1. Remove the server from the cluster with RemoveServer, using its network -// address as its ServerID. -// 2. Update the server's config to use a UUID or something else that is -// not tied to the machine as the ServerID (restarting the server). -// 3. Add the server back to the cluster with AddVoter, using its new ID. +// 1. Remove the server from the cluster with RemoveServer, using its network +// address as its ServerID. +// 2. Update the server's config to use a UUID or something else that is +// not tied to the machine as the ServerID (restarting the server). +// 3. Add the server back to the cluster with AddVoter, using its new ID. // // You can do this during the rolling upgrade from N+1 to N+2 of your app, or // as a rolling change at any time after the upgrade. // -// Version History +// # Version History // // 0: Original Raft library before versioning was added. Servers running this -// version of the Raft library use AddPeerDeprecated/RemovePeerDeprecated -// for all configuration changes, and have no support for LogConfiguration. +// +// version of the Raft library use AddPeerDeprecated/RemovePeerDeprecated +// for all configuration changes, and have no support for LogConfiguration. +// // 1: First versioned protocol, used to interoperate with old servers, and begin -// the migration path to newer versions of the protocol. Under this version -// all configuration changes are propagated using the now-deprecated -// RemovePeerDeprecated Raft log entry. This means that server IDs are always -// set to be the same as the server addresses (since the old log entry type -// cannot transmit an ID), and only AddPeer/RemovePeer APIs are supported. -// Servers running this version of the protocol can understand the new -// LogConfiguration Raft log entry but will never generate one so they can -// remain compatible with version 0 Raft servers in the cluster. +// +// the migration path to newer versions of the protocol. Under this version +// all configuration changes are propagated using the now-deprecated +// RemovePeerDeprecated Raft log entry. This means that server IDs are always +// set to be the same as the server addresses (since the old log entry type +// cannot transmit an ID), and only AddPeer/RemovePeer APIs are supported. +// Servers running this version of the protocol can understand the new +// LogConfiguration Raft log entry but will never generate one so they can +// remain compatible with version 0 Raft servers in the cluster. +// // 2: Transitional protocol used when migrating an existing cluster to the new -// server ID system. Server IDs are still set to be the same as server -// addresses, but all configuration changes are propagated using the new -// LogConfiguration Raft log entry type, which can carry full ID information. -// This version supports the old AddPeer/RemovePeer APIs as well as the new -// ID-based AddVoter/RemoveServer APIs which should be used when adding -// version 3 servers to the cluster later. This version sheds all -// interoperability with version 0 servers, but can interoperate with newer -// Raft servers running with protocol version 1 since they can understand the -// new LogConfiguration Raft log entry, and this version can still understand -// their RemovePeerDeprecated Raft log entries. We need this protocol version -// as an intermediate step between 1 and 3 so that servers will propagate the -// ID information that will come from newly-added (or -rolled) servers using -// protocol version 3, but since they are still using their address-based IDs -// from the previous step they will still be able to track commitments and -// their own voting status properly. If we skipped this step, servers would -// be started with their new IDs, but they wouldn't see themselves in the old -// address-based configuration, so none of the servers would think they had a -// vote. +// +// server ID system. Server IDs are still set to be the same as server +// addresses, but all configuration changes are propagated using the new +// LogConfiguration Raft log entry type, which can carry full ID information. +// This version supports the old AddPeer/RemovePeer APIs as well as the new +// ID-based AddVoter/RemoveServer APIs which should be used when adding +// version 3 servers to the cluster later. This version sheds all +// interoperability with version 0 servers, but can interoperate with newer +// Raft servers running with protocol version 1 since they can understand the +// new LogConfiguration Raft log entry, and this version can still understand +// their RemovePeerDeprecated Raft log entries. We need this protocol version +// as an intermediate step between 1 and 3 so that servers will propagate the +// ID information that will come from newly-added (or -rolled) servers using +// protocol version 3, but since they are still using their address-based IDs +// from the previous step they will still be able to track commitments and +// their own voting status properly. If we skipped this step, servers would +// be started with their new IDs, but they wouldn't see themselves in the old +// address-based configuration, so none of the servers would think they had a +// vote. +// // 3: Protocol adding full support for server IDs and new ID-based server APIs -// (AddVoter, AddNonvoter, etc.), old AddPeer/RemovePeer APIs are no longer -// supported. Version 2 servers should be swapped out by removing them from -// the cluster one-by-one and re-adding them with updated configuration for -// this protocol version, along with their server ID. The remove/add cycle -// is required to populate their server ID. Note that removing must be done -// by ID, which will be the old server's address. +// +// (AddVoter, AddNonvoter, etc.), old AddPeer/RemovePeer APIs are no longer +// supported. Version 2 servers should be swapped out by removing them from +// the cluster one-by-one and re-adding them with updated configuration for +// this protocol version, along with their server ID. The remove/add cycle +// is required to populate their server ID. Note that removing must be done +// by ID, which will be the old server's address. type ProtocolVersion int const ( @@ -99,19 +106,22 @@ const ( // Currently, it is always assumed that the server generates the latest version, // though this may be changed in the future to include a configurable version. // -// Version History +// # Version History // // 0: Original Raft library before versioning was added. The peers portion of -// these snapshots is encoded in the legacy format which requires decodePeers -// to parse. This version of snapshots should only be produced by the -// unversioned Raft library. +// +// these snapshots is encoded in the legacy format which requires decodePeers +// to parse. This version of snapshots should only be produced by the +// unversioned Raft library. +// // 1: New format which adds support for a full configuration structure and its -// associated log index, with support for server IDs and non-voting server -// modes. To ease upgrades, this also includes the legacy peers structure but -// that will never be used by servers that understand version 1 snapshots. -// Since the original Raft library didn't enforce any versioning, we must -// include the legacy peers structure for this version, but we can deprecate -// it in the next snapshot version. +// +// associated log index, with support for server IDs and non-voting server +// modes. To ease upgrades, this also includes the legacy peers structure but +// that will never be used by servers that understand version 1 snapshots. +// Since the original Raft library didn't enforce any versioning, we must +// include the legacy peers structure for this version, but we can deprecate +// it in the next snapshot version. type SnapshotVersion int const ( @@ -221,6 +231,9 @@ type Config struct { // skipStartup allows NewRaft() to bypass all background work goroutines skipStartup bool + + // PreVote activate the pre-vote feature + PreVote bool } func (conf *Config) getOrCreateLogger() hclog.Logger { diff --git a/raft.go b/raft.go index 374b9912e..0b93505e9 100644 --- a/raft.go +++ b/raft.go @@ -284,7 +284,13 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - prevoteCh := r.electSelf(true) + var voteCh <-chan *voteResult + var prevoteCh <-chan *voteResult + if r.preVote { + prevoteCh = r.electSelf(true) + } else { + voteCh = r.electSelf(false) + } // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -301,7 +307,7 @@ func (r *Raft) runCandidate() { preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - var voteCh <-chan *voteResult + for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -1565,13 +1571,16 @@ func (r *Raft) processConfigurationLogEntry(entry *Log) error { func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) r.observe(*req) - + preVote := req.PreVote + if !r.preVote { + preVote = false + } // Setup a response resp := &RequestVoteResponse{ RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: req.PreVote, + PreVote: preVote, } var rpcErr error defer func() { @@ -1627,7 +1636,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !req.PreVote { + if !preVote { r.setState(Follower) r.setCurrentTerm(req.Term) } @@ -1687,7 +1696,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if !req.PreVote { + if !preVote { if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return diff --git a/raft_test.go b/raft_test.go index 653eb9167..4498c4b3e 100644 --- a/raft_test.go +++ b/raft_test.go @@ -1916,6 +1916,62 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } +func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { + + tcs := []struct { + name string + prevoteNum int + noprevoteNum int + }{ + {"majority no pre-vote", 2, 3}, + {"majority pre-vote", 3, 2}, + {"all pre-vote", 3, 0}, + {"all no pre-vote", 0, 3}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + + // Make majority cluster. + majority := tc.prevoteNum + minority := tc.noprevoteNum + if tc.prevoteNum < tc.noprevoteNum { + majority = tc.noprevoteNum + minority = tc.prevoteNum + } + + conf := inmemConfig(t) + conf.PreVote = tc.prevoteNum > tc.noprevoteNum + c := MakeCluster(majority, t, conf) + defer c.Close() + + // Set up another server speaking protocol version 2. + conf = inmemConfig(t) + conf.PreVote = tc.prevoteNum < tc.noprevoteNum + c1 := MakeClusterNoBootstrap(minority, t, conf) + + // Merge clusters. + c.Merge(c1) + c.FullyConnect() + + if len(c1.rafts) > 0 { + future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + if err := future.Error(); err != nil { + t.Fatalf("err: %v", err) + } + } + time.Sleep(c.propagateTimeout * 10) + + leaderOld := c.Leader() + c.Followers() + c.Partition([]ServerAddress{leaderOld.localAddr}) + time.Sleep(c.propagateTimeout * 3) + leader := c.Leader() + require.NotEqual(t, leader.leaderID, leaderOld.leaderID) + }) + } + +} + func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { conf := inmemConfig(t) conf.ProtocolVersion = 3 From c57bed86557bc5ad9e7eecbcb85703916a86edc5 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 16:11:36 -0500 Subject: [PATCH 03/87] remove extra comments, fix a case where newer term is discovered for prevote --- raft.go | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/raft.go b/raft.go index 0b93505e9..1e374d218 100644 --- a/raft.go +++ b/raft.go @@ -315,8 +315,9 @@ func (r *Raft) runCandidate() { case rpc := <-r.rpcCh: r.mainThreadSaturation.working() r.processRPC(rpc) - case vote := <-prevoteCh: + // This a pre-vote case but could lead to winning an election, in the case that majority of other nodes + // don't support pre-vote or have pre-vote deactivated. r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail @@ -326,6 +327,12 @@ func (r *Raft) runCandidate() { r.setCurrentTerm(vote.Term) return } + if vote.Term > term && vote.PreVote { + r.logger.Debug("newer term discovered on pre-vote, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } // Check if the vote is granted if vote.Granted { @@ -339,14 +346,17 @@ func (r *Raft) runCandidate() { } // Check if we've become the leader + // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) + // set our state to leader and our term to the pre-vote term. if grantedVotes >= votesNeeded { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) - //r.setCurrentTerm(term) + + r.setCurrentTerm(term) return } - // Check if we've become the leader + // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 @@ -1900,7 +1910,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { resp.Granted = false resp.PreVote = req.PreVote } - r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1950,7 +1959,6 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { - r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) From 3082935ce714797a16a0516c194c72675640266d Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 10 Nov 2022 11:23:51 -0500 Subject: [PATCH 04/87] fix to reset timeout after pre-vote and fix split vote (pre-vote,vote) case. --- raft.go | 5 ++++- raft_test.go | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index 1e374d218..e889bfcd2 100644 --- a/raft.go +++ b/raft.go @@ -348,7 +348,9 @@ func (r *Raft) runCandidate() { // Check if we've become the leader // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) // set our state to leader and our term to the pre-vote term. - if grantedVotes >= votesNeeded { + // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 + // we can flip our vote to an actual vote. + if grantedVotes >= votesNeeded-1 { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) @@ -361,6 +363,7 @@ func (r *Raft) runCandidate() { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 grantedVotes = 0 + electionTimer = randomTimeout(electionTimeout) voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index 4498c4b3e..4e2c68570 100644 --- a/raft_test.go +++ b/raft_test.go @@ -1916,7 +1916,7 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } -func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { +func TestRaft_PreVoteMixedCluster(t *testing.T) { tcs := []struct { name string @@ -1925,6 +1925,8 @@ func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { }{ {"majority no pre-vote", 2, 3}, {"majority pre-vote", 3, 2}, + {"majority no pre-vote", 1, 2}, + {"majority pre-vote", 2, 1}, {"all pre-vote", 3, 0}, {"all no pre-vote", 0, 3}, } From 2c44c94b2dcad41c42401d7f98a2cdad2753d9e7 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 14 Nov 2022 13:59:32 -0500 Subject: [PATCH 05/87] fix a case where granted votes and prevotes don't reach quorum but the sum can reach quorum --- raft.go | 5 +++-- raft_test.go | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/raft.go b/raft.go index e889bfcd2..5f58b80b4 100644 --- a/raft.go +++ b/raft.go @@ -359,11 +359,12 @@ func (r *Raft) runCandidate() { return } // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + if preVoteGrantedVotes+grantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 grantedVotes = 0 electionTimer = randomTimeout(electionTimeout) + prevoteCh = nil voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index 4e2c68570..7e23820eb 100644 --- a/raft_test.go +++ b/raft_test.go @@ -1955,8 +1955,8 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { c.Merge(c1) c.FullyConnect() - if len(c1.rafts) > 0 { - future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + for _, r := range c1.rafts { + future := c.Leader().AddVoter(r.localID, r.localAddr, 0, 0) if err := future.Error(); err != nil { t.Fatalf("err: %v", err) } From baf623c65b9988234cd83fa3ad0080d159c242a5 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 6 Jun 2023 09:48:32 -0400 Subject: [PATCH 06/87] add submodule and first iteration of multi-version tests rename test and pin submodule to version 1.5.0 rename test file --- .gitmodules | 3 + integ_test.go | 2 +- raft-compat/go.mod | 29 +++++++ raft-compat/go.sum | 128 ++++++++++++++++++++++++++++ raft-compat/raftrs | 1 + raft-compat/rolling_upgrade_test.go | 87 +++++++++++++++++++ raft_test.go | 62 +++++++------- testing.go | 37 +++++++- 8 files changed, 315 insertions(+), 34 deletions(-) create mode 100644 .gitmodules create mode 100644 raft-compat/go.mod create mode 100644 raft-compat/go.sum create mode 160000 raft-compat/raftrs create mode 100644 raft-compat/rolling_upgrade_test.go diff --git a/.gitmodules b/.gitmodules new file mode 100644 index 000000000..ed959a7cf --- /dev/null +++ b/.gitmodules @@ -0,0 +1,3 @@ +[submodule "raft-compat/raftrs"] + path = raft-compat/raftrs + url = git@github.com:hashicorp/raft diff --git a/integ_test.go b/integ_test.go index 905663496..a39d21535 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,7 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(t *testing.T, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { - conf = inmemConfig(t) + conf = InmemConfig(t) } dir, err := os.MkdirTemp("", "raft") diff --git a/raft-compat/go.mod b/raft-compat/go.mod new file mode 100644 index 000000000..40db0858b --- /dev/null +++ b/raft-compat/go.mod @@ -0,0 +1,29 @@ +module github.com/hashicorp/raft/compat + +go 1.20 + +require github.com/stretchr/testify v1.8.4 + +require ( + github.com/armon/go-metrics v0.4.1 // indirect + github.com/fatih/color v1.13.0 // indirect + github.com/hashicorp/go-hclog v1.5.0 // indirect + github.com/hashicorp/go-immutable-radix v1.0.0 // indirect + github.com/hashicorp/go-msgpack v0.5.5 // indirect + github.com/hashicorp/golang-lru v0.5.0 // indirect + github.com/mattn/go-colorable v0.1.12 // indirect + github.com/mattn/go-isatty v0.0.14 // indirect + golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect +) + +replace github.com/dhiayachi/raft v1.0.0 => ./raftrs + +replace github.com/hashicorp/raft => ../ + +require ( + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dhiayachi/raft v1.0.0 + github.com/hashicorp/raft v1.2.0 + github.com/pmezard/go-difflib v1.0.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) diff --git a/raft-compat/go.sum b/raft-compat/go.sum new file mode 100644 index 000000000..ed3b7f040 --- /dev/null +++ b/raft-compat/go.sum @@ -0,0 +1,128 @@ +github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA= +github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= +github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= +github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= +github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= +github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= +github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= +github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= +github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= +github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= +github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY= +github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= +github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 h1:nonptSpoQ4vQjyraW20DXPAglgQfVnM9ZC6MmNLMR60= +golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/raft-compat/raftrs b/raft-compat/raftrs new file mode 160000 index 000000000..8fdc4ce5b --- /dev/null +++ b/raft-compat/raftrs @@ -0,0 +1 @@ +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go new file mode 100644 index 000000000..65c511e61 --- /dev/null +++ b/raft-compat/rolling_upgrade_test.go @@ -0,0 +1,87 @@ +package raft_compat + +import ( + "fmt" + raftrs "github.com/dhiayachi/raft" + "github.com/hashicorp/raft" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +func TestRaft_RollingUpgrade_NoLeaderUpgrade(t *testing.T) { + + initCount := 3 + rafts := make([]*raft.Raft, initCount) + trans := make([]*raft.NetworkTransport, initCount) + confs := make([]*raft.Config, initCount) + stores := make([]*raft.InmemStore, initCount) + snaps := make([]*raft.InmemSnapshotStore, initCount) + id := make([]raft.ServerID, initCount) + configuration := raft.Configuration{} + + for i := 0; i < initCount; i++ { + confs[i] = raft.InmemConfig(t) + id[i] = raft.ServerID(fmt.Sprintf("grpc%d", i)) + confs[i].LocalID = id[i] + stores[i] = raft.NewInmemStore() + snaps[i] = raft.NewInmemSnapshotStore() + var err error + trans[i], err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + rafts[i], err = raft.NewRaft(confs[i], &raft.MockFSM{}, stores[i], stores[i], snaps[i], trans[i]) + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raft.Server{ + ID: id[i], + Address: trans[i].LocalAddr(), + }) + } + boot := rafts[0].BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + lCh := rafts[0].LeaderCh() + + after := time.After(time.Second) + + select { + case <-after: + t.Fatalf("timedout") + case <-lCh: + } + a := rafts[0].Leader() + require.NotEmpty(t, a) + future := rafts[0].Apply([]byte("test"), time.Second) + require.NoError(t, future.Error()) + + newRafts := make([]*raftrs.Raft, initCount) + newTrans := make([]*raftrs.NetworkTransport, initCount) + newConfs := make([]*raftrs.Config, initCount) + newStores := make([]*raftrs.InmemStore, initCount) + newSnaps := make([]*raftrs.InmemSnapshotStore, initCount) + leader, _ := rafts[0].LeaderWithID() + require.NotEmpty(t, leader) + for i := 1; i < len(rafts); i++ { + newConfs[i] = raftrs.DefaultConfig() + newConfs[i].LocalID = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) + newStores[i] = raftrs.NewInmemStore() + newSnaps[i] = raftrs.NewInmemSnapshotStore() + var err error + newTrans[i], err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + newRafts[i], err = raftrs.NewRaft(newConfs[i], &raftrs.MockFSM{}, newStores[i], newStores[i], newSnaps[i], newTrans[i]) + rafts[0].AddVoter("grpc4", raft.ServerAddress(newTrans[i].LocalAddr()), 0, 0) + + time.Sleep(1 * time.Second) + + // Check Leader haven't changed as we are not replacing the leader + a := rafts[0].Leader() + require.Equal(t, a, leader) + rafts[0].RemoveServer(id[i], 0, 0) + rafts[i].Shutdown() + + } + future = rafts[0].Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + +} diff --git a/raft_test.go b/raft_test.go index df9712b1c..21f2af63a 100644 --- a/raft_test.go +++ b/raft_test.go @@ -153,7 +153,7 @@ func TestRaft_RecoverCluster(t *testing.T) { snapshotThreshold := 5 runRecover := func(t *testing.T, applies int) { var err error - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 conf.SnapshotThreshold = uint64(snapshotThreshold) c := MakeCluster(3, t, conf) @@ -296,7 +296,7 @@ func TestRaft_HasExistingState(t *testing.T) { } func TestRaft_SingleNode(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -502,7 +502,7 @@ func TestRaft_ApplyNonLeader(t *testing.T) { func TestRaft_ApplyConcurrent(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout c := MakeCluster(3, t, conf) @@ -553,7 +553,7 @@ func TestRaft_ApplyConcurrent(t *testing.T) { func TestRaft_ApplyConcurrent_Timeout(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.CommitTimeout = 1 * time.Millisecond conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout @@ -621,7 +621,7 @@ func TestRaft_JoinNode(t *testing.T) { func TestRaft_JoinNode_ConfigStore(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := makeCluster(t, &MakeClusterOpts{ Peers: 1, Bootstrap: true, @@ -779,7 +779,7 @@ func TestRaft_RemoveLeader(t *testing.T) { func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -842,7 +842,7 @@ func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { func TestRaft_RemoveFollower_SplitCluster(t *testing.T) { // Make a cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(4, t, conf) defer c.Close() @@ -957,7 +957,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { func TestRaft_SnapshotRestore(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1017,7 +1017,7 @@ func TestRaft_SnapshotRestore(t *testing.T) { func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 opts := &MakeClusterOpts{ Peers: 1, @@ -1099,7 +1099,7 @@ func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { func TestRaft_SnapshotRestore_Progress(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1217,7 +1217,7 @@ func (b *lockedBytesBuffer) String() string { // up. func TestRaft_NoRestoreOnStart(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 conf.NoSnapshotRestoreOnStart = true c := MakeCluster(1, t, conf) @@ -1262,7 +1262,7 @@ func TestRaft_NoRestoreOnStart(t *testing.T) { func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { var err error // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 1 conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) @@ -1367,7 +1367,7 @@ func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { func TestRaft_AutoSnapshot(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.SnapshotInterval = conf.CommitTimeout * 2 conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 @@ -1397,7 +1397,7 @@ func TestRaft_AutoSnapshot(t *testing.T) { func TestRaft_UserSnapshot(t *testing.T) { // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) @@ -1437,7 +1437,7 @@ func snapshotAndRestore(t *testing.T, offset uint64, monotonicLogStore bool, res t.Helper() // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) // snapshot operations perform some file IO operations. // increase times out to account for that @@ -1599,7 +1599,7 @@ func TestRaft_UserRestore(t *testing.T) { func TestRaft_SendSnapshotFollower(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1641,7 +1641,7 @@ func TestRaft_SendSnapshotFollower(t *testing.T) { func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1695,7 +1695,7 @@ func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { func TestRaft_ReJoinFollower(t *testing.T) { // Enable operation after a remove. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -1772,7 +1772,7 @@ func TestRaft_ReJoinFollower(t *testing.T) { func TestRaft_LeaderLeaseExpire(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1901,7 +1901,7 @@ func TestRaft_VerifyLeader_Single(t *testing.T) { func TestRaft_VerifyLeader_Fail(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1941,7 +1941,7 @@ func TestRaft_VerifyLeader_Fail(t *testing.T) { func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() @@ -1975,7 +1975,7 @@ func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { func TestRaft_NotifyCh(t *testing.T) { ch := make(chan bool, 1) - conf := inmemConfig(t) + conf := InmemConfig(t) conf.NotifyCh = ch c := MakeCluster(1, t, conf) defer c.Close() @@ -2064,7 +2064,7 @@ func TestRaft_AppendEntry(t *testing.T) { } func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 3 c := MakeCluster(3, t, conf) defer c.Close() @@ -2144,13 +2144,13 @@ func TestRaft_ProtocolVersion_RejectRPC(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { // Make a cluster back on protocol version 1. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 1 c := MakeCluster(2, t, conf) defer c.Close() // Set up another server speaking protocol version 2. - conf = inmemConfig(t) + conf = InmemConfig(t) conf.ProtocolVersion = 2 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2187,14 +2187,14 @@ func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { // Make a cluster back on protocol version 2. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 2 c := MakeCluster(2, t, conf) defer c.Close() oldAddr := c.Followers()[0].localAddr // Set up another server speaking protocol version 3. - conf = inmemConfig(t) + conf = InmemConfig(t) conf.ProtocolVersion = 3 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2221,7 +2221,7 @@ func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { func TestRaft_LeaderID_Propagated(t *testing.T) { // Make a cluster on protocol version 3. - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() err := waitForLeader(c) @@ -2648,7 +2648,7 @@ func TestRaft_CacheLogWithStoreError(t *testing.T) { } func TestRaft_ReloadConfig(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.LeaderLeaseTimeout = 40 * time.Millisecond c := MakeCluster(1, t, conf) defer c.Close() @@ -2679,7 +2679,7 @@ func TestRaft_ReloadConfig(t *testing.T) { } func TestRaft_ReloadConfigValidates(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -2921,7 +2921,7 @@ func TestRaft_ClusterCanRegainStability_WhenNonVoterWithHigherTermJoin(t *testin // started when a standby is shut down and restarted. func TestRaft_FollowerRemovalNoElection(t *testing.T) { // Make a cluster - inmemConf := inmemConfig(t) + inmemConf := InmemConfig(t) inmemConf.HeartbeatTimeout = 100 * time.Millisecond inmemConf.ElectionTimeout = 100 * time.Millisecond c := MakeCluster(3, t, inmemConf) diff --git a/testing.go b/testing.go index fd770d423..e3859e1e2 100644 --- a/testing.go +++ b/testing.go @@ -23,7 +23,7 @@ var ( ) // Return configurations optimized for in-memory -func inmemConfig(t *testing.T) *Config { +func InmemConfig(t *testing.T) *Config { conf := DefaultConfig() conf.HeartbeatTimeout = 50 * time.Millisecond conf.ElectionTimeout = 50 * time.Millisecond @@ -222,6 +222,39 @@ func newTestLoggerWithPrefix(t *testing.T, prefix string) hclog.Logger { }) } +func (c *cluster) NodeID(index int) ServerID { + if index >= len(c.rafts) { + return "" + } + return c.rafts[index].localID +} + +// ConnectExternal connects all the transports together. +func (c *cluster) ConnectExternal(a ServerAddress) { + c.logger.Debug("fully connecting") + for _, t := range c.trans { + t.Connect(a, t) + } +} + +func (c *cluster) Transport(index int) LoopbackTransport { + if index >= len(c.trans) { + return nil + } + return c.trans[index] +} + +func (c *cluster) NodeAddress(index int) ServerAddress { + if index >= len(c.rafts) { + return "" + } + return c.rafts[index].localAddr +} + +func (c *cluster) RaftsLen() int { + return len(c.rafts) +} + type cluster struct { dirs []string stores []*InmemStore @@ -727,7 +760,7 @@ type MakeClusterOpts struct { // each other. func makeCluster(t *testing.T, opts *MakeClusterOpts) *cluster { if opts.Conf == nil { - opts.Conf = inmemConfig(t) + opts.Conf = InmemConfig(t) } c := &cluster{ From 22b88e4bfd0b786dbb973706ace27cf023f267fd Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 7 Jun 2023 15:13:48 -0400 Subject: [PATCH 07/87] refactor test --- raft-compat/rolling_upgrade_test.go | 183 +++++++++++++++++++++------- 1 file changed, 141 insertions(+), 42 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 65c511e61..01d54c0a2 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -9,79 +9,178 @@ import ( "time" ) -func TestRaft_RollingUpgrade_NoLeaderUpgrade(t *testing.T) { +type raftUIT struct { + raft *raft.Raft + trans *raft.NetworkTransport + Config *raft.Config + Store *raft.InmemStore + Snap *raft.InmemSnapshotStore + id raft.ServerID + fsm *raft.MockFSM +} + +func (r raftUIT) getLocalID() string { + return string(r.id) +} + +func (r raftUIT) getLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +type raftLatest struct { + raft *raftrs.Raft + trans *raftrs.NetworkTransport + Config *raftrs.Config + Store *raftrs.InmemStore + Snap *raftrs.InmemSnapshotStore + id raftrs.ServerID + fsm *raftrs.MockFSM +} + +func (r raftLatest) getLocalID() string { + return string(r.id) +} + +func (r raftLatest) getLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +type raftNode interface { + getLocalID() string + getLeaderID() string +} + +type raftCluster[T raftNode] struct { + rafts []T +} + +func newRaftCluster[T raftNode](count int) raftCluster[T] { + rc := raftCluster[T]{} + rc.rafts = make([]T, count) + return rc +} + +func (r *raftCluster[T]) getLeader() T { + var empty T + for _, n := range r.rafts { + if n.getLocalID() == n.getLeaderID() { + return n + } + } + return empty +} + +func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rafts := make([]*raft.Raft, initCount) - trans := make([]*raft.NetworkTransport, initCount) - confs := make([]*raft.Config, initCount) - stores := make([]*raft.InmemStore, initCount) - snaps := make([]*raft.InmemSnapshotStore, initCount) - id := make([]raft.ServerID, initCount) - configuration := raft.Configuration{} + rLatest := newRaftCluster[raftLatest](initCount) + configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { - confs[i] = raft.InmemConfig(t) - id[i] = raft.ServerID(fmt.Sprintf("grpc%d", i)) - confs[i].LocalID = id[i] - stores[i] = raft.NewInmemStore() - snaps[i] = raft.NewInmemSnapshotStore() + rLatest.rafts[i].Config = raftrs.DefaultConfig() + rLatest.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.CommitTimeout = 5 * time.Millisecond + rLatest.rafts[i].id = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) + rLatest.rafts[i].Config.LocalID = rLatest.rafts[i].id + rLatest.rafts[i].Store = raftrs.NewInmemStore() + rLatest.rafts[i].Snap = raftrs.NewInmemSnapshotStore() + rLatest.rafts[i].fsm = &raftrs.MockFSM{} var err error - trans[i], err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - rafts[i], err = raft.NewRaft(confs[i], &raft.MockFSM{}, stores[i], stores[i], snaps[i], trans[i]) + rLatest.rafts[i].raft, err = raftrs.NewRaft(rLatest.rafts[i].Config, rLatest.rafts[i].fsm, rLatest.rafts[i].Store, + rLatest.rafts[i].Store, rLatest.rafts[i].Snap, rLatest.rafts[i].trans) require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raft.Server{ - ID: id[i], - Address: trans[i].LocalAddr(), + configuration.Servers = append(configuration.Servers, raftrs.Server{ + ID: rLatest.rafts[i].id, + Address: rLatest.rafts[i].trans.LocalAddr(), }) } - boot := rafts[0].BootstrapCluster(configuration) + boot := rLatest.rafts[0].raft.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := rafts[0].LeaderCh() + lCh := rLatest.rafts[0].raft.LeaderCh() - after := time.After(time.Second) + after := time.After(5 * time.Second) select { case <-after: t.Fatalf("timedout") case <-lCh: } - a := rafts[0].Leader() + getLeader := rLatest.getLeader() + require.NotNil(t, getLeader) + a := getLeader.raft.Leader() require.NotEmpty(t, a) - future := rafts[0].Apply([]byte("test"), time.Second) + future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - newRafts := make([]*raftrs.Raft, initCount) - newTrans := make([]*raftrs.NetworkTransport, initCount) - newConfs := make([]*raftrs.Config, initCount) - newStores := make([]*raftrs.InmemStore, initCount) - newSnaps := make([]*raftrs.InmemSnapshotStore, initCount) - leader, _ := rafts[0].LeaderWithID() + rUIT := newRaftCluster[raftUIT](initCount) + leader, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, leader) - for i := 1; i < len(rafts); i++ { - newConfs[i] = raftrs.DefaultConfig() - newConfs[i].LocalID = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) - newStores[i] = raftrs.NewInmemStore() - newSnaps[i] = raftrs.NewInmemSnapshotStore() + + // Upgrade all the followers + leaderIdx := 0 + for i := 0; i < len(rLatest.rafts); i++ { + if getLeader.getLocalID() == rLatest.rafts[i].getLocalID() { + leaderIdx = i + continue + } + rUIT.rafts[i].Config = raft.DefaultConfig() + rUIT.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.CommitTimeout = 5 * time.Millisecond + rUIT.rafts[i].id = raft.ServerID(fmt.Sprintf("newGrpc%d", i)) + rUIT.rafts[i].Config.LocalID = rUIT.rafts[i].id + rUIT.rafts[i].Store = raft.NewInmemStore() + rUIT.rafts[i].Snap = raft.NewInmemSnapshotStore() + rUIT.rafts[i].fsm = &raft.MockFSM{} var err error - newTrans[i], err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - newRafts[i], err = raftrs.NewRaft(newConfs[i], &raftrs.MockFSM{}, newStores[i], newStores[i], newSnaps[i], newTrans[i]) - rafts[0].AddVoter("grpc4", raft.ServerAddress(newTrans[i].LocalAddr()), 0, 0) + rUIT.rafts[i].raft, err = raft.NewRaft(rUIT.rafts[i].Config, rUIT.rafts[i].fsm, rUIT.rafts[i].Store, + rUIT.rafts[i].Store, rUIT.rafts[i].Snap, rUIT.rafts[i].trans) + getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[i].getLocalID()), raftrs.ServerAddress(rUIT.rafts[i].trans.LocalAddr()), 0, 0) time.Sleep(1 * time.Second) // Check Leader haven't changed as we are not replacing the leader - a := rafts[0].Leader() + a := getLeader.raft.Leader() require.Equal(t, a, leader) - rafts[0].RemoveServer(id[i], 0, 0) - rafts[i].Shutdown() - + getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) + rLatest.rafts[i].raft.Shutdown() } - future = rafts[0].Apply([]byte("test2"), time.Second) + future = getLeader.raft.Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) + rUIT.rafts[leaderIdx].Config = raft.InmemConfig(t) + rUIT.rafts[leaderIdx].id = raft.ServerID(fmt.Sprintf("newGrpc%d", leaderIdx)) + rUIT.rafts[leaderIdx].Config.LocalID = rUIT.rafts[leaderIdx].id + rUIT.rafts[leaderIdx].Store = raft.NewInmemStore() + rUIT.rafts[leaderIdx].Snap = raft.NewInmemSnapshotStore() + rUIT.rafts[leaderIdx].fsm = &raft.MockFSM{} + var err error + rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + rUIT.rafts[leaderIdx].raft, err = raft.NewRaft(rUIT.rafts[leaderIdx].Config, rUIT.rafts[leaderIdx].fsm, rUIT.rafts[leaderIdx].Store, + rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) + getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) + // Check Leader haven't changed as we are not replacing the leader + a = getLeader.raft.Leader() + require.Equal(t, a, leader) + getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) + time.Sleep(1 * time.Second) + rLatest.rafts[leaderIdx].raft.Shutdown() + time.Sleep(1 * time.Second) + aNew := rUIT.getLeader().raft.Leader() + require.NotEqual(t, aNew, leader) + + require.Len(t, rUIT.getLeader().fsm.Logs(), 2) + } From 0882d3bb0af96456e12c083afdef11cfa44d3363 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 7 Jun 2023 16:03:28 -0400 Subject: [PATCH 08/87] clean up node init --- raft-compat/rolling_upgrade_test.go | 66 +++++++++++++++++++---------- 1 file changed, 43 insertions(+), 23 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 01d54c0a2..ed7cf1ba3 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -56,9 +56,12 @@ type raftCluster[T raftNode] struct { rafts []T } -func newRaftCluster[T raftNode](count int) raftCluster[T] { +func newRaftCluster[T raftNode](count int, name string) raftCluster[T] { rc := raftCluster[T]{} rc.rafts = make([]T, count) + for i := 0; i < count; i++ { + initNode(&rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) + } return rc } @@ -72,23 +75,50 @@ func (r *raftCluster[T]) getLeader() T { return empty } +func initNode(node interface{}, id string) { + switch node.(type) { + case *raftLatest: + initLatest(node.(*raftLatest), id) + case *raftUIT: + initUIT(node.(*raftUIT), id) + default: + panic("invalid node type") + } +} + +func initUIT(node *raftUIT, id string) { + node.Config = raft.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raft.ServerID(id) + node.Config.LocalID = node.id + node.Store = raft.NewInmemStore() + node.Snap = raft.NewInmemSnapshotStore() + node.fsm = &raft.MockFSM{} +} + +func initLatest(node *raftLatest, id string) { + node.Config = raftrs.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raftrs.ServerID(id) + node.Config.LocalID = node.id + node.Store = raftrs.NewInmemStore() + node.Snap = raftrs.NewInmemSnapshotStore() + node.fsm = &raftrs.MockFSM{} +} + func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := newRaftCluster[raftLatest](initCount) + rLatest := newRaftCluster[raftLatest](initCount, "raftOld") configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { - rLatest.rafts[i].Config = raftrs.DefaultConfig() - rLatest.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.CommitTimeout = 5 * time.Millisecond - rLatest.rafts[i].id = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) - rLatest.rafts[i].Config.LocalID = rLatest.rafts[i].id - rLatest.rafts[i].Store = raftrs.NewInmemStore() - rLatest.rafts[i].Snap = raftrs.NewInmemSnapshotStore() - rLatest.rafts[i].fsm = &raftrs.MockFSM{} var err error rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) @@ -120,7 +150,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - rUIT := newRaftCluster[raftUIT](initCount) + rUIT := newRaftCluster[raftUIT](initCount, "raftNew") leader, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, leader) @@ -131,16 +161,6 @@ func TestRaft_RollingUpgrade(t *testing.T) { leaderIdx = i continue } - rUIT.rafts[i].Config = raft.DefaultConfig() - rUIT.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.CommitTimeout = 5 * time.Millisecond - rUIT.rafts[i].id = raft.ServerID(fmt.Sprintf("newGrpc%d", i)) - rUIT.rafts[i].Config.LocalID = rUIT.rafts[i].id - rUIT.rafts[i].Store = raft.NewInmemStore() - rUIT.rafts[i].Snap = raft.NewInmemSnapshotStore() - rUIT.rafts[i].fsm = &raft.MockFSM{} var err error rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) From 14dfa683093b3a8d6817760a9d5a418aef66efec Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 8 Jun 2023 16:19:01 -0400 Subject: [PATCH 09/87] clean up leader rolling upgrade --- raft-compat/rolling_upgrade_test.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index ed7cf1ba3..cc5bfb6ee 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -179,12 +179,6 @@ func TestRaft_RollingUpgrade(t *testing.T) { future = getLeader.raft.Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - rUIT.rafts[leaderIdx].Config = raft.InmemConfig(t) - rUIT.rafts[leaderIdx].id = raft.ServerID(fmt.Sprintf("newGrpc%d", leaderIdx)) - rUIT.rafts[leaderIdx].Config.LocalID = rUIT.rafts[leaderIdx].id - rUIT.rafts[leaderIdx].Store = raft.NewInmemStore() - rUIT.rafts[leaderIdx].Snap = raft.NewInmemSnapshotStore() - rUIT.rafts[leaderIdx].fsm = &raft.MockFSM{} var err error rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) From a617c6d83ba7f4cc046f6e108343eee55f97b966 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 8 Jun 2023 16:20:08 -0400 Subject: [PATCH 10/87] fix use of deprecate Leader method --- raft-compat/rolling_upgrade_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index cc5bfb6ee..ddf901c71 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -145,7 +145,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { } getLeader := rLatest.getLeader() require.NotNil(t, getLeader) - a := getLeader.raft.Leader() + a, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, a) future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) @@ -171,7 +171,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { time.Sleep(1 * time.Second) // Check Leader haven't changed as we are not replacing the leader - a := getLeader.raft.Leader() + a, _ := getLeader.raft.LeaderWithID() require.Equal(t, a, leader) getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) rLatest.rafts[i].raft.Shutdown() @@ -186,13 +186,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) // Check Leader haven't changed as we are not replacing the leader - a = getLeader.raft.Leader() + a, _ = getLeader.raft.LeaderWithID() require.Equal(t, a, leader) getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) time.Sleep(1 * time.Second) rLatest.rafts[leaderIdx].raft.Shutdown() time.Sleep(1 * time.Second) - aNew := rUIT.getLeader().raft.Leader() + aNew, _ := getLeader.raft.LeaderWithID() require.NotEqual(t, aNew, leader) require.Len(t, rUIT.getLeader().fsm.Logs(), 2) From 60f32320ef4ff64170f3d22c35097172304c3313 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 10:43:10 -0400 Subject: [PATCH 11/87] extract cluster package --- raft-compat/rolling_upgrade_test.go | 172 +++++----------------------- raft-compat/testcluster/cluster.go | 163 ++++++++++++++++++++++++++ 2 files changed, 193 insertions(+), 142 deletions(-) create mode 100644 raft-compat/testcluster/cluster.go diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index ddf901c71..35dd73680 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,140 +1,34 @@ package raft_compat import ( - "fmt" raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" "time" ) -type raftUIT struct { - raft *raft.Raft - trans *raft.NetworkTransport - Config *raft.Config - Store *raft.InmemStore - Snap *raft.InmemSnapshotStore - id raft.ServerID - fsm *raft.MockFSM -} - -func (r raftUIT) getLocalID() string { - return string(r.id) -} - -func (r raftUIT) getLeaderID() string { - _, id := r.raft.LeaderWithID() - return string(id) -} - -type raftLatest struct { - raft *raftrs.Raft - trans *raftrs.NetworkTransport - Config *raftrs.Config - Store *raftrs.InmemStore - Snap *raftrs.InmemSnapshotStore - id raftrs.ServerID - fsm *raftrs.MockFSM -} - -func (r raftLatest) getLocalID() string { - return string(r.id) -} - -func (r raftLatest) getLeaderID() string { - _, id := r.raft.LeaderWithID() - return string(id) -} - -type raftNode interface { - getLocalID() string - getLeaderID() string -} - -type raftCluster[T raftNode] struct { - rafts []T -} - -func newRaftCluster[T raftNode](count int, name string) raftCluster[T] { - rc := raftCluster[T]{} - rc.rafts = make([]T, count) - for i := 0; i < count; i++ { - initNode(&rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) - } - return rc -} - -func (r *raftCluster[T]) getLeader() T { - var empty T - for _, n := range r.rafts { - if n.getLocalID() == n.getLeaderID() { - return n - } - } - return empty -} - -func initNode(node interface{}, id string) { - switch node.(type) { - case *raftLatest: - initLatest(node.(*raftLatest), id) - case *raftUIT: - initUIT(node.(*raftUIT), id) - default: - panic("invalid node type") - } -} - -func initUIT(node *raftUIT, id string) { - node.Config = raft.DefaultConfig() - node.Config.HeartbeatTimeout = 50 * time.Millisecond - node.Config.ElectionTimeout = 50 * time.Millisecond - node.Config.LeaderLeaseTimeout = 50 * time.Millisecond - node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raft.ServerID(id) - node.Config.LocalID = node.id - node.Store = raft.NewInmemStore() - node.Snap = raft.NewInmemSnapshotStore() - node.fsm = &raft.MockFSM{} -} - -func initLatest(node *raftLatest, id string) { - node.Config = raftrs.DefaultConfig() - node.Config.HeartbeatTimeout = 50 * time.Millisecond - node.Config.ElectionTimeout = 50 * time.Millisecond - node.Config.LeaderLeaseTimeout = 50 * time.Millisecond - node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftrs.ServerID(id) - node.Config.LocalID = node.id - node.Store = raftrs.NewInmemStore() - node.Snap = raftrs.NewInmemSnapshotStore() - node.fsm = &raftrs.MockFSM{} -} - func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := newRaftCluster[raftLatest](initCount, "raftOld") + rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { var err error - rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rLatest.rafts[i].raft, err = raftrs.NewRaft(rLatest.rafts[i].Config, rLatest.rafts[i].fsm, rLatest.rafts[i].Store, - rLatest.rafts[i].Store, rLatest.rafts[i].Snap, rLatest.rafts[i].trans) require.NoError(t, err) configuration.Servers = append(configuration.Servers, raftrs.Server{ - ID: rLatest.rafts[i].id, - Address: rLatest.rafts[i].trans.LocalAddr(), + ID: raftrs.ServerID(rLatest.ID(i)), + Address: raftrs.ServerAddress(rLatest.Addr(i)), }) } - boot := rLatest.rafts[0].raft.BootstrapCluster(configuration) + raft0 := rLatest.Raft(0).(*raftrs.Raft) + boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := rLatest.rafts[0].raft.LeaderCh() + lCh := raft0.LeaderCh() after := time.After(5 * time.Second) @@ -143,58 +37,52 @@ func TestRaft_RollingUpgrade(t *testing.T) { t.Fatalf("timedout") case <-lCh: } - getLeader := rLatest.getLeader() + getLeader := rLatest.GetLeader() require.NotNil(t, getLeader) - a, _ := getLeader.raft.LeaderWithID() + a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.raft.Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - rUIT := newRaftCluster[raftUIT](initCount, "raftNew") - leader, _ := getLeader.raft.LeaderWithID() + rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") + leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers leaderIdx := 0 - for i := 0; i < len(rLatest.rafts); i++ { - if getLeader.getLocalID() == rLatest.rafts[i].getLocalID() { + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == rLatest.ID(i) { leaderIdx = i continue } - var err error - rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rUIT.rafts[i].raft, err = raft.NewRaft(rUIT.rafts[i].Config, rUIT.rafts[i].fsm, rUIT.rafts[i].Store, - rUIT.rafts[i].Store, rUIT.rafts[i].Snap, rUIT.rafts[i].trans) - getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[i].getLocalID()), raftrs.ServerAddress(rUIT.rafts[i].trans.LocalAddr()), 0, 0) - time.Sleep(1 * time.Second) + future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) + time.Sleep(1 * time.Second) + require.NoError(t, future.Error()) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.raft.LeaderWithID() + a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) - rLatest.rafts[i].raft.Shutdown() + getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(i).(*raftrs.Raft).Shutdown() } - future = getLeader.raft.Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - var err error - rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rUIT.rafts[leaderIdx].raft, err = raft.NewRaft(rUIT.rafts[leaderIdx].Config, rUIT.rafts[leaderIdx].fsm, rUIT.rafts[leaderIdx].Store, - rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) - getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) + f := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + time.Sleep(1 * time.Second) + require.NoError(t, f.Error()) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.raft.LeaderWithID() + a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) + getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) time.Sleep(1 * time.Second) - rLatest.rafts[leaderIdx].raft.Shutdown() + rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() time.Sleep(1 * time.Second) - aNew, _ := getLeader.raft.LeaderWithID() + newLeader := rUIT.GetLeader() + aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() require.NotEqual(t, aNew, leader) - require.Len(t, rUIT.getLeader().fsm.Logs(), 2) + require.Equal(t, newLeader.NumLogs(), 2) } diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go new file mode 100644 index 000000000..79a0cdb70 --- /dev/null +++ b/raft-compat/testcluster/cluster.go @@ -0,0 +1,163 @@ +package testcluster + +import ( + "fmt" + raftrs "github.com/dhiayachi/raft" + "github.com/hashicorp/raft" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +type RaftUIT struct { + raft *raft.Raft + trans *raft.NetworkTransport + Config *raft.Config + Store *raft.InmemStore + Snap *raft.InmemSnapshotStore + id raft.ServerID + fsm *raft.MockFSM +} + +func (r RaftUIT) NumLogs() int { + return len(r.fsm.Logs()) +} + +func (r RaftUIT) GetLocalAddr() string { + return string(r.trans.LocalAddr()) +} + +func (r RaftUIT) GetRaft() interface{} { + return r.raft +} + +func (r RaftUIT) GetLocalID() string { + return string(r.id) +} + +func (r RaftUIT) GetLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +func (r *raftCluster[T]) ID(i int) string { + return r.rafts[i].GetLocalID() +} +func (r *raftCluster[T]) Addr(i int) string { + return r.rafts[i].GetLocalAddr() +} + +func (r *raftCluster[T]) Raft(i int) interface{} { + return r.rafts[i].GetRaft() +} + +type RaftLatest struct { + raft *raftrs.Raft + trans *raftrs.NetworkTransport + Config *raftrs.Config + Store *raftrs.InmemStore + Snap *raftrs.InmemSnapshotStore + id raftrs.ServerID + fsm *raftrs.MockFSM +} + +func (r RaftLatest) NumLogs() int { + return len(r.fsm.Logs()) +} + +func (r RaftLatest) GetLocalAddr() string { + return string(r.trans.LocalAddr()) +} + +func (r RaftLatest) GetRaft() interface{} { + return r.raft +} + +func (r RaftLatest) GetLocalID() string { + return string(r.id) +} + +func (r RaftLatest) GetLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +type RaftNode interface { + GetLocalID() string + GetLocalAddr() string + GetLeaderID() string + GetRaft() interface{} + NumLogs() int +} + +type raftCluster[T RaftNode] struct { + rafts []T +} + +func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluster[T] { + rc := raftCluster[T]{} + rc.rafts = make([]T, count) + for i := 0; i < count; i++ { + initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) + } + return rc +} + +func (r *raftCluster[T]) GetLeader() T { + var empty T + for _, n := range r.rafts { + if n.GetLocalID() == n.GetLeaderID() { + return n + } + } + return empty +} + +func initNode(t *testing.T, node interface{}, id string) { + switch node.(type) { + case *RaftLatest: + initLatest(t, node.(*RaftLatest), id) + case *RaftUIT: + initUIT(t, node.(*RaftUIT), id) + default: + panic("invalid node type") + } +} + +func initUIT(t *testing.T, node *RaftUIT, id string) { + node.Config = raft.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raft.ServerID(id) + node.Config.LocalID = node.id + node.Store = raft.NewInmemStore() + node.Snap = raft.NewInmemSnapshotStore() + node.fsm = &raft.MockFSM{} + var err error + node.trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + node.raft, err = raft.NewRaft(node.Config, node.fsm, node.Store, + node.Store, node.Snap, node.trans) + require.NoError(t, err) +} + +func initLatest(t *testing.T, node *RaftLatest, id string) { + node.Config = raftrs.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raftrs.ServerID(id) + node.Config.LocalID = node.id + node.Store = raftrs.NewInmemStore() + node.Snap = raftrs.NewInmemSnapshotStore() + node.fsm = &raftrs.MockFSM{} + var err error + node.trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + node.raft, err = raftrs.NewRaft(node.Config, node.fsm, node.Store, + node.Store, node.Snap, node.trans) + require.NoError(t, err) +} From 16eebcd33861f8d42bf1b4f70dc14dea06000636 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 10:43:51 -0400 Subject: [PATCH 12/87] export cluster Type --- raft-compat/testcluster/cluster.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 79a0cdb70..590424418 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -40,14 +40,14 @@ func (r RaftUIT) GetLeaderID() string { return string(id) } -func (r *raftCluster[T]) ID(i int) string { +func (r *RaftCluster[T]) ID(i int) string { return r.rafts[i].GetLocalID() } -func (r *raftCluster[T]) Addr(i int) string { +func (r *RaftCluster[T]) Addr(i int) string { return r.rafts[i].GetLocalAddr() } -func (r *raftCluster[T]) Raft(i int) interface{} { +func (r *RaftCluster[T]) Raft(i int) interface{} { return r.rafts[i].GetRaft() } @@ -90,12 +90,12 @@ type RaftNode interface { NumLogs() int } -type raftCluster[T RaftNode] struct { +type RaftCluster[T RaftNode] struct { rafts []T } -func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluster[T] { - rc := raftCluster[T]{} +func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) RaftCluster[T] { + rc := RaftCluster[T]{} rc.rafts = make([]T, count) for i := 0; i < count; i++ { initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) @@ -103,7 +103,7 @@ func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluste return rc } -func (r *raftCluster[T]) GetLeader() T { +func (r *RaftCluster[T]) GetLeader() T { var empty T for _, n := range r.rafts { if n.GetLocalID() == n.GetLeaderID() { From 3e9efafc12771f05fb1dd387d0a006e651263437 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 11:54:23 -0400 Subject: [PATCH 13/87] clean up tests and add test utils --- raft-compat/go.mod | 4 +- raft-compat/rolling_upgrade_test.go | 33 +++++++---------- raft-compat/testcluster/cluster.go | 6 ++- raft-compat/utils/test_utils.go | 57 +++++++++++++++++++++++++++++ 4 files changed, 77 insertions(+), 23 deletions(-) create mode 100644 raft-compat/utils/test_utils.go diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 40db0858b..96b3e9626 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,14 +16,14 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/dhiayachi/raft v1.0.0 => ./raftrs +replace github.com/hashicorp/raft-latest => ./raftrs replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dhiayachi/raft v1.0.0 github.com/hashicorp/raft v1.2.0 + github.com/hashicorp/raft-latest v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 35dd73680..b9c9f0ff6 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,9 +1,10 @@ package raft_compat import ( - raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + raftrs "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" + "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" "testing" "time" @@ -28,21 +29,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := raft0.LeaderCh() - - after := time.After(5 * time.Second) - - select { - case <-after: - t.Fatalf("timedout") - case <-lCh: - } + utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) getLeader := rLatest.GetLeader() - require.NotNil(t, getLeader) + require.NotEmpty(t, getLeader) a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, a) future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) - require.NoError(t, future.Error()) + utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() @@ -58,8 +51,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) - time.Sleep(1 * time.Second) - require.NoError(t, future.Error()) + utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) @@ -69,17 +61,18 @@ func TestRaft_RollingUpgrade(t *testing.T) { future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - f := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) - time.Sleep(1 * time.Second) - require.NoError(t, f.Error()) + fa := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + utils.WaitFuture(t, fa) + // Check Leader haven't changed as we are not replacing the leader a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) - time.Sleep(1 * time.Second) + fr := getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) + utils.WaitFuture(t, fr) rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() - time.Sleep(1 * time.Second) + utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() + require.NotEmpty(t, newLeader) aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() require.NotEqual(t, aNew, leader) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 590424418..962453bec 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -2,8 +2,8 @@ package testcluster import ( "fmt" - raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + raftrs "github.com/hashicorp/raft-latest" "github.com/stretchr/testify/require" "testing" "time" @@ -113,6 +113,10 @@ func (r *RaftCluster[T]) GetLeader() T { return empty } +func (r *RaftCluster[T]) Len() int { + return len(r.rafts) +} + func initNode(t *testing.T, node interface{}, id string) { switch node.(type) { case *RaftLatest: diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go new file mode 100644 index 000000000..d69d5366c --- /dev/null +++ b/raft-compat/utils/test_utils.go @@ -0,0 +1,57 @@ +package utils + +import ( + "fmt" + "github.com/hashicorp/raft" + raftrs "github.com/hashicorp/raft-latest" + "github.com/hashicorp/raft/compat/testcluster" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c testcluster.RaftCluster[T]) { + + leader := func() string { + for i := 0; i < c.Len(); i++ { + switch r := c.Raft(i).(type) { + case *raft.Raft: + if r.State() == raft.Leader { + return c.ID(i) + } + case *raftrs.Raft: + if r.State() == raftrs.Leader { + return c.ID(i) + } + } + } + return "" + } + after := time.After(5 * time.Second) + ticker := time.NewTicker(100 * time.Millisecond) + for { + select { + case <-after: + t.Fatalf("timedout") + case <-ticker.C: + id := leader() + if id != "" { + if id != oldLeader || oldLeader == "" { + return + } + } + } + } +} + +type future interface { + Error() error +} + +func WaitFuture(t *testing.T, f future) { + timer := time.AfterFunc(1000*time.Millisecond, func() { + panic(fmt.Errorf("timeout waiting for future %v", f)) + }) + defer timer.Stop() + require.NoError(t, f.Error()) +} From 8b56ef33ed443199c0c38099df3d75d404a0ceda Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 11:55:22 -0400 Subject: [PATCH 14/87] rename package to raftlatest --- raft-compat/rolling_upgrade_test.go | 36 ++++++++++++++--------------- raft-compat/testcluster/cluster.go | 30 ++++++++++++------------ raft-compat/utils/test_utils.go | 6 ++--- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index b9c9f0ff6..c7e0db013 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -2,7 +2,7 @@ package raft_compat import ( "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" @@ -14,17 +14,17 @@ func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") - configuration := raftrs.Configuration{} + configuration := raftlatest.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftrs.Server{ - ID: raftrs.ServerID(rLatest.ID(i)), - Address: raftrs.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftlatest.Server{ + ID: raftlatest.ServerID(rLatest.ID(i)), + Address: raftlatest.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(0).(*raftrs.Raft) + raft0 := rLatest.Raft(0).(*raftlatest.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -32,13 +32,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers @@ -49,27 +49,27 @@ func TestRaft_RollingUpgrade(t *testing.T) { continue } - future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) + future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(i)), raftlatest.ServerAddress(rUIT.Addr(i)), 0, 0) utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(i).(*raftrs.Raft).Shutdown() + getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(i).(*raftlatest.Raft).Shutdown() } - future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - fa := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + fa := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(leaderIdx)), raftlatest.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) + fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() + rLatest.Raft(leaderIdx).(*raftlatest.Raft).Shutdown() utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 962453bec..71ef6432e 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -3,7 +3,7 @@ package testcluster import ( "fmt" "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/stretchr/testify/require" "testing" "time" @@ -52,13 +52,13 @@ func (r *RaftCluster[T]) Raft(i int) interface{} { } type RaftLatest struct { - raft *raftrs.Raft - trans *raftrs.NetworkTransport - Config *raftrs.Config - Store *raftrs.InmemStore - Snap *raftrs.InmemSnapshotStore - id raftrs.ServerID - fsm *raftrs.MockFSM + raft *raftlatest.Raft + trans *raftlatest.NetworkTransport + Config *raftlatest.Config + Store *raftlatest.InmemStore + Snap *raftlatest.InmemSnapshotStore + id raftlatest.ServerID + fsm *raftlatest.MockFSM } func (r RaftLatest) NumLogs() int { @@ -148,20 +148,20 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { } func initLatest(t *testing.T, node *RaftLatest, id string) { - node.Config = raftrs.DefaultConfig() + node.Config = raftlatest.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftrs.ServerID(id) + node.id = raftlatest.ServerID(id) node.Config.LocalID = node.id - node.Store = raftrs.NewInmemStore() - node.Snap = raftrs.NewInmemSnapshotStore() - node.fsm = &raftrs.MockFSM{} + node.Store = raftlatest.NewInmemStore() + node.Snap = raftlatest.NewInmemSnapshotStore() + node.fsm = &raftlatest.MockFSM{} var err error - node.trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + node.trans, err = raftlatest.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - node.raft, err = raftrs.NewRaft(node.Config, node.fsm, node.Store, + node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index d69d5366c..4b6eb9ff6 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -3,7 +3,7 @@ package utils import ( "fmt" "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" @@ -19,8 +19,8 @@ func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c if r.State() == raft.Leader { return c.ID(i) } - case *raftrs.Raft: - if r.State() == raftrs.Leader { + case *raftlatest.Raft: + if r.State() == raftlatest.Leader { return c.ID(i) } } From 2e8a26af1450efff893b649e98452895651a9990 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:05:10 -0400 Subject: [PATCH 15/87] remove submodule --- .gitmodules | 3 --- raft-compat/raftrs | 1 - 2 files changed, 4 deletions(-) delete mode 160000 raft-compat/raftrs diff --git a/.gitmodules b/.gitmodules index ed959a7cf..e69de29bb 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +0,0 @@ -[submodule "raft-compat/raftrs"] - path = raft-compat/raftrs - url = git@github.com:hashicorp/raft diff --git a/raft-compat/raftrs b/raft-compat/raftrs deleted file mode 160000 index 8fdc4ce5b..000000000 --- a/raft-compat/raftrs +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From 01b88ade3fc6bf45cdc208d6c131bd986ea24af6 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:05:51 -0400 Subject: [PATCH 16/87] new submodule --- .gitmodules | 3 +++ raft-compat/raft-latest | 1 + 2 files changed, 4 insertions(+) create mode 160000 raft-compat/raft-latest diff --git a/.gitmodules b/.gitmodules index e69de29bb..96d94f795 100644 --- a/.gitmodules +++ b/.gitmodules @@ -0,0 +1,3 @@ +[submodule "raft-compat/raft-latest"] + path = raft-compat/raft-latest + url = git@github.com:hashicorp/raft diff --git a/raft-compat/raft-latest b/raft-compat/raft-latest new file mode 160000 index 000000000..8fdc4ce5b --- /dev/null +++ b/raft-compat/raft-latest @@ -0,0 +1 @@ +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From a7aefface84feea246fb4194bb21bcc9a33ed6f4 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:06:20 -0400 Subject: [PATCH 17/87] fix go.mod --- raft-compat/go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 96b3e9626..28e134a74 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,7 +16,7 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-latest => ./raftrs +replace github.com/hashicorp/raft-latest => ./raft-latest replace github.com/hashicorp/raft => ../ From 8ade0def9ded3113b8eb8b066ed16b42a7b243a8 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 13:19:01 -0400 Subject: [PATCH 18/87] change inmemConfig to be not exported --- integ_test.go | 2 +- raft_test.go | 62 +++++++++++++++++++++++++-------------------------- testing.go | 4 ++-- 3 files changed, 34 insertions(+), 34 deletions(-) diff --git a/integ_test.go b/integ_test.go index a39d21535..905663496 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,7 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(t *testing.T, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { - conf = InmemConfig(t) + conf = inmemConfig(t) } dir, err := os.MkdirTemp("", "raft") diff --git a/raft_test.go b/raft_test.go index 21f2af63a..df9712b1c 100644 --- a/raft_test.go +++ b/raft_test.go @@ -153,7 +153,7 @@ func TestRaft_RecoverCluster(t *testing.T) { snapshotThreshold := 5 runRecover := func(t *testing.T, applies int) { var err error - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 conf.SnapshotThreshold = uint64(snapshotThreshold) c := MakeCluster(3, t, conf) @@ -296,7 +296,7 @@ func TestRaft_HasExistingState(t *testing.T) { } func TestRaft_SingleNode(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -502,7 +502,7 @@ func TestRaft_ApplyNonLeader(t *testing.T) { func TestRaft_ApplyConcurrent(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout c := MakeCluster(3, t, conf) @@ -553,7 +553,7 @@ func TestRaft_ApplyConcurrent(t *testing.T) { func TestRaft_ApplyConcurrent_Timeout(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.CommitTimeout = 1 * time.Millisecond conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout @@ -621,7 +621,7 @@ func TestRaft_JoinNode(t *testing.T) { func TestRaft_JoinNode_ConfigStore(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := makeCluster(t, &MakeClusterOpts{ Peers: 1, Bootstrap: true, @@ -779,7 +779,7 @@ func TestRaft_RemoveLeader(t *testing.T) { func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -842,7 +842,7 @@ func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { func TestRaft_RemoveFollower_SplitCluster(t *testing.T) { // Make a cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(4, t, conf) defer c.Close() @@ -957,7 +957,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { func TestRaft_SnapshotRestore(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1017,7 +1017,7 @@ func TestRaft_SnapshotRestore(t *testing.T) { func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 opts := &MakeClusterOpts{ Peers: 1, @@ -1099,7 +1099,7 @@ func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { func TestRaft_SnapshotRestore_Progress(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1217,7 +1217,7 @@ func (b *lockedBytesBuffer) String() string { // up. func TestRaft_NoRestoreOnStart(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 conf.NoSnapshotRestoreOnStart = true c := MakeCluster(1, t, conf) @@ -1262,7 +1262,7 @@ func TestRaft_NoRestoreOnStart(t *testing.T) { func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { var err error // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 1 conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) @@ -1367,7 +1367,7 @@ func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { func TestRaft_AutoSnapshot(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.SnapshotInterval = conf.CommitTimeout * 2 conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 @@ -1397,7 +1397,7 @@ func TestRaft_AutoSnapshot(t *testing.T) { func TestRaft_UserSnapshot(t *testing.T) { // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) @@ -1437,7 +1437,7 @@ func snapshotAndRestore(t *testing.T, offset uint64, monotonicLogStore bool, res t.Helper() // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) // snapshot operations perform some file IO operations. // increase times out to account for that @@ -1599,7 +1599,7 @@ func TestRaft_UserRestore(t *testing.T) { func TestRaft_SendSnapshotFollower(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1641,7 +1641,7 @@ func TestRaft_SendSnapshotFollower(t *testing.T) { func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1695,7 +1695,7 @@ func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { func TestRaft_ReJoinFollower(t *testing.T) { // Enable operation after a remove. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -1772,7 +1772,7 @@ func TestRaft_ReJoinFollower(t *testing.T) { func TestRaft_LeaderLeaseExpire(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1901,7 +1901,7 @@ func TestRaft_VerifyLeader_Single(t *testing.T) { func TestRaft_VerifyLeader_Fail(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1941,7 +1941,7 @@ func TestRaft_VerifyLeader_Fail(t *testing.T) { func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() @@ -1975,7 +1975,7 @@ func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { func TestRaft_NotifyCh(t *testing.T) { ch := make(chan bool, 1) - conf := InmemConfig(t) + conf := inmemConfig(t) conf.NotifyCh = ch c := MakeCluster(1, t, conf) defer c.Close() @@ -2064,7 +2064,7 @@ func TestRaft_AppendEntry(t *testing.T) { } func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 3 c := MakeCluster(3, t, conf) defer c.Close() @@ -2144,13 +2144,13 @@ func TestRaft_ProtocolVersion_RejectRPC(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { // Make a cluster back on protocol version 1. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 1 c := MakeCluster(2, t, conf) defer c.Close() // Set up another server speaking protocol version 2. - conf = InmemConfig(t) + conf = inmemConfig(t) conf.ProtocolVersion = 2 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2187,14 +2187,14 @@ func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { // Make a cluster back on protocol version 2. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 2 c := MakeCluster(2, t, conf) defer c.Close() oldAddr := c.Followers()[0].localAddr // Set up another server speaking protocol version 3. - conf = InmemConfig(t) + conf = inmemConfig(t) conf.ProtocolVersion = 3 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2221,7 +2221,7 @@ func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { func TestRaft_LeaderID_Propagated(t *testing.T) { // Make a cluster on protocol version 3. - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() err := waitForLeader(c) @@ -2648,7 +2648,7 @@ func TestRaft_CacheLogWithStoreError(t *testing.T) { } func TestRaft_ReloadConfig(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.LeaderLeaseTimeout = 40 * time.Millisecond c := MakeCluster(1, t, conf) defer c.Close() @@ -2679,7 +2679,7 @@ func TestRaft_ReloadConfig(t *testing.T) { } func TestRaft_ReloadConfigValidates(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -2921,7 +2921,7 @@ func TestRaft_ClusterCanRegainStability_WhenNonVoterWithHigherTermJoin(t *testin // started when a standby is shut down and restarted. func TestRaft_FollowerRemovalNoElection(t *testing.T) { // Make a cluster - inmemConf := InmemConfig(t) + inmemConf := inmemConfig(t) inmemConf.HeartbeatTimeout = 100 * time.Millisecond inmemConf.ElectionTimeout = 100 * time.Millisecond c := MakeCluster(3, t, inmemConf) diff --git a/testing.go b/testing.go index e3859e1e2..81765f3b5 100644 --- a/testing.go +++ b/testing.go @@ -23,7 +23,7 @@ var ( ) // Return configurations optimized for in-memory -func InmemConfig(t *testing.T) *Config { +func inmemConfig(t *testing.T) *Config { conf := DefaultConfig() conf.HeartbeatTimeout = 50 * time.Millisecond conf.ElectionTimeout = 50 * time.Millisecond @@ -760,7 +760,7 @@ type MakeClusterOpts struct { // each other. func makeCluster(t *testing.T, opts *MakeClusterOpts) *cluster { if opts.Conf == nil { - opts.Conf = InmemConfig(t) + opts.Conf = inmemConfig(t) } c := &cluster{ From 164e0c9d4745aba0bceb520091eb3e2ecc60ab6c Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 13:20:18 -0400 Subject: [PATCH 19/87] remove unused func --- testing.go | 33 --------------------------------- 1 file changed, 33 deletions(-) diff --git a/testing.go b/testing.go index 81765f3b5..fd770d423 100644 --- a/testing.go +++ b/testing.go @@ -222,39 +222,6 @@ func newTestLoggerWithPrefix(t *testing.T, prefix string) hclog.Logger { }) } -func (c *cluster) NodeID(index int) ServerID { - if index >= len(c.rafts) { - return "" - } - return c.rafts[index].localID -} - -// ConnectExternal connects all the transports together. -func (c *cluster) ConnectExternal(a ServerAddress) { - c.logger.Debug("fully connecting") - for _, t := range c.trans { - t.Connect(a, t) - } -} - -func (c *cluster) Transport(index int) LoopbackTransport { - if index >= len(c.trans) { - return nil - } - return c.trans[index] -} - -func (c *cluster) NodeAddress(index int) ServerAddress { - if index >= len(c.rafts) { - return "" - } - return c.rafts[index].localAddr -} - -func (c *cluster) RaftsLen() int { - return len(c.rafts) -} - type cluster struct { dirs []string stores []*InmemStore From 6b0d70d5e98d5eb7c57506af32285d7f168ecb61 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:17:35 -0400 Subject: [PATCH 20/87] add replace rolling upgrade tests --- raft-compat/rolling_upgrade_test.go | 123 +++++++++++++++++++++-- raft-compat/testcluster/cluster.go | 149 +++++++++++++++++++++++----- raft-compat/utils/test_utils.go | 5 +- 3 files changed, 244 insertions(+), 33 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index c7e0db013..f1d213a2f 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,6 +1,7 @@ package raft_compat import ( + "fmt" "github.com/hashicorp/raft" raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" @@ -10,10 +11,13 @@ import ( "time" ) +// TestRaft_RollingUpgrade This test perform a rolling upgrade by adding a new node, +// wait for it to join the cluster and remove one of the old nodes, until all nodes +// are cycled func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") + rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") configuration := raftlatest.Configuration{} for i := 0; i < initCount; i++ { @@ -24,12 +28,12 @@ func TestRaft_RollingUpgrade(t *testing.T) { Address: raftlatest.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(0).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) + utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() @@ -37,7 +41,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) - rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") + rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, leader) @@ -56,7 +60,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(i).(*raftlatest.Raft).Shutdown() + rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() } future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) @@ -69,8 +73,8 @@ func TestRaft_RollingUpgrade(t *testing.T) { require.Equal(t, a, leader) fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(leaderIdx).(*raftlatest.Raft).Shutdown() - utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) + rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() @@ -79,3 +83,108 @@ func TestRaft_RollingUpgrade(t *testing.T) { require.Equal(t, newLeader.NumLogs(), 2) } + +// TestRaft_ReplaceUpgrade This test perform a rolling upgrade by removing an old node, +// and create a new node with the same store until all old nodes are cycled to new nodes. +// This simulate the advised way of upgrading in Consul. +func TestRaft_ReplaceUpgrade(t *testing.T) { + + initCount := 3 + rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + configuration := raftlatest.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftlatest.Server{ + ID: raftlatest.ServerID(rLatest.ID(i)), + Address: raftlatest.ServerAddress(rLatest.Addr(i)), + }) + } + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", rLatest) + getLeader := rLatest.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + leaderIdx := 0 + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == rLatest.ID(i) { + leaderIdx = i + continue + } + + // Check Leader haven't changed as we are not replacing the leader + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.Equal(t, a, leader) + getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + + fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) + rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + + // Keep the store, to be passed to the upgraded node. + store := rLatest.Store(rLatest.ID(i)) + + //Delete the node from the cluster + rLatest.DeleteNode(rLatest.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftlatest.InmemStore)) + future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.GetLocalID()), raftlatest.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + rLatest.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := rLatest.Store(getLeader.GetLocalID()) + + //Remove and shutdown the leader node + fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(getLeader.GetLocalID()), 0, 0) + utils.WaitFuture(t, fr) + rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + + // Delete the old leader node from the cluster + rLatest.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, rLatest) + getLeader = rLatest.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftlatest.InmemStore)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", rLatest) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) +} diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 71ef6432e..9e04972ff 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -31,6 +31,10 @@ func (r RaftUIT) GetRaft() interface{} { return r.raft } +func (r RaftUIT) GetStore() interface{} { + return r.Store +} + func (r RaftUIT) GetLocalID() string { return string(r.id) } @@ -40,17 +44,23 @@ func (r RaftUIT) GetLeaderID() string { return string(id) } -func (r *RaftCluster[T]) ID(i int) string { +func (r *RaftCluster) ID(i int) string { return r.rafts[i].GetLocalID() } -func (r *RaftCluster[T]) Addr(i int) string { +func (r *RaftCluster) Addr(i int) string { return r.rafts[i].GetLocalAddr() } -func (r *RaftCluster[T]) Raft(i int) interface{} { +func (r *RaftCluster) Raft(id string) interface{} { + i := r.GetIndex(id) return r.rafts[i].GetRaft() } +func (r *RaftCluster) Store(id string) interface{} { + i := r.GetIndex(id) + return r.rafts[i].GetStore() +} + type RaftLatest struct { raft *raftlatest.Raft trans *raftlatest.NetworkTransport @@ -72,6 +82,9 @@ func (r RaftLatest) GetLocalAddr() string { func (r RaftLatest) GetRaft() interface{} { return r.raft } +func (r RaftLatest) GetStore() interface{} { + return r.Store +} func (r RaftLatest) GetLocalID() string { return string(r.id) @@ -87,48 +100,79 @@ type RaftNode interface { GetLocalAddr() string GetLeaderID() string GetRaft() interface{} + GetStore() interface{} NumLogs() int } -type RaftCluster[T RaftNode] struct { - rafts []T +type RaftCluster struct { + rafts []RaftNode } -func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) RaftCluster[T] { - rc := RaftCluster[T]{} - rc.rafts = make([]T, count) +func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, count int, name string) RaftCluster { + rc := RaftCluster{} + rc.rafts = make([]RaftNode, count) for i := 0; i < count; i++ { - initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) + rc.rafts[i] = f(t, fmt.Sprintf("%s-%d", name, i)) } return rc } -func (r *RaftCluster[T]) GetLeader() T { - var empty T +//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftlatest.InmemStore) *T { +// raft := new(T) +// initNode(t, raft, name, store) +// return raft +//} + +func (r *RaftCluster) GetLeader() RaftNode { for _, n := range r.rafts { if n.GetLocalID() == n.GetLeaderID() { return n } } - return empty + return nil } -func (r *RaftCluster[T]) Len() int { +func (r *RaftCluster) Len() int { return len(r.rafts) } -func initNode(t *testing.T, node interface{}, id string) { - switch node.(type) { - case *RaftLatest: - initLatest(t, node.(*RaftLatest), id) - case *RaftUIT: - initUIT(t, node.(*RaftUIT), id) - default: - panic("invalid node type") +//func initNode(t *testing.T, node interface{}, id string, store *raftlatest.InmemStore) { +// switch node.(type) { +// case *RaftLatest: +// initLatest(t, node.(*RaftLatest), id) +// case *RaftUIT: +// initUIT(t, node.(*RaftUIT), id, convertInMemStore(store)) +// default: +// panic("invalid node type") +// } +//} + +func (r *RaftCluster) AddNode(node RaftNode) { + r.rafts = append([]RaftNode{node}, r.rafts...) +} + +func (r *RaftCluster) DeleteNode(id string) { + i := r.GetIndex(id) + r.rafts = append(r.rafts[:i], r.rafts[i+1:]...) +} + +func (r *RaftCluster) GetIndex(id string) int { + i := 0 + for _, r := range r.rafts { + if r.GetLocalID() == id { + return i + } + i++ } + return -1 } -func initUIT(t *testing.T, node *RaftUIT, id string) { +func InitUIT(t *testing.T, id string) RaftNode { + return InitUITWithStore(t, id, nil) +} + +func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) RaftNode { + node := RaftUIT{} node.Config = raft.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond @@ -136,7 +180,12 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { node.Config.CommitTimeout = 5 * time.Millisecond node.id = raft.ServerID(id) node.Config.LocalID = node.id - node.Store = raft.NewInmemStore() + if store != nil { + node.Store = convertInMemStore(store) + } else { + node.Store = raft.NewInmemStore() + } + node.Snap = raft.NewInmemSnapshotStore() node.fsm = &raft.MockFSM{} var err error @@ -145,9 +194,11 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { node.raft, err = raft.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) + return node } -func initLatest(t *testing.T, node *RaftLatest, id string) { +func InitLatest(t *testing.T, id string) RaftNode { + node := RaftLatest{} node.Config = raftlatest.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond @@ -155,6 +206,7 @@ func initLatest(t *testing.T, node *RaftLatest, id string) { node.Config.CommitTimeout = 5 * time.Millisecond node.id = raftlatest.ServerID(id) node.Config.LocalID = node.id + node.Store = raftlatest.NewInmemStore() node.Snap = raftlatest.NewInmemSnapshotStore() node.fsm = &raftlatest.MockFSM{} @@ -164,4 +216,53 @@ func initLatest(t *testing.T, node *RaftLatest, id string) { node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) + return node +} + +func convertLog(ll *raftlatest.Log) *raft.Log { + l := new(raft.Log) + l.Index = ll.Index + l.AppendedAt = ll.AppendedAt + l.Type = raft.LogType(ll.Type) + l.Term = ll.Term + l.Data = ll.Data + l.Extensions = ll.Extensions + return l +} + +var ( + keyCurrentTerm = []byte("CurrentTerm") + keyLastVoteTerm = []byte("LastVoteTerm") + keyLastVoteCand = []byte("LastVoteCand") +) + +func convertInMemStore(s *raftlatest.InmemStore) *raft.InmemStore { + ss := raft.NewInmemStore() + fi, _ := s.FirstIndex() + li, _ := s.LastIndex() + for i := fi; i <= li; i++ { + log := new(raftlatest.Log) + s.GetLog(i, log) + ss.StoreLog(convertLog(log)) + } + + get, _ := ss.Get(keyCurrentTerm) + ss.Set(keyCurrentTerm, get) + + get, _ = ss.Get(keyLastVoteTerm) + ss.Set(keyLastVoteTerm, get) + + get, _ = ss.Get(keyLastVoteCand) + ss.Set(keyLastVoteCand, get) + + get64, _ := ss.GetUint64(keyCurrentTerm) + ss.SetUint64(keyCurrentTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteTerm) + ss.SetUint64(keyLastVoteTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteCand) + ss.SetUint64(keyLastVoteCand, get64) + + return ss } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 4b6eb9ff6..0f69594ca 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -10,11 +10,11 @@ import ( "time" ) -func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c testcluster.RaftCluster[T]) { +func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) { leader := func() string { for i := 0; i < c.Len(); i++ { - switch r := c.Raft(i).(type) { + switch r := c.Raft(c.ID(i)).(type) { case *raft.Raft: if r.State() == raft.Leader { return c.ID(i) @@ -36,6 +36,7 @@ func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c case <-ticker.C: id := leader() if id != "" { + fmt.Printf("got a leader %s\n", id) if id != oldLeader || oldLeader == "" { return } From 89da6428206de119badc5eae0d9f240cd65ebf9f Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:21:57 -0400 Subject: [PATCH 21/87] rename raft-latest to raft-previous --- raft-compat/go.mod | 4 +- raft-compat/rolling_upgrade_test.go | 72 ++++++++++++++--------------- raft-compat/testcluster/cluster.go | 51 ++++++++------------ raft-compat/utils/test_utils.go | 6 +-- 4 files changed, 61 insertions(+), 72 deletions(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 28e134a74..4fb0906a0 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,14 +16,14 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-latest => ./raft-latest +replace github.com/hashicorp/raft-previous-version => ./raft-latest replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/hashicorp/raft v1.2.0 - github.com/hashicorp/raft-latest v1.2.0 + github.com/hashicorp/raft-previous-version v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index f1d213a2f..6517a4b86 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -3,7 +3,7 @@ package raft_compat import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" @@ -18,17 +18,17 @@ func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftlatest.Configuration{} + configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftlatest.Server{ - ID: raftlatest.ServerID(rLatest.ID(i)), - Address: raftlatest.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(rLatest.ID(i)), + Address: raftprevious.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -36,13 +36,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers @@ -53,27 +53,27 @@ func TestRaft_RollingUpgrade(t *testing.T) { continue } - future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(i)), raftlatest.ServerAddress(rUIT.Addr(i)), 0, 0) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(i)), raftprevious.ServerAddress(rUIT.Addr(i)), 0, 0) utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() } - future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - fa := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(leaderIdx)), raftlatest.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) + fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) @@ -91,17 +91,17 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftlatest.Configuration{} + configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftlatest.Server{ - ID: raftlatest.ServerID(rLatest.ID(i)), - Address: raftlatest.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(rLatest.ID(i)), + Address: raftprevious.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -109,12 +109,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) - leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers leaderIdx := 0 @@ -125,12 +125,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { } // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) - rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() // Keep the store, to be passed to the upgraded node. store := rLatest.Store(rLatest.ID(i)) @@ -139,8 +139,8 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { rLatest.DeleteNode(rLatest.ID(i)) //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftlatest.InmemStore)) - future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.GetLocalID()), raftlatest.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftprevious.InmemStore)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) //Add the new node to the cluster rLatest.AddNode(rUIT) @@ -150,20 +150,20 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { time.Sleep(time.Second) //Apply some logs - future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) // keep a reference to the store store := rLatest.Store(getLeader.GetLocalID()) //Remove and shutdown the leader node - fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(getLeader.GetLocalID()), 0, 0) + fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(getLeader.GetLocalID()), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() // Delete the old leader node from the cluster rLatest.DeleteNode(getLeader.GetLocalID()) @@ -175,7 +175,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftlatest.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftprevious.InmemStore)) fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 9e04972ff..0e1c5b49a 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -3,7 +3,7 @@ package testcluster import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/stretchr/testify/require" "testing" "time" @@ -62,13 +62,13 @@ func (r *RaftCluster) Store(id string) interface{} { } type RaftLatest struct { - raft *raftlatest.Raft - trans *raftlatest.NetworkTransport - Config *raftlatest.Config - Store *raftlatest.InmemStore - Snap *raftlatest.InmemSnapshotStore - id raftlatest.ServerID - fsm *raftlatest.MockFSM + raft *raftprevious.Raft + trans *raftprevious.NetworkTransport + Config *raftprevious.Config + Store *raftprevious.InmemStore + Snap *raftprevious.InmemSnapshotStore + id raftprevious.ServerID + fsm *raftprevious.MockFSM } func (r RaftLatest) NumLogs() int { @@ -117,7 +117,7 @@ func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, coun return rc } -//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftlatest.InmemStore) *T { +//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftprevious.InmemStore) *T { // raft := new(T) // initNode(t, raft, name, store) // return raft @@ -136,17 +136,6 @@ func (r *RaftCluster) Len() int { return len(r.rafts) } -//func initNode(t *testing.T, node interface{}, id string, store *raftlatest.InmemStore) { -// switch node.(type) { -// case *RaftLatest: -// initLatest(t, node.(*RaftLatest), id) -// case *RaftUIT: -// initUIT(t, node.(*RaftUIT), id, convertInMemStore(store)) -// default: -// panic("invalid node type") -// } -//} - func (r *RaftCluster) AddNode(node RaftNode) { r.rafts = append([]RaftNode{node}, r.rafts...) } @@ -171,7 +160,7 @@ func InitUIT(t *testing.T, id string) RaftNode { return InitUITWithStore(t, id, nil) } -func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) RaftNode { +func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) RaftNode { node := RaftUIT{} node.Config = raft.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond @@ -199,27 +188,27 @@ func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) Raf func InitLatest(t *testing.T, id string) RaftNode { node := RaftLatest{} - node.Config = raftlatest.DefaultConfig() + node.Config = raftprevious.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftlatest.ServerID(id) + node.id = raftprevious.ServerID(id) node.Config.LocalID = node.id - node.Store = raftlatest.NewInmemStore() - node.Snap = raftlatest.NewInmemSnapshotStore() - node.fsm = &raftlatest.MockFSM{} + node.Store = raftprevious.NewInmemStore() + node.Snap = raftprevious.NewInmemSnapshotStore() + node.fsm = &raftprevious.MockFSM{} var err error - node.trans, err = raftlatest.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + node.trans, err = raftprevious.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, + node.raft, err = raftprevious.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) return node } -func convertLog(ll *raftlatest.Log) *raft.Log { +func convertLog(ll *raftprevious.Log) *raft.Log { l := new(raft.Log) l.Index = ll.Index l.AppendedAt = ll.AppendedAt @@ -236,12 +225,12 @@ var ( keyLastVoteCand = []byte("LastVoteCand") ) -func convertInMemStore(s *raftlatest.InmemStore) *raft.InmemStore { +func convertInMemStore(s *raftprevious.InmemStore) *raft.InmemStore { ss := raft.NewInmemStore() fi, _ := s.FirstIndex() li, _ := s.LastIndex() for i := fi; i <= li; i++ { - log := new(raftlatest.Log) + log := new(raftprevious.Log) s.GetLog(i, log) ss.StoreLog(convertLog(log)) } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 0f69594ca..035575d0b 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -3,7 +3,7 @@ package utils import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" @@ -19,8 +19,8 @@ func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) if r.State() == raft.Leader { return c.ID(i) } - case *raftlatest.Raft: - if r.State() == raftlatest.Leader { + case *raftprevious.Raft: + if r.State() == raftprevious.Leader { return c.ID(i) } } From 7cde48c0d8539301514e800a7828a85ab62e66e9 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:25:09 -0400 Subject: [PATCH 22/87] rename raft-latest to raft-previous submodule --- raft-compat/go.mod | 2 +- raft-compat/{raft-latest => raft-previous-version} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename raft-compat/{raft-latest => raft-previous-version} (100%) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 4fb0906a0..23ff53c7e 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,7 +16,7 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-previous-version => ./raft-latest +replace github.com/hashicorp/raft-previous-version => ./raft-previous-version replace github.com/hashicorp/raft => ../ diff --git a/raft-compat/raft-latest b/raft-compat/raft-previous-version similarity index 100% rename from raft-compat/raft-latest rename to raft-compat/raft-previous-version From 9b03730487188bd4541304689fbfb826fb653e18 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:26:33 -0400 Subject: [PATCH 23/87] fix submodule --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 96d94f795..28e632c85 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,3 @@ [submodule "raft-compat/raft-latest"] - path = raft-compat/raft-latest + path = raft-compat/raft-previous-version url = git@github.com:hashicorp/raft From 0c3c5079f9a058f416c8c5625aa4d089ccba9a45 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:29:47 -0400 Subject: [PATCH 24/87] remove printf --- raft-compat/rolling_upgrade_test.go | 8 ++++---- raft-compat/utils/test_utils.go | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 6517a4b86..b9664f623 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -68,7 +68,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) - // Check Leader haven't changed as we are not replacing the leader + // Check Leader haven't changed as we haven't replaced it yet a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) @@ -124,12 +124,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { continue } - // Check Leader haven't changed as we are not replacing the leader + // Check Leader haven't changed a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) + // + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() // Keep the store, to be passed to the upgraded node. diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 035575d0b..0b0b18c6d 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -36,7 +36,6 @@ func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) case <-ticker.C: id := leader() if id != "" { - fmt.Printf("got a leader %s\n", id) if id != oldLeader || oldLeader == "" { return } From b3dec1bb914e8dac0842e1c0997484c56f94ef2c Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 27 Jun 2023 10:20:16 -0400 Subject: [PATCH 25/87] use same name for recycled servers, add other leave scenarios --- raft-compat/rolling_upgrade_test.go | 228 ++++++++++++++++------------ 1 file changed, 134 insertions(+), 94 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index b9664f623..a18eaa25b 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,7 +1,6 @@ package raft_compat import ( - "fmt" "github.com/hashicorp/raft" raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" @@ -89,102 +88,143 @@ func TestRaft_RollingUpgrade(t *testing.T) { // This simulate the advised way of upgrading in Consul. func TestRaft_ReplaceUpgrade(t *testing.T) { - initCount := 3 - rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} - - for i := 0; i < initCount; i++ { - var err error - require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(rLatest.ID(i)), - Address: raftprevious.ServerAddress(rLatest.Addr(i)), - }) - } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) - if err := boot.Error(); err != nil { - t.Fatalf("bootstrap err: %v", err) + tcs := []struct { + Name string + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + { + Name: "leave before shutdown", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "shutdown without leave", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "leader transfer", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, } - utils.WaitForNewLeader(t, "", rLatest) - getLeader := rLatest.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - // Upgrade all the followers - leaderIdx := 0 - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == rLatest.ID(i) { - leaderIdx = i - continue - } - - // Check Leader haven't changed - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() - - // Keep the store, to be passed to the upgraded node. - store := rLatest.Store(rLatest.ID(i)) - - //Delete the node from the cluster - rLatest.DeleteNode(rLatest.ID(i)) - - //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftprevious.InmemStore)) - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, future) - //Add the new node to the cluster - rLatest.AddNode(rUIT) + for _, tc := range tcs { + t.Run(tc.Name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + configuration := raftprevious.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + continue + } + + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // + tc.Leave(t, cluster, cluster.ID(i)) + + // Keep the store, to be passed to the upgraded node. + store := cluster.Store(cluster.ID(i)) + id := cluster.ID(i) + + //Delete the node from the cluster + cluster.DeleteNode(cluster.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + cluster.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := cluster.Store(getLeader.GetLocalID()) + id := getLeader.GetLocalID() + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + cluster.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) + }) } +} - // Wait enough to have the configuration propagated. - time.Sleep(time.Second) - - //Apply some logs - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) - require.NoError(t, future.Error()) - - // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // keep a reference to the store - store := rLatest.Store(getLeader.GetLocalID()) - - //Remove and shutdown the leader node - fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(getLeader.GetLocalID()), 0, 0) +func leave(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() - - // Delete the old leader node from the cluster - rLatest.DeleteNode(getLeader.GetLocalID()) - oldLeaderID := getLeader.GetLocalID() - - // Wait for a new leader to be elected - utils.WaitForNewLeader(t, oldLeaderID, rLatest) - getLeader = rLatest.GetLeader() - require.NotEmpty(t, getLeader) - - // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftprevious.InmemStore)) - fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, fa) - - // Wait for new leader, (this happens because of not having prevote) - utils.WaitForNewLeader(t, "", rLatest) - newLeader := rUIT.GetLeaderID() - require.NotEmpty(t, newLeader) - - require.NotEqual(t, newLeader, leader) - - require.Equal(t, rUIT.NumLogs(), 2) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) } From 02e042f04bc63747b2e91a1644e0f2d0f76a7866 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 8 Nov 2022 16:21:38 -0500 Subject: [PATCH 26/87] prevote initial implementation --- commands.go | 4 +++ raft.go | 82 ++++++++++++++++++++++++++++++++++++++++++---------- raft_test.go | 1 - 3 files changed, 70 insertions(+), 17 deletions(-) diff --git a/commands.go b/commands.go index 1a74e0525..c36295e80 100644 --- a/commands.go +++ b/commands.go @@ -92,6 +92,7 @@ type RequestVoteRequest struct { // transfer. It is required for leadership transfer to work, because servers // wouldn't vote otherwise if they are aware of an existing leader. LeadershipTransfer bool + PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -113,6 +114,9 @@ type RequestVoteResponse struct { // Is the vote granted. Granted bool + + // Is it a preVote response + PreVote bool } // GetRPCHeader - See WithRPCHeader. diff --git a/raft.go b/raft.go index be6743441..cf9bdf9a6 100644 --- a/raft.go +++ b/raft.go @@ -286,7 +286,7 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - voteCh := r.electSelf() + prevoteCh := r.electSelf(true) // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -300,9 +300,10 @@ func (r *Raft) runCandidate() { // Tally the votes, need a simple majority grantedVotes := 0 + preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - + var voteCh <-chan *voteResult for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -311,6 +312,43 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.processRPC(rpc) + case vote := <-prevoteCh: + r.mainThreadSaturation.working() + r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + // Check if the term is greater than ours, bail + if vote.Term > r.getCurrentTerm() && !vote.PreVote { + r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } + + // Check if the vote is granted + if vote.Granted { + if !vote.PreVote { + grantedVotes++ + r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + } else { + preVoteGrantedVotes++ + r.logger.Debug("prevote granted", "from", vote.voterID, "term", vote.Term, "tally", preVoteGrantedVotes) + } + } + + // Check if we've become the leader + if grantedVotes >= votesNeeded { + r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) + r.setState(Leader) + r.setLeader(r.localAddr, r.localID) + //r.setCurrentTerm(term) + return + } + // Check if we've become the leader + if preVoteGrantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + preVoteGrantedVotes = 0 + grantedVotes = 0 + voteCh = r.electSelf(false) + } case vote := <-voteCh: r.mainThreadSaturation.working() // Check if the term is greater than ours, bail @@ -334,7 +372,6 @@ func (r *Raft) runCandidate() { r.setLeader(r.localAddr, r.localID) return } - case c := <-r.configurationChangeCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader @@ -1542,6 +1579,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, + PreVote: req.PreVote, } var rpcErr error defer func() { @@ -1597,8 +1635,10 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - r.setState(Follower) - r.setCurrentTerm(req.Term) + if !req.PreVote { + r.setState(Follower) + r.setCurrentTerm(req.Term) + } resp.Term = req.Term } @@ -1655,11 +1695,12 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if err := r.persistVote(req.Term, candidateBytes); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return + if !req.PreVote { + if err := r.persistVote(req.Term, candidateBytes); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return + } } - resp.Granted = true r.setLastContact() } @@ -1825,23 +1866,26 @@ type voteResult struct { // ourself. This has the side affecting of incrementing the current term. The // response channel returned is used to wait for all the responses (including a // vote for ourself). This must only be called from the main thread. -func (r *Raft) electSelf() <-chan *voteResult { +func (r *Raft) electSelf(preVote bool) <-chan *voteResult { // Create a response channel respCh := make(chan *voteResult, len(r.configurations.latest.Servers)) // Increment the term - r.setCurrentTerm(r.getCurrentTerm() + 1) - + newTerm := r.getCurrentTerm() + 1 + if !preVote { + r.setCurrentTerm(newTerm) + } // Construct the request lastIdx, lastTerm := r.getLastEntry() req := &RequestVoteRequest{ RPCHeader: r.getRPCHeader(), - Term: r.getCurrentTerm(), + Term: newTerm, // this is needed for retro compatibility, before RPCHeader.Addr was added Candidate: r.trans.EncodePeer(r.localID, r.localAddr), LastLogIndex: lastIdx, LastLogTerm: lastTerm, LeadershipTransfer: r.candidateFromLeadershipTransfer, + PreVote: preVote, } // Construct a function to ask for a vote @@ -1857,7 +1901,9 @@ func (r *Raft) electSelf() <-chan *voteResult { "term", req.Term) resp.Term = req.Term resp.Granted = false + resp.PreVote = req.PreVote } + r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1868,9 +1914,11 @@ func (r *Raft) electSelf() <-chan *voteResult { if server.ID == r.localID { r.logger.Debug("voting for self", "term", req.Term, "id", r.localID) // Persist a vote for ourselves - if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return nil + if !preVote { + if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return nil + } } // Include our own vote respCh <- &voteResult{ @@ -1878,6 +1926,7 @@ func (r *Raft) electSelf() <-chan *voteResult { RPCHeader: r.getRPCHeader(), Term: req.Term, Granted: true, + PreVote: req.PreVote, }, voterID: r.localID, } @@ -1904,6 +1953,7 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { + r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) diff --git a/raft_test.go b/raft_test.go index df9712b1c..06c6b7bba 100644 --- a/raft_test.go +++ b/raft_test.go @@ -3013,7 +3013,6 @@ func TestRaft_VoteWithNoIDNoAddr(t *testing.T) { var resp RequestVoteResponse followerT := c.trans[c.IndexOf(followers[1])] c.Partition([]ServerAddress{leader.localAddr}) - time.Sleep(c.propagateTimeout) // wait for the remaining follower to trigger an election waitForState(follower, Candidate) From a359f5c6ce144de38ccb02e0d79a5058de7e9003 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 14:30:30 -0500 Subject: [PATCH 27/87] add config and relevant tests --- api.go | 4 ++++ config.go | 3 +++ raft.go | 21 ++++++++++++++------ raft_test.go | 56 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 78 insertions(+), 6 deletions(-) diff --git a/api.go b/api.go index 23077dc35..6796aafce 100644 --- a/api.go +++ b/api.go @@ -213,6 +213,9 @@ type Raft struct { // mainThreadSaturation measures the saturation of the main raft goroutine. mainThreadSaturation *saturationMetric + + // preVote control if the pre-vote feature is activated + preVote bool } // BootstrapCluster initializes a server's storage with the given cluster @@ -560,6 +563,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna leaderNotifyCh: make(chan struct{}, 1), followerNotifyCh: make(chan struct{}, 1), mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), + preVote: conf.PreVote, } r.conf.Store(*conf) diff --git a/config.go b/config.go index b97b4338f..715049896 100644 --- a/config.go +++ b/config.go @@ -234,6 +234,9 @@ type Config struct { // skipStartup allows NewRaft() to bypass all background work goroutines skipStartup bool + + // PreVote activate the pre-vote feature + PreVote bool } func (conf *Config) getOrCreateLogger() hclog.Logger { diff --git a/raft.go b/raft.go index cf9bdf9a6..d6b431c8f 100644 --- a/raft.go +++ b/raft.go @@ -286,7 +286,13 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - prevoteCh := r.electSelf(true) + var voteCh <-chan *voteResult + var prevoteCh <-chan *voteResult + if r.preVote { + prevoteCh = r.electSelf(true) + } else { + voteCh = r.electSelf(false) + } // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -303,7 +309,7 @@ func (r *Raft) runCandidate() { preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - var voteCh <-chan *voteResult + for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -1573,13 +1579,16 @@ func (r *Raft) processConfigurationLogEntry(entry *Log) error { func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) r.observe(*req) - + preVote := req.PreVote + if !r.preVote { + preVote = false + } // Setup a response resp := &RequestVoteResponse{ RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: req.PreVote, + PreVote: preVote, } var rpcErr error defer func() { @@ -1635,7 +1644,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !req.PreVote { + if !preVote { r.setState(Follower) r.setCurrentTerm(req.Term) } @@ -1695,7 +1704,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if !req.PreVote { + if !preVote { if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return diff --git a/raft_test.go b/raft_test.go index 06c6b7bba..24b7ec789 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2063,6 +2063,62 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } +func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { + + tcs := []struct { + name string + prevoteNum int + noprevoteNum int + }{ + {"majority no pre-vote", 2, 3}, + {"majority pre-vote", 3, 2}, + {"all pre-vote", 3, 0}, + {"all no pre-vote", 0, 3}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + + // Make majority cluster. + majority := tc.prevoteNum + minority := tc.noprevoteNum + if tc.prevoteNum < tc.noprevoteNum { + majority = tc.noprevoteNum + minority = tc.prevoteNum + } + + conf := inmemConfig(t) + conf.PreVote = tc.prevoteNum > tc.noprevoteNum + c := MakeCluster(majority, t, conf) + defer c.Close() + + // Set up another server speaking protocol version 2. + conf = inmemConfig(t) + conf.PreVote = tc.prevoteNum < tc.noprevoteNum + c1 := MakeClusterNoBootstrap(minority, t, conf) + + // Merge clusters. + c.Merge(c1) + c.FullyConnect() + + if len(c1.rafts) > 0 { + future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + if err := future.Error(); err != nil { + t.Fatalf("err: %v", err) + } + } + time.Sleep(c.propagateTimeout * 10) + + leaderOld := c.Leader() + c.Followers() + c.Partition([]ServerAddress{leaderOld.localAddr}) + time.Sleep(c.propagateTimeout * 3) + leader := c.Leader() + require.NotEqual(t, leader.leaderID, leaderOld.leaderID) + }) + } + +} + func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { conf := inmemConfig(t) conf.ProtocolVersion = 3 From e0c539dd40c34c9992ef3d25db7515ab50717acb Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 16:11:36 -0500 Subject: [PATCH 28/87] remove extra comments, fix a case where newer term is discovered for prevote --- raft.go | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/raft.go b/raft.go index d6b431c8f..904b6e453 100644 --- a/raft.go +++ b/raft.go @@ -317,8 +317,9 @@ func (r *Raft) runCandidate() { case rpc := <-r.rpcCh: r.mainThreadSaturation.working() r.processRPC(rpc) - case vote := <-prevoteCh: + // This a pre-vote case but could lead to winning an election, in the case that majority of other nodes + // don't support pre-vote or have pre-vote deactivated. r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail @@ -328,6 +329,12 @@ func (r *Raft) runCandidate() { r.setCurrentTerm(vote.Term) return } + if vote.Term > term && vote.PreVote { + r.logger.Debug("newer term discovered on pre-vote, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } // Check if the vote is granted if vote.Granted { @@ -341,14 +348,17 @@ func (r *Raft) runCandidate() { } // Check if we've become the leader + // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) + // set our state to leader and our term to the pre-vote term. if grantedVotes >= votesNeeded { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) - //r.setCurrentTerm(term) + + r.setCurrentTerm(term) return } - // Check if we've become the leader + // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 @@ -1912,7 +1922,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { resp.Granted = false resp.PreVote = req.PreVote } - r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1962,7 +1971,6 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { - r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) From cfa7e87be42d208c4d0c2d0ddadb28255e0ce1d7 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 10 Nov 2022 11:23:51 -0500 Subject: [PATCH 29/87] fix to reset timeout after pre-vote and fix split vote (pre-vote,vote) case. --- raft.go | 5 ++++- raft_test.go | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index 904b6e453..8ff2b5903 100644 --- a/raft.go +++ b/raft.go @@ -350,7 +350,9 @@ func (r *Raft) runCandidate() { // Check if we've become the leader // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) // set our state to leader and our term to the pre-vote term. - if grantedVotes >= votesNeeded { + // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 + // we can flip our vote to an actual vote. + if grantedVotes >= votesNeeded-1 { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) @@ -363,6 +365,7 @@ func (r *Raft) runCandidate() { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 grantedVotes = 0 + electionTimer = randomTimeout(electionTimeout) voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index 24b7ec789..f82f166fc 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2063,7 +2063,7 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } -func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { +func TestRaft_PreVoteMixedCluster(t *testing.T) { tcs := []struct { name string @@ -2072,6 +2072,8 @@ func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { }{ {"majority no pre-vote", 2, 3}, {"majority pre-vote", 3, 2}, + {"majority no pre-vote", 1, 2}, + {"majority pre-vote", 2, 1}, {"all pre-vote", 3, 0}, {"all no pre-vote", 0, 3}, } From 2dad2f307f4c714ddb1aa46d898a8f7a2a2aeb07 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 14 Nov 2022 13:59:32 -0500 Subject: [PATCH 30/87] write upgrade tests that include prevotes --- fuzzy/go.sum | 1 + raft-compat/prevote_test.go | 133 ++++++++++++++++++ raft-compat/raft-previous-version | 2 +- raft-compat/rolling_upgrade_test.go | 210 ++++++++++++++++++---------- raft-compat/testcluster/cluster.go | 9 +- raft.go | 7 +- raft_test.go | 4 +- 7 files changed, 285 insertions(+), 81 deletions(-) create mode 100644 raft-compat/prevote_test.go diff --git a/fuzzy/go.sum b/fuzzy/go.sum index e013c3a5d..c2025336c 100644 --- a/fuzzy/go.sum +++ b/fuzzy/go.sum @@ -95,6 +95,7 @@ github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1F github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go new file mode 100644 index 000000000..e3f27410e --- /dev/null +++ b/raft-compat/prevote_test.go @@ -0,0 +1,133 @@ +package raft_compat + +import ( + "github.com/hashicorp/raft" + raftprevious "github.com/hashicorp/raft-previous-version" + "github.com/hashicorp/raft/compat/testcluster" + "github.com/hashicorp/raft/compat/utils" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { + leave := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + tcs := []struct { + name string + prevoteNum int + noprevoteNum int + preVoteEnabled bool + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + {"majority latest, prevote off", 1, 2, false, leave}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + configuration := raftprevious.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + continue + } + + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // + tc.Leave(t, cluster, cluster.ID(i)) + + // Keep the store, to be passed to the upgraded node. + store := cluster.Store(cluster.ID(i)) + id := cluster.ID(i) + + //Delete the node from the cluster + cluster.DeleteNode(cluster.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { + config.PreVote = true + }) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + cluster.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := cluster.Store(getLeader.GetLocalID()) + id := getLeader.GetLocalID() + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + cluster.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVote = true }) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) + }) + } + +} diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index 8fdc4ce5b..df0cdeebe 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 +Subproject commit df0cdeebead4b031824392c2bb19568eeb300112 diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index a18eaa25b..516e3da44 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,6 +1,7 @@ package raft_compat import ( + "fmt" "github.com/hashicorp/raft" raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" @@ -14,73 +15,143 @@ import ( // wait for it to join the cluster and remove one of the old nodes, until all nodes // are cycled func TestRaft_RollingUpgrade(t *testing.T) { + tcs := []struct { + Name string + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + { + Name: "leave before shutdown", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "leader transfer", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + utils.WaitForNewLeader(t, id, cluster) + } + switch cluster.GetLeader().GetRaft().(type) { + case *raftprevious.Raft: + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + case *raft.Raft: + fr := cluster.GetLeader().GetRaft().(*raft.Raft).RemoveServer(raft.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + + }, + }, + } + + for _, tc := range tcs { + t.Run(tc.Name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + configuration := raftprevious.Configuration{} - initCount := 3 - rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + leaderIdx := 0 - for i := 0; i < initCount; i++ { - var err error - require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(rLatest.ID(i)), - Address: raftprevious.ServerAddress(rLatest.Addr(i)), + followers := make([]string, 0) + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + leaderIdx = i + continue + } + followers = append(followers, cluster.ID(i)) + } + + for _, f := range followers { + require.NotEqual(t, f, getLeader.GetLocalID()) + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + //Create an upgraded node with the store + rUIT := testcluster.InitUIT(t, fmt.Sprintf("%s-new", f)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + + //Add the new node to the cluster + + tc.Leave(t, cluster, f) + + //Delete the node from the cluster + cluster.AddNode(rUIT) + cluster.DeleteNode(f) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUIT(t, fmt.Sprintf("raftNew-%d", leaderIdx)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + require.NotEqual(t, newLeader, leader) + + cluster.DeleteNode(getLeader.GetLocalID()) + require.Equal(t, rUIT.NumLogs(), 2) }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) - if err := boot.Error(); err != nil { - t.Fatalf("bootstrap err: %v", err) - } - utils.WaitForNewLeader(t, "", rLatest) - getLeader := rLatest.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - - rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - - // Upgrade all the followers - leaderIdx := 0 - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == rLatest.ID(i) { - leaderIdx = i - continue - } - - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(i)), raftprevious.ServerAddress(rUIT.Addr(i)), 0, 0) - - utils.WaitFuture(t, future) - // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() - } - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) - require.NoError(t, future.Error()) - - fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) - utils.WaitFuture(t, fa) - - // Check Leader haven't changed as we haven't replaced it yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) - utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() - utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) - newLeader := rUIT.GetLeader() - require.NotEmpty(t, newLeader) - aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() - require.NotEqual(t, aNew, leader) - - require.Equal(t, newLeader.NumLogs(), 2) - } // TestRaft_ReplaceUpgrade This test perform a rolling upgrade by removing an old node, @@ -124,7 +195,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { for _, tc := range tcs { t.Run(tc.Name, func(t *testing.T) { initCount := 3 - cluster := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { @@ -171,7 +242,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { cluster.DeleteNode(cluster.ID(i)) //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) {}) future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) //Add the new node to the cluster @@ -206,7 +277,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) {}) fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) @@ -221,10 +292,3 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { }) } } - -func leave(t *testing.T, cluster testcluster.RaftCluster, id string) { - fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) - utils.WaitFuture(t, fr) - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) -} diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 0e1c5b49a..64fe04c9c 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -117,6 +117,10 @@ func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, coun return rc } +func NewPreviousRaftCluster(t *testing.T, count int, name string) RaftCluster { + return NewRaftCluster(t, InitLatest, count, name) +} + //func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftprevious.InmemStore) *T { // raft := new(T) // initNode(t, raft, name, store) @@ -157,12 +161,13 @@ func (r *RaftCluster) GetIndex(id string) int { } func InitUIT(t *testing.T, id string) RaftNode { - return InitUITWithStore(t, id, nil) + return InitUITWithStore(t, id, nil, func(config *raft.Config) {}) } -func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) RaftNode { +func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore, cfgMod func(config *raft.Config)) RaftNode { node := RaftUIT{} node.Config = raft.DefaultConfig() + cfgMod(node.Config) node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond diff --git a/raft.go b/raft.go index 8ff2b5903..9a2290d36 100644 --- a/raft.go +++ b/raft.go @@ -323,7 +323,7 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail - if vote.Term > r.getCurrentTerm() && !vote.PreVote { + if vote.Term > term && !vote.PreVote { r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) r.setState(Follower) r.setCurrentTerm(vote.Term) @@ -361,11 +361,12 @@ func (r *Raft) runCandidate() { return } // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + if preVoteGrantedVotes+grantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 grantedVotes = 0 electionTimer = randomTimeout(electionTimeout) + prevoteCh = nil voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index f82f166fc..928763b2d 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2102,8 +2102,8 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { c.Merge(c1) c.FullyConnect() - if len(c1.rafts) > 0 { - future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + for _, r := range c1.rafts { + future := c.Leader().AddVoter(r.localID, r.localAddr, 0, 0) if err := future.Error(); err != nil { t.Fatalf("err: %v", err) } From 2b09f8cbea32c4b471cf13b9adff631454d50cff Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 22 Aug 2023 10:31:25 -0400 Subject: [PATCH 31/87] add more test cases --- raft-compat/prevote_test.go | 38 +++++++++++++++++++++++++------------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index e3f27410e..c80ec731e 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -11,7 +11,15 @@ import ( ) func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { - leave := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + leaveTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { if cluster.GetLeader().GetLocalID() == id { transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() utils.WaitFuture(t, transfer) @@ -20,21 +28,27 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { utils.WaitFuture(t, f) } tcs := []struct { - name string - prevoteNum int - noprevoteNum int - preVoteEnabled bool - Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + name string + numNodes int + preVote bool + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) }{ - {"majority latest, prevote off", 1, 2, false, leave}, + {"no prevote -> prevote (leave transfer)", 3, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer)", 3, true, leaveNoTransfer}, + {"no prevote -> prevote (leave transfer) 5", 5, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer) 5", 5, true, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer)", 3, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer)", 3, false, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer) 5", 5, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer) 5", 5, false, leaveNoTransfer}, } for _, tc := range tcs { t.Run(tc.name, func(t *testing.T) { - initCount := 3 - cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + + cluster := testcluster.NewPreviousRaftCluster(t, tc.numNodes, "raftNode") configuration := raftprevious.Configuration{} - for i := 0; i < initCount; i++ { + for i := 0; i < tc.numNodes; i++ { var err error require.NoError(t, err) configuration.Servers = append(configuration.Servers, raftprevious.Server{ @@ -58,7 +72,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers - for i := 0; i < initCount; i++ { + for i := 0; i < tc.numNodes; i++ { if getLeader.GetLocalID() == cluster.ID(i) { continue } @@ -79,7 +93,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { //Create an upgraded node with the store rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { - config.PreVote = true + config.PreVote = tc.preVote }) future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) From 785b127115dc43d291887265f55c686a3c445987 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 22 Aug 2023 10:33:17 -0400 Subject: [PATCH 32/87] fix submodule version --- raft-compat/raft-previous-version | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index df0cdeebe..8fdc4ce5b 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit df0cdeebead4b031824392c2bb19568eeb300112 +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From ea82333b65c85da60497660cbffcd2b055739755 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:51:46 -0500 Subject: [PATCH 33/87] go mod tidy --- fuzzy/go.mod | 2 +- fuzzy/go.sum | 1 + raft-compat/go.mod | 4 ++-- raft-compat/go.sum | 2 ++ 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/fuzzy/go.mod b/fuzzy/go.mod index c1c51543a..196abb440 100644 --- a/fuzzy/go.mod +++ b/fuzzy/go.mod @@ -3,7 +3,7 @@ module github.com/hashicorp/raft/fuzzy go 1.20 require ( - github.com/hashicorp/go-hclog v1.5.0 + github.com/hashicorp/go-hclog v1.6.2 github.com/hashicorp/go-msgpack/v2 v2.1.1 github.com/hashicorp/raft v1.2.0 github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea diff --git a/fuzzy/go.sum b/fuzzy/go.sum index 1f99677f0..ca609c3b3 100644 --- a/fuzzy/go.sum +++ b/fuzzy/go.sum @@ -33,6 +33,7 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 23ff53c7e..f1a5a5866 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -7,13 +7,13 @@ require github.com/stretchr/testify v1.8.4 require ( github.com/armon/go-metrics v0.4.1 // indirect github.com/fatih/color v1.13.0 // indirect - github.com/hashicorp/go-hclog v1.5.0 // indirect + github.com/hashicorp/go-hclog v1.6.2 // indirect github.com/hashicorp/go-immutable-radix v1.0.0 // indirect github.com/hashicorp/go-msgpack v0.5.5 // indirect github.com/hashicorp/golang-lru v0.5.0 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect - golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect + golang.org/x/sys v0.13.0 // indirect ) replace github.com/hashicorp/raft-previous-version => ./raft-previous-version diff --git a/raft-compat/go.sum b/raft-compat/go.sum index ed3b7f040..fffbbd864 100644 --- a/raft-compat/go.sum +++ b/raft-compat/go.sum @@ -31,6 +31,7 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= @@ -113,6 +114,7 @@ golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 h1:nonptSpoQ4vQjyraW20DXPAglgQfVnM9ZC6MmNLMR60= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= From 8786e2a934020b67f86f1ea637581d5173f6360e Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:53:18 -0500 Subject: [PATCH 34/87] update pervious version to v1.6.0 --- raft-compat/raft-previous-version | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index 8fdc4ce5b..b96f998ff 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 +Subproject commit b96f998ff7e752c7eb68615f086a9c52008a40b6 From 16a8c764ffca378e0e47c5a7bc4a95c84c2401e2 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:54:21 -0500 Subject: [PATCH 35/87] fix merge duplication --- raft.go | 1 - 1 file changed, 1 deletion(-) diff --git a/raft.go b/raft.go index 02ba3505a..4ef3ade06 100644 --- a/raft.go +++ b/raft.go @@ -1925,7 +1925,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { LastLogTerm: lastTerm, LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), PreVote: preVote, - PreVote: preVote, } // Construct a function to ask for a vote From ae369f3d65de93c5c57ce46cfb227ffc7eeba34e Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 6 Jun 2023 09:48:32 -0400 Subject: [PATCH 36/87] add submodule and first iteration of multi-version tests rename test and pin submodule to version 1.5.0 rename test file --- .gitmodules | 6 +- integ_test.go | 2 +- raft-compat/go.mod | 4 +- raft-compat/raftrs | 1 + raft-compat/rolling_upgrade_test.go | 255 ++++++---------------------- raft_test.go | 62 +++---- testing.go | 37 +++- 7 files changed, 129 insertions(+), 238 deletions(-) create mode 160000 raft-compat/raftrs diff --git a/.gitmodules b/.gitmodules index cbcd5cc91..ed959a7cf 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,3 @@ -[submodule "raft-compat/raft-latest"] - path = raft-compat/raft-previous-version - url = https://github.com/hashicorp/raft.git +[submodule "raft-compat/raftrs"] + path = raft-compat/raftrs + url = git@github.com:hashicorp/raft diff --git a/integ_test.go b/integ_test.go index 61557a355..897857794 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,7 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(tb testing.TB, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { - conf = inmemConfig(tb) + conf = InmemConfig(t) } dir, err := os.MkdirTemp("", "raft") diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 23ff53c7e..40db0858b 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,14 +16,14 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-previous-version => ./raft-previous-version +replace github.com/dhiayachi/raft v1.0.0 => ./raftrs replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dhiayachi/raft v1.0.0 github.com/hashicorp/raft v1.2.0 - github.com/hashicorp/raft-previous-version v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/raft-compat/raftrs b/raft-compat/raftrs new file mode 160000 index 000000000..8fdc4ce5b --- /dev/null +++ b/raft-compat/raftrs @@ -0,0 +1 @@ +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index a18eaa25b..65c511e61 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,230 +1,87 @@ package raft_compat import ( + "fmt" + raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" - raftprevious "github.com/hashicorp/raft-previous-version" - "github.com/hashicorp/raft/compat/testcluster" - "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" "testing" "time" ) -// TestRaft_RollingUpgrade This test perform a rolling upgrade by adding a new node, -// wait for it to join the cluster and remove one of the old nodes, until all nodes -// are cycled -func TestRaft_RollingUpgrade(t *testing.T) { +func TestRaft_RollingUpgrade_NoLeaderUpgrade(t *testing.T) { initCount := 3 - rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} + rafts := make([]*raft.Raft, initCount) + trans := make([]*raft.NetworkTransport, initCount) + confs := make([]*raft.Config, initCount) + stores := make([]*raft.InmemStore, initCount) + snaps := make([]*raft.InmemSnapshotStore, initCount) + id := make([]raft.ServerID, initCount) + configuration := raft.Configuration{} for i := 0; i < initCount; i++ { + confs[i] = raft.InmemConfig(t) + id[i] = raft.ServerID(fmt.Sprintf("grpc%d", i)) + confs[i].LocalID = id[i] + stores[i] = raft.NewInmemStore() + snaps[i] = raft.NewInmemSnapshotStore() var err error + trans[i], err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(rLatest.ID(i)), - Address: raftprevious.ServerAddress(rLatest.Addr(i)), + rafts[i], err = raft.NewRaft(confs[i], &raft.MockFSM{}, stores[i], stores[i], snaps[i], trans[i]) + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raft.Server{ + ID: id[i], + Address: trans[i].LocalAddr(), }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) + boot := rafts[0].BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - utils.WaitForNewLeader(t, "", rLatest) - getLeader := rLatest.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - - rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - - // Upgrade all the followers - leaderIdx := 0 - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == rLatest.ID(i) { - leaderIdx = i - continue - } + lCh := rafts[0].LeaderCh() - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(i)), raftprevious.ServerAddress(rUIT.Addr(i)), 0, 0) + after := time.After(time.Second) - utils.WaitFuture(t, future) - // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() + select { + case <-after: + t.Fatalf("timedout") + case <-lCh: } - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + a := rafts[0].Leader() + require.NotEmpty(t, a) + future := rafts[0].Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) - utils.WaitFuture(t, fa) - - // Check Leader haven't changed as we haven't replaced it yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) - utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() - utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) - newLeader := rUIT.GetLeader() - require.NotEmpty(t, newLeader) - aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() - require.NotEqual(t, aNew, leader) - - require.Equal(t, newLeader.NumLogs(), 2) - -} - -// TestRaft_ReplaceUpgrade This test perform a rolling upgrade by removing an old node, -// and create a new node with the same store until all old nodes are cycled to new nodes. -// This simulate the advised way of upgrading in Consul. -func TestRaft_ReplaceUpgrade(t *testing.T) { - - tcs := []struct { - Name string - Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) - }{ - { - Name: "leave before shutdown", - Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { - fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) - utils.WaitFuture(t, fr) - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) - }, - }, - { - Name: "shutdown without leave", - Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) - }, - }, - { - Name: "leader transfer", - Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { - if cluster.GetLeader().GetLocalID() == id { - transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() - utils.WaitFuture(t, transfer) - } - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) - }, - }, - } - - for _, tc := range tcs { - t.Run(tc.Name, func(t *testing.T) { - initCount := 3 - cluster := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} - - for i := 0; i < initCount; i++ { - var err error - require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(cluster.ID(i)), - Address: raftprevious.ServerAddress(cluster.Addr(i)), - }) - } - raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) - if err := boot.Error(); err != nil { - t.Fatalf("bootstrap err: %v", err) - } - utils.WaitForNewLeader(t, "", cluster) - getLeader := cluster.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - // Upgrade all the followers - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == cluster.ID(i) { - continue - } - - // Check Leader haven't changed - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // - tc.Leave(t, cluster, cluster.ID(i)) - - // Keep the store, to be passed to the upgraded node. - store := cluster.Store(cluster.ID(i)) - id := cluster.ID(i) - - //Delete the node from the cluster - cluster.DeleteNode(cluster.ID(i)) - - //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, future) - //Add the new node to the cluster - cluster.AddNode(rUIT) - } - - // Wait enough to have the configuration propagated. - time.Sleep(time.Second) - - //Apply some logs - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) - require.NoError(t, future.Error()) - - // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // keep a reference to the store - store := cluster.Store(getLeader.GetLocalID()) - id := getLeader.GetLocalID() - - //Remove and shutdown the leader node - tc.Leave(t, cluster, getLeader.GetLocalID()) - - // Delete the old leader node from the cluster - cluster.DeleteNode(getLeader.GetLocalID()) - oldLeaderID := getLeader.GetLocalID() - - // Wait for a new leader to be elected - utils.WaitForNewLeader(t, oldLeaderID, cluster) - getLeader = cluster.GetLeader() - require.NotEmpty(t, getLeader) - - // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) - fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, fa) + newRafts := make([]*raftrs.Raft, initCount) + newTrans := make([]*raftrs.NetworkTransport, initCount) + newConfs := make([]*raftrs.Config, initCount) + newStores := make([]*raftrs.InmemStore, initCount) + newSnaps := make([]*raftrs.InmemSnapshotStore, initCount) + leader, _ := rafts[0].LeaderWithID() + require.NotEmpty(t, leader) + for i := 1; i < len(rafts); i++ { + newConfs[i] = raftrs.DefaultConfig() + newConfs[i].LocalID = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) + newStores[i] = raftrs.NewInmemStore() + newSnaps[i] = raftrs.NewInmemSnapshotStore() + var err error + newTrans[i], err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + newRafts[i], err = raftrs.NewRaft(newConfs[i], &raftrs.MockFSM{}, newStores[i], newStores[i], newSnaps[i], newTrans[i]) + rafts[0].AddVoter("grpc4", raft.ServerAddress(newTrans[i].LocalAddr()), 0, 0) - // Wait for new leader, (this happens because of not having prevote) - utils.WaitForNewLeader(t, "", cluster) - newLeader := rUIT.GetLeaderID() - require.NotEmpty(t, newLeader) + time.Sleep(1 * time.Second) - require.NotEqual(t, newLeader, leader) + // Check Leader haven't changed as we are not replacing the leader + a := rafts[0].Leader() + require.Equal(t, a, leader) + rafts[0].RemoveServer(id[i], 0, 0) + rafts[i].Shutdown() - require.Equal(t, rUIT.NumLogs(), 2) - }) } -} + future = rafts[0].Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) -func leave(t *testing.T, cluster testcluster.RaftCluster, id string) { - fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) - utils.WaitFuture(t, fr) - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) } diff --git a/raft_test.go b/raft_test.go index 3eaf1e3ce..85f3f6fa8 100644 --- a/raft_test.go +++ b/raft_test.go @@ -152,7 +152,7 @@ func TestRaft_RecoverCluster(t *testing.T) { snapshotThreshold := 5 runRecover := func(t *testing.T, applies int) { var err error - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 conf.SnapshotThreshold = uint64(snapshotThreshold) c := MakeCluster(3, t, conf) @@ -295,7 +295,7 @@ func TestRaft_HasExistingState(t *testing.T) { } func TestRaft_SingleNode(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -501,7 +501,7 @@ func TestRaft_ApplyNonLeader(t *testing.T) { func TestRaft_ApplyConcurrent(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout c := MakeCluster(3, t, conf) @@ -552,7 +552,7 @@ func TestRaft_ApplyConcurrent(t *testing.T) { func TestRaft_ApplyConcurrent_Timeout(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.CommitTimeout = 1 * time.Millisecond conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout @@ -620,7 +620,7 @@ func TestRaft_JoinNode(t *testing.T) { func TestRaft_JoinNode_ConfigStore(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := makeCluster(t, &MakeClusterOpts{ Peers: 1, Bootstrap: true, @@ -777,7 +777,7 @@ func TestRaft_RemoveLeader(t *testing.T) { func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -840,7 +840,7 @@ func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { func TestRaft_RemoveFollower_SplitCluster(t *testing.T) { // Make a cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(4, t, conf) defer c.Close() @@ -955,7 +955,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { func TestRaft_SnapshotRestore(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1015,7 +1015,7 @@ func TestRaft_SnapshotRestore(t *testing.T) { func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 opts := &MakeClusterOpts{ Peers: 1, @@ -1097,7 +1097,7 @@ func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { func TestRaft_SnapshotRestore_Progress(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1215,7 +1215,7 @@ func (b *lockedBytesBuffer) String() string { // up. func TestRaft_NoRestoreOnStart(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 conf.NoSnapshotRestoreOnStart = true c := MakeCluster(1, t, conf) @@ -1260,7 +1260,7 @@ func TestRaft_NoRestoreOnStart(t *testing.T) { func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { var err error // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 1 conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) @@ -1365,7 +1365,7 @@ func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { func TestRaft_AutoSnapshot(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.SnapshotInterval = conf.CommitTimeout * 2 conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 @@ -1395,7 +1395,7 @@ func TestRaft_AutoSnapshot(t *testing.T) { func TestRaft_UserSnapshot(t *testing.T) { // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) @@ -1435,7 +1435,7 @@ func snapshotAndRestore(t *testing.T, offset uint64, monotonicLogStore bool, res t.Helper() // Make the cluster. - conf := inmemConfig(t) + conf := InmemConfig(t) // snapshot operations perform some file IO operations. // increase times out to account for that @@ -1597,7 +1597,7 @@ func TestRaft_UserRestore(t *testing.T) { func TestRaft_SendSnapshotFollower(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1639,7 +1639,7 @@ func TestRaft_SendSnapshotFollower(t *testing.T) { func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { // Make the cluster - conf := inmemConfig(t) + conf := InmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1693,7 +1693,7 @@ func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { func TestRaft_ReJoinFollower(t *testing.T) { // Enable operation after a remove. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -1770,7 +1770,7 @@ func TestRaft_ReJoinFollower(t *testing.T) { func TestRaft_LeaderLeaseExpire(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1899,7 +1899,7 @@ func TestRaft_VerifyLeader_Single(t *testing.T) { func TestRaft_VerifyLeader_Fail(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1939,7 +1939,7 @@ func TestRaft_VerifyLeader_Fail(t *testing.T) { func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { // Make a cluster - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() @@ -1973,7 +1973,7 @@ func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { func TestRaft_NotifyCh(t *testing.T) { ch := make(chan bool, 1) - conf := inmemConfig(t) + conf := InmemConfig(t) conf.NotifyCh = ch c := MakeCluster(1, t, conf) defer c.Close() @@ -2062,7 +2062,7 @@ func TestRaft_AppendEntry(t *testing.T) { } func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 3 c := MakeCluster(3, t, conf) defer c.Close() @@ -2142,13 +2142,13 @@ func TestRaft_ProtocolVersion_RejectRPC(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { // Make a cluster back on protocol version 1. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 1 c := MakeCluster(2, t, conf) defer c.Close() // Set up another server speaking protocol version 2. - conf = inmemConfig(t) + conf = InmemConfig(t) conf.ProtocolVersion = 2 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2185,14 +2185,14 @@ func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { // Make a cluster back on protocol version 2. - conf := inmemConfig(t) + conf := InmemConfig(t) conf.ProtocolVersion = 2 c := MakeCluster(2, t, conf) defer c.Close() oldAddr := c.Followers()[0].localAddr // Set up another server speaking protocol version 3. - conf = inmemConfig(t) + conf = InmemConfig(t) conf.ProtocolVersion = 3 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2219,7 +2219,7 @@ func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { func TestRaft_LeaderID_Propagated(t *testing.T) { // Make a cluster on protocol version 3. - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() err := waitForLeader(c) @@ -2700,7 +2700,7 @@ func TestRaft_CacheLogWithStoreError(t *testing.T) { } func TestRaft_ReloadConfig(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) conf.LeaderLeaseTimeout = 40 * time.Millisecond c := MakeCluster(1, t, conf) defer c.Close() @@ -2731,7 +2731,7 @@ func TestRaft_ReloadConfig(t *testing.T) { } func TestRaft_ReloadConfigValidates(t *testing.T) { - conf := inmemConfig(t) + conf := InmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -2973,7 +2973,7 @@ func TestRaft_ClusterCanRegainStability_WhenNonVoterWithHigherTermJoin(t *testin // started when a standby is shut down and restarted. func TestRaft_FollowerRemovalNoElection(t *testing.T) { // Make a cluster - inmemConf := inmemConfig(t) + inmemConf := InmemConfig(t) inmemConf.HeartbeatTimeout = 100 * time.Millisecond inmemConf.ElectionTimeout = 100 * time.Millisecond c := MakeCluster(3, t, inmemConf) diff --git a/testing.go b/testing.go index e0885714f..9355eb94a 100644 --- a/testing.go +++ b/testing.go @@ -21,7 +21,7 @@ import ( var userSnapshotErrorsOnNoData = true // Return configurations optimized for in-memory -func inmemConfig(tb testing.TB) *Config { +func InmemConfig(t *testing.T) *Config { conf := DefaultConfig() conf.HeartbeatTimeout = 50 * time.Millisecond conf.ElectionTimeout = 50 * time.Millisecond @@ -220,6 +220,39 @@ func newTestLoggerWithPrefix(tb testing.TB, prefix string) hclog.Logger { }) } +func (c *cluster) NodeID(index int) ServerID { + if index >= len(c.rafts) { + return "" + } + return c.rafts[index].localID +} + +// ConnectExternal connects all the transports together. +func (c *cluster) ConnectExternal(a ServerAddress) { + c.logger.Debug("fully connecting") + for _, t := range c.trans { + t.Connect(a, t) + } +} + +func (c *cluster) Transport(index int) LoopbackTransport { + if index >= len(c.trans) { + return nil + } + return c.trans[index] +} + +func (c *cluster) NodeAddress(index int) ServerAddress { + if index >= len(c.rafts) { + return "" + } + return c.rafts[index].localAddr +} + +func (c *cluster) RaftsLen() int { + return len(c.rafts) +} + type cluster struct { dirs []string stores []*InmemStore @@ -726,7 +759,7 @@ type MakeClusterOpts struct { // each other. func makeCluster(t *testing.T, opts *MakeClusterOpts) *cluster { if opts.Conf == nil { - opts.Conf = inmemConfig(t) + opts.Conf = InmemConfig(t) } c := &cluster{ From 4c678d820b42d3d416711bfcec3f60fb2b0e4064 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 7 Jun 2023 15:13:48 -0400 Subject: [PATCH 37/87] refactor test --- raft-compat/rolling_upgrade_test.go | 183 +++++++++++++++++++++------- 1 file changed, 141 insertions(+), 42 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 65c511e61..01d54c0a2 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -9,79 +9,178 @@ import ( "time" ) -func TestRaft_RollingUpgrade_NoLeaderUpgrade(t *testing.T) { +type raftUIT struct { + raft *raft.Raft + trans *raft.NetworkTransport + Config *raft.Config + Store *raft.InmemStore + Snap *raft.InmemSnapshotStore + id raft.ServerID + fsm *raft.MockFSM +} + +func (r raftUIT) getLocalID() string { + return string(r.id) +} + +func (r raftUIT) getLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +type raftLatest struct { + raft *raftrs.Raft + trans *raftrs.NetworkTransport + Config *raftrs.Config + Store *raftrs.InmemStore + Snap *raftrs.InmemSnapshotStore + id raftrs.ServerID + fsm *raftrs.MockFSM +} + +func (r raftLatest) getLocalID() string { + return string(r.id) +} + +func (r raftLatest) getLeaderID() string { + _, id := r.raft.LeaderWithID() + return string(id) +} + +type raftNode interface { + getLocalID() string + getLeaderID() string +} + +type raftCluster[T raftNode] struct { + rafts []T +} + +func newRaftCluster[T raftNode](count int) raftCluster[T] { + rc := raftCluster[T]{} + rc.rafts = make([]T, count) + return rc +} + +func (r *raftCluster[T]) getLeader() T { + var empty T + for _, n := range r.rafts { + if n.getLocalID() == n.getLeaderID() { + return n + } + } + return empty +} + +func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rafts := make([]*raft.Raft, initCount) - trans := make([]*raft.NetworkTransport, initCount) - confs := make([]*raft.Config, initCount) - stores := make([]*raft.InmemStore, initCount) - snaps := make([]*raft.InmemSnapshotStore, initCount) - id := make([]raft.ServerID, initCount) - configuration := raft.Configuration{} + rLatest := newRaftCluster[raftLatest](initCount) + configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { - confs[i] = raft.InmemConfig(t) - id[i] = raft.ServerID(fmt.Sprintf("grpc%d", i)) - confs[i].LocalID = id[i] - stores[i] = raft.NewInmemStore() - snaps[i] = raft.NewInmemSnapshotStore() + rLatest.rafts[i].Config = raftrs.DefaultConfig() + rLatest.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond + rLatest.rafts[i].Config.CommitTimeout = 5 * time.Millisecond + rLatest.rafts[i].id = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) + rLatest.rafts[i].Config.LocalID = rLatest.rafts[i].id + rLatest.rafts[i].Store = raftrs.NewInmemStore() + rLatest.rafts[i].Snap = raftrs.NewInmemSnapshotStore() + rLatest.rafts[i].fsm = &raftrs.MockFSM{} var err error - trans[i], err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - rafts[i], err = raft.NewRaft(confs[i], &raft.MockFSM{}, stores[i], stores[i], snaps[i], trans[i]) + rLatest.rafts[i].raft, err = raftrs.NewRaft(rLatest.rafts[i].Config, rLatest.rafts[i].fsm, rLatest.rafts[i].Store, + rLatest.rafts[i].Store, rLatest.rafts[i].Snap, rLatest.rafts[i].trans) require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raft.Server{ - ID: id[i], - Address: trans[i].LocalAddr(), + configuration.Servers = append(configuration.Servers, raftrs.Server{ + ID: rLatest.rafts[i].id, + Address: rLatest.rafts[i].trans.LocalAddr(), }) } - boot := rafts[0].BootstrapCluster(configuration) + boot := rLatest.rafts[0].raft.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := rafts[0].LeaderCh() + lCh := rLatest.rafts[0].raft.LeaderCh() - after := time.After(time.Second) + after := time.After(5 * time.Second) select { case <-after: t.Fatalf("timedout") case <-lCh: } - a := rafts[0].Leader() + getLeader := rLatest.getLeader() + require.NotNil(t, getLeader) + a := getLeader.raft.Leader() require.NotEmpty(t, a) - future := rafts[0].Apply([]byte("test"), time.Second) + future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - newRafts := make([]*raftrs.Raft, initCount) - newTrans := make([]*raftrs.NetworkTransport, initCount) - newConfs := make([]*raftrs.Config, initCount) - newStores := make([]*raftrs.InmemStore, initCount) - newSnaps := make([]*raftrs.InmemSnapshotStore, initCount) - leader, _ := rafts[0].LeaderWithID() + rUIT := newRaftCluster[raftUIT](initCount) + leader, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, leader) - for i := 1; i < len(rafts); i++ { - newConfs[i] = raftrs.DefaultConfig() - newConfs[i].LocalID = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) - newStores[i] = raftrs.NewInmemStore() - newSnaps[i] = raftrs.NewInmemSnapshotStore() + + // Upgrade all the followers + leaderIdx := 0 + for i := 0; i < len(rLatest.rafts); i++ { + if getLeader.getLocalID() == rLatest.rafts[i].getLocalID() { + leaderIdx = i + continue + } + rUIT.rafts[i].Config = raft.DefaultConfig() + rUIT.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond + rUIT.rafts[i].Config.CommitTimeout = 5 * time.Millisecond + rUIT.rafts[i].id = raft.ServerID(fmt.Sprintf("newGrpc%d", i)) + rUIT.rafts[i].Config.LocalID = rUIT.rafts[i].id + rUIT.rafts[i].Store = raft.NewInmemStore() + rUIT.rafts[i].Snap = raft.NewInmemSnapshotStore() + rUIT.rafts[i].fsm = &raft.MockFSM{} var err error - newTrans[i], err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - newRafts[i], err = raftrs.NewRaft(newConfs[i], &raftrs.MockFSM{}, newStores[i], newStores[i], newSnaps[i], newTrans[i]) - rafts[0].AddVoter("grpc4", raft.ServerAddress(newTrans[i].LocalAddr()), 0, 0) + rUIT.rafts[i].raft, err = raft.NewRaft(rUIT.rafts[i].Config, rUIT.rafts[i].fsm, rUIT.rafts[i].Store, + rUIT.rafts[i].Store, rUIT.rafts[i].Snap, rUIT.rafts[i].trans) + getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[i].getLocalID()), raftrs.ServerAddress(rUIT.rafts[i].trans.LocalAddr()), 0, 0) time.Sleep(1 * time.Second) // Check Leader haven't changed as we are not replacing the leader - a := rafts[0].Leader() + a := getLeader.raft.Leader() require.Equal(t, a, leader) - rafts[0].RemoveServer(id[i], 0, 0) - rafts[i].Shutdown() - + getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) + rLatest.rafts[i].raft.Shutdown() } - future = rafts[0].Apply([]byte("test2"), time.Second) + future = getLeader.raft.Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) + rUIT.rafts[leaderIdx].Config = raft.InmemConfig(t) + rUIT.rafts[leaderIdx].id = raft.ServerID(fmt.Sprintf("newGrpc%d", leaderIdx)) + rUIT.rafts[leaderIdx].Config.LocalID = rUIT.rafts[leaderIdx].id + rUIT.rafts[leaderIdx].Store = raft.NewInmemStore() + rUIT.rafts[leaderIdx].Snap = raft.NewInmemSnapshotStore() + rUIT.rafts[leaderIdx].fsm = &raft.MockFSM{} + var err error + rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + require.NoError(t, err) + rUIT.rafts[leaderIdx].raft, err = raft.NewRaft(rUIT.rafts[leaderIdx].Config, rUIT.rafts[leaderIdx].fsm, rUIT.rafts[leaderIdx].Store, + rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) + getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) + // Check Leader haven't changed as we are not replacing the leader + a = getLeader.raft.Leader() + require.Equal(t, a, leader) + getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) + time.Sleep(1 * time.Second) + rLatest.rafts[leaderIdx].raft.Shutdown() + time.Sleep(1 * time.Second) + aNew := rUIT.getLeader().raft.Leader() + require.NotEqual(t, aNew, leader) + + require.Len(t, rUIT.getLeader().fsm.Logs(), 2) + } From f83d29815dc14bf491f4b2aebac72c998d734910 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 7 Jun 2023 16:03:28 -0400 Subject: [PATCH 38/87] clean up node init --- raft-compat/rolling_upgrade_test.go | 66 +++++++++++++++++++---------- 1 file changed, 43 insertions(+), 23 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 01d54c0a2..ed7cf1ba3 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -56,9 +56,12 @@ type raftCluster[T raftNode] struct { rafts []T } -func newRaftCluster[T raftNode](count int) raftCluster[T] { +func newRaftCluster[T raftNode](count int, name string) raftCluster[T] { rc := raftCluster[T]{} rc.rafts = make([]T, count) + for i := 0; i < count; i++ { + initNode(&rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) + } return rc } @@ -72,23 +75,50 @@ func (r *raftCluster[T]) getLeader() T { return empty } +func initNode(node interface{}, id string) { + switch node.(type) { + case *raftLatest: + initLatest(node.(*raftLatest), id) + case *raftUIT: + initUIT(node.(*raftUIT), id) + default: + panic("invalid node type") + } +} + +func initUIT(node *raftUIT, id string) { + node.Config = raft.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raft.ServerID(id) + node.Config.LocalID = node.id + node.Store = raft.NewInmemStore() + node.Snap = raft.NewInmemSnapshotStore() + node.fsm = &raft.MockFSM{} +} + +func initLatest(node *raftLatest, id string) { + node.Config = raftrs.DefaultConfig() + node.Config.HeartbeatTimeout = 50 * time.Millisecond + node.Config.ElectionTimeout = 50 * time.Millisecond + node.Config.LeaderLeaseTimeout = 50 * time.Millisecond + node.Config.CommitTimeout = 5 * time.Millisecond + node.id = raftrs.ServerID(id) + node.Config.LocalID = node.id + node.Store = raftrs.NewInmemStore() + node.Snap = raftrs.NewInmemSnapshotStore() + node.fsm = &raftrs.MockFSM{} +} + func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := newRaftCluster[raftLatest](initCount) + rLatest := newRaftCluster[raftLatest](initCount, "raftOld") configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { - rLatest.rafts[i].Config = raftrs.DefaultConfig() - rLatest.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond - rLatest.rafts[i].Config.CommitTimeout = 5 * time.Millisecond - rLatest.rafts[i].id = raftrs.ServerID(fmt.Sprintf("grpc%d", i)) - rLatest.rafts[i].Config.LocalID = rLatest.rafts[i].id - rLatest.rafts[i].Store = raftrs.NewInmemStore() - rLatest.rafts[i].Snap = raftrs.NewInmemSnapshotStore() - rLatest.rafts[i].fsm = &raftrs.MockFSM{} var err error rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) @@ -120,7 +150,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - rUIT := newRaftCluster[raftUIT](initCount) + rUIT := newRaftCluster[raftUIT](initCount, "raftNew") leader, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, leader) @@ -131,16 +161,6 @@ func TestRaft_RollingUpgrade(t *testing.T) { leaderIdx = i continue } - rUIT.rafts[i].Config = raft.DefaultConfig() - rUIT.rafts[i].Config.HeartbeatTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.ElectionTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.LeaderLeaseTimeout = 50 * time.Millisecond - rUIT.rafts[i].Config.CommitTimeout = 5 * time.Millisecond - rUIT.rafts[i].id = raft.ServerID(fmt.Sprintf("newGrpc%d", i)) - rUIT.rafts[i].Config.LocalID = rUIT.rafts[i].id - rUIT.rafts[i].Store = raft.NewInmemStore() - rUIT.rafts[i].Snap = raft.NewInmemSnapshotStore() - rUIT.rafts[i].fsm = &raft.MockFSM{} var err error rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) From fafbf062b869c4aba99cfd42823de339c11881b0 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 8 Jun 2023 16:19:01 -0400 Subject: [PATCH 39/87] clean up leader rolling upgrade --- raft-compat/rolling_upgrade_test.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index ed7cf1ba3..cc5bfb6ee 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -179,12 +179,6 @@ func TestRaft_RollingUpgrade(t *testing.T) { future = getLeader.raft.Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - rUIT.rafts[leaderIdx].Config = raft.InmemConfig(t) - rUIT.rafts[leaderIdx].id = raft.ServerID(fmt.Sprintf("newGrpc%d", leaderIdx)) - rUIT.rafts[leaderIdx].Config.LocalID = rUIT.rafts[leaderIdx].id - rUIT.rafts[leaderIdx].Store = raft.NewInmemStore() - rUIT.rafts[leaderIdx].Snap = raft.NewInmemSnapshotStore() - rUIT.rafts[leaderIdx].fsm = &raft.MockFSM{} var err error rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) From 80e46f0fb0d862ed6f08036adca4b65e2648e84e Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 8 Jun 2023 16:20:08 -0400 Subject: [PATCH 40/87] fix use of deprecate Leader method --- raft-compat/rolling_upgrade_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index cc5bfb6ee..ddf901c71 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -145,7 +145,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { } getLeader := rLatest.getLeader() require.NotNil(t, getLeader) - a := getLeader.raft.Leader() + a, _ := getLeader.raft.LeaderWithID() require.NotEmpty(t, a) future := getLeader.raft.Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) @@ -171,7 +171,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { time.Sleep(1 * time.Second) // Check Leader haven't changed as we are not replacing the leader - a := getLeader.raft.Leader() + a, _ := getLeader.raft.LeaderWithID() require.Equal(t, a, leader) getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) rLatest.rafts[i].raft.Shutdown() @@ -186,13 +186,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) // Check Leader haven't changed as we are not replacing the leader - a = getLeader.raft.Leader() + a, _ = getLeader.raft.LeaderWithID() require.Equal(t, a, leader) getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) time.Sleep(1 * time.Second) rLatest.rafts[leaderIdx].raft.Shutdown() time.Sleep(1 * time.Second) - aNew := rUIT.getLeader().raft.Leader() + aNew, _ := getLeader.raft.LeaderWithID() require.NotEqual(t, aNew, leader) require.Len(t, rUIT.getLeader().fsm.Logs(), 2) From bb62335fc5d3902ad9f22b860a76118ee896dd89 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 10:43:10 -0400 Subject: [PATCH 41/87] extract cluster package --- raft-compat/rolling_upgrade_test.go | 172 +++++----------------------- raft-compat/testcluster/cluster.go | 164 ++++++-------------------- 2 files changed, 68 insertions(+), 268 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index ddf901c71..35dd73680 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,140 +1,34 @@ package raft_compat import ( - "fmt" raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" "time" ) -type raftUIT struct { - raft *raft.Raft - trans *raft.NetworkTransport - Config *raft.Config - Store *raft.InmemStore - Snap *raft.InmemSnapshotStore - id raft.ServerID - fsm *raft.MockFSM -} - -func (r raftUIT) getLocalID() string { - return string(r.id) -} - -func (r raftUIT) getLeaderID() string { - _, id := r.raft.LeaderWithID() - return string(id) -} - -type raftLatest struct { - raft *raftrs.Raft - trans *raftrs.NetworkTransport - Config *raftrs.Config - Store *raftrs.InmemStore - Snap *raftrs.InmemSnapshotStore - id raftrs.ServerID - fsm *raftrs.MockFSM -} - -func (r raftLatest) getLocalID() string { - return string(r.id) -} - -func (r raftLatest) getLeaderID() string { - _, id := r.raft.LeaderWithID() - return string(id) -} - -type raftNode interface { - getLocalID() string - getLeaderID() string -} - -type raftCluster[T raftNode] struct { - rafts []T -} - -func newRaftCluster[T raftNode](count int, name string) raftCluster[T] { - rc := raftCluster[T]{} - rc.rafts = make([]T, count) - for i := 0; i < count; i++ { - initNode(&rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) - } - return rc -} - -func (r *raftCluster[T]) getLeader() T { - var empty T - for _, n := range r.rafts { - if n.getLocalID() == n.getLeaderID() { - return n - } - } - return empty -} - -func initNode(node interface{}, id string) { - switch node.(type) { - case *raftLatest: - initLatest(node.(*raftLatest), id) - case *raftUIT: - initUIT(node.(*raftUIT), id) - default: - panic("invalid node type") - } -} - -func initUIT(node *raftUIT, id string) { - node.Config = raft.DefaultConfig() - node.Config.HeartbeatTimeout = 50 * time.Millisecond - node.Config.ElectionTimeout = 50 * time.Millisecond - node.Config.LeaderLeaseTimeout = 50 * time.Millisecond - node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raft.ServerID(id) - node.Config.LocalID = node.id - node.Store = raft.NewInmemStore() - node.Snap = raft.NewInmemSnapshotStore() - node.fsm = &raft.MockFSM{} -} - -func initLatest(node *raftLatest, id string) { - node.Config = raftrs.DefaultConfig() - node.Config.HeartbeatTimeout = 50 * time.Millisecond - node.Config.ElectionTimeout = 50 * time.Millisecond - node.Config.LeaderLeaseTimeout = 50 * time.Millisecond - node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftrs.ServerID(id) - node.Config.LocalID = node.id - node.Store = raftrs.NewInmemStore() - node.Snap = raftrs.NewInmemSnapshotStore() - node.fsm = &raftrs.MockFSM{} -} - func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := newRaftCluster[raftLatest](initCount, "raftOld") + rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") configuration := raftrs.Configuration{} for i := 0; i < initCount; i++ { var err error - rLatest.rafts[i].trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rLatest.rafts[i].raft, err = raftrs.NewRaft(rLatest.rafts[i].Config, rLatest.rafts[i].fsm, rLatest.rafts[i].Store, - rLatest.rafts[i].Store, rLatest.rafts[i].Snap, rLatest.rafts[i].trans) require.NoError(t, err) configuration.Servers = append(configuration.Servers, raftrs.Server{ - ID: rLatest.rafts[i].id, - Address: rLatest.rafts[i].trans.LocalAddr(), + ID: raftrs.ServerID(rLatest.ID(i)), + Address: raftrs.ServerAddress(rLatest.Addr(i)), }) } - boot := rLatest.rafts[0].raft.BootstrapCluster(configuration) + raft0 := rLatest.Raft(0).(*raftrs.Raft) + boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := rLatest.rafts[0].raft.LeaderCh() + lCh := raft0.LeaderCh() after := time.After(5 * time.Second) @@ -143,58 +37,52 @@ func TestRaft_RollingUpgrade(t *testing.T) { t.Fatalf("timedout") case <-lCh: } - getLeader := rLatest.getLeader() + getLeader := rLatest.GetLeader() require.NotNil(t, getLeader) - a, _ := getLeader.raft.LeaderWithID() + a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.raft.Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) require.NoError(t, future.Error()) - rUIT := newRaftCluster[raftUIT](initCount, "raftNew") - leader, _ := getLeader.raft.LeaderWithID() + rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") + leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers leaderIdx := 0 - for i := 0; i < len(rLatest.rafts); i++ { - if getLeader.getLocalID() == rLatest.rafts[i].getLocalID() { + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == rLatest.ID(i) { leaderIdx = i continue } - var err error - rUIT.rafts[i].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rUIT.rafts[i].raft, err = raft.NewRaft(rUIT.rafts[i].Config, rUIT.rafts[i].fsm, rUIT.rafts[i].Store, - rUIT.rafts[i].Store, rUIT.rafts[i].Snap, rUIT.rafts[i].trans) - getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[i].getLocalID()), raftrs.ServerAddress(rUIT.rafts[i].trans.LocalAddr()), 0, 0) - time.Sleep(1 * time.Second) + future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) + time.Sleep(1 * time.Second) + require.NoError(t, future.Error()) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.raft.LeaderWithID() + a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.raft.RemoveServer(rLatest.rafts[i].id, 0, 0) - rLatest.rafts[i].raft.Shutdown() + getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(i).(*raftrs.Raft).Shutdown() } - future = getLeader.raft.Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - var err error - rUIT.rafts[leaderIdx].trans, err = raft.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) - require.NoError(t, err) - rUIT.rafts[leaderIdx].raft, err = raft.NewRaft(rUIT.rafts[leaderIdx].Config, rUIT.rafts[leaderIdx].fsm, rUIT.rafts[leaderIdx].Store, - rUIT.rafts[leaderIdx].Store, rUIT.rafts[leaderIdx].Snap, rUIT.rafts[leaderIdx].trans) - getLeader.raft.AddVoter(raftrs.ServerID(rUIT.rafts[leaderIdx].getLocalID()), raftrs.ServerAddress(rUIT.rafts[leaderIdx].trans.LocalAddr()), 0, 0) + f := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + time.Sleep(1 * time.Second) + require.NoError(t, f.Error()) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.raft.LeaderWithID() + a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.raft.RemoveServer(rLatest.rafts[leaderIdx].id, 0, 0) + getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) time.Sleep(1 * time.Second) - rLatest.rafts[leaderIdx].raft.Shutdown() + rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() time.Sleep(1 * time.Second) - aNew, _ := getLeader.raft.LeaderWithID() + newLeader := rUIT.GetLeader() + aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() require.NotEqual(t, aNew, leader) - require.Len(t, rUIT.getLeader().fsm.Logs(), 2) + require.Equal(t, newLeader.NumLogs(), 2) } diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 4096b606d..79a0cdb70 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -2,8 +2,8 @@ package testcluster import ( "fmt" + raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" - raftprevious "github.com/hashicorp/raft-previous-version" "github.com/stretchr/testify/require" "testing" "time" @@ -31,10 +31,6 @@ func (r RaftUIT) GetRaft() interface{} { return r.raft } -func (r RaftUIT) GetStore() interface{} { - return r.Store -} - func (r RaftUIT) GetLocalID() string { return string(r.id) } @@ -44,31 +40,25 @@ func (r RaftUIT) GetLeaderID() string { return string(id) } -func (r *RaftCluster) ID(i int) string { +func (r *raftCluster[T]) ID(i int) string { return r.rafts[i].GetLocalID() } -func (r *RaftCluster) Addr(i int) string { +func (r *raftCluster[T]) Addr(i int) string { return r.rafts[i].GetLocalAddr() } -func (r *RaftCluster) Raft(id string) interface{} { - i := r.GetIndex(id) +func (r *raftCluster[T]) Raft(i int) interface{} { return r.rafts[i].GetRaft() } -func (r *RaftCluster) Store(id string) interface{} { - i := r.GetIndex(id) - return r.rafts[i].GetStore() -} - type RaftLatest struct { - raft *raftprevious.Raft - trans *raftprevious.NetworkTransport - Config *raftprevious.Config - Store *raftprevious.InmemStore - Snap *raftprevious.InmemSnapshotStore - id raftprevious.ServerID - fsm *raftprevious.MockFSM + raft *raftrs.Raft + trans *raftrs.NetworkTransport + Config *raftrs.Config + Store *raftrs.InmemStore + Snap *raftrs.InmemSnapshotStore + id raftrs.ServerID + fsm *raftrs.MockFSM } func (r RaftLatest) NumLogs() int { @@ -82,9 +72,6 @@ func (r RaftLatest) GetLocalAddr() string { func (r RaftLatest) GetRaft() interface{} { return r.raft } -func (r RaftLatest) GetStore() interface{} { - return r.Store -} func (r RaftLatest) GetLocalID() string { return string(r.id) @@ -100,62 +87,44 @@ type RaftNode interface { GetLocalAddr() string GetLeaderID() string GetRaft() interface{} - GetStore() interface{} NumLogs() int } -type RaftCluster struct { - rafts []RaftNode +type raftCluster[T RaftNode] struct { + rafts []T } -func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, count int, name string) RaftCluster { - rc := RaftCluster{} - rc.rafts = make([]RaftNode, count) +func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluster[T] { + rc := raftCluster[T]{} + rc.rafts = make([]T, count) for i := 0; i < count; i++ { - rc.rafts[i] = f(t, fmt.Sprintf("%s-%d", name, i)) + initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) } return rc } -func (r *RaftCluster) GetLeader() RaftNode { +func (r *raftCluster[T]) GetLeader() T { + var empty T for _, n := range r.rafts { if n.GetLocalID() == n.GetLeaderID() { return n } } - return nil -} - -func (r *RaftCluster) Len() int { - return len(r.rafts) -} - -func (r *RaftCluster) AddNode(node RaftNode) { - r.rafts = append([]RaftNode{node}, r.rafts...) -} - -func (r *RaftCluster) DeleteNode(id string) { - i := r.GetIndex(id) - r.rafts = append(r.rafts[:i], r.rafts[i+1:]...) + return empty } -func (r *RaftCluster) GetIndex(id string) int { - i := 0 - for _, r := range r.rafts { - if r.GetLocalID() == id { - return i - } - i++ +func initNode(t *testing.T, node interface{}, id string) { + switch node.(type) { + case *RaftLatest: + initLatest(t, node.(*RaftLatest), id) + case *RaftUIT: + initUIT(t, node.(*RaftUIT), id) + default: + panic("invalid node type") } - return -1 } -func InitUIT(t *testing.T, id string) RaftNode { - return InitUITWithStore(t, id, nil) -} - -func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) RaftNode { - node := RaftUIT{} +func initUIT(t *testing.T, node *RaftUIT, id string) { node.Config = raft.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond @@ -163,12 +132,7 @@ func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) R node.Config.CommitTimeout = 5 * time.Millisecond node.id = raft.ServerID(id) node.Config.LocalID = node.id - if store != nil { - node.Store = convertInMemStore(store) - } else { - node.Store = raft.NewInmemStore() - } - + node.Store = raft.NewInmemStore() node.Snap = raft.NewInmemSnapshotStore() node.fsm = &raft.MockFSM{} var err error @@ -177,75 +141,23 @@ func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) R node.raft, err = raft.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) - return node } -func InitLatest(t *testing.T, id string) RaftNode { - node := RaftLatest{} - node.Config = raftprevious.DefaultConfig() +func initLatest(t *testing.T, node *RaftLatest, id string) { + node.Config = raftrs.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftprevious.ServerID(id) + node.id = raftrs.ServerID(id) node.Config.LocalID = node.id - - node.Store = raftprevious.NewInmemStore() - node.Snap = raftprevious.NewInmemSnapshotStore() - node.fsm = &raftprevious.MockFSM{} + node.Store = raftrs.NewInmemStore() + node.Snap = raftrs.NewInmemSnapshotStore() + node.fsm = &raftrs.MockFSM{} var err error - node.trans, err = raftprevious.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + node.trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - node.raft, err = raftprevious.NewRaft(node.Config, node.fsm, node.Store, + node.raft, err = raftrs.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) - return node -} - -func convertLog(ll *raftprevious.Log) *raft.Log { - l := new(raft.Log) - l.Index = ll.Index - l.AppendedAt = ll.AppendedAt - l.Type = raft.LogType(ll.Type) - l.Term = ll.Term - l.Data = ll.Data - l.Extensions = ll.Extensions - return l -} - -var ( - keyCurrentTerm = []byte("CurrentTerm") - keyLastVoteTerm = []byte("LastVoteTerm") - keyLastVoteCand = []byte("LastVoteCand") -) - -func convertInMemStore(s *raftprevious.InmemStore) *raft.InmemStore { - ss := raft.NewInmemStore() - fi, _ := s.FirstIndex() - li, _ := s.LastIndex() - for i := fi; i <= li; i++ { - log := new(raftprevious.Log) - s.GetLog(i, log) - ss.StoreLog(convertLog(log)) - } - - get, _ := ss.Get(keyCurrentTerm) - ss.Set(keyCurrentTerm, get) - - get, _ = ss.Get(keyLastVoteTerm) - ss.Set(keyLastVoteTerm, get) - - get, _ = ss.Get(keyLastVoteCand) - ss.Set(keyLastVoteCand, get) - - get64, _ := ss.GetUint64(keyCurrentTerm) - ss.SetUint64(keyCurrentTerm, get64) - - get64, _ = ss.GetUint64(keyLastVoteTerm) - ss.SetUint64(keyLastVoteTerm, get64) - - get64, _ = ss.GetUint64(keyLastVoteCand) - ss.SetUint64(keyLastVoteCand, get64) - - return ss } From 9ed608a3c003146838511fe7b00f1bb733fcc434 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 10:43:51 -0400 Subject: [PATCH 42/87] export cluster Type --- raft-compat/testcluster/cluster.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 79a0cdb70..590424418 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -40,14 +40,14 @@ func (r RaftUIT) GetLeaderID() string { return string(id) } -func (r *raftCluster[T]) ID(i int) string { +func (r *RaftCluster[T]) ID(i int) string { return r.rafts[i].GetLocalID() } -func (r *raftCluster[T]) Addr(i int) string { +func (r *RaftCluster[T]) Addr(i int) string { return r.rafts[i].GetLocalAddr() } -func (r *raftCluster[T]) Raft(i int) interface{} { +func (r *RaftCluster[T]) Raft(i int) interface{} { return r.rafts[i].GetRaft() } @@ -90,12 +90,12 @@ type RaftNode interface { NumLogs() int } -type raftCluster[T RaftNode] struct { +type RaftCluster[T RaftNode] struct { rafts []T } -func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluster[T] { - rc := raftCluster[T]{} +func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) RaftCluster[T] { + rc := RaftCluster[T]{} rc.rafts = make([]T, count) for i := 0; i < count; i++ { initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) @@ -103,7 +103,7 @@ func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) raftCluste return rc } -func (r *raftCluster[T]) GetLeader() T { +func (r *RaftCluster[T]) GetLeader() T { var empty T for _, n := range r.rafts { if n.GetLocalID() == n.GetLeaderID() { From 4dcdf1d900d180d4f756ca81c61f599eaf382647 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 11:54:23 -0400 Subject: [PATCH 43/87] clean up tests and add test utils --- raft-compat/go.mod | 4 ++-- raft-compat/rolling_upgrade_test.go | 33 ++++++++++++----------------- raft-compat/testcluster/cluster.go | 6 +++++- raft-compat/utils/test_utils.go | 10 ++++----- 4 files changed, 25 insertions(+), 28 deletions(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 40db0858b..96b3e9626 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,14 +16,14 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/dhiayachi/raft v1.0.0 => ./raftrs +replace github.com/hashicorp/raft-latest => ./raftrs replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dhiayachi/raft v1.0.0 github.com/hashicorp/raft v1.2.0 + github.com/hashicorp/raft-latest v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 35dd73680..b9c9f0ff6 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,9 +1,10 @@ package raft_compat import ( - raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + raftrs "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" + "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" "testing" "time" @@ -28,21 +29,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - lCh := raft0.LeaderCh() - - after := time.After(5 * time.Second) - - select { - case <-after: - t.Fatalf("timedout") - case <-lCh: - } + utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) getLeader := rLatest.GetLeader() - require.NotNil(t, getLeader) + require.NotEmpty(t, getLeader) a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.NotEmpty(t, a) future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) - require.NoError(t, future.Error()) + utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() @@ -58,8 +51,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) - time.Sleep(1 * time.Second) - require.NoError(t, future.Error()) + utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) @@ -69,17 +61,18 @@ func TestRaft_RollingUpgrade(t *testing.T) { future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - f := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) - time.Sleep(1 * time.Second) - require.NoError(t, f.Error()) + fa := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + utils.WaitFuture(t, fa) + // Check Leader haven't changed as we are not replacing the leader a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) - time.Sleep(1 * time.Second) + fr := getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) + utils.WaitFuture(t, fr) rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() - time.Sleep(1 * time.Second) + utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() + require.NotEmpty(t, newLeader) aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() require.NotEqual(t, aNew, leader) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 590424418..962453bec 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -2,8 +2,8 @@ package testcluster import ( "fmt" - raftrs "github.com/dhiayachi/raft" "github.com/hashicorp/raft" + raftrs "github.com/hashicorp/raft-latest" "github.com/stretchr/testify/require" "testing" "time" @@ -113,6 +113,10 @@ func (r *RaftCluster[T]) GetLeader() T { return empty } +func (r *RaftCluster[T]) Len() int { + return len(r.rafts) +} + func initNode(t *testing.T, node interface{}, id string) { switch node.(type) { case *RaftLatest: diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 0b0b18c6d..d69d5366c 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -3,24 +3,24 @@ package utils import ( "fmt" "github.com/hashicorp/raft" - raftprevious "github.com/hashicorp/raft-previous-version" + raftrs "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" "time" ) -func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) { +func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c testcluster.RaftCluster[T]) { leader := func() string { for i := 0; i < c.Len(); i++ { - switch r := c.Raft(c.ID(i)).(type) { + switch r := c.Raft(i).(type) { case *raft.Raft: if r.State() == raft.Leader { return c.ID(i) } - case *raftprevious.Raft: - if r.State() == raftprevious.Leader { + case *raftrs.Raft: + if r.State() == raftrs.Leader { return c.ID(i) } } From 475d620f734eeccf610135deeb504793c4d8025a Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 11:55:22 -0400 Subject: [PATCH 44/87] rename package to raftlatest --- raft-compat/rolling_upgrade_test.go | 36 ++++++++++++++--------------- raft-compat/testcluster/cluster.go | 30 ++++++++++++------------ raft-compat/utils/test_utils.go | 6 ++--- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index b9c9f0ff6..c7e0db013 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -2,7 +2,7 @@ package raft_compat import ( "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" @@ -14,17 +14,17 @@ func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") - configuration := raftrs.Configuration{} + configuration := raftlatest.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftrs.Server{ - ID: raftrs.ServerID(rLatest.ID(i)), - Address: raftrs.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftlatest.Server{ + ID: raftlatest.ServerID(rLatest.ID(i)), + Address: raftlatest.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(0).(*raftrs.Raft) + raft0 := rLatest.Raft(0).(*raftlatest.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -32,13 +32,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers @@ -49,27 +49,27 @@ func TestRaft_RollingUpgrade(t *testing.T) { continue } - future := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(i)), raftrs.ServerAddress(rUIT.Addr(i)), 0, 0) + future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(i)), raftlatest.ServerAddress(rUIT.Addr(i)), 0, 0) utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(i).(*raftrs.Raft).Shutdown() + getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(i).(*raftlatest.Raft).Shutdown() } - future = getLeader.GetRaft().(*raftrs.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - fa := getLeader.GetRaft().(*raftrs.Raft).AddVoter(raftrs.ServerID(rUIT.ID(leaderIdx)), raftrs.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + fa := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(leaderIdx)), raftlatest.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.GetRaft().(*raftrs.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftrs.Raft).RemoveServer(raftrs.ServerID(rLatest.ID(leaderIdx)), 0, 0) + fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(leaderIdx).(*raftrs.Raft).Shutdown() + rLatest.Raft(leaderIdx).(*raftlatest.Raft).Shutdown() utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 962453bec..71ef6432e 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -3,7 +3,7 @@ package testcluster import ( "fmt" "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/stretchr/testify/require" "testing" "time" @@ -52,13 +52,13 @@ func (r *RaftCluster[T]) Raft(i int) interface{} { } type RaftLatest struct { - raft *raftrs.Raft - trans *raftrs.NetworkTransport - Config *raftrs.Config - Store *raftrs.InmemStore - Snap *raftrs.InmemSnapshotStore - id raftrs.ServerID - fsm *raftrs.MockFSM + raft *raftlatest.Raft + trans *raftlatest.NetworkTransport + Config *raftlatest.Config + Store *raftlatest.InmemStore + Snap *raftlatest.InmemSnapshotStore + id raftlatest.ServerID + fsm *raftlatest.MockFSM } func (r RaftLatest) NumLogs() int { @@ -148,20 +148,20 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { } func initLatest(t *testing.T, node *RaftLatest, id string) { - node.Config = raftrs.DefaultConfig() + node.Config = raftlatest.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftrs.ServerID(id) + node.id = raftlatest.ServerID(id) node.Config.LocalID = node.id - node.Store = raftrs.NewInmemStore() - node.Snap = raftrs.NewInmemSnapshotStore() - node.fsm = &raftrs.MockFSM{} + node.Store = raftlatest.NewInmemStore() + node.Snap = raftlatest.NewInmemSnapshotStore() + node.fsm = &raftlatest.MockFSM{} var err error - node.trans, err = raftrs.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + node.trans, err = raftlatest.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - node.raft, err = raftrs.NewRaft(node.Config, node.fsm, node.Store, + node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index d69d5366c..4b6eb9ff6 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -3,7 +3,7 @@ package utils import ( "fmt" "github.com/hashicorp/raft" - raftrs "github.com/hashicorp/raft-latest" + raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" @@ -19,8 +19,8 @@ func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c if r.State() == raft.Leader { return c.ID(i) } - case *raftrs.Raft: - if r.State() == raftrs.Leader { + case *raftlatest.Raft: + if r.State() == raftlatest.Leader { return c.ID(i) } } From 2bad9386b24788109824c3ec58026cd49ed316b0 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:05:10 -0400 Subject: [PATCH 45/87] remove submodule --- .gitmodules | 3 --- raft-compat/raftrs | 1 - 2 files changed, 4 deletions(-) delete mode 160000 raft-compat/raftrs diff --git a/.gitmodules b/.gitmodules index ed959a7cf..e69de29bb 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +0,0 @@ -[submodule "raft-compat/raftrs"] - path = raft-compat/raftrs - url = git@github.com:hashicorp/raft diff --git a/raft-compat/raftrs b/raft-compat/raftrs deleted file mode 160000 index 8fdc4ce5b..000000000 --- a/raft-compat/raftrs +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From b3970e0a71262945e731f0b194f277ef9bdda646 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:05:51 -0400 Subject: [PATCH 46/87] new submodule --- .gitmodules | 3 +++ raft-compat/raft-latest | 1 + 2 files changed, 4 insertions(+) create mode 160000 raft-compat/raft-latest diff --git a/.gitmodules b/.gitmodules index e69de29bb..96d94f795 100644 --- a/.gitmodules +++ b/.gitmodules @@ -0,0 +1,3 @@ +[submodule "raft-compat/raft-latest"] + path = raft-compat/raft-latest + url = git@github.com:hashicorp/raft diff --git a/raft-compat/raft-latest b/raft-compat/raft-latest new file mode 160000 index 000000000..8fdc4ce5b --- /dev/null +++ b/raft-compat/raft-latest @@ -0,0 +1 @@ +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From 0a2e3b5b260757e9134ef2ea84687f13bf8cd374 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 12:06:20 -0400 Subject: [PATCH 47/87] fix go.mod --- raft-compat/go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 96b3e9626..28e134a74 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,7 +16,7 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-latest => ./raftrs +replace github.com/hashicorp/raft-latest => ./raft-latest replace github.com/hashicorp/raft => ../ From e9260228db1c9e09cc144b30153070fd00f39bac Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 13:19:01 -0400 Subject: [PATCH 48/87] change inmemConfig to be not exported --- integ_test.go | 2 +- raft_test.go | 62 +++++++++++++++++++++++++-------------------------- testing.go | 4 ++-- 3 files changed, 34 insertions(+), 34 deletions(-) diff --git a/integ_test.go b/integ_test.go index 897857794..8f4c198f6 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,7 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(tb testing.TB, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { - conf = InmemConfig(t) + conf = inmemConfig(t) } dir, err := os.MkdirTemp("", "raft") diff --git a/raft_test.go b/raft_test.go index 85f3f6fa8..3eaf1e3ce 100644 --- a/raft_test.go +++ b/raft_test.go @@ -152,7 +152,7 @@ func TestRaft_RecoverCluster(t *testing.T) { snapshotThreshold := 5 runRecover := func(t *testing.T, applies int) { var err error - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 conf.SnapshotThreshold = uint64(snapshotThreshold) c := MakeCluster(3, t, conf) @@ -295,7 +295,7 @@ func TestRaft_HasExistingState(t *testing.T) { } func TestRaft_SingleNode(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -501,7 +501,7 @@ func TestRaft_ApplyNonLeader(t *testing.T) { func TestRaft_ApplyConcurrent(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout c := MakeCluster(3, t, conf) @@ -552,7 +552,7 @@ func TestRaft_ApplyConcurrent(t *testing.T) { func TestRaft_ApplyConcurrent_Timeout(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.CommitTimeout = 1 * time.Millisecond conf.HeartbeatTimeout = 2 * conf.HeartbeatTimeout conf.ElectionTimeout = 2 * conf.ElectionTimeout @@ -620,7 +620,7 @@ func TestRaft_JoinNode(t *testing.T) { func TestRaft_JoinNode_ConfigStore(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := makeCluster(t, &MakeClusterOpts{ Peers: 1, Bootstrap: true, @@ -777,7 +777,7 @@ func TestRaft_RemoveLeader(t *testing.T) { func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -840,7 +840,7 @@ func TestRaft_RemoveLeader_NoShutdown(t *testing.T) { func TestRaft_RemoveFollower_SplitCluster(t *testing.T) { // Make a cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(4, t, conf) defer c.Close() @@ -955,7 +955,7 @@ func TestRaft_RemoveUnknownPeer(t *testing.T) { func TestRaft_SnapshotRestore(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1015,7 +1015,7 @@ func TestRaft_SnapshotRestore(t *testing.T) { func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 opts := &MakeClusterOpts{ Peers: 1, @@ -1097,7 +1097,7 @@ func TestRaft_RestoreSnapshotOnStartup_Monotonic(t *testing.T) { func TestRaft_SnapshotRestore_Progress(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) defer c.Close() @@ -1215,7 +1215,7 @@ func (b *lockedBytesBuffer) String() string { // up. func TestRaft_NoRestoreOnStart(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 conf.NoSnapshotRestoreOnStart = true c := MakeCluster(1, t, conf) @@ -1260,7 +1260,7 @@ func TestRaft_NoRestoreOnStart(t *testing.T) { func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { var err error // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 1 conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) @@ -1365,7 +1365,7 @@ func TestRaft_SnapshotRestore_PeerChange(t *testing.T) { func TestRaft_AutoSnapshot(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.SnapshotInterval = conf.CommitTimeout * 2 conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 @@ -1395,7 +1395,7 @@ func TestRaft_AutoSnapshot(t *testing.T) { func TestRaft_UserSnapshot(t *testing.T) { // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.SnapshotThreshold = 50 conf.TrailingLogs = 10 c := MakeCluster(1, t, conf) @@ -1435,7 +1435,7 @@ func snapshotAndRestore(t *testing.T, offset uint64, monotonicLogStore bool, res t.Helper() // Make the cluster. - conf := InmemConfig(t) + conf := inmemConfig(t) // snapshot operations perform some file IO operations. // increase times out to account for that @@ -1597,7 +1597,7 @@ func TestRaft_UserRestore(t *testing.T) { func TestRaft_SendSnapshotFollower(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1639,7 +1639,7 @@ func TestRaft_SendSnapshotFollower(t *testing.T) { func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { // Make the cluster - conf := InmemConfig(t) + conf := inmemConfig(t) conf.TrailingLogs = 10 c := MakeCluster(3, t, conf) defer c.Close() @@ -1693,7 +1693,7 @@ func TestRaft_SendSnapshotAndLogsFollower(t *testing.T) { func TestRaft_ReJoinFollower(t *testing.T) { // Enable operation after a remove. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ShutdownOnRemove = false c := MakeCluster(3, t, conf) defer c.Close() @@ -1770,7 +1770,7 @@ func TestRaft_ReJoinFollower(t *testing.T) { func TestRaft_LeaderLeaseExpire(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1899,7 +1899,7 @@ func TestRaft_VerifyLeader_Single(t *testing.T) { func TestRaft_VerifyLeader_Fail(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(2, t, conf) defer c.Close() @@ -1939,7 +1939,7 @@ func TestRaft_VerifyLeader_Fail(t *testing.T) { func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { // Make a cluster - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() @@ -1973,7 +1973,7 @@ func TestRaft_VerifyLeader_PartialConnect(t *testing.T) { func TestRaft_NotifyCh(t *testing.T) { ch := make(chan bool, 1) - conf := InmemConfig(t) + conf := inmemConfig(t) conf.NotifyCh = ch c := MakeCluster(1, t, conf) defer c.Close() @@ -2062,7 +2062,7 @@ func TestRaft_AppendEntry(t *testing.T) { } func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 3 c := MakeCluster(3, t, conf) defer c.Close() @@ -2142,13 +2142,13 @@ func TestRaft_ProtocolVersion_RejectRPC(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { // Make a cluster back on protocol version 1. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 1 c := MakeCluster(2, t, conf) defer c.Close() // Set up another server speaking protocol version 2. - conf = InmemConfig(t) + conf = inmemConfig(t) conf.ProtocolVersion = 2 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2185,14 +2185,14 @@ func TestRaft_ProtocolVersion_Upgrade_1_2(t *testing.T) { func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { // Make a cluster back on protocol version 2. - conf := InmemConfig(t) + conf := inmemConfig(t) conf.ProtocolVersion = 2 c := MakeCluster(2, t, conf) defer c.Close() oldAddr := c.Followers()[0].localAddr // Set up another server speaking protocol version 3. - conf = InmemConfig(t) + conf = inmemConfig(t) conf.ProtocolVersion = 3 c1 := MakeClusterNoBootstrap(1, t, conf) @@ -2219,7 +2219,7 @@ func TestRaft_ProtocolVersion_Upgrade_2_3(t *testing.T) { func TestRaft_LeaderID_Propagated(t *testing.T) { // Make a cluster on protocol version 3. - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(3, t, conf) defer c.Close() err := waitForLeader(c) @@ -2700,7 +2700,7 @@ func TestRaft_CacheLogWithStoreError(t *testing.T) { } func TestRaft_ReloadConfig(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) conf.LeaderLeaseTimeout = 40 * time.Millisecond c := MakeCluster(1, t, conf) defer c.Close() @@ -2731,7 +2731,7 @@ func TestRaft_ReloadConfig(t *testing.T) { } func TestRaft_ReloadConfigValidates(t *testing.T) { - conf := InmemConfig(t) + conf := inmemConfig(t) c := MakeCluster(1, t, conf) defer c.Close() raft := c.rafts[0] @@ -2973,7 +2973,7 @@ func TestRaft_ClusterCanRegainStability_WhenNonVoterWithHigherTermJoin(t *testin // started when a standby is shut down and restarted. func TestRaft_FollowerRemovalNoElection(t *testing.T) { // Make a cluster - inmemConf := InmemConfig(t) + inmemConf := inmemConfig(t) inmemConf.HeartbeatTimeout = 100 * time.Millisecond inmemConf.ElectionTimeout = 100 * time.Millisecond c := MakeCluster(3, t, inmemConf) diff --git a/testing.go b/testing.go index 9355eb94a..6f0a668f0 100644 --- a/testing.go +++ b/testing.go @@ -21,7 +21,7 @@ import ( var userSnapshotErrorsOnNoData = true // Return configurations optimized for in-memory -func InmemConfig(t *testing.T) *Config { +func inmemConfig(t *testing.T) *Config { conf := DefaultConfig() conf.HeartbeatTimeout = 50 * time.Millisecond conf.ElectionTimeout = 50 * time.Millisecond @@ -759,7 +759,7 @@ type MakeClusterOpts struct { // each other. func makeCluster(t *testing.T, opts *MakeClusterOpts) *cluster { if opts.Conf == nil { - opts.Conf = InmemConfig(t) + opts.Conf = inmemConfig(t) } c := &cluster{ From 74e7f53c63515e3841325b9384b7bc167a019cb9 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 9 Jun 2023 13:20:18 -0400 Subject: [PATCH 49/87] remove unused func --- testing.go | 33 --------------------------------- 1 file changed, 33 deletions(-) diff --git a/testing.go b/testing.go index 6f0a668f0..7ec53786f 100644 --- a/testing.go +++ b/testing.go @@ -220,39 +220,6 @@ func newTestLoggerWithPrefix(tb testing.TB, prefix string) hclog.Logger { }) } -func (c *cluster) NodeID(index int) ServerID { - if index >= len(c.rafts) { - return "" - } - return c.rafts[index].localID -} - -// ConnectExternal connects all the transports together. -func (c *cluster) ConnectExternal(a ServerAddress) { - c.logger.Debug("fully connecting") - for _, t := range c.trans { - t.Connect(a, t) - } -} - -func (c *cluster) Transport(index int) LoopbackTransport { - if index >= len(c.trans) { - return nil - } - return c.trans[index] -} - -func (c *cluster) NodeAddress(index int) ServerAddress { - if index >= len(c.rafts) { - return "" - } - return c.rafts[index].localAddr -} - -func (c *cluster) RaftsLen() int { - return len(c.rafts) -} - type cluster struct { dirs []string stores []*InmemStore From afca2823d54cf5bb1808b2c757ac1f162ab19788 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:17:35 -0400 Subject: [PATCH 50/87] add replace rolling upgrade tests --- raft-compat/rolling_upgrade_test.go | 123 +++++++++++++++++++++-- raft-compat/testcluster/cluster.go | 149 +++++++++++++++++++++++----- raft-compat/utils/test_utils.go | 5 +- 3 files changed, 244 insertions(+), 33 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index c7e0db013..f1d213a2f 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,6 +1,7 @@ package raft_compat import ( + "fmt" "github.com/hashicorp/raft" raftlatest "github.com/hashicorp/raft-latest" "github.com/hashicorp/raft/compat/testcluster" @@ -10,10 +11,13 @@ import ( "time" ) +// TestRaft_RollingUpgrade This test perform a rolling upgrade by adding a new node, +// wait for it to join the cluster and remove one of the old nodes, until all nodes +// are cycled func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 - rLatest := testcluster.NewRaftCluster[testcluster.RaftLatest](t, initCount, "raftOld") + rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") configuration := raftlatest.Configuration{} for i := 0; i < initCount; i++ { @@ -24,12 +28,12 @@ func TestRaft_RollingUpgrade(t *testing.T) { Address: raftlatest.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(0).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) } - utils.WaitForNewLeader[testcluster.RaftLatest](t, "", rLatest) + utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() @@ -37,7 +41,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) - rUIT := testcluster.NewRaftCluster[testcluster.RaftUIT](t, initCount, "raftNew") + rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.NotEmpty(t, leader) @@ -56,7 +60,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() require.Equal(t, a, leader) getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(i).(*raftlatest.Raft).Shutdown() + rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() } future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) @@ -69,8 +73,8 @@ func TestRaft_RollingUpgrade(t *testing.T) { require.Equal(t, a, leader) fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(leaderIdx).(*raftlatest.Raft).Shutdown() - utils.WaitForNewLeader[testcluster.RaftUIT](t, getLeader.GetLocalID(), rUIT) + rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() @@ -79,3 +83,108 @@ func TestRaft_RollingUpgrade(t *testing.T) { require.Equal(t, newLeader.NumLogs(), 2) } + +// TestRaft_ReplaceUpgrade This test perform a rolling upgrade by removing an old node, +// and create a new node with the same store until all old nodes are cycled to new nodes. +// This simulate the advised way of upgrading in Consul. +func TestRaft_ReplaceUpgrade(t *testing.T) { + + initCount := 3 + rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + configuration := raftlatest.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftlatest.Server{ + ID: raftlatest.ServerID(rLatest.ID(i)), + Address: raftlatest.ServerAddress(rLatest.Addr(i)), + }) + } + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", rLatest) + getLeader := rLatest.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + leaderIdx := 0 + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == rLatest.ID(i) { + leaderIdx = i + continue + } + + // Check Leader haven't changed as we are not replacing the leader + a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.Equal(t, a, leader) + getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + + fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) + rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + + // Keep the store, to be passed to the upgraded node. + store := rLatest.Store(rLatest.ID(i)) + + //Delete the node from the cluster + rLatest.DeleteNode(rLatest.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftlatest.InmemStore)) + future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.GetLocalID()), raftlatest.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + rLatest.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := rLatest.Store(getLeader.GetLocalID()) + + //Remove and shutdown the leader node + fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(getLeader.GetLocalID()), 0, 0) + utils.WaitFuture(t, fr) + rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + + // Delete the old leader node from the cluster + rLatest.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, rLatest) + getLeader = rLatest.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftlatest.InmemStore)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", rLatest) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) +} diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 71ef6432e..9e04972ff 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -31,6 +31,10 @@ func (r RaftUIT) GetRaft() interface{} { return r.raft } +func (r RaftUIT) GetStore() interface{} { + return r.Store +} + func (r RaftUIT) GetLocalID() string { return string(r.id) } @@ -40,17 +44,23 @@ func (r RaftUIT) GetLeaderID() string { return string(id) } -func (r *RaftCluster[T]) ID(i int) string { +func (r *RaftCluster) ID(i int) string { return r.rafts[i].GetLocalID() } -func (r *RaftCluster[T]) Addr(i int) string { +func (r *RaftCluster) Addr(i int) string { return r.rafts[i].GetLocalAddr() } -func (r *RaftCluster[T]) Raft(i int) interface{} { +func (r *RaftCluster) Raft(id string) interface{} { + i := r.GetIndex(id) return r.rafts[i].GetRaft() } +func (r *RaftCluster) Store(id string) interface{} { + i := r.GetIndex(id) + return r.rafts[i].GetStore() +} + type RaftLatest struct { raft *raftlatest.Raft trans *raftlatest.NetworkTransport @@ -72,6 +82,9 @@ func (r RaftLatest) GetLocalAddr() string { func (r RaftLatest) GetRaft() interface{} { return r.raft } +func (r RaftLatest) GetStore() interface{} { + return r.Store +} func (r RaftLatest) GetLocalID() string { return string(r.id) @@ -87,48 +100,79 @@ type RaftNode interface { GetLocalAddr() string GetLeaderID() string GetRaft() interface{} + GetStore() interface{} NumLogs() int } -type RaftCluster[T RaftNode] struct { - rafts []T +type RaftCluster struct { + rafts []RaftNode } -func NewRaftCluster[T RaftNode](t *testing.T, count int, name string) RaftCluster[T] { - rc := RaftCluster[T]{} - rc.rafts = make([]T, count) +func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, count int, name string) RaftCluster { + rc := RaftCluster{} + rc.rafts = make([]RaftNode, count) for i := 0; i < count; i++ { - initNode(t, &rc.rafts[i], fmt.Sprintf("%s-%d", name, i)) + rc.rafts[i] = f(t, fmt.Sprintf("%s-%d", name, i)) } return rc } -func (r *RaftCluster[T]) GetLeader() T { - var empty T +//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftlatest.InmemStore) *T { +// raft := new(T) +// initNode(t, raft, name, store) +// return raft +//} + +func (r *RaftCluster) GetLeader() RaftNode { for _, n := range r.rafts { if n.GetLocalID() == n.GetLeaderID() { return n } } - return empty + return nil } -func (r *RaftCluster[T]) Len() int { +func (r *RaftCluster) Len() int { return len(r.rafts) } -func initNode(t *testing.T, node interface{}, id string) { - switch node.(type) { - case *RaftLatest: - initLatest(t, node.(*RaftLatest), id) - case *RaftUIT: - initUIT(t, node.(*RaftUIT), id) - default: - panic("invalid node type") +//func initNode(t *testing.T, node interface{}, id string, store *raftlatest.InmemStore) { +// switch node.(type) { +// case *RaftLatest: +// initLatest(t, node.(*RaftLatest), id) +// case *RaftUIT: +// initUIT(t, node.(*RaftUIT), id, convertInMemStore(store)) +// default: +// panic("invalid node type") +// } +//} + +func (r *RaftCluster) AddNode(node RaftNode) { + r.rafts = append([]RaftNode{node}, r.rafts...) +} + +func (r *RaftCluster) DeleteNode(id string) { + i := r.GetIndex(id) + r.rafts = append(r.rafts[:i], r.rafts[i+1:]...) +} + +func (r *RaftCluster) GetIndex(id string) int { + i := 0 + for _, r := range r.rafts { + if r.GetLocalID() == id { + return i + } + i++ } + return -1 } -func initUIT(t *testing.T, node *RaftUIT, id string) { +func InitUIT(t *testing.T, id string) RaftNode { + return InitUITWithStore(t, id, nil) +} + +func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) RaftNode { + node := RaftUIT{} node.Config = raft.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond @@ -136,7 +180,12 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { node.Config.CommitTimeout = 5 * time.Millisecond node.id = raft.ServerID(id) node.Config.LocalID = node.id - node.Store = raft.NewInmemStore() + if store != nil { + node.Store = convertInMemStore(store) + } else { + node.Store = raft.NewInmemStore() + } + node.Snap = raft.NewInmemSnapshotStore() node.fsm = &raft.MockFSM{} var err error @@ -145,9 +194,11 @@ func initUIT(t *testing.T, node *RaftUIT, id string) { node.raft, err = raft.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) + return node } -func initLatest(t *testing.T, node *RaftLatest, id string) { +func InitLatest(t *testing.T, id string) RaftNode { + node := RaftLatest{} node.Config = raftlatest.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond @@ -155,6 +206,7 @@ func initLatest(t *testing.T, node *RaftLatest, id string) { node.Config.CommitTimeout = 5 * time.Millisecond node.id = raftlatest.ServerID(id) node.Config.LocalID = node.id + node.Store = raftlatest.NewInmemStore() node.Snap = raftlatest.NewInmemSnapshotStore() node.fsm = &raftlatest.MockFSM{} @@ -164,4 +216,53 @@ func initLatest(t *testing.T, node *RaftLatest, id string) { node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) + return node +} + +func convertLog(ll *raftlatest.Log) *raft.Log { + l := new(raft.Log) + l.Index = ll.Index + l.AppendedAt = ll.AppendedAt + l.Type = raft.LogType(ll.Type) + l.Term = ll.Term + l.Data = ll.Data + l.Extensions = ll.Extensions + return l +} + +var ( + keyCurrentTerm = []byte("CurrentTerm") + keyLastVoteTerm = []byte("LastVoteTerm") + keyLastVoteCand = []byte("LastVoteCand") +) + +func convertInMemStore(s *raftlatest.InmemStore) *raft.InmemStore { + ss := raft.NewInmemStore() + fi, _ := s.FirstIndex() + li, _ := s.LastIndex() + for i := fi; i <= li; i++ { + log := new(raftlatest.Log) + s.GetLog(i, log) + ss.StoreLog(convertLog(log)) + } + + get, _ := ss.Get(keyCurrentTerm) + ss.Set(keyCurrentTerm, get) + + get, _ = ss.Get(keyLastVoteTerm) + ss.Set(keyLastVoteTerm, get) + + get, _ = ss.Get(keyLastVoteCand) + ss.Set(keyLastVoteCand, get) + + get64, _ := ss.GetUint64(keyCurrentTerm) + ss.SetUint64(keyCurrentTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteTerm) + ss.SetUint64(keyLastVoteTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteCand) + ss.SetUint64(keyLastVoteCand, get64) + + return ss } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 4b6eb9ff6..0f69594ca 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -10,11 +10,11 @@ import ( "time" ) -func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c testcluster.RaftCluster[T]) { +func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) { leader := func() string { for i := 0; i < c.Len(); i++ { - switch r := c.Raft(i).(type) { + switch r := c.Raft(c.ID(i)).(type) { case *raft.Raft: if r.State() == raft.Leader { return c.ID(i) @@ -36,6 +36,7 @@ func WaitForNewLeader[T testcluster.RaftNode](t *testing.T, oldLeader string, c case <-ticker.C: id := leader() if id != "" { + fmt.Printf("got a leader %s\n", id) if id != oldLeader || oldLeader == "" { return } From 34b71eb52f89565e7109012284869c56b754750a Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:21:57 -0400 Subject: [PATCH 51/87] rename raft-latest to raft-previous --- raft-compat/go.mod | 4 +- raft-compat/rolling_upgrade_test.go | 72 ++++++++++++++--------------- raft-compat/testcluster/cluster.go | 51 ++++++++------------ raft-compat/utils/test_utils.go | 6 +-- 4 files changed, 61 insertions(+), 72 deletions(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 28e134a74..4fb0906a0 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,14 +16,14 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-latest => ./raft-latest +replace github.com/hashicorp/raft-previous-version => ./raft-latest replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/hashicorp/raft v1.2.0 - github.com/hashicorp/raft-latest v1.2.0 + github.com/hashicorp/raft-previous-version v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index f1d213a2f..6517a4b86 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -3,7 +3,7 @@ package raft_compat import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" "github.com/hashicorp/raft/compat/utils" "github.com/stretchr/testify/require" @@ -18,17 +18,17 @@ func TestRaft_RollingUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftlatest.Configuration{} + configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftlatest.Server{ - ID: raftlatest.ServerID(rLatest.ID(i)), - Address: raftlatest.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(rLatest.ID(i)), + Address: raftprevious.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -36,13 +36,13 @@ func TestRaft_RollingUpgrade(t *testing.T) { utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers @@ -53,27 +53,27 @@ func TestRaft_RollingUpgrade(t *testing.T) { continue } - future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(i)), raftlatest.ServerAddress(rUIT.Addr(i)), 0, 0) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(i)), raftprevious.ServerAddress(rUIT.Addr(i)), 0, 0) utils.WaitFuture(t, future) // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) + rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() } - future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) - fa := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.ID(leaderIdx)), raftlatest.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) + fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) // Check Leader haven't changed as we are not replacing the leader - a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(leaderIdx)), 0, 0) + fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) newLeader := rUIT.GetLeader() require.NotEmpty(t, newLeader) @@ -91,17 +91,17 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { initCount := 3 rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftlatest.Configuration{} + configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { var err error require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftlatest.Server{ - ID: raftlatest.ServerID(rLatest.ID(i)), - Address: raftlatest.ServerAddress(rLatest.Addr(i)), + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(rLatest.ID(i)), + Address: raftprevious.ServerAddress(rLatest.Addr(i)), }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftlatest.Raft) + raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) boot := raft0.BootstrapCluster(configuration) if err := boot.Error(); err != nil { t.Fatalf("bootstrap err: %v", err) @@ -109,12 +109,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { utils.WaitForNewLeader(t, "", rLatest) getLeader := rLatest.GetLeader() require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test"), time.Second) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) utils.WaitFuture(t, future) - leader, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers leaderIdx := 0 @@ -125,12 +125,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { } // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(rLatest.ID(i)), 0, 0) + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) - rLatest.Raft(rLatest.ID(i)).(*raftlatest.Raft).Shutdown() + rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() // Keep the store, to be passed to the upgraded node. store := rLatest.Store(rLatest.ID(i)) @@ -139,8 +139,8 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { rLatest.DeleteNode(rLatest.ID(i)) //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftlatest.InmemStore)) - future := getLeader.GetRaft().(*raftlatest.Raft).AddVoter(raftlatest.ServerID(rUIT.GetLocalID()), raftlatest.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftprevious.InmemStore)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) //Add the new node to the cluster rLatest.AddNode(rUIT) @@ -150,20 +150,20 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { time.Sleep(time.Second) //Apply some logs - future = getLeader.GetRaft().(*raftlatest.Raft).Apply([]byte("test2"), time.Second) + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftlatest.Raft).LeaderWithID() + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) // keep a reference to the store store := rLatest.Store(getLeader.GetLocalID()) //Remove and shutdown the leader node - fr := getLeader.GetRaft().(*raftlatest.Raft).RemoveServer(raftlatest.ServerID(getLeader.GetLocalID()), 0, 0) + fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(getLeader.GetLocalID()), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftlatest.Raft).Shutdown() + rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() // Delete the old leader node from the cluster rLatest.DeleteNode(getLeader.GetLocalID()) @@ -175,7 +175,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftlatest.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftprevious.InmemStore)) fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 9e04972ff..0e1c5b49a 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -3,7 +3,7 @@ package testcluster import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/stretchr/testify/require" "testing" "time" @@ -62,13 +62,13 @@ func (r *RaftCluster) Store(id string) interface{} { } type RaftLatest struct { - raft *raftlatest.Raft - trans *raftlatest.NetworkTransport - Config *raftlatest.Config - Store *raftlatest.InmemStore - Snap *raftlatest.InmemSnapshotStore - id raftlatest.ServerID - fsm *raftlatest.MockFSM + raft *raftprevious.Raft + trans *raftprevious.NetworkTransport + Config *raftprevious.Config + Store *raftprevious.InmemStore + Snap *raftprevious.InmemSnapshotStore + id raftprevious.ServerID + fsm *raftprevious.MockFSM } func (r RaftLatest) NumLogs() int { @@ -117,7 +117,7 @@ func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, coun return rc } -//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftlatest.InmemStore) *T { +//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftprevious.InmemStore) *T { // raft := new(T) // initNode(t, raft, name, store) // return raft @@ -136,17 +136,6 @@ func (r *RaftCluster) Len() int { return len(r.rafts) } -//func initNode(t *testing.T, node interface{}, id string, store *raftlatest.InmemStore) { -// switch node.(type) { -// case *RaftLatest: -// initLatest(t, node.(*RaftLatest), id) -// case *RaftUIT: -// initUIT(t, node.(*RaftUIT), id, convertInMemStore(store)) -// default: -// panic("invalid node type") -// } -//} - func (r *RaftCluster) AddNode(node RaftNode) { r.rafts = append([]RaftNode{node}, r.rafts...) } @@ -171,7 +160,7 @@ func InitUIT(t *testing.T, id string) RaftNode { return InitUITWithStore(t, id, nil) } -func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) RaftNode { +func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) RaftNode { node := RaftUIT{} node.Config = raft.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond @@ -199,27 +188,27 @@ func InitUITWithStore(t *testing.T, id string, store *raftlatest.InmemStore) Raf func InitLatest(t *testing.T, id string) RaftNode { node := RaftLatest{} - node.Config = raftlatest.DefaultConfig() + node.Config = raftprevious.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond node.Config.CommitTimeout = 5 * time.Millisecond - node.id = raftlatest.ServerID(id) + node.id = raftprevious.ServerID(id) node.Config.LocalID = node.id - node.Store = raftlatest.NewInmemStore() - node.Snap = raftlatest.NewInmemSnapshotStore() - node.fsm = &raftlatest.MockFSM{} + node.Store = raftprevious.NewInmemStore() + node.Snap = raftprevious.NewInmemSnapshotStore() + node.fsm = &raftprevious.MockFSM{} var err error - node.trans, err = raftlatest.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) + node.trans, err = raftprevious.NewTCPTransport("localhost:0", nil, 2, time.Second, nil) require.NoError(t, err) - node.raft, err = raftlatest.NewRaft(node.Config, node.fsm, node.Store, + node.raft, err = raftprevious.NewRaft(node.Config, node.fsm, node.Store, node.Store, node.Snap, node.trans) require.NoError(t, err) return node } -func convertLog(ll *raftlatest.Log) *raft.Log { +func convertLog(ll *raftprevious.Log) *raft.Log { l := new(raft.Log) l.Index = ll.Index l.AppendedAt = ll.AppendedAt @@ -236,12 +225,12 @@ var ( keyLastVoteCand = []byte("LastVoteCand") ) -func convertInMemStore(s *raftlatest.InmemStore) *raft.InmemStore { +func convertInMemStore(s *raftprevious.InmemStore) *raft.InmemStore { ss := raft.NewInmemStore() fi, _ := s.FirstIndex() li, _ := s.LastIndex() for i := fi; i <= li; i++ { - log := new(raftlatest.Log) + log := new(raftprevious.Log) s.GetLog(i, log) ss.StoreLog(convertLog(log)) } diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 0f69594ca..035575d0b 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -3,7 +3,7 @@ package utils import ( "fmt" "github.com/hashicorp/raft" - raftlatest "github.com/hashicorp/raft-latest" + raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" "github.com/stretchr/testify/require" "testing" @@ -19,8 +19,8 @@ func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) if r.State() == raft.Leader { return c.ID(i) } - case *raftlatest.Raft: - if r.State() == raftlatest.Leader { + case *raftprevious.Raft: + if r.State() == raftprevious.Leader { return c.ID(i) } } From 2988d0766b58a9438cec42691f2e2dd526e39d9b Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:25:09 -0400 Subject: [PATCH 52/87] rename raft-latest to raft-previous submodule --- raft-compat/go.mod | 2 +- raft-compat/raft-latest | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) delete mode 160000 raft-compat/raft-latest diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 4fb0906a0..23ff53c7e 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -16,7 +16,7 @@ require ( golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect ) -replace github.com/hashicorp/raft-previous-version => ./raft-latest +replace github.com/hashicorp/raft-previous-version => ./raft-previous-version replace github.com/hashicorp/raft => ../ diff --git a/raft-compat/raft-latest b/raft-compat/raft-latest deleted file mode 160000 index 8fdc4ce5b..000000000 --- a/raft-compat/raft-latest +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From e9fcf377c3e9c92e89198b64d563665151eb700f Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:26:33 -0400 Subject: [PATCH 53/87] fix submodule --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 96d94f795..28e632c85 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,3 @@ [submodule "raft-compat/raft-latest"] - path = raft-compat/raft-latest + path = raft-compat/raft-previous-version url = git@github.com:hashicorp/raft From 07bd0e282a87b437ea7729f909df91369463741b Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 26 Jun 2023 15:29:47 -0400 Subject: [PATCH 54/87] remove printf --- raft-compat/rolling_upgrade_test.go | 8 ++++---- raft-compat/utils/test_utils.go | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index 6517a4b86..b9664f623 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -68,7 +68,7 @@ func TestRaft_RollingUpgrade(t *testing.T) { fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) utils.WaitFuture(t, fa) - // Check Leader haven't changed as we are not replacing the leader + // Check Leader haven't changed as we haven't replaced it yet a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) @@ -124,12 +124,12 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { continue } - // Check Leader haven't changed as we are not replacing the leader + // Check Leader haven't changed a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - fmt.Printf("dhayachi:: shutting down %s", rLatest.ID(i)) + // + getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() // Keep the store, to be passed to the upgraded node. diff --git a/raft-compat/utils/test_utils.go b/raft-compat/utils/test_utils.go index 035575d0b..0b0b18c6d 100644 --- a/raft-compat/utils/test_utils.go +++ b/raft-compat/utils/test_utils.go @@ -36,7 +36,6 @@ func WaitForNewLeader(t *testing.T, oldLeader string, c testcluster.RaftCluster) case <-ticker.C: id := leader() if id != "" { - fmt.Printf("got a leader %s\n", id) if id != oldLeader || oldLeader == "" { return } From 95e17a0f6e933c98d7aa9e215e07266382a17e5a Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 27 Jun 2023 10:20:16 -0400 Subject: [PATCH 55/87] use same name for recycled servers, add other leave scenarios --- raft-compat/rolling_upgrade_test.go | 228 ++++++++++++++++------------ 1 file changed, 134 insertions(+), 94 deletions(-) diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index b9664f623..a18eaa25b 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,7 +1,6 @@ package raft_compat import ( - "fmt" "github.com/hashicorp/raft" raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" @@ -89,102 +88,143 @@ func TestRaft_RollingUpgrade(t *testing.T) { // This simulate the advised way of upgrading in Consul. func TestRaft_ReplaceUpgrade(t *testing.T) { - initCount := 3 - rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} - - for i := 0; i < initCount; i++ { - var err error - require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(rLatest.ID(i)), - Address: raftprevious.ServerAddress(rLatest.Addr(i)), - }) - } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) - if err := boot.Error(); err != nil { - t.Fatalf("bootstrap err: %v", err) + tcs := []struct { + Name string + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + { + Name: "leave before shutdown", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "shutdown without leave", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "leader transfer", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, } - utils.WaitForNewLeader(t, "", rLatest) - getLeader := rLatest.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - // Upgrade all the followers - leaderIdx := 0 - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == rLatest.ID(i) { - leaderIdx = i - continue - } - - // Check Leader haven't changed - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() - - // Keep the store, to be passed to the upgraded node. - store := rLatest.Store(rLatest.ID(i)) - - //Delete the node from the cluster - rLatest.DeleteNode(rLatest.ID(i)) - - //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", i), store.(*raftprevious.InmemStore)) - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, future) - //Add the new node to the cluster - rLatest.AddNode(rUIT) + for _, tc := range tcs { + t.Run(tc.Name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + configuration := raftprevious.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + continue + } + + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // + tc.Leave(t, cluster, cluster.ID(i)) + + // Keep the store, to be passed to the upgraded node. + store := cluster.Store(cluster.ID(i)) + id := cluster.ID(i) + + //Delete the node from the cluster + cluster.DeleteNode(cluster.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + cluster.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := cluster.Store(getLeader.GetLocalID()) + id := getLeader.GetLocalID() + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + cluster.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) + }) } +} - // Wait enough to have the configuration propagated. - time.Sleep(time.Second) - - //Apply some logs - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) - require.NoError(t, future.Error()) - - // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - - // keep a reference to the store - store := rLatest.Store(getLeader.GetLocalID()) - - //Remove and shutdown the leader node - fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(getLeader.GetLocalID()), 0, 0) +func leave(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() - - // Delete the old leader node from the cluster - rLatest.DeleteNode(getLeader.GetLocalID()) - oldLeaderID := getLeader.GetLocalID() - - // Wait for a new leader to be elected - utils.WaitForNewLeader(t, oldLeaderID, rLatest) - getLeader = rLatest.GetLeader() - require.NotEmpty(t, getLeader) - - // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, fmt.Sprintf("New-Raft-%d", leaderIdx), store.(*raftprevious.InmemStore)) - fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) - utils.WaitFuture(t, fa) - - // Wait for new leader, (this happens because of not having prevote) - utils.WaitForNewLeader(t, "", rLatest) - newLeader := rUIT.GetLeaderID() - require.NotEmpty(t, newLeader) - - require.NotEqual(t, newLeader, leader) - - require.Equal(t, rUIT.NumLogs(), 2) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) } From 104ceaa52f4756a56b61aee91fb18b3118480b65 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 8 Nov 2022 16:21:38 -0500 Subject: [PATCH 56/87] prevote initial implementation --- commands.go | 4 +++ raft.go | 84 +++++++++++++++++++++++++++++++++++++++++----------- raft_test.go | 1 - 3 files changed, 71 insertions(+), 18 deletions(-) diff --git a/commands.go b/commands.go index 1a74e0525..c36295e80 100644 --- a/commands.go +++ b/commands.go @@ -92,6 +92,7 @@ type RequestVoteRequest struct { // transfer. It is required for leadership transfer to work, because servers // wouldn't vote otherwise if they are aware of an existing leader. LeadershipTransfer bool + PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -113,6 +114,9 @@ type RequestVoteResponse struct { // Is the vote granted. Granted bool + + // Is it a preVote response + PreVote bool } // GetRPCHeader - See WithRPCHeader. diff --git a/raft.go b/raft.go index 28c112832..abee104b0 100644 --- a/raft.go +++ b/raft.go @@ -286,7 +286,7 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - voteCh := r.electSelf() + prevoteCh := r.electSelf(true) // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -300,9 +300,10 @@ func (r *Raft) runCandidate() { // Tally the votes, need a simple majority grantedVotes := 0 + preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - + var voteCh <-chan *voteResult for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -311,6 +312,43 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.processRPC(rpc) + case vote := <-prevoteCh: + r.mainThreadSaturation.working() + r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + // Check if the term is greater than ours, bail + if vote.Term > r.getCurrentTerm() && !vote.PreVote { + r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } + + // Check if the vote is granted + if vote.Granted { + if !vote.PreVote { + grantedVotes++ + r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + } else { + preVoteGrantedVotes++ + r.logger.Debug("prevote granted", "from", vote.voterID, "term", vote.Term, "tally", preVoteGrantedVotes) + } + } + + // Check if we've become the leader + if grantedVotes >= votesNeeded { + r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) + r.setState(Leader) + r.setLeader(r.localAddr, r.localID) + //r.setCurrentTerm(term) + return + } + // Check if we've become the leader + if preVoteGrantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + preVoteGrantedVotes = 0 + grantedVotes = 0 + voteCh = r.electSelf(false) + } case vote := <-voteCh: r.mainThreadSaturation.working() // Check if the term is greater than ours, bail @@ -334,7 +372,6 @@ func (r *Raft) runCandidate() { r.setLeader(r.localAddr, r.localID) return } - case c := <-r.configurationChangeCh: r.mainThreadSaturation.working() // Reject any operations since we are not the leader @@ -1558,6 +1595,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, + PreVote: req.PreVote, } var rpcErr error defer func() { @@ -1613,8 +1651,10 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - r.setState(Follower) - r.setCurrentTerm(req.Term) + if !req.PreVote { + r.setState(Follower) + r.setCurrentTerm(req.Term) + } resp.Term = req.Term } @@ -1671,11 +1711,12 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if err := r.persistVote(req.Term, candidateBytes); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return + if !req.PreVote { + if err := r.persistVote(req.Term, candidateBytes); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return + } } - resp.Granted = true r.setLastContact() } @@ -1841,23 +1882,26 @@ type voteResult struct { // ourself. This has the side affecting of incrementing the current term. The // response channel returned is used to wait for all the responses (including a // vote for ourself). This must only be called from the main thread. -func (r *Raft) electSelf() <-chan *voteResult { +func (r *Raft) electSelf(preVote bool) <-chan *voteResult { // Create a response channel respCh := make(chan *voteResult, len(r.configurations.latest.Servers)) // Increment the term - r.setCurrentTerm(r.getCurrentTerm() + 1) - + newTerm := r.getCurrentTerm() + 1 + if !preVote { + r.setCurrentTerm(newTerm) + } // Construct the request lastIdx, lastTerm := r.getLastEntry() req := &RequestVoteRequest{ RPCHeader: r.getRPCHeader(), - Term: r.getCurrentTerm(), + Term: newTerm, // this is needed for retro compatibility, before RPCHeader.Addr was added Candidate: r.trans.EncodePeer(r.localID, r.localAddr), LastLogIndex: lastIdx, LastLogTerm: lastTerm, - LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), + LeadershipTransfer: r.candidateFromLeadershipTransfer, + PreVote: preVote, } // Construct a function to ask for a vote @@ -1873,7 +1917,9 @@ func (r *Raft) electSelf() <-chan *voteResult { "term", req.Term) resp.Term = req.Term resp.Granted = false + resp.PreVote = req.PreVote } + r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1884,9 +1930,11 @@ func (r *Raft) electSelf() <-chan *voteResult { if server.ID == r.localID { r.logger.Debug("voting for self", "term", req.Term, "id", r.localID) // Persist a vote for ourselves - if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return nil + if !preVote { + if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return nil + } } // Include our own vote respCh <- &voteResult{ @@ -1894,6 +1942,7 @@ func (r *Raft) electSelf() <-chan *voteResult { RPCHeader: r.getRPCHeader(), Term: req.Term, Granted: true, + PreVote: req.PreVote, }, voterID: r.localID, } @@ -1920,6 +1969,7 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { + r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) diff --git a/raft_test.go b/raft_test.go index 3eaf1e3ce..41c95809c 100644 --- a/raft_test.go +++ b/raft_test.go @@ -3065,7 +3065,6 @@ func TestRaft_VoteWithNoIDNoAddr(t *testing.T) { var resp RequestVoteResponse followerT := c.trans[c.IndexOf(followers[1])] c.Partition([]ServerAddress{leader.localAddr}) - time.Sleep(c.propagateTimeout) // wait for the remaining follower to trigger an election waitForState(follower, Candidate) From faa005e267cfd04e314652566927bda5eb8410a2 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 14:30:30 -0500 Subject: [PATCH 57/87] add config and relevant tests --- api.go | 4 ++++ config.go | 3 +++ raft.go | 21 ++++++++++++++------ raft_test.go | 56 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 78 insertions(+), 6 deletions(-) diff --git a/api.go b/api.go index 01f573ce6..8b3875c5b 100644 --- a/api.go +++ b/api.go @@ -213,6 +213,9 @@ type Raft struct { // mainThreadSaturation measures the saturation of the main raft goroutine. mainThreadSaturation *saturationMetric + + // preVote control if the pre-vote feature is activated + preVote bool } // BootstrapCluster initializes a server's storage with the given cluster @@ -560,6 +563,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna leaderNotifyCh: make(chan struct{}, 1), followerNotifyCh: make(chan struct{}, 1), mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), + preVote: conf.PreVote, } r.conf.Store(*conf) diff --git a/config.go b/config.go index b97b4338f..715049896 100644 --- a/config.go +++ b/config.go @@ -234,6 +234,9 @@ type Config struct { // skipStartup allows NewRaft() to bypass all background work goroutines skipStartup bool + + // PreVote activate the pre-vote feature + PreVote bool } func (conf *Config) getOrCreateLogger() hclog.Logger { diff --git a/raft.go b/raft.go index abee104b0..66db29410 100644 --- a/raft.go +++ b/raft.go @@ -286,7 +286,13 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - prevoteCh := r.electSelf(true) + var voteCh <-chan *voteResult + var prevoteCh <-chan *voteResult + if r.preVote { + prevoteCh = r.electSelf(true) + } else { + voteCh = r.electSelf(false) + } // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -303,7 +309,7 @@ func (r *Raft) runCandidate() { preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - var voteCh <-chan *voteResult + for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -1589,13 +1595,16 @@ func (r *Raft) processConfigurationLogEntry(entry *Log) error { func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) r.observe(*req) - + preVote := req.PreVote + if !r.preVote { + preVote = false + } // Setup a response resp := &RequestVoteResponse{ RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: req.PreVote, + PreVote: preVote, } var rpcErr error defer func() { @@ -1651,7 +1660,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !req.PreVote { + if !preVote { r.setState(Follower) r.setCurrentTerm(req.Term) } @@ -1711,7 +1720,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if !req.PreVote { + if !preVote { if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return diff --git a/raft_test.go b/raft_test.go index 41c95809c..6f45cb825 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2061,6 +2061,62 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } +func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { + + tcs := []struct { + name string + prevoteNum int + noprevoteNum int + }{ + {"majority no pre-vote", 2, 3}, + {"majority pre-vote", 3, 2}, + {"all pre-vote", 3, 0}, + {"all no pre-vote", 0, 3}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + + // Make majority cluster. + majority := tc.prevoteNum + minority := tc.noprevoteNum + if tc.prevoteNum < tc.noprevoteNum { + majority = tc.noprevoteNum + minority = tc.prevoteNum + } + + conf := inmemConfig(t) + conf.PreVote = tc.prevoteNum > tc.noprevoteNum + c := MakeCluster(majority, t, conf) + defer c.Close() + + // Set up another server speaking protocol version 2. + conf = inmemConfig(t) + conf.PreVote = tc.prevoteNum < tc.noprevoteNum + c1 := MakeClusterNoBootstrap(minority, t, conf) + + // Merge clusters. + c.Merge(c1) + c.FullyConnect() + + if len(c1.rafts) > 0 { + future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + if err := future.Error(); err != nil { + t.Fatalf("err: %v", err) + } + } + time.Sleep(c.propagateTimeout * 10) + + leaderOld := c.Leader() + c.Followers() + c.Partition([]ServerAddress{leaderOld.localAddr}) + time.Sleep(c.propagateTimeout * 3) + leader := c.Leader() + require.NotEqual(t, leader.leaderID, leaderOld.leaderID) + }) + } + +} + func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { conf := inmemConfig(t) conf.ProtocolVersion = 3 From d15db63fc537427b0d478874ac3d9090fb5ee6c2 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 16:11:36 -0500 Subject: [PATCH 58/87] remove extra comments, fix a case where newer term is discovered for prevote --- raft.go | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/raft.go b/raft.go index 66db29410..6db9adcf2 100644 --- a/raft.go +++ b/raft.go @@ -317,8 +317,9 @@ func (r *Raft) runCandidate() { case rpc := <-r.rpcCh: r.mainThreadSaturation.working() r.processRPC(rpc) - case vote := <-prevoteCh: + // This a pre-vote case but could lead to winning an election, in the case that majority of other nodes + // don't support pre-vote or have pre-vote deactivated. r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail @@ -328,6 +329,12 @@ func (r *Raft) runCandidate() { r.setCurrentTerm(vote.Term) return } + if vote.Term > term && vote.PreVote { + r.logger.Debug("newer term discovered on pre-vote, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } // Check if the vote is granted if vote.Granted { @@ -341,14 +348,17 @@ func (r *Raft) runCandidate() { } // Check if we've become the leader + // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) + // set our state to leader and our term to the pre-vote term. if grantedVotes >= votesNeeded { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) - //r.setCurrentTerm(term) + + r.setCurrentTerm(term) return } - // Check if we've become the leader + // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 @@ -1928,7 +1938,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { resp.Granted = false resp.PreVote = req.PreVote } - r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1978,7 +1987,6 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { - r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) From e7e4476982b6ca79745c0a5795639188f0851b1c Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 10 Nov 2022 11:23:51 -0500 Subject: [PATCH 59/87] fix to reset timeout after pre-vote and fix split vote (pre-vote,vote) case. --- raft.go | 5 ++++- raft_test.go | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index 6db9adcf2..fdfde55cb 100644 --- a/raft.go +++ b/raft.go @@ -350,7 +350,9 @@ func (r *Raft) runCandidate() { // Check if we've become the leader // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) // set our state to leader and our term to the pre-vote term. - if grantedVotes >= votesNeeded { + // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 + // we can flip our vote to an actual vote. + if grantedVotes >= votesNeeded-1 { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) @@ -363,6 +365,7 @@ func (r *Raft) runCandidate() { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 grantedVotes = 0 + electionTimer = randomTimeout(electionTimeout) voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index 6f45cb825..e7b633599 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2061,7 +2061,7 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } -func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { +func TestRaft_PreVoteMixedCluster(t *testing.T) { tcs := []struct { name string @@ -2070,6 +2070,8 @@ func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { }{ {"majority no pre-vote", 2, 3}, {"majority pre-vote", 3, 2}, + {"majority no pre-vote", 1, 2}, + {"majority pre-vote", 2, 1}, {"all pre-vote", 3, 0}, {"all no pre-vote", 0, 3}, } From 60653280744aa414f7dc61a7b69835cfab96702e Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 14 Nov 2022 13:59:32 -0500 Subject: [PATCH 60/87] write upgrade tests that include prevotes --- fuzzy/go.sum | 5 +- raft-compat/prevote_test.go | 133 ++++++++++++++++++ raft-compat/raft-previous-version | 2 +- raft-compat/rolling_upgrade_test.go | 210 ++++++++++++++++++---------- raft-compat/testcluster/cluster.go | 9 +- raft.go | 7 +- raft_test.go | 4 +- 7 files changed, 288 insertions(+), 82 deletions(-) create mode 100644 raft-compat/prevote_test.go diff --git a/fuzzy/go.sum b/fuzzy/go.sum index 1f99677f0..3f8067c79 100644 --- a/fuzzy/go.sum +++ b/fuzzy/go.sum @@ -91,7 +91,10 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go new file mode 100644 index 000000000..e3f27410e --- /dev/null +++ b/raft-compat/prevote_test.go @@ -0,0 +1,133 @@ +package raft_compat + +import ( + "github.com/hashicorp/raft" + raftprevious "github.com/hashicorp/raft-previous-version" + "github.com/hashicorp/raft/compat/testcluster" + "github.com/hashicorp/raft/compat/utils" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { + leave := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + tcs := []struct { + name string + prevoteNum int + noprevoteNum int + preVoteEnabled bool + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + {"majority latest, prevote off", 1, 2, false, leave}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + configuration := raftprevious.Configuration{} + + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + continue + } + + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // + tc.Leave(t, cluster, cluster.ID(i)) + + // Keep the store, to be passed to the upgraded node. + store := cluster.Store(cluster.ID(i)) + id := cluster.ID(i) + + //Delete the node from the cluster + cluster.DeleteNode(cluster.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { + config.PreVote = true + }) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + cluster.AddNode(rUIT) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // keep a reference to the store + store := cluster.Store(getLeader.GetLocalID()) + id := getLeader.GetLocalID() + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + cluster.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVote = true }) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + + require.NotEqual(t, newLeader, leader) + + require.Equal(t, rUIT.NumLogs(), 2) + }) + } + +} diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index 8fdc4ce5b..df0cdeebe 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 +Subproject commit df0cdeebead4b031824392c2bb19568eeb300112 diff --git a/raft-compat/rolling_upgrade_test.go b/raft-compat/rolling_upgrade_test.go index a18eaa25b..516e3da44 100644 --- a/raft-compat/rolling_upgrade_test.go +++ b/raft-compat/rolling_upgrade_test.go @@ -1,6 +1,7 @@ package raft_compat import ( + "fmt" "github.com/hashicorp/raft" raftprevious "github.com/hashicorp/raft-previous-version" "github.com/hashicorp/raft/compat/testcluster" @@ -14,73 +15,143 @@ import ( // wait for it to join the cluster and remove one of the old nodes, until all nodes // are cycled func TestRaft_RollingUpgrade(t *testing.T) { + tcs := []struct { + Name string + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + { + Name: "leave before shutdown", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + }, + }, + { + Name: "leader transfer", + Leave: func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + utils.WaitForNewLeader(t, id, cluster) + } + switch cluster.GetLeader().GetRaft().(type) { + case *raftprevious.Raft: + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + case *raft.Raft: + fr := cluster.GetLeader().GetRaft().(*raft.Raft).RemoveServer(raft.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + + }, + }, + } + + for _, tc := range tcs { + t.Run(tc.Name, func(t *testing.T) { + initCount := 3 + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + configuration := raftprevious.Configuration{} - initCount := 3 - rLatest := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") - configuration := raftprevious.Configuration{} + for i := 0; i < initCount; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raftprevious.Server{ + ID: raftprevious.ServerID(cluster.ID(i)), + Address: raftprevious.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raftprevious.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + leaderIdx := 0 - for i := 0; i < initCount; i++ { - var err error - require.NoError(t, err) - configuration.Servers = append(configuration.Servers, raftprevious.Server{ - ID: raftprevious.ServerID(rLatest.ID(i)), - Address: raftprevious.ServerAddress(rLatest.Addr(i)), + followers := make([]string, 0) + for i := 0; i < initCount; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + leaderIdx = i + continue + } + followers = append(followers, cluster.ID(i)) + } + + for _, f := range followers { + require.NotEqual(t, f, getLeader.GetLocalID()) + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + //Create an upgraded node with the store + rUIT := testcluster.InitUIT(t, fmt.Sprintf("%s-new", f)) + future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + + //Add the new node to the cluster + + tc.Leave(t, cluster, f) + + //Delete the node from the cluster + cluster.AddNode(rUIT) + cluster.DeleteNode(f) + } + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUIT(t, fmt.Sprintf("raftNew-%d", leaderIdx)) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeader := rUIT.GetLeaderID() + require.NotEmpty(t, newLeader) + require.NotEqual(t, newLeader, leader) + + cluster.DeleteNode(getLeader.GetLocalID()) + require.Equal(t, rUIT.NumLogs(), 2) }) } - raft0 := rLatest.Raft(rLatest.ID(0)).(*raftprevious.Raft) - boot := raft0.BootstrapCluster(configuration) - if err := boot.Error(); err != nil { - t.Fatalf("bootstrap err: %v", err) - } - utils.WaitForNewLeader(t, "", rLatest) - getLeader := rLatest.GetLeader() - require.NotEmpty(t, getLeader) - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, a) - future := getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test"), time.Second) - utils.WaitFuture(t, future) - - rUIT := testcluster.NewRaftCluster(t, testcluster.InitUIT, initCount, "raftNew") - leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.NotEmpty(t, leader) - - // Upgrade all the followers - leaderIdx := 0 - for i := 0; i < initCount; i++ { - if getLeader.GetLocalID() == rLatest.ID(i) { - leaderIdx = i - continue - } - - future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(i)), raftprevious.ServerAddress(rUIT.Addr(i)), 0, 0) - - utils.WaitFuture(t, future) - // Check Leader haven't changed as we are not replacing the leader - a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(i)), 0, 0) - rLatest.Raft(rLatest.ID(i)).(*raftprevious.Raft).Shutdown() - } - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) - require.NoError(t, future.Error()) - - fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.ID(leaderIdx)), raftprevious.ServerAddress(rUIT.Addr(leaderIdx)), 0, 0) - utils.WaitFuture(t, fa) - - // Check Leader haven't changed as we haven't replaced it yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) - fr := getLeader.GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(rLatest.ID(leaderIdx)), 0, 0) - utils.WaitFuture(t, fr) - rLatest.Raft(getLeader.GetLocalID()).(*raftprevious.Raft).Shutdown() - utils.WaitForNewLeader(t, getLeader.GetLocalID(), rUIT) - newLeader := rUIT.GetLeader() - require.NotEmpty(t, newLeader) - aNew, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() - require.NotEqual(t, aNew, leader) - - require.Equal(t, newLeader.NumLogs(), 2) - } // TestRaft_ReplaceUpgrade This test perform a rolling upgrade by removing an old node, @@ -124,7 +195,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { for _, tc := range tcs { t.Run(tc.Name, func(t *testing.T) { initCount := 3 - cluster := testcluster.NewRaftCluster(t, testcluster.InitLatest, initCount, "raftOld") + cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") configuration := raftprevious.Configuration{} for i := 0; i < initCount; i++ { @@ -171,7 +242,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { cluster.DeleteNode(cluster.ID(i)) //Create an upgraded node with the store - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) {}) future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) //Add the new node to the cluster @@ -206,7 +277,7 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore)) + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) {}) fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) @@ -221,10 +292,3 @@ func TestRaft_ReplaceUpgrade(t *testing.T) { }) } } - -func leave(t *testing.T, cluster testcluster.RaftCluster, id string) { - fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) - utils.WaitFuture(t, fr) - f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() - utils.WaitFuture(t, f) -} diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 0e1c5b49a..64fe04c9c 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -117,6 +117,10 @@ func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, coun return rc } +func NewPreviousRaftCluster(t *testing.T, count int, name string) RaftCluster { + return NewRaftCluster(t, InitLatest, count, name) +} + //func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftprevious.InmemStore) *T { // raft := new(T) // initNode(t, raft, name, store) @@ -157,12 +161,13 @@ func (r *RaftCluster) GetIndex(id string) int { } func InitUIT(t *testing.T, id string) RaftNode { - return InitUITWithStore(t, id, nil) + return InitUITWithStore(t, id, nil, func(config *raft.Config) {}) } -func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore) RaftNode { +func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore, cfgMod func(config *raft.Config)) RaftNode { node := RaftUIT{} node.Config = raft.DefaultConfig() + cfgMod(node.Config) node.Config.HeartbeatTimeout = 50 * time.Millisecond node.Config.ElectionTimeout = 50 * time.Millisecond node.Config.LeaderLeaseTimeout = 50 * time.Millisecond diff --git a/raft.go b/raft.go index fdfde55cb..06378c98b 100644 --- a/raft.go +++ b/raft.go @@ -323,7 +323,7 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail - if vote.Term > r.getCurrentTerm() && !vote.PreVote { + if vote.Term > term && !vote.PreVote { r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) r.setState(Follower) r.setCurrentTerm(vote.Term) @@ -361,11 +361,12 @@ func (r *Raft) runCandidate() { return } // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + if preVoteGrantedVotes+grantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 grantedVotes = 0 electionTimer = randomTimeout(electionTimeout) + prevoteCh = nil voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index e7b633599..e462427e1 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2100,8 +2100,8 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { c.Merge(c1) c.FullyConnect() - if len(c1.rafts) > 0 { - future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + for _, r := range c1.rafts { + future := c.Leader().AddVoter(r.localID, r.localAddr, 0, 0) if err := future.Error(); err != nil { t.Fatalf("err: %v", err) } From 257ba386cf736d6696259be58d8e28d50c292807 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 22 Aug 2023 10:31:25 -0400 Subject: [PATCH 61/87] add more test cases --- raft-compat/prevote_test.go | 38 +++++++++++++++++++++++++------------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index e3f27410e..c80ec731e 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -11,7 +11,15 @@ import ( ) func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { - leave := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + leaveTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() + utils.WaitFuture(t, f) + } + leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { if cluster.GetLeader().GetLocalID() == id { transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() utils.WaitFuture(t, transfer) @@ -20,21 +28,27 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { utils.WaitFuture(t, f) } tcs := []struct { - name string - prevoteNum int - noprevoteNum int - preVoteEnabled bool - Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + name string + numNodes int + preVote bool + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) }{ - {"majority latest, prevote off", 1, 2, false, leave}, + {"no prevote -> prevote (leave transfer)", 3, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer)", 3, true, leaveNoTransfer}, + {"no prevote -> prevote (leave transfer) 5", 5, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer) 5", 5, true, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer)", 3, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer)", 3, false, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer) 5", 5, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer) 5", 5, false, leaveNoTransfer}, } for _, tc := range tcs { t.Run(tc.name, func(t *testing.T) { - initCount := 3 - cluster := testcluster.NewPreviousRaftCluster(t, initCount, "raftNode") + + cluster := testcluster.NewPreviousRaftCluster(t, tc.numNodes, "raftNode") configuration := raftprevious.Configuration{} - for i := 0; i < initCount; i++ { + for i := 0; i < tc.numNodes; i++ { var err error require.NoError(t, err) configuration.Servers = append(configuration.Servers, raftprevious.Server{ @@ -58,7 +72,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { leader, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.NotEmpty(t, leader) // Upgrade all the followers - for i := 0; i < initCount; i++ { + for i := 0; i < tc.numNodes; i++ { if getLeader.GetLocalID() == cluster.ID(i) { continue } @@ -79,7 +93,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { //Create an upgraded node with the store rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { - config.PreVote = true + config.PreVote = tc.preVote }) future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) From 12a2bf1d01cb6afd03c0d454c94b403e06ec0cd8 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 22 Aug 2023 10:33:17 -0400 Subject: [PATCH 62/87] fix submodule version --- raft-compat/raft-previous-version | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index df0cdeebe..8fdc4ce5b 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit df0cdeebead4b031824392c2bb19568eeb300112 +Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 From 3c6af465d089e0fa3893bff8e03416a3a75572ce Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 8 Nov 2022 16:21:38 -0500 Subject: [PATCH 63/87] prevote initial implementation --- raft.go | 46 +++++++++++++++++++++++----------------------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/raft.go b/raft.go index 06378c98b..cce2bcfd0 100644 --- a/raft.go +++ b/raft.go @@ -286,13 +286,7 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - var voteCh <-chan *voteResult - var prevoteCh <-chan *voteResult - if r.preVote { - prevoteCh = r.electSelf(true) - } else { - voteCh = r.electSelf(false) - } + prevoteCh := r.electSelf(true) // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -309,7 +303,7 @@ func (r *Raft) runCandidate() { preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - + var voteCh <-chan *voteResult for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -336,6 +330,17 @@ func (r *Raft) runCandidate() { return } + case vote := <-prevoteCh: + r.mainThreadSaturation.working() + r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + // Check if the term is greater than ours, bail + if vote.Term > r.getCurrentTerm() && !vote.PreVote { + r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) + r.setState(Follower) + r.setCurrentTerm(vote.Term) + return + } + // Check if the vote is granted if vote.Granted { if !vote.PreVote { @@ -348,25 +353,18 @@ func (r *Raft) runCandidate() { } // Check if we've become the leader - // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) - // set our state to leader and our term to the pre-vote term. - // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 - // we can flip our vote to an actual vote. - if grantedVotes >= votesNeeded-1 { + if grantedVotes >= votesNeeded { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) - - r.setCurrentTerm(term) + //r.setCurrentTerm(term) return } - // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes+grantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + // Check if we've become the leader + if preVoteGrantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 grantedVotes = 0 - electionTimer = randomTimeout(electionTimeout) - prevoteCh = nil voteCh = r.electSelf(false) } case vote := <-voteCh: @@ -1618,7 +1616,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: preVote, + PreVote: req.PreVote, } var rpcErr error defer func() { @@ -1674,7 +1672,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !preVote { + if !req.PreVote { r.setState(Follower) r.setCurrentTerm(req.Term) } @@ -1734,7 +1732,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if !preVote { + if !req.PreVote { if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return @@ -1942,6 +1940,7 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { resp.Granted = false resp.PreVote = req.PreVote } + r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1991,6 +1990,7 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { + r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) From 989ca74d6b4620e43c8d1543707b594a5c1914af Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 14:30:30 -0500 Subject: [PATCH 64/87] add config and relevant tests --- raft.go | 16 +++++++++++----- raft_test.go | 8 +++----- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/raft.go b/raft.go index cce2bcfd0..8805c90b1 100644 --- a/raft.go +++ b/raft.go @@ -286,7 +286,13 @@ func (r *Raft) runCandidate() { metrics.IncrCounter([]string{"raft", "state", "candidate"}, 1) // Start vote for us, and set a timeout - prevoteCh := r.electSelf(true) + var voteCh <-chan *voteResult + var prevoteCh <-chan *voteResult + if r.preVote { + prevoteCh = r.electSelf(true) + } else { + voteCh = r.electSelf(false) + } // Make sure the leadership transfer flag is reset after each run. Having this // flag will set the field LeadershipTransfer in a RequestVoteRequst to true, @@ -303,7 +309,7 @@ func (r *Raft) runCandidate() { preVoteGrantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) - var voteCh <-chan *voteResult + for r.getState() == Candidate { r.mainThreadSaturation.sleeping() @@ -1616,7 +1622,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: req.PreVote, + PreVote: preVote, } var rpcErr error defer func() { @@ -1672,7 +1678,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !req.PreVote { + if !preVote { r.setState(Follower) r.setCurrentTerm(req.Term) } @@ -1732,7 +1738,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { } // Persist a vote for safety - if !req.PreVote { + if !preVote { if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return diff --git a/raft_test.go b/raft_test.go index e462427e1..6f45cb825 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2061,7 +2061,7 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } -func TestRaft_PreVoteMixedCluster(t *testing.T) { +func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { tcs := []struct { name string @@ -2070,8 +2070,6 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { }{ {"majority no pre-vote", 2, 3}, {"majority pre-vote", 3, 2}, - {"majority no pre-vote", 1, 2}, - {"majority pre-vote", 2, 1}, {"all pre-vote", 3, 0}, {"all no pre-vote", 0, 3}, } @@ -2100,8 +2098,8 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { c.Merge(c1) c.FullyConnect() - for _, r := range c1.rafts { - future := c.Leader().AddVoter(r.localID, r.localAddr, 0, 0) + if len(c1.rafts) > 0 { + future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) if err := future.Error(); err != nil { t.Fatalf("err: %v", err) } From eba1435dac1de83679938c3b58155e2824ec5dee Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 9 Nov 2022 16:11:36 -0500 Subject: [PATCH 65/87] remove extra comments, fix a case where newer term is discovered for prevote --- raft.go | 22 ++++++---------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/raft.go b/raft.go index 8805c90b1..6db9adcf2 100644 --- a/raft.go +++ b/raft.go @@ -323,7 +323,7 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) // Check if the term is greater than ours, bail - if vote.Term > term && !vote.PreVote { + if vote.Term > r.getCurrentTerm() && !vote.PreVote { r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) r.setState(Follower) r.setCurrentTerm(vote.Term) @@ -336,17 +336,6 @@ func (r *Raft) runCandidate() { return } - case vote := <-prevoteCh: - r.mainThreadSaturation.working() - r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) - // Check if the term is greater than ours, bail - if vote.Term > r.getCurrentTerm() && !vote.PreVote { - r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) - r.setState(Follower) - r.setCurrentTerm(vote.Term) - return - } - // Check if the vote is granted if vote.Granted { if !vote.PreVote { @@ -359,14 +348,17 @@ func (r *Raft) runCandidate() { } // Check if we've become the leader + // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) + // set our state to leader and our term to the pre-vote term. if grantedVotes >= votesNeeded { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) - //r.setCurrentTerm(term) + + r.setCurrentTerm(term) return } - // Check if we've become the leader + // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 @@ -1946,7 +1938,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { resp.Granted = false resp.PreVote = req.PreVote } - r.logger.Warn("dhayachi::", "prevote-req", req.PreVote, "resp-prevote", resp.PreVote) respCh <- resp }) } @@ -1996,7 +1987,6 @@ func (r *Raft) persistVote(term uint64, candidate []byte) error { // setCurrentTerm is used to set the current term in a durable manner. func (r *Raft) setCurrentTerm(t uint64) { - r.logger.Warn("dhayachi setting term", "term", t) // Persist to disk first if err := r.stable.SetUint64(keyCurrentTerm, t); err != nil { panic(fmt.Errorf("failed to save current term: %v", err)) From 348e418e5ad1461f8c790ce22ddee874fa6f8b9f Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 10 Nov 2022 11:23:51 -0500 Subject: [PATCH 66/87] fix to reset timeout after pre-vote and fix split vote (pre-vote,vote) case. --- raft.go | 5 ++++- raft_test.go | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index 6db9adcf2..fdfde55cb 100644 --- a/raft.go +++ b/raft.go @@ -350,7 +350,9 @@ func (r *Raft) runCandidate() { // Check if we've become the leader // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) // set our state to leader and our term to the pre-vote term. - if grantedVotes >= votesNeeded { + // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 + // we can flip our vote to an actual vote. + if grantedVotes >= votesNeeded-1 { r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) r.setState(Leader) r.setLeader(r.localAddr, r.localID) @@ -363,6 +365,7 @@ func (r *Raft) runCandidate() { r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) preVoteGrantedVotes = 0 grantedVotes = 0 + electionTimer = randomTimeout(electionTimeout) voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index 6f45cb825..e7b633599 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2061,7 +2061,7 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } -func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { +func TestRaft_PreVoteMixedCluster(t *testing.T) { tcs := []struct { name string @@ -2070,6 +2070,8 @@ func TestRaft_PreVoteMixedCluster_MajorityNoPreVote(t *testing.T) { }{ {"majority no pre-vote", 2, 3}, {"majority pre-vote", 3, 2}, + {"majority no pre-vote", 1, 2}, + {"majority pre-vote", 2, 1}, {"all pre-vote", 3, 0}, {"all no pre-vote", 0, 3}, } From f4e3442e7ef10dfb6ed92f192f71ffc53f3cbf4e Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 14 Nov 2022 13:59:32 -0500 Subject: [PATCH 67/87] fix a case where granted votes and prevotes don't reach quorum but the sum can reach quorum --- raft.go | 5 +++-- raft_test.go | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/raft.go b/raft.go index fdfde55cb..52196d195 100644 --- a/raft.go +++ b/raft.go @@ -361,11 +361,12 @@ func (r *Raft) runCandidate() { return } // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes) + if preVoteGrantedVotes+grantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 grantedVotes = 0 electionTimer = randomTimeout(electionTimeout) + prevoteCh = nil voteCh = r.electSelf(false) } case vote := <-voteCh: diff --git a/raft_test.go b/raft_test.go index e7b633599..e462427e1 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2100,8 +2100,8 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { c.Merge(c1) c.FullyConnect() - if len(c1.rafts) > 0 { - future := c.Leader().AddVoter(c1.rafts[0].localID, c1.rafts[0].localAddr, 0, 0) + for _, r := range c1.rafts { + future := c.Leader().AddVoter(r.localID, r.localAddr, 0, 0) if err := future.Error(); err != nil { t.Fatalf("err: %v", err) } From 6744aa799955c4456076b3bd0d94dbf54feca3d8 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:51:46 -0500 Subject: [PATCH 68/87] go mod tidy --- fuzzy/go.mod | 2 +- fuzzy/go.sum | 1 + raft-compat/go.mod | 4 ++-- raft-compat/go.sum | 2 ++ 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/fuzzy/go.mod b/fuzzy/go.mod index c1c51543a..196abb440 100644 --- a/fuzzy/go.mod +++ b/fuzzy/go.mod @@ -3,7 +3,7 @@ module github.com/hashicorp/raft/fuzzy go 1.20 require ( - github.com/hashicorp/go-hclog v1.5.0 + github.com/hashicorp/go-hclog v1.6.2 github.com/hashicorp/go-msgpack/v2 v2.1.1 github.com/hashicorp/raft v1.2.0 github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea diff --git a/fuzzy/go.sum b/fuzzy/go.sum index 3f8067c79..de7a9127c 100644 --- a/fuzzy/go.sum +++ b/fuzzy/go.sum @@ -33,6 +33,7 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= diff --git a/raft-compat/go.mod b/raft-compat/go.mod index 23ff53c7e..f1a5a5866 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -7,13 +7,13 @@ require github.com/stretchr/testify v1.8.4 require ( github.com/armon/go-metrics v0.4.1 // indirect github.com/fatih/color v1.13.0 // indirect - github.com/hashicorp/go-hclog v1.5.0 // indirect + github.com/hashicorp/go-hclog v1.6.2 // indirect github.com/hashicorp/go-immutable-radix v1.0.0 // indirect github.com/hashicorp/go-msgpack v0.5.5 // indirect github.com/hashicorp/golang-lru v0.5.0 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect - golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 // indirect + golang.org/x/sys v0.13.0 // indirect ) replace github.com/hashicorp/raft-previous-version => ./raft-previous-version diff --git a/raft-compat/go.sum b/raft-compat/go.sum index ed3b7f040..fffbbd864 100644 --- a/raft-compat/go.sum +++ b/raft-compat/go.sum @@ -31,6 +31,7 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= @@ -113,6 +114,7 @@ golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 h1:nonptSpoQ4vQjyraW20DXPAglgQfVnM9ZC6MmNLMR60= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= From 89a5e179c5a1c5130835a3e6ef47dc6006266643 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:53:18 -0500 Subject: [PATCH 69/87] update pervious version to v1.6.0 --- raft-compat/raft-previous-version | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft-compat/raft-previous-version b/raft-compat/raft-previous-version index 8fdc4ce5b..b96f998ff 160000 --- a/raft-compat/raft-previous-version +++ b/raft-compat/raft-previous-version @@ -1 +1 @@ -Subproject commit 8fdc4ce5b75cd34904974859fb8e57a2fc4dc145 +Subproject commit b96f998ff7e752c7eb68615f086a9c52008a40b6 From b27f8ddb7e2a23aa1d407a3a34217e2e24decad3 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Fri, 22 Dec 2023 14:54:21 -0500 Subject: [PATCH 70/87] fix merge duplication --- raft.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/raft.go b/raft.go index 52196d195..64abe3825 100644 --- a/raft.go +++ b/raft.go @@ -1923,7 +1923,7 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { Candidate: r.trans.EncodePeer(r.localID, r.localAddr), LastLogIndex: lastIdx, LastLogTerm: lastTerm, - LeadershipTransfer: r.candidateFromLeadershipTransfer, + LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), PreVote: preVote, } From 1d84a92da916b0ec5b44a522a894e7e7001cc126 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 25 Mar 2024 10:01:38 -0400 Subject: [PATCH 71/87] fix rebase issues --- raft-compat/go.mod | 3 ++- raft-compat/go.sum | 4 ++++ testing.go | 2 +- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/raft-compat/go.mod b/raft-compat/go.mod index f1a5a5866..5d86c2a96 100644 --- a/raft-compat/go.mod +++ b/raft-compat/go.mod @@ -10,6 +10,7 @@ require ( github.com/hashicorp/go-hclog v1.6.2 // indirect github.com/hashicorp/go-immutable-radix v1.0.0 // indirect github.com/hashicorp/go-msgpack v0.5.5 // indirect + github.com/hashicorp/go-msgpack/v2 v2.1.1 // indirect github.com/hashicorp/golang-lru v0.5.0 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect @@ -22,7 +23,7 @@ replace github.com/hashicorp/raft => ../ require ( github.com/davecgh/go-spew v1.1.1 // indirect - github.com/hashicorp/raft v1.2.0 + github.com/hashicorp/raft v1.6.1 github.com/hashicorp/raft-previous-version v1.2.0 github.com/pmezard/go-difflib v1.0.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/raft-compat/go.sum b/raft-compat/go.sum index fffbbd864..9608c05c1 100644 --- a/raft-compat/go.sum +++ b/raft-compat/go.sum @@ -31,11 +31,14 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-hclog v1.5.0 h1:bI2ocEMgcVlz55Oj1xZNBsVi900c7II+fWDyV9o+13c= github.com/hashicorp/go-hclog v1.5.0/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-hclog v1.6.2 h1:NOtoftovWkDheyUM/8JW3QMiXyxJK3uHRK7wV04nD2I= github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-msgpack/v2 v2.1.1 h1:xQEY9yB2wnHitoSzk/B9UjXWRQ67QKu5AOm8aFp8N3I= +github.com/hashicorp/go-msgpack/v2 v2.1.1/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4= github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -114,6 +117,7 @@ golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6 h1:nonptSpoQ4vQjyraW20DXPAglgQfVnM9ZC6MmNLMR60= golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= diff --git a/testing.go b/testing.go index 7ec53786f..bbfee16aa 100644 --- a/testing.go +++ b/testing.go @@ -27,7 +27,7 @@ func inmemConfig(t *testing.T) *Config { conf.ElectionTimeout = 50 * time.Millisecond conf.LeaderLeaseTimeout = 50 * time.Millisecond conf.CommitTimeout = 5 * time.Millisecond - conf.Logger = newTestLogger(tb) + conf.Logger = newTestLogger(t) return conf } From e8a78bac9c9f61bf83bf85baf5fd26aa6871675c Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 25 Mar 2024 13:08:29 -0400 Subject: [PATCH 72/87] use a different RPC command for prevote. --- commands.go | 51 +++++++- config.go | 2 +- fuzzy/transport.go | 5 + inmem_transport.go | 12 ++ integ_test.go | 1 + net_transport.go | 13 ++ raft.go | 313 ++++++++++++++++++++++++++++++++++----------- testing.go | 2 +- transport.go | 17 ++- 9 files changed, 332 insertions(+), 84 deletions(-) diff --git a/commands.go b/commands.go index c36295e80..9f500b2e5 100644 --- a/commands.go +++ b/commands.go @@ -92,7 +92,6 @@ type RequestVoteRequest struct { // transfer. It is required for leadership transfer to work, because servers // wouldn't vote otherwise if they are aware of an existing leader. LeadershipTransfer bool - PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -114,9 +113,6 @@ type RequestVoteResponse struct { // Is the vote granted. Granted bool - - // Is it a preVote response - PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -124,6 +120,53 @@ func (r *RequestVoteResponse) GetRPCHeader() RPCHeader { return r.RPCHeader } +// RequestPreVoteRequest is the command used by a candidate to ask a Raft peer +// for a vote in an election. +type RequestPreVoteRequest struct { + RPCHeader + + // Provide the term and our id + Term uint64 + + // Deprecated: use RPCHeader.Addr instead + Candidate []byte + + // Used to ensure safety + LastLogIndex uint64 + LastLogTerm uint64 + + // Used to indicate to peers if this vote was triggered by a leadership + // transfer. It is required for leadership transfer to work, because servers + // wouldn't vote otherwise if they are aware of an existing leader. + LeadershipTransfer bool +} + +// GetRPCHeader - See WithRPCHeader. +func (r *RequestPreVoteRequest) GetRPCHeader() RPCHeader { + return r.RPCHeader +} + +// RequestPreVoteResponse is the response returned from a RequestPreVoteRequest. +type RequestPreVoteResponse struct { + RPCHeader + + // Newer term if leader is out of date. + Term uint64 + + // Peers is deprecated, but required by servers that only understand + // protocol version 0. This is not populated in protocol version 2 + // and later. + Peers []byte + + // Is the vote granted. + Granted bool +} + +// GetRPCHeader - See WithRPCHeader. +func (r *RequestPreVoteResponse) GetRPCHeader() RPCHeader { + return r.RPCHeader +} + // InstallSnapshotRequest is the command sent to a Raft peer to bootstrap its // log (and state machine) from a snapshot on another peer. type InstallSnapshotRequest struct { diff --git a/config.go b/config.go index 715049896..aad2ebd26 100644 --- a/config.go +++ b/config.go @@ -320,7 +320,7 @@ func DefaultConfig() *Config { SnapshotInterval: 120 * time.Second, SnapshotThreshold: 8192, LeaderLeaseTimeout: 500 * time.Millisecond, - LogLevel: "DEBUG", + LogLevel: "TRACE", } } diff --git a/fuzzy/transport.go b/fuzzy/transport.go index 5fdb4c7b4..d6030d9b0 100644 --- a/fuzzy/transport.go +++ b/fuzzy/transport.go @@ -221,6 +221,11 @@ func (t *transport) RequestVote(id raft.ServerID, target raft.ServerAddress, arg return t.sendRPC(string(target), args, resp) } +// RequestPreVote sends the appropriate RPC to the target node. +func (t *transport) RequestPreVote(id raft.ServerID, target raft.ServerAddress, args *raft.RequestPreVoteRequest, resp *raft.RequestPreVoteResponse) error { + return t.sendRPC(string(target), args, resp) +} + // InstallSnapshot is used to push a snapshot down to a follower. The data is read from // the ReadCloser and streamed to the client. func (t *transport) InstallSnapshot(id raft.ServerID, target raft.ServerAddress, args *raft.InstallSnapshotRequest, resp *raft.InstallSnapshotResponse, data io.Reader) error { diff --git a/inmem_transport.go b/inmem_transport.go index 5d9365b78..561ba73d7 100644 --- a/inmem_transport.go +++ b/inmem_transport.go @@ -125,6 +125,18 @@ func (i *InmemTransport) RequestVote(id ServerID, target ServerAddress, args *Re return nil } +func (i *InmemTransport) RequestPreVote(id ServerID, target ServerAddress, args *RequestPreVoteRequest, resp *RequestPreVoteResponse) error { + rpcResp, err := i.makeRPC(target, args, nil, i.timeout) + if err != nil { + return err + } + + // Copy the result back + out := rpcResp.Response.(*RequestPreVoteResponse) + *resp = *out + return nil +} + // InstallSnapshot implements the Transport interface. func (i *InmemTransport) InstallSnapshot(id ServerID, target ServerAddress, args *InstallSnapshotRequest, resp *InstallSnapshotResponse, data io.Reader) error { rpcResp, err := i.makeRPC(target, args, data, 10*i.timeout) diff --git a/integ_test.go b/integ_test.go index 8f4c198f6..c16ffd94a 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,6 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(tb testing.TB, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { + t := tb.(*testing.T) conf = inmemConfig(t) } diff --git a/net_transport.go b/net_transport.go index bc34285e1..5d91fb97c 100644 --- a/net_transport.go +++ b/net_transport.go @@ -24,6 +24,7 @@ const ( rpcRequestVote rpcInstallSnapshot rpcTimeoutNow + rpcRequestPreVote // DefaultTimeoutScale is the default TimeoutScale in a NetworkTransport. DefaultTimeoutScale = 256 * 1024 // 256KB @@ -470,6 +471,11 @@ func (n *NetworkTransport) RequestVote(id ServerID, target ServerAddress, args * return n.genericRPC(id, target, rpcRequestVote, args, resp) } +// RequestPreVote implements the Transport interface. +func (n *NetworkTransport) RequestPreVote(id ServerID, target ServerAddress, args *RequestPreVoteRequest, resp *RequestPreVoteResponse) error { + return n.genericRPC(id, target, rpcRequestPreVote, args, resp) +} + // genericRPC handles a simple request/response RPC. func (n *NetworkTransport) genericRPC(id ServerID, target ServerAddress, rpcType uint8, args interface{}, resp interface{}) error { // Get a conn @@ -682,6 +688,13 @@ func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, en } rpc.Command = &req labels = []metrics.Label{{Name: "rpcType", Value: "RequestVote"}} + case rpcRequestPreVote: + var req RequestPreVoteRequest + if err := dec.Decode(&req); err != nil { + return err + } + rpc.Command = &req + labels = []metrics.Label{{Name: "rpcType", Value: "RequestPreVote"}} case rpcInstallSnapshot: var req InstallSnapshotRequest if err := dec.Decode(&req); err != nil { diff --git a/raft.go b/raft.go index 64abe3825..2eba75994 100644 --- a/raft.go +++ b/raft.go @@ -8,6 +8,7 @@ import ( "container/list" "fmt" "io" + "strings" "sync/atomic" "time" @@ -17,8 +18,9 @@ import ( ) const ( - minCheckInterval = 10 * time.Millisecond - oldestLogGaugeInterval = 10 * time.Second + minCheckInterval = 10 * time.Millisecond + oldestLogGaugeInterval = 10 * time.Second + rpcUnexpectedCommandError = "unexpected command" ) var ( @@ -287,11 +289,15 @@ func (r *Raft) runCandidate() { // Start vote for us, and set a timeout var voteCh <-chan *voteResult - var prevoteCh <-chan *voteResult - if r.preVote { - prevoteCh = r.electSelf(true) + var prevoteCh <-chan *preVoteResult + + // check if the transport support prevote requests + _, ok := r.trans.(WithPreVote) + + if r.preVote && ok { + prevoteCh = r.preElectSelf() } else { - voteCh = r.electSelf(false) + voteCh = r.electSelf() } // Make sure the leadership transfer flag is reset after each run. Having this @@ -305,8 +311,9 @@ func (r *Raft) runCandidate() { electionTimer := randomTimeout(electionTimeout) // Tally the votes, need a simple majority - grantedVotes := 0 preVoteGrantedVotes := 0 + preVoteRefusedVotes := 0 + grantedVotes := 0 votesNeeded := r.quorumSize() r.logger.Debug("calculated votes needed", "needed", votesNeeded, "term", term) @@ -317,57 +324,41 @@ func (r *Raft) runCandidate() { case rpc := <-r.rpcCh: r.mainThreadSaturation.working() r.processRPC(rpc) - case vote := <-prevoteCh: - // This a pre-vote case but could lead to winning an election, in the case that majority of other nodes - // don't support pre-vote or have pre-vote deactivated. + case preVote := <-prevoteCh: + // This a pre-preVote case but could lead to winning an election, in the case that majority of other nodes + // don't support pre-preVote or have pre-preVote deactivated. r.mainThreadSaturation.working() - r.logger.Debug("got a prevote!!", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + r.logger.Debug("got a prevote!!", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) // Check if the term is greater than ours, bail - if vote.Term > r.getCurrentTerm() && !vote.PreVote { - r.logger.Debug("newer term discovered, fallback to follower", "term", vote.Term) + if preVote.Term > term { + r.logger.Debug("newer term discovered on pre-preVote, fallback to follower", "term", preVote.Term) r.setState(Follower) - r.setCurrentTerm(vote.Term) - return - } - if vote.Term > term && vote.PreVote { - r.logger.Debug("newer term discovered on pre-vote, fallback to follower", "term", vote.Term) - r.setState(Follower) - r.setCurrentTerm(vote.Term) + r.setCurrentTerm(preVote.Term) return } - // Check if the vote is granted - if vote.Granted { - if !vote.PreVote { - grantedVotes++ - r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) - } else { - preVoteGrantedVotes++ - r.logger.Debug("prevote granted", "from", vote.voterID, "term", vote.Term, "tally", preVoteGrantedVotes) - } + // Check if the preVote is granted + if preVote.Granted { + preVoteGrantedVotes++ + r.logger.Debug("prevote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) + } else { + preVoteRefusedVotes++ + r.logger.Debug("prevote refused", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) } - // Check if we've become the leader - // If we won the election because the majority of nodes don't support pre-vote (or pre-vote is deactivated) - // set our state to leader and our term to the pre-vote term. - // we only need votesNeeded-1 as our vote was cast as a prevote and if we have votesNeeded-1 - // we can flip our vote to an actual vote. - if grantedVotes >= votesNeeded-1 { - r.logger.Info("election won", "term", vote.Term, "tally", grantedVotes) - r.setState(Leader) - r.setLeader(r.localAddr, r.localID) - - r.setCurrentTerm(term) - return - } - // Check if we've won the pre-vote and proceed to election if so - if preVoteGrantedVotes+grantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + // Check if we've won the pre-preVote and proceed to election if so + if preVoteGrantedVotes >= votesNeeded { + r.logger.Info("pre election won", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 - grantedVotes = 0 + preVoteRefusedVotes = 0 electionTimer = randomTimeout(electionTimeout) prevoteCh = nil - voteCh = r.electSelf(false) + voteCh = r.electSelf() + } + // Check if we've lost the pre-Vote and wait for the election to timeout so we can do another time of + // prevote. + if preVoteRefusedVotes >= votesNeeded { + r.logger.Info("pre election lost, wait for election to timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) } case vote := <-voteCh: r.mainThreadSaturation.working() @@ -379,10 +370,10 @@ func (r *Raft) runCandidate() { return } - // Check if the vote is granted + // Check if the preVote is granted if vote.Granted { grantedVotes++ - r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + r.logger.Debug("v ote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) } // Check if we've become the leader @@ -1407,6 +1398,8 @@ func (r *Raft) processRPC(rpc RPC) { r.appendEntries(rpc, cmd) case *RequestVoteRequest: r.requestVote(rpc, cmd) + case *RequestPreVoteRequest: + r.requestPreVote(rpc, cmd) case *InstallSnapshotRequest: r.installSnapshot(rpc, cmd) case *TimeoutNowRequest: @@ -1414,7 +1407,8 @@ func (r *Raft) processRPC(rpc RPC) { default: r.logger.Error("got unexpected command", "command", hclog.Fmt("%#v", rpc.Command)) - rpc.Respond(nil, fmt.Errorf("unexpected command")) + + rpc.Respond(nil, fmt.Errorf(rpcUnexpectedCommandError)) } } @@ -1609,16 +1603,12 @@ func (r *Raft) processConfigurationLogEntry(entry *Log) error { func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) r.observe(*req) - preVote := req.PreVote - if !r.preVote { - preVote = false - } + // Setup a response resp := &RequestVoteResponse{ RPCHeader: r.getRPCHeader(), Term: r.getCurrentTerm(), Granted: false, - PreVote: preVote, } var rpcErr error defer func() { @@ -1674,10 +1664,9 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { if req.Term > r.getCurrentTerm() { // Ensure transition to follower r.logger.Debug("lost leadership because received a requestVote with a newer term") - if !preVote { - r.setState(Follower) - r.setCurrentTerm(req.Term) - } + r.setState(Follower) + r.setCurrentTerm(req.Term) + resp.Term = req.Term } @@ -1733,13 +1722,108 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { return } - // Persist a vote for safety - if !preVote { - if err := r.persistVote(req.Term, candidateBytes); err != nil { - r.logger.Error("failed to persist vote", "error", err) + // Persist a vote for safety\ + if err := r.persistVote(req.Term, candidateBytes); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return + } + + resp.Granted = true + r.setLastContact() +} + +// requestPreVote is invoked when we get a request vote RPC call. +func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { + defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) + r.observe(*req) + + // Setup a response + resp := &RequestPreVoteResponse{ + RPCHeader: r.getRPCHeader(), + Term: r.getCurrentTerm(), + Granted: false, + } + var rpcErr error + defer func() { + rpc.Respond(resp, rpcErr) + }() + + // Version 0 servers will panic unless the peers is present. It's only + // used on them to produce a warning message. + if r.protocolVersion < 2 { + resp.Peers = encodePeers(r.configurations.latest, r.trans) + } + + // Check if we have an existing leader [who's not the candidate] and also + // check the LeadershipTransfer flag is set. Usually votes are rejected if + // there is a known leader. But if the leader initiated a leadership transfer, + // vote! + var candidate ServerAddress + + // For older raft version ID is not part of the packed message + // We assume that the peer is part of the configuration and skip this check + if len(req.ID) > 0 { + candidateID := ServerID(req.ID) + // if the Servers list is empty that mean the cluster is very likely trying to bootstrap, + // Grant the vote + if len(r.configurations.latest.Servers) > 0 && !inConfiguration(r.configurations.latest, candidateID) { + r.logger.Warn("rejecting pre-vote request since node is not in configuration", + "from", candidate) return } } + if leaderAddr, leaderID := r.LeaderWithID(); leaderAddr != "" && leaderAddr != candidate && !req.LeadershipTransfer { + r.logger.Warn("rejecting pre-vote request since we have a leader", + "from", candidate, + "leader", leaderAddr, + "leader-id", string(leaderID)) + return + } + + // Ignore an older term + if req.Term < r.getCurrentTerm() { + return + } + + // Increase the term if we see a newer one + if req.Term > r.getCurrentTerm() { + // Ensure we grant the pre-vote as in a "real" vote this will transition us to follower + r.logger.Debug("received a requestPreVote with a newer term, grant the pre-vote") + resp.Term = req.Term + resp.Granted = true + } + + // if we get a request for vote from a nonVoter and the request term is higher, + // step down and update term, but reject the vote request + // This could happen when a node, previously voter, is converted to non-voter + // The reason we need to step in is to permit to the cluster to make progress in such a scenario + // More details about that in https://github.com/hashicorp/raft/pull/526 + if len(req.ID) > 0 { + candidateID := ServerID(req.ID) + if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) { + r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate) + return + } + } + + // Reject if their term is older + lastIdx, lastTerm := r.getLastEntry() + if lastTerm > req.LastLogTerm { + r.logger.Warn("rejecting vote request since our last term is greater", + "candidate", candidate, + "last-term", lastTerm, + "last-candidate-term", req.LastLogTerm) + return + } + + if lastTerm == req.LastLogTerm && lastIdx > req.LastLogIndex { + r.logger.Warn("rejecting vote request since our last index is greater", + "candidate", candidate, + "last-index", lastIdx, + "last-candidate-index", req.LastLogIndex) + return + } + resp.Granted = true r.setLastContact() } @@ -1901,19 +1985,23 @@ type voteResult struct { voterID ServerID } +type preVoteResult struct { + RequestPreVoteResponse + voterID ServerID +} + // electSelf is used to send a RequestVote RPC to all peers, and vote for // ourself. This has the side affecting of incrementing the current term. The // response channel returned is used to wait for all the responses (including a // vote for ourself). This must only be called from the main thread. -func (r *Raft) electSelf(preVote bool) <-chan *voteResult { +func (r *Raft) electSelf() <-chan *voteResult { // Create a response channel respCh := make(chan *voteResult, len(r.configurations.latest.Servers)) // Increment the term newTerm := r.getCurrentTerm() + 1 - if !preVote { - r.setCurrentTerm(newTerm) - } + + r.setCurrentTerm(newTerm) // Construct the request lastIdx, lastTerm := r.getLastEntry() req := &RequestVoteRequest{ @@ -1924,7 +2012,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { LastLogIndex: lastIdx, LastLogTerm: lastTerm, LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), - PreVote: preVote, } // Construct a function to ask for a vote @@ -1940,7 +2027,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { "term", req.Term) resp.Term = req.Term resp.Granted = false - resp.PreVote = req.PreVote } respCh <- resp }) @@ -1951,12 +2037,12 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { if server.Suffrage == Voter { if server.ID == r.localID { r.logger.Debug("voting for self", "term", req.Term, "id", r.localID) + // Persist a vote for ourselves - if !preVote { - if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { - r.logger.Error("failed to persist vote", "error", err) - return nil - } + if err := r.persistVote(req.Term, req.RPCHeader.Addr); err != nil { + r.logger.Error("failed to persist vote", "error", err) + return nil + } // Include our own vote respCh <- &voteResult{ @@ -1964,7 +2050,6 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { RPCHeader: r.getRPCHeader(), Term: req.Term, Granted: true, - PreVote: req.PreVote, }, voterID: r.localID, } @@ -1978,6 +2063,84 @@ func (r *Raft) electSelf(preVote bool) <-chan *voteResult { return respCh } +// preElectSelf is used to send a RequestVote RPC to all peers, and vote for +// ourself. This has the side affecting of incrementing the current term. The +// response channel returned is used to wait for all the responses (including a +// vote for ourself). This must only be called from the main thread. +func (r *Raft) preElectSelf() <-chan *preVoteResult { + // Create a response channel + respCh := make(chan *preVoteResult, len(r.configurations.latest.Servers)) + + // Increment the term + newTerm := r.getCurrentTerm() + 1 + + // Construct the request + lastIdx, lastTerm := r.getLastEntry() + req := &RequestPreVoteRequest{ + RPCHeader: r.getRPCHeader(), + Term: newTerm, + // this is needed for retro compatibility, before RPCHeader.Addr was added + Candidate: r.trans.EncodePeer(r.localID, r.localAddr), + LastLogIndex: lastIdx, + LastLogTerm: lastTerm, + LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), + } + + // Construct a function to ask for a vote + askPeer := func(peer Server) { + r.goFunc(func() { + defer metrics.MeasureSince([]string{"raft", "candidate", "electSelf"}, time.Now()) + resp := &preVoteResult{voterID: peer.ID} + if prevoteTrans, ok := r.trans.(WithPreVote); ok { + err := prevoteTrans.RequestPreVote(peer.ID, peer.Address, req, &resp.RequestPreVoteResponse) + + // If the target server do not support Pre-vote RPC we count this as a granted vote to allow + // the cluster to progress. + if err != nil && strings.Contains(err.Error(), rpcUnexpectedCommandError) { + r.logger.Error("target do not support pre-vote RPC", + "target", peer, + "error", err, + "term", req.Term) + resp.Term = req.Term + resp.Granted = true + } else if err != nil { + r.logger.Error("failed to make requestVote RPC", + "target", peer, + "error", err, + "term", req.Term) + resp.Term = req.Term + resp.Granted = false + } + respCh <- resp + } + }) + } + + // For each peer, request a vote + for _, server := range r.configurations.latest.Servers { + if server.Suffrage == Voter { + if server.ID == r.localID { + r.logger.Debug("pre-voting for self", "term", req.Term, "id", r.localID) + // Persist a vote for ourselves + // Include our own vote + respCh <- &preVoteResult{ + RequestPreVoteResponse: RequestPreVoteResponse{ + RPCHeader: r.getRPCHeader(), + Term: req.Term, + Granted: true, + }, + voterID: r.localID, + } + } else { + r.logger.Debug("asking for pre-vote", "term", req.Term, "from", server.ID, "address", server.Address) + askPeer(server) + } + } + } + + return respCh +} + // persistVote is used to persist our vote for safety. func (r *Raft) persistVote(term uint64, candidate []byte) error { if err := r.stable.SetUint64(keyLastVoteTerm, term); err != nil { diff --git a/testing.go b/testing.go index bbfee16aa..3de4de3f8 100644 --- a/testing.go +++ b/testing.go @@ -211,7 +211,7 @@ func newTestLogger(tb testing.TB) hclog.Logger { // is logged after the test is complete. func newTestLoggerWithPrefix(tb testing.TB, prefix string) hclog.Logger { if testing.Verbose() { - return hclog.New(&hclog.LoggerOptions{Name: prefix}) + return hclog.New(&hclog.LoggerOptions{Name: prefix, Level: hclog.Trace}) } return hclog.New(&hclog.LoggerOptions{ diff --git a/transport.go b/transport.go index 054fa6241..c64fff6ec 100644 --- a/transport.go +++ b/transport.go @@ -66,6 +66,16 @@ type Transport interface { TimeoutNow(id ServerID, target ServerAddress, args *TimeoutNowRequest, resp *TimeoutNowResponse) error } +// WithPreVote is an interface that a transport may provide which +// allows a transport to support a PreVote request. +// +// It is defined separately from Transport as unfortunately it wasn't in the +// original interface specification. +type WithPreVote interface { + // RequestPreVote sends the appropriate RPC to the target node. + RequestPreVote(id ServerID, target ServerAddress, args *RequestPreVoteRequest, resp *RequestPreVoteResponse) error +} + // WithClose is an interface that a transport may provide which // allows a transport to be shut down cleanly when a Raft instance // shuts down. @@ -81,9 +91,10 @@ type WithClose interface { // LoopbackTransport is an interface that provides a loopback transport suitable for testing // e.g. InmemTransport. It's there so we don't have to rewrite tests. type LoopbackTransport interface { - Transport // Embedded transport reference - WithPeers // Embedded peer management - WithClose // with a close routine + Transport // Embedded transport reference + WithPeers // Embedded peer management + WithClose // with a close routine + WithPreVote // with a prevote } // WithPeers is an interface that a transport may provide which allows for connection and From 62bcfe721cd4210f7252c4d63efbd060349a8c57 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 25 Mar 2024 13:09:09 -0400 Subject: [PATCH 73/87] fix prevote tests and add rollback tests --- raft-compat/prevote_test.go | 173 ++++++++++++++++++++++++++--- raft-compat/testcluster/cluster.go | 73 ++++++++++-- 2 files changed, 219 insertions(+), 27 deletions(-) diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index c80ec731e..34971b893 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -20,10 +20,6 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { utils.WaitFuture(t, f) } leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { - if cluster.GetLeader().GetLocalID() == id { - transfer := cluster.Raft(id).(*raftprevious.Raft).LeadershipTransfer() - utils.WaitFuture(t, transfer) - } f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() utils.WaitFuture(t, f) } @@ -99,19 +95,157 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { utils.WaitFuture(t, future) //Add the new node to the cluster cluster.AddNode(rUIT) + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, a, leader) } + // keep a reference to the store + store := cluster.Store(getLeader.GetLocalID()) + id := getLeader.GetLocalID() + + //Remove and shutdown the leader node + tc.Leave(t, cluster, getLeader.GetLocalID()) + + // Delete the old leader node from the cluster + cluster.DeleteNode(getLeader.GetLocalID()) + oldLeaderID := getLeader.GetLocalID() + + // Wait for a new leader to be elected + utils.WaitForNewLeader(t, oldLeaderID, cluster) + getLeader = cluster.GetLeader() + require.NotEmpty(t, getLeader) + + // Create a new node to replace the deleted one + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVote = true }) + fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, fa) + + // Wait for new leader, (this happens because of not having prevote) + utils.WaitForNewLeader(t, "", cluster) + newLeaderID := rUIT.GetLeaderID() + require.NotEmpty(t, newLeaderID) - // Wait enough to have the configuration propagated. - time.Sleep(time.Second) + require.NotEqual(t, newLeaderID, leader) + newLeader := cluster.GetLeader() //Apply some logs - future = getLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + future = newLeader.GetRaft().(*raft.Raft).Apply([]byte("test2"), time.Second) require.NoError(t, future.Error()) // Check Leader haven't changed as we haven't replaced the leader yet - a, _ = getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() - require.Equal(t, a, leader) + newAddr, _ := newLeader.GetRaft().(*raft.Raft).LeaderWithID() + require.Equal(t, string(newAddr), newLeader.GetLocalAddr()) + + require.Equal(t, tc.numNodes, rUIT.NumLogs()) + }) + } + +} + +func TestRaft_PreVote_Rollback(t *testing.T) { + leaveTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + if cluster.GetLeader().GetLocalID() == id { + transfer := cluster.Raft(id).(*raft.Raft).LeadershipTransfer() + utils.WaitFuture(t, transfer) + } + f := cluster.Raft(id).(*raft.Raft).Shutdown() + utils.WaitFuture(t, f) + } + leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + f := cluster.Raft(id).(*raft.Raft).Shutdown() + utils.WaitFuture(t, f) + } + tcs := []struct { + name string + numNodes int + preVote bool + Leave func(t *testing.T, cluster testcluster.RaftCluster, id string) + }{ + {"no prevote -> prevote (leave transfer)", 3, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer)", 3, true, leaveNoTransfer}, + {"no prevote -> prevote (leave transfer) 5", 5, true, leaveTransfer}, + {"no prevote -> prevote (leave no transfer) 5", 5, true, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer)", 3, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer)", 3, false, leaveNoTransfer}, + {"no prevote -> no prevote (leave transfer) 5", 5, false, leaveTransfer}, + {"no prevote -> no prevote (leave no transfer) 5", 5, false, leaveNoTransfer}, + } + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + + cluster := testcluster.NewUITRaftCluster(t, tc.numNodes, "raftIUTNode") + configuration := raft.Configuration{} + + for i := 0; i < tc.numNodes; i++ { + var err error + require.NoError(t, err) + configuration.Servers = append(configuration.Servers, raft.Server{ + ID: raft.ServerID(cluster.ID(i)), + Address: raft.ServerAddress(cluster.Addr(i)), + }) + } + raft0 := cluster.Raft(cluster.ID(0)).(*raft.Raft) + boot := raft0.BootstrapCluster(configuration) + if err := boot.Error(); err != nil { + t.Fatalf("bootstrap err: %v", err) + } + utils.WaitForNewLeader(t, "", cluster) + getLeader := cluster.GetLeader() + require.NotEmpty(t, getLeader) + a, _ := getLeader.GetRaft().(*raft.Raft).LeaderWithID() + require.NotEmpty(t, a) + future := getLeader.GetRaft().(*raft.Raft).Apply([]byte("test"), time.Second) + utils.WaitFuture(t, future) + + leader, _ := getLeader.GetRaft().(*raft.Raft).LeaderWithID() + require.NotEmpty(t, leader) + // Upgrade all the followers + for i := 0; i < tc.numNodes; i++ { + if getLeader.GetLocalID() == cluster.ID(i) { + continue + } + + // Check Leader haven't changed + a, _ := getLeader.GetRaft().(*raft.Raft).LeaderWithID() + require.Equal(t, a, leader) + + // + tc.Leave(t, cluster, cluster.ID(i)) + // Keep the store, to be passed to the upgraded node. + store := cluster.Store(cluster.ID(i)) + id := cluster.ID(i) + + //Delete the node from the cluster + cluster.DeleteNode(cluster.ID(i)) + + //Create an upgraded node with the store + rUIT := testcluster.InitPreviousWithStore(t, id, store.(*raft.InmemStore), func(config *raftprevious.Config) { + }) + future := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + utils.WaitFuture(t, future) + //Add the new node to the cluster + cluster.AddNode(rUIT) + + // Wait enough to have the configuration propagated. + time.Sleep(time.Second) + + //Apply some logs + future = getLeader.GetRaft().(*raft.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + a, _ = getLeader.GetRaft().(*raft.Raft).LeaderWithID() + require.Equal(t, a, leader) + } // keep a reference to the store store := cluster.Store(getLeader.GetLocalID()) id := getLeader.GetLocalID() @@ -129,18 +263,27 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVote = true }) - fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) + rUIT := testcluster.InitPreviousWithStore(t, id, store.(*raft.InmemStore), func(config *raftprevious.Config) {}) + fa := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) // Wait for new leader, (this happens because of not having prevote) utils.WaitForNewLeader(t, "", cluster) - newLeader := rUIT.GetLeaderID() - require.NotEmpty(t, newLeader) + newLeaderID := rUIT.GetLeaderID() + require.NotEmpty(t, newLeaderID) - require.NotEqual(t, newLeader, leader) + require.NotEqual(t, newLeaderID, leader) + + newLeader := cluster.GetLeader() + //Apply some logs + future = newLeader.GetRaft().(*raftprevious.Raft).Apply([]byte("test2"), time.Second) + require.NoError(t, future.Error()) + + // Check Leader haven't changed as we haven't replaced the leader yet + newAddr, _ := newLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() + require.Equal(t, string(newAddr), newLeader.GetLocalAddr()) - require.Equal(t, rUIT.NumLogs(), 2) + require.Equal(t, tc.numNodes, rUIT.NumLogs()) }) } diff --git a/raft-compat/testcluster/cluster.go b/raft-compat/testcluster/cluster.go index 64fe04c9c..8930287a6 100644 --- a/raft-compat/testcluster/cluster.go +++ b/raft-compat/testcluster/cluster.go @@ -118,14 +118,12 @@ func NewRaftCluster(t *testing.T, f func(t *testing.T, id string) RaftNode, coun } func NewPreviousRaftCluster(t *testing.T, count int, name string) RaftCluster { - return NewRaftCluster(t, InitLatest, count, name) + return NewRaftCluster(t, InitPrevious, count, name) } -//func NewRaftNodeWitStore[T RaftNode](t *testing.T, name string, store *raftprevious.InmemStore) *T { -// raft := new(T) -// initNode(t, raft, name, store) -// return raft -//} +func NewUITRaftCluster(t *testing.T, count int, name string) RaftCluster { + return NewRaftCluster(t, InitUIT, count, name) +} func (r *RaftCluster) GetLeader() RaftNode { for _, n := range r.rafts { @@ -175,7 +173,7 @@ func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore, c node.id = raft.ServerID(id) node.Config.LocalID = node.id if store != nil { - node.Store = convertInMemStore(store) + node.Store = convertInMemStoreToUIT(store) } else { node.Store = raft.NewInmemStore() } @@ -191,7 +189,12 @@ func InitUITWithStore(t *testing.T, id string, store *raftprevious.InmemStore, c return node } -func InitLatest(t *testing.T, id string) RaftNode { +func InitPrevious(t *testing.T, id string) RaftNode { + return InitPreviousWithStore(t, id, nil, func(config *raftprevious.Config) { + }) +} + +func InitPreviousWithStore(t *testing.T, id string, store *raft.InmemStore, f func(config *raftprevious.Config)) RaftNode { node := RaftLatest{} node.Config = raftprevious.DefaultConfig() node.Config.HeartbeatTimeout = 50 * time.Millisecond @@ -200,8 +203,13 @@ func InitLatest(t *testing.T, id string) RaftNode { node.Config.CommitTimeout = 5 * time.Millisecond node.id = raftprevious.ServerID(id) node.Config.LocalID = node.id + f(node.Config) - node.Store = raftprevious.NewInmemStore() + if store != nil { + node.Store = convertInMemStoreToPrevious(store) + } else { + node.Store = raftprevious.NewInmemStore() + } node.Snap = raftprevious.NewInmemSnapshotStore() node.fsm = &raftprevious.MockFSM{} var err error @@ -213,7 +221,7 @@ func InitLatest(t *testing.T, id string) RaftNode { return node } -func convertLog(ll *raftprevious.Log) *raft.Log { +func convertLogToUIT(ll *raftprevious.Log) *raft.Log { l := new(raft.Log) l.Index = ll.Index l.AppendedAt = ll.AppendedAt @@ -223,6 +231,16 @@ func convertLog(ll *raftprevious.Log) *raft.Log { l.Extensions = ll.Extensions return l } +func convertLogToPrevious(ll *raft.Log) *raftprevious.Log { + l := new(raftprevious.Log) + l.Index = ll.Index + l.AppendedAt = ll.AppendedAt + l.Type = raftprevious.LogType(ll.Type) + l.Term = ll.Term + l.Data = ll.Data + l.Extensions = ll.Extensions + return l +} var ( keyCurrentTerm = []byte("CurrentTerm") @@ -230,14 +248,45 @@ var ( keyLastVoteCand = []byte("LastVoteCand") ) -func convertInMemStore(s *raftprevious.InmemStore) *raft.InmemStore { +func convertInMemStoreToPrevious(s *raft.InmemStore) *raftprevious.InmemStore { + ss := raftprevious.NewInmemStore() + fi, _ := s.FirstIndex() + li, _ := s.LastIndex() + for i := fi; i <= li; i++ { + log := new(raft.Log) + s.GetLog(i, log) + ss.StoreLog(convertLogToPrevious(log)) + } + + get, _ := ss.Get(keyCurrentTerm) + ss.Set(keyCurrentTerm, get) + + get, _ = ss.Get(keyLastVoteTerm) + ss.Set(keyLastVoteTerm, get) + + get, _ = ss.Get(keyLastVoteCand) + ss.Set(keyLastVoteCand, get) + + get64, _ := ss.GetUint64(keyCurrentTerm) + ss.SetUint64(keyCurrentTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteTerm) + ss.SetUint64(keyLastVoteTerm, get64) + + get64, _ = ss.GetUint64(keyLastVoteCand) + ss.SetUint64(keyLastVoteCand, get64) + + return ss +} + +func convertInMemStoreToUIT(s *raftprevious.InmemStore) *raft.InmemStore { ss := raft.NewInmemStore() fi, _ := s.FirstIndex() li, _ := s.LastIndex() for i := fi; i <= li; i++ { log := new(raftprevious.Log) s.GetLog(i, log) - ss.StoreLog(convertLog(log)) + ss.StoreLog(convertLogToUIT(log)) } get, _ := ss.Get(keyCurrentTerm) From 97f6bd4422c78b447f03190d47bb6785400b107a Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 25 Mar 2024 13:37:53 -0400 Subject: [PATCH 74/87] add a partitioned node prevote test --- raft_test.go | 31 +++++++++++++++++++++++++++++++ testing.go | 6 ++++++ 2 files changed, 37 insertions(+) diff --git a/raft_test.go b/raft_test.go index e462427e1..d20cdc2fd 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2119,6 +2119,37 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { } +func TestRaft_PreVoteAvoidElectionWithPartition(t *testing.T) { + // Make a prevote cluster. + conf := inmemConfig(t) + conf.PreVote = true + c := MakeCluster(5, t, conf) + defer c.Close() + + oldLeaderTerm := c.Leader().getCurrentTerm() + followers := c.Followers() + require.Len(t, followers, 4) + + //Partition a node and wait enough for it to increase its term + c.Partition([]ServerAddress{followers[0].localAddr}) + time.Sleep(10 * c.propagateTimeout) + + // Check the leader is stable and the followers are as expected + leaderTerm := c.Leader().getCurrentTerm() + require.Equal(t, leaderTerm, oldLeaderTerm) + require.Len(t, c.WaitForFollowers(3), 3) + + // reconnect the partitioned node + c.FullyConnect() + time.Sleep(3 * c.propagateTimeout) + + // Check that the number of followers increase and + require.Len(t, c.Followers(), 4) + leaderTerm = c.Leader().getCurrentTerm() + require.Equal(t, leaderTerm, oldLeaderTerm) + +} + func TestRaft_VotingGrant_WhenLeaderAvailable(t *testing.T) { conf := inmemConfig(t) conf.ProtocolVersion = 3 diff --git a/testing.go b/testing.go index 3de4de3f8..3b3185602 100644 --- a/testing.go +++ b/testing.go @@ -501,6 +501,12 @@ func (c *cluster) Leader() *Raft { // state. func (c *cluster) Followers() []*Raft { expFollowers := len(c.rafts) - 1 + return c.WaitForFollowers(expFollowers) +} + +// WaitForFollowers waits for the cluster to have a given number of followers and stay in a stable +// state. +func (c *cluster) WaitForFollowers(expFollowers int) []*Raft { followers := c.GetInState(Follower) if len(followers) != expFollowers { c.t.Fatalf("timeout waiting for %d followers (followers are %v)", expFollowers, followers) From 4e3bb65a573cf35c1fa30ed09d7d5ed8533b7adc Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 25 Mar 2024 14:06:10 -0400 Subject: [PATCH 75/87] remove server from config before shutting down, fix raft submodule --- .gitmodules | 2 +- raft-compat/prevote_test.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 28e632c85..cbcd5cc91 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,3 @@ [submodule "raft-compat/raft-latest"] path = raft-compat/raft-previous-version - url = git@github.com:hashicorp/raft + url = https://github.com/hashicorp/raft.git diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index 34971b893..86cbe1c5c 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -20,6 +20,8 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { utils.WaitFuture(t, f) } leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raftprevious.Raft).RemoveServer(raftprevious.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) f := cluster.Raft(id).(*raftprevious.Raft).Shutdown() utils.WaitFuture(t, f) } @@ -160,6 +162,8 @@ func TestRaft_PreVote_Rollback(t *testing.T) { utils.WaitFuture(t, f) } leaveNoTransfer := func(t *testing.T, cluster testcluster.RaftCluster, id string) { + fr := cluster.GetLeader().GetRaft().(*raft.Raft).RemoveServer(raft.ServerID(id), 0, 0) + utils.WaitFuture(t, fr) f := cluster.Raft(id).(*raft.Raft).Shutdown() utils.WaitFuture(t, f) } From 9ce972d9c62eb7dc8ec4fac9b1fa31dc1e9da633 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 26 Mar 2024 09:59:37 -0400 Subject: [PATCH 76/87] remove extra comment --- raft-compat/prevote_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index 86cbe1c5c..f2e1b161e 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -78,8 +78,6 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { // Check Leader haven't changed a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID() require.Equal(t, a, leader) - - // tc.Leave(t, cluster, cluster.ID(i)) // Keep the store, to be passed to the upgraded node. From 36690e5c6ed26ee4288f7f157f35e7a2892498de Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 26 Mar 2024 11:06:26 -0400 Subject: [PATCH 77/87] change `inmemConfig` to accept testing.TB --- integ_test.go | 3 +-- testing.go | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/integ_test.go b/integ_test.go index c16ffd94a..61557a355 100644 --- a/integ_test.go +++ b/integ_test.go @@ -73,8 +73,7 @@ func (r *RaftEnv) Restart(t *testing.T) { func MakeRaft(tb testing.TB, conf *Config, bootstrap bool) *RaftEnv { // Set the config if conf == nil { - t := tb.(*testing.T) - conf = inmemConfig(t) + conf = inmemConfig(tb) } dir, err := os.MkdirTemp("", "raft") diff --git a/testing.go b/testing.go index 3b3185602..351a9abab 100644 --- a/testing.go +++ b/testing.go @@ -21,7 +21,7 @@ import ( var userSnapshotErrorsOnNoData = true // Return configurations optimized for in-memory -func inmemConfig(t *testing.T) *Config { +func inmemConfig(t testing.TB) *Config { conf := DefaultConfig() conf.HeartbeatTimeout = 50 * time.Millisecond conf.ElectionTimeout = 50 * time.Millisecond From 43974d22809f9c597ba14e712f2b367b90098170 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 26 Mar 2024 11:07:12 -0400 Subject: [PATCH 78/87] remove stray comment --- raft-compat/prevote_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index f2e1b161e..383d19be6 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -218,8 +218,6 @@ func TestRaft_PreVote_Rollback(t *testing.T) { // Check Leader haven't changed a, _ := getLeader.GetRaft().(*raft.Raft).LeaderWithID() require.Equal(t, a, leader) - - // tc.Leave(t, cluster, cluster.ID(i)) // Keep the store, to be passed to the upgraded node. From 7543a3d80aeb304c8c734bf961dff517fe23f16c Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 26 Mar 2024 11:56:33 -0400 Subject: [PATCH 79/87] fix comments and remove extra fields --- commands.go | 4 ---- raft.go | 41 +++++++++++++++++++---------------------- raft_test.go | 2 +- 3 files changed, 20 insertions(+), 27 deletions(-) diff --git a/commands.go b/commands.go index b499f299a..9f500b2e5 100644 --- a/commands.go +++ b/commands.go @@ -92,7 +92,6 @@ type RequestVoteRequest struct { // transfer. It is required for leadership transfer to work, because servers // wouldn't vote otherwise if they are aware of an existing leader. LeadershipTransfer bool - PreVote bool } // GetRPCHeader - See WithRPCHeader. @@ -114,9 +113,6 @@ type RequestVoteResponse struct { // Is the vote granted. Granted bool - - // Is it a preVote response - PreVote bool } // GetRPCHeader - See WithRPCHeader. diff --git a/raft.go b/raft.go index 2eba75994..8d9da2870 100644 --- a/raft.go +++ b/raft.go @@ -325,8 +325,7 @@ func (r *Raft) runCandidate() { r.mainThreadSaturation.working() r.processRPC(rpc) case preVote := <-prevoteCh: - // This a pre-preVote case but could lead to winning an election, in the case that majority of other nodes - // don't support pre-preVote or have pre-preVote deactivated. + // This a pre-vote case it should trigger a "real" election if the pre-vote is won. r.mainThreadSaturation.working() r.logger.Debug("got a prevote!!", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) // Check if the term is greater than ours, bail @@ -346,7 +345,7 @@ func (r *Raft) runCandidate() { r.logger.Debug("prevote refused", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) } - // Check if we've won the pre-preVote and proceed to election if so + // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { r.logger.Info("pre election won", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 @@ -355,7 +354,7 @@ func (r *Raft) runCandidate() { prevoteCh = nil voteCh = r.electSelf() } - // Check if we've lost the pre-Vote and wait for the election to timeout so we can do another time of + // Check if we've lost the pre-vote and wait for the election to timeout so we can do another time of // prevote. if preVoteRefusedVotes >= votesNeeded { r.logger.Info("pre election lost, wait for election to timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) @@ -373,7 +372,7 @@ func (r *Raft) runCandidate() { // Check if the preVote is granted if vote.Granted { grantedVotes++ - r.logger.Debug("v ote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) + r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) } // Check if we've become the leader @@ -1762,16 +1761,16 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // For older raft version ID is not part of the packed message // We assume that the peer is part of the configuration and skip this check - if len(req.ID) > 0 { - candidateID := ServerID(req.ID) - // if the Servers list is empty that mean the cluster is very likely trying to bootstrap, - // Grant the vote - if len(r.configurations.latest.Servers) > 0 && !inConfiguration(r.configurations.latest, candidateID) { - r.logger.Warn("rejecting pre-vote request since node is not in configuration", - "from", candidate) - return - } + + candidateID := ServerID(req.ID) + // if the Servers list is empty that mean the cluster is very likely trying to bootstrap, + // Grant the vote + if len(r.configurations.latest.Servers) > 0 && !inConfiguration(r.configurations.latest, candidateID) { + r.logger.Warn("rejecting pre-vote request since node is not in configuration", + "from", candidate) + return } + if leaderAddr, leaderID := r.LeaderWithID(); leaderAddr != "" && leaderAddr != candidate && !req.LeadershipTransfer { r.logger.Warn("rejecting pre-vote request since we have a leader", "from", candidate, @@ -1787,10 +1786,9 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // Increase the term if we see a newer one if req.Term > r.getCurrentTerm() { - // Ensure we grant the pre-vote as in a "real" vote this will transition us to follower + // continue processing here to possibly grant the pre-vote as in a "real" vote this will transition us to follower r.logger.Debug("received a requestPreVote with a newer term, grant the pre-vote") resp.Term = req.Term - resp.Granted = true } // if we get a request for vote from a nonVoter and the request term is higher, @@ -1798,12 +1796,11 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // This could happen when a node, previously voter, is converted to non-voter // The reason we need to step in is to permit to the cluster to make progress in such a scenario // More details about that in https://github.com/hashicorp/raft/pull/526 - if len(req.ID) > 0 { - candidateID := ServerID(req.ID) - if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) { - r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate) - return - } + + candidateID := ServerID(req.ID) + if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) { + r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate) + return } // Reject if their term is older diff --git a/raft_test.go b/raft_test.go index d20cdc2fd..2501296bb 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2143,7 +2143,7 @@ func TestRaft_PreVoteAvoidElectionWithPartition(t *testing.T) { c.FullyConnect() time.Sleep(3 * c.propagateTimeout) - // Check that the number of followers increase and + // Check that the number of followers increase and the term is not increased require.Len(t, c.Followers(), 4) leaderTerm = c.Leader().getCurrentTerm() require.Equal(t, leaderTerm, oldLeaderTerm) From de78bf8e5ef5e06e41e2e91d9a9141e9b1942798 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 26 Mar 2024 12:22:09 -0400 Subject: [PATCH 80/87] remove duplicate var --- raft.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/raft.go b/raft.go index 8d9da2870..d78178b6c 100644 --- a/raft.go +++ b/raft.go @@ -1758,11 +1758,8 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // there is a known leader. But if the leader initiated a leadership transfer, // vote! var candidate ServerAddress - - // For older raft version ID is not part of the packed message - // We assume that the peer is part of the configuration and skip this check - candidateID := ServerID(req.ID) + // if the Servers list is empty that mean the cluster is very likely trying to bootstrap, // Grant the vote if len(r.configurations.latest.Servers) > 0 && !inConfiguration(r.configurations.latest, candidateID) { @@ -1796,8 +1793,6 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // This could happen when a node, previously voter, is converted to non-voter // The reason we need to step in is to permit to the cluster to make progress in such a scenario // More details about that in https://github.com/hashicorp/raft/pull/526 - - candidateID := ServerID(req.ID) if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) { r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate) return From 921be2d449a45b8f021ef1d1f9507f345edc3154 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 28 Mar 2024 09:16:08 -0400 Subject: [PATCH 81/87] remove leader transfer from pre-vote path, fix logs and comments. --- api.go | 3 ++- commands.go | 10 --------- config.go | 8 +++---- raft.go | 61 ++++++++++++++++++++++------------------------------ raft_test.go | 4 ++++ 5 files changed, 36 insertions(+), 50 deletions(-) diff --git a/api.go b/api.go index 8b3875c5b..6aa5c96c7 100644 --- a/api.go +++ b/api.go @@ -534,6 +534,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna applyCh = make(chan *logFuture, conf.MaxAppendEntries) } + _, transportSupportPreVote := trans.(WithPreVote) // Create Raft struct. r := &Raft{ protocolVersion: protocolVersion, @@ -563,7 +564,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna leaderNotifyCh: make(chan struct{}, 1), followerNotifyCh: make(chan struct{}, 1), mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), - preVote: conf.PreVote, + preVote: conf.PreVote && transportSupportPreVote, } r.conf.Store(*conf) diff --git a/commands.go b/commands.go index 9f500b2e5..af6a8505e 100644 --- a/commands.go +++ b/commands.go @@ -134,11 +134,6 @@ type RequestPreVoteRequest struct { // Used to ensure safety LastLogIndex uint64 LastLogTerm uint64 - - // Used to indicate to peers if this vote was triggered by a leadership - // transfer. It is required for leadership transfer to work, because servers - // wouldn't vote otherwise if they are aware of an existing leader. - LeadershipTransfer bool } // GetRPCHeader - See WithRPCHeader. @@ -153,11 +148,6 @@ type RequestPreVoteResponse struct { // Newer term if leader is out of date. Term uint64 - // Peers is deprecated, but required by servers that only understand - // protocol version 0. This is not populated in protocol version 2 - // and later. - Peers []byte - // Is the vote granted. Granted bool } diff --git a/config.go b/config.go index aad2ebd26..ba718ae7e 100644 --- a/config.go +++ b/config.go @@ -232,11 +232,11 @@ type Config struct { // raft's configuration and index values. NoSnapshotRestoreOnStart bool - // skipStartup allows NewRaft() to bypass all background work goroutines - skipStartup bool - // PreVote activate the pre-vote feature PreVote bool + + // skipStartup allows NewRaft() to bypass all background work goroutines + skipStartup bool } func (conf *Config) getOrCreateLogger() hclog.Logger { @@ -320,7 +320,7 @@ func DefaultConfig() *Config { SnapshotInterval: 120 * time.Second, SnapshotThreshold: 8192, LeaderLeaseTimeout: 500 * time.Millisecond, - LogLevel: "TRACE", + LogLevel: "DEBUG", } } diff --git a/raft.go b/raft.go index d78178b6c..6215f3ee0 100644 --- a/raft.go +++ b/raft.go @@ -291,10 +291,11 @@ func (r *Raft) runCandidate() { var voteCh <-chan *voteResult var prevoteCh <-chan *preVoteResult - // check if the transport support prevote requests - _, ok := r.trans.(WithPreVote) + // check if pre-vote is active and that this is not a leader transfer - if r.preVote && ok { + // Leader transfer do not perform prevote by design, as the selected server is very likely to be fit + // and an election will happen in all cases. + if r.preVote && !r.candidateFromLeadershipTransfer.Load() { prevoteCh = r.preElectSelf() } else { voteCh = r.electSelf() @@ -327,10 +328,10 @@ func (r *Raft) runCandidate() { case preVote := <-prevoteCh: // This a pre-vote case it should trigger a "real" election if the pre-vote is won. r.mainThreadSaturation.working() - r.logger.Debug("got a prevote!!", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) + r.logger.Debug("pre-vote received", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) // Check if the term is greater than ours, bail if preVote.Term > term { - r.logger.Debug("newer term discovered on pre-preVote, fallback to follower", "term", preVote.Term) + r.logger.Debug("pre-vote denied: found newer term, falling back to follower", "term", preVote.Term) r.setState(Follower) r.setCurrentTerm(preVote.Term) return @@ -339,15 +340,15 @@ func (r *Raft) runCandidate() { // Check if the preVote is granted if preVote.Granted { preVoteGrantedVotes++ - r.logger.Debug("prevote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) + r.logger.Debug("pre-vote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) } else { preVoteRefusedVotes++ - r.logger.Debug("prevote refused", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) + r.logger.Debug("pre-vote denied", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes) } // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre election won", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) preVoteGrantedVotes = 0 preVoteRefusedVotes = 0 electionTimer = randomTimeout(electionTimeout) @@ -357,7 +358,7 @@ func (r *Raft) runCandidate() { // Check if we've lost the pre-vote and wait for the election to timeout so we can do another time of // prevote. if preVoteRefusedVotes >= votesNeeded { - r.logger.Info("pre election lost, wait for election to timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + r.logger.Info("pre-vote campaign failed, waiting for election timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) } case vote := <-voteCh: r.mainThreadSaturation.working() @@ -369,7 +370,7 @@ func (r *Raft) runCandidate() { return } - // Check if the preVote is granted + // Check if the vote is granted if vote.Granted { grantedVotes++ r.logger.Debug("vote granted", "from", vote.voterID, "term", vote.Term, "tally", grantedVotes) @@ -1721,7 +1722,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { return } - // Persist a vote for safety\ + // Persist a vote for safety if err := r.persistVote(req.Term, candidateBytes); err != nil { r.logger.Error("failed to persist vote", "error", err) return @@ -1731,7 +1732,7 @@ func (r *Raft) requestVote(rpc RPC, req *RequestVoteRequest) { r.setLastContact() } -// requestPreVote is invoked when we get a request vote RPC call. +// requestPreVote is invoked when we get a request Pre-Vote RPC call. func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { defer metrics.MeasureSince([]string{"raft", "rpc", "requestVote"}, time.Now()) r.observe(*req) @@ -1747,12 +1748,6 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { rpc.Respond(resp, rpcErr) }() - // Version 0 servers will panic unless the peers is present. It's only - // used on them to produce a warning message. - if r.protocolVersion < 2 { - resp.Peers = encodePeers(r.configurations.latest, r.trans) - } - // Check if we have an existing leader [who's not the candidate] and also // check the LeadershipTransfer flag is set. Usually votes are rejected if // there is a known leader. But if the leader initiated a leadership transfer, @@ -1768,7 +1763,7 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { return } - if leaderAddr, leaderID := r.LeaderWithID(); leaderAddr != "" && leaderAddr != candidate && !req.LeadershipTransfer { + if leaderAddr, leaderID := r.LeaderWithID(); leaderAddr != "" && leaderAddr != candidate { r.logger.Warn("rejecting pre-vote request since we have a leader", "from", candidate, "leader", leaderAddr, @@ -1781,18 +1776,14 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { return } - // Increase the term if we see a newer one if req.Term > r.getCurrentTerm() { // continue processing here to possibly grant the pre-vote as in a "real" vote this will transition us to follower r.logger.Debug("received a requestPreVote with a newer term, grant the pre-vote") resp.Term = req.Term } - // if we get a request for vote from a nonVoter and the request term is higher, - // step down and update term, but reject the vote request + // if we get a request for a pre-vote from a nonVoter and the request term is higher, do not grant the Pre-Vote // This could happen when a node, previously voter, is converted to non-voter - // The reason we need to step in is to permit to the cluster to make progress in such a scenario - // More details about that in https://github.com/hashicorp/raft/pull/526 if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) { r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate) return @@ -1801,7 +1792,7 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { // Reject if their term is older lastIdx, lastTerm := r.getLastEntry() if lastTerm > req.LastLogTerm { - r.logger.Warn("rejecting vote request since our last term is greater", + r.logger.Warn("rejecting pre-vote request since our last term is greater", "candidate", candidate, "last-term", lastTerm, "last-candidate-term", req.LastLogTerm) @@ -1809,7 +1800,7 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { } if lastTerm == req.LastLogTerm && lastIdx > req.LastLogIndex { - r.logger.Warn("rejecting vote request since our last index is greater", + r.logger.Warn("rejecting pre-vote request since our last index is greater", "candidate", candidate, "last-index", lastIdx, "last-candidate-index", req.LastLogIndex) @@ -2055,10 +2046,11 @@ func (r *Raft) electSelf() <-chan *voteResult { return respCh } -// preElectSelf is used to send a RequestVote RPC to all peers, and vote for -// ourself. This has the side affecting of incrementing the current term. The +// preElectSelf is used to send a RequestPreVote RPC to all peers, and vote for +// ourself. This will not increment the current term. The // response channel returned is used to wait for all the responses (including a -// vote for ourself). This must only be called from the main thread. +// vote for ourself). +// This must only be called from the main thread. func (r *Raft) preElectSelf() <-chan *preVoteResult { // Create a response channel respCh := make(chan *preVoteResult, len(r.configurations.latest.Servers)) @@ -2072,10 +2064,9 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { RPCHeader: r.getRPCHeader(), Term: newTerm, // this is needed for retro compatibility, before RPCHeader.Addr was added - Candidate: r.trans.EncodePeer(r.localID, r.localAddr), - LastLogIndex: lastIdx, - LastLogTerm: lastTerm, - LeadershipTransfer: r.candidateFromLeadershipTransfer.Load(), + Candidate: r.trans.EncodePeer(r.localID, r.localAddr), + LastLogIndex: lastIdx, + LastLogTerm: lastTerm, } // Construct a function to ask for a vote @@ -2113,8 +2104,8 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { if server.Suffrage == Voter { if server.ID == r.localID { r.logger.Debug("pre-voting for self", "term", req.Term, "id", r.localID) - // Persist a vote for ourselves - // Include our own vote + + // cast a pre-vote for our self respCh <- &preVoteResult{ RequestPreVoteResponse: RequestPreVoteResponse{ RPCHeader: r.getRPCHeader(), diff --git a/raft_test.go b/raft_test.go index 2501296bb..55aaba220 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2061,6 +2061,10 @@ func TestRaft_AppendEntry(t *testing.T) { require.True(t, resp2.Success) } +// TestRaft_PreVoteMixedCluster focus on testing a cluster with +// a mix of nodes that have pre-vote activated and deactivated. +// Once the cluster is created, we force an election by partioning the leader +// and verify that the cluster regain stability. func TestRaft_PreVoteMixedCluster(t *testing.T) { tcs := []struct { From 4f6fc13a5b5b9c1d5e3eae680775fa8bf1f12ecc Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Thu, 28 Mar 2024 09:26:55 -0400 Subject: [PATCH 82/87] make pre-vote enabled by default --- api.go | 7 ++++--- config.go | 4 ++-- raft-compat/prevote_test.go | 4 ++-- raft.go | 8 +++----- raft_test.go | 6 +++--- 5 files changed, 14 insertions(+), 15 deletions(-) diff --git a/api.go b/api.go index 6aa5c96c7..5bc9d70f5 100644 --- a/api.go +++ b/api.go @@ -214,8 +214,9 @@ type Raft struct { // mainThreadSaturation measures the saturation of the main raft goroutine. mainThreadSaturation *saturationMetric - // preVote control if the pre-vote feature is activated - preVote bool + // preVoteDisabled control if the pre-vote feature is activated, + // prevote feature is disabled if set to true. + preVoteDisabled bool } // BootstrapCluster initializes a server's storage with the given cluster @@ -564,7 +565,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna leaderNotifyCh: make(chan struct{}, 1), followerNotifyCh: make(chan struct{}, 1), mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), - preVote: conf.PreVote && transportSupportPreVote, + preVoteDisabled: conf.PreVoteDisabled || !transportSupportPreVote, } r.conf.Store(*conf) diff --git a/config.go b/config.go index ba718ae7e..d14392fc3 100644 --- a/config.go +++ b/config.go @@ -232,8 +232,8 @@ type Config struct { // raft's configuration and index values. NoSnapshotRestoreOnStart bool - // PreVote activate the pre-vote feature - PreVote bool + // PreVoteDisabled deactivate the pre-vote feature when set to true + PreVoteDisabled bool // skipStartup allows NewRaft() to bypass all background work goroutines skipStartup bool diff --git a/raft-compat/prevote_test.go b/raft-compat/prevote_test.go index 383d19be6..b21cc02dc 100644 --- a/raft-compat/prevote_test.go +++ b/raft-compat/prevote_test.go @@ -89,7 +89,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { //Create an upgraded node with the store rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { - config.PreVote = tc.preVote + config.PreVoteDisabled = !tc.preVote }) future := getLeader.GetRaft().(*raftprevious.Raft).AddVoter(raftprevious.ServerID(rUIT.GetLocalID()), raftprevious.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, future) @@ -124,7 +124,7 @@ func TestRaft_PreVote_BootStrap_PreVote(t *testing.T) { require.NotEmpty(t, getLeader) // Create a new node to replace the deleted one - rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVote = true }) + rUIT := testcluster.InitUITWithStore(t, id, store.(*raftprevious.InmemStore), func(config *raft.Config) { config.PreVoteDisabled = false }) fa := getLeader.GetRaft().(*raft.Raft).AddVoter(raft.ServerID(rUIT.GetLocalID()), raft.ServerAddress(rUIT.GetLocalAddr()), 0, 0) utils.WaitFuture(t, fa) diff --git a/raft.go b/raft.go index 6215f3ee0..cbb12d947 100644 --- a/raft.go +++ b/raft.go @@ -291,11 +291,9 @@ func (r *Raft) runCandidate() { var voteCh <-chan *voteResult var prevoteCh <-chan *preVoteResult - // check if pre-vote is active and that this is not a leader transfer - - // Leader transfer do not perform prevote by design, as the selected server is very likely to be fit - // and an election will happen in all cases. - if r.preVote && !r.candidateFromLeadershipTransfer.Load() { + // check if pre-vote is active and that this is not a leader transfer. + // Leader transfer do not perform prevote by design + if !r.preVoteDisabled && !r.candidateFromLeadershipTransfer.Load() { prevoteCh = r.preElectSelf() } else { voteCh = r.electSelf() diff --git a/raft_test.go b/raft_test.go index 55aaba220..d6d2c0307 100644 --- a/raft_test.go +++ b/raft_test.go @@ -2091,13 +2091,13 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { } conf := inmemConfig(t) - conf.PreVote = tc.prevoteNum > tc.noprevoteNum + conf.PreVoteDisabled = tc.prevoteNum <= tc.noprevoteNum c := MakeCluster(majority, t, conf) defer c.Close() // Set up another server speaking protocol version 2. conf = inmemConfig(t) - conf.PreVote = tc.prevoteNum < tc.noprevoteNum + conf.PreVoteDisabled = tc.prevoteNum >= tc.noprevoteNum c1 := MakeClusterNoBootstrap(minority, t, conf) // Merge clusters. @@ -2126,7 +2126,7 @@ func TestRaft_PreVoteMixedCluster(t *testing.T) { func TestRaft_PreVoteAvoidElectionWithPartition(t *testing.T) { // Make a prevote cluster. conf := inmemConfig(t) - conf.PreVote = true + conf.PreVoteDisabled = false c := MakeCluster(5, t, conf) defer c.Close() From 1912201252579c050610c4a93ccfc05356647872 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Tue, 2 Apr 2024 10:27:42 -0400 Subject: [PATCH 83/87] remove `Candidate` field from pre-vote request --- commands.go | 3 --- raft.go | 6 ++---- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/commands.go b/commands.go index af6a8505e..1ec76cb27 100644 --- a/commands.go +++ b/commands.go @@ -128,9 +128,6 @@ type RequestPreVoteRequest struct { // Provide the term and our id Term uint64 - // Deprecated: use RPCHeader.Addr instead - Candidate []byte - // Used to ensure safety LastLogIndex uint64 LastLogTerm uint64 diff --git a/raft.go b/raft.go index cbb12d947..d809257b6 100644 --- a/raft.go +++ b/raft.go @@ -2059,10 +2059,8 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { // Construct the request lastIdx, lastTerm := r.getLastEntry() req := &RequestPreVoteRequest{ - RPCHeader: r.getRPCHeader(), - Term: newTerm, - // this is needed for retro compatibility, before RPCHeader.Addr was added - Candidate: r.trans.EncodePeer(r.localID, r.localAddr), + RPCHeader: r.getRPCHeader(), + Term: newTerm, LastLogIndex: lastIdx, LastLogTerm: lastTerm, } From 27a2e7d7ce5dc2a49b04e2135c91607c8a634925 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 8 Apr 2024 10:26:43 -0400 Subject: [PATCH 84/87] add warning when transport don't support prevote --- api.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/api.go b/api.go index 5bc9d70f5..cff2eaac2 100644 --- a/api.go +++ b/api.go @@ -567,6 +567,9 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second), preVoteDisabled: conf.PreVoteDisabled || !transportSupportPreVote, } + if !transportSupportPreVote && !conf.PreVoteDisabled { + r.logger.Warn("pre-vote is disabled because it is not supported by the Transport") + } r.conf.Store(*conf) From ddcbc506e2c04c0d580e63d7f734f43adf2bc063 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Mon, 8 Apr 2024 10:27:40 -0400 Subject: [PATCH 85/87] panic if transport is not supported in preElectSelf. --- raft.go | 55 ++++++++++++++++++++++++++++++------------------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/raft.go b/raft.go index d809257b6..94520c228 100644 --- a/raft.go +++ b/raft.go @@ -1747,9 +1747,6 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) { }() // Check if we have an existing leader [who's not the candidate] and also - // check the LeadershipTransfer flag is set. Usually votes are rejected if - // there is a known leader. But if the leader initiated a leadership transfer, - // vote! var candidate ServerAddress candidateID := ServerID(req.ID) @@ -2050,6 +2047,14 @@ func (r *Raft) electSelf() <-chan *voteResult { // vote for ourself). // This must only be called from the main thread. func (r *Raft) preElectSelf() <-chan *preVoteResult { + + // At this point transport should support pre-vote + // but check just in case + prevoteTrans, prevoteTransSupported := r.trans.(WithPreVote) + if !prevoteTransSupported { + panic("preElection is not possible if the transport don't support pre-vote") + } + // Create a response channel respCh := make(chan *preVoteResult, len(r.configurations.latest.Servers)) @@ -2068,30 +2073,30 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { // Construct a function to ask for a vote askPeer := func(peer Server) { r.goFunc(func() { - defer metrics.MeasureSince([]string{"raft", "candidate", "electSelf"}, time.Now()) + defer metrics.MeasureSince([]string{"raft", "candidate", "preElectSelf"}, time.Now()) resp := &preVoteResult{voterID: peer.ID} - if prevoteTrans, ok := r.trans.(WithPreVote); ok { - err := prevoteTrans.RequestPreVote(peer.ID, peer.Address, req, &resp.RequestPreVoteResponse) - - // If the target server do not support Pre-vote RPC we count this as a granted vote to allow - // the cluster to progress. - if err != nil && strings.Contains(err.Error(), rpcUnexpectedCommandError) { - r.logger.Error("target do not support pre-vote RPC", - "target", peer, - "error", err, - "term", req.Term) - resp.Term = req.Term - resp.Granted = true - } else if err != nil { - r.logger.Error("failed to make requestVote RPC", - "target", peer, - "error", err, - "term", req.Term) - resp.Term = req.Term - resp.Granted = false - } - respCh <- resp + + err := prevoteTrans.RequestPreVote(peer.ID, peer.Address, req, &resp.RequestPreVoteResponse) + + // If the target server do not support Pre-vote RPC we count this as a granted vote to allow + // the cluster to progress. + if err != nil && strings.Contains(err.Error(), rpcUnexpectedCommandError) { + r.logger.Error("target do not support pre-vote RPC", + "target", peer, + "error", err, + "term", req.Term) + resp.Term = req.Term + resp.Granted = true + } else if err != nil { + r.logger.Error("failed to make requestVote RPC", + "target", peer, + "error", err, + "term", req.Term) + resp.Term = req.Term + resp.Granted = false } + respCh <- resp + }) } From 083b811c2460e2e2d60923fc8666a3a480dbeb4d Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 29 May 2024 14:34:09 -0400 Subject: [PATCH 86/87] Fix comments and log string Co-authored-by: Paul Banks --- raft.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index 94520c228..a22842947 100644 --- a/raft.go +++ b/raft.go @@ -2058,7 +2058,7 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { // Create a response channel respCh := make(chan *preVoteResult, len(r.configurations.latest.Servers)) - // Increment the term + // Propose the next term without actually changing our state newTerm := r.getCurrentTerm() + 1 // Construct the request @@ -2081,7 +2081,7 @@ func (r *Raft) preElectSelf() <-chan *preVoteResult { // If the target server do not support Pre-vote RPC we count this as a granted vote to allow // the cluster to progress. if err != nil && strings.Contains(err.Error(), rpcUnexpectedCommandError) { - r.logger.Error("target do not support pre-vote RPC", + r.logger.Error("target does not support pre-vote RPC, treating as granted", "target", peer, "error", err, "term", req.Term) From d962c2418a0ff11c900e4f5eb5a70cfbe94bd987 Mon Sep 17 00:00:00 2001 From: Dhia Ayachi Date: Wed, 29 May 2024 14:54:02 -0400 Subject: [PATCH 87/87] Fix to log the right number for votesNeeded, added preVoteRefusedVotes to the log --- raft.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/raft.go b/raft.go index a22842947..183f041a4 100644 --- a/raft.go +++ b/raft.go @@ -346,7 +346,8 @@ func (r *Raft) runCandidate() { // Check if we've won the pre-vote and proceed to election if so if preVoteGrantedVotes >= votesNeeded { - r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, + "tally", preVoteGrantedVotes, "refused", preVoteRefusedVotes, "votesNeeded", votesNeeded) preVoteGrantedVotes = 0 preVoteRefusedVotes = 0 electionTimer = randomTimeout(electionTimeout) @@ -356,7 +357,8 @@ func (r *Raft) runCandidate() { // Check if we've lost the pre-vote and wait for the election to timeout so we can do another time of // prevote. if preVoteRefusedVotes >= votesNeeded { - r.logger.Info("pre-vote campaign failed, waiting for election timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1) + r.logger.Info("pre-vote campaign failed, waiting for election timeout", "term", preVote.Term, + "tally", preVoteGrantedVotes, "refused", preVoteRefusedVotes, "votesNeeded", votesNeeded) } case vote := <-voteCh: r.mainThreadSaturation.working()