Add simple cluster leader election.
parent
c8b9d783b7
commit
458c2d9d7c
7
log.go
7
log.go
|
@ -117,6 +117,9 @@ func (l *Log) NewCommand(name string) (Command, error) {
|
|||
// deserialized each time a new log entry is read. This function will panic
|
||||
// if a command type with the same name already exists.
|
||||
func (l *Log) AddCommandType(command Command) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
|
||||
if command == nil {
|
||||
panic(fmt.Sprintf("raft.Log: Command type cannot be nil"))
|
||||
} else if l.commandTypes[command.CommandName()] != nil {
|
||||
|
@ -281,7 +284,7 @@ func (l *Log) SetCommitIndex(index uint64) error {
|
|||
func (l *Log) Truncate(index uint64, term uint64) error {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
|
||||
|
||||
// Do not allow committed entries to be truncated.
|
||||
if index < l.CommitIndex() {
|
||||
return fmt.Errorf("raft.Log: Index is already committed (%v): (IDX=%v, TERM=%v)", l.CommitIndex(), index, term)
|
||||
|
@ -307,7 +310,7 @@ func (l *Log) Truncate(index uint64, term uint64) error {
|
|||
l.entries = l.entries[0:index]
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -217,12 +217,12 @@ func TestLogTruncate(t *testing.T) {
|
|||
t.Fatalf("Truncating mismatched entries shouldn't work: %v", err)
|
||||
}
|
||||
// Truncate end of log.
|
||||
if err := log.Truncate(3, 2); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1,entry2,entry3})) {
|
||||
t.Fatalf("Truncating end of log should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1,entry2,entry3})
|
||||
if err := log.Truncate(3, 2); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1, entry2, entry3})) {
|
||||
t.Fatalf("Truncating end of log should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1, entry2, entry3})
|
||||
}
|
||||
// Truncate at last commit.
|
||||
if err := log.Truncate(2, 1); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1,entry2})) {
|
||||
t.Fatalf("Truncating at last commit should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1,entry2})
|
||||
if err := log.Truncate(2, 1); !(err == nil && reflect.DeepEqual(log.entries, []*LogEntry{entry1, entry2})) {
|
||||
t.Fatalf("Truncating at last commit should work: %v\n\nEntries:\nActual: %v\nExpected: %v", err, log.entries, []*LogEntry{entry1, entry2})
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -8,6 +8,7 @@ package raft
|
|||
|
||||
// The request sent to a server to vote for a candidate to become a leader.
|
||||
type RequestVoteRequest struct {
|
||||
peer *Peer
|
||||
Term uint64 `json:"term"`
|
||||
CandidateName string `json:"candidateName"`
|
||||
LastLogIndex uint64 `json:"lastLogIndex"`
|
||||
|
@ -16,6 +17,7 @@ type RequestVoteRequest struct {
|
|||
|
||||
// The response returned from a server after a vote for a candidate to become a leader.
|
||||
type RequestVoteResponse struct {
|
||||
peer *Peer
|
||||
Term uint64 `json:"term"`
|
||||
VoteGranted bool `json:"voteGranted"`
|
||||
}
|
||||
|
|
173
server.go
173
server.go
|
@ -35,6 +35,9 @@ const (
|
|||
// candidate or a leader.
|
||||
type Server struct {
|
||||
ApplyFunc func(*Server, Command)
|
||||
DoHandler func(*Server, *Peer, Command) error
|
||||
RequestVoteHandler func(*Server, *Peer, *RequestVoteRequest) (*RequestVoteResponse, error)
|
||||
AppendEntriesHandler func(*Server, *Peer, *AppendEntriesRequest) (*AppendEntriesResponse, error)
|
||||
name string
|
||||
path string
|
||||
state string
|
||||
|
@ -45,8 +48,6 @@ type Server struct {
|
|||
peers map[string]*Peer
|
||||
mutex sync.Mutex
|
||||
electionTimer *ElectionTimer
|
||||
DoHandler func(*Server, *Peer, Command) error
|
||||
AppendEntriesHandler func(*Server, *AppendEntriesRequest) (*AppendEntriesResponse, error)
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
|
@ -64,6 +65,7 @@ func NewServer(name string, path string) (*Server, error) {
|
|||
name: name,
|
||||
path: path,
|
||||
state: Stopped,
|
||||
peers: make(map[string]*Peer),
|
||||
log: NewLog(),
|
||||
electionTimer: NewElectionTimer(DefaultElectionTimeout),
|
||||
}
|
||||
|
@ -106,8 +108,8 @@ func (s *Server) State() string {
|
|||
}
|
||||
|
||||
// Retrieves the number of member servers in the consensus.
|
||||
func (s *Server) MemberCount() uint64 {
|
||||
var count uint64 = 1
|
||||
func (s *Server) MemberCount() int {
|
||||
count := 1
|
||||
for _, _ = range s.peers {
|
||||
count++
|
||||
}
|
||||
|
@ -115,7 +117,7 @@ func (s *Server) MemberCount() uint64 {
|
|||
}
|
||||
|
||||
// Retrieves the number of servers required to make a quorum.
|
||||
func (s *Server) QuorumSize() uint64 {
|
||||
func (s *Server) QuorumSize() int {
|
||||
return (s.MemberCount() / 2) + 1
|
||||
}
|
||||
|
||||
|
@ -173,21 +175,6 @@ func (s *Server) Running() bool {
|
|||
return s.state != Stopped
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Handlers
|
||||
//--------------------------------------
|
||||
|
||||
// Executes the handler for executing a command.
|
||||
func (s *Server) executeDoHandler(peer *Peer, command Command) error {
|
||||
if s.DoHandler == nil {
|
||||
return errors.New("raft.Server: DoHandler not registered")
|
||||
} else if peer == nil {
|
||||
return errors.New("raft.Server: Peer required")
|
||||
}
|
||||
|
||||
return s.DoHandler(s, peer, command)
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Commands
|
||||
//--------------------------------------
|
||||
|
@ -196,8 +183,6 @@ func (s *Server) executeDoHandler(peer *Peer, command Command) error {
|
|||
// deserialized each time a new log entry is read. This function will panic
|
||||
// if a command type with the same name already exists.
|
||||
func (s *Server) AddCommandType(command Command) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
s.log.AddCommandType(command)
|
||||
}
|
||||
|
||||
|
@ -212,16 +197,12 @@ func (s *Server) Do(command Command) error {
|
|||
// This function is the low-level interface to execute commands. This function
|
||||
// does not obtain a lock so one must be obtained before executing.
|
||||
func (s *Server) do(command Command) error {
|
||||
warn("[%s] do.1 (%s) %v", s.name, command.CommandName(), command)
|
||||
|
||||
// Send the request to the leader if we're not the leader.
|
||||
if s.state != Leader {
|
||||
// TODO: If we don't have a leader then elect one.
|
||||
return s.executeDoHandler(s.leader, command)
|
||||
}
|
||||
|
||||
warn("[%s] do.2", s.name)
|
||||
|
||||
// If we are the leader then create a new log entry.
|
||||
commitIndex := s.log.CommitIndex()
|
||||
prevLogIndex, prevLogTerm := s.log.CurrentIndex(), s.log.CurrentTerm()
|
||||
|
@ -244,7 +225,7 @@ func (s *Server) do(command Command) error {
|
|||
}
|
||||
|
||||
// Send response through the channel that gets collected below.
|
||||
if resp, err := s.AppendEntriesHandler(s, request); err != nil {
|
||||
if resp, err := s.AppendEntriesHandler(s, peer, request); err != nil {
|
||||
resp.peer = peer
|
||||
warn("raft.Server: Error in AppendEntriesHandler: %v", err)
|
||||
} else {
|
||||
|
@ -258,7 +239,7 @@ func (s *Server) do(command Command) error {
|
|||
timeoutChannel, success := time.After(DefaultElectionTimeout), false
|
||||
for {
|
||||
// If we have reached a quorum then exit.
|
||||
var voteCount uint64 = 1
|
||||
voteCount := 1
|
||||
for _, _ = range votes {
|
||||
voteCount++
|
||||
}
|
||||
|
@ -295,6 +276,14 @@ func (s *Server) do(command Command) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// Executes the handler for doing a command on a particular peer.
|
||||
func (s *Server) executeDoHandler(peer *Peer, command Command) error {
|
||||
if s.DoHandler == nil {
|
||||
panic("raft.Server: DoHandler not registered")
|
||||
}
|
||||
return s.DoHandler(s, peer, command)
|
||||
}
|
||||
|
||||
// Appends a log entry from the leader to this server.
|
||||
func (s *Server) AppendEntries(req *AppendEntriesRequest) (*AppendEntriesResponse, error) {
|
||||
s.mutex.Lock()
|
||||
|
@ -329,7 +318,125 @@ func (s *Server) AppendEntries(req *AppendEntriesRequest) (*AppendEntriesRespons
|
|||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Elections
|
||||
// Promotion
|
||||
//--------------------------------------
|
||||
|
||||
// Promotes the server to a candidate and then requests votes from peers. If
|
||||
// enough votes are received then the server becomes the leader. If this
|
||||
// server is elected then true is returned. If another server is elected then
|
||||
// false is returned.
|
||||
func (s *Server) promote() bool {
|
||||
for {
|
||||
// Start a new election.
|
||||
term, lastLogIndex, lastLogTerm := s.promoteToCandidate()
|
||||
|
||||
// Request votes from each of our peers.
|
||||
c := make(chan *RequestVoteResponse, len(s.peers))
|
||||
for _, _peer := range s.peers {
|
||||
peer := _peer
|
||||
go func() {
|
||||
req := NewRequestVoteRequest(term, s.name, lastLogIndex, lastLogTerm)
|
||||
req.peer = peer
|
||||
resp, _ := s.executeRequestVoteHandler(peer, req)
|
||||
resp.peer = peer
|
||||
c <- resp
|
||||
}()
|
||||
}
|
||||
|
||||
// Collect votes until we have a quorum.
|
||||
votes := map[string]bool{}
|
||||
elected := false
|
||||
timeout := time.After(DefaultElectionTimeout)
|
||||
for {
|
||||
// Add up all our votes.
|
||||
votesGranted := 1
|
||||
for _, value := range votes {
|
||||
if value {
|
||||
votesGranted++
|
||||
}
|
||||
}
|
||||
// If we received enough votes then stop waiting for more votes.
|
||||
if votesGranted >= s.QuorumSize() {
|
||||
elected = true
|
||||
break
|
||||
}
|
||||
|
||||
// Collect votes from peers.
|
||||
select {
|
||||
case resp := <-c:
|
||||
if resp != nil && resp.peer != nil {
|
||||
votes[resp.peer.Name()] = resp.VoteGranted
|
||||
}
|
||||
case <-timeout:
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// If we received enough votes then promote to leader and stop this election.
|
||||
if elected && s.promoteToLeader(term, lastLogIndex, lastLogTerm) {
|
||||
break
|
||||
}
|
||||
|
||||
// If we are no longer in the same term then another server must have been elected.
|
||||
if s.currentTerm != term {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// Promotes the server to a candidate and increases the election term. The
|
||||
// term and log state are returned for use in the RPCs.
|
||||
func (s *Server) promoteToCandidate() (term uint64, lastLogIndex uint64, lastLogTerm uint64) {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
// Ignore promotions if we're already the leader or trying to become the leader.
|
||||
if s.state != Follower {
|
||||
return
|
||||
}
|
||||
|
||||
// Move server to become a candidate and increase our term.
|
||||
s.state = Candidate
|
||||
s.currentTerm++
|
||||
|
||||
// Pause the election timer while we're a candidate.
|
||||
s.electionTimer.Pause()
|
||||
|
||||
// Return server state so we can check for it during leader promotion.
|
||||
lastLogIndex, lastLogTerm = s.log.CommitInfo()
|
||||
return s.currentTerm, lastLogIndex, lastLogTerm
|
||||
}
|
||||
|
||||
// Promotes the server from a candidate to a leader. This can only occur if
|
||||
// the server is in the state that it assumed when the candidate election
|
||||
// began. This is because another server may have won the election and caused
|
||||
// the state to change.
|
||||
func (s *Server) promoteToLeader(term uint64, lastLogIndex uint64, lastLogTerm uint64) bool {
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
// Ignore promotion if we are not a candidate.
|
||||
if s.state != Candidate {
|
||||
return false
|
||||
}
|
||||
|
||||
// Disallow promotion if the term or log does not match what we currently have.
|
||||
logIndex, logTerm := s.log.CommitInfo()
|
||||
if s.currentTerm != term || logIndex != lastLogIndex || logTerm != lastLogTerm {
|
||||
return false
|
||||
}
|
||||
|
||||
// Move server to become a leader.
|
||||
s.state = Leader
|
||||
// TODO: Begin heartbeat to peers.
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Request Vote
|
||||
//--------------------------------------
|
||||
|
||||
// Requests a vote from a server. A vote can be obtained if the vote's term is
|
||||
|
@ -362,6 +469,14 @@ func (s *Server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse {
|
|||
return NewRequestVoteResponse(s.currentTerm, true)
|
||||
}
|
||||
|
||||
// Executes the handler for sending a RequestVote RPC.
|
||||
func (s *Server) executeRequestVoteHandler(peer *Peer, req *RequestVoteRequest) (*RequestVoteResponse, error) {
|
||||
if s.RequestVoteHandler == nil {
|
||||
panic("raft.Server: RequestVoteHandler not registered")
|
||||
}
|
||||
return s.RequestVoteHandler(s, peer, req)
|
||||
}
|
||||
|
||||
// Updates the current term on the server if the term is greater than the
|
||||
// server's current term. When the term is changed then the server's vote is
|
||||
// cleared and its state is changed to be a follower.
|
||||
|
|
|
@ -92,6 +92,31 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Promotion
|
||||
//--------------------------------------
|
||||
|
||||
// Ensure that we can self-promote a server to candidate, obtain votes and become a fearless leader.
|
||||
func TestServerPromoteSelf(t *testing.T) {
|
||||
server := newTestServer("1")
|
||||
server.Start()
|
||||
if success := server.promote(); !(success && server.state == Leader) {
|
||||
t.Fatalf("Server self-promotion failed: %v", server.state)
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure that we can promote a server within a cluster to a leader.
|
||||
func TestServerPromote(t *testing.T) {
|
||||
servers, lookup := newTestCluster([]string{"1", "2", "3"})
|
||||
servers.SetRequestVoteHandler(func(server *Server, peer *Peer, req *RequestVoteRequest) (*RequestVoteResponse, error) {
|
||||
return lookup[peer.Name()].RequestVote(req), nil
|
||||
})
|
||||
leader := servers[0]
|
||||
if success := leader.promote(); !(success && leader.state == Leader) {
|
||||
t.Fatalf("Server promotion in cluster failed: %v", leader.state)
|
||||
}
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Append Entries
|
||||
//--------------------------------------
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
package raft
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
//
|
||||
// Typedefs
|
||||
//
|
||||
//------------------------------------------------------------------------------
|
||||
|
||||
// A collection of servers.
|
||||
type Servers []*Server
|
||||
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
//
|
||||
// Methods
|
||||
//
|
||||
//------------------------------------------------------------------------------
|
||||
|
||||
//--------------------------------------
|
||||
// Handlers
|
||||
//--------------------------------------
|
||||
|
||||
// Sets the ApplyFunc handler for a set of servers.
|
||||
func (s Servers) SetApplyFunc(f func(*Server, Command)) {
|
||||
for _, server := range s {
|
||||
server.ApplyFunc = f
|
||||
}
|
||||
}
|
||||
|
||||
// Sets the RequestVoteHandler for a set of servers.
|
||||
func (s Servers) SetRequestVoteHandler(f func(*Server, *Peer, *RequestVoteRequest) (*RequestVoteResponse, error)) {
|
||||
for _, server := range s {
|
||||
server.RequestVoteHandler = f
|
||||
}
|
||||
}
|
23
test.go
23
test.go
|
@ -1,6 +1,7 @@
|
|||
package raft
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
)
|
||||
|
@ -60,6 +61,28 @@ func newTestServerWithLog(name string, content string) *Server {
|
|||
return server
|
||||
}
|
||||
|
||||
func newTestCluster(names []string) (Servers, map[string]*Server) {
|
||||
servers := make(Servers, 0)
|
||||
lookup := make(map[string]*Server, 0)
|
||||
for _, name := range names {
|
||||
if lookup[name] != nil {
|
||||
panic(fmt.Sprintf("Duplicate server in test cluster! %v", name))
|
||||
}
|
||||
server := newTestServer(name)
|
||||
servers = append(servers, server)
|
||||
lookup[name] = server
|
||||
}
|
||||
for _, server := range servers {
|
||||
for _, peer := range servers {
|
||||
if server != peer {
|
||||
server.peers[peer.Name()] = NewPeer(peer.Name())
|
||||
}
|
||||
}
|
||||
server.Start()
|
||||
}
|
||||
return servers, lookup
|
||||
}
|
||||
|
||||
//--------------------------------------
|
||||
// Command1
|
||||
//--------------------------------------
|
||||
|
|
Loading…
Reference in New Issue