From 4c7e095e7015e24caf3211aa75bf819faa4acff5 Mon Sep 17 00:00:00 2001 From: "Derrick J. Wippler" Date: Tue, 24 Nov 2020 16:56:10 -0600 Subject: [PATCH 01/15] initial leader election implementation --- election/cluster.go | 113 +++++++ election/election.go | 681 ++++++++++++++++++++++++++++++++++++++ election/election_test.go | 123 +++++++ election/rpc.go | 134 ++++++++ election/rpc_test.go | 126 +++++++ 5 files changed, 1177 insertions(+) create mode 100644 election/cluster.go create mode 100644 election/election.go create mode 100644 election/election_test.go create mode 100644 election/rpc.go create mode 100644 election/rpc_test.go diff --git a/election/cluster.go b/election/cluster.go new file mode 100644 index 00000000..ce9e65eb --- /dev/null +++ b/election/cluster.go @@ -0,0 +1,113 @@ +package election + +import ( + "context" + "sync" + + "github.com/mailgun/holster/v3/setter" +) + +type ObsPair struct { + From string + Leader string +} + +// Useful in tests where you need to simulate an election cluster +type TestCluster struct { + Nodes map[string]Candidate + ObserverCh chan ObsPair + errors map[string]error + lock sync.Mutex +} + +func NewTestCluster() *TestCluster { + return &TestCluster{ + Nodes: make(map[string]Candidate), + errors: make(map[string]error), + ObserverCh: make(chan ObsPair, 500), + } +} + +// Spawns a new node and adds it to the cluster +func (c *TestCluster) SpawnNode(name string, conf *Config) error { + setter.SetDefault(&conf, &Config{}) + var err error + + conf.Self = name + conf.SendRPC = func(ctx context.Context, peer string, req RPCRequest, resp *RPCResponse) error { + return c.sendRPC(name, peer, req, resp) + } + conf.Observer = func(s string) { + c.ObserverCh <- ObsPair{ + From: name, + Leader: s, + } + } + c.Nodes[name], err = SpawnCandidate(*conf) + + // Build a list of all the peers + var peers []string + for k, _ := range c.Nodes { + peers = append(peers, k) + } + + // Update our list of known peers + for _, v := range c.Nodes { + v.SetPeers(peers) + } + + return err +} + +type ClusterStatus map[string]string + +func (c *TestCluster) GetClusterStatus() ClusterStatus { + status := make(ClusterStatus) + for k, v := range c.Nodes { + status[k] = v.Leader() + } + return status +} + +func (c *TestCluster) GetLeader() Candidate { + for _, v := range c.Nodes { + if v.IsLeader() { + return v + } + } + return nil +} + +func (c *TestCluster) AddNetworkError(peer string, err error) { + c.lock.Lock() + defer c.lock.Unlock() + c.errors[peer] = err +} + +func (c *TestCluster) DelNetworkError(peer string) { + c.lock.Lock() + defer c.lock.Unlock() + delete(c.errors, peer) +} + +func (c *TestCluster) sendRPC(from string, to string, req RPCRequest, resp *RPCResponse) error { + c.lock.Lock() + defer c.lock.Unlock() + + if err, ok := c.errors[from]; ok { + return err + } + + if err, ok := c.errors[to]; ok { + return err + } + + c.Nodes[to].ReceiveRPC(req, resp) + return nil +} + +func (c *TestCluster) Close() { + for _, v := range c.Nodes { + v.Close() + } +} diff --git a/election/election.go b/election/election.go new file mode 100644 index 00000000..2be3a712 --- /dev/null +++ b/election/election.go @@ -0,0 +1,681 @@ +package election + +import ( + "context" + "errors" + "fmt" + "math/rand" + "sync" + "time" + + "github.com/mailgun/holster/v3/setter" + + "github.com/mailgun/holster/v3/slice" + "github.com/mailgun/holster/v3/syncutil" + "github.com/sirupsen/logrus" +) + +var ( + ErrInShutdown = errors.New("node is shutting down") +) + +type state uint32 + +const ( + // FollowerState means we are following the leader and expect + // to get heart beats regularly. This is the initial state, as + // we don't want to force an election when a new node joins + // the cluster. + FollowerState state = iota + // CandidateState means we are actively attempting to become leader + CandidateState + // LeaderState means we have received a quorum of votes while + // in CandidateState and have assumed leadership. + LeaderState + // ShutdownState means we are in the process of shutting down + ShutdownState +) + +func (s state) String() string { + switch s { + case FollowerState: + return "Follower" + case CandidateState: + return "Candidate" + case LeaderState: + return "Leader" + case ShutdownState: + return "Shutdown" + default: + return "Unknown" + } +} + +type Config struct { + // How long we should wait for a single network operation to complete. + NetworkTimeout time.Duration + + // How long followers should wait before they decide the leader + // lost connection to peers and therefore start a new election. + HeartBeatTimeout time.Duration + + // How long candidates should wait for an election to complete + // before starting a new one. + ElectionTimeout time.Duration + + // How long the leader should wait on heart beat responses from + // followers before it decides to step down as leader and start a + // new election. + LeaderQuorumTimeout time.Duration + + // The Initial list of peers to be considered in the election, including ourself. + Peers []string + + // The peer name of our self, as found in the Peers list + Self string + + // Called when the leader changes + Observer Observer + + // The logger used errors and warning + Log logrus.FieldLogger + + // Sends an RPC request to a peer, This function must be provided and can + // utilize any network communication the implementer wishes. If context cancelled + // should return an error. + SendRPC func(context.Context, string, RPCRequest, *RPCResponse) error +} + +type Observer func(string) + +type Candidate interface { + // Set the list of peers to be considered for the election, this list MUST + // include ourself as defined by `Config.Self`. + SetPeers([]string) error + + // If leader, resigns as leader and starts a new election that we will not + // participate in. + Resign() bool + + // Returns true if we are currently leader + IsLeader() bool + + // Returns the current leader + Leader() string + + // Returns the current state of this node + State() state + + // Called + ReceiveRPC(RPCRequest, *RPCResponse) + + // Cancels the election, resigns if we are leader and waits for all go + // routines to complete before returning. + Close() +} + +type candidate struct { + conf Config // The election configuration + state state // Current state of our node + vote struct { + CurrentTerm uint64 + LastTerm uint64 + LastCandidate string + } // Current state of the vote + currentTerm uint64 // The current term of the election when in candidate state + rpcCh chan RPCRequest // RPC Response channel, listen for for RPC responses on this channel + self string // Our name + lock sync.RWMutex // lock for peers and leader + peers []string + leader string + lastContact time.Time // The last successful contact with the leader (if we are a follower) + shutdownCh chan struct{} // Signals we are in shutdown + log logrus.FieldLogger + wg syncutil.WaitGroup +} + +func SpawnCandidate(conf Config) (Candidate, error) { + + if conf.Self == "" { + return nil, errors.New("refusing to spawn a new election candidate with no Config.Self defined") + } + + setter.SetDefault(&conf.Log, logrus.WithField("name", conf.Self)) + setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*30) + setter.SetDefault(&conf.HeartBeatTimeout, time.Second*5) + setter.SetDefault(&conf.ElectionTimeout, time.Second*10) + setter.SetDefault(&conf.NetworkTimeout, time.Second*2) + + c := &candidate{ + shutdownCh: make(chan struct{}), + rpcCh: make(chan RPCRequest, 5_000), + self: conf.Self, + conf: conf, + log: conf.Log, + } + c.wg.Go(c.run) + return c, c.SetPeers(conf.Peers) +} + +func (e *candidate) ReceiveRPC(req RPCRequest, resp *RPCResponse) { + req.respChan = make(chan RPCResponse, 1) + e.rpcCh <- req + + select { + case rpcResp := <-req.respChan: + *resp = rpcResp + case <-e.shutdownCh: + } +} + +func (e *candidate) SetPeers(peers []string) error { + e.lock.Lock() + defer e.lock.Unlock() + if !slice.ContainsString(e.self, peers, nil) { + return fmt.Errorf("peer list does not include self '%s'; refusing peer list", e.self) + } + e.peers = peers + return nil +} + +func (e *candidate) GetPeers() []string { + e.lock.RLock() + defer e.lock.RUnlock() + return e.peers +} + +func (e *candidate) State() state { + e.lock.RLock() + defer e.lock.RUnlock() + return e.state +} + +func (e *candidate) setState(state state) { + e.log.Debugf("State Change (%s)", state) + e.lock.RLock() + defer e.lock.RUnlock() + e.state = state +} + +func (e *candidate) IsLeader() bool { + e.lock.RLock() + defer e.lock.RUnlock() + return e.self == e.leader +} + +func (e *candidate) Leader() string { + e.lock.RLock() + defer e.lock.RUnlock() + return e.leader +} + +func (e *candidate) setLeader(leader string) { + e.lock.Lock() + defer e.lock.Unlock() + if e.leader != leader { + e.log.Debugf("Set Leader (%s)", leader) + e.leader = leader + if e.conf.Observer != nil { + e.conf.Observer(leader) + } + } +} + +func (e *candidate) Resign() bool { + respCh := make(chan RPCResponse, 1) + e.rpcCh <- RPCRequest{ + Request: ResignReq{}, + respChan: respCh, + } + + select { + case rpcResp := <-respCh: + resp, ok := rpcResp.Response.(ResignResp) + if !ok { + return false + } + if rpcResp.Error != "" { + return false + } + return resp.Success + case <-e.shutdownCh: + return false + } +} + +func (e *candidate) Close() { + close(e.shutdownCh) + e.wg.Wait() +} + +func (e *candidate) run() { + for { + e.log.Debug("main loop") + select { + case <-e.shutdownCh: + e.setLeader("") + e.setState(ShutdownState) + return + default: + } + + switch e.state { + case FollowerState: + e.runFollower() + case CandidateState: + e.runCandidate() + case LeaderState: + e.runLeader() + } + } +} + +func (e *candidate) runFollower() { + e.log.Infof("entering follower state, current leader is '%s'", e.Leader()) + heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) + + for e.state == FollowerState { + select { + case rpc := <-e.rpcCh: + e.processRPC(rpc) + case <-heartbeatTimer.C: + + // Check if we have had successful contact with the leader + if time.Now().Sub(e.lastContact) < e.conf.HeartBeatTimeout { + continue + } + + // Heartbeat failed! Transition to the candidate state + e.log.Warnf("heartbeat timeout, starting election; previous leader was '%s'", e.Leader()) + e.setLeader("") + e.setState(CandidateState) + return + case <-e.shutdownCh: + heartbeatTimer.Stop() + return + } + } +} + +func (e *candidate) runCandidate() { + e.log.Infof("entering candidate state; current term '%d'", e.currentTerm+1) + voteCh := make(<-chan VoteResp) + + // Each node will choose a random time to send their vote. This makes it more + // likely that the first node to send vote requests will win the election, and avoid + // a stalemate. + voteTimer := time.NewTimer(randomDuration(e.conf.HeartBeatTimeout / 10)) + // We re-start the vote if we have not received a heart beat from a chosen leader before + // this timer expires. + electionTimer := time.NewTimer(randomDuration(e.conf.ElectionTimeout)) + + // Tally the votes, need a simple majority + grantedVotes := 0 + votesNeeded := e.quorumSize() + e.log.Debugf("votes needed: %d", votesNeeded) + + for e.State() == CandidateState { + select { + case <-voteTimer.C: + voteCh = e.electSelf() + voteTimer.Stop() + case rpc := <-e.rpcCh: + e.processRPC(rpc) + case vote := <-voteCh: + // Check if the term is greater than ours, bail + if vote.Term > e.currentTerm { + e.log.Debug("newer term discovered, fallback to follower") + e.state = FollowerState + e.currentTerm = vote.Term + return + } + + // Check if the vote is granted + if vote.Granted { + grantedVotes++ + e.log.Debugf("vote granted from '%s' term '%d', tally '%d'", vote.Candidate, vote.Term, grantedVotes) + } + + // Check if we've become the leader + if grantedVotes >= votesNeeded { + e.log.Infof("election won! tally is '%d'", grantedVotes) + e.state = LeaderState + e.setLeader(e.self) + return + } + case <-electionTimer.C: + // Election failed! Restart the election. We simply return, which will kick us back into runCandidate + e.log.Warn("Election timeout reached, restarting election") + electionTimer.Stop() + return + case <-e.shutdownCh: + return + } + } +} + +// electSelf is used to send a SendVote() RPC to all peers with a 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. +func (e *candidate) electSelf() <-chan VoteResp { + peers := e.GetPeers() + respCh := make(chan VoteResp, len(peers)) + + // Increment the term + e.currentTerm++ + + // Construct a function to ask for a vote + askPeer := func(peer string, term uint64, self string) { + e.wg.Go(func() { + ctx, cancel := context.WithTimeout(context.Background(), e.conf.NetworkTimeout) + defer cancel() + + // Construct the request + req := RPCRequest{ + RPC: VoteRPC, + Request: VoteReq{ + Term: term, + Candidate: self, + }, + } + + var resp RPCResponse + if err := e.conf.SendRPC(ctx, peer, req, &resp); err != nil { + e.log.WithFields(logrus.Fields{"err": err, "peer": peer}). + Error("error during vote rpc") + vResp, ok := resp.Response.(VoteResp) + if !ok { + return + } + vResp.Term = term + vResp.Granted = false + respCh <- vResp + } + vResp, ok := resp.Response.(VoteResp) + if !ok { + return + } + respCh <- vResp + }) + } + + // For each peer, request a vote + for _, peer := range peers { + if peer == e.self { + // Persist a vote for ourselves + e.vote.LastCandidate = e.self + e.vote.LastTerm = e.currentTerm + + // Include our own vote + respCh <- VoteResp{ + Candidate: e.self, + Term: e.currentTerm, + Granted: true, + } + } else { + askPeer(peer, e.currentTerm, e.self) + } + } + return respCh +} + +func (e *candidate) runLeader() { + quorumTicker := time.NewTicker(e.conf.LeaderQuorumTimeout) + heartBeatTicker := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout / 10)) + heartBeatReplyCh := make(chan HeartBeatResp, 5_000) + peersLastContact := make(map[string]time.Time, len(e.GetPeers())) + + for e.state == LeaderState { + select { + case rpc := <-e.rpcCh: + e.processRPC(rpc) + case reply := <-heartBeatReplyCh: + // Is the reply from a peer we are familiar with? + if !slice.ContainsString(reply.From, e.GetPeers(), nil) { + e.log.WithField("peer", reply.From). + Debug("leader received heartbeat reply from peer not in our peer list; ignoring") + break + } + peersLastContact[reply.From] = time.Now() + case <-heartBeatTicker.C: + for _, peer := range e.GetPeers() { + e.sendHeartBeat(peer, heartBeatReplyCh) + } + case <-quorumTicker.C: + // Check if we have received contact from a quorum of nodes within the leader quorum timeout interval. + // If not, we step down as we may have lost connectivity. + contacted := 0 + now := time.Now() + for _, peer := range e.GetPeers() { + if peer == e.self { + contacted++ + continue + } + + lc, ok := peersLastContact[peer] + if !ok { + continue + } + diff := now.Sub(lc) + if diff >= e.conf.HeartBeatTimeout { + e.log.Warnf("no heartbeat response from '%s' for '%s'", peer, diff) + continue + } + contacted++ + } + + // Verify we can contact a quorum + quorum := e.quorumSize() + if contacted < quorum { + e.log.Warn("failed to receive heart beats from a quorum of peers; stepping down") + e.state = FollowerState + // TODO: Perhaps we send ResetElection to what peers we can? + // This would avoid having to wait for the heartbeat timeout + // to start a new election. + } + case <-e.shutdownCh: + e.state = ShutdownState + heartBeatTicker.Stop() + quorumTicker.Stop() + if e.IsLeader() { + // Notify all followers we are no longer leader + for _, peer := range e.GetPeers() { + e.sendElectionReset(peer) + } + } + } + } + e.lastContact = time.Now() + if e.IsLeader() { + e.setLeader("") + } + quorumTicker.Stop() +} + +func (e *candidate) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { + // Don't heartbeat ourself + if peer == e.self { + return + } + + e.wg.Go(func() { + var resp RPCResponse + req := RPCRequest{ + RPC: HeartBeatRPC, + Request: HeartBeatReq{ + From: e.self, + Term: e.currentTerm, + }, + } + + ctx, cancel := context.WithTimeout(context.Background(), e.conf.NetworkTimeout) + defer cancel() + if err := e.conf.SendRPC(ctx, peer, req, &resp); err != nil { + e.log.WithFields(logrus.Fields{"err": err, "peer": peer}). + Debug("error during heart beat rpc") + return + } + hResp, ok := resp.Response.(HeartBeatResp) + if !ok { + return + } + heartBeatReplyCh <- hResp + }) +} + +func (e *candidate) sendElectionReset(peer string) { + // Don't send election reset to ourself + if peer == e.self { + return + } + + e.wg.Go(func() { + ctx, cancel := context.WithTimeout(context.Background(), e.conf.NetworkTimeout) + defer cancel() + req := RPCRequest{RPC: ResetElectionRPC, Request: ResetElectionReq{}} + if err := e.conf.SendRPC(ctx, peer, req, &RPCResponse{}); err != nil { + e.log.WithFields(logrus.Fields{"err": err, "peer": peer}). + Debug("error during reset election rpc") + } + }) +} + +func (e *candidate) processRPC(rpc RPCRequest) { + // TODO: Should check for state = shutdown? + switch cmd := rpc.Request.(type) { + case VoteReq: + e.handleVote(rpc, cmd) + case ResetElectionReq: + e.handleResetElection(rpc) + case HeartBeatReq: + e.handleHeartBeat(rpc, cmd) + case ResignReq: + e.handleResign(rpc) + default: + e.log.Errorf("got unexpected command %#v", rpc.Request) + rpc.respond(rpc.RPC, nil, "unexpected command") + } +} + +// handleResign Notifies all followers that we are stepping down as leader. +// if we are leader returns Success = true +func (e *candidate) handleResign(rpc RPCRequest) { + e.setLeader("") + e.state = FollowerState + for _, peer := range e.GetPeers() { + e.sendElectionReset(peer) + } + rpc.respond(rpc.RPC, ResignReq{}, "") +} + +// handleResetElection resets our state and starts a new election +func (e *candidate) handleResetElection(rpc RPCRequest) { + e.setLeader("") + e.state = CandidateState + rpc.respond(rpc.RPC, ResetElectionResp{}, "") +} + +// handleHeartBeat handles heartbeat requests from the elected leader +func (e *candidate) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { + resp := HeartBeatResp{ + From: e.self, + Term: e.currentTerm, + Success: false, + } + + defer func() { + rpc.respond(rpc.RPC, resp, "") + }() + + // Ignore an older term + if req.Term < e.currentTerm { + return + } + + // Increase the term if we see a newer one, and transition to a follower if we + // receive a heart beat from someone else who thinks they are leader, as only + // leaders are allowed to send heartbeats. + // + // This might occur if 2 or more nodes think they are elected leader. In this + // case all leaders will emit heartbeats and both fall back to follower, from + // there the followers will timeout waiting for a heartbeat and the vote will + // occur again, hopefully this time without electing 2 leaders. + if req.Term > e.currentTerm || e.state != FollowerState { + e.state = FollowerState + e.currentTerm = req.Term + resp.Term = req.Term + return + } + + // Only the node with the most votes is the leader and should report heartbeats + e.setLeader(req.From) + + resp.Success = true + e.lastContact = time.Now() +} + +// handleVote determines who we will vote for this term +func (e *candidate) handleVote(rpc RPCRequest, req VoteReq) { + resp := VoteResp{ + Term: e.currentTerm, + Candidate: e.self, + Granted: false, + } + + defer func() { + rpc.respond(rpc.RPC, resp, "") + }() + + // Check if we have an existing leader (who's not the candidate). Votes are rejected + // if there is a known leader. If a leader wants to step down, they notify followers + // with the ResetElection RPC call. + leader := e.Leader() + if leader != "" && leader != req.Candidate { + e.log.Warnf("rejecting vote request from '%s' since we have leader '%s'", req.Candidate, leader) + return + } + + // Ignore an older term + if req.Term < e.currentTerm { + return + } + + // Increase the term if we see a newer one + if req.Term > e.currentTerm { + // Ensure transition to follower + e.log.Debugf("received a vote request with a newer term '%d'", req.Term) + e.setState(FollowerState) + e.currentTerm = req.Term + resp.Term = req.Term + } + + // Check if we've voted in this election before + if e.vote.LastTerm == req.Term && e.vote.LastCandidate != "" { + e.log.Infof("ignoring vote request from '%s'; already voted for '%s' in election '%d'", + req.Candidate, e.vote.LastCandidate, req.Term) + if e.vote.LastCandidate == req.Candidate { + e.log.Warnf("duplicate requestVote from candidate '%s'", req.Candidate) + resp.Granted = true + } + return + } + + // Always vote for the first candidate we receive a request from for this term + e.vote.LastTerm = req.Term + e.vote.LastCandidate = req.Candidate + + // Tell the requester we voted for him + resp.Granted = true + e.lastContact = time.Now() + return +} + +func (e *candidate) quorumSize() int { + return len(e.GetPeers())/2 + 1 +} + +// randomDuration returns a value that is between the minDur and 2x minDur. +func randomDuration(minDur time.Duration) time.Duration { + return minDur + time.Duration(rand.Int63())%minDur +} diff --git a/election/election_test.go b/election/election_test.go new file mode 100644 index 00000000..481b89b3 --- /dev/null +++ b/election/election_test.go @@ -0,0 +1,123 @@ +package election_test + +import ( + "errors" + "testing" + "time" + + "github.com/mailgun/holster/v3/election" + "github.com/mailgun/holster/v3/testutil" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" +) + +var ( + cfg *election.Config + ErrConnRefused = errors.New("connection refused") +) + +func init() { + logrus.SetLevel(logrus.DebugLevel) + cfg = &election.Config{ + NetworkTimeout: time.Second, + HeartBeatTimeout: time.Second, + LeaderQuorumTimeout: time.Second * 2, + ElectionTimeout: time.Second * 2, + } +} + +func TestSimpleElection(t *testing.T) { + c := election.NewTestCluster() + defer c.Close() + + // Ensure a single node becomes leader + c.SpawnNode("n0", cfg) + + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, election.ClusterStatus{ + "n0": "n0", + }, status) + }) + + // Added nodes should become followers + c.SpawnNode("n1", cfg) + c.SpawnNode("n2", cfg) + c.SpawnNode("n3", cfg) + c.SpawnNode("n4", cfg) + + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, election.ClusterStatus{ + "n0": "n0", + "n1": "n0", + "n2": "n0", + "n3": "n0", + "n4": "n0", + }, status) + }) + + c.Nodes["n0"].Resign() + + // Wait until n0 is no longer leader + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + candidate := c.GetLeader() + if !assert.NotNil(t, candidate) { + return + } + assert.NotEqual(t, "n0", candidate.Leader()) + }) + + for k, v := range c.Nodes { + t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) + } +} + +func TestLeaderDisconnect(t *testing.T) { + c := election.NewTestCluster() + defer c.Close() + + // Start with a known leader + c.SpawnNode("n0", cfg) + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, election.ClusterStatus{ + "n0": "n0", + }, status) + }) + + // Added nodes should become followers + c.SpawnNode("n1", cfg) + c.SpawnNode("n2", cfg) + c.SpawnNode("n3", cfg) + c.SpawnNode("n4", cfg) + + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, election.ClusterStatus{ + "n0": "n0", + "n1": "n0", + "n2": "n0", + "n3": "n0", + "n4": "n0", + }, status) + }) + + c.AddNetworkError("n0", ErrConnRefused) + defer c.DelNetworkError("n0") + + // Should lose leadership + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + candidate := c.Nodes["n0"] + if !assert.NotNil(t, candidate) { + return + } + assert.NotEqual(t, "n0", candidate.Leader()) + }) + + for k, v := range c.Nodes { + t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) + } +} + +// TODO: Test Follower disconnect (Should NOT force leader to diff --git a/election/rpc.go b/election/rpc.go new file mode 100644 index 00000000..7ccf2492 --- /dev/null +++ b/election/rpc.go @@ -0,0 +1,134 @@ +package election + +import ( + "encoding/json" +) + +type VoteResp struct { + // The address of the candidate + Candidate string `json:"candidate"` + // Newer term if leader is out of date. + Term uint64 `json:"term"` + // Is the vote granted. + Granted bool `json:"granted"` +} + +type VoteReq struct { + // The address of the candidate + Candidate string `json:"candidate-address"` + // Newer term if leader is out of date. + Term uint64 `json:"term"` +} + +type ResetElectionReq struct{} +type ResetElectionResp struct{} + +type ResignReq struct{} +type ResignResp struct { + Success bool `json:"success"` +} + +type HeartBeatReq struct { + From string `json:"from"` + Term uint64 `json:"term"` +} +type HeartBeatResp struct { + From string `json:"from"` + Term uint64 `json:"term"` + Success bool `json:"success"` +} + +type RPC string + +const ( + HeartBeatRPC = RPC("heartbeat") + VoteRPC = RPC("vote") + ResetElectionRPC = RPC("reset-election") + UnknownRPC = RPC("unknown") +) + +type RPCPayload struct { + RPC RPC `json:"rpc"` + Request json.RawMessage `json:"request,omitempty"` + Response json.RawMessage `json:"response,omitempty"` + Error string `json:"error,omitempty"` +} + +type RPCResponse struct { + RPC RPC + Response interface{} + Error string +} + +func (r *RPCResponse) UnmarshalJSON(s []byte) error { + var in RPCPayload + if err := json.Unmarshal(s, &in); err != nil { + return err + } + r.Error = in.Error + r.RPC = in.RPC + + switch in.RPC { + case HeartBeatRPC: + resp := HeartBeatResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp + } + return nil +} + +func (r RPCResponse) MarshalJSON() ([]byte, error) { + out, err := json.Marshal(r.Response) + if err != nil { + return nil, err + } + p := RPCPayload{ + Error: r.Error, + RPC: r.RPC, + Response: out, + } + return json.Marshal(p) +} + +type RPCRequest struct { + RPC RPC + Request interface{} + respChan chan RPCResponse +} + +func (r RPCRequest) MarshalJSON() ([]byte, error) { + out, err := json.Marshal(r.Request) + if err != nil { + return nil, err + } + p := RPCPayload{ + RPC: r.RPC, + Request: out, + } + return json.Marshal(p) +} + +func (r *RPCRequest) UnmarshalJSON(s []byte) error { + var in RPCPayload + if err := json.Unmarshal(s, &in); err != nil { + return err + } + r.RPC = in.RPC + + switch in.RPC { + case HeartBeatRPC: + req := HeartBeatReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req + } + return nil +} + +// respond is used to respond with a response, error or both +func (r *RPCRequest) respond(rpc RPC, resp interface{}, errorMsg string) { + r.respChan <- RPCResponse{RPC: rpc, Response: resp, Error: errorMsg} +} diff --git a/election/rpc_test.go b/election/rpc_test.go new file mode 100644 index 00000000..533c6120 --- /dev/null +++ b/election/rpc_test.go @@ -0,0 +1,126 @@ +package election_test + +import ( + "bytes" + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "net/http/httptest" + "testing" + + "github.com/mailgun/holster/v3/election" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestRPCRequest(t *testing.T) { + out := election.RPCRequest{ + RPC: election.HeartBeatRPC, + Request: election.HeartBeatReq{ + From: "node1", + Term: 1, + }, + } + + b, err := json.Marshal(out) + require.NoError(t, err) + assert.Equal(t, `{"rpc":"heartbeat","request":{"from":"node1","term":1}}`, string(b)) + + var in election.RPCRequest + err = json.Unmarshal(b, &in) + require.NoError(t, err) + assert.Equal(t, out, in) +} + +func TestRPCResponse(t *testing.T) { + out := election.RPCResponse{ + RPC: election.HeartBeatRPC, + Response: election.HeartBeatResp{ + From: "node1", + Success: true, + Term: 1, + }, + } + b, err := json.Marshal(out) + require.NoError(t, err) + assert.Equal(t, `{"rpc":"heartbeat","response":{"from":"node1","term":1,"success":true}}`, string(b)) + + var in election.RPCResponse + err = json.Unmarshal(b, &in) + require.NoError(t, err) + assert.Equal(t, out, in) +} + +func TestHTTPServer(t *testing.T) { + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var in election.RPCRequest + + b, err := ioutil.ReadAll(r.Body) + require.NoError(t, err) + require.NoError(t, json.Unmarshal(b, &in)) + var resp election.RPCResponse + + switch in.RPC { + case election.HeartBeatRPC: + resp = election.RPCResponse{ + RPC: election.HeartBeatRPC, + Response: election.HeartBeatResp{ + From: "node1", + Term: 10, + Success: true, + }, + } + default: + resp = election.RPCResponse{ + Error: fmt.Sprintf("invalid rpc request '%s'", in.RPC), + } + } + out, err := json.Marshal(resp) + require.NoError(t, err) + w.Write(out) + })) + defer ts.Close() + + // Marshall our request + b, err := json.Marshal(election.RPCRequest{ + RPC: election.HeartBeatRPC, + Request: election.HeartBeatReq{ + Term: 10, + From: "node10", + }, + }) + require.NoError(t, err) + + // Send the request to the server + req, err := http.NewRequest(http.MethodPost, ts.URL, bytes.NewBuffer(b)) + require.NoError(t, err) + resp, err := http.DefaultClient.Do(req) + require.NoError(t, err) + + // Unmarshall the response + var rpcResp election.RPCResponse + b, err = ioutil.ReadAll(resp.Body) + err = json.Unmarshal(b, &rpcResp) + require.NoError(t, err) + + // Should have the response we expect + hb := rpcResp.Response.(election.HeartBeatResp) + assert.Equal(t, uint64(10), hb.Term) + assert.Equal(t, "node1", hb.From) + assert.Equal(t, true, hb.Success) + + // Send an unknown rpc request to the server + req, err = http.NewRequest(http.MethodPost, ts.URL, bytes.NewBuffer([]byte(`{"rpc":"unknown"}`))) + require.NoError(t, err) + resp, err = http.DefaultClient.Do(req) + require.NoError(t, err) + + // Unmarshall the response + b, err = ioutil.ReadAll(resp.Body) + err = json.Unmarshal(b, &rpcResp) + require.NoError(t, err) + + // Should have the response we expect + assert.Equal(t, "invalid rpc request 'unknown'", rpcResp.Error) +} From e4e04fa55ecbb3207898a63667edd473c126857e Mon Sep 17 00:00:00 2001 From: "Derrick J. Wippler" Date: Tue, 1 Dec 2020 11:00:19 -0600 Subject: [PATCH 02/15] leader election testing --- election/election.go | 12 +++--- election/election_test.go | 87 ++++++++++++++++++++------------------- 2 files changed, 49 insertions(+), 50 deletions(-) diff --git a/election/election.go b/election/election.go index 2be3a712..b28251ff 100644 --- a/election/election.go +++ b/election/election.go @@ -593,19 +593,17 @@ func (e *candidate) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { return } - // Increase the term if we see a newer one, and transition to a follower if we - // receive a heart beat from someone else who thinks they are leader, as only - // leaders are allowed to send heartbeats. - // // This might occur if 2 or more nodes think they are elected leader. In this - // case all leaders will emit heartbeats and both fall back to follower, from + // case all leaders that emit heartbeats will both fall back to follower, from // there the followers will timeout waiting for a heartbeat and the vote will // occur again, hopefully this time without electing 2 leaders. - if req.Term > e.currentTerm || e.state != FollowerState { + // + // This can also occur if a follower loses connectivity to the rest of the cluster. + // In this case we become the follower of who ever sent us the heartbeat. + if e.state != FollowerState { e.state = FollowerState e.currentTerm = req.Term resp.Term = req.Term - return } // Only the node with the most votes is the leader and should report heartbeats diff --git a/election/election_test.go b/election/election_test.go index 481b89b3..92e43bf2 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -28,40 +28,36 @@ func init() { func TestSimpleElection(t *testing.T) { c := election.NewTestCluster() + createCluster(t, c) defer c.Close() - // Ensure a single node becomes leader - c.SpawnNode("n0", cfg) + c.Nodes["n0"].Resign() + // Wait until n0 is no longer leader testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { - status := c.GetClusterStatus() - assert.Equal(t, election.ClusterStatus{ - "n0": "n0", - }, status) + candidate := c.GetLeader() + if !assert.NotNil(t, candidate) { + return + } + assert.NotEqual(t, "n0", candidate.Leader()) }) - // Added nodes should become followers - c.SpawnNode("n1", cfg) - c.SpawnNode("n2", cfg) - c.SpawnNode("n3", cfg) - c.SpawnNode("n4", cfg) + for k, v := range c.Nodes { + t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) + } +} - testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { - status := c.GetClusterStatus() - assert.Equal(t, election.ClusterStatus{ - "n0": "n0", - "n1": "n0", - "n2": "n0", - "n3": "n0", - "n4": "n0", - }, status) - }) +func TestLeaderDisconnect(t *testing.T) { + c := election.NewTestCluster() + createCluster(t, c) + defer c.Close() - c.Nodes["n0"].Resign() + c.AddNetworkError("n0", ErrConnRefused) + defer c.DelNetworkError("n0") - // Wait until n0 is no longer leader + // Should lose leadership testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { - candidate := c.GetLeader() + candidate := c.Nodes["n0"] if !assert.NotNil(t, candidate) { return } @@ -73,10 +69,33 @@ func TestSimpleElection(t *testing.T) { } } -func TestLeaderDisconnect(t *testing.T) { +func TestFollowerDisconnect(t *testing.T) { c := election.NewTestCluster() + createCluster(t, c) defer c.Close() + c.AddNetworkError("n4", ErrConnRefused) + defer c.DelNetworkError("n4") + + // Wait until n4 loses leader + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.NotEqual(t, "n0", status["n4"]) + }) + + c.DelNetworkError("n4") + + // Follower should resume being a follower without forcing a new election. + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, "n0", status["n4"]) + }) + +} + +func createCluster(t *testing.T, c *election.TestCluster) { + t.Helper() + // Start with a known leader c.SpawnNode("n0", cfg) testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { @@ -102,22 +121,4 @@ func TestLeaderDisconnect(t *testing.T) { "n4": "n0", }, status) }) - - c.AddNetworkError("n0", ErrConnRefused) - defer c.DelNetworkError("n0") - - // Should lose leadership - testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { - candidate := c.Nodes["n0"] - if !assert.NotNil(t, candidate) { - return - } - assert.NotEqual(t, "n0", candidate.Leader()) - }) - - for k, v := range c.Nodes { - t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) - } } - -// TODO: Test Follower disconnect (Should NOT force leader to From 03f1862d0798f8f08fb2cbea9f45641decaebe9c Mon Sep 17 00:00:00 2001 From: "Derrick J. Wippler" Date: Tue, 1 Dec 2020 14:31:58 -0600 Subject: [PATCH 03/15] Refactor election --- election/cluster.go | 113 ---------------------------- election/cluster_test.go | 151 ++++++++++++++++++++++++++++++++++++++ election/election.go | 141 +++++++++++++++++++---------------- election/election_test.go | 125 +++++++++++++++++++++++-------- 4 files changed, 320 insertions(+), 210 deletions(-) delete mode 100644 election/cluster.go create mode 100644 election/cluster_test.go diff --git a/election/cluster.go b/election/cluster.go deleted file mode 100644 index ce9e65eb..00000000 --- a/election/cluster.go +++ /dev/null @@ -1,113 +0,0 @@ -package election - -import ( - "context" - "sync" - - "github.com/mailgun/holster/v3/setter" -) - -type ObsPair struct { - From string - Leader string -} - -// Useful in tests where you need to simulate an election cluster -type TestCluster struct { - Nodes map[string]Candidate - ObserverCh chan ObsPair - errors map[string]error - lock sync.Mutex -} - -func NewTestCluster() *TestCluster { - return &TestCluster{ - Nodes: make(map[string]Candidate), - errors: make(map[string]error), - ObserverCh: make(chan ObsPair, 500), - } -} - -// Spawns a new node and adds it to the cluster -func (c *TestCluster) SpawnNode(name string, conf *Config) error { - setter.SetDefault(&conf, &Config{}) - var err error - - conf.Self = name - conf.SendRPC = func(ctx context.Context, peer string, req RPCRequest, resp *RPCResponse) error { - return c.sendRPC(name, peer, req, resp) - } - conf.Observer = func(s string) { - c.ObserverCh <- ObsPair{ - From: name, - Leader: s, - } - } - c.Nodes[name], err = SpawnCandidate(*conf) - - // Build a list of all the peers - var peers []string - for k, _ := range c.Nodes { - peers = append(peers, k) - } - - // Update our list of known peers - for _, v := range c.Nodes { - v.SetPeers(peers) - } - - return err -} - -type ClusterStatus map[string]string - -func (c *TestCluster) GetClusterStatus() ClusterStatus { - status := make(ClusterStatus) - for k, v := range c.Nodes { - status[k] = v.Leader() - } - return status -} - -func (c *TestCluster) GetLeader() Candidate { - for _, v := range c.Nodes { - if v.IsLeader() { - return v - } - } - return nil -} - -func (c *TestCluster) AddNetworkError(peer string, err error) { - c.lock.Lock() - defer c.lock.Unlock() - c.errors[peer] = err -} - -func (c *TestCluster) DelNetworkError(peer string) { - c.lock.Lock() - defer c.lock.Unlock() - delete(c.errors, peer) -} - -func (c *TestCluster) sendRPC(from string, to string, req RPCRequest, resp *RPCResponse) error { - c.lock.Lock() - defer c.lock.Unlock() - - if err, ok := c.errors[from]; ok { - return err - } - - if err, ok := c.errors[to]; ok { - return err - } - - c.Nodes[to].ReceiveRPC(req, resp) - return nil -} - -func (c *TestCluster) Close() { - for _, v := range c.Nodes { - v.Close() - } -} diff --git a/election/cluster_test.go b/election/cluster_test.go new file mode 100644 index 00000000..eb30fdae --- /dev/null +++ b/election/cluster_test.go @@ -0,0 +1,151 @@ +package election_test + +import ( + "context" + "fmt" + "sync" + + "github.com/mailgun/holster/v3/election" + "github.com/mailgun/holster/v3/setter" +) + +type ObsPair struct { + From string + Leader string +} + +// Useful in tests where you need to simulate an election cluster +type TestCluster struct { + Nodes map[string]*ClusterNode + ObserverCh chan ObsPair + errors map[string]error + lock sync.Mutex +} + +type ClusterNode struct { + lock sync.RWMutex + Node election.Node + SendRPC func(from string, to string, req election.RPCRequest, resp *election.RPCResponse) error +} + +func NewTestCluster() *TestCluster { + return &TestCluster{ + Nodes: make(map[string]*ClusterNode), + errors: make(map[string]error), + ObserverCh: make(chan ObsPair, 500), + } +} + +// Spawns a new node and adds it to the cluster +func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { + setter.SetDefault(&conf, &election.Config{}) + n := &ClusterNode{ + SendRPC: c.sendRPC, + } + + conf.Self = name + conf.SendRPC = func(ctx context.Context, peer string, req election.RPCRequest, resp *election.RPCResponse) error { + n.lock.RLock() + defer n.lock.RUnlock() + return n.SendRPC(name, peer, req, resp) + } + conf.Observer = func(s string) { + c.ObserverCh <- ObsPair{ + From: name, + Leader: s, + } + } + var err error + n.Node, err = election.SpawnNode(*conf) + if err != nil { + return err + } + c.Add(name, n) + return nil +} + +func (c *TestCluster) Add(name string, node *ClusterNode) { + c.Nodes[name] = node + node.lock.Lock() + defer node.lock.Unlock() + node.SendRPC = c.sendRPC + c.updatePeers() +} + +func (c *TestCluster) Remove(name string) *ClusterNode { + n := c.Nodes[name] + delete(c.Nodes, name) + c.updatePeers() + return n +} + +func (c *TestCluster) updatePeers() { + // Build a list of all the peers + var peers []string + for k, _ := range c.Nodes { + peers = append(peers, k) + } + + // Update our list of known peers + for _, v := range c.Nodes { + v.Node.SetPeers(peers) + } +} + +type ClusterStatus map[string]string + +func (c *TestCluster) GetClusterStatus() ClusterStatus { + status := make(ClusterStatus) + for k, v := range c.Nodes { + status[k] = v.Node.Leader() + } + return status +} + +func (c *TestCluster) GetLeader() election.Node { + for _, v := range c.Nodes { + if v.Node.IsLeader() { + return v.Node + } + } + return nil +} + +func (c *TestCluster) AddNetworkError(peer string, err error) { + c.lock.Lock() + defer c.lock.Unlock() + c.errors[peer] = err +} + +func (c *TestCluster) DelNetworkError(peer string) { + c.lock.Lock() + defer c.lock.Unlock() + delete(c.errors, peer) +} + +func (c *TestCluster) sendRPC(from string, to string, req election.RPCRequest, resp *election.RPCResponse) error { + c.lock.Lock() + defer c.lock.Unlock() + + if err, ok := c.errors[from]; ok { + return err + } + + if err, ok := c.errors[to]; ok { + return err + } + + n, ok := c.Nodes[to] + if !ok { + return fmt.Errorf("unknown peer '%s'", to) + } + n.Node.ReceiveRPC(req, resp) + + return nil +} + +func (c *TestCluster) Close() { + for _, v := range c.Nodes { + v.Node.Close() + } +} diff --git a/election/election.go b/election/election.go index b28251ff..995e6be7 100644 --- a/election/election.go +++ b/election/election.go @@ -3,30 +3,24 @@ package election import ( "context" "errors" - "fmt" "math/rand" "sync" "time" "github.com/mailgun/holster/v3/setter" - "github.com/mailgun/holster/v3/slice" "github.com/mailgun/holster/v3/syncutil" "github.com/sirupsen/logrus" ) -var ( - ErrInShutdown = errors.New("node is shutting down") -) - -type state uint32 +type State uint32 const ( // FollowerState means we are following the leader and expect // to get heart beats regularly. This is the initial state, as // we don't want to force an election when a new node joins // the cluster. - FollowerState state = iota + FollowerState State = iota // CandidateState means we are actively attempting to become leader CandidateState // LeaderState means we have received a quorum of votes while @@ -36,7 +30,7 @@ const ( ShutdownState ) -func (s state) String() string { +func (s State) String() string { switch s { case FollowerState: return "Follower" @@ -88,7 +82,7 @@ type Config struct { type Observer func(string) -type Candidate interface { +type Node interface { // Set the list of peers to be considered for the election, this list MUST // include ourself as defined by `Config.Self`. SetPeers([]string) error @@ -104,7 +98,7 @@ type Candidate interface { Leader() string // Returns the current state of this node - State() state + State() State // Called ReceiveRPC(RPCRequest, *RPCResponse) @@ -114,9 +108,9 @@ type Candidate interface { Close() } -type candidate struct { +type node struct { conf Config // The election configuration - state state // Current state of our node + state State // Current state of our node vote struct { CurrentTerm uint64 LastTerm uint64 @@ -134,10 +128,11 @@ type candidate struct { wg syncutil.WaitGroup } -func SpawnCandidate(conf Config) (Candidate, error) { +// Spawns a new node that will participate in the election. +func SpawnNode(conf Config) (Node, error) { if conf.Self == "" { - return nil, errors.New("refusing to spawn a new election candidate with no Config.Self defined") + return nil, errors.New("refusing to spawn a new node with no Config.Self defined") } setter.SetDefault(&conf.Log, logrus.WithField("name", conf.Self)) @@ -146,7 +141,7 @@ func SpawnCandidate(conf Config) (Candidate, error) { setter.SetDefault(&conf.ElectionTimeout, time.Second*10) setter.SetDefault(&conf.NetworkTimeout, time.Second*2) - c := &candidate{ + c := &node{ shutdownCh: make(chan struct{}), rpcCh: make(chan RPCRequest, 5_000), self: conf.Self, @@ -157,7 +152,8 @@ func SpawnCandidate(conf Config) (Candidate, error) { return c, c.SetPeers(conf.Peers) } -func (e *candidate) ReceiveRPC(req RPCRequest, resp *RPCResponse) { +// Called by the implementer when an RPC is received from another node +func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { req.respChan = make(chan RPCResponse, 1) e.rpcCh <- req @@ -168,48 +164,51 @@ func (e *candidate) ReceiveRPC(req RPCRequest, resp *RPCResponse) { } } -func (e *candidate) SetPeers(peers []string) error { +// SetPeers is a thread safe way to dynamically add or remove peers in a running cluster. +// These peers will be contacted when requesting votes during leader election. +func (e *node) SetPeers(peers []string) error { e.lock.Lock() defer e.lock.Unlock() - if !slice.ContainsString(e.self, peers, nil) { - return fmt.Errorf("peer list does not include self '%s'; refusing peer list", e.self) - } e.peers = peers return nil } -func (e *candidate) GetPeers() []string { +// GetPeers returns the current peers this node knows about. +func (e *node) GetPeers() []string { e.lock.RLock() defer e.lock.RUnlock() return e.peers } -func (e *candidate) State() state { +// State returns the current state of this node +func (e *node) State() State { e.lock.RLock() defer e.lock.RUnlock() return e.state } -func (e *candidate) setState(state state) { +func (e *node) setState(state State) { e.log.Debugf("State Change (%s)", state) e.lock.RLock() defer e.lock.RUnlock() e.state = state } -func (e *candidate) IsLeader() bool { +// IsLeader returns true if this node was elected leader +func (e *node) IsLeader() bool { e.lock.RLock() defer e.lock.RUnlock() return e.self == e.leader } -func (e *candidate) Leader() string { +// Leader returns the name of the node that is currently leader +func (e *node) Leader() string { e.lock.RLock() defer e.lock.RUnlock() return e.leader } -func (e *candidate) setLeader(leader string) { +func (e *node) setLeader(leader string) { e.lock.Lock() defer e.lock.Unlock() if e.leader != leader { @@ -221,7 +220,9 @@ func (e *candidate) setLeader(leader string) { } } -func (e *candidate) Resign() bool { +// Resign will cause this node to step down as leader, if this +// node is NOT leader, this does nothing and returns 'false' +func (e *node) Resign() bool { respCh := make(chan RPCResponse, 1) e.rpcCh <- RPCRequest{ Request: ResignReq{}, @@ -243,12 +244,14 @@ func (e *candidate) Resign() bool { } } -func (e *candidate) Close() { +// Close closes all internal go routines and if this node is currently +// leader, resigns as leader. +func (e *node) Close() { close(e.shutdownCh) e.wg.Wait() } -func (e *candidate) run() { +func (e *node) run() { for { e.log.Debug("main loop") select { @@ -270,8 +273,8 @@ func (e *candidate) run() { } } -func (e *candidate) runFollower() { - e.log.Infof("entering follower state, current leader is '%s'", e.Leader()) +func (e *node) runFollower() { + e.log.Debugf("entering follower state, current leader is '%s'", e.Leader()) heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) for e.state == FollowerState { @@ -286,7 +289,7 @@ func (e *candidate) runFollower() { } // Heartbeat failed! Transition to the candidate state - e.log.Warnf("heartbeat timeout, starting election; previous leader was '%s'", e.Leader()) + e.log.Debugf("heartbeat timeout, starting election; previous leader was '%s'", e.Leader()) e.setLeader("") e.setState(CandidateState) return @@ -297,8 +300,8 @@ func (e *candidate) runFollower() { } } -func (e *candidate) runCandidate() { - e.log.Infof("entering candidate state; current term '%d'", e.currentTerm+1) +func (e *node) runCandidate() { + e.log.Debugf("entering candidate state; current term '%d'", e.currentTerm+1) voteCh := make(<-chan VoteResp) // Each node will choose a random time to send their vote. This makes it more @@ -338,14 +341,14 @@ func (e *candidate) runCandidate() { // Check if we've become the leader if grantedVotes >= votesNeeded { - e.log.Infof("election won! tally is '%d'", grantedVotes) + e.log.Debugf("election won! tally is '%d'", grantedVotes) e.state = LeaderState e.setLeader(e.self) return } case <-electionTimer.C: // Election failed! Restart the election. We simply return, which will kick us back into runCandidate - e.log.Warn("Election timeout reached, restarting election") + e.log.Debug("Election timeout reached, restarting election") electionTimer.Stop() return case <-e.shutdownCh: @@ -358,7 +361,7 @@ func (e *candidate) runCandidate() { // 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. -func (e *candidate) electSelf() <-chan VoteResp { +func (e *node) electSelf() <-chan VoteResp { peers := e.GetPeers() respCh := make(chan VoteResp, len(peers)) @@ -400,27 +403,28 @@ func (e *candidate) electSelf() <-chan VoteResp { }) } + // Vote for ourselves first + e.vote.LastCandidate = e.self + e.vote.LastTerm = e.currentTerm + + // Include our own vote + respCh <- VoteResp{ + Candidate: e.self, + Term: e.currentTerm, + Granted: true, + } + // For each peer, request a vote for _, peer := range peers { if peer == e.self { - // Persist a vote for ourselves - e.vote.LastCandidate = e.self - e.vote.LastTerm = e.currentTerm - - // Include our own vote - respCh <- VoteResp{ - Candidate: e.self, - Term: e.currentTerm, - Granted: true, - } - } else { - askPeer(peer, e.currentTerm, e.self) + continue } + askPeer(peer, e.currentTerm, e.self) } return respCh } -func (e *candidate) runLeader() { +func (e *node) runLeader() { quorumTicker := time.NewTicker(e.conf.LeaderQuorumTimeout) heartBeatTicker := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout / 10)) heartBeatReplyCh := make(chan HeartBeatResp, 5_000) @@ -459,7 +463,7 @@ func (e *candidate) runLeader() { } diff := now.Sub(lc) if diff >= e.conf.HeartBeatTimeout { - e.log.Warnf("no heartbeat response from '%s' for '%s'", peer, diff) + e.log.Debugf("no heartbeat response from '%s' for '%s'", peer, diff) continue } contacted++ @@ -468,7 +472,7 @@ func (e *candidate) runLeader() { // Verify we can contact a quorum quorum := e.quorumSize() if contacted < quorum { - e.log.Warn("failed to receive heart beats from a quorum of peers; stepping down") + e.log.Debug("failed to receive heart beats from a quorum of peers; stepping down") e.state = FollowerState // TODO: Perhaps we send ResetElection to what peers we can? // This would avoid having to wait for the heartbeat timeout @@ -493,7 +497,7 @@ func (e *candidate) runLeader() { quorumTicker.Stop() } -func (e *candidate) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { +func (e *node) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { // Don't heartbeat ourself if peer == e.self { return @@ -524,7 +528,7 @@ func (e *candidate) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatRe }) } -func (e *candidate) sendElectionReset(peer string) { +func (e *node) sendElectionReset(peer string) { // Don't send election reset to ourself if peer == e.self { return @@ -541,7 +545,7 @@ func (e *candidate) sendElectionReset(peer string) { }) } -func (e *candidate) processRPC(rpc RPCRequest) { +func (e *node) processRPC(rpc RPCRequest) { // TODO: Should check for state = shutdown? switch cmd := rpc.Request.(type) { case VoteReq: @@ -560,7 +564,7 @@ func (e *candidate) processRPC(rpc RPCRequest) { // handleResign Notifies all followers that we are stepping down as leader. // if we are leader returns Success = true -func (e *candidate) handleResign(rpc RPCRequest) { +func (e *node) handleResign(rpc RPCRequest) { e.setLeader("") e.state = FollowerState for _, peer := range e.GetPeers() { @@ -570,14 +574,14 @@ func (e *candidate) handleResign(rpc RPCRequest) { } // handleResetElection resets our state and starts a new election -func (e *candidate) handleResetElection(rpc RPCRequest) { +func (e *node) handleResetElection(rpc RPCRequest) { e.setLeader("") e.state = CandidateState rpc.respond(rpc.RPC, ResetElectionResp{}, "") } // handleHeartBeat handles heartbeat requests from the elected leader -func (e *candidate) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { +func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { resp := HeartBeatResp{ From: e.self, Term: e.currentTerm, @@ -598,6 +602,9 @@ func (e *candidate) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // there the followers will timeout waiting for a heartbeat and the vote will // occur again, hopefully this time without electing 2 leaders. // + // It's also possible that one leader sends it's heartbeats before the other leader, + // in that case the first leader to send a heartbeat becomes leader. + // // This can also occur if a follower loses connectivity to the rest of the cluster. // In this case we become the follower of who ever sent us the heartbeat. if e.state != FollowerState { @@ -614,7 +621,7 @@ func (e *candidate) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { } // handleVote determines who we will vote for this term -func (e *candidate) handleVote(rpc RPCRequest, req VoteReq) { +func (e *node) handleVote(rpc RPCRequest, req VoteReq) { resp := VoteResp{ Term: e.currentTerm, Candidate: e.self, @@ -630,7 +637,7 @@ func (e *candidate) handleVote(rpc RPCRequest, req VoteReq) { // with the ResetElection RPC call. leader := e.Leader() if leader != "" && leader != req.Candidate { - e.log.Warnf("rejecting vote request from '%s' since we have leader '%s'", req.Candidate, leader) + e.log.Debugf("rejecting vote request from '%s' since we have leader '%s'", req.Candidate, leader) return } @@ -650,10 +657,10 @@ func (e *candidate) handleVote(rpc RPCRequest, req VoteReq) { // Check if we've voted in this election before if e.vote.LastTerm == req.Term && e.vote.LastCandidate != "" { - e.log.Infof("ignoring vote request from '%s'; already voted for '%s' in election '%d'", + e.log.Debugf("ignoring vote request from '%s'; already voted for '%s' in election '%d'", req.Candidate, e.vote.LastCandidate, req.Term) if e.vote.LastCandidate == req.Candidate { - e.log.Warnf("duplicate requestVote from candidate '%s'", req.Candidate) + e.log.Debugf("duplicate requestVote from candidate '%s'", req.Candidate) resp.Granted = true } return @@ -669,8 +676,12 @@ func (e *candidate) handleVote(rpc RPCRequest, req VoteReq) { return } -func (e *candidate) quorumSize() int { - return len(e.GetPeers())/2 + 1 +func (e *node) quorumSize() int { + size := len(e.GetPeers()) + if size == 0 { + return 1 + } + return size/2 + 1 } // randomDuration returns a value that is between the minDur and 2x minDur. diff --git a/election/election_test.go b/election/election_test.go index 92e43bf2..bea32f63 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -6,9 +6,11 @@ import ( "time" "github.com/mailgun/holster/v3/election" + "github.com/mailgun/holster/v3/slice" "github.com/mailgun/holster/v3/testutil" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) var ( @@ -26,12 +28,43 @@ func init() { } } +func createCluster(t *testing.T, c *TestCluster) { + t.Helper() + + // Start with a known leader + err := c.SpawnNode("n0", cfg) + require.NoError(t, err) + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, ClusterStatus{ + "n0": "n0", + }, status) + }) + + // Added nodes should become followers + c.SpawnNode("n1", cfg) + c.SpawnNode("n2", cfg) + c.SpawnNode("n3", cfg) + c.SpawnNode("n4", cfg) + + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, ClusterStatus{ + "n0": "n0", + "n1": "n0", + "n2": "n0", + "n3": "n0", + "n4": "n0", + }, status) + }) +} + func TestSimpleElection(t *testing.T) { - c := election.NewTestCluster() + c := NewTestCluster() createCluster(t, c) defer c.Close() - c.Nodes["n0"].Resign() + c.Nodes["n0"].Node.Resign() // Wait until n0 is no longer leader testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { @@ -43,12 +76,12 @@ func TestSimpleElection(t *testing.T) { }) for k, v := range c.Nodes { - t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) + t.Logf("Node: %s Leader: %t\n", k, v.Node.IsLeader()) } } func TestLeaderDisconnect(t *testing.T) { - c := election.NewTestCluster() + c := NewTestCluster() createCluster(t, c) defer c.Close() @@ -57,20 +90,20 @@ func TestLeaderDisconnect(t *testing.T) { // Should lose leadership testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { - candidate := c.Nodes["n0"] - if !assert.NotNil(t, candidate) { + node := c.Nodes["n0"] + if !assert.NotNil(t, node.Node) { return } - assert.NotEqual(t, "n0", candidate.Leader()) + assert.NotEqual(t, "n0", node.Node.Leader()) }) for k, v := range c.Nodes { - t.Logf("Node: %s Leader: %t\n", k, v.IsLeader()) + t.Logf("Node: %s Leader: %t\n", k, v.Node.IsLeader()) } } func TestFollowerDisconnect(t *testing.T) { - c := election.NewTestCluster() + c := NewTestCluster() createCluster(t, c) defer c.Close() @@ -90,35 +123,63 @@ func TestFollowerDisconnect(t *testing.T) { status := c.GetClusterStatus() assert.Equal(t, "n0", status["n4"]) }) - } -func createCluster(t *testing.T, c *election.TestCluster) { - t.Helper() +func TestSplitBrain(t *testing.T) { + c1 := NewTestCluster() + createCluster(t, c1) + defer c1.Close() - // Start with a known leader - c.SpawnNode("n0", cfg) - testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { - status := c.GetClusterStatus() - assert.Equal(t, election.ClusterStatus{ - "n0": "n0", - }, status) + c2 := NewTestCluster() + + // Now take 2 nodes from cluster 1 and put them in their own cluster. + // This causes n0 to lose contact with n2-n4 and should update the member list + // such that n0 only knows about n1. + + // Since n0 was leader previously, it should remain leader + c2.Add("n0", c1.Remove("n0")) + c2.Add("n1", c1.Remove("n1")) + + // Cluster 1 should elect a new leader + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + assert.NotNil(t, c1.GetLeader()) }) - // Added nodes should become followers - c.SpawnNode("n1", cfg) - c.SpawnNode("n2", cfg) - c.SpawnNode("n3", cfg) - c.SpawnNode("n4", cfg) + for k, v := range c1.Nodes { + t.Logf("C1 Node: %s Leader: %t\n", k, v.Node.IsLeader()) + } + + // Cluster 2 should elect a new leader + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + assert.NotNil(t, c2.GetLeader()) + }) + for k, v := range c2.Nodes { + t.Logf("C2 Node: %s Leader: %t\n", k, v.Node.IsLeader()) + } + + // Move the nodes in cluster2, back to the cluster1 + c1.Add("n0", c2.Remove("n0")) + c1.Add("n1", c2.Remove("n1")) + + // The nodes should detect 2 leaders and start a new vote. testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { - status := c.GetClusterStatus() - assert.Equal(t, election.ClusterStatus{ - "n0": "n0", - "n1": "n0", - "n2": "n0", - "n3": "n0", - "n4": "n0", - }, status) + status := c1.GetClusterStatus() + var leaders []string + for _, v := range status { + if slice.ContainsString(v, leaders, nil) { + continue + } + leaders = append(leaders, v) + } + if !assert.NotNil(t, leaders) { + return + } + assert.Equal(t, 1, len(leaders)) + assert.NotEmpty(t, leaders[0]) }) + + for k, v := range c1.Nodes { + t.Logf("Node: %s Leader: %t\n", k, v.Node.IsLeader()) + } } From baa9f6c9b00f264a1992f101933f60d72a2166fb Mon Sep 17 00:00:00 2001 From: "Derrick J. Wippler" Date: Tue, 1 Dec 2020 17:12:11 -0600 Subject: [PATCH 04/15] more tests and documentation --- election/README.md | 15 +++ election/election.go | 6 +- election/example_test.go | 106 ++++++++++++++++++ election/gen_proto.sh | 29 +++++ election/rpc.go | 71 ++++++------ election/rpc_test.go | 141 ++++++++++++++++++------ election/structs.pb.go | 226 +++++++++++++++++++++++++++++++++++++++ election/structs.proto | 48 +++++++++ go.mod | 1 + 9 files changed, 574 insertions(+), 69 deletions(-) create mode 100644 election/README.md create mode 100644 election/example_test.go create mode 100755 election/gen_proto.sh create mode 100644 election/structs.pb.go create mode 100644 election/structs.proto diff --git a/election/README.md b/election/README.md new file mode 100644 index 00000000..07132409 --- /dev/null +++ b/election/README.md @@ -0,0 +1,15 @@ +## Election Library +This is a network agnostic implementation of the leader election portion of the RAFT protocol. This library provides no +peer discovery mechanism, as such the user of this library must call `SetPeers()` on the node when the list of peers +changes. Users can use any third party service discovery mechanism, such as consul, etc, k8s, or memberlist. + +For our internal uses we choose https://github.com/hashicorp/memberlist such that our services have as few external +dependencies as possible. + +### Usage +In order to use the library on the network the user must provide a `SendRPC()` function at initialization time. This +function will be called when RPC communication between `election.Node` is needed. A node that wishes to receive +an RPC call must in turn call `Node.ReceiveRPC()` when the RPC request is received by what ever network protocol the +user implements. + +You can see a simple example of this using http by looking at `example_test.go` \ No newline at end of file diff --git a/election/election.go b/election/election.go index 995e6be7..efd99a2e 100644 --- a/election/election.go +++ b/election/election.go @@ -583,9 +583,8 @@ func (e *node) handleResetElection(rpc RPCRequest) { // handleHeartBeat handles heartbeat requests from the elected leader func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { resp := HeartBeatResp{ - From: e.self, - Term: e.currentTerm, - Success: false, + From: e.self, + Term: e.currentTerm, } defer func() { @@ -616,7 +615,6 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // Only the node with the most votes is the leader and should report heartbeats e.setLeader(req.From) - resp.Success = true e.lastContact = time.Now() } diff --git a/election/example_test.go b/election/example_test.go new file mode 100644 index 00000000..07345296 --- /dev/null +++ b/election/example_test.go @@ -0,0 +1,106 @@ +package election_test + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "log" + "net/http" + "testing" + + "github.com/mailgun/holster/v3/election" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" +) + +func sendRPC(ctx context.Context, peer string, req election.RPCRequest, resp *election.RPCResponse) error { + // Marshall the RPC request to json + b, err := json.Marshal(req) + if err != nil { + return errors.Wrap(err, "while encoding request") + } + + // Create a new http request with context + hr, err := http.NewRequest(http.MethodPost, fmt.Sprintf("http://%s/rpc", peer), bytes.NewBuffer(b)) + if err != nil { + return errors.Wrap(err, "while creating request") + } + hr.WithContext(ctx) + + // Send the request + hp, err := http.DefaultClient.Do(hr) + if err != nil { + return errors.Wrap(err, "while sending http request") + } + + // Decode the response from JSON + dec := json.NewDecoder(hp.Body) + if err := dec.Decode(&resp); err != nil { + return errors.Wrap(err, "while decoding response") + } + return nil +} + +func newHandler(node election.Node) func(w http.ResponseWriter, r *http.Request) { + return func(w http.ResponseWriter, r *http.Request) { + dec := json.NewDecoder(r.Body) + var req election.RPCRequest + if err := dec.Decode(&req); err != nil { + w.WriteHeader(http.StatusBadRequest) + w.Write([]byte(err.Error())) + } + var resp election.RPCResponse + node.ReceiveRPC(req, &resp) + + enc := json.NewEncoder(w) + if err := enc.Encode(resp); err != nil { + w.WriteHeader(http.StatusInternalServerError) + w.Write([]byte(err.Error())) + } + } +} + +// This example spawns 2 nodes, in a real application you would +// only spawn a single node which would represent your application +// in the election. +func SimpleExample(t *testing.T) { + logrus.SetLevel(logrus.DebugLevel) + + node1, err := election.SpawnNode(election.Config{ + // A list of known peers at startup + Peers: []string{"localhost:7080", "localhost:7081"}, + // A unique identifier used to identify us in a list of peers + Self: "localhost:7080", + // Called whenever the library detects a change in leadership + Observer: func(leader string) { + log.Printf("Current Leader: %s\n", leader) + }, + // Called when the library wants to contact other peers + SendRPC: sendRPC, + }) + if err != nil { + log.Fatal(err) + } + defer node1.Close() + + node2, err := election.SpawnNode(election.Config{ + Peers: []string{"localhost:7080", "localhost:7081"}, + Self: "localhost:7081", + SendRPC: sendRPC, + }) + if err != nil { + log.Fatal(err) + } + defer node2.Close() + + go func() { + mux := http.NewServeMux() + mux.HandleFunc("/rpc", newHandler(node1)) + log.Fatal(http.ListenAndServe(":7080", mux)) + }() + + mux := http.NewServeMux() + mux.HandleFunc("/rpc", newHandler(node2)) + log.Fatal(http.ListenAndServe(":7081", mux)) +} diff --git a/election/gen_proto.sh b/election/gen_proto.sh new file mode 100755 index 00000000..cd1dba9a --- /dev/null +++ b/election/gen_proto.sh @@ -0,0 +1,29 @@ +#! /bin/sh + +# Copyright 2018-2019 Mailgun Technologies Inc +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +# Make sure the script fails fast. +set -e +set -u +set -x + +PROTO_DIR=. +GO_DIR=. + +protoc -I=$PROTO_DIR \ + --go_out=plugins=grpc:$GO_DIR \ + $PROTO_DIR/*.proto diff --git a/election/rpc.go b/election/rpc.go index 7ccf2492..93190d3a 100644 --- a/election/rpc.go +++ b/election/rpc.go @@ -4,46 +4,13 @@ import ( "encoding/json" ) -type VoteResp struct { - // The address of the candidate - Candidate string `json:"candidate"` - // Newer term if leader is out of date. - Term uint64 `json:"term"` - // Is the vote granted. - Granted bool `json:"granted"` -} - -type VoteReq struct { - // The address of the candidate - Candidate string `json:"candidate-address"` - // Newer term if leader is out of date. - Term uint64 `json:"term"` -} - -type ResetElectionReq struct{} -type ResetElectionResp struct{} - -type ResignReq struct{} -type ResignResp struct { - Success bool `json:"success"` -} - -type HeartBeatReq struct { - From string `json:"from"` - Term uint64 `json:"term"` -} -type HeartBeatResp struct { - From string `json:"from"` - Term uint64 `json:"term"` - Success bool `json:"success"` -} - type RPC string const ( HeartBeatRPC = RPC("heartbeat") VoteRPC = RPC("vote") ResetElectionRPC = RPC("reset-election") + ResignRPC = RPC("resign") UnknownRPC = RPC("unknown") ) @@ -75,6 +42,24 @@ func (r *RPCResponse) UnmarshalJSON(s []byte) error { return err } r.Response = resp + case VoteRPC: + resp := VoteResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp + case ResetElectionRPC: + resp := ResetElectionResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp + case ResignRPC: + resp := ResignResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp } return nil } @@ -124,6 +109,24 @@ func (r *RPCRequest) UnmarshalJSON(s []byte) error { return err } r.Request = req + case VoteRPC: + req := VoteReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req + case ResetElectionRPC: + req := ResetElectionReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req + case ResignRPC: + req := ResignReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req } return nil } diff --git a/election/rpc_test.go b/election/rpc_test.go index 533c6120..330ad791 100644 --- a/election/rpc_test.go +++ b/election/rpc_test.go @@ -15,41 +15,122 @@ import ( ) func TestRPCRequest(t *testing.T) { - out := election.RPCRequest{ - RPC: election.HeartBeatRPC, - Request: election.HeartBeatReq{ - From: "node1", - Term: 1, + for _, tt := range []struct { + name string + in election.RPCRequest + out string + }{ + { + name: "heartbeat", + in: election.RPCRequest{ + RPC: election.HeartBeatRPC, + Request: election.HeartBeatReq{ + From: "node1", + Term: 1, + }, + }, + out: `{"rpc":"heartbeat","request":{"from":"node1","term":1}}`, }, - } + { + name: "vote", + in: election.RPCRequest{ + RPC: election.VoteRPC, + Request: election.VoteReq{ + Candidate: "node1", + Term: 1, + }, + }, + out: `{"rpc":"vote","request":{"candidate":"node1","term":1}}`, + }, + { + name: "reset", + in: election.RPCRequest{ + RPC: election.ResetElectionRPC, + Request: election.ResetElectionReq{}, + }, + out: `{"rpc":"reset-election","request":{}}`, + }, + { + name: "resign", + in: election.RPCRequest{ + RPC: election.ResignRPC, + Request: election.ResignReq{}, + }, + out: `{"rpc":"resign","request":{}}`, + }, + } { + b, err := json.Marshal(tt.in) + require.NoError(t, err) + assert.Equal(t, tt.out, string(b)) - b, err := json.Marshal(out) - require.NoError(t, err) - assert.Equal(t, `{"rpc":"heartbeat","request":{"from":"node1","term":1}}`, string(b)) + var in election.RPCRequest + err = json.Unmarshal(b, &in) + require.NoError(t, err) + assert.Equal(t, tt.in, in) + } - var in election.RPCRequest - err = json.Unmarshal(b, &in) - require.NoError(t, err) - assert.Equal(t, out, in) } func TestRPCResponse(t *testing.T) { - out := election.RPCResponse{ - RPC: election.HeartBeatRPC, - Response: election.HeartBeatResp{ - From: "node1", - Success: true, - Term: 1, + for _, tt := range []struct { + name string + in election.RPCResponse + out string + }{ + { + name: "heartbeat", + in: election.RPCResponse{ + RPC: election.HeartBeatRPC, + Response: election.HeartBeatResp{ + From: "node1", + Term: 1, + }, + }, + out: `{"rpc":"heartbeat","response":{"from":"node1","term":1}}`, + }, + { + name: "vote", + in: election.RPCResponse{ + RPC: election.VoteRPC, + Response: election.VoteResp{ + Candidate: "node1", + Term: 1, + Granted: true, + }, + }, + out: `{"rpc":"vote","response":{"candidate":"node1","term":1,"granted":true}}`, + }, + { + name: "reset", + in: election.RPCResponse{ + RPC: election.ResetElectionRPC, + Response: election.ResetElectionResp{}, + }, + out: `{"rpc":"reset-election","response":{}}`, }, + { + name: "resign", + in: election.RPCResponse{ + RPC: election.ResignRPC, + Response: election.ResignResp{ + Success: true, + }, + }, + out: `{"rpc":"resign","response":{"success":true}}`, + }, + } { + t.Run(tt.name, func(t *testing.T) { + b, err := json.Marshal(tt.in) + require.NoError(t, err) + assert.Equal(t, tt.out, string(b)) + + var in election.RPCResponse + err = json.Unmarshal(b, &in) + require.NoError(t, err) + assert.Equal(t, tt.in, in) + + }) } - b, err := json.Marshal(out) - require.NoError(t, err) - assert.Equal(t, `{"rpc":"heartbeat","response":{"from":"node1","term":1,"success":true}}`, string(b)) - - var in election.RPCResponse - err = json.Unmarshal(b, &in) - require.NoError(t, err) - assert.Equal(t, out, in) } func TestHTTPServer(t *testing.T) { @@ -66,9 +147,8 @@ func TestHTTPServer(t *testing.T) { resp = election.RPCResponse{ RPC: election.HeartBeatRPC, Response: election.HeartBeatResp{ - From: "node1", - Term: 10, - Success: true, + From: "node1", + Term: 10, }, } default: @@ -108,7 +188,6 @@ func TestHTTPServer(t *testing.T) { hb := rpcResp.Response.(election.HeartBeatResp) assert.Equal(t, uint64(10), hb.Term) assert.Equal(t, "node1", hb.From) - assert.Equal(t, true, hb.Success) // Send an unknown rpc request to the server req, err = http.NewRequest(http.MethodPost, ts.URL, bytes.NewBuffer([]byte(`{"rpc":"unknown"}`))) diff --git a/election/structs.pb.go b/election/structs.pb.go new file mode 100644 index 00000000..7f653c54 --- /dev/null +++ b/election/structs.pb.go @@ -0,0 +1,226 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// source: structs.proto + +/* +Package election is a generated protocol buffer package. + +It is generated from these files: + structs.proto + +It has these top-level messages: + ResetElectionReq + ResetElectionResp + ResignReq + ResignResp + HeartBeatReq + HeartBeatResp + VoteResp + VoteReq +*/ +package election + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +// Resets the current state of a node to 'candidate' +type ResetElectionReq struct { +} + +func (m *ResetElectionReq) Reset() { *m = ResetElectionReq{} } +func (m *ResetElectionReq) String() string { return proto.CompactTextString(m) } +func (*ResetElectionReq) ProtoMessage() {} +func (*ResetElectionReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +type ResetElectionResp struct { +} + +func (m *ResetElectionResp) Reset() { *m = ResetElectionResp{} } +func (m *ResetElectionResp) String() string { return proto.CompactTextString(m) } +func (*ResetElectionResp) ProtoMessage() {} +func (*ResetElectionResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +// Asks the node to resign as leader +type ResignReq struct { +} + +func (m *ResignReq) Reset() { *m = ResignReq{} } +func (m *ResignReq) String() string { return proto.CompactTextString(m) } +func (*ResignReq) ProtoMessage() {} +func (*ResignReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +type ResignResp struct { + // True if the receiver is leader and stepped down + Success bool `protobuf:"varint,1,opt,name=success" json:"success,omitempty"` +} + +func (m *ResignResp) Reset() { *m = ResignResp{} } +func (m *ResignResp) String() string { return proto.CompactTextString(m) } +func (*ResignResp) ProtoMessage() {} +func (*ResignResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } + +func (m *ResignResp) GetSuccess() bool { + if m != nil { + return m.Success + } + return false +} + +// Sent by the leader of the election to all followers +type HeartBeatReq struct { + // The leader this heart beat is from + From string `protobuf:"bytes,1,opt,name=from" json:"from,omitempty"` + // The current term of the leader + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term,omitempty"` +} + +func (m *HeartBeatReq) Reset() { *m = HeartBeatReq{} } +func (m *HeartBeatReq) String() string { return proto.CompactTextString(m) } +func (*HeartBeatReq) ProtoMessage() {} +func (*HeartBeatReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } + +func (m *HeartBeatReq) GetFrom() string { + if m != nil { + return m.From + } + return "" +} + +func (m *HeartBeatReq) GetTerm() uint64 { + if m != nil { + return m.Term + } + return 0 +} + +// Response to a heart beat request +type HeartBeatResp struct { + // The follower who is responding + From string `protobuf:"bytes,1,opt,name=from" json:"from,omitempty"` + // The term the heart beat is for + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term,omitempty"` +} + +func (m *HeartBeatResp) Reset() { *m = HeartBeatResp{} } +func (m *HeartBeatResp) String() string { return proto.CompactTextString(m) } +func (*HeartBeatResp) ProtoMessage() {} +func (*HeartBeatResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } + +func (m *HeartBeatResp) GetFrom() string { + if m != nil { + return m.From + } + return "" +} + +func (m *HeartBeatResp) GetTerm() uint64 { + if m != nil { + return m.Term + } + return 0 +} + +type VoteResp struct { + // The candidate who responded + Candidate string `protobuf:"bytes,1,opt,name=candidate" json:"candidate,omitempty"` + // The term this vote response is for + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term,omitempty"` + // If the Vote was granted by this node + Granted bool `protobuf:"varint,3,opt,name=granted" json:"granted,omitempty"` +} + +func (m *VoteResp) Reset() { *m = VoteResp{} } +func (m *VoteResp) String() string { return proto.CompactTextString(m) } +func (*VoteResp) ProtoMessage() {} +func (*VoteResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } + +func (m *VoteResp) GetCandidate() string { + if m != nil { + return m.Candidate + } + return "" +} + +func (m *VoteResp) GetTerm() uint64 { + if m != nil { + return m.Term + } + return 0 +} + +func (m *VoteResp) GetGranted() bool { + if m != nil { + return m.Granted + } + return false +} + +// A vote request sent to all peers at +// the start of an election. +type VoteReq struct { + // The candidate who is requesting the targets vote + Candidate string `protobuf:"bytes,1,opt,name=candidate" json:"candidate,omitempty"` + // The term this vote is for. + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term,omitempty"` +} + +func (m *VoteReq) Reset() { *m = VoteReq{} } +func (m *VoteReq) String() string { return proto.CompactTextString(m) } +func (*VoteReq) ProtoMessage() {} +func (*VoteReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } + +func (m *VoteReq) GetCandidate() string { + if m != nil { + return m.Candidate + } + return "" +} + +func (m *VoteReq) GetTerm() uint64 { + if m != nil { + return m.Term + } + return 0 +} + +func init() { + proto.RegisterType((*ResetElectionReq)(nil), "ResetElectionReq") + proto.RegisterType((*ResetElectionResp)(nil), "ResetElectionResp") + proto.RegisterType((*ResignReq)(nil), "ResignReq") + proto.RegisterType((*ResignResp)(nil), "ResignResp") + proto.RegisterType((*HeartBeatReq)(nil), "HeartBeatReq") + proto.RegisterType((*HeartBeatResp)(nil), "HeartBeatResp") + proto.RegisterType((*VoteResp)(nil), "VoteResp") + proto.RegisterType((*VoteReq)(nil), "VoteReq") +} + +func init() { proto.RegisterFile("structs.proto", fileDescriptor0) } + +var fileDescriptor0 = []byte{ + // 216 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x90, 0x31, 0x4b, 0xc0, 0x30, + 0x10, 0x85, 0xa9, 0x16, 0xdb, 0x9e, 0x16, 0x34, 0x2e, 0x19, 0x1c, 0x4a, 0x06, 0xe9, 0xe4, 0x22, + 0xe8, 0xe0, 0x56, 0x10, 0x9c, 0x33, 0x74, 0x70, 0x8b, 0xe9, 0x59, 0x0a, 0xb6, 0x49, 0x73, 0xd7, + 0xff, 0x2f, 0x4d, 0x5b, 0x14, 0x71, 0xd0, 0xed, 0xde, 0xe3, 0xbe, 0x07, 0xef, 0x41, 0x49, 0x1c, + 0x16, 0xcb, 0x74, 0xe7, 0x83, 0x63, 0xa7, 0x04, 0x5c, 0x6a, 0x24, 0xe4, 0xe7, 0x0f, 0xb4, 0x3c, + 0xb8, 0x49, 0xe3, 0xac, 0xae, 0xe1, 0xea, 0x87, 0x47, 0x5e, 0x9d, 0x43, 0xa1, 0x91, 0x86, 0x3e, + 0x7e, 0xdc, 0x02, 0x1c, 0x82, 0xbc, 0x90, 0x90, 0xd1, 0x62, 0x2d, 0x12, 0xc9, 0xa4, 0x4a, 0xea, + 0x5c, 0x1f, 0x52, 0x3d, 0xc0, 0xc5, 0x0b, 0x9a, 0xc0, 0x0d, 0x1a, 0xd6, 0x38, 0x0b, 0x01, 0xe9, + 0x7b, 0x70, 0x63, 0x7c, 0x2b, 0x74, 0xbc, 0x57, 0x8f, 0x31, 0x8c, 0xf2, 0xa4, 0x4a, 0xea, 0x54, + 0xc7, 0x5b, 0x3d, 0x42, 0xf9, 0x8d, 0x23, 0xff, 0x67, 0xb0, 0x85, 0xbc, 0x75, 0x8c, 0x91, 0xb9, + 0x81, 0xc2, 0x9a, 0xa9, 0x1b, 0x3a, 0xc3, 0xb8, 0x83, 0x5f, 0xc6, 0x6f, 0xf4, 0x5a, 0xa4, 0x0f, + 0x66, 0x62, 0xec, 0xe4, 0xe9, 0x56, 0x64, 0x97, 0xea, 0x09, 0xb2, 0x2d, 0x77, 0xfe, 0x7f, 0x6c, + 0x03, 0xaf, 0x39, 0xee, 0x53, 0xbe, 0x9d, 0xc5, 0xd9, 0xef, 0x3f, 0x03, 0x00, 0x00, 0xff, 0xff, + 0xe1, 0x04, 0x54, 0x6d, 0x87, 0x01, 0x00, 0x00, +} diff --git a/election/structs.proto b/election/structs.proto new file mode 100644 index 00000000..c212641b --- /dev/null +++ b/election/structs.proto @@ -0,0 +1,48 @@ +syntax = "proto3"; +option go_package = "election"; + +// Resets the current state of a node to 'candidate' +message ResetElectionReq {} +message ResetElectionResp {} + +// Asks the node to resign as leader +message ResignReq {} +message ResignResp { + // True if the receiver is leader and stepped down + bool success = 1; +} + +// Sent by the leader of the election to all followers +message HeartBeatReq { + // The leader this heart beat is from + string from = 1; + // The current term of the leader + uint64 term = 2; +} + +// Response to a heart beat request +message HeartBeatResp { + // The follower who is responding + string from = 1; + // The term the heart beat is for + uint64 term = 2; +} + +message VoteResp { + // The candidate who responded + string candidate = 1; + // The term this vote response is for + uint64 term = 2; + // If the Vote was granted by this node + bool granted = 3; +} + +// A vote request sent to all peers at +// the start of an election. +message VoteReq { + // The candidate who is requesting the targets vote + string candidate = 1; + // The term this vote is for. + uint64 term = 2; +} + diff --git a/go.mod b/go.mod index 8a0fca6e..4dea9f92 100644 --- a/go.mod +++ b/go.mod @@ -13,6 +13,7 @@ require ( github.com/dgrijalva/jwt-go v3.2.0+incompatible // indirect github.com/gogo/protobuf v1.2.1 // indirect github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef // indirect + github.com/golang/protobuf v1.3.2 github.com/google/btree v1.0.0 // indirect github.com/google/uuid v1.1.2 // indirect github.com/gorilla/mux v1.7.3 // indirect From 46983a697612aa7d66827ff5be5225bdb0ecdc81 Mon Sep 17 00:00:00 2001 From: "Derrick J. Wippler" Date: Wed, 16 Dec 2020 11:30:38 -0600 Subject: [PATCH 05/15] Added memberlist for peer discovery --- cmd/election/main.go | 92 ++++++++++++++ election/cluster_test.go | 23 +++- election/election.go | 14 +-- election/election_test.go | 55 +++++++++ election/example_test.go | 4 +- election/memberlist.go | 246 ++++++++++++++++++++++++++++++++++++++ go.mod | 1 + 7 files changed, 425 insertions(+), 10 deletions(-) create mode 100644 cmd/election/main.go create mode 100644 election/memberlist.go diff --git a/cmd/election/main.go b/cmd/election/main.go new file mode 100644 index 00000000..3fdc9171 --- /dev/null +++ b/cmd/election/main.go @@ -0,0 +1,92 @@ +package main + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "log" + "net/http" + "os" + + "github.com/mailgun/holster/v3/election" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" +) + +func sendRPC(ctx context.Context, peer string, req election.RPCRequest, resp *election.RPCResponse) error { + // Marshall the RPC request to json + b, err := json.Marshal(req) + if err != nil { + return errors.Wrap(err, "while encoding request") + } + + // Create a new http request with context + hr, err := http.NewRequest(http.MethodPost, fmt.Sprintf("http://%s/rpc", peer), bytes.NewBuffer(b)) + if err != nil { + return errors.Wrap(err, "while creating request") + } + hr.WithContext(ctx) + + // Send the request + hp, err := http.DefaultClient.Do(hr) + if err != nil { + return errors.Wrap(err, "while sending http request") + } + + // Decode the response from JSON + dec := json.NewDecoder(hp.Body) + if err := dec.Decode(&resp); err != nil { + return errors.Wrap(err, "while decoding response") + } + return nil +} + +func newHandler(node election.Node) func(w http.ResponseWriter, r *http.Request) { + return func(w http.ResponseWriter, r *http.Request) { + dec := json.NewDecoder(r.Body) + var req election.RPCRequest + if err := dec.Decode(&req); err != nil { + w.WriteHeader(http.StatusBadRequest) + w.Write([]byte(err.Error())) + } + var resp election.RPCResponse + node.ReceiveRPC(req, &resp) + + enc := json.NewEncoder(w) + if err := enc.Encode(resp); err != nil { + w.WriteHeader(http.StatusInternalServerError) + w.Write([]byte(err.Error())) + } + } +} + +func main() { + address := os.Args[0] + if address == "" { + log.Fatal("please provide an address IE: 'localhost:8080'") + } + + logrus.SetLevel(logrus.DebugLevel) + + node1, err := election.SpawnNode(election.Config{ + // A list of known peers at startup + Peers: []string{"localhost:7080", "localhost:7081"}, + // A unique identifier used to identify us in a list of peers + Name: "localhost:7080", + // Called whenever the library detects a change in leadership + Observer: func(leader string) { + log.Printf("Current Leader: %s\n", leader) + }, + // Called when the library wants to contact other peers + SendRPC: sendRPC, + }) + if err != nil { + log.Fatal(err) + } + defer node1.Close() + + mux := http.NewServeMux() + mux.HandleFunc("/rpc", newHandler(node1)) + log.Fatal(http.ListenAndServe(":7080", mux)) +} diff --git a/election/cluster_test.go b/election/cluster_test.go index eb30fdae..345540b4 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -43,7 +43,7 @@ func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { SendRPC: c.sendRPC, } - conf.Self = name + conf.Name = name conf.SendRPC = func(ctx context.Context, peer string, req election.RPCRequest, resp *election.RPCResponse) error { n.lock.RLock() defer n.lock.RUnlock() @@ -111,6 +111,23 @@ func (c *TestCluster) GetLeader() election.Node { return nil } +func (c *TestCluster) peerKey(from, to string) string { + return fmt.Sprintf("%s|%s", from, to) +} + +func (c *TestCluster) ClearErrors() { + c.lock.Lock() + defer c.lock.Unlock() + c.errors = make(map[string]error) +} + +// Add a specific peer to peer error +func (c *TestCluster) AddPeerToPeerError(from string, to string, err error) { + c.lock.Lock() + defer c.lock.Unlock() + c.errors[c.peerKey(from, to)] = err +} + func (c *TestCluster) AddNetworkError(peer string, err error) { c.lock.Lock() defer c.lock.Unlock() @@ -135,6 +152,10 @@ func (c *TestCluster) sendRPC(from string, to string, req election.RPCRequest, r return err } + if err, ok := c.errors[c.peerKey(from, to)]; ok { + return err + } + n, ok := c.Nodes[to] if !ok { return fmt.Errorf("unknown peer '%s'", to) diff --git a/election/election.go b/election/election.go index efd99a2e..d2191c09 100644 --- a/election/election.go +++ b/election/election.go @@ -65,8 +65,8 @@ type Config struct { // The Initial list of peers to be considered in the election, including ourself. Peers []string - // The peer name of our self, as found in the Peers list - Self string + // The name this peer identifies itself as, as found in the Peers list + Name string // Called when the leader changes Observer Observer @@ -84,7 +84,7 @@ type Observer func(string) type Node interface { // Set the list of peers to be considered for the election, this list MUST - // include ourself as defined by `Config.Self`. + // include ourself as defined by `Config.Name`. SetPeers([]string) error // If leader, resigns as leader and starts a new election that we will not @@ -131,11 +131,11 @@ type node struct { // Spawns a new node that will participate in the election. func SpawnNode(conf Config) (Node, error) { - if conf.Self == "" { - return nil, errors.New("refusing to spawn a new node with no Config.Self defined") + if conf.Name == "" { + return nil, errors.New("refusing to spawn a new node with no Config.Name defined") } - setter.SetDefault(&conf.Log, logrus.WithField("name", conf.Self)) + setter.SetDefault(&conf.Log, logrus.WithField("name", conf.Name)) setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*30) setter.SetDefault(&conf.HeartBeatTimeout, time.Second*5) setter.SetDefault(&conf.ElectionTimeout, time.Second*10) @@ -144,7 +144,7 @@ func SpawnNode(conf Config) (Node, error) { c := &node{ shutdownCh: make(chan struct{}), rpcCh: make(chan RPCRequest, 5_000), - self: conf.Self, + self: conf.Name, conf: conf, log: conf.Log, } diff --git a/election/election_test.go b/election/election_test.go index bea32f63..33cdad7b 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -183,3 +183,58 @@ func TestSplitBrain(t *testing.T) { t.Logf("Node: %s Leader: %t\n", k, v.Node.IsLeader()) } } + +func TestOmissionFaults(t *testing.T) { + c1 := NewTestCluster() + createCluster(t, c1) + defer c1.Close() + + // Create an unstable cluster with n3 and n4 only able to contact n1 and n2 respectively. + // The end result should be an omission fault of less than quorum. + // + // Diagram: lines indicate connectivity between nodes + // (n0)-----(n1)----(n4) + // \ / + // \ / + // \ / + // (n2)----(n3) + // + + // n3 and n4 can't talk + c1.AddPeerToPeerError("n3", "n4", ErrConnRefused) + c1.AddPeerToPeerError("n4", "n3", ErrConnRefused) + + // Leader can't talk to n4 + c1.AddPeerToPeerError("n0", "n4", ErrConnRefused) + c1.AddPeerToPeerError("n4", "n0", ErrConnRefused) + + // Leader can't talk to n3 + c1.AddPeerToPeerError("n0", "n3", ErrConnRefused) + c1.AddPeerToPeerError("n3", "n0", ErrConnRefused) + + // n2 and n4 can't talk + c1.AddPeerToPeerError("n2", "n4", ErrConnRefused) + c1.AddPeerToPeerError("n4", "n2", ErrConnRefused) + + // n1 and n3 can't talk + c1.AddPeerToPeerError("n1", "n3", ErrConnRefused) + c1.AddPeerToPeerError("n3", "n1", ErrConnRefused) + + // Cluster should retain n0 as leader in the face on unstable cluster + for i := 0; i < 12; i++ { + leader := c1.GetLeader() + require.NotNil(t, leader) + require.Equal(t, leader.Leader(), "n0") + time.Sleep(time.Millisecond * 400) + } + + // Should retain leader once communication is restored + c1.ClearErrors() + + for i := 0; i < 12; i++ { + leader := c1.GetLeader() + require.NotNil(t, leader) + require.Equal(t, leader.Leader(), "n0") + time.Sleep(time.Millisecond * 400) + } +} diff --git a/election/example_test.go b/election/example_test.go index 07345296..7e862e37 100644 --- a/election/example_test.go +++ b/election/example_test.go @@ -71,7 +71,7 @@ func SimpleExample(t *testing.T) { // A list of known peers at startup Peers: []string{"localhost:7080", "localhost:7081"}, // A unique identifier used to identify us in a list of peers - Self: "localhost:7080", + Name: "localhost:7080", // Called whenever the library detects a change in leadership Observer: func(leader string) { log.Printf("Current Leader: %s\n", leader) @@ -86,7 +86,7 @@ func SimpleExample(t *testing.T) { node2, err := election.SpawnNode(election.Config{ Peers: []string{"localhost:7080", "localhost:7081"}, - Self: "localhost:7081", + Name: "localhost:7081", SendRPC: sendRPC, }) if err != nil { diff --git a/election/memberlist.go b/election/memberlist.go new file mode 100644 index 00000000..faceec1d --- /dev/null +++ b/election/memberlist.go @@ -0,0 +1,246 @@ +package election + +import ( + "bufio" + "context" + "encoding/json" + "io" + "net" + "runtime" + "strconv" + + ml "github.com/hashicorp/memberlist" + "github.com/mailgun/holster/v3/clock" + "github.com/mailgun/holster/v3/errors" + "github.com/mailgun/holster/v3/retry" + "github.com/mailgun/holster/v3/setter" + "github.com/sirupsen/logrus" +) + +type PeerInfo struct { + // The http address:port of the peer + HTTPAddress string `json:"http-address"` + // The grpc address:port of the peer + GRPCAddress string `json:"grpc-address"` + // Is true if PeerInfo matches the Name as given in the memberlist config + IsOwner bool `json:"is-owner,omitempty"` +} + +type UpdateFunc func([]PeerInfo) + +type MemberListPool struct { + log logrus.FieldLogger + memberList *ml.Memberlist + conf MemberListPoolConfig + events *memberListEventHandler +} + +type MemberListPoolConfig struct { + // This is the address:port the member list protocol listen for other members on + MemberListAddress string + + // The information about this peer which should be shared with all other members + PeerInfo PeerInfo + + // A list of nodes this member list instance can contact to find other members. + KnownNodes []string + + // A callback function which is called when the member list changes + OnUpdate UpdateFunc + + // An interface through which logging will occur (Usually *logrus.Entry) + Logger logrus.FieldLogger +} + +func NewMemberListPool(ctx context.Context, conf MemberListPoolConfig) (*MemberListPool, error) { + setter.SetDefault(conf.Logger, logrus.WithField("category", "gubernator")) + m := &MemberListPool{ + log: conf.Logger, + conf: conf, + } + + host, port, err := splitAddress(conf.MemberListAddress) + if err != nil { + return nil, errors.Wrap(err, "MemberListAddress=`%s` is invalid;") + } + + // Member list requires the address to be an ip address + if ip := net.ParseIP(host); ip == nil { + addrs, err := net.LookupHost(host) + if err != nil { + return nil, errors.Wrapf(err, "while preforming host lookup for '%s'", host) + } + if len(addrs) == 0 { + return nil, errors.Wrapf(err, "net.LookupHost() returned no addresses for '%s'", host) + } + host = addrs[0] + } + + // Configure member list event handler + m.events = &memberListEventHandler{ + conf: conf, + log: m.log, + } + m.events.peers = make(map[string]PeerInfo) + + // Configure member list + config := ml.DefaultWANConfig() + config.Events = m.events + config.AdvertiseAddr = host + config.AdvertisePort = port + config.Name = conf.PeerInfo.HTTPAddress + config.LogOutput = newLogWriter(m.log) + + // Create and set member list + memberList, err := ml.Create(config) + if err != nil { + return nil, err + } + m.memberList = memberList + conf.PeerInfo.IsOwner = false + + // Join member list pool + err = m.joinPool(ctx) + if err != nil { + return nil, errors.Wrap(err, "while attempting to join the member-list pool") + } + + return m, nil +} + +func (m *MemberListPool) joinPool(ctx context.Context) error { + // Get local node and set metadata + node := m.memberList.LocalNode() + b, err := json.Marshal(&m.conf.PeerInfo) + if err != nil { + return errors.Wrap(err, "error marshalling metadata") + } + node.Meta = b + + err = retry.Until(ctx, retry.Interval(clock.Millisecond*300), func(ctx context.Context, i int) error { + // Join member list + _, err = m.memberList.Join(m.conf.KnownNodes) + if err != nil { + return errors.Wrap(err, "while joining member-list") + } + return nil + }) + if err != nil { + return errors.Wrap(err, "timed out attempting to join member list") + } + + // Add the local node to the event handler's peer list + m.events.addPeer(node) + return nil +} + +func (m *MemberListPool) Close() { + err := m.memberList.Leave(clock.Second) + if err != nil { + m.log.Warn(errors.Wrap(err, "while leaving member-list")) + } +} + +type memberListEventHandler struct { + peers map[string]PeerInfo + log logrus.FieldLogger + conf MemberListPoolConfig +} + +func (e *memberListEventHandler) addPeer(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + var metadata PeerInfo + if err := json.Unmarshal(node.Meta, &metadata); err != nil { + e.log.WithError(err).Warnf("while adding to peers") + return + } + e.peers[ip] = metadata + e.callOnUpdate() +} + +func (e *memberListEventHandler) NotifyJoin(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + var metadata PeerInfo + if err := json.Unmarshal(node.Meta, &metadata); err != nil { + e.log.WithError(err).Warn("while deserialize member-list metadata") + return + } + e.peers[ip] = metadata + e.callOnUpdate() +} + +func (e *memberListEventHandler) NotifyLeave(node *ml.Node) { + ip := getIP(node.Address()) + + // Remove PeerInfo + delete(e.peers, ip) + + e.callOnUpdate() +} + +func (e *memberListEventHandler) NotifyUpdate(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + var metadata PeerInfo + if err := json.Unmarshal(node.Meta, &metadata); err != nil { + e.log.WithError(err).Warn("while updating member-list") + return + } + e.peers[ip] = metadata + e.callOnUpdate() +} + +func (e *memberListEventHandler) callOnUpdate() { + var peers []PeerInfo + + for _, p := range e.peers { + if p.HTTPAddress == e.conf.PeerInfo.HTTPAddress { + p.IsOwner = true + } + peers = append(peers, p) + } + e.conf.OnUpdate(peers) +} + +func getIP(address string) string { + addr, _, _ := net.SplitHostPort(address) + return addr +} + +func newLogWriter(log logrus.FieldLogger) *io.PipeWriter { + reader, writer := io.Pipe() + + go func() { + scanner := bufio.NewScanner(reader) + for scanner.Scan() { + log.Info(scanner.Text()) + } + if err := scanner.Err(); err != nil { + log.Errorf("Error while reading from Writer: %s", err) + } + reader.Close() + }() + runtime.SetFinalizer(writer, func(w *io.PipeWriter) { + writer.Close() + }) + + return writer +} + +func splitAddress(addr string) (string, int, error) { + host, port, err := net.SplitHostPort(addr) + if err != nil { + return host, 0, errors.New(" expected format is `address:port`") + } + + intPort, err := strconv.Atoi(port) + if err != nil { + return host, intPort, errors.Wrap(err, "port must be a number") + } + return host, intPort, nil +} diff --git a/go.mod b/go.mod index 4dea9f92..d60b17d4 100644 --- a/go.mod +++ b/go.mod @@ -23,6 +23,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.10.0 // indirect github.com/hashicorp/consul/api v1.7.0 github.com/hashicorp/golang-lru v0.5.1 // indirect + github.com/hashicorp/memberlist v0.2.2 github.com/jonboulle/clockwork v0.1.0 // indirect github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.1.0 // indirect From 16dec3e14d9e102f96818046ec09b8fa1dc74cf5 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Wed, 13 Jan 2021 16:47:46 -0600 Subject: [PATCH 06/15] Added discovery package --- cancel/context.go | 38 +++++ cmd/election/main.go | 69 +++++++-- consul/log.go | 183 ++++++++++++++++++++++ discovery/consul.go | 241 +++++++++++++++++++++++++++++ discovery/consul_test.go | 90 +++++++++++ discovery/docker-compose.yaml | 14 ++ discovery/memberlist.go | 276 ++++++++++++++++++++++++++++++++++ discovery/memberlist_test.go | 40 +++++ election/cluster_test.go | 12 +- election/election.go | 79 +++++----- election/election_test.go | 27 ++++ election/example_test.go | 10 +- election/memberlist.go | 246 ------------------------------ go.mod | 2 + 14 files changed, 1017 insertions(+), 310 deletions(-) create mode 100644 cancel/context.go create mode 100644 consul/log.go create mode 100644 discovery/consul.go create mode 100644 discovery/consul_test.go create mode 100644 discovery/docker-compose.yaml create mode 100644 discovery/memberlist.go create mode 100644 discovery/memberlist_test.go delete mode 100644 election/memberlist.go diff --git a/cancel/context.go b/cancel/context.go new file mode 100644 index 00000000..e553d794 --- /dev/null +++ b/cancel/context.go @@ -0,0 +1,38 @@ +package cancel + +import ( + "context" + "time" +) + +type Context interface { + context.Context + Cancel() +} + +type cancelCtx struct { + ctx context.Context + cancel context.CancelFunc +} + +// Creates a context that wraps the given context and returns an obj that can be cancelled. +// This allows an object which desires to cancel a long running operation to store a single +// cancel.Context in it's struct variables instead of having to store both the context.Context +// and context.CancelFunc. +func New(ctx context.Context) Context { + if ctx == nil { + ctx = context.Background() + } + + ctx, cancel := context.WithCancel(ctx) + return &cancelCtx{ + cancel: cancel, + ctx: ctx, + } +} + +func (c *cancelCtx) Cancel() { c.cancel() } +func (c *cancelCtx) Deadline() (deadline time.Time, ok bool) { return c.ctx.Deadline() } +func (c *cancelCtx) Done() <-chan struct{} { return c.ctx.Done() } +func (c *cancelCtx) Err() error { return c.ctx.Err() } +func (c *cancelCtx) Value(key interface{}) interface{} { return c.ctx.Value(key) } diff --git a/cmd/election/main.go b/cmd/election/main.go index 3fdc9171..3d5d48f7 100644 --- a/cmd/election/main.go +++ b/cmd/election/main.go @@ -5,10 +5,14 @@ import ( "context" "encoding/json" "fmt" - "log" "net/http" "os" + "os/signal" + "syscall" + "time" + "github.com/google/uuid" + "github.com/mailgun/holster/v3/discovery" "github.com/mailgun/holster/v3/election" "github.com/pkg/errors" "github.com/sirupsen/logrus" @@ -62,31 +66,66 @@ func newHandler(node election.Node) func(w http.ResponseWriter, r *http.Request) } func main() { - address := os.Args[0] - if address == "" { - log.Fatal("please provide an address IE: 'localhost:8080'") + if len(os.Args) != 4 { + logrus.Fatal("usage: ") } - logrus.SetLevel(logrus.DebugLevel) + electionAddr, memberListAddr, knownAddr := os.Args[1], os.Args[2], os.Args[3] + //logrus.SetLevel(logrus.DebugLevel) - node1, err := election.SpawnNode(election.Config{ - // A list of known peers at startup - Peers: []string{"localhost:7080", "localhost:7081"}, + node, err := election.SpawnNode(election.Config{ // A unique identifier used to identify us in a list of peers - Name: "localhost:7080", + UniqueID: electionAddr, // Called whenever the library detects a change in leadership - Observer: func(leader string) { - log.Printf("Current Leader: %s\n", leader) + OnUpdate: func(leader string) { + logrus.Printf("Current Leader: %s\n", leader) }, // Called when the library wants to contact other peers SendRPC: sendRPC, }) if err != nil { - log.Fatal(err) + logrus.Fatal(err) + } + + // Create a member list catalog + ml, err := discovery.NewMemberList(context.Background(), discovery.MemberListConfig{ + BindAddress: memberListAddr, + Peer: discovery.Peer{ + ID: uuid.New().String(), + Metadata: []byte(electionAddr), + }, + KnownPeers: []string{knownAddr}, + OnUpdate: func(peers []discovery.Peer) { + var result []string + for _, p := range peers { + result = append(result, string(p.Metadata)) + } + logrus.Infof("Update Peers: %s", result) + if err := node.SetPeers(result); err != nil { + logrus.Fatal(err) + } + }, + }) + if err != nil { + logrus.Fatal(err) } - defer node1.Close() mux := http.NewServeMux() - mux.HandleFunc("/rpc", newHandler(node1)) - log.Fatal(http.ListenAndServe(":7080", mux)) + mux.HandleFunc("/rpc", newHandler(node)) + go func() { + logrus.Fatal(http.ListenAndServe(electionAddr, mux)) + }() + + // Wait here for signals to clean up our mess + c := make(chan os.Signal, 1) + signal.Notify(c, os.Interrupt, syscall.SIGTERM) + for range c { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + if err := ml.Close(ctx); err != nil { + logrus.WithError(err).Error("during member list catalog close") + } + cancel() + node.Close() + os.Exit(0) + } } diff --git a/consul/log.go b/consul/log.go new file mode 100644 index 00000000..c61eaf4c --- /dev/null +++ b/consul/log.go @@ -0,0 +1,183 @@ +package consul + +import ( + "bytes" + "io" + "log" + "os" + + "github.com/hashicorp/go-hclog" + "github.com/sirupsen/logrus" +) + +// HCLogAdapter implements the hclog interface, and wraps it +// around a Logrus entry +type HCLogAdapter struct { + log logrus.FieldLogger + name string + args []interface{} // key/value pairs if this logger was created via With() +} + +func NewHCLogAdapter(log logrus.FieldLogger, name string) *HCLogAdapter { + return &HCLogAdapter{ + log: log, + name: name, + } +} + +// HCLog has one more level than we do. As such, we will never +// set trace level. +func (*HCLogAdapter) Trace(_ string, _ ...interface{}) { +} + +func (a *HCLogAdapter) Debug(msg string, args ...interface{}) { + a.CreateEntry(args).Debug(msg) +} + +func (a *HCLogAdapter) Info(msg string, args ...interface{}) { + a.CreateEntry(args).Info(msg) +} + +func (a *HCLogAdapter) Warn(msg string, args ...interface{}) { + a.CreateEntry(args).Warn(msg) +} + +func (a *HCLogAdapter) Error(msg string, args ...interface{}) { + a.CreateEntry(args).Error(msg) +} + +func (a *HCLogAdapter) Log(level hclog.Level, msg string, args ...interface{}) { + switch level { + case hclog.Trace: + a.Trace(msg, args...) + case hclog.Debug: + a.Debug(msg, args...) + case hclog.Info: + a.Info(msg, args...) + case hclog.Warn: + a.Warn(msg, args...) + case hclog.Error: + a.Error(msg, args...) + } +} + +func (a *HCLogAdapter) IsTrace() bool { + return false +} + +func (a *HCLogAdapter) IsDebug() bool { + return a.shouldEmit(logrus.DebugLevel) +} + +func (a *HCLogAdapter) IsInfo() bool { + return a.shouldEmit(logrus.InfoLevel) +} + +func (a *HCLogAdapter) IsWarn() bool { + return a.shouldEmit(logrus.WarnLevel) +} + +func (a *HCLogAdapter) IsError() bool { + return a.shouldEmit(logrus.ErrorLevel) +} + +func (a *HCLogAdapter) SetLevel(hclog.Level) { + // interface definition says it is ok for this to be a noop if + // implementations don't need/want to support dynamic level changing, which + // we don't currently. +} + +func (a *HCLogAdapter) With(args ...interface{}) hclog.Logger { + e := a.CreateEntry(args) + return &HCLogAdapter{ + log: e, + args: concatFields(a.args, args), + } +} + +// concatFields combines two sets of key/value pairs. +// It allocates a new slice to avoid using append() and +// accidentally overriding the original slice a, e.g. +// when logger.With() is called multiple times to create +// sub-scoped loggers. +func concatFields(a, b []interface{}) []interface{} { + c := make([]interface{}, len(a)+len(b)) + copy(c, a) + copy(c[len(a):], b) + return c +} + +// ImpliedArgs returns With key/value pairs +func (a *HCLogAdapter) ImpliedArgs() []interface{} { + return a.args +} + +func (a *HCLogAdapter) Name() string { + return a.name +} + +func (a *HCLogAdapter) Named(name string) hclog.Logger { + var newName bytes.Buffer + if a.name != "" { + newName.WriteString(a.name) + newName.WriteString(".") + } + newName.WriteString(name) + + return a.ResetNamed(newName.String()) +} + +func (a *HCLogAdapter) ResetNamed(name string) hclog.Logger { + fields := []interface{}{"subsystem_name", name} + e := a.CreateEntry(fields) + return &HCLogAdapter{log: e, name: name} +} + +// StandardLogger is meant to return a stdlib Logger type which wraps around +// hclog. It does this by providing an io.Writer and instantiating a new +// Logger. It then tries to interpret the log level by parsing the message. +// +// Since we are not using `hclog` in a generic way, and I cannot find any +// calls to this method from go-plugin, we will poorly support this method. +// Rather than pull in all of hclog writer parsing logic, pass it a Logrus +// writer, and hardcode the level to INFO. +// +// Apologies to those who find themselves here. +func (a *HCLogAdapter) StandardLogger(opts *hclog.StandardLoggerOptions) *log.Logger { + entry := a.log.WithFields(logrus.Fields{}) + return log.New(entry.WriterLevel(logrus.InfoLevel), "", 0) +} + +func (a *HCLogAdapter) StandardWriter(opts *hclog.StandardLoggerOptions) io.Writer { + var w io.Writer + logger, ok := a.log.(*logrus.Logger) + if ok { + w = logger.Out + } + if w == nil { + w = os.Stderr + } + return w +} + +func (a *HCLogAdapter) shouldEmit(level logrus.Level) bool { + return a.log.WithFields(logrus.Fields{}).Level >= level +} + +func (a *HCLogAdapter) CreateEntry(args []interface{}) *logrus.Entry { + if len(args)%2 != 0 { + args = append(args, "") + } + + fields := make(logrus.Fields) + for i := 0; i < len(args); i += 2 { + k, ok := args[i].(string) + if !ok { + continue + } + v := args[i+1] + fields[k] = v + } + + return a.log.WithFields(fields) +} diff --git a/discovery/consul.go b/discovery/consul.go new file mode 100644 index 00000000..ec79c544 --- /dev/null +++ b/discovery/consul.go @@ -0,0 +1,241 @@ +package discovery + +import ( + "context" + "fmt" + "reflect" + "sort" + "time" + + "github.com/hashicorp/consul/api" + "github.com/hashicorp/consul/api/watch" + "github.com/mailgun/holster/v3/cancel" + "github.com/mailgun/holster/v3/consul" + "github.com/mailgun/holster/v3/errors" + "github.com/mailgun/holster/v3/setter" + "github.com/mailgun/holster/v3/syncutil" + "github.com/sirupsen/logrus" +) + +type ConsulConfig struct { + // This is the consul client config; typically created by calling api.DefaultConfig() + ClientConfig *api.Config + + // The name of the catalog we should register under; should be common to all peers in the catalog + CatalogName string + + // Information about this peer which should be shared with all other peers in the catalog + Peer Peer + + // This is an address the will be registered with consul so it can preform liveliness checks + LivelinessAddress string + + // A callback function which is called when the member list changes + OnUpdate OnUpdateFunc + + // An interface through which logging will occur; usually *logrus.Entry + Logger logrus.FieldLogger +} + +type consulStore struct { + wg syncutil.WaitGroup + log logrus.FieldLogger + client *api.Client + plan *watch.Plan + conf *ConsulConfig + ctx cancel.Context +} + +func NewConsul(conf *ConsulConfig) (Catalog, error) { + setter.SetDefault(&conf.Logger, logrus.WithField("category", "consul-catalog")) + setter.SetDefault(&conf.ClientConfig, api.DefaultConfig()) + var err error + + if conf.Peer.ID == "" { + return nil, errors.New("Peer.ID cannot be empty") + } + + if conf.CatalogName == "" { + return nil, errors.New("CatalogName cannot be empty") + } + + cs := consulStore{ + ctx: cancel.New(context.Background()), + log: conf.Logger, + conf: conf, + } + + cs.client, err = api.NewClient(cs.conf.ClientConfig) + if err != nil { + return nil, errors.Wrap(err, "while creating a new client") + } + + // Register ourselves in consul as a member of the cluster + err = cs.client.Agent().ServiceRegisterOpts(&api.AgentServiceRegistration{ + Name: conf.CatalogName, + ID: conf.Peer.ID, + Tags: []string{"scout-bloom"}, + Address: conf.LivelinessAddress, + Check: &api.AgentServiceCheck{ + DeregisterCriticalServiceAfter: "10m", + TTL: "10s", + }, + Meta: map[string]string{ + "peer": string(conf.Peer.Metadata), + }, + }, api.ServiceRegisterOpts{ReplaceExistingChecks: true}) + if err != nil { + return nil, errors.Wrapf(err, "while registering the peer '%s' to the service catalog '%s'", + conf.Peer.ID, cs.conf.CatalogName) + } + + // Update the service check TTL + err = cs.client.Agent().UpdateTTL(fmt.Sprintf("service:%s", conf.Peer.ID), "", api.HealthPassing) + if err != nil { + return nil, errors.Wrap(err, "while updating service TTL after registration") + } + + cs.log.Debugf("Registered '%s' with consul catalog '%s'", conf.Peer.ID, conf.CatalogName) + + // Periodically update the TTL check on the registered service + ticker := time.NewTicker(time.Second * 4) + cs.wg.Until(func(done chan struct{}) bool { + select { + case <-ticker.C: + err := cs.client.Agent().UpdateTTL(fmt.Sprintf("service:%s", conf.Peer.ID), "", api.HealthPassing) + if err != nil { + cs.log.WithError(err).Warn("while updating consul TTL") + } + case <-done: + ticker.Stop() + return false + } + return true + }) + + // Watch for changes to the service list and partition config changes + if err := cs.watch(); err != nil { + return nil, err + } + + return &cs, nil +} + +func (cs *consulStore) watch() error { + changeCh := make(chan []*api.ServiceEntry, 100) + var previousPeers map[string]Peer + var err error + + cs.plan, err = watch.Parse(map[string]interface{}{ + "type": "service", + "service": cs.conf.CatalogName, + }) + if err != nil { + return fmt.Errorf("while creating watch plan: %s", err) + } + + cs.plan.HybridHandler = func(blockParamVal watch.BlockingParamVal, raw interface{}) { + if raw == nil { + cs.log.Info("Raw == nil") + } + if v, ok := raw.([]*api.ServiceEntry); ok && v != nil { + changeCh <- v + } + } + + allChecksPassing := func(checks api.HealthChecks) bool { + for _, c := range checks { + if c.Status != "passing" { + return false + } + } + return true + } + + cs.wg.Go(func() { + if err := cs.plan.RunWithClientAndHclog(cs.client, consul.NewHCLogAdapter(cs.log, "consul-store")); err != nil { + cs.log.WithError(err).Error("Service watch failed") + } + }) + + cs.wg.Until(func(done chan struct{}) bool { + select { + case <-done: + return false + case serviceEntries := <-changeCh: + if cs.conf.OnUpdate == nil { + return true + } + peers := make(map[string]Peer) + for _, se := range serviceEntries { + if !allChecksPassing(se.Checks) { + break + } + meta, ok := se.Service.Meta["peer"] + if !ok { + cs.log.Errorf("service entry missing 'peer' metadata '%s'", se.Service.ID) + } + p := Peer{ID: se.Service.ID, Metadata: []byte(meta)} + if meta == string(cs.conf.Peer.Metadata) { + p.IsSelf = true + } + peers[p.ID] = p + } + + if !reflect.DeepEqual(previousPeers, peers) { + var result []Peer + for _, v := range peers { + result = append(result, v) + } + // Sort the results to make it easy to compare peer lists + sort.Slice(result, func(i, j int) bool { + return result[i].ID < result[j].ID + }) + cs.conf.OnUpdate(result) + previousPeers = peers + } + } + return true + }) + return nil +} + +func (cs *consulStore) GetPeers(ctx context.Context) ([]Peer, error) { + opts := &api.QueryOptions{LocalOnly: true} + services, _, err := cs.client.Health().Service(cs.conf.CatalogName, "", true, opts.WithContext(ctx)) + if err != nil { + return nil, errors.Wrap(err, "while fetching healthy catalog listing") + } + var peers []Peer + for _, i := range services { + v, ok := i.Service.Meta["peer"] + if !ok { + return nil, fmt.Errorf("service entry missing 'peer' metadata '%s'", i.Service.ID) + } + var p Peer + p.Metadata = []byte(v) + p.ID = i.Service.ID + if v == string(cs.conf.Peer.Metadata) { + p.IsSelf = true + } + peers = append(peers, p) + } + return peers, nil +} + +func (cs *consulStore) Close(ctx context.Context) error { + errCh := make(chan error) + go func() { + cs.plan.Stop() + cs.wg.Stop() + errCh <- cs.client.Agent().ServiceDeregister(cs.conf.Peer.ID) + }() + + select { + case <-ctx.Done(): + cs.ctx.Cancel() + return ctx.Err() + case err := <-errCh: + return err + } +} diff --git a/discovery/consul_test.go b/discovery/consul_test.go new file mode 100644 index 00000000..e3291dec --- /dev/null +++ b/discovery/consul_test.go @@ -0,0 +1,90 @@ +package discovery_test + +import ( + "context" + "fmt" + "testing" + + "github.com/hashicorp/consul/api" + "github.com/mailgun/holster/v3/discovery" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func printCatalog(t *testing.T, catalog string, client *api.Client) { + t.Helper() + fmt.Printf("============\n") + l, _, err := client.Health().Service(catalog, "", false, nil) + require.NoError(t, err) + for _, i := range l { + t.Logf("Service: %s", i.Service.ID) + } + fmt.Printf("======\n") +} + +func TestConsulSinglePeer(t *testing.T) { + const catalog = "TestConsulSinglePeer" + p := discovery.Peer{ID: "id-1", Metadata: []byte("address-0"), IsSelf: true} + + //client, err := api.NewClient(api.DefaultConfig()) + //require.NoError(t, err) + + //printCatalog(t, catalog, client) + + onUpdateCh := make(chan []discovery.Peer, 1) + cs, err := discovery.NewConsul(&discovery.ConsulConfig{ + CatalogName: catalog, + Peer: p, + OnUpdate: func(peers []discovery.Peer) { + onUpdateCh <- peers + }, + }) + //printCatalog(t, catalog, client) + + e := <-onUpdateCh + assert.Equal(t, p, e[0]) + + err = cs.Close(context.Background()) + require.NoError(t, err) + + //printCatalog(t, catalog, client) +} + +func TestConsulMultiplePeers(t *testing.T) { + const catalog = "TestConsulMultiplePeers" + p0 := discovery.Peer{ID: "id-0", Metadata: []byte("address-0"), IsSelf: true} + p1 := discovery.Peer{ID: "id-1", Metadata: []byte("address-1")} + + //client, err := api.NewClient(api.DefaultConfig()) + //require.NoError(t, err) + + //printCatalog(t, catalog, client) + + onUpdateCh := make(chan []discovery.Peer, 2) + cs0, err := discovery.NewConsul(&discovery.ConsulConfig{ + CatalogName: catalog, + Peer: p0, + OnUpdate: func(peers []discovery.Peer) { + onUpdateCh <- peers + }, + }) + require.NoError(t, err) + defer cs0.Close(context.Background()) + + e := <-onUpdateCh + assert.Equal(t, e[0], p0) + + //printCatalog(t, catalog, client) + + cs1, err := discovery.NewConsul(&discovery.ConsulConfig{ + CatalogName: catalog, + Peer: p1, + }) + require.NoError(t, err) + defer cs1.Close(context.Background()) + + e = <-onUpdateCh + assert.Equal(t, []discovery.Peer{p0, p1}, e) + + //printCatalog(t, catalog, client) +} diff --git a/discovery/docker-compose.yaml b/discovery/docker-compose.yaml new file mode 100644 index 00000000..7cfb27e2 --- /dev/null +++ b/discovery/docker-compose.yaml @@ -0,0 +1,14 @@ +version: '3' +services: + consul-agent: + image: consul:latest + command: "agent -retry-join consul-server-bootstrap -client 0.0.0.0" + + consul-server-bootstrap: + image: consul:latest + ports: + - "8400:8400" + - "8500:8500" + - "8600:8600" + - "8600:8600/udp" + command: "agent -server -bootstrap-expect 1 -ui -client 0.0.0.0" diff --git a/discovery/memberlist.go b/discovery/memberlist.go new file mode 100644 index 00000000..8dee0d1c --- /dev/null +++ b/discovery/memberlist.go @@ -0,0 +1,276 @@ +package discovery + +import ( + "bufio" + "context" + "io" + "net" + "runtime" + "sort" + "strconv" + "sync" + + ml "github.com/hashicorp/memberlist" + "github.com/mailgun/holster/v3/clock" + "github.com/mailgun/holster/v3/errors" + "github.com/mailgun/holster/v3/retry" + "github.com/mailgun/holster/v3/setter" + "github.com/sirupsen/logrus" +) + +type Peer struct { + // An ID the uniquely identifies this peer in the catalog + ID string + // The metadata associated with this peer + Metadata []byte + // Is true if this Peer refers to our instance in the catalog + IsSelf bool +} + +type OnUpdateFunc func([]Peer) + +type Catalog interface { + // Returns the peers currently registered in the catalog + GetPeers(context.Context) ([]Peer, error) + // Removes our peer from the catalog and closes all connections + Close(context.Context) error + // TODO: Updates the Peer metadata shared with peers in the catalog + //UpdatePeer(context.Context, Peer) error +} + +type memberList struct { + log logrus.FieldLogger + memberList *ml.Memberlist + conf MemberListConfig + events *eventDelegate +} + +type MemberListConfig struct { + // This is the address:port the member list protocol listen for other peers on. + BindAddress string + + // This is the address:port the member list protocol will advertise to other peers. (Defaults to BindAddress) + AdvertiseAddress string + + // Metadata about this peer which should be shared with all other peers in the same catalog. + Peer Peer + + // A list of peers this member list instance can contact to find other peers. + KnownPeers []string + + // A callback function which is called when the member list changes. + OnUpdate OnUpdateFunc + + // An interface through which logging will occur; usually *logrus.Entry + Logger logrus.FieldLogger +} + +func NewMemberList(ctx context.Context, conf MemberListConfig) (Catalog, error) { + setter.SetDefault(&conf.Logger, logrus.WithField("category", "member-list")) + setter.SetDefault(&conf.AdvertiseAddress, conf.BindAddress) + if conf.Peer.ID == "" { + return nil, errors.New("Peer.ID cannot be empty") + } + if conf.BindAddress == "" { + return nil, errors.New("BindAddress cannot be empty") + } + + conf.Peer.IsSelf = false + + m := &memberList{ + log: conf.Logger, + conf: conf, + events: &eventDelegate{ + peers: make(map[string]Peer, 1), + conf: conf, + log: conf.Logger, + }, + } + + // Create the member list config + config, err := m.newMLConfig(conf) + if err != nil { + return nil, err + } + + // Create a new member list instance + m.memberList, err = ml.Create(config) + if err != nil { + return nil, err + } + + // Attempt to join the member list using a list of known nodes + err = retry.Until(ctx, retry.Interval(clock.Millisecond*300), func(ctx context.Context, i int) error { + _, err = m.memberList.Join(m.conf.KnownPeers) + if err != nil { + return errors.Wrap(err, "while joining member-list") + } + return nil + }) + return m, errors.Wrap(err, "timed out attempting to join member list") +} + +func (m *memberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { + config := ml.DefaultWANConfig() + config.Name = conf.Peer.ID + config.LogOutput = newLogWriter(conf.Logger) + + var err error + config.BindAddr, config.BindPort, err = splitAddress(conf.BindAddress) + if err != nil { + return nil, errors.Wrap(err, "BindAddress=`%s` is invalid;") + } + + config.AdvertiseAddr, config.AdvertisePort, err = splitAddress(conf.AdvertiseAddress) + if err != nil { + return nil, errors.Wrap(err, "LivelinessAddress=`%s` is invalid;") + } + + config.Delegate = &delegate{meta: conf.Peer.Metadata} + config.Events = m.events + return config, nil +} + +func (m *memberList) Close(ctx context.Context) error { + errCh := make(chan error) + go func() { + if err := m.memberList.Leave(clock.Second * 30); err != nil { + errCh <- err + return + } + errCh <- m.memberList.Shutdown() + }() + + select { + case <-ctx.Done(): + return ctx.Err() + case err := <-errCh: + return err + } +} + +func (m *memberList) GetPeers(_ context.Context) ([]Peer, error) { + return m.events.GetPeers() +} + +type eventDelegate struct { + peers map[string]Peer + log logrus.FieldLogger + conf MemberListConfig + mutex sync.Mutex +} + +func (e *eventDelegate) NotifyJoin(node *ml.Node) { + defer e.mutex.Unlock() + e.mutex.Lock() + e.peers[node.Name] = Peer{ID: node.Name, Metadata: node.Meta} + e.callOnUpdate() +} + +func (e *eventDelegate) NotifyLeave(node *ml.Node) { + defer e.mutex.Unlock() + e.mutex.Lock() + delete(e.peers, node.Name) + e.callOnUpdate() +} + +func (e *eventDelegate) NotifyUpdate(node *ml.Node) { + defer e.mutex.Unlock() + e.mutex.Lock() + e.peers[node.Name] = Peer{ID: node.Name, Metadata: node.Meta} + e.callOnUpdate() +} +func (e *eventDelegate) GetPeers() ([]Peer, error) { + defer e.mutex.Unlock() + e.mutex.Lock() + return e.getPeers(), nil +} + +func (e *eventDelegate) getPeers() []Peer { + var peers []Peer + for _, p := range e.peers { + if p.ID == e.conf.Peer.ID { + p.IsSelf = true + } + peers = append(peers, p) + } + return peers +} + +func (e *eventDelegate) callOnUpdate() { + if e.conf.OnUpdate == nil { + return + } + + // Sort the results to make it easy to compare peer lists + peers := e.getPeers() + sort.Slice(peers, func(i, j int) bool { + return peers[i].ID < peers[j].ID + }) + + e.conf.OnUpdate(peers) +} + +type delegate struct { + meta []byte +} + +func (m *delegate) NodeMeta(int) []byte { + return m.meta +} +func (m *delegate) NotifyMsg([]byte) {} +func (m *delegate) GetBroadcasts(int, int) [][]byte { return nil } +func (m *delegate) LocalState(bool) []byte { return nil } +func (m *delegate) MergeRemoteState([]byte, bool) {} + +func newLogWriter(log logrus.FieldLogger) *io.PipeWriter { + reader, writer := io.Pipe() + + go func() { + scanner := bufio.NewScanner(reader) + for scanner.Scan() { + log.Info(scanner.Text()) + } + if err := scanner.Err(); err != nil { + log.Errorf("Error while reading from Writer: %s", err) + } + reader.Close() + }() + runtime.SetFinalizer(writer, func(w *io.PipeWriter) { + writer.Close() + }) + + return writer +} + +func split(addr string) (string, int, error) { + host, port, err := net.SplitHostPort(addr) + if err != nil { + return host, 0, errors.New(" expected format is `address:port`") + } + + intPort, err := strconv.Atoi(port) + if err != nil { + return host, intPort, errors.Wrap(err, "port must be a number") + } + return host, intPort, nil +} + +func splitAddress(addr string) (string, int, error) { + host, port, err := split(addr) + if err != nil { + return "", 0, err + } + // Member list requires the address to be an ip address + if ip := net.ParseIP(host); ip == nil { + addresses, err := net.LookupHost(host) + if err != nil { + return "", 0, errors.Wrapf(err, "while preforming host lookup for '%s'", host) + } + if len(addresses) == 0 { + return "", 0, errors.Wrapf(err, "net.LookupHost() returned no addresses for '%s'", host) + } + host = addresses[0] + } + return host, port, nil +} diff --git a/discovery/memberlist_test.go b/discovery/memberlist_test.go new file mode 100644 index 00000000..fc9adb12 --- /dev/null +++ b/discovery/memberlist_test.go @@ -0,0 +1,40 @@ +package discovery_test + +import ( + "context" + "testing" + + "github.com/mailgun/holster/v3/discovery" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestMemberListMultiplePeers(t *testing.T) { + p0 := discovery.Peer{ID: "id-0", Metadata: []byte("address-0"), IsSelf: true} + p1 := discovery.Peer{ID: "id-1", Metadata: []byte("address-1")} + + onUpdateCh := make(chan []discovery.Peer, 2) + ml0, err := discovery.NewMemberList(context.Background(), discovery.MemberListConfig{ + BindAddress: "localhost:8519", + Peer: p0, + OnUpdate: func(peers []discovery.Peer) { + onUpdateCh <- peers + }, + }) + require.NoError(t, err) + defer ml0.Close(context.Background()) + + e := <-onUpdateCh + assert.Equal(t, e[0], p0) + + ml1, err := discovery.NewMemberList(context.Background(), discovery.MemberListConfig{ + KnownPeers: []string{"localhost:8519"}, + BindAddress: "localhost:8518", + Peer: p1, + }) + require.NoError(t, err) + defer ml1.Close(context.Background()) + + e = <-onUpdateCh + assert.Equal(t, []discovery.Peer{p0, p1}, e) +} diff --git a/election/cluster_test.go b/election/cluster_test.go index 345540b4..ab9c4849 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -9,7 +9,7 @@ import ( "github.com/mailgun/holster/v3/setter" ) -type ObsPair struct { +type ChangePair struct { From string Leader string } @@ -17,7 +17,7 @@ type ObsPair struct { // Useful in tests where you need to simulate an election cluster type TestCluster struct { Nodes map[string]*ClusterNode - ObserverCh chan ObsPair + OnChangeCh chan ChangePair errors map[string]error lock sync.Mutex } @@ -32,7 +32,7 @@ func NewTestCluster() *TestCluster { return &TestCluster{ Nodes: make(map[string]*ClusterNode), errors: make(map[string]error), - ObserverCh: make(chan ObsPair, 500), + OnChangeCh: make(chan ChangePair, 500), } } @@ -43,14 +43,14 @@ func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { SendRPC: c.sendRPC, } - conf.Name = name + conf.UniqueID = name conf.SendRPC = func(ctx context.Context, peer string, req election.RPCRequest, resp *election.RPCResponse) error { n.lock.RLock() defer n.lock.RUnlock() return n.SendRPC(name, peer, req, resp) } - conf.Observer = func(s string) { - c.ObserverCh <- ObsPair{ + conf.OnUpdate = func(s string) { + c.OnChangeCh <- ChangePair{ From: name, Leader: s, } diff --git a/election/election.go b/election/election.go index d2191c09..f8c7c0f7 100644 --- a/election/election.go +++ b/election/election.go @@ -65,11 +65,12 @@ type Config struct { // The Initial list of peers to be considered in the election, including ourself. Peers []string - // The name this peer identifies itself as, as found in the Peers list - Name string + // The unique id this peer identifies itself as, as found in the Peers list. + // This is typically an ip:port address the node is listening to for RPC requests. + UniqueID string // Called when the leader changes - Observer Observer + OnUpdate OnUpdate // The logger used errors and warning Log logrus.FieldLogger @@ -80,11 +81,11 @@ type Config struct { SendRPC func(context.Context, string, RPCRequest, *RPCResponse) error } -type Observer func(string) +type OnUpdate func(string) type Node interface { // Set the list of peers to be considered for the election, this list MUST - // include ourself as defined by `Config.Name`. + // include ourself as defined by `Config.UniqueID`. SetPeers([]string) error // If leader, resigns as leader and starts a new election that we will not @@ -130,21 +131,20 @@ type node struct { // Spawns a new node that will participate in the election. func SpawnNode(conf Config) (Node, error) { - - if conf.Name == "" { - return nil, errors.New("refusing to spawn a new node with no Config.Name defined") + if conf.UniqueID == "" { + return nil, errors.New("refusing to spawn a new node with no Config.UniqueID defined") } - setter.SetDefault(&conf.Log, logrus.WithField("name", conf.Name)) - setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*30) - setter.SetDefault(&conf.HeartBeatTimeout, time.Second*5) - setter.SetDefault(&conf.ElectionTimeout, time.Second*10) - setter.SetDefault(&conf.NetworkTimeout, time.Second*2) + setter.SetDefault(&conf.Log, logrus.WithField("id", conf.UniqueID)) + setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*60) + setter.SetDefault(&conf.HeartBeatTimeout, time.Second*20) + setter.SetDefault(&conf.ElectionTimeout, time.Second*15) + setter.SetDefault(&conf.NetworkTimeout, time.Second*3) c := &node{ shutdownCh: make(chan struct{}), rpcCh: make(chan RPCRequest, 5_000), - self: conf.Name, + self: conf.UniqueID, conf: conf, log: conf.Log, } @@ -169,6 +169,11 @@ func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { func (e *node) SetPeers(peers []string) error { e.lock.Lock() defer e.lock.Unlock() + + if len(peers) == 0 { + return nil + } + e.peers = peers return nil } @@ -214,8 +219,8 @@ func (e *node) setLeader(leader string) { if e.leader != leader { e.log.Debugf("Set Leader (%s)", leader) e.leader = leader - if e.conf.Observer != nil { - e.conf.Observer(leader) + if e.conf.OnUpdate != nil { + e.conf.OnUpdate(leader) } } } @@ -253,10 +258,8 @@ func (e *node) Close() { func (e *node) run() { for { - e.log.Debug("main loop") select { case <-e.shutdownCh: - e.setLeader("") e.setState(ShutdownState) return default: @@ -276,13 +279,13 @@ func (e *node) run() { func (e *node) runFollower() { e.log.Debugf("entering follower state, current leader is '%s'", e.Leader()) heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) + defer heartbeatTimer.Stop() for e.state == FollowerState { select { case rpc := <-e.rpcCh: e.processRPC(rpc) case <-heartbeatTimer.C: - // Check if we have had successful contact with the leader if time.Now().Sub(e.lastContact) < e.conf.HeartBeatTimeout { continue @@ -294,7 +297,6 @@ func (e *node) runFollower() { e.setState(CandidateState) return case <-e.shutdownCh: - heartbeatTimer.Stop() return } } @@ -308,9 +310,11 @@ func (e *node) runCandidate() { // likely that the first node to send vote requests will win the election, and avoid // a stalemate. voteTimer := time.NewTimer(randomDuration(e.conf.HeartBeatTimeout / 10)) + defer voteTimer.Stop() // We re-start the vote if we have not received a heart beat from a chosen leader before // this timer expires. electionTimer := time.NewTimer(randomDuration(e.conf.ElectionTimeout)) + defer electionTimer.Stop() // Tally the votes, need a simple majority grantedVotes := 0 @@ -321,7 +325,6 @@ func (e *node) runCandidate() { select { case <-voteTimer.C: voteCh = e.electSelf() - voteTimer.Stop() case rpc := <-e.rpcCh: e.processRPC(rpc) case vote := <-voteCh: @@ -349,7 +352,6 @@ func (e *node) runCandidate() { case <-electionTimer.C: // Election failed! Restart the election. We simply return, which will kick us back into runCandidate e.log.Debug("Election timeout reached, restarting election") - electionTimer.Stop() return case <-e.shutdownCh: return @@ -357,13 +359,13 @@ func (e *node) runCandidate() { } } -// electSelf is used to send a SendVote() RPC to all peers with a vote for -// ourself. This has the side affecting of incrementing the current term. The +// electSelf is used to send a VoteReq RPC to all peers with a vote for +// ourself. This has the side affect of incrementing the current term. The // response channel returned is used to wait for all the responses, including a // vote for ourself. func (e *node) electSelf() <-chan VoteResp { peers := e.GetPeers() - respCh := make(chan VoteResp, len(peers)) + respCh := make(chan VoteResp, len(peers)+1) // Increment the term e.currentTerm++ @@ -425,10 +427,12 @@ func (e *node) electSelf() <-chan VoteResp { } func (e *node) runLeader() { + heartBeatTicker := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout / 3)) + defer heartBeatTicker.Stop() quorumTicker := time.NewTicker(e.conf.LeaderQuorumTimeout) - heartBeatTicker := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout / 10)) - heartBeatReplyCh := make(chan HeartBeatResp, 5_000) + defer quorumTicker.Stop() peersLastContact := make(map[string]time.Time, len(e.GetPeers())) + heartBeatReplyCh := make(chan HeartBeatResp, 5_000) for e.state == LeaderState { select { @@ -469,19 +473,19 @@ func (e *node) runLeader() { contacted++ } - // Verify we can contact a quorum + // Verify we can contact a quorum (Minus ourself) quorum := e.quorumSize() - if contacted < quorum { + if contacted < (quorum - 1) { e.log.Debug("failed to receive heart beats from a quorum of peers; stepping down") e.state = FollowerState - // TODO: Perhaps we send ResetElection to what peers we can? - // This would avoid having to wait for the heartbeat timeout - // to start a new election. + + // Inform the other peers we are stepping down + for _, peer := range e.GetPeers() { + e.sendElectionReset(peer) + } } case <-e.shutdownCh: e.state = ShutdownState - heartBeatTicker.Stop() - quorumTicker.Stop() if e.IsLeader() { // Notify all followers we are no longer leader for _, peer := range e.GetPeers() { @@ -494,7 +498,6 @@ func (e *node) runLeader() { if e.IsLeader() { e.setLeader("") } - quorumTicker.Stop() } func (e *node) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { @@ -546,7 +549,7 @@ func (e *node) sendElectionReset(peer string) { } func (e *node) processRPC(rpc RPCRequest) { - // TODO: Should check for state = shutdown? + e.log.Debugf("RPC: %#v\n", rpc) switch cmd := rpc.Request.(type) { case VoteReq: e.handleVote(rpc, cmd) @@ -604,8 +607,8 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // It's also possible that one leader sends it's heartbeats before the other leader, // in that case the first leader to send a heartbeat becomes leader. // - // This can also occur if a follower loses connectivity to the rest of the cluster. - // In this case we become the follower of who ever sent us the heartbeat. + // This can also occur if a leader loses connectivity to the rest of the cluster. + // In this case we become the follower of who ever sent us a heartbeat. if e.state != FollowerState { e.state = FollowerState e.currentTerm = req.Term diff --git a/election/election_test.go b/election/election_test.go index 33cdad7b..6c282d53 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -59,6 +59,33 @@ func createCluster(t *testing.T, c *TestCluster) { }) } +func TestSingleNodeLeader(t *testing.T) { + c := NewTestCluster() + err := c.SpawnNode("n0", cfg) + require.NoError(t, err) + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, ClusterStatus{ + "n0": "n0", + }, status) + }) + + // Consume first leader election event + event := <-c.OnChangeCh + assert.Equal(t, "n0", event.Leader) + assert.Equal(t, "n0", event.From) + + assert.True(t, c.Nodes["n0"].Node.IsLeader()) + + select { + // Should NOT receive a leadership change as we are the only node + case <-c.OnChangeCh: + t.Log("received un-expected leader change") + t.FailNow() + case <-time.After(cfg.HeartBeatTimeout * 3): + } +} + func TestSimpleElection(t *testing.T) { c := NewTestCluster() createCluster(t, c) diff --git a/election/example_test.go b/election/example_test.go index 7e862e37..53fe123b 100644 --- a/election/example_test.go +++ b/election/example_test.go @@ -71,9 +71,9 @@ func SimpleExample(t *testing.T) { // A list of known peers at startup Peers: []string{"localhost:7080", "localhost:7081"}, // A unique identifier used to identify us in a list of peers - Name: "localhost:7080", + UniqueID: "localhost:7080", // Called whenever the library detects a change in leadership - Observer: func(leader string) { + OnUpdate: func(leader string) { log.Printf("Current Leader: %s\n", leader) }, // Called when the library wants to contact other peers @@ -85,9 +85,9 @@ func SimpleExample(t *testing.T) { defer node1.Close() node2, err := election.SpawnNode(election.Config{ - Peers: []string{"localhost:7080", "localhost:7081"}, - Name: "localhost:7081", - SendRPC: sendRPC, + Peers: []string{"localhost:7080", "localhost:7081"}, + UniqueID: "localhost:7081", + SendRPC: sendRPC, }) if err != nil { log.Fatal(err) diff --git a/election/memberlist.go b/election/memberlist.go deleted file mode 100644 index faceec1d..00000000 --- a/election/memberlist.go +++ /dev/null @@ -1,246 +0,0 @@ -package election - -import ( - "bufio" - "context" - "encoding/json" - "io" - "net" - "runtime" - "strconv" - - ml "github.com/hashicorp/memberlist" - "github.com/mailgun/holster/v3/clock" - "github.com/mailgun/holster/v3/errors" - "github.com/mailgun/holster/v3/retry" - "github.com/mailgun/holster/v3/setter" - "github.com/sirupsen/logrus" -) - -type PeerInfo struct { - // The http address:port of the peer - HTTPAddress string `json:"http-address"` - // The grpc address:port of the peer - GRPCAddress string `json:"grpc-address"` - // Is true if PeerInfo matches the Name as given in the memberlist config - IsOwner bool `json:"is-owner,omitempty"` -} - -type UpdateFunc func([]PeerInfo) - -type MemberListPool struct { - log logrus.FieldLogger - memberList *ml.Memberlist - conf MemberListPoolConfig - events *memberListEventHandler -} - -type MemberListPoolConfig struct { - // This is the address:port the member list protocol listen for other members on - MemberListAddress string - - // The information about this peer which should be shared with all other members - PeerInfo PeerInfo - - // A list of nodes this member list instance can contact to find other members. - KnownNodes []string - - // A callback function which is called when the member list changes - OnUpdate UpdateFunc - - // An interface through which logging will occur (Usually *logrus.Entry) - Logger logrus.FieldLogger -} - -func NewMemberListPool(ctx context.Context, conf MemberListPoolConfig) (*MemberListPool, error) { - setter.SetDefault(conf.Logger, logrus.WithField("category", "gubernator")) - m := &MemberListPool{ - log: conf.Logger, - conf: conf, - } - - host, port, err := splitAddress(conf.MemberListAddress) - if err != nil { - return nil, errors.Wrap(err, "MemberListAddress=`%s` is invalid;") - } - - // Member list requires the address to be an ip address - if ip := net.ParseIP(host); ip == nil { - addrs, err := net.LookupHost(host) - if err != nil { - return nil, errors.Wrapf(err, "while preforming host lookup for '%s'", host) - } - if len(addrs) == 0 { - return nil, errors.Wrapf(err, "net.LookupHost() returned no addresses for '%s'", host) - } - host = addrs[0] - } - - // Configure member list event handler - m.events = &memberListEventHandler{ - conf: conf, - log: m.log, - } - m.events.peers = make(map[string]PeerInfo) - - // Configure member list - config := ml.DefaultWANConfig() - config.Events = m.events - config.AdvertiseAddr = host - config.AdvertisePort = port - config.Name = conf.PeerInfo.HTTPAddress - config.LogOutput = newLogWriter(m.log) - - // Create and set member list - memberList, err := ml.Create(config) - if err != nil { - return nil, err - } - m.memberList = memberList - conf.PeerInfo.IsOwner = false - - // Join member list pool - err = m.joinPool(ctx) - if err != nil { - return nil, errors.Wrap(err, "while attempting to join the member-list pool") - } - - return m, nil -} - -func (m *MemberListPool) joinPool(ctx context.Context) error { - // Get local node and set metadata - node := m.memberList.LocalNode() - b, err := json.Marshal(&m.conf.PeerInfo) - if err != nil { - return errors.Wrap(err, "error marshalling metadata") - } - node.Meta = b - - err = retry.Until(ctx, retry.Interval(clock.Millisecond*300), func(ctx context.Context, i int) error { - // Join member list - _, err = m.memberList.Join(m.conf.KnownNodes) - if err != nil { - return errors.Wrap(err, "while joining member-list") - } - return nil - }) - if err != nil { - return errors.Wrap(err, "timed out attempting to join member list") - } - - // Add the local node to the event handler's peer list - m.events.addPeer(node) - return nil -} - -func (m *MemberListPool) Close() { - err := m.memberList.Leave(clock.Second) - if err != nil { - m.log.Warn(errors.Wrap(err, "while leaving member-list")) - } -} - -type memberListEventHandler struct { - peers map[string]PeerInfo - log logrus.FieldLogger - conf MemberListPoolConfig -} - -func (e *memberListEventHandler) addPeer(node *ml.Node) { - ip := getIP(node.Address()) - - // Deserialize metadata - var metadata PeerInfo - if err := json.Unmarshal(node.Meta, &metadata); err != nil { - e.log.WithError(err).Warnf("while adding to peers") - return - } - e.peers[ip] = metadata - e.callOnUpdate() -} - -func (e *memberListEventHandler) NotifyJoin(node *ml.Node) { - ip := getIP(node.Address()) - - // Deserialize metadata - var metadata PeerInfo - if err := json.Unmarshal(node.Meta, &metadata); err != nil { - e.log.WithError(err).Warn("while deserialize member-list metadata") - return - } - e.peers[ip] = metadata - e.callOnUpdate() -} - -func (e *memberListEventHandler) NotifyLeave(node *ml.Node) { - ip := getIP(node.Address()) - - // Remove PeerInfo - delete(e.peers, ip) - - e.callOnUpdate() -} - -func (e *memberListEventHandler) NotifyUpdate(node *ml.Node) { - ip := getIP(node.Address()) - - // Deserialize metadata - var metadata PeerInfo - if err := json.Unmarshal(node.Meta, &metadata); err != nil { - e.log.WithError(err).Warn("while updating member-list") - return - } - e.peers[ip] = metadata - e.callOnUpdate() -} - -func (e *memberListEventHandler) callOnUpdate() { - var peers []PeerInfo - - for _, p := range e.peers { - if p.HTTPAddress == e.conf.PeerInfo.HTTPAddress { - p.IsOwner = true - } - peers = append(peers, p) - } - e.conf.OnUpdate(peers) -} - -func getIP(address string) string { - addr, _, _ := net.SplitHostPort(address) - return addr -} - -func newLogWriter(log logrus.FieldLogger) *io.PipeWriter { - reader, writer := io.Pipe() - - go func() { - scanner := bufio.NewScanner(reader) - for scanner.Scan() { - log.Info(scanner.Text()) - } - if err := scanner.Err(); err != nil { - log.Errorf("Error while reading from Writer: %s", err) - } - reader.Close() - }() - runtime.SetFinalizer(writer, func(w *io.PipeWriter) { - writer.Close() - }) - - return writer -} - -func splitAddress(addr string) (string, int, error) { - host, port, err := net.SplitHostPort(addr) - if err != nil { - return host, 0, errors.New(" expected format is `address:port`") - } - - intPort, err := strconv.Atoi(port) - if err != nil { - return host, intPort, errors.Wrap(err, "port must be a number") - } - return host, intPort, nil -} diff --git a/go.mod b/go.mod index d60b17d4..e5da04cc 100644 --- a/go.mod +++ b/go.mod @@ -10,6 +10,7 @@ require ( github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect + github.com/davecgh/go-spew v1.1.1 github.com/dgrijalva/jwt-go v3.2.0+incompatible // indirect github.com/gogo/protobuf v1.2.1 // indirect github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef // indirect @@ -22,6 +23,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect github.com/grpc-ecosystem/grpc-gateway v1.10.0 // indirect github.com/hashicorp/consul/api v1.7.0 + github.com/hashicorp/go-hclog v0.12.0 github.com/hashicorp/golang-lru v0.5.1 // indirect github.com/hashicorp/memberlist v0.2.2 github.com/jonboulle/clockwork v0.1.0 // indirect From 16c9217bbac3eb632f55df9118fd1580fa46ea28 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Thu, 21 Jan 2021 15:50:42 -0600 Subject: [PATCH 07/15] Added GetSRVAddresses() to discovery package --- .DS_Store | Bin 6148 -> 0 bytes discovery/memberlist.go | 2 +- discovery/srv.go | 87 ++++++++++++++++++++++++++++++++++++++++ discovery/srv_test.go | 52 ++++++++++++++++++++++++ go.mod | 4 +- 5 files changed, 142 insertions(+), 3 deletions(-) delete mode 100644 .DS_Store create mode 100644 discovery/srv.go create mode 100644 discovery/srv_test.go diff --git a/.DS_Store b/.DS_Store deleted file mode 100644 index 22608d3e05b13c826def38ce22e2b50118bfa556..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeHKI|>3Z5S>vG!N$@uSMUZw^aNf&{8hq&V!xH=@@T&K6v}F+h4Ka_FPY3s$SZbs zL`3J8-ArU6A|tq=Ty5x@?VES3mk|ZRamGo`huwL*X_{jv`*py$L)n9QlID?b8#F3F z1*iZOpaN9j;|gSn9gRPJG7qEzRN&VYuHg$K#o*^3Op;&kLASb{}TSD z|9>WNMFpt9Un!u2#cDCfld`sU9%r?-z}IlgxxvjacM1kC$3QQ~SXeopdQ#*Sn`6Hw Uwt-GZ-0496445u7D)4Ou?iJM)uK)l5 diff --git a/discovery/memberlist.go b/discovery/memberlist.go index 8dee0d1c..6c74804c 100644 --- a/discovery/memberlist.go +++ b/discovery/memberlist.go @@ -103,7 +103,7 @@ func NewMemberList(ctx context.Context, conf MemberListConfig) (Catalog, error) err = retry.Until(ctx, retry.Interval(clock.Millisecond*300), func(ctx context.Context, i int) error { _, err = m.memberList.Join(m.conf.KnownPeers) if err != nil { - return errors.Wrap(err, "while joining member-list") + return errors.Wrapf(err, "while joining member list known peers %#v", m.conf.KnownPeers) } return nil }) diff --git a/discovery/srv.go b/discovery/srv.go new file mode 100644 index 00000000..ec8c5cb5 --- /dev/null +++ b/discovery/srv.go @@ -0,0 +1,87 @@ +package discovery + +import ( + "fmt" + "net" + "time" + + "github.com/mailgun/holster/v3/errors" + "github.com/miekg/dns" +) + +// Given a DNS name return a list of addresses returned by the +// getting the SRV records from DNS for that name. +func GetSRVAddresses(dnsName, dnsServer string) ([]string, error) { + if dnsServer != "" { + return directLookupSRV(dnsName, dnsServer) + } + return builtinLookupSRV(dnsName) +} + +// Queries the DNS server, by passing /etc/resolv.conf +func directLookupSRV(dnsName, dnsAddr string) ([]string, error) { + if !dns.IsFqdn(dnsName) { + dnsName += "." + } + + c := new(dns.Client) + c.Timeout = time.Second * 3 + + m := new(dns.Msg) + m.SetQuestion(dnsName, dns.TypeSRV) + r, _, err := c.Exchange(m, dnsAddr) + if err != nil { + return nil, err + } + + if r.Rcode != dns.RcodeSuccess { + return nil, fmt.Errorf("no SRV records found for '%s'", dnsName) + } + + var results []string + for _, a := range r.Answer { + srv := a.(*dns.SRV) + if net.ParseIP(srv.Target) == nil { + srv.Target = findARecord(srv.Target, r.Extra) + } + results = append(results, fmt.Sprintf("%s:%d", srv.Target, srv.Port)) + } + return results, nil +} + +// Attempts to find an 'A' record within the extra (additional answer section) of the DNS response +func findARecord(target string, extra []dns.RR) string { + for _, item := range extra { + if a, ok := item.(*dns.A); ok { + if a.Hdr.Name == target { + return a.A.String() + } + } + } + return target +} + +// Uses the builtin golang net.LookupSRV on systems that have their +// /etc/resolv.conf configured correctly +func builtinLookupSRV(dnsName string) ([]string, error) { + _, records, err := net.LookupSRV("", "", dnsName) + if err != nil { + return nil, err + } + + var results []string + for _, srv := range records { + if net.ParseIP(srv.Target) == nil { + addresses, err := net.LookupHost(srv.Target) + if err != nil { + return results, errors.Wrapf(err, "while looking up A record for '%s'", srv.Target) + } + if len(addresses) == 0 { + return results, errors.Wrapf(err, "no A records found for '%s'", srv.Target) + } + srv.Target = addresses[0] + } + results = append(results, fmt.Sprintf("%s:%d", srv.Target, srv.Port)) + } + return results, nil +} diff --git a/discovery/srv_test.go b/discovery/srv_test.go new file mode 100644 index 00000000..5d4da025 --- /dev/null +++ b/discovery/srv_test.go @@ -0,0 +1,52 @@ +package discovery_test + +import ( + "testing" + + "github.com/hashicorp/consul/api" + "github.com/mailgun/holster/v3/discovery" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestGetSRVAddressesDirect(t *testing.T) { + client, err := api.NewClient(api.DefaultConfig()) + require.NoError(t, err) + + // Register ourselves in consul as a member of the cluster + err = client.Agent().ServiceRegisterOpts(&api.AgentServiceRegistration{ + Name: "scout", + ID: "1234123", + Tags: []string{"ml"}, + Address: "127.0.0.1", + Port: 2319, + }, api.ServiceRegisterOpts{ReplaceExistingChecks: true}) + require.NoError(t, err) + //defer client.Agent().ServiceDeregister("1234123") + + addresses, err := discovery.GetSRVAddresses("ml.scout.service.consul", "127.0.0.1:8600") + require.NoError(t, err) + + assert.Equal(t, []string{"127.0.0.1:2319"}, addresses) +} + +func TestGetSRVAddresses(t *testing.T) { + client, err := api.NewClient(api.DefaultConfig()) + require.NoError(t, err) + + // Register ourselves in consul as a member of the cluster + err = client.Agent().ServiceRegisterOpts(&api.AgentServiceRegistration{ + Name: "scout", + ID: "123-2319", + Tags: []string{"mll"}, + Address: "127.0.0.1", + Port: 2319, + }, api.ServiceRegisterOpts{ReplaceExistingChecks: true}) + require.NoError(t, err) + defer client.Agent().ServiceDeregister("123-2319") + + addresses, err := discovery.GetSRVAddresses("mll.scout.service.consul", "") + require.NoError(t, err) + + assert.Equal(t, []string{"127.0.0.1:2319"}, addresses) +} diff --git a/go.mod b/go.mod index e5da04cc..84106cb6 100644 --- a/go.mod +++ b/go.mod @@ -10,13 +10,12 @@ require ( github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect - github.com/davecgh/go-spew v1.1.1 github.com/dgrijalva/jwt-go v3.2.0+incompatible // indirect github.com/gogo/protobuf v1.2.1 // indirect github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef // indirect github.com/golang/protobuf v1.3.2 github.com/google/btree v1.0.0 // indirect - github.com/google/uuid v1.1.2 // indirect + github.com/google/uuid v1.1.2 github.com/gorilla/mux v1.7.3 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.0 // indirect @@ -27,6 +26,7 @@ require ( github.com/hashicorp/golang-lru v0.5.1 // indirect github.com/hashicorp/memberlist v0.2.2 github.com/jonboulle/clockwork v0.1.0 // indirect + github.com/miekg/dns v1.1.26 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.1.0 // indirect github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 // indirect From 15cc65db71614cac559142e9d18a839834a87628 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Tue, 2 Feb 2021 16:51:18 -0600 Subject: [PATCH 08/15] Single node elections occur much faster --- discovery/memberlist.go | 16 ++++++++-------- election/election.go | 22 ++++++++++++++++++++++ election/election_test.go | 4 ++-- 3 files changed, 32 insertions(+), 10 deletions(-) diff --git a/discovery/memberlist.go b/discovery/memberlist.go index 6c74804c..7872c74b 100644 --- a/discovery/memberlist.go +++ b/discovery/memberlist.go @@ -19,22 +19,22 @@ import ( ) type Peer struct { - // An ID the uniquely identifies this peer in the catalog + // An ID the uniquely identifies this peer ID string // The metadata associated with this peer Metadata []byte - // Is true if this Peer refers to our instance in the catalog + // Is true if this Peer refers to our instance IsSelf bool } type OnUpdateFunc func([]Peer) -type Catalog interface { - // Returns the peers currently registered in the catalog +type Members interface { + // Returns the peers currently registered GetPeers(context.Context) ([]Peer, error) - // Removes our peer from the catalog and closes all connections + // Removes our peer from the member list and closes all connections Close(context.Context) error - // TODO: Updates the Peer metadata shared with peers in the catalog + // TODO: Updates the Peer metadata shared with peers //UpdatePeer(context.Context, Peer) error } @@ -52,7 +52,7 @@ type MemberListConfig struct { // This is the address:port the member list protocol will advertise to other peers. (Defaults to BindAddress) AdvertiseAddress string - // Metadata about this peer which should be shared with all other peers in the same catalog. + // Metadata about this peer which should be shared with other peers Peer Peer // A list of peers this member list instance can contact to find other peers. @@ -65,7 +65,7 @@ type MemberListConfig struct { Logger logrus.FieldLogger } -func NewMemberList(ctx context.Context, conf MemberListConfig) (Catalog, error) { +func NewMemberList(ctx context.Context, conf MemberListConfig) (Members, error) { setter.SetDefault(&conf.Logger, logrus.WithField("category", "member-list")) setter.SetDefault(&conf.AdvertiseAddress, conf.BindAddress) if conf.Peer.ID == "" { diff --git a/election/election.go b/election/election.go index f8c7c0f7..b50d8aee 100644 --- a/election/election.go +++ b/election/election.go @@ -225,6 +225,12 @@ func (e *node) setLeader(leader string) { } } +func (e *node) getLeader() string { + e.lock.Lock() + defer e.lock.Unlock() + return e.leader +} + // Resign will cause this node to step down as leader, if this // node is NOT leader, this does nothing and returns 'false' func (e *node) Resign() bool { @@ -280,6 +286,8 @@ func (e *node) runFollower() { e.log.Debugf("entering follower state, current leader is '%s'", e.Leader()) heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) defer heartbeatTimer.Stop() + noPeersTimer := time.NewTimer(e.conf.HeartBeatTimeout / 5) + defer noPeersTimer.Stop() for e.state == FollowerState { select { @@ -296,6 +304,20 @@ func (e *node) runFollower() { e.setLeader("") e.setState(CandidateState) return + case <-noPeersTimer.C: + peers := e.GetPeers() + + // If we already have leader, ignore peer list + if e.getLeader() != "" { + return + } + + // If we have no peers, or if we are the only peer, no need to wait + // for the heartbeat timeout. Change state to candidate and start the election. + if len(peers) == 0 || len(peers) == 1 && peers[0] == e.self { + e.setState(CandidateState) + return + } case <-e.shutdownCh: return } diff --git a/election/election_test.go b/election/election_test.go index 6c282d53..ae1ecfc4 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -138,7 +138,7 @@ func TestFollowerDisconnect(t *testing.T) { defer c.DelNetworkError("n4") // Wait until n4 loses leader - testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { status := c.GetClusterStatus() assert.NotEqual(t, "n0", status["n4"]) }) @@ -146,7 +146,7 @@ func TestFollowerDisconnect(t *testing.T) { c.DelNetworkError("n4") // Follower should resume being a follower without forcing a new election. - testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + testutil.UntilPass(t, 60, time.Second, func(t testutil.TestingT) { status := c.GetClusterStatus() assert.Equal(t, "n0", status["n4"]) }) From ffb707da348173152e0c056134ac7c9fe6713f92 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Thu, 4 Feb 2021 17:01:42 -0600 Subject: [PATCH 09/15] SetPeer and Get accessors are now available via RPC calls --- cmd/election/main.go | 9 ++ collections/lru_cache.go | 4 +- collections/priority_queue_test.go | 2 +- discovery/consul.go | 2 +- election/cluster_test.go | 5 +- election/election.go | 181 ++++++++++++++++------------- election/election_test.go | 10 +- election/rpc.go | 26 +++++ election/rpc_test.go | 52 +++++++-- election/structs.pb.go | 109 ++++++++++++++--- election/structs.proto | 17 +++ syncutil/broadcast_test.go | 2 +- 12 files changed, 306 insertions(+), 113 deletions(-) diff --git a/cmd/election/main.go b/cmd/election/main.go index 3d5d48f7..a5a630f4 100644 --- a/cmd/election/main.go +++ b/cmd/election/main.go @@ -54,6 +54,15 @@ func newHandler(node election.Node) func(w http.ResponseWriter, r *http.Request) w.WriteHeader(http.StatusBadRequest) w.Write([]byte(err.Error())) } + + // Example of how a peer might exclude RPC + // commands it doesn't want made. + if req.RPC == election.SetPeersRPC { + w.WriteHeader(http.StatusBadRequest) + w.Write([]byte(fmt.Sprintf("RPC request '%s' not allowed", req.RPC))) + return + } + var resp election.RPCResponse node.ReceiveRPC(req, &resp) diff --git a/collections/lru_cache.go b/collections/lru_cache.go index 80eb102f..9af33d1d 100644 --- a/collections/lru_cache.go +++ b/collections/lru_cache.go @@ -198,7 +198,7 @@ func (c *LRUCache) Peek(key interface{}) (value interface{}, ok bool) { // Processes each item in the cache in a thread safe way, such that the cache can be in use // while processing items in the cache. Processing the cache with `Each()` does not update // the expiration or last used. -func (c LRUCache) Each(concurrent int, callBack func(key interface{}, value interface{}) error) []error { +func (c *LRUCache) Each(concurrent int, callBack func(key interface{}, value interface{}) error) []error { fanOut := syncutil.NewFanOut(concurrent) keys := c.Keys() @@ -239,4 +239,4 @@ func (c *LRUCache) Map(mapping func(item *CacheItem) bool) { c.removeElement(v) } } -} \ No newline at end of file +} diff --git a/collections/priority_queue_test.go b/collections/priority_queue_test.go index 3b167742..adc72824 100644 --- a/collections/priority_queue_test.go +++ b/collections/priority_queue_test.go @@ -89,7 +89,7 @@ func TestUpdate(t *testing.T) { assert.Equal(t, 1, toInt(mh.Peek().Value)) } -func Example_Priority_Queue_Usage() { +func ExampleNewPriorityQueue() { queue := collections.NewPriorityQueue() queue.Push(&collections.PQItem{ diff --git a/discovery/consul.go b/discovery/consul.go index ec79c544..e765f804 100644 --- a/discovery/consul.go +++ b/discovery/consul.go @@ -46,7 +46,7 @@ type consulStore struct { ctx cancel.Context } -func NewConsul(conf *ConsulConfig) (Catalog, error) { +func NewConsul(conf *ConsulConfig) (Members, error) { setter.SetDefault(&conf.Logger, logrus.WithField("category", "consul-catalog")) setter.SetDefault(&conf.ClientConfig, api.DefaultConfig()) var err error diff --git a/election/cluster_test.go b/election/cluster_test.go index ab9c4849..d67d9f27 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -65,7 +65,10 @@ func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { } func (c *TestCluster) Add(name string, node *ClusterNode) { + c.lock.Lock() + defer c.lock.Unlock() c.Nodes[name] = node + node.lock.Lock() defer node.lock.Unlock() node.SendRPC = c.sendRPC @@ -97,7 +100,7 @@ type ClusterStatus map[string]string func (c *TestCluster) GetClusterStatus() ClusterStatus { status := make(ClusterStatus) for k, v := range c.Nodes { - status[k] = v.Node.Leader() + status[k] = v.Node.GetLeader() } return status } diff --git a/election/election.go b/election/election.go index b50d8aee..f1800dfd 100644 --- a/election/election.go +++ b/election/election.go @@ -4,7 +4,6 @@ import ( "context" "errors" "math/rand" - "sync" "time" "github.com/mailgun/holster/v3/setter" @@ -13,14 +12,15 @@ import ( "github.com/sirupsen/logrus" ) -type State uint32 +type NodeState GetStateResp +type state uint32 const ( // FollowerState means we are following the leader and expect // to get heart beats regularly. This is the initial state, as // we don't want to force an election when a new node joins // the cluster. - FollowerState State = iota + FollowerState state = iota // CandidateState means we are actively attempting to become leader CandidateState // LeaderState means we have received a quorum of votes while @@ -30,7 +30,7 @@ const ( ShutdownState ) -func (s State) String() string { +func (s state) String() string { switch s { case FollowerState: return "Follower" @@ -96,12 +96,12 @@ type Node interface { IsLeader() bool // Returns the current leader - Leader() string + GetLeader() string // Returns the current state of this node - State() State + GetState() NodeState - // Called + // Called when this peer receives a RPC request from a peer ReceiveRPC(RPCRequest, *RPCResponse) // Cancels the election, resigns if we are leader and waits for all go @@ -111,7 +111,7 @@ type Node interface { type node struct { conf Config // The election configuration - state State // Current state of our node + state state // Current state of our node vote struct { CurrentTerm uint64 LastTerm uint64 @@ -120,7 +120,6 @@ type node struct { currentTerm uint64 // The current term of the election when in candidate state rpcCh chan RPCRequest // RPC Response channel, listen for for RPC responses on this channel self string // Our name - lock sync.RWMutex // lock for peers and leader peers []string leader string lastContact time.Time // The last successful contact with the leader (if we are a follower) @@ -167,55 +166,41 @@ func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { // SetPeers is a thread safe way to dynamically add or remove peers in a running cluster. // These peers will be contacted when requesting votes during leader election. func (e *node) SetPeers(peers []string) error { - e.lock.Lock() - defer e.lock.Unlock() - - if len(peers) == 0 { - return nil - } - - e.peers = peers + _ = <-e.send(SetPeersReq{Peers: peers}) return nil } // GetPeers returns the current peers this node knows about. func (e *node) GetPeers() []string { - e.lock.RLock() - defer e.lock.RUnlock() - return e.peers -} - -// State returns the current state of this node -func (e *node) State() State { - e.lock.RLock() - defer e.lock.RUnlock() - return e.state + return e.GetState().Peers } -func (e *node) setState(state State) { - e.log.Debugf("State Change (%s)", state) - e.lock.RLock() - defer e.lock.RUnlock() - e.state = state +// GetState returns the current state of this node +func (e *node) GetState() NodeState { + select { + case resp := <-e.send(GetStateReq{}): + if s, ok := resp.Response.(GetStateResp); ok { + return NodeState(s) + } + } + return NodeState{} } // IsLeader returns true if this node was elected leader func (e *node) IsLeader() bool { - e.lock.RLock() - defer e.lock.RUnlock() - return e.self == e.leader + return e.self == e.GetState().Leader } // Leader returns the name of the node that is currently leader -func (e *node) Leader() string { - e.lock.RLock() - defer e.lock.RUnlock() - return e.leader +func (e *node) GetLeader() string { + return e.GetState().Leader +} + +func (e *node) isLeader() bool { + return e.self == e.leader } func (e *node) setLeader(leader string) { - e.lock.Lock() - defer e.lock.Unlock() if e.leader != leader { e.log.Debugf("Set Leader (%s)", leader) e.leader = leader @@ -226,22 +211,14 @@ func (e *node) setLeader(leader string) { } func (e *node) getLeader() string { - e.lock.Lock() - defer e.lock.Unlock() return e.leader } // Resign will cause this node to step down as leader, if this // node is NOT leader, this does nothing and returns 'false' func (e *node) Resign() bool { - respCh := make(chan RPCResponse, 1) - e.rpcCh <- RPCRequest{ - Request: ResignReq{}, - respChan: respCh, - } - select { - case rpcResp := <-respCh: + case rpcResp := <-e.send(ResignReq{}): resp, ok := rpcResp.Response.(ResignResp) if !ok { return false @@ -262,11 +239,12 @@ func (e *node) Close() { e.wg.Wait() } +// Main thread loop func (e *node) run() { for { select { case <-e.shutdownCh: - e.setState(ShutdownState) + e.state = ShutdownState return default: } @@ -283,7 +261,7 @@ func (e *node) run() { } func (e *node) runFollower() { - e.log.Debugf("entering follower state, current leader is '%s'", e.Leader()) + e.log.Debugf("entering follower state, current leader is '%s'", e.leader) heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) defer heartbeatTimer.Stop() noPeersTimer := time.NewTimer(e.conf.HeartBeatTimeout / 5) @@ -300,22 +278,20 @@ func (e *node) runFollower() { } // Heartbeat failed! Transition to the candidate state - e.log.Debugf("heartbeat timeout, starting election; previous leader was '%s'", e.Leader()) + e.log.Debugf("heartbeat timeout, starting election; previous leader was '%s'", e.leader) e.setLeader("") - e.setState(CandidateState) + e.state = CandidateState return case <-noPeersTimer.C: - peers := e.GetPeers() - - // If we already have leader, ignore peer list + // If we already have leader, don't check for no peers if e.getLeader() != "" { - return + continue } // If we have no peers, or if we are the only peer, no need to wait // for the heartbeat timeout. Change state to candidate and start the election. - if len(peers) == 0 || len(peers) == 1 && peers[0] == e.self { - e.setState(CandidateState) + if len(e.peers) == 0 || len(e.peers) == 1 && e.peers[0] == e.self { + e.state = CandidateState return } case <-e.shutdownCh: @@ -343,7 +319,7 @@ func (e *node) runCandidate() { votesNeeded := e.quorumSize() e.log.Debugf("votes needed: %d", votesNeeded) - for e.State() == CandidateState { + for e.state == CandidateState { select { case <-voteTimer.C: voteCh = e.electSelf() @@ -386,8 +362,7 @@ func (e *node) runCandidate() { // response channel returned is used to wait for all the responses, including a // vote for ourself. func (e *node) electSelf() <-chan VoteResp { - peers := e.GetPeers() - respCh := make(chan VoteResp, len(peers)+1) + respCh := make(chan VoteResp, len(e.peers)+1) // Increment the term e.currentTerm++ @@ -439,7 +414,7 @@ func (e *node) electSelf() <-chan VoteResp { } // For each peer, request a vote - for _, peer := range peers { + for _, peer := range e.peers { if peer == e.self { continue } @@ -453,23 +428,29 @@ func (e *node) runLeader() { defer heartBeatTicker.Stop() quorumTicker := time.NewTicker(e.conf.LeaderQuorumTimeout) defer quorumTicker.Stop() - peersLastContact := make(map[string]time.Time, len(e.GetPeers())) + peersLastContact := make(map[string]time.Time, len(e.peers)) heartBeatReplyCh := make(chan HeartBeatResp, 5_000) for e.state == LeaderState { select { case rpc := <-e.rpcCh: e.processRPC(rpc) + // If the RPC was a set peers request, immediately send heart beats to all nodes + if _, ok := rpc.Request.(SetPeersReq); ok { + for _, peer := range e.peers { + e.sendHeartBeat(peer, heartBeatReplyCh) + } + } case reply := <-heartBeatReplyCh: // Is the reply from a peer we are familiar with? - if !slice.ContainsString(reply.From, e.GetPeers(), nil) { + if !slice.ContainsString(reply.From, e.peers, nil) { e.log.WithField("peer", reply.From). Debug("leader received heartbeat reply from peer not in our peer list; ignoring") break } peersLastContact[reply.From] = time.Now() case <-heartBeatTicker.C: - for _, peer := range e.GetPeers() { + for _, peer := range e.peers { e.sendHeartBeat(peer, heartBeatReplyCh) } case <-quorumTicker.C: @@ -477,7 +458,7 @@ func (e *node) runLeader() { // If not, we step down as we may have lost connectivity. contacted := 0 now := time.Now() - for _, peer := range e.GetPeers() { + for _, peer := range e.peers { if peer == e.self { contacted++ continue @@ -502,22 +483,22 @@ func (e *node) runLeader() { e.state = FollowerState // Inform the other peers we are stepping down - for _, peer := range e.GetPeers() { + for _, peer := range e.peers { e.sendElectionReset(peer) } } case <-e.shutdownCh: e.state = ShutdownState - if e.IsLeader() { + if e.isLeader() { // Notify all followers we are no longer leader - for _, peer := range e.GetPeers() { + for _, peer := range e.peers { e.sendElectionReset(peer) } } } } e.lastContact = time.Now() - if e.IsLeader() { + if e.isLeader() { e.setLeader("") } } @@ -527,6 +508,8 @@ func (e *node) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { if peer == e.self { return } + // Avoid race by localizing the current term + term := e.currentTerm e.wg.Go(func() { var resp RPCResponse @@ -534,7 +517,7 @@ func (e *node) sendHeartBeat(peer string, heartBeatReplyCh chan HeartBeatResp) { RPC: HeartBeatRPC, Request: HeartBeatReq{ From: e.self, - Term: e.currentTerm, + Term: term, }, } @@ -571,7 +554,6 @@ func (e *node) sendElectionReset(peer string) { } func (e *node) processRPC(rpc RPCRequest) { - e.log.Debugf("RPC: %#v\n", rpc) switch cmd := rpc.Request.(type) { case VoteReq: e.handleVote(rpc, cmd) @@ -581,6 +563,10 @@ func (e *node) processRPC(rpc RPCRequest) { e.handleHeartBeat(rpc, cmd) case ResignReq: e.handleResign(rpc) + case SetPeersReq: + e.handleSetPeers(rpc, cmd) + case GetStateReq: + e.handleGetState(rpc) default: e.log.Errorf("got unexpected command %#v", rpc.Request) rpc.respond(rpc.RPC, nil, "unexpected command") @@ -590,9 +576,10 @@ func (e *node) processRPC(rpc RPCRequest) { // handleResign Notifies all followers that we are stepping down as leader. // if we are leader returns Success = true func (e *node) handleResign(rpc RPCRequest) { + e.log.Debug("RPC: election.ResignReq{}") e.setLeader("") e.state = FollowerState - for _, peer := range e.GetPeers() { + for _, peer := range e.peers { e.sendElectionReset(peer) } rpc.respond(rpc.RPC, ResignReq{}, "") @@ -600,6 +587,7 @@ func (e *node) handleResign(rpc RPCRequest) { // handleResetElection resets our state and starts a new election func (e *node) handleResetElection(rpc RPCRequest) { + e.log.Debug("RPC: election.ResetElectionReq{}") e.setLeader("") e.state = CandidateState rpc.respond(rpc.RPC, ResetElectionResp{}, "") @@ -607,6 +595,7 @@ func (e *node) handleResetElection(rpc RPCRequest) { // handleHeartBeat handles heartbeat requests from the elected leader func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { + e.log.Debugf("RPC: %#v", req) resp := HeartBeatResp{ From: e.self, Term: e.currentTerm, @@ -645,6 +634,7 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // handleVote determines who we will vote for this term func (e *node) handleVote(rpc RPCRequest, req VoteReq) { + e.log.Debugf("RPC: %#v", req) resp := VoteResp{ Term: e.currentTerm, Candidate: e.self, @@ -658,9 +648,8 @@ func (e *node) handleVote(rpc RPCRequest, req VoteReq) { // Check if we have an existing leader (who's not the candidate). Votes are rejected // if there is a known leader. If a leader wants to step down, they notify followers // with the ResetElection RPC call. - leader := e.Leader() - if leader != "" && leader != req.Candidate { - e.log.Debugf("rejecting vote request from '%s' since we have leader '%s'", req.Candidate, leader) + if e.leader != "" && e.leader != req.Candidate { + e.log.Debugf("rejecting vote request from '%s' since we have leader '%s'", req.Candidate, e.leader) return } @@ -673,7 +662,7 @@ func (e *node) handleVote(rpc RPCRequest, req VoteReq) { if req.Term > e.currentTerm { // Ensure transition to follower e.log.Debugf("received a vote request with a newer term '%d'", req.Term) - e.setState(FollowerState) + e.state = FollowerState e.currentTerm = req.Term resp.Term = req.Term } @@ -699,14 +688,46 @@ func (e *node) handleVote(rpc RPCRequest, req VoteReq) { return } +func (e *node) handleSetPeers(rpc RPCRequest, req SetPeersReq) { + e.log.Debugf("RPC: %#v", req) + e.peers = req.Peers + rpc.respond(rpc.RPC, SetPeersResp{}, "") +} + +func (e *node) handleGetState(rpc RPCRequest) { + e.log.Debug("RPC: election.GetStateReq{}") + + rpc.respond(rpc.RPC, GetStateResp{ + Peers: e.peers, + State: e.state.String(), + Leader: e.leader, + }, "") +} + func (e *node) quorumSize() int { - size := len(e.GetPeers()) + size := len(e.peers) if size == 0 { return 1 } return size/2 + 1 } +func (e *node) send(req interface{}) chan RPCResponse { + respCh := make(chan RPCResponse, 1) + + select { + case e.rpcCh <- RPCRequest{ + Request: req, + respChan: respCh, + }: + // Avoid blocking if the rpcCh is full + default: + e.conf.Log.Error("RPC send failed; rpc channel is full") + respCh <- RPCResponse{} + } + return respCh +} + // randomDuration returns a value that is between the minDur and 2x minDur. func randomDuration(minDur time.Duration) time.Duration { return minDur + time.Duration(rand.Int63())%minDur diff --git a/election/election_test.go b/election/election_test.go index ae1ecfc4..b8b350a0 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -99,7 +99,7 @@ func TestSimpleElection(t *testing.T) { if !assert.NotNil(t, candidate) { return } - assert.NotEqual(t, "n0", candidate.Leader()) + assert.NotEqual(t, "n0", candidate.GetLeader()) }) for k, v := range c.Nodes { @@ -121,7 +121,7 @@ func TestLeaderDisconnect(t *testing.T) { if !assert.NotNil(t, node.Node) { return } - assert.NotEqual(t, "n0", node.Node.Leader()) + assert.NotEqual(t, "n0", node.Node.GetLeader()) }) for k, v := range c.Nodes { @@ -138,7 +138,7 @@ func TestFollowerDisconnect(t *testing.T) { defer c.DelNetworkError("n4") // Wait until n4 loses leader - testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + testutil.UntilPass(t, 5, time.Second, func(t testutil.TestingT) { status := c.GetClusterStatus() assert.NotEqual(t, "n0", status["n4"]) }) @@ -251,7 +251,7 @@ func TestOmissionFaults(t *testing.T) { for i := 0; i < 12; i++ { leader := c1.GetLeader() require.NotNil(t, leader) - require.Equal(t, leader.Leader(), "n0") + require.Equal(t, leader.GetLeader(), "n0") time.Sleep(time.Millisecond * 400) } @@ -261,7 +261,7 @@ func TestOmissionFaults(t *testing.T) { for i := 0; i < 12; i++ { leader := c1.GetLeader() require.NotNil(t, leader) - require.Equal(t, leader.Leader(), "n0") + require.Equal(t, leader.GetLeader(), "n0") time.Sleep(time.Millisecond * 400) } } diff --git a/election/rpc.go b/election/rpc.go index 93190d3a..414aaf4c 100644 --- a/election/rpc.go +++ b/election/rpc.go @@ -11,6 +11,8 @@ const ( VoteRPC = RPC("vote") ResetElectionRPC = RPC("reset-election") ResignRPC = RPC("resign") + SetPeersRPC = RPC("set-peers") + GetStateRPC = RPC("get-state") UnknownRPC = RPC("unknown") ) @@ -60,6 +62,18 @@ func (r *RPCResponse) UnmarshalJSON(s []byte) error { return err } r.Response = resp + case SetPeersRPC: + resp := SetPeersResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp + case GetStateRPC: + resp := GetStateResp{} + if err := json.Unmarshal(in.Response, &resp); err != nil { + return err + } + r.Response = resp } return nil } @@ -127,6 +141,18 @@ func (r *RPCRequest) UnmarshalJSON(s []byte) error { return err } r.Request = req + case SetPeersRPC: + req := SetPeersReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req + case GetStateRPC: + req := GetStateReq{} + if err := json.Unmarshal(in.Request, &req); err != nil { + return err + } + r.Request = req } return nil } diff --git a/election/rpc_test.go b/election/rpc_test.go index 330ad791..9ef874a7 100644 --- a/election/rpc_test.go +++ b/election/rpc_test.go @@ -58,15 +58,33 @@ func TestRPCRequest(t *testing.T) { }, out: `{"rpc":"resign","request":{}}`, }, + { + name: "set-peers", + in: election.RPCRequest{ + RPC: election.SetPeersRPC, + Request: election.SetPeersReq{Peers: []string{"n0", "n1"}}, + }, + out: `{"rpc":"set-peers","request":{"peers":["n0","n1"]}}`, + }, + { + name: "get-state", + in: election.RPCRequest{ + RPC: election.GetStateRPC, + Request: election.GetStateReq{}, + }, + out: `{"rpc":"get-state","request":{}}`, + }, } { - b, err := json.Marshal(tt.in) - require.NoError(t, err) - assert.Equal(t, tt.out, string(b)) + t.Run(tt.name, func(t *testing.T) { + b, err := json.Marshal(tt.in) + require.NoError(t, err) + assert.Equal(t, tt.out, string(b)) - var in election.RPCRequest - err = json.Unmarshal(b, &in) - require.NoError(t, err) - assert.Equal(t, tt.in, in) + var in election.RPCRequest + err = json.Unmarshal(b, &in) + require.NoError(t, err) + assert.Equal(t, tt.in, in) + }) } } @@ -118,6 +136,26 @@ func TestRPCResponse(t *testing.T) { }, out: `{"rpc":"resign","response":{"success":true}}`, }, + { + name: "set-peers", + in: election.RPCResponse{ + RPC: election.SetPeersRPC, + Response: election.SetPeersResp{}, + }, + out: `{"rpc":"set-peers","response":{}}`, + }, + { + name: "get-state", + in: election.RPCResponse{ + RPC: election.GetStateRPC, + Response: election.GetStateResp{ + Leader: "n0", + Peers: []string{"n0", "n1"}, + State: "follower", + }, + }, + out: `{"rpc":"get-state","response":{"leader":"n0","state":"follower","peers":["n0","n1"]}}`, + }, } { t.Run(tt.name, func(t *testing.T) { b, err := json.Marshal(tt.in) diff --git a/election/structs.pb.go b/election/structs.pb.go index 7f653c54..0a738540 100644 --- a/election/structs.pb.go +++ b/election/structs.pb.go @@ -16,6 +16,10 @@ It has these top-level messages: HeartBeatResp VoteResp VoteReq + SetPeersReq + SetPeersResp + GetStateReq + GetStateResp */ package election @@ -194,6 +198,73 @@ func (m *VoteReq) GetTerm() uint64 { return 0 } +// Set the peers this node will consider during the election +type SetPeersReq struct { + // A list of peers + Peers []string `protobuf:"bytes,1,rep,name=peers" json:"peers,omitempty"` +} + +func (m *SetPeersReq) Reset() { *m = SetPeersReq{} } +func (m *SetPeersReq) String() string { return proto.CompactTextString(m) } +func (*SetPeersReq) ProtoMessage() {} +func (*SetPeersReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } + +func (m *SetPeersReq) GetPeers() []string { + if m != nil { + return m.Peers + } + return nil +} + +type SetPeersResp struct { +} + +func (m *SetPeersResp) Reset() { *m = SetPeersResp{} } +func (m *SetPeersResp) String() string { return proto.CompactTextString(m) } +func (*SetPeersResp) ProtoMessage() {} +func (*SetPeersResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } + +// Get the current state of the node +type GetStateReq struct { +} + +func (m *GetStateReq) Reset() { *m = GetStateReq{} } +func (m *GetStateReq) String() string { return proto.CompactTextString(m) } +func (*GetStateReq) ProtoMessage() {} +func (*GetStateReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } + +type GetStateResp struct { + Leader string `protobuf:"bytes,1,opt,name=leader" json:"leader,omitempty"` + State string `protobuf:"bytes,2,opt,name=state" json:"state,omitempty"` + Peers []string `protobuf:"bytes,3,rep,name=peers" json:"peers,omitempty"` +} + +func (m *GetStateResp) Reset() { *m = GetStateResp{} } +func (m *GetStateResp) String() string { return proto.CompactTextString(m) } +func (*GetStateResp) ProtoMessage() {} +func (*GetStateResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } + +func (m *GetStateResp) GetLeader() string { + if m != nil { + return m.Leader + } + return "" +} + +func (m *GetStateResp) GetState() string { + if m != nil { + return m.State + } + return "" +} + +func (m *GetStateResp) GetPeers() []string { + if m != nil { + return m.Peers + } + return nil +} + func init() { proto.RegisterType((*ResetElectionReq)(nil), "ResetElectionReq") proto.RegisterType((*ResetElectionResp)(nil), "ResetElectionResp") @@ -203,24 +274,32 @@ func init() { proto.RegisterType((*HeartBeatResp)(nil), "HeartBeatResp") proto.RegisterType((*VoteResp)(nil), "VoteResp") proto.RegisterType((*VoteReq)(nil), "VoteReq") + proto.RegisterType((*SetPeersReq)(nil), "SetPeersReq") + proto.RegisterType((*SetPeersResp)(nil), "SetPeersResp") + proto.RegisterType((*GetStateReq)(nil), "GetStateReq") + proto.RegisterType((*GetStateResp)(nil), "GetStateResp") } func init() { proto.RegisterFile("structs.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 216 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x90, 0x31, 0x4b, 0xc0, 0x30, - 0x10, 0x85, 0xa9, 0x16, 0xdb, 0x9e, 0x16, 0x34, 0x2e, 0x19, 0x1c, 0x4a, 0x06, 0xe9, 0xe4, 0x22, - 0xe8, 0xe0, 0x56, 0x10, 0x9c, 0x33, 0x74, 0x70, 0x8b, 0xe9, 0x59, 0x0a, 0xb6, 0x49, 0x73, 0xd7, - 0xff, 0x2f, 0x4d, 0x5b, 0x14, 0x71, 0xd0, 0xed, 0xde, 0xe3, 0xbe, 0x07, 0xef, 0x41, 0x49, 0x1c, - 0x16, 0xcb, 0x74, 0xe7, 0x83, 0x63, 0xa7, 0x04, 0x5c, 0x6a, 0x24, 0xe4, 0xe7, 0x0f, 0xb4, 0x3c, - 0xb8, 0x49, 0xe3, 0xac, 0xae, 0xe1, 0xea, 0x87, 0x47, 0x5e, 0x9d, 0x43, 0xa1, 0x91, 0x86, 0x3e, - 0x7e, 0xdc, 0x02, 0x1c, 0x82, 0xbc, 0x90, 0x90, 0xd1, 0x62, 0x2d, 0x12, 0xc9, 0xa4, 0x4a, 0xea, - 0x5c, 0x1f, 0x52, 0x3d, 0xc0, 0xc5, 0x0b, 0x9a, 0xc0, 0x0d, 0x1a, 0xd6, 0x38, 0x0b, 0x01, 0xe9, - 0x7b, 0x70, 0x63, 0x7c, 0x2b, 0x74, 0xbc, 0x57, 0x8f, 0x31, 0x8c, 0xf2, 0xa4, 0x4a, 0xea, 0x54, - 0xc7, 0x5b, 0x3d, 0x42, 0xf9, 0x8d, 0x23, 0xff, 0x67, 0xb0, 0x85, 0xbc, 0x75, 0x8c, 0x91, 0xb9, - 0x81, 0xc2, 0x9a, 0xa9, 0x1b, 0x3a, 0xc3, 0xb8, 0x83, 0x5f, 0xc6, 0x6f, 0xf4, 0x5a, 0xa4, 0x0f, - 0x66, 0x62, 0xec, 0xe4, 0xe9, 0x56, 0x64, 0x97, 0xea, 0x09, 0xb2, 0x2d, 0x77, 0xfe, 0x7f, 0x6c, - 0x03, 0xaf, 0x39, 0xee, 0x53, 0xbe, 0x9d, 0xc5, 0xd9, 0xef, 0x3f, 0x03, 0x00, 0x00, 0xff, 0xff, - 0xe1, 0x04, 0x54, 0x6d, 0x87, 0x01, 0x00, 0x00, + // 287 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x92, 0x41, 0x4b, 0xc3, 0x40, + 0x10, 0x85, 0x89, 0xa9, 0x6d, 0x32, 0x49, 0x44, 0xa3, 0x48, 0x0e, 0x1e, 0xc2, 0x0a, 0x92, 0x93, + 0x17, 0x41, 0x0f, 0xde, 0x0a, 0xa2, 0x47, 0xd9, 0x42, 0x0f, 0xde, 0xd6, 0x64, 0x2c, 0x81, 0x36, + 0xd9, 0xec, 0x4c, 0xff, 0xbf, 0xec, 0x66, 0x4b, 0x8a, 0x78, 0xd0, 0xdb, 0x7c, 0xc3, 0x7b, 0x6f, + 0x66, 0xd8, 0x85, 0x8c, 0xd8, 0xec, 0x6b, 0xa6, 0x7b, 0x6d, 0x7a, 0xee, 0x45, 0x0e, 0xe7, 0x12, + 0x09, 0xf9, 0x65, 0x8b, 0x35, 0xb7, 0x7d, 0x27, 0x71, 0x10, 0x97, 0x70, 0xf1, 0xa3, 0x47, 0x5a, + 0x24, 0x10, 0x4b, 0xa4, 0x76, 0xe3, 0x14, 0x77, 0x00, 0x07, 0x20, 0x9d, 0x17, 0xb0, 0xa0, 0x7d, + 0x5d, 0x23, 0x51, 0x11, 0x94, 0x41, 0x15, 0xc9, 0x03, 0x8a, 0x47, 0x48, 0xdf, 0x50, 0x19, 0x5e, + 0xa2, 0x62, 0x89, 0x43, 0x9e, 0xc3, 0xec, 0xcb, 0xf4, 0x3b, 0x27, 0x8b, 0xa5, 0xab, 0x6d, 0x8f, + 0xd1, 0xec, 0x8a, 0x93, 0x32, 0xa8, 0x66, 0xd2, 0xd5, 0xe2, 0x09, 0xb2, 0x23, 0x1f, 0xe9, 0x3f, + 0x1b, 0xd7, 0x10, 0xad, 0x7b, 0x46, 0xe7, 0xb9, 0x81, 0xb8, 0x56, 0x5d, 0xd3, 0x36, 0x8a, 0xd1, + 0x1b, 0xa7, 0xc6, 0x6f, 0x6e, 0x7b, 0xc8, 0xc6, 0xa8, 0x8e, 0xb1, 0x29, 0xc2, 0xf1, 0x10, 0x8f, + 0xe2, 0x19, 0x16, 0x63, 0xee, 0xf0, 0xff, 0x58, 0x71, 0x0b, 0xc9, 0x0a, 0xf9, 0x1d, 0xd1, 0x90, + 0x0d, 0xb8, 0x82, 0x53, 0x6d, 0xeb, 0x22, 0x28, 0xc3, 0x2a, 0x96, 0x23, 0x88, 0x33, 0x48, 0x27, + 0x11, 0x69, 0x91, 0x41, 0xf2, 0x8a, 0xbc, 0x62, 0xe5, 0xa6, 0x0a, 0x09, 0xe9, 0x84, 0xa4, 0xf3, + 0x6b, 0x98, 0x6f, 0x51, 0x35, 0x68, 0xfc, 0x0a, 0x9e, 0x6c, 0x38, 0x59, 0x91, 0x5b, 0x20, 0x96, + 0x23, 0x4c, 0x23, 0xc3, 0xa3, 0x91, 0x4b, 0xf8, 0x88, 0xd0, 0x3f, 0xf1, 0xe7, 0xdc, 0x7d, 0x87, + 0x87, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc7, 0xdf, 0xef, 0x28, 0x1f, 0x02, 0x00, 0x00, } diff --git a/election/structs.proto b/election/structs.proto index c212641b..5f4055a3 100644 --- a/election/structs.proto +++ b/election/structs.proto @@ -46,3 +46,20 @@ message VoteReq { uint64 term = 2; } +// Set the peers this node will consider during the election. +// This is typically set by calling Node.SetPeers() but the +// implementor could choose to allow a remote entity to set the +// peer list via Node.ReceiveRPC(). This can be blocked by refusing +// to accept election.SetPeersRPC calls. +message SetPeersReq { + repeated string peers = 1; +} +message SetPeersResp { } + +// Get the current state of the node +message GetStateReq {} +message GetStateResp { + string leader = 1; + string state = 2; + repeated string peers = 3; +} diff --git a/syncutil/broadcast_test.go b/syncutil/broadcast_test.go index e2b53c9c..058a437a 100644 --- a/syncutil/broadcast_test.go +++ b/syncutil/broadcast_test.go @@ -40,7 +40,7 @@ func TestBroadcast(t *testing.T) { // Wait for more chats to be added to chat[] select { - case <-broadcaster.WaitChan(string(idx)): + case <-broadcaster.WaitChan(fmt.Sprint(idx)): case <-done: return } From 077ca962689c121b7027c36dcd1b9a630a2c8cb7 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Mon, 22 Feb 2021 17:24:50 -0600 Subject: [PATCH 10/15] Separated SpawnNode into NewNode and Start to avoid race during init of RPC calls --- cmd/election/main.go | 17 +++++-- discovery/memberlist.go | 2 + election/cluster_test.go | 6 ++- election/election.go | 101 ++++++++++++++++++++++++++++++-------- election/example_test.go | 43 +++++++++++++--- etcdutil/election.go | 4 +- etcdutil/election_test.go | 4 +- 7 files changed, 138 insertions(+), 39 deletions(-) diff --git a/cmd/election/main.go b/cmd/election/main.go index a5a630f4..33895d81 100644 --- a/cmd/election/main.go +++ b/cmd/election/main.go @@ -82,7 +82,7 @@ func main() { electionAddr, memberListAddr, knownAddr := os.Args[1], os.Args[2], os.Args[3] //logrus.SetLevel(logrus.DebugLevel) - node, err := election.SpawnNode(election.Config{ + node, err := election.NewNode(election.Config{ // A unique identifier used to identify us in a list of peers UniqueID: electionAddr, // Called whenever the library detects a change in leadership @@ -110,9 +110,7 @@ func main() { result = append(result, string(p.Metadata)) } logrus.Infof("Update Peers: %s", result) - if err := node.SetPeers(result); err != nil { - logrus.Fatal(err) - } + node.SetPeers(result) }, }) if err != nil { @@ -125,6 +123,15 @@ func main() { logrus.Fatal(http.ListenAndServe(electionAddr, mux)) }() + // Wait until the http server is up and can receive RPC requests + if err := election.WaitForConnect(electionAddr, 10, time.Millisecond*100); err != nil { + logrus.Fatal(err) + } + + // Now that our http handler is listening for requests we + // can safely start the election. + node.Start() + // Wait here for signals to clean up our mess c := make(chan os.Signal, 1) signal.Notify(c, os.Interrupt, syscall.SIGTERM) @@ -134,7 +141,7 @@ func main() { logrus.WithError(err).Error("during member list catalog close") } cancel() - node.Close() + node.Stop() os.Exit(0) } } diff --git a/discovery/memberlist.go b/discovery/memberlist.go index 7872c74b..ac3c9a00 100644 --- a/discovery/memberlist.go +++ b/discovery/memberlist.go @@ -126,6 +126,8 @@ func (m *memberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { return nil, errors.Wrap(err, "LivelinessAddress=`%s` is invalid;") } + m.conf.Logger.Infof("BindAddr: %s Port: %d", config.BindAddr, config.BindPort) + m.conf.Logger.Infof("AdvAddr: %s Port: %d", config.AdvertiseAddr, config.AdvertisePort) config.Delegate = &delegate{meta: conf.Peer.Metadata} config.Events = m.events return config, nil diff --git a/election/cluster_test.go b/election/cluster_test.go index d67d9f27..c03ae4f6 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -56,11 +56,13 @@ func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { } } var err error - n.Node, err = election.SpawnNode(*conf) + n.Node, err = election.NewNode(*conf) if err != nil { return err } + // Add the node to our list of nodes c.Add(name, n) + n.Node.Start() return nil } @@ -170,6 +172,6 @@ func (c *TestCluster) sendRPC(from string, to string, req election.RPCRequest, r func (c *TestCluster) Close() { for _, v := range c.Nodes { - v.Node.Close() + v.Node.Stop() } } diff --git a/election/election.go b/election/election.go index f1800dfd..9a5f7430 100644 --- a/election/election.go +++ b/election/election.go @@ -3,7 +3,10 @@ package election import ( "context" "errors" + "fmt" "math/rand" + "net" + "sync/atomic" "time" "github.com/mailgun/holster/v3/setter" @@ -84,9 +87,15 @@ type Config struct { type OnUpdate func(string) type Node interface { - // Set the list of peers to be considered for the election, this list MUST - // include ourself as defined by `Config.UniqueID`. - SetPeers([]string) error + // Starts the main election loop. + Start() + + // Cancels the election, resigns if we are leader and waits for all go + // routines to complete before returning. + Stop() + + // Set the list of peers to be considered for the election + SetPeers([]string) // If leader, resigns as leader and starts a new election that we will not // participate in. @@ -103,10 +112,6 @@ type Node interface { // Called when this peer receives a RPC request from a peer ReceiveRPC(RPCRequest, *RPCResponse) - - // Cancels the election, resigns if we are leader and waits for all go - // routines to complete before returning. - Close() } type node struct { @@ -126,10 +131,11 @@ type node struct { shutdownCh chan struct{} // Signals we are in shutdown log logrus.FieldLogger wg syncutil.WaitGroup + running int64 } -// Spawns a new node that will participate in the election. -func SpawnNode(conf Config) (Node, error) { +// Creates a new node. You must call Start() to be participate in the election. +func NewNode(conf Config) (Node, error) { if conf.UniqueID == "" { return nil, errors.New("refusing to spawn a new node with no Config.UniqueID defined") } @@ -141,14 +147,13 @@ func SpawnNode(conf Config) (Node, error) { setter.SetDefault(&conf.NetworkTimeout, time.Second*3) c := &node{ - shutdownCh: make(chan struct{}), - rpcCh: make(chan RPCRequest, 5_000), - self: conf.UniqueID, - conf: conf, - log: conf.Log, + rpcCh: make(chan RPCRequest, 5_000), + self: conf.UniqueID, + peers: conf.Peers, + conf: conf, + log: conf.Log, } - c.wg.Go(c.run) - return c, c.SetPeers(conf.Peers) + return c, nil } // Called by the implementer when an RPC is received from another node @@ -165,9 +170,15 @@ func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { // SetPeers is a thread safe way to dynamically add or remove peers in a running cluster. // These peers will be contacted when requesting votes during leader election. -func (e *node) SetPeers(peers []string) error { - _ = <-e.send(SetPeersReq{Peers: peers}) - return nil +func (e *node) SetPeers(peers []string) { + + // If the main loop is not running, there is no risk of race + if atomic.LoadInt64(&e.running) != 1 { + e.peers = peers + return + } + + e.send(SetPeersReq{Peers: peers}) } // GetPeers returns the current peers this node knows about. @@ -177,6 +188,16 @@ func (e *node) GetPeers() []string { // GetState returns the current state of this node func (e *node) GetState() NodeState { + + // If the main loop is not running, there is no risk of race + if atomic.LoadInt64(&e.running) != 1 { + return NodeState{ + Peers: e.peers, + State: e.state.String(), + Leader: e.leader, + } + } + select { case resp := <-e.send(GetStateReq{}): if s, ok := resp.Response.(GetStateResp); ok { @@ -217,6 +238,12 @@ func (e *node) getLeader() string { // Resign will cause this node to step down as leader, if this // node is NOT leader, this does nothing and returns 'false' func (e *node) Resign() bool { + + // Avoid blocking if main loop is not running + if atomic.LoadInt64(&e.running) != 1 { + return false + } + select { case rpcResp := <-e.send(ResignReq{}): resp, ok := rpcResp.Response.(ResignResp) @@ -232,9 +259,24 @@ func (e *node) Resign() bool { } } -// Close closes all internal go routines and if this node is currently +// Start the main event loop which allows the election to proceed. +// Call this method when the node is ready to be considered in the election. +func (e *node) Start() { + if atomic.LoadInt64(&e.running) == 1 { + return + } + atomic.StoreInt64(&e.running, 1) + e.shutdownCh = make(chan struct{}) + e.wg.Go(e.run) +} + +// Stop stops all internal go routines and if this node is currently // leader, resigns as leader. -func (e *node) Close() { +func (e *node) Stop() { + if atomic.LoadInt64(&e.running) != 1 { + return + } + atomic.StoreInt64(&e.running, 0) close(e.shutdownCh) e.wg.Wait() } @@ -732,3 +774,20 @@ func (e *node) send(req interface{}) chan RPCResponse { func randomDuration(minDur time.Duration) time.Duration { return minDur + time.Duration(rand.Int63())%minDur } + +// WaitForConnect waits for the specified address to accept connections then returns nil. +// Returns an error if all attempts have been exhausted. +func WaitForConnect(address string, attempts int, interval time.Duration) error { + var err error + var conn net.Conn + for i := 0; i < attempts; i++ { + conn, err = net.Dial("tcp", address) + if err != nil { + continue + } + conn.Close() + time.Sleep(interval) + return nil + } + return fmt.Errorf("while connecting to '%s' - '%s' after %d attempts", address, err, attempts) +} diff --git a/election/example_test.go b/election/example_test.go index 53fe123b..ed8001fb 100644 --- a/election/example_test.go +++ b/election/example_test.go @@ -7,7 +7,11 @@ import ( "fmt" "log" "net/http" + "os" + "os/signal" + "syscall" "testing" + "time" "github.com/mailgun/holster/v3/election" "github.com/pkg/errors" @@ -67,7 +71,7 @@ func newHandler(node election.Node) func(w http.ResponseWriter, r *http.Request) func SimpleExample(t *testing.T) { logrus.SetLevel(logrus.DebugLevel) - node1, err := election.SpawnNode(election.Config{ + node1, err := election.NewNode(election.Config{ // A list of known peers at startup Peers: []string{"localhost:7080", "localhost:7081"}, // A unique identifier used to identify us in a list of peers @@ -82,9 +86,9 @@ func SimpleExample(t *testing.T) { if err != nil { log.Fatal(err) } - defer node1.Close() + defer node1.Stop() - node2, err := election.SpawnNode(election.Config{ + node2, err := election.NewNode(election.Config{ Peers: []string{"localhost:7080", "localhost:7081"}, UniqueID: "localhost:7081", SendRPC: sendRPC, @@ -92,7 +96,7 @@ func SimpleExample(t *testing.T) { if err != nil { log.Fatal(err) } - defer node2.Close() + defer node2.Stop() go func() { mux := http.NewServeMux() @@ -100,7 +104,32 @@ func SimpleExample(t *testing.T) { log.Fatal(http.ListenAndServe(":7080", mux)) }() - mux := http.NewServeMux() - mux.HandleFunc("/rpc", newHandler(node2)) - log.Fatal(http.ListenAndServe(":7081", mux)) + go func() { + mux := http.NewServeMux() + mux.HandleFunc("/rpc", newHandler(node2)) + log.Fatal(http.ListenAndServe(":7081", mux)) + }() + + // Wait for each of the http listeners to start fielding requests + if err := election.WaitForConnect("localhost:7080", 3, time.Second); err != nil { + log.Fatal(err) + } + + if err := election.WaitForConnect("localhost:7081", 3, time.Second); err != nil { + log.Fatal(err) + } + + // Now that both http handlers are listening for requests we + // can safely start the election. + node1.Start() + node2.Start() + + // Wait here for signals to clean up our mess + c := make(chan os.Signal, 1) + signal.Notify(c, os.Interrupt, syscall.SIGTERM) + for range c { + node1.Stop() + node2.Stop() + os.Exit(0) + } } diff --git a/etcdutil/election.go b/etcdutil/election.go index 0c7c0464..ef8939e7 100644 --- a/etcdutil/election.go +++ b/etcdutil/election.go @@ -96,7 +96,7 @@ type ElectionConfig struct { // // // Concede the election if leader and cancel our candidacy // // for the election. -// election.Close() +// election.Stop() // func NewElection(ctx context.Context, client *etcd.Client, conf ElectionConfig) (*Election, error) { var initialElectionErr error @@ -145,7 +145,7 @@ func NewElection(ctx context.Context, client *etcd.Client, conf ElectionConfig) // }) // // // Cancels the election and concedes the election if we are leader. -// election.Close() +// election.Stop() // func NewElectionAsync(client *etcd.Client, conf ElectionConfig) *Election { setter.SetDefault(&conf.Election, "null") diff --git a/etcdutil/election_test.go b/etcdutil/election_test.go index 98367569..e6781354 100644 --- a/etcdutil/election_test.go +++ b/etcdutil/election_test.go @@ -225,7 +225,7 @@ func (s *ElectionsSuite) TestLostLeaderStop() { s.assertElectionClosed(ch0, 3*clock.Second) } -// FIXME: This test gets stuck on e0.Close(). +// FIXME: This test gets stuck on e0.Stop(). //// If Etcd is down on start the candidate keeps trying to connect. //func (s *ElectionsSuite) TestEtcdDownOnStart() { // s.toxiProxies[0].Stop() @@ -237,7 +237,7 @@ func (s *ElectionsSuite) TestLostLeaderStop() { // // // Then // s.assertElectionWinner(ch0, 3*clock.Second) -// e0.Close() +// e0.Stop() //} // If provided etcd endpoint candidate keeps trying to connect until it is From 17862cbe7e75f29bce50c958723f31bca2c67e2e Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Tue, 23 Feb 2021 20:49:03 -0600 Subject: [PATCH 11/15] Added context.Context to most election methods --- cmd/election/main.go | 6 +-- election/cluster_test.go | 6 +-- election/election.go | 105 ++++++++++++++++++++++++-------------- election/election_test.go | 3 +- election/example_test.go | 11 ++-- 5 files changed, 81 insertions(+), 50 deletions(-) diff --git a/cmd/election/main.go b/cmd/election/main.go index 33895d81..75abff3e 100644 --- a/cmd/election/main.go +++ b/cmd/election/main.go @@ -110,7 +110,7 @@ func main() { result = append(result, string(p.Metadata)) } logrus.Infof("Update Peers: %s", result) - node.SetPeers(result) + node.SetPeers(context.Background(), result) }, }) if err != nil { @@ -130,7 +130,7 @@ func main() { // Now that our http handler is listening for requests we // can safely start the election. - node.Start() + node.Start(context.Background()) // Wait here for signals to clean up our mess c := make(chan os.Signal, 1) @@ -141,7 +141,7 @@ func main() { logrus.WithError(err).Error("during member list catalog close") } cancel() - node.Stop() + node.Stop(context.Background()) os.Exit(0) } } diff --git a/election/cluster_test.go b/election/cluster_test.go index c03ae4f6..8b846e36 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -62,7 +62,7 @@ func (c *TestCluster) SpawnNode(name string, conf *election.Config) error { } // Add the node to our list of nodes c.Add(name, n) - n.Node.Start() + n.Node.Start(context.Background()) return nil } @@ -93,7 +93,7 @@ func (c *TestCluster) updatePeers() { // Update our list of known peers for _, v := range c.Nodes { - v.Node.SetPeers(peers) + v.Node.SetPeers(context.Background(), peers) } } @@ -172,6 +172,6 @@ func (c *TestCluster) sendRPC(from string, to string, req election.RPCRequest, r func (c *TestCluster) Close() { for _, v := range c.Nodes { - v.Node.Stop() + v.Node.Stop(context.Background()) } } diff --git a/election/election.go b/election/election.go index 9a5f7430..710c4a41 100644 --- a/election/election.go +++ b/election/election.go @@ -33,6 +33,8 @@ const ( ShutdownState ) +var ErrNotLeader = errors.New("not the leader") + func (s state) String() string { switch s { case FollowerState: @@ -88,27 +90,29 @@ type OnUpdate func(string) type Node interface { // Starts the main election loop. - Start() + Start(ctx context.Context) error // Cancels the election, resigns if we are leader and waits for all go // routines to complete before returning. - Stop() + Stop(ctx context.Context) error // Set the list of peers to be considered for the election - SetPeers([]string) + SetPeers(ctx context.Context, peers []string) error // If leader, resigns as leader and starts a new election that we will not - // participate in. - Resign() bool + // participate in. returns ErrNotLeader if not currently the leader + Resign(ctx context.Context) error - // Returns true if we are currently leader + // IsLeader is a convenience function that calls GetState() and returns true + // if this node was elected leader. May block if main loop is occupied. IsLeader() bool - // Returns the current leader + // GetLeader is a convenience function that calls GetState() returns the + // unique id of the node that is currently leader. May block if main loop is occupied. GetLeader() string // Returns the current state of this node - GetState() NodeState + GetState(ctx context.Context) (NodeState, error) // Called when this peer receives a RPC request from a peer ReceiveRPC(RPCRequest, *RPCResponse) @@ -158,6 +162,13 @@ func NewNode(conf Config) (Node, error) { // Called by the implementer when an RPC is received from another node func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { + // Ignore requests received when we are not running. If + // we don't we can create a race when initializing e.shutdownCh + // and we could fill up the rpcCh with requests that are never handled + if atomic.LoadInt64(&e.running) != 1 { + return + } + req.respChan = make(chan RPCResponse, 1) e.rpcCh <- req @@ -170,24 +181,24 @@ func (e *node) ReceiveRPC(req RPCRequest, resp *RPCResponse) { // SetPeers is a thread safe way to dynamically add or remove peers in a running cluster. // These peers will be contacted when requesting votes during leader election. -func (e *node) SetPeers(peers []string) { +func (e *node) SetPeers(ctx context.Context, peers []string) error { // If the main loop is not running, there is no risk of race if atomic.LoadInt64(&e.running) != 1 { e.peers = peers - return + return nil } - e.send(SetPeersReq{Peers: peers}) -} - -// GetPeers returns the current peers this node knows about. -func (e *node) GetPeers() []string { - return e.GetState().Peers + select { + case <-e.send(SetPeersReq{Peers: peers}): + return nil + case <-ctx.Done(): + return ctx.Err() + } } // GetState returns the current state of this node -func (e *node) GetState() NodeState { +func (e *node) GetState(ctx context.Context) (NodeState, error) { // If the main loop is not running, there is no risk of race if atomic.LoadInt64(&e.running) != 1 { @@ -195,26 +206,32 @@ func (e *node) GetState() NodeState { Peers: e.peers, State: e.state.String(), Leader: e.leader, - } + }, nil } select { case resp := <-e.send(GetStateReq{}): if s, ok := resp.Response.(GetStateResp); ok { - return NodeState(s) + return NodeState(s), nil } + case <-ctx.Done(): + return NodeState{}, ctx.Err() } - return NodeState{} + return NodeState{}, nil } -// IsLeader returns true if this node was elected leader +// IsLeader is a convenience function that calls GetState() and returns true +// if this node was elected leader. May block if main loop is occupied. func (e *node) IsLeader() bool { - return e.self == e.GetState().Leader + s, _ := e.GetState(context.Background()) + return e.self == s.Leader } -// Leader returns the name of the node that is currently leader +// GetLeader is a convenience function that calls GetState() returns the +// unique id of the node that is currently leader. May block if main loop is occupied. func (e *node) GetLeader() string { - return e.GetState().Leader + s, _ := e.GetState(context.Background()) + return s.Leader } func (e *node) isLeader() bool { @@ -236,49 +253,63 @@ func (e *node) getLeader() string { } // Resign will cause this node to step down as leader, if this -// node is NOT leader, this does nothing and returns 'false' -func (e *node) Resign() bool { +// node is NOT leader, this does nothing and returns ErrNotLeader +func (e *node) Resign(ctx context.Context) error { // Avoid blocking if main loop is not running if atomic.LoadInt64(&e.running) != 1 { - return false + return ErrNotLeader } select { case rpcResp := <-e.send(ResignReq{}): resp, ok := rpcResp.Response.(ResignResp) if !ok { - return false + return errors.New("resign response channel closed") } if rpcResp.Error != "" { - return false + return errors.New(rpcResp.Error) + } + if resp.Success { + return nil } - return resp.Success + return ErrNotLeader case <-e.shutdownCh: - return false + return nil + case <-ctx.Done(): + return ctx.Err() } } // Start the main event loop which allows the election to proceed. // Call this method when the node is ready to be considered in the election. -func (e *node) Start() { +func (e *node) Start(ctx context.Context) error { if atomic.LoadInt64(&e.running) == 1 { - return + return nil } - atomic.StoreInt64(&e.running, 1) e.shutdownCh = make(chan struct{}) + atomic.StoreInt64(&e.running, 1) e.wg.Go(e.run) + return nil } // Stop stops all internal go routines and if this node is currently // leader, resigns as leader. -func (e *node) Stop() { +func (e *node) Stop(ctx context.Context) error { if atomic.LoadInt64(&e.running) != 1 { - return + return nil } atomic.StoreInt64(&e.running, 0) close(e.shutdownCh) - e.wg.Wait() + done := make(chan struct{}) + go func() { + e.wg.Wait() + close(done) + }() + select { + case <-done: + return ctx.Err() + } } // Main thread loop diff --git a/election/election_test.go b/election/election_test.go index b8b350a0..ffe7aedb 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -1,6 +1,7 @@ package election_test import ( + "context" "errors" "testing" "time" @@ -91,7 +92,7 @@ func TestSimpleElection(t *testing.T) { createCluster(t, c) defer c.Close() - c.Nodes["n0"].Node.Resign() + c.Nodes["n0"].Node.Resign(context.Background()) // Wait until n0 is no longer leader testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { diff --git a/election/example_test.go b/election/example_test.go index ed8001fb..78a0109b 100644 --- a/election/example_test.go +++ b/election/example_test.go @@ -86,7 +86,7 @@ func SimpleExample(t *testing.T) { if err != nil { log.Fatal(err) } - defer node1.Stop() + defer node1.Stop(context.Background()) node2, err := election.NewNode(election.Config{ Peers: []string{"localhost:7080", "localhost:7081"}, @@ -96,7 +96,6 @@ func SimpleExample(t *testing.T) { if err != nil { log.Fatal(err) } - defer node2.Stop() go func() { mux := http.NewServeMux() @@ -121,15 +120,15 @@ func SimpleExample(t *testing.T) { // Now that both http handlers are listening for requests we // can safely start the election. - node1.Start() - node2.Start() + node1.Start(context.Background()) + node2.Start(context.Background()) // Wait here for signals to clean up our mess c := make(chan os.Signal, 1) signal.Notify(c, os.Interrupt, syscall.SIGTERM) for range c { - node1.Stop() - node2.Stop() + node1.Stop(context.Background()) + node2.Stop(context.Background()) os.Exit(0) } } From d92a0d07ceeccf8cc9ee18c490717b13e4e5d76c Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Fri, 26 Feb 2021 17:14:31 -0600 Subject: [PATCH 12/15] Added udp package with a Proxy --- discovery/memberlist.go | 5 ++- udp/client.go | 44 ++++++++++++++++++ udp/proxy.go | 95 +++++++++++++++++++++++++++++++++++++++ udp/proxy_test.go | 99 +++++++++++++++++++++++++++++++++++++++++ udp/server.go | 62 ++++++++++++++++++++++++++ 5 files changed, 303 insertions(+), 2 deletions(-) create mode 100644 udp/client.go create mode 100644 udp/proxy.go create mode 100644 udp/proxy_test.go create mode 100644 udp/server.go diff --git a/discovery/memberlist.go b/discovery/memberlist.go index ac3c9a00..0f5519ea 100644 --- a/discovery/memberlist.go +++ b/discovery/memberlist.go @@ -126,8 +126,8 @@ func (m *memberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { return nil, errors.Wrap(err, "LivelinessAddress=`%s` is invalid;") } - m.conf.Logger.Infof("BindAddr: %s Port: %d", config.BindAddr, config.BindPort) - m.conf.Logger.Infof("AdvAddr: %s Port: %d", config.AdvertiseAddr, config.AdvertisePort) + m.conf.Logger.Debugf("BindAddr: %s Port: %d", config.BindAddr, config.BindPort) + m.conf.Logger.Debugf("AdvAddr: %s Port: %d", config.AdvertiseAddr, config.AdvertisePort) config.Delegate = &delegate{meta: conf.Peer.Metadata} config.Events = m.events return config, nil @@ -149,6 +149,7 @@ func (m *memberList) Close(ctx context.Context) error { case err := <-errCh: return err } + return nil } func (m *memberList) GetPeers(_ context.Context) ([]Peer, error) { diff --git a/udp/client.go b/udp/client.go new file mode 100644 index 00000000..33a7c615 --- /dev/null +++ b/udp/client.go @@ -0,0 +1,44 @@ +package udp + +import ( + "fmt" + "net" + "time" +) + +type Client struct { + conn *net.UDPConn + addr net.Addr +} + +func NewClient(address string) (*Client, error) { + addr, err := net.ResolveUDPAddr("udp", address) + if err != nil { + return nil, fmt.Errorf("while resolving Upstream address: %w", err) + } + + conn, err := net.DialUDP("udp", nil, addr) + if err != nil { + return nil, fmt.Errorf("while dialing udp '%s' - %w", addr.String(), err) + } + return &Client{ + conn: conn, + addr: addr, + }, nil +} + +func (c *Client) Recv(b []byte, timeout time.Duration) (int, net.Addr, error) { + deadline := time.Now().Add(timeout) + if err := c.conn.SetReadDeadline(deadline); err != nil { + return 0, nil, err + } + return c.conn.ReadFrom(b) +} + +func (c *Client) Send(b []byte) (int, error) { + return c.conn.Write(b) +} + +func (c *Client) Close() { + c.conn.Close() +} diff --git a/udp/proxy.go b/udp/proxy.go new file mode 100644 index 00000000..3bf9576c --- /dev/null +++ b/udp/proxy.go @@ -0,0 +1,95 @@ +package udp + +import ( + "errors" + "fmt" + "net" + "sync" + "time" + + "github.com/sirupsen/logrus" +) + +type Proxy struct { + conf ProxyConfig + listen *Server + upstream *Client + mutex sync.Mutex // ensure the handler isn't still executing when start()/stop() are called +} + +type ProxyConfig struct { + // The address:port we should bind too + Listen string + // The address:port of the upstream udp server we should forward requests too + Upstream string + // How long we should wait for a response from the upstream udp server + UpstreamTimeout time.Duration +} + +func NewProxy(conf ProxyConfig) *Proxy { + return &Proxy{ + conf: conf, + } +} + +func (p *Proxy) Start() error { + p.mutex.Lock() + defer p.mutex.Unlock() + + if p.conf.Listen == "" { + return errors.New("variable Listen cannot be empty") + } + + if p.conf.Upstream == "" { + return errors.New("variable Upstream cannot be empty") + } + + var err error + p.upstream, err = NewClient(p.conf.Upstream) + if err != nil { + return fmt.Errorf("while dialing upstream '%s' - %w", p.conf.Upstream, err) + } + + p.listen, err = NewServer(ServerConfig{ + BindAddress: p.conf.Listen, + Handler: p.handler, + }) + if err != nil { + return fmt.Errorf("while attempting to listen on '%s' - %w", p.conf.Listen, err) + } + return nil +} + +func (p *Proxy) handler(conn net.PacketConn, buf []byte, addr net.Addr) { + p.mutex.Lock() + defer p.mutex.Unlock() + logrus.Debugf("proxy %d bytes %s -> %s", len(buf), addr.String(), p.upstream.addr.String()) + // Forward the request to upstream + if _, err := p.upstream.Send(buf); err != nil { + logrus.WithError(err).Errorf("failed to forward '%d' bytes from '%s' to upstream '%s'", len(buf), addr.String(), p.upstream.addr.String()) + return + } + + // Wait for a response until timeout + b := make([]byte, 10_000) + n, _, _ := p.upstream.Recv(b, p.conf.UpstreamTimeout) + + // Nothing to send to upstream + if n == 0 { + return + } + + // Send response to upstream + if _, err := conn.WriteTo(b[:n], addr); err != nil { + logrus.WithError(err).Errorf("failed to forward '%d' bytes from '%s' to downstream '%s'", n, p.upstream.addr.String(), addr.String()) + return + } + logrus.Debugf("proxy %d bytes %s <- %s", n, p.upstream.addr.String(), addr.String()) +} + +func (p *Proxy) Stop() { + p.mutex.Lock() + defer p.mutex.Unlock() + p.listen.Close() + p.upstream.Close() +} diff --git a/udp/proxy_test.go b/udp/proxy_test.go new file mode 100644 index 00000000..6e1aafe3 --- /dev/null +++ b/udp/proxy_test.go @@ -0,0 +1,99 @@ +package udp_test + +import ( + "fmt" + "net" + "testing" + "time" + + "github.com/sirupsen/logrus" + + "github.com/mailgun/holster/v3/udp" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestServerClient(t *testing.T) { + srv, err := udp.NewServer(udp.ServerConfig{ + BindAddress: "127.0.0.1:5001", + Handler: func(conn net.PacketConn, recv []byte, addr net.Addr) { + resp := fmt.Sprintf("Hello, %s", string(recv)) + _, err := conn.WriteTo([]byte(resp), addr) + require.NoError(t, err) + }, + }) + require.NoError(t, err) + defer srv.Close() + + conn, err := udp.NewClient("127.0.0.1:5001") + require.NoError(t, err) + defer conn.Close() + + _, err = conn.Send([]byte("Thrawn")) + require.NoError(t, err) + + b := make([]byte, 50) + n, _, err := conn.Recv(b, time.Second) + + assert.Equal(t, "Hello, Thrawn", string(b[:n])) +} + +func TestProxy(t *testing.T) { + logrus.SetLevel(logrus.DebugLevel) + srv, err := udp.NewServer(udp.ServerConfig{ + BindAddress: "127.0.0.1:5001", + Handler: func(conn net.PacketConn, recv []byte, addr net.Addr) { + resp := fmt.Sprintf("Hello, %s", string(recv)) + _, err := conn.WriteTo([]byte(resp), addr) + require.NoError(t, err) + }, + }) + require.NoError(t, err) + defer srv.Close() + + p := udp.NewProxy(udp.ProxyConfig{ + Listen: "127.0.0.1:5000", + Upstream: "127.0.0.1:5001", + UpstreamTimeout: time.Second, + }) + + // Start the proxy for our udp server + err = p.Start() + require.NoError(t, err) + defer p.Stop() + + conn, err := udp.NewClient("127.0.0.1:5000") + require.NoError(t, err) + defer conn.Close() + + _, err = conn.Send([]byte("Admiral")) + require.NoError(t, err) + + b := make([]byte, 50) + n, _, err := conn.Recv(b, time.Second) + require.NoError(t, err) + + assert.Equal(t, "Hello, Admiral", string(b[:n])) + + // Shutdown the proxy + p.Stop() + + // Should not get a response from the upstream server + _, err = conn.Send([]byte("Not expecting a response")) + require.NoError(t, err) + n, _, err = conn.Recv(b, time.Second) + require.Error(t, err) + assert.Contains(t, err.Error(), "recvfrom: connection refused") + + // Start the proxy again + p.Start() + + // Should get a response + _, err = conn.Send([]byte("World")) + require.NoError(t, err) + + n, _, err = conn.Recv(b, time.Second) + require.NoError(t, err) + + assert.Equal(t, "Hello, World", string(b[:n])) +} diff --git a/udp/server.go b/udp/server.go new file mode 100644 index 00000000..61d48ec8 --- /dev/null +++ b/udp/server.go @@ -0,0 +1,62 @@ +package udp + +import ( + "fmt" + "net" + "strings" + + "github.com/sirupsen/logrus" +) + +type Server struct { + conn net.PacketConn + shutdown chan struct{} +} + +type Handler func(net.PacketConn, []byte, net.Addr) + +type ServerConfig struct { + BindAddress string + Handler Handler +} + +func NewServer(conf ServerConfig) (*Server, error) { + conn, err := net.ListenPacket("udp", conf.BindAddress) + if err != nil { + return nil, fmt.Errorf("while listening on '%s' - %w", conf.BindAddress, err) + } + + shutdown := make(chan struct{}) + logrus.Debugf("Listening [%s]...\n", conf.BindAddress) + + go func() { + for { + select { + case <-shutdown: + return + default: + } + + b := make([]byte, 10_000) + n, addr, err := conn.ReadFrom(b) + if err != nil { + if strings.HasSuffix(err.Error(), "use of closed network connection") { + return + } + logrus.WithError(err).Error("ReadFrom() failed") + return + } + //fmt.Printf("packet-received: bytes=%d from=%s\n", n, addr.String()) + conf.Handler(conn, b[:n], addr) + } + }() + return &Server{ + conn: conn, + shutdown: shutdown, + }, nil +} + +func (u *Server) Close() { + close(u.shutdown) + u.conn.Close() +} From 64b30567aeb5390bfc15aa1bb1fba52b479f5665 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Fri, 12 Mar 2021 12:08:52 -0600 Subject: [PATCH 13/15] Isolated leader now properly steps down --- discovery/consul.go | 10 ++--- discovery/memberlist.go | 28 ++++++------ election/cluster_test.go | 2 +- election/election.go | 77 +++++++++++++++++-------------- election/election_test.go | 95 ++++++++++++++++++++++++++++++++++----- go.mod | 1 + udp/proxy.go | 35 +++++++++++++++ udp/proxy_test.go | 1 + 8 files changed, 185 insertions(+), 64 deletions(-) diff --git a/discovery/consul.go b/discovery/consul.go index e765f804..b2b1186c 100644 --- a/discovery/consul.go +++ b/discovery/consul.go @@ -37,7 +37,7 @@ type ConsulConfig struct { Logger logrus.FieldLogger } -type consulStore struct { +type Consul struct { wg syncutil.WaitGroup log logrus.FieldLogger client *api.Client @@ -59,7 +59,7 @@ func NewConsul(conf *ConsulConfig) (Members, error) { return nil, errors.New("CatalogName cannot be empty") } - cs := consulStore{ + cs := Consul{ ctx: cancel.New(context.Background()), log: conf.Logger, conf: conf, @@ -121,7 +121,7 @@ func NewConsul(conf *ConsulConfig) (Members, error) { return &cs, nil } -func (cs *consulStore) watch() error { +func (cs *Consul) watch() error { changeCh := make(chan []*api.ServiceEntry, 100) var previousPeers map[string]Peer var err error @@ -200,7 +200,7 @@ func (cs *consulStore) watch() error { return nil } -func (cs *consulStore) GetPeers(ctx context.Context) ([]Peer, error) { +func (cs *Consul) GetPeers(ctx context.Context) ([]Peer, error) { opts := &api.QueryOptions{LocalOnly: true} services, _, err := cs.client.Health().Service(cs.conf.CatalogName, "", true, opts.WithContext(ctx)) if err != nil { @@ -223,7 +223,7 @@ func (cs *consulStore) GetPeers(ctx context.Context) ([]Peer, error) { return peers, nil } -func (cs *consulStore) Close(ctx context.Context) error { +func (cs *Consul) Close(ctx context.Context) error { errCh := make(chan error) go func() { cs.plan.Stop() diff --git a/discovery/memberlist.go b/discovery/memberlist.go index 0f5519ea..1202f01c 100644 --- a/discovery/memberlist.go +++ b/discovery/memberlist.go @@ -9,6 +9,7 @@ import ( "sort" "strconv" "sync" + "time" ml "github.com/hashicorp/memberlist" "github.com/mailgun/holster/v3/clock" @@ -38,29 +39,27 @@ type Members interface { //UpdatePeer(context.Context, Peer) error } -type memberList struct { +type MemberList struct { log logrus.FieldLogger memberList *ml.Memberlist conf MemberListConfig events *eventDelegate + mutex sync.Mutex } type MemberListConfig struct { // This is the address:port the member list protocol listen for other peers on. BindAddress string - // This is the address:port the member list protocol will advertise to other peers. (Defaults to BindAddress) AdvertiseAddress string - // Metadata about this peer which should be shared with other peers Peer Peer - // A list of peers this member list instance can contact to find other peers. KnownPeers []string - // A callback function which is called when the member list changes. OnUpdate OnUpdateFunc - + // If not nil, use this config instead of ml.DefaultLANConfig() + MemberListConfig *ml.Config // An interface through which logging will occur; usually *logrus.Entry Logger logrus.FieldLogger } @@ -74,10 +73,9 @@ func NewMemberList(ctx context.Context, conf MemberListConfig) (Members, error) if conf.BindAddress == "" { return nil, errors.New("BindAddress cannot be empty") } - conf.Peer.IsSelf = false - m := &memberList{ + m := &MemberList{ log: conf.Logger, conf: conf, events: &eventDelegate{ @@ -110,10 +108,12 @@ func NewMemberList(ctx context.Context, conf MemberListConfig) (Members, error) return m, errors.Wrap(err, "timed out attempting to join member list") } -func (m *memberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { - config := ml.DefaultWANConfig() +func (m *MemberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { + config := conf.MemberListConfig + setter.SetDefault(&config, ml.DefaultLANConfig()) config.Name = conf.Peer.ID - config.LogOutput = newLogWriter(conf.Logger) + config.LogOutput = NewLogWriter(conf.Logger) + config.PushPullInterval = time.Second * 5 var err error config.BindAddr, config.BindPort, err = splitAddress(conf.BindAddress) @@ -133,7 +133,7 @@ func (m *memberList) newMLConfig(conf MemberListConfig) (*ml.Config, error) { return config, nil } -func (m *memberList) Close(ctx context.Context) error { +func (m *MemberList) Close(ctx context.Context) error { errCh := make(chan error) go func() { if err := m.memberList.Leave(clock.Second * 30); err != nil { @@ -152,7 +152,7 @@ func (m *memberList) Close(ctx context.Context) error { return nil } -func (m *memberList) GetPeers(_ context.Context) ([]Peer, error) { +func (m *MemberList) GetPeers(_ context.Context) ([]Peer, error) { return m.events.GetPeers() } @@ -226,7 +226,7 @@ func (m *delegate) GetBroadcasts(int, int) [][]byte { return nil } func (m *delegate) LocalState(bool) []byte { return nil } func (m *delegate) MergeRemoteState([]byte, bool) {} -func newLogWriter(log logrus.FieldLogger) *io.PipeWriter { +func NewLogWriter(log logrus.FieldLogger) *io.PipeWriter { reader, writer := io.Pipe() go func() { diff --git a/election/cluster_test.go b/election/cluster_test.go index 8b846e36..22b472ce 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -127,7 +127,7 @@ func (c *TestCluster) ClearErrors() { } // Add a specific peer to peer error -func (c *TestCluster) AddPeerToPeerError(from string, to string, err error) { +func (c *TestCluster) Disconnect(from string, to string, err error) { c.lock.Lock() defer c.lock.Unlock() c.errors[c.peerKey(from, to)] = err diff --git a/election/election.go b/election/election.go index 710c4a41..a946bfd2 100644 --- a/election/election.go +++ b/election/election.go @@ -19,37 +19,39 @@ type NodeState GetStateResp type state uint32 const ( - // FollowerState means we are following the leader and expect + // followerState means we are following the leader and expect // to get heart beats regularly. This is the initial state, as // we don't want to force an election when a new node joins // the cluster. - FollowerState state = iota - // CandidateState means we are actively attempting to become leader - CandidateState - // LeaderState means we have received a quorum of votes while - // in CandidateState and have assumed leadership. - LeaderState - // ShutdownState means we are in the process of shutting down - ShutdownState + followerState state = iota + // candidateState means we are actively attempting to become leader + candidateState + // leaderState means we have received a quorum of votes while + // in candidateState and have assumed leadership. + leaderState + // shutdownState means we are in the process of shutting down + shutdownState ) var ErrNotLeader = errors.New("not the leader") func (s state) String() string { switch s { - case FollowerState: + case followerState: return "Follower" - case CandidateState: + case candidateState: return "Candidate" - case LeaderState: + case leaderState: return "Leader" - case ShutdownState: + case shutdownState: return "Shutdown" default: return "Unknown" } } +type SendRPCFunc func(context.Context, string, RPCRequest, *RPCResponse) error + type Config struct { // How long we should wait for a single network operation to complete. NetworkTimeout time.Duration @@ -83,7 +85,7 @@ type Config struct { // Sends an RPC request to a peer, This function must be provided and can // utilize any network communication the implementer wishes. If context cancelled // should return an error. - SendRPC func(context.Context, string, RPCRequest, *RPCResponse) error + SendRPC SendRPCFunc } type OnUpdate func(string) @@ -144,6 +146,10 @@ func NewNode(conf Config) (Node, error) { return nil, errors.New("refusing to spawn a new node with no Config.UniqueID defined") } + if conf.SendRPC == nil { + return nil, errors.New("refusing to spawn a new node with no Config.SendRPC defined") + } + setter.SetDefault(&conf.Log, logrus.WithField("id", conf.UniqueID)) setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*60) setter.SetDefault(&conf.HeartBeatTimeout, time.Second*20) @@ -240,7 +246,7 @@ func (e *node) isLeader() bool { func (e *node) setLeader(leader string) { if e.leader != leader { - e.log.Debugf("Set Leader (%s)", leader) + e.log.Debugf("Set Leader '%s'", leader) e.leader = leader if e.conf.OnUpdate != nil { e.conf.OnUpdate(leader) @@ -317,17 +323,17 @@ func (e *node) run() { for { select { case <-e.shutdownCh: - e.state = ShutdownState + e.state = shutdownState return default: } switch e.state { - case FollowerState: + case followerState: e.runFollower() - case CandidateState: + case candidateState: e.runCandidate() - case LeaderState: + case leaderState: e.runLeader() } } @@ -340,7 +346,7 @@ func (e *node) runFollower() { noPeersTimer := time.NewTimer(e.conf.HeartBeatTimeout / 5) defer noPeersTimer.Stop() - for e.state == FollowerState { + for e.state == followerState { select { case rpc := <-e.rpcCh: e.processRPC(rpc) @@ -353,7 +359,7 @@ func (e *node) runFollower() { // Heartbeat failed! Transition to the candidate state e.log.Debugf("heartbeat timeout, starting election; previous leader was '%s'", e.leader) e.setLeader("") - e.state = CandidateState + e.state = candidateState return case <-noPeersTimer.C: // If we already have leader, don't check for no peers @@ -364,7 +370,7 @@ func (e *node) runFollower() { // If we have no peers, or if we are the only peer, no need to wait // for the heartbeat timeout. Change state to candidate and start the election. if len(e.peers) == 0 || len(e.peers) == 1 && e.peers[0] == e.self { - e.state = CandidateState + e.state = candidateState return } case <-e.shutdownCh: @@ -392,7 +398,7 @@ func (e *node) runCandidate() { votesNeeded := e.quorumSize() e.log.Debugf("votes needed: %d", votesNeeded) - for e.state == CandidateState { + for e.state == candidateState { select { case <-voteTimer.C: voteCh = e.electSelf() @@ -402,7 +408,7 @@ func (e *node) runCandidate() { // Check if the term is greater than ours, bail if vote.Term > e.currentTerm { e.log.Debug("newer term discovered, fallback to follower") - e.state = FollowerState + e.state = followerState e.currentTerm = vote.Term return } @@ -416,7 +422,7 @@ func (e *node) runCandidate() { // Check if we've become the leader if grantedVotes >= votesNeeded { e.log.Debugf("election won! tally is '%d'", grantedVotes) - e.state = LeaderState + e.state = leaderState e.setLeader(e.self) return } @@ -504,7 +510,7 @@ func (e *node) runLeader() { peersLastContact := make(map[string]time.Time, len(e.peers)) heartBeatReplyCh := make(chan HeartBeatResp, 5_000) - for e.state == LeaderState { + for e.state == leaderState { select { case rpc := <-e.rpcCh: e.processRPC(rpc) @@ -533,15 +539,16 @@ func (e *node) runLeader() { now := time.Now() for _, peer := range e.peers { if peer == e.self { - contacted++ continue } lc, ok := peersLastContact[peer] if !ok { + e.log.Debugf("quorum check - peer '%s' not found", peer) continue } diff := now.Sub(lc) + e.log.Debugf("quorum check - peer '%s' diff '%f", peer, diff.Seconds()) if diff >= e.conf.HeartBeatTimeout { e.log.Debugf("no heartbeat response from '%s' for '%s'", peer, diff) continue @@ -551,9 +558,10 @@ func (e *node) runLeader() { // Verify we can contact a quorum (Minus ourself) quorum := e.quorumSize() + e.log.Debugf("quorum check - quorum='%d' contacted='%d'", quorum-1, contacted) if contacted < (quorum - 1) { e.log.Debug("failed to receive heart beats from a quorum of peers; stepping down") - e.state = FollowerState + e.state = followerState // Inform the other peers we are stepping down for _, peer := range e.peers { @@ -561,7 +569,8 @@ func (e *node) runLeader() { } } case <-e.shutdownCh: - e.state = ShutdownState + e.state = shutdownState + e.log.Debug("leader shutdown") if e.isLeader() { // Notify all followers we are no longer leader for _, peer := range e.peers { @@ -651,7 +660,7 @@ func (e *node) processRPC(rpc RPCRequest) { func (e *node) handleResign(rpc RPCRequest) { e.log.Debug("RPC: election.ResignReq{}") e.setLeader("") - e.state = FollowerState + e.state = followerState for _, peer := range e.peers { e.sendElectionReset(peer) } @@ -662,7 +671,7 @@ func (e *node) handleResign(rpc RPCRequest) { func (e *node) handleResetElection(rpc RPCRequest) { e.log.Debug("RPC: election.ResetElectionReq{}") e.setLeader("") - e.state = CandidateState + e.state = candidateState rpc.respond(rpc.RPC, ResetElectionResp{}, "") } @@ -693,8 +702,8 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // // This can also occur if a leader loses connectivity to the rest of the cluster. // In this case we become the follower of who ever sent us a heartbeat. - if e.state != FollowerState { - e.state = FollowerState + if e.state != followerState { + e.state = followerState e.currentTerm = req.Term resp.Term = req.Term } @@ -735,7 +744,7 @@ func (e *node) handleVote(rpc RPCRequest, req VoteReq) { if req.Term > e.currentTerm { // Ensure transition to follower e.log.Debugf("received a vote request with a newer term '%d'", req.Term) - e.state = FollowerState + e.state = followerState e.currentTerm = req.Term resp.Term = req.Term } diff --git a/election/election_test.go b/election/election_test.go index ffe7aedb..c8a0cdf6 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -229,24 +229,24 @@ func TestOmissionFaults(t *testing.T) { // // n3 and n4 can't talk - c1.AddPeerToPeerError("n3", "n4", ErrConnRefused) - c1.AddPeerToPeerError("n4", "n3", ErrConnRefused) + c1.Disconnect("n3", "n4", ErrConnRefused) + c1.Disconnect("n4", "n3", ErrConnRefused) // Leader can't talk to n4 - c1.AddPeerToPeerError("n0", "n4", ErrConnRefused) - c1.AddPeerToPeerError("n4", "n0", ErrConnRefused) + c1.Disconnect("n0", "n4", ErrConnRefused) + c1.Disconnect("n4", "n0", ErrConnRefused) // Leader can't talk to n3 - c1.AddPeerToPeerError("n0", "n3", ErrConnRefused) - c1.AddPeerToPeerError("n3", "n0", ErrConnRefused) + c1.Disconnect("n0", "n3", ErrConnRefused) + c1.Disconnect("n3", "n0", ErrConnRefused) // n2 and n4 can't talk - c1.AddPeerToPeerError("n2", "n4", ErrConnRefused) - c1.AddPeerToPeerError("n4", "n2", ErrConnRefused) + c1.Disconnect("n2", "n4", ErrConnRefused) + c1.Disconnect("n4", "n2", ErrConnRefused) // n1 and n3 can't talk - c1.AddPeerToPeerError("n1", "n3", ErrConnRefused) - c1.AddPeerToPeerError("n3", "n1", ErrConnRefused) + c1.Disconnect("n1", "n3", ErrConnRefused) + c1.Disconnect("n3", "n1", ErrConnRefused) // Cluster should retain n0 as leader in the face on unstable cluster for i := 0; i < 12; i++ { @@ -266,3 +266,78 @@ func TestOmissionFaults(t *testing.T) { time.Sleep(time.Millisecond * 400) } } + +func TestIsolatedLeader(t *testing.T) { + c1 := NewTestCluster() + createCluster(t, c1) + defer c1.Close() + + // Create a cluster where the leader become isolated from the rest + // of the cluster. + // + // Diagram: lines indicate connectivity + // between nodes and n0 is leader + // + // (n0)----(n1)----(n4) + // / \ / + // / \ / + // / \ / + // (n2)----(n3) + // + require.Equal(t, c1.GetLeader().GetLeader(), "n0") + + // Leader can't talk to n2 + c1.Disconnect("n0", "n2", ErrConnRefused) + c1.Disconnect("n2", "n0", ErrConnRefused) + + // Leader can't talk to n3 + c1.Disconnect("n0", "n3", ErrConnRefused) + c1.Disconnect("n3", "n0", ErrConnRefused) + + // Leader can't talk to n4 + c1.Disconnect("n0", "n4", ErrConnRefused) + c1.Disconnect("n4", "n0", ErrConnRefused) + + // Leader should realize it doesn't have a quorum of + // heartbeats and step down and remaining cluster should + // elect a new leader + for i := 0; i < 20; i++ { + leader := c1.GetLeader() + if leader == nil { + goto sleep + } + + // Leader should no longer be n0 + if leader.GetLeader() != "n0" { + // A node in the new cluster must agree and have elected a new leader + l := c1.Nodes["n4"].Node.GetLeader() + if l != "" && l == "n0" { + break + } + } + sleep: + time.Sleep(time.Millisecond * 500) + } + require.NotNil(t, c1.GetLeader()) + require.NotEqual(t, c1.GetLeader().GetLeader(), "n0") + require.Equal(t, c1.GetLeader().GetLeader(), "n0") + // Note: In the case where n1 is elected the new leader, + // n0 will know that n1 is the new leader sooner than later + // since connectivity from n0 to n1 was never interrupted. + //fmt.Printf("Cluster: %#v\n", c1.GetClusterStatus()) + + // Should persist new leader once communication is restored + c1.ClearErrors() + + for i := 0; i < 20; i++ { + if c1.Nodes["n0"].Node.GetLeader() == "" { + time.Sleep(time.Millisecond * 500) + continue + } + break + } + + s, err := c1.Nodes["n0"].Node.GetState(context.Background()) + require.NoError(t, err) + assert.Equal(t, "Follower", s.State) +} diff --git a/go.mod b/go.mod index 84106cb6..fd7dea07 100644 --- a/go.mod +++ b/go.mod @@ -23,6 +23,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.10.0 // indirect github.com/hashicorp/consul/api v1.7.0 github.com/hashicorp/go-hclog v0.12.0 + github.com/hashicorp/go-msgpack v0.5.3 github.com/hashicorp/golang-lru v0.5.1 // indirect github.com/hashicorp/memberlist v0.2.2 github.com/jonboulle/clockwork v0.1.0 // indirect diff --git a/udp/proxy.go b/udp/proxy.go index 3bf9576c..70b5aca5 100644 --- a/udp/proxy.go +++ b/udp/proxy.go @@ -15,6 +15,8 @@ type Proxy struct { listen *Server upstream *Client mutex sync.Mutex // ensure the handler isn't still executing when start()/stop() are called + // A list of addresses that we deny udp proxy for + blocked []string } type ProxyConfig struct { @@ -63,7 +65,15 @@ func (p *Proxy) Start() error { func (p *Proxy) handler(conn net.PacketConn, buf []byte, addr net.Addr) { p.mutex.Lock() defer p.mutex.Unlock() + + for _, block := range p.blocked { + if block == addr.String() { + logrus.Debugf("Blocked proxy of %d bytes from '%s' ", len(buf), addr.String()) + return + } + } logrus.Debugf("proxy %d bytes %s -> %s", len(buf), addr.String(), p.upstream.addr.String()) + // Forward the request to upstream if _, err := p.upstream.Send(buf); err != nil { logrus.WithError(err).Errorf("failed to forward '%d' bytes from '%s' to upstream '%s'", len(buf), addr.String(), p.upstream.addr.String()) @@ -87,6 +97,31 @@ func (p *Proxy) handler(conn net.PacketConn, buf []byte, addr net.Addr) { logrus.Debugf("proxy %d bytes %s <- %s", n, p.upstream.addr.String(), addr.String()) } +func (p *Proxy) Block(addr string) { + p.mutex.Lock() + defer p.mutex.Unlock() + p.blocked = append(p.blocked, addr) +} + +func (p *Proxy) UnBlock(addr string) { + p.mutex.Lock() + defer p.mutex.Unlock() + + // Short cut + if len(p.blocked) == 1 { + p.blocked = nil + } + + var blocked []string + for _, b := range p.blocked { + if b == addr { + continue + } + blocked = append(blocked, addr) + } + p.blocked = blocked +} + func (p *Proxy) Stop() { p.mutex.Lock() defer p.mutex.Unlock() diff --git a/udp/proxy_test.go b/udp/proxy_test.go index 6e1aafe3..64d4194d 100644 --- a/udp/proxy_test.go +++ b/udp/proxy_test.go @@ -96,4 +96,5 @@ func TestProxy(t *testing.T) { require.NoError(t, err) assert.Equal(t, "Hello, World", string(b[:n])) + } From 14c93af8acfbbd9b9a066087d4526cc30ae6a2e4 Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Wed, 17 Mar 2021 16:49:04 -0500 Subject: [PATCH 14/15] Introduced MinimumQuorum to avoid small split brains from becoming their own cluster --- election/cluster_test.go | 3 ++ election/election.go | 52 ++++++++++++++++++++++------- election/election_test.go | 70 ++++++++++++++++++++++++++++++++++----- 3 files changed, 105 insertions(+), 20 deletions(-) diff --git a/election/cluster_test.go b/election/cluster_test.go index 22b472ce..594d8b3d 100644 --- a/election/cluster_test.go +++ b/election/cluster_test.go @@ -78,6 +78,9 @@ func (c *TestCluster) Add(name string, node *ClusterNode) { } func (c *TestCluster) Remove(name string) *ClusterNode { + c.lock.Lock() + defer c.lock.Unlock() + n := c.Nodes[name] delete(c.Nodes, name) c.updatePeers() diff --git a/election/election.go b/election/election.go index a946bfd2..2614dcb8 100644 --- a/election/election.go +++ b/election/election.go @@ -69,6 +69,16 @@ type Config struct { // new election. LeaderQuorumTimeout time.Duration + // The minimum number of peers that are required to form a cluster and elect a leader. + // This is to prevent a small number of nodes (or a single node) that gets disconnected + // from the cluster to elect a leader (assuming the peer list is updated to exclude the + // disconnected peers). Instead nodes will wait until connectivity is restored + // to the quorum of the cluster. The default is zero, which means if a single node is + // disconnected from the cluster, and it's peer list only includes it's self, it will elect + // itself leader. If we set MinimumQuorum = 2 then no leader will be elected until the peer + // list includes at least 2 peers and a successful vote has completed. + MinimumQuorum int + // The Initial list of peers to be considered in the election, including ourself. Peers []string @@ -151,9 +161,9 @@ func NewNode(conf Config) (Node, error) { } setter.SetDefault(&conf.Log, logrus.WithField("id", conf.UniqueID)) - setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*60) - setter.SetDefault(&conf.HeartBeatTimeout, time.Second*20) - setter.SetDefault(&conf.ElectionTimeout, time.Second*15) + setter.SetDefault(&conf.LeaderQuorumTimeout, time.Second*12) + setter.SetDefault(&conf.HeartBeatTimeout, time.Second*6) + setter.SetDefault(&conf.ElectionTimeout, time.Second*6) setter.SetDefault(&conf.NetworkTimeout, time.Second*3) c := &node{ @@ -341,9 +351,10 @@ func (e *node) run() { func (e *node) runFollower() { e.log.Debugf("entering follower state, current leader is '%s'", e.leader) - heartbeatTimer := time.NewTicker(randomDuration(e.conf.HeartBeatTimeout)) + timeout := randomDuration(e.conf.HeartBeatTimeout) + heartbeatTimer := time.NewTicker(timeout) defer heartbeatTimer.Stop() - noPeersTimer := time.NewTimer(e.conf.HeartBeatTimeout / 5) + noPeersTimer := time.NewTimer(timeout / 5) defer noPeersTimer.Stop() for e.state == followerState { @@ -401,6 +412,12 @@ func (e *node) runCandidate() { for e.state == candidateState { select { case <-voteTimer.C: + // Do not start a vote if we are below our minimum quorum + if len(e.peers) < e.conf.MinimumQuorum { + e.log.Warnf("peer count '%d' below minimum quorum of '%d'; sleeping...", + len(e.peers), e.conf.MinimumQuorum) + continue + } voteCh = e.electSelf() case rpc := <-e.rpcCh: e.processRPC(rpc) @@ -533,6 +550,19 @@ func (e *node) runLeader() { e.sendHeartBeat(peer, heartBeatReplyCh) } case <-quorumTicker.C: + // If the number of peers falls below our MinimumQuorum then we step down. + if len(e.peers) < e.conf.MinimumQuorum { + e.log.Warnf("peer count '%d' below minimum quorum of '%d'; stepping down", + len(e.peers), e.conf.MinimumQuorum) + e.state = followerState + e.setLeader("") + // Inform the other peers we are stepping down + for _, peer := range e.peers { + e.sendElectionReset(peer) + } + return + } + // Check if we have received contact from a quorum of nodes within the leader quorum timeout interval. // If not, we step down as we may have lost connectivity. contacted := 0 @@ -562,11 +592,12 @@ func (e *node) runLeader() { if contacted < (quorum - 1) { e.log.Debug("failed to receive heart beats from a quorum of peers; stepping down") e.state = followerState - + e.setLeader("") // Inform the other peers we are stepping down for _, peer := range e.peers { e.sendElectionReset(peer) } + return } case <-e.shutdownCh: e.state = shutdownState @@ -577,6 +608,7 @@ func (e *node) runLeader() { e.sendElectionReset(peer) } } + return } } e.lastContact = time.Now() @@ -687,11 +719,6 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { rpc.respond(rpc.RPC, resp, "") }() - // Ignore an older term - if req.Term < e.currentTerm { - return - } - // This might occur if 2 or more nodes think they are elected leader. In this // case all leaders that emit heartbeats will both fall back to follower, from // there the followers will timeout waiting for a heartbeat and the vote will @@ -701,7 +728,8 @@ func (e *node) handleHeartBeat(rpc RPCRequest, req HeartBeatReq) { // in that case the first leader to send a heartbeat becomes leader. // // This can also occur if a leader loses connectivity to the rest of the cluster. - // In this case we become the follower of who ever sent us a heartbeat. + // In this case we become the follower of who ever sent us a heartbeat, regardless + // of our current term compared the one who sent us the heartbeat. if e.state != followerState { e.state = followerState e.currentTerm = req.Term diff --git a/election/election_test.go b/election/election_test.go index c8a0cdf6..25064c43 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -3,6 +3,7 @@ package election_test import ( "context" "errors" + "fmt" "testing" "time" @@ -320,7 +321,6 @@ func TestIsolatedLeader(t *testing.T) { } require.NotNil(t, c1.GetLeader()) require.NotEqual(t, c1.GetLeader().GetLeader(), "n0") - require.Equal(t, c1.GetLeader().GetLeader(), "n0") // Note: In the case where n1 is elected the new leader, // n0 will know that n1 is the new leader sooner than later // since connectivity from n0 to n1 was never interrupted. @@ -329,15 +329,69 @@ func TestIsolatedLeader(t *testing.T) { // Should persist new leader once communication is restored c1.ClearErrors() - for i := 0; i < 20; i++ { - if c1.Nodes["n0"].Node.GetLeader() == "" { - time.Sleep(time.Millisecond * 500) - continue - } - break - } + //for i := 0; i < 20; i++ { + // if c1.Nodes["n0"].Node.GetLeader() == "" { + // time.Sleep(time.Millisecond * 500) + // continue + // } + // break + //} + + // Should pick up the leadership from the rest of the cluster + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + leader := c1.Nodes["n0"].Node.GetLeader() + assert.NotEqual(t, leader, "") + }) s, err := c1.Nodes["n0"].Node.GetState(context.Background()) + fmt.Printf("State: %#v\n", s) require.NoError(t, err) assert.Equal(t, "Follower", s.State) } + +func TestMinimumQuorum(t *testing.T) { + c := NewTestCluster() + + cfg := &election.Config{ + NetworkTimeout: time.Second, + HeartBeatTimeout: time.Second, + LeaderQuorumTimeout: time.Second * 2, + ElectionTimeout: time.Second * 2, + MinimumQuorum: 2, + } + + err := c.SpawnNode("n0", cfg) + require.NoError(t, err) + + time.Sleep(time.Second * 5) + + // Ensure n0 is not leader + status := c.GetClusterStatus() + require.NotEqual(t, "n0", status["n0"]) + + c.SpawnNode("n1", cfg) + + // Should elect a leader + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.NotEqual(t, status["n0"], "") + }) + + status = c.GetClusterStatus() + var leader string + + // Shutdown the follower + if status["n0"] == "n0" { + c.Remove("n1").Node.Stop(context.Background()) + leader = "n0" + } else { + c.Remove("n0").Node.Stop(context.Background()) + leader = "n1" + } + + // The leader should detect it no longer has MinimumQuorum and step down + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, status[leader], "") + }) +} From a150f599897906c8b779f82085800b2dc700b88e Mon Sep 17 00:00:00 2001 From: Derrick Wippler Date: Thu, 18 Mar 2021 15:25:04 -0500 Subject: [PATCH 15/15] Resigning a follower is now ignored --- election/election.go | 8 +++++- election/election_test.go | 59 +++++++++++++++++++++++++++++++++------ 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/election/election.go b/election/election.go index 2614dcb8..f6670fcc 100644 --- a/election/election.go +++ b/election/election.go @@ -691,12 +691,18 @@ func (e *node) processRPC(rpc RPCRequest) { // if we are leader returns Success = true func (e *node) handleResign(rpc RPCRequest) { e.log.Debug("RPC: election.ResignReq{}") + // If not leader, do nothing + if !e.isLeader() { + rpc.respond(rpc.RPC, ResignResp{}, "") + return + } + e.setLeader("") e.state = followerState for _, peer := range e.peers { e.sendElectionReset(peer) } - rpc.respond(rpc.RPC, ResignReq{}, "") + rpc.respond(rpc.RPC, ResignResp{Success: true}, "") } // handleResetElection resets our state and starts a new election diff --git a/election/election_test.go b/election/election_test.go index 25064c43..c4a0f2f9 100644 --- a/election/election_test.go +++ b/election/election_test.go @@ -329,14 +329,6 @@ func TestIsolatedLeader(t *testing.T) { // Should persist new leader once communication is restored c1.ClearErrors() - //for i := 0; i < 20; i++ { - // if c1.Nodes["n0"].Node.GetLeader() == "" { - // time.Sleep(time.Millisecond * 500) - // continue - // } - // break - //} - // Should pick up the leadership from the rest of the cluster testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { leader := c1.Nodes["n0"].Node.GetLeader() @@ -395,3 +387,54 @@ func TestMinimumQuorum(t *testing.T) { assert.Equal(t, status[leader], "") }) } + +func TestResign(t *testing.T) { + c1 := NewTestCluster() + createCluster(t, c1) + defer c1.Close() + + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + assert.NotNil(t, c1.GetLeader()) + }) + + leader := c1.GetLeader() + + // Calling resign on a follower should have no effect + c1.Nodes["n1"].Node.Resign(context.Background()) + + for i := 0; i < 10; i++ { + if c1.GetLeader() != leader { + require.FailNow(t, "leader should not have changed") + } + time.Sleep(time.Millisecond * 500) + } + // Calling resign on the leader should give up leader + c1.Nodes["n0"].Node.Resign(context.Background()) + + testutil.UntilPass(t, 30, time.Second, func(t testutil.TestingT) { + assert.NotEqual(t, leader, c1.GetLeader()) + }) +} + +func TestResignSingleNode(t *testing.T) { + c := NewTestCluster() + c.SpawnNode("n0", cfg) + defer c.Close() + + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, ClusterStatus{ + "n0": "n0", + }, status) + }) + + c.Nodes["n0"].Node.Resign(context.Background()) + + // n0 will eventually become leader again + testutil.UntilPass(t, 10, time.Second, func(t testutil.TestingT) { + status := c.GetClusterStatus() + assert.Equal(t, ClusterStatus{ + "n0": "n0", + }, status) + }) +}