Add simple cluster leader election.

pull/820/head
Ben Johnson 2013-04-30 23:11:23 -06:00
parent c8b9d783b7
commit 458c2d9d7c
7 changed files with 238 additions and 35 deletions

7
log.go
View File

@ -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
}

View File

@ -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})
}
}

View File

@ -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
View File

@ -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.

View File

@ -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
//--------------------------------------

35
servers.go Normal file
View File

@ -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
View File

@ -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
//--------------------------------------