2013-04-14 21:37:33 +00:00
|
|
|
|
package raft
|
|
|
|
|
|
2013-04-17 02:28:08 +00:00
|
|
|
|
import (
|
2013-06-24 16:52:51 +00:00
|
|
|
|
"encoding/json"
|
2013-04-17 02:28:08 +00:00
|
|
|
|
"errors"
|
|
|
|
|
"fmt"
|
2013-06-08 02:19:18 +00:00
|
|
|
|
"io/ioutil"
|
2013-06-03 21:58:12 +00:00
|
|
|
|
"os"
|
2013-06-05 17:38:49 +00:00
|
|
|
|
"path"
|
2014-10-20 17:12:59 +00:00
|
|
|
|
"path/filepath"
|
2013-06-08 02:19:18 +00:00
|
|
|
|
"sort"
|
|
|
|
|
"sync"
|
|
|
|
|
"time"
|
2013-04-17 02:28:08 +00:00
|
|
|
|
)
|
|
|
|
|
|
2013-04-14 21:37:33 +00:00
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Constants
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
|
|
|
|
|
const (
|
2013-07-17 00:40:19 +00:00
|
|
|
|
Stopped = "stopped"
|
2014-03-21 02:10:48 +00:00
|
|
|
|
Initialized = "initialized"
|
2013-07-17 00:40:19 +00:00
|
|
|
|
Follower = "follower"
|
|
|
|
|
Candidate = "candidate"
|
|
|
|
|
Leader = "leader"
|
|
|
|
|
Snapshotting = "snapshotting"
|
2013-04-17 02:32:49 +00:00
|
|
|
|
)
|
|
|
|
|
|
2013-07-15 05:48:41 +00:00
|
|
|
|
const (
|
2013-08-02 00:58:03 +00:00
|
|
|
|
MaxLogEntriesPerRequest = 2000
|
2013-07-18 23:44:01 +00:00
|
|
|
|
NumberOfLogEntriesAfterSnapshot = 200
|
2013-07-15 05:48:41 +00:00
|
|
|
|
)
|
|
|
|
|
|
2013-04-17 02:32:49 +00:00
|
|
|
|
const (
|
2014-01-23 20:50:29 +00:00
|
|
|
|
// DefaultHeartbeatInterval is the interval that the leader will send
|
|
|
|
|
// AppendEntriesRequests to followers to maintain leadership.
|
|
|
|
|
DefaultHeartbeatInterval = 50 * time.Millisecond
|
|
|
|
|
|
|
|
|
|
DefaultElectionTimeout = 150 * time.Millisecond
|
2013-04-14 21:37:33 +00:00
|
|
|
|
)
|
|
|
|
|
|
2013-12-29 19:54:00 +00:00
|
|
|
|
// ElectionTimeoutThresholdPercent specifies the threshold at which the server
|
|
|
|
|
// will dispatch warning events that the heartbeat RTT is too close to the
|
|
|
|
|
// election timeout.
|
2013-12-29 19:54:15 +00:00
|
|
|
|
const ElectionTimeoutThresholdPercent = 0.8
|
2013-12-29 19:54:00 +00:00
|
|
|
|
|
2013-05-28 18:46:27 +00:00
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Errors
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
|
|
|
|
|
var NotLeaderError = errors.New("raft.Server: Not current leader")
|
|
|
|
|
var DuplicatePeerError = errors.New("raft.Server: Duplicate peer")
|
2013-07-07 20:21:04 +00:00
|
|
|
|
var CommandTimeoutError = errors.New("raft: Command timeout")
|
2014-04-01 17:21:31 +00:00
|
|
|
|
var StopError = errors.New("raft: Has been stopped")
|
2013-05-27 02:04:41 +00:00
|
|
|
|
|
2013-04-14 21:37:33 +00:00
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Typedefs
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
|
|
|
|
|
// A server is involved in the consensus protocol and can act as a follower,
|
|
|
|
|
// candidate or a leader.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
type Server interface {
|
|
|
|
|
Name() string
|
|
|
|
|
Context() interface{}
|
|
|
|
|
StateMachine() StateMachine
|
|
|
|
|
Leader() string
|
|
|
|
|
State() string
|
|
|
|
|
Path() string
|
|
|
|
|
LogPath() string
|
2013-10-14 19:33:01 +00:00
|
|
|
|
SnapshotPath(lastIndex uint64, lastTerm uint64) string
|
2013-10-14 17:54:49 +00:00
|
|
|
|
Term() uint64
|
|
|
|
|
CommitIndex() uint64
|
|
|
|
|
VotedFor() string
|
|
|
|
|
MemberCount() int
|
|
|
|
|
QuorumSize() int
|
|
|
|
|
IsLogEmpty() bool
|
2013-10-14 19:40:20 +00:00
|
|
|
|
LogEntries() []*LogEntry
|
|
|
|
|
LastCommandName() string
|
|
|
|
|
GetState() string
|
2013-10-14 17:54:49 +00:00
|
|
|
|
ElectionTimeout() time.Duration
|
|
|
|
|
SetElectionTimeout(duration time.Duration)
|
2014-01-23 20:50:29 +00:00
|
|
|
|
HeartbeatInterval() time.Duration
|
|
|
|
|
SetHeartbeatInterval(duration time.Duration)
|
2013-10-14 17:54:49 +00:00
|
|
|
|
Transporter() Transporter
|
|
|
|
|
SetTransporter(t Transporter)
|
|
|
|
|
AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse
|
|
|
|
|
RequestVote(req *RequestVoteRequest) *RequestVoteResponse
|
2013-10-14 19:33:01 +00:00
|
|
|
|
RequestSnapshot(req *SnapshotRequest) *SnapshotResponse
|
|
|
|
|
SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse
|
2013-10-14 17:54:49 +00:00
|
|
|
|
AddPeer(name string, connectiongString string) error
|
|
|
|
|
RemovePeer(name string) error
|
|
|
|
|
Peers() map[string]*Peer
|
2014-03-21 02:10:48 +00:00
|
|
|
|
Init() error
|
2013-10-14 17:54:49 +00:00
|
|
|
|
Start() error
|
|
|
|
|
Stop()
|
|
|
|
|
Running() bool
|
|
|
|
|
Do(command Command) (interface{}, error)
|
2013-10-14 19:33:01 +00:00
|
|
|
|
TakeSnapshot() error
|
|
|
|
|
LoadSnapshot() error
|
2013-12-19 23:39:37 +00:00
|
|
|
|
AddEventListener(string, EventListener)
|
2014-03-21 01:13:38 +00:00
|
|
|
|
FlushCommitIndex()
|
2013-10-14 17:54:49 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
type server struct {
|
2013-12-19 23:39:37 +00:00
|
|
|
|
*eventDispatcher
|
|
|
|
|
|
2013-06-24 16:52:51 +00:00
|
|
|
|
name string
|
|
|
|
|
path string
|
|
|
|
|
state string
|
|
|
|
|
transporter Transporter
|
|
|
|
|
context interface{}
|
|
|
|
|
currentTerm uint64
|
|
|
|
|
|
2013-07-10 23:07:14 +00:00
|
|
|
|
votedFor string
|
|
|
|
|
log *Log
|
|
|
|
|
leader string
|
|
|
|
|
peers map[string]*Peer
|
|
|
|
|
mutex sync.RWMutex
|
|
|
|
|
syncedPeer map[string]bool
|
2013-06-13 18:03:32 +00:00
|
|
|
|
|
2014-04-01 17:21:31 +00:00
|
|
|
|
stopped chan bool
|
2014-01-23 20:50:29 +00:00
|
|
|
|
c chan *ev
|
|
|
|
|
electionTimeout time.Duration
|
|
|
|
|
heartbeatInterval time.Duration
|
2013-06-13 18:03:32 +00:00
|
|
|
|
|
2014-02-27 16:12:39 +00:00
|
|
|
|
snapshot *Snapshot
|
|
|
|
|
|
|
|
|
|
// PendingSnapshot is an unfinished snapshot.
|
|
|
|
|
// After the pendingSnapshot is saved to disk,
|
|
|
|
|
// it will be set to snapshot and also will be
|
|
|
|
|
// set to nil.
|
|
|
|
|
pendingSnapshot *Snapshot
|
|
|
|
|
|
2013-07-15 05:48:41 +00:00
|
|
|
|
stateMachine StateMachine
|
|
|
|
|
maxLogEntriesPerRequest uint64
|
2013-09-18 04:19:46 +00:00
|
|
|
|
|
|
|
|
|
connectionString string
|
2014-04-01 17:21:31 +00:00
|
|
|
|
|
|
|
|
|
routineGroup sync.WaitGroup
|
2013-04-14 21:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-12-19 23:39:37 +00:00
|
|
|
|
// An internal event to be processed by the server's event loop.
|
|
|
|
|
type ev struct {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
target interface{}
|
|
|
|
|
returnValue interface{}
|
|
|
|
|
c chan error
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-14 21:37:33 +00:00
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Constructor
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
|
2013-11-24 23:15:02 +00:00
|
|
|
|
// Creates a new server with a log at the given path. transporter must
|
|
|
|
|
// not be nil. stateMachine can be nil if snapshotting and log
|
|
|
|
|
// compaction is to be disabled. context can be anything (including nil)
|
|
|
|
|
// and is not used by the raft package except returned by
|
|
|
|
|
// Server.Context(). connectionString can be anything.
|
2013-12-23 14:16:37 +00:00
|
|
|
|
func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, ctx interface{}, connectionString string) (Server, error) {
|
2013-04-17 02:28:08 +00:00
|
|
|
|
if name == "" {
|
|
|
|
|
return nil, errors.New("raft.Server: Name cannot be blank")
|
|
|
|
|
}
|
2013-05-28 19:57:38 +00:00
|
|
|
|
if transporter == nil {
|
2013-06-08 02:41:36 +00:00
|
|
|
|
panic("raft: Transporter required")
|
2013-05-28 19:57:38 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
s := &server{
|
2013-07-15 05:48:41 +00:00
|
|
|
|
name: name,
|
|
|
|
|
path: path,
|
|
|
|
|
transporter: transporter,
|
|
|
|
|
stateMachine: stateMachine,
|
2013-12-23 14:16:37 +00:00
|
|
|
|
context: ctx,
|
2013-07-15 05:48:41 +00:00
|
|
|
|
state: Stopped,
|
|
|
|
|
peers: make(map[string]*Peer),
|
|
|
|
|
log: newLog(),
|
2013-12-19 23:39:37 +00:00
|
|
|
|
c: make(chan *ev, 256),
|
2013-07-15 05:48:41 +00:00
|
|
|
|
electionTimeout: DefaultElectionTimeout,
|
2014-01-23 20:50:29 +00:00
|
|
|
|
heartbeatInterval: DefaultHeartbeatInterval,
|
2013-07-15 05:48:41 +00:00
|
|
|
|
maxLogEntriesPerRequest: MaxLogEntriesPerRequest,
|
2013-09-22 04:08:18 +00:00
|
|
|
|
connectionString: connectionString,
|
2013-04-14 21:37:33 +00:00
|
|
|
|
}
|
2013-12-19 23:39:37 +00:00
|
|
|
|
s.eventDispatcher = newEventDispatcher(s)
|
2013-04-30 04:13:50 +00:00
|
|
|
|
|
|
|
|
|
// Setup apply function.
|
2014-02-10 16:36:17 +00:00
|
|
|
|
s.log.ApplyFunc = func(e *LogEntry, c Command) (interface{}, error) {
|
|
|
|
|
// Dispatch commit event.
|
|
|
|
|
s.DispatchEvent(newEvent(CommitEventType, e, nil))
|
|
|
|
|
|
|
|
|
|
// Apply command to the state machine.
|
2013-12-23 14:16:37 +00:00
|
|
|
|
switch c := c.(type) {
|
|
|
|
|
case CommandApply:
|
2013-12-23 23:07:40 +00:00
|
|
|
|
return c.Apply(&context{
|
|
|
|
|
server: s,
|
|
|
|
|
currentTerm: s.currentTerm,
|
2013-12-27 22:23:59 +00:00
|
|
|
|
currentIndex: s.log.internalCurrentIndex(),
|
2013-12-23 23:07:40 +00:00
|
|
|
|
commitIndex: s.log.commitIndex,
|
|
|
|
|
})
|
2013-12-23 14:16:37 +00:00
|
|
|
|
case deprecatedCommandApply:
|
|
|
|
|
return c.Apply(s)
|
|
|
|
|
default:
|
|
|
|
|
return nil, fmt.Errorf("Command does not implement Apply()")
|
|
|
|
|
}
|
2013-04-30 04:13:50 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-04-17 02:28:08 +00:00
|
|
|
|
return s, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Accessors
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
|
2013-05-01 05:21:56 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// General
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-04-17 02:28:08 +00:00
|
|
|
|
// Retrieves the name of the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Name() string {
|
2013-04-17 02:28:08 +00:00
|
|
|
|
return s.name
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Retrieves the storage path for the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Path() string {
|
2013-04-17 02:28:08 +00:00
|
|
|
|
return s.path
|
|
|
|
|
}
|
2013-05-05 19:36:23 +00:00
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// The name of the current leader.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Leader() string {
|
2013-06-03 01:18:25 +00:00
|
|
|
|
return s.leader
|
2013-06-08 02:19:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-06-26 18:25:22 +00:00
|
|
|
|
// Retrieves a copy of the peer data.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Peers() map[string]*Peer {
|
2013-06-26 18:25:22 +00:00
|
|
|
|
s.mutex.Lock()
|
|
|
|
|
defer s.mutex.Unlock()
|
|
|
|
|
|
|
|
|
|
peers := make(map[string]*Peer)
|
|
|
|
|
for name, peer := range s.peers {
|
|
|
|
|
peers[name] = peer.clone()
|
|
|
|
|
}
|
|
|
|
|
return peers
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-28 19:57:38 +00:00
|
|
|
|
// Retrieves the object that transports requests.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Transporter() Transporter {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-05-28 19:57:38 +00:00
|
|
|
|
return s.transporter
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) SetTransporter(t Transporter) {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.Lock()
|
|
|
|
|
defer s.mutex.Unlock()
|
2013-06-11 22:30:13 +00:00
|
|
|
|
s.transporter = t
|
|
|
|
|
}
|
|
|
|
|
|
2013-06-03 02:43:40 +00:00
|
|
|
|
// Retrieves the context passed into the constructor.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Context() interface{} {
|
2013-06-03 02:43:40 +00:00
|
|
|
|
return s.context
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-10 03:15:57 +00:00
|
|
|
|
// Retrieves the state machine passed into the constructor.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) StateMachine() StateMachine {
|
2013-10-10 03:15:57 +00:00
|
|
|
|
return s.stateMachine
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-17 02:28:08 +00:00
|
|
|
|
// Retrieves the log path for the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) LogPath() string {
|
2013-07-23 22:30:14 +00:00
|
|
|
|
return path.Join(s.path, "log")
|
2013-04-17 02:28:08 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Retrieves the current state of the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) State() string {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-04-17 02:28:08 +00:00
|
|
|
|
return s.state
|
2013-04-14 21:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 22:12:24 +00:00
|
|
|
|
// Sets the state of the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) setState(state string) {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.Lock()
|
|
|
|
|
defer s.mutex.Unlock()
|
2013-12-19 23:39:37 +00:00
|
|
|
|
|
|
|
|
|
// Temporarily store previous values.
|
|
|
|
|
prevState := s.state
|
|
|
|
|
prevLeader := s.leader
|
|
|
|
|
|
|
|
|
|
// Update state and leader.
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.state = state
|
2013-07-08 00:01:55 +00:00
|
|
|
|
if state == Leader {
|
|
|
|
|
s.leader = s.Name()
|
2014-01-12 08:01:38 +00:00
|
|
|
|
s.syncedPeer = make(map[string]bool)
|
2013-07-08 00:01:55 +00:00
|
|
|
|
}
|
2013-12-19 23:39:37 +00:00
|
|
|
|
|
|
|
|
|
// Dispatch state and leader change events.
|
2014-02-16 00:14:24 +00:00
|
|
|
|
s.DispatchEvent(newEvent(StateChangeEventType, s.state, prevState))
|
|
|
|
|
|
2013-12-19 23:39:37 +00:00
|
|
|
|
if prevLeader != s.leader {
|
|
|
|
|
s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
|
|
|
|
|
}
|
2013-07-07 22:12:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-06-10 04:47:59 +00:00
|
|
|
|
// Retrieves the current term of the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Term() uint64 {
|
2013-10-22 22:39:32 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-06-10 04:47:59 +00:00
|
|
|
|
return s.currentTerm
|
2013-04-14 21:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// Retrieves the current commit index of the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) CommitIndex() uint64 {
|
2013-10-22 22:39:32 +00:00
|
|
|
|
s.log.mutex.RLock()
|
|
|
|
|
defer s.log.mutex.RUnlock()
|
2013-07-06 04:49:47 +00:00
|
|
|
|
return s.log.commitIndex
|
2013-06-28 23:14:41 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-05-03 04:16:39 +00:00
|
|
|
|
// Retrieves the name of the candidate this server voted for in this term.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) VotedFor() string {
|
2013-05-03 04:16:39 +00:00
|
|
|
|
return s.votedFor
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-08 03:56:32 +00:00
|
|
|
|
// Retrieves whether the server's log has no entries.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) IsLogEmpty() bool {
|
2013-07-06 04:49:47 +00:00
|
|
|
|
return s.log.isEmpty()
|
2013-05-08 03:56:32 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-05-08 20:22:08 +00:00
|
|
|
|
// A list of all the log entries. This should only be used for debugging purposes.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) LogEntries() []*LogEntry {
|
2014-04-10 17:21:34 +00:00
|
|
|
|
s.log.mutex.RLock()
|
|
|
|
|
defer s.log.mutex.RUnlock()
|
2013-07-06 19:41:42 +00:00
|
|
|
|
return s.log.entries
|
2013-05-08 20:22:08 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-06-03 19:13:38 +00:00
|
|
|
|
// A reference to the command name of the last entry.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) LastCommandName() string {
|
2013-07-06 19:41:42 +00:00
|
|
|
|
return s.log.lastCommandName()
|
2013-06-03 19:13:38 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-02 18:42:14 +00:00
|
|
|
|
// Get the state of the server for debugging
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) GetState() string {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-08-03 02:00:11 +00:00
|
|
|
|
return fmt.Sprintf("Name: %s, State: %s, Term: %v, CommitedIndex: %v ", s.name, s.state, s.currentTerm, s.log.commitIndex)
|
2013-07-02 18:42:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-25 23:16:06 +00:00
|
|
|
|
// Check if the server is promotable
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) promotable() bool {
|
2013-07-25 23:16:06 +00:00
|
|
|
|
return s.log.currentIndex() > 0
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-01 05:21:56 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Membership
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-04-28 04:51:17 +00:00
|
|
|
|
// Retrieves the number of member servers in the consensus.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) MemberCount() int {
|
2014-03-10 20:09:51 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-06-22 15:35:30 +00:00
|
|
|
|
return len(s.peers) + 1
|
2013-04-28 04:51:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Retrieves the number of servers required to make a quorum.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) QuorumSize() int {
|
2013-04-28 04:51:17 +00:00
|
|
|
|
return (s.MemberCount() / 2) + 1
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-01 05:21:56 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Election timeout
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
|
|
|
|
// Retrieves the election timeout.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) ElectionTimeout() time.Duration {
|
2013-08-07 04:02:30 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2013-07-07 22:12:24 +00:00
|
|
|
|
return s.electionTimeout
|
2013-05-01 05:21:56 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Sets the election timeout.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) SetElectionTimeout(duration time.Duration) {
|
2013-08-07 04:02:30 +00:00
|
|
|
|
s.mutex.Lock()
|
|
|
|
|
defer s.mutex.Unlock()
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.electionTimeout = duration
|
2013-05-01 05:21:56 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-05-05 21:41:55 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Heartbeat timeout
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
|
|
|
|
// Retrieves the heartbeat timeout.
|
2014-01-23 20:50:29 +00:00
|
|
|
|
func (s *server) HeartbeatInterval() time.Duration {
|
2013-08-07 04:02:30 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2014-01-23 20:50:29 +00:00
|
|
|
|
return s.heartbeatInterval
|
2013-05-05 21:41:55 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Sets the heartbeat timeout.
|
2014-01-23 20:50:29 +00:00
|
|
|
|
func (s *server) SetHeartbeatInterval(duration time.Duration) {
|
2013-05-05 21:41:55 +00:00
|
|
|
|
s.mutex.Lock()
|
|
|
|
|
defer s.mutex.Unlock()
|
|
|
|
|
|
2014-01-23 20:50:29 +00:00
|
|
|
|
s.heartbeatInterval = duration
|
2013-05-05 21:41:55 +00:00
|
|
|
|
for _, peer := range s.peers {
|
2014-01-23 20:50:29 +00:00
|
|
|
|
peer.setHeartbeatInterval(duration)
|
2013-05-05 21:41:55 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-14 21:37:33 +00:00
|
|
|
|
//------------------------------------------------------------------------------
|
|
|
|
|
//
|
|
|
|
|
// Methods
|
|
|
|
|
//
|
|
|
|
|
//------------------------------------------------------------------------------
|
2013-04-17 02:28:08 +00:00
|
|
|
|
|
|
|
|
|
//--------------------------------------
|
2013-06-24 16:52:51 +00:00
|
|
|
|
// Initialization
|
2013-04-17 02:28:08 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-07-11 05:19:57 +00:00
|
|
|
|
// Reg the NOPCommand
|
|
|
|
|
func init() {
|
|
|
|
|
RegisterCommand(&NOPCommand{})
|
2013-07-26 19:13:52 +00:00
|
|
|
|
RegisterCommand(&DefaultJoinCommand{})
|
2013-07-26 20:33:58 +00:00
|
|
|
|
RegisterCommand(&DefaultLeaveCommand{})
|
2013-07-11 05:19:57 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-03-21 02:10:48 +00:00
|
|
|
|
// Start the raft server
|
2013-07-25 22:47:35 +00:00
|
|
|
|
// If log entries exist then allow promotion to candidate if no AEs received.
|
|
|
|
|
// If no log entries exist then wait for AEs from another node.
|
|
|
|
|
// If no log entries exist and a self-join command is issued then
|
|
|
|
|
// immediately become leader and commit entry.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Start() error {
|
2013-07-25 21:26:27 +00:00
|
|
|
|
// Exit if the server is already running.
|
2014-03-21 02:10:48 +00:00
|
|
|
|
if s.Running() {
|
|
|
|
|
return fmt.Errorf("raft.Server: Server already running[%v]", s.state)
|
2013-07-25 21:26:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-03-21 02:10:48 +00:00
|
|
|
|
if err := s.Init(); err != nil {
|
|
|
|
|
return err
|
2013-07-25 21:26:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-01 17:21:31 +00:00
|
|
|
|
// stopped needs to be allocated each time server starts
|
|
|
|
|
// because it is closed at `Stop`.
|
|
|
|
|
s.stopped = make(chan bool)
|
2013-07-25 21:26:27 +00:00
|
|
|
|
s.setState(Follower)
|
|
|
|
|
|
2013-07-25 22:40:20 +00:00
|
|
|
|
// If no log entries exist then
|
|
|
|
|
// 1. wait for AEs from another node
|
|
|
|
|
// 2. wait for self-join command
|
|
|
|
|
// to set itself promotable
|
2013-07-25 23:16:06 +00:00
|
|
|
|
if !s.promotable() {
|
2013-07-25 21:26:27 +00:00
|
|
|
|
s.debugln("start as a new raft server")
|
|
|
|
|
|
2013-07-25 22:40:20 +00:00
|
|
|
|
// If log entries exist then allow promotion to candidate
|
|
|
|
|
// if no AEs received.
|
2013-07-25 21:26:27 +00:00
|
|
|
|
} else {
|
|
|
|
|
s.debugln("start from previous saved state")
|
|
|
|
|
}
|
|
|
|
|
|
2013-08-03 02:00:11 +00:00
|
|
|
|
debugln(s.GetState())
|
|
|
|
|
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Add(1)
|
|
|
|
|
go func() {
|
|
|
|
|
defer s.routineGroup.Done()
|
|
|
|
|
s.loop()
|
|
|
|
|
}()
|
2013-07-25 21:26:27 +00:00
|
|
|
|
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-10 03:21:31 +00:00
|
|
|
|
// Init initializes the raft server.
|
|
|
|
|
// If there is no previous log file under the given path, Init() will create an empty log file.
|
|
|
|
|
// Otherwise, Init() will load in the log entries from the log file.
|
2014-03-21 02:10:48 +00:00
|
|
|
|
func (s *server) Init() error {
|
|
|
|
|
if s.Running() {
|
|
|
|
|
return fmt.Errorf("raft.Server: Server already running[%v]", s.state)
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-04 01:08:36 +00:00
|
|
|
|
// Server has been initialized or server was stopped after initialized
|
|
|
|
|
// If log has been initialized, we know that the server was stopped after
|
|
|
|
|
// running.
|
|
|
|
|
if s.state == Initialized || s.log.initialized {
|
2014-03-21 02:10:48 +00:00
|
|
|
|
s.state = Initialized
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Create snapshot directory if it does not exist
|
2014-03-22 17:30:14 +00:00
|
|
|
|
err := os.Mkdir(path.Join(s.path, "snapshot"), 0700)
|
2014-03-22 23:19:00 +00:00
|
|
|
|
if err != nil && !os.IsExist(err) {
|
2014-03-21 02:10:48 +00:00
|
|
|
|
s.debugln("raft: Snapshot dir error: ", err)
|
|
|
|
|
return fmt.Errorf("raft: Initialization error: %s", err)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if err := s.readConf(); err != nil {
|
|
|
|
|
s.debugln("raft: Conf file error: ", err)
|
|
|
|
|
return fmt.Errorf("raft: Initialization error: %s", err)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Initialize the log and load it up.
|
|
|
|
|
if err := s.log.open(s.LogPath()); err != nil {
|
|
|
|
|
s.debugln("raft: Log error: ", err)
|
|
|
|
|
return fmt.Errorf("raft: Initialization error: %s", err)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Update the term to the last term in the log.
|
|
|
|
|
_, s.currentTerm = s.log.lastInfo()
|
|
|
|
|
|
|
|
|
|
s.state = Initialized
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2013-07-03 16:53:46 +00:00
|
|
|
|
// Shuts down the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Stop() {
|
2014-03-25 21:58:38 +00:00
|
|
|
|
if s.State() == Stopped {
|
|
|
|
|
return
|
2014-04-01 17:21:31 +00:00
|
|
|
|
}
|
2014-03-25 21:58:38 +00:00
|
|
|
|
|
2014-04-04 21:43:02 +00:00
|
|
|
|
close(s.stopped)
|
2013-11-27 05:30:03 +00:00
|
|
|
|
|
2014-04-04 23:36:55 +00:00
|
|
|
|
// make sure all goroutines have stopped before we close the log
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Wait()
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.log.close()
|
2014-03-24 19:02:15 +00:00
|
|
|
|
s.setState(Stopped)
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Checks if the server is currently running.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Running() bool {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.mutex.RLock()
|
|
|
|
|
defer s.mutex.RUnlock()
|
2014-03-21 02:10:48 +00:00
|
|
|
|
return (s.state != Stopped && s.state != Initialized)
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Term
|
|
|
|
|
//--------------------------------------
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2014-02-16 00:14:24 +00:00
|
|
|
|
// updates the current term for the server. This is only used when a larger
|
|
|
|
|
// external term is found.
|
|
|
|
|
func (s *server) updateCurrentTerm(term uint64, leaderName string) {
|
2014-02-18 00:51:01 +00:00
|
|
|
|
_assert(term > s.currentTerm,
|
|
|
|
|
"upadteCurrentTerm: update is called when term is not larger than currentTerm")
|
|
|
|
|
|
2013-12-19 23:39:37 +00:00
|
|
|
|
// Store previous values temporarily.
|
|
|
|
|
prevTerm := s.currentTerm
|
|
|
|
|
prevLeader := s.leader
|
|
|
|
|
|
2014-02-16 00:14:24 +00:00
|
|
|
|
// set currentTerm = T, convert to follower (§5.1)
|
|
|
|
|
// stop heartbeats before step-down
|
|
|
|
|
if s.state == Leader {
|
|
|
|
|
for _, peer := range s.peers {
|
|
|
|
|
peer.stopHeartbeat(false)
|
2014-01-27 14:05:54 +00:00
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
2014-02-16 00:14:24 +00:00
|
|
|
|
// update the term and clear vote for
|
|
|
|
|
if s.state != Follower {
|
|
|
|
|
s.setState(Follower)
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
2014-03-10 20:09:51 +00:00
|
|
|
|
|
|
|
|
|
s.mutex.Lock()
|
2014-02-16 00:14:24 +00:00
|
|
|
|
s.currentTerm = term
|
|
|
|
|
s.leader = leaderName
|
|
|
|
|
s.votedFor = ""
|
2014-03-10 20:09:51 +00:00
|
|
|
|
s.mutex.Unlock()
|
2013-07-26 00:49:01 +00:00
|
|
|
|
|
2013-12-19 23:39:37 +00:00
|
|
|
|
// Dispatch change events.
|
2014-02-16 00:14:24 +00:00
|
|
|
|
s.DispatchEvent(newEvent(TermChangeEventType, s.currentTerm, prevTerm))
|
|
|
|
|
|
2013-12-19 23:39:37 +00:00
|
|
|
|
if prevLeader != s.leader {
|
|
|
|
|
s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
|
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Event Loop
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-07-17 00:40:19 +00:00
|
|
|
|
// ________
|
|
|
|
|
// --|Snapshot| timeout
|
|
|
|
|
// | -------- ______
|
|
|
|
|
// recover | ^ | |
|
|
|
|
|
// snapshot / | |snapshot | |
|
|
|
|
|
// higher | | v | recv majority votes
|
|
|
|
|
// term | -------- timeout ----------- -----------
|
|
|
|
|
// |-> |Follower| ----------> | Candidate |--------------------> | Leader |
|
|
|
|
|
// -------- ----------- -----------
|
|
|
|
|
// ^ higher term/ | higher term |
|
|
|
|
|
// | new leader | |
|
|
|
|
|
// |_______________________|____________________________________ |
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// The main event loop for the server
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) loop() {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
defer s.debugln("server.loop.end")
|
2013-07-06 19:41:42 +00:00
|
|
|
|
|
2014-03-24 19:32:39 +00:00
|
|
|
|
state := s.State()
|
2013-07-07 23:37:11 +00:00
|
|
|
|
|
2014-03-24 19:32:39 +00:00
|
|
|
|
for state != Stopped {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
s.debugln("server.loop.run ", state)
|
|
|
|
|
switch state {
|
2013-07-06 19:41:42 +00:00
|
|
|
|
case Follower:
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.followerLoop()
|
2013-07-06 19:41:42 +00:00
|
|
|
|
case Candidate:
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.candidateLoop()
|
2013-07-06 19:41:42 +00:00
|
|
|
|
case Leader:
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.leaderLoop()
|
2013-07-17 00:40:19 +00:00
|
|
|
|
case Snapshotting:
|
|
|
|
|
s.snapshotLoop()
|
2013-07-06 19:41:42 +00:00
|
|
|
|
}
|
2014-03-24 19:32:39 +00:00
|
|
|
|
state = s.State()
|
2013-07-06 19:41:42 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Sends an event to the event loop to be processed. The function will wait
|
|
|
|
|
// until the event is actually processed before returning.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) send(value interface{}) (interface{}, error) {
|
2014-04-06 17:19:13 +00:00
|
|
|
|
if !s.Running() {
|
|
|
|
|
return nil, StopError
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-09 10:18:06 +00:00
|
|
|
|
event := &ev{target: value, c: make(chan error, 1)}
|
2014-04-04 21:56:33 +00:00
|
|
|
|
select {
|
|
|
|
|
case s.c <- event:
|
|
|
|
|
case <-s.stopped:
|
|
|
|
|
return nil, StopError
|
|
|
|
|
}
|
2014-04-01 17:21:31 +00:00
|
|
|
|
select {
|
|
|
|
|
case <-s.stopped:
|
|
|
|
|
return nil, StopError
|
|
|
|
|
case err := <-event.c:
|
|
|
|
|
return event.returnValue, err
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-11-27 04:36:42 +00:00
|
|
|
|
func (s *server) sendAsync(value interface{}) {
|
2014-04-06 17:19:13 +00:00
|
|
|
|
if !s.Running() {
|
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-09 10:54:58 +00:00
|
|
|
|
event := &ev{target: value, c: make(chan error, 1)}
|
|
|
|
|
// try a non-blocking send first
|
|
|
|
|
// in most cases, this should not be blocking
|
|
|
|
|
// avoid create unnecessary go routines
|
|
|
|
|
select {
|
|
|
|
|
case s.c <- event:
|
|
|
|
|
return
|
|
|
|
|
default:
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Add(1)
|
2013-11-27 04:36:42 +00:00
|
|
|
|
go func() {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
defer s.routineGroup.Done()
|
2014-04-04 21:56:33 +00:00
|
|
|
|
select {
|
|
|
|
|
case s.c <- event:
|
|
|
|
|
case <-s.stopped:
|
|
|
|
|
}
|
2013-11-27 04:36:42 +00:00
|
|
|
|
}()
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// The event loop that is run when the server is in a Follower state.
|
|
|
|
|
// Responds to RPCs from candidates and leaders.
|
|
|
|
|
// Converts to candidate if election timeout elapses without either:
|
|
|
|
|
// 1.Receiving valid AppendEntries RPC, or
|
|
|
|
|
// 2.Granting vote to candidate
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) followerLoop() {
|
2013-12-29 19:54:00 +00:00
|
|
|
|
since := time.Now()
|
|
|
|
|
electionTimeout := s.ElectionTimeout()
|
2013-07-08 04:31:58 +00:00
|
|
|
|
timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2014-01-28 15:58:23 +00:00
|
|
|
|
for s.State() == Follower {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
var err error
|
2013-07-12 01:03:24 +00:00
|
|
|
|
update := false
|
2013-07-07 20:21:04 +00:00
|
|
|
|
select {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
case <-s.stopped:
|
2014-02-16 01:16:53 +00:00
|
|
|
|
s.setState(Stopped)
|
|
|
|
|
return
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
case e := <-s.c:
|
2014-02-16 01:16:53 +00:00
|
|
|
|
switch req := e.target.(type) {
|
|
|
|
|
case JoinCommand:
|
|
|
|
|
//If no log entries exist and a self-join command is issued
|
|
|
|
|
//then immediately become leader and commit entry.
|
|
|
|
|
if s.log.currentIndex() == 0 && req.NodeName() == s.Name() {
|
|
|
|
|
s.debugln("selfjoin and promote to leader")
|
|
|
|
|
s.setState(Leader)
|
|
|
|
|
s.processCommand(req, e)
|
|
|
|
|
} else {
|
2013-07-25 21:26:27 +00:00
|
|
|
|
err = NotLeaderError
|
|
|
|
|
}
|
2014-02-16 01:16:53 +00:00
|
|
|
|
case *AppendEntriesRequest:
|
|
|
|
|
// If heartbeats get too close to the election timeout then send an event.
|
|
|
|
|
elapsedTime := time.Now().Sub(since)
|
|
|
|
|
if elapsedTime > time.Duration(float64(electionTimeout)*ElectionTimeoutThresholdPercent) {
|
|
|
|
|
s.DispatchEvent(newEvent(ElectionTimeoutThresholdEventType, elapsedTime, nil))
|
|
|
|
|
}
|
|
|
|
|
e.returnValue, update = s.processAppendEntriesRequest(req)
|
|
|
|
|
case *RequestVoteRequest:
|
|
|
|
|
e.returnValue, update = s.processRequestVoteRequest(req)
|
|
|
|
|
case *SnapshotRequest:
|
|
|
|
|
e.returnValue = s.processSnapshotRequest(req)
|
|
|
|
|
default:
|
|
|
|
|
err = NotLeaderError
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
|
|
|
|
// Callback to event.
|
|
|
|
|
e.c <- err
|
2013-07-07 20:55:55 +00:00
|
|
|
|
|
2013-07-08 04:31:58 +00:00
|
|
|
|
case <-timeoutChan:
|
2013-07-12 01:03:24 +00:00
|
|
|
|
// only allow synced follower to promote to candidate
|
2013-07-25 23:16:06 +00:00
|
|
|
|
if s.promotable() {
|
2013-07-12 01:03:24 +00:00
|
|
|
|
s.setState(Candidate)
|
|
|
|
|
} else {
|
|
|
|
|
update = true
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
2013-07-07 20:55:55 +00:00
|
|
|
|
|
2013-07-08 04:31:58 +00:00
|
|
|
|
// Converts to candidate if election timeout elapses without either:
|
|
|
|
|
// 1.Receiving valid AppendEntries RPC, or
|
|
|
|
|
// 2.Granting vote to candidate
|
|
|
|
|
if update {
|
2013-12-29 19:54:00 +00:00
|
|
|
|
since = time.Now()
|
2013-07-09 03:00:14 +00:00
|
|
|
|
timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
|
2013-07-08 04:31:58 +00:00
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// The event loop that is run when the server is in a Candidate state.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) candidateLoop() {
|
2013-12-19 23:39:37 +00:00
|
|
|
|
// Clear leader value.
|
|
|
|
|
prevLeader := s.leader
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.leader = ""
|
2013-12-19 23:39:37 +00:00
|
|
|
|
if prevLeader != s.leader {
|
|
|
|
|
s.DispatchEvent(newEvent(LeaderChangeEventType, s.leader, prevLeader))
|
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
lastLogIndex, lastLogTerm := s.log.lastInfo()
|
|
|
|
|
doVote := true
|
|
|
|
|
votesGranted := 0
|
|
|
|
|
var timeoutChan <-chan time.Time
|
|
|
|
|
var respChan chan *RequestVoteResponse
|
|
|
|
|
|
2014-01-28 15:58:23 +00:00
|
|
|
|
for s.State() == Candidate {
|
2014-02-13 22:05:42 +00:00
|
|
|
|
if doVote {
|
|
|
|
|
// Increment current term, vote for self.
|
|
|
|
|
s.currentTerm++
|
|
|
|
|
s.votedFor = s.name
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// Send RequestVote RPCs to all other servers.
|
|
|
|
|
respChan = make(chan *RequestVoteResponse, len(s.peers))
|
|
|
|
|
for _, peer := range s.peers {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Add(1)
|
|
|
|
|
go func(peer *Peer) {
|
|
|
|
|
defer s.routineGroup.Done()
|
|
|
|
|
peer.sendVoteRequest(newRequestVoteRequest(s.currentTerm, s.name, lastLogIndex, lastLogTerm), respChan)
|
|
|
|
|
}(peer)
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// Wait for either:
|
|
|
|
|
// * Votes received from majority of servers: become leader
|
|
|
|
|
// * AppendEntries RPC received from new leader: step down.
|
|
|
|
|
// * Election timeout elapses without election resolution: increment term, start new election
|
|
|
|
|
// * Discover higher term: step down (§5.1)
|
|
|
|
|
votesGranted = 1
|
|
|
|
|
timeoutChan = afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2)
|
|
|
|
|
doVote = false
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// If we received enough votes then stop waiting for more votes.
|
|
|
|
|
// And return from the candidate loop
|
|
|
|
|
if votesGranted == s.QuorumSize() {
|
|
|
|
|
s.debugln("server.candidate.recv.enough.votes")
|
|
|
|
|
s.setState(Leader)
|
|
|
|
|
return
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// Collect votes from peers.
|
|
|
|
|
select {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
case <-s.stopped:
|
2014-02-16 01:16:53 +00:00
|
|
|
|
s.setState(Stopped)
|
|
|
|
|
return
|
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
case resp := <-respChan:
|
|
|
|
|
if success := s.processVoteResponse(resp); success {
|
|
|
|
|
s.debugln("server.candidate.vote.granted: ", votesGranted)
|
|
|
|
|
votesGranted++
|
2013-07-07 23:52:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
case e := <-s.c:
|
|
|
|
|
var err error
|
2014-02-16 01:16:53 +00:00
|
|
|
|
switch req := e.target.(type) {
|
|
|
|
|
case Command:
|
|
|
|
|
err = NotLeaderError
|
|
|
|
|
case *AppendEntriesRequest:
|
|
|
|
|
e.returnValue, _ = s.processAppendEntriesRequest(req)
|
|
|
|
|
case *RequestVoteRequest:
|
|
|
|
|
e.returnValue, _ = s.processRequestVoteRequest(req)
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
2014-02-16 01:16:53 +00:00
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// Callback to event.
|
|
|
|
|
e.c <- err
|
|
|
|
|
|
|
|
|
|
case <-timeoutChan:
|
|
|
|
|
doVote = true
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-08-07 04:33:37 +00:00
|
|
|
|
// The event loop that is run when the server is in a Leader state.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) leaderLoop() {
|
2013-07-06 04:49:47 +00:00
|
|
|
|
logIndex, _ := s.log.lastInfo()
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2013-07-06 19:41:42 +00:00
|
|
|
|
// Update the peers prevLogIndex to leader's lastLogIndex and start heartbeat.
|
2013-07-21 02:07:16 +00:00
|
|
|
|
s.debugln("leaderLoop.set.PrevIndex to ", logIndex)
|
2013-07-03 16:53:46 +00:00
|
|
|
|
for _, peer := range s.peers {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
peer.setPrevLogIndex(logIndex)
|
2013-07-06 04:49:47 +00:00
|
|
|
|
peer.startHeartbeat()
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
2013-07-01 15:46:53 +00:00
|
|
|
|
|
2014-01-23 20:42:06 +00:00
|
|
|
|
// Commit a NOP after the server becomes leader. From the Raft paper:
|
|
|
|
|
// "Upon election: send initial empty AppendEntries RPCs (heartbeat) to
|
|
|
|
|
// each server; repeat during idle periods to prevent election timeouts
|
|
|
|
|
// (§5.2)". The heartbeats started above do the "idle" period work.
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Add(1)
|
|
|
|
|
go func() {
|
|
|
|
|
defer s.routineGroup.Done()
|
|
|
|
|
s.Do(NOPCommand{})
|
|
|
|
|
}()
|
2013-07-10 23:07:14 +00:00
|
|
|
|
|
2013-07-03 16:53:46 +00:00
|
|
|
|
// Begin to collect response from followers
|
2014-01-28 15:58:23 +00:00
|
|
|
|
for s.State() == Leader {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
var err error
|
2013-07-06 19:41:42 +00:00
|
|
|
|
select {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
case <-s.stopped:
|
2014-02-16 01:16:53 +00:00
|
|
|
|
// Stop all peers before stop
|
|
|
|
|
for _, peer := range s.peers {
|
|
|
|
|
peer.stopHeartbeat(false)
|
|
|
|
|
}
|
|
|
|
|
s.setState(Stopped)
|
|
|
|
|
return
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
case e := <-s.c:
|
2014-02-16 01:16:53 +00:00
|
|
|
|
switch req := e.target.(type) {
|
|
|
|
|
case Command:
|
|
|
|
|
s.processCommand(req, e)
|
|
|
|
|
continue
|
|
|
|
|
case *AppendEntriesRequest:
|
|
|
|
|
e.returnValue, _ = s.processAppendEntriesRequest(req)
|
|
|
|
|
case *AppendEntriesResponse:
|
|
|
|
|
s.processAppendEntriesResponse(req)
|
|
|
|
|
case *RequestVoteRequest:
|
|
|
|
|
e.returnValue, _ = s.processRequestVoteRequest(req)
|
2013-07-06 19:41:42 +00:00
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Callback to event.
|
|
|
|
|
e.c <- err
|
|
|
|
|
}
|
2013-07-03 16:53:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-10 23:07:14 +00:00
|
|
|
|
s.syncedPeer = nil
|
2013-06-23 18:42:31 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) snapshotLoop() {
|
2014-01-28 15:58:23 +00:00
|
|
|
|
for s.State() == Snapshotting {
|
2013-07-17 00:40:19 +00:00
|
|
|
|
var err error
|
2014-02-16 01:16:53 +00:00
|
|
|
|
select {
|
2014-04-01 17:21:31 +00:00
|
|
|
|
case <-s.stopped:
|
2013-07-17 00:40:19 +00:00
|
|
|
|
s.setState(Stopped)
|
2014-02-16 01:16:53 +00:00
|
|
|
|
return
|
|
|
|
|
|
|
|
|
|
case e := <-s.c:
|
2013-08-07 04:37:50 +00:00
|
|
|
|
switch req := e.target.(type) {
|
|
|
|
|
case Command:
|
|
|
|
|
err = NotLeaderError
|
|
|
|
|
case *AppendEntriesRequest:
|
|
|
|
|
e.returnValue, _ = s.processAppendEntriesRequest(req)
|
|
|
|
|
case *RequestVoteRequest:
|
|
|
|
|
e.returnValue, _ = s.processRequestVoteRequest(req)
|
|
|
|
|
case *SnapshotRecoveryRequest:
|
|
|
|
|
e.returnValue = s.processSnapshotRecoveryRequest(req)
|
|
|
|
|
}
|
2014-02-16 01:16:53 +00:00
|
|
|
|
// Callback to event.
|
|
|
|
|
e.c <- err
|
2013-07-17 00:40:19 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-17 02:28:08 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Commands
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-04-28 04:51:17 +00:00
|
|
|
|
// Attempts to execute a command and replicate it. The function will return
|
|
|
|
|
// when the command has been successfully committed or an error has occurred.
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) Do(command Command) (interface{}, error) {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
return s.send(command)
|
|
|
|
|
}
|
2013-07-05 17:44:03 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Processes a command.
|
2013-12-19 23:39:37 +00:00
|
|
|
|
func (s *server) processCommand(command Command, e *ev) {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.command.process")
|
2013-07-05 17:44:03 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Create an entry for the command in the log.
|
2013-12-30 08:58:38 +00:00
|
|
|
|
entry, err := s.log.createEntry(s.currentTerm, command, e)
|
2013-07-21 02:07:16 +00:00
|
|
|
|
|
2013-07-17 13:45:53 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
s.debugln("server.command.log.entry.error:", err)
|
|
|
|
|
e.c <- err
|
|
|
|
|
return
|
|
|
|
|
}
|
2013-07-24 18:03:20 +00:00
|
|
|
|
|
2013-07-06 04:49:47 +00:00
|
|
|
|
if err := s.log.appendEntry(entry); err != nil {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.command.log.error:", err)
|
|
|
|
|
e.c <- err
|
|
|
|
|
return
|
2013-05-05 19:36:23 +00:00
|
|
|
|
}
|
2013-05-05 20:26:04 +00:00
|
|
|
|
|
2014-01-12 08:01:38 +00:00
|
|
|
|
s.syncedPeer[s.Name()] = true
|
|
|
|
|
if len(s.peers) == 0 {
|
|
|
|
|
commitIndex := s.log.currentIndex()
|
|
|
|
|
s.log.setCommitIndex(commitIndex)
|
|
|
|
|
s.debugln("commit index ", commitIndex)
|
|
|
|
|
}
|
2013-04-28 04:51:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Append Entries
|
|
|
|
|
//--------------------------------------
|
2013-07-06 19:41:42 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Appends zero or more log entry from the leader to this server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
ret, _ := s.send(req)
|
|
|
|
|
resp, _ := ret.(*AppendEntriesResponse)
|
|
|
|
|
return resp
|
|
|
|
|
}
|
2013-05-05 21:41:55 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Processes the "append entries" request.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) {
|
2013-07-09 02:55:00 +00:00
|
|
|
|
s.traceln("server.ae.process")
|
|
|
|
|
|
2013-04-30 04:13:50 +00:00
|
|
|
|
if req.Term < s.currentTerm {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.ae.error: stale term")
|
2013-07-10 23:07:14 +00:00
|
|
|
|
return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), false
|
2013-04-30 04:13:50 +00:00
|
|
|
|
}
|
2013-06-24 16:52:51 +00:00
|
|
|
|
|
2014-02-16 00:14:24 +00:00
|
|
|
|
if req.Term == s.currentTerm {
|
2014-03-24 19:32:39 +00:00
|
|
|
|
_assert(s.State() != Leader, "leader.elected.at.same.term.%d\n", s.currentTerm)
|
2014-04-05 22:51:33 +00:00
|
|
|
|
|
|
|
|
|
// step-down to follower when it is a candidate
|
|
|
|
|
if s.state == Candidate {
|
|
|
|
|
// change state to follower
|
|
|
|
|
s.setState(Follower)
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-16 00:14:24 +00:00
|
|
|
|
// discover new leader when candidate
|
|
|
|
|
// save leader name when follower
|
|
|
|
|
s.leader = req.LeaderName
|
|
|
|
|
} else {
|
|
|
|
|
// Update term and leader.
|
|
|
|
|
s.updateCurrentTerm(req.Term, req.LeaderName)
|
|
|
|
|
}
|
2013-06-08 02:19:18 +00:00
|
|
|
|
|
2013-04-30 04:13:50 +00:00
|
|
|
|
// Reject if log doesn't contain a matching previous entry.
|
2013-07-06 04:49:47 +00:00
|
|
|
|
if err := s.log.truncate(req.PrevLogIndex, req.PrevLogTerm); err != nil {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.ae.truncate.error: ", err)
|
2013-07-10 23:07:14 +00:00
|
|
|
|
return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
|
2013-04-30 04:13:50 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Append entries to the log.
|
2013-07-06 04:49:47 +00:00
|
|
|
|
if err := s.log.appendEntries(req.Entries); err != nil {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.ae.append.error: ", err)
|
2013-07-10 23:07:14 +00:00
|
|
|
|
return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
|
2013-04-30 04:13:50 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Commit up to the commit index.
|
2013-07-06 04:49:47 +00:00
|
|
|
|
if err := s.log.setCommitIndex(req.CommitIndex); err != nil {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.ae.commit.error: ", err)
|
2013-07-10 23:07:14 +00:00
|
|
|
|
return newAppendEntriesResponse(s.currentTerm, false, s.log.currentIndex(), s.log.CommitIndex()), true
|
2013-06-08 02:19:18 +00:00
|
|
|
|
}
|
2013-04-30 04:13:50 +00:00
|
|
|
|
|
2014-01-09 03:59:13 +00:00
|
|
|
|
// once the server appended and committed all the log entries from the leader
|
2013-07-12 01:03:24 +00:00
|
|
|
|
|
2013-07-10 23:07:14 +00:00
|
|
|
|
return newAppendEntriesResponse(s.currentTerm, true, s.log.currentIndex(), s.log.CommitIndex()), true
|
2013-04-28 04:51:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Processes the "append entries" response from the peer. This is only
|
|
|
|
|
// processed when the server is a leader. Responses received during other
|
|
|
|
|
// states are dropped.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// If we find a higher term then change to a follower and exit.
|
2014-01-12 07:40:55 +00:00
|
|
|
|
if resp.Term() > s.Term() {
|
2014-02-16 00:14:24 +00:00
|
|
|
|
s.updateCurrentTerm(resp.Term(), "")
|
2013-07-07 20:21:04 +00:00
|
|
|
|
return
|
|
|
|
|
}
|
2013-07-07 20:55:55 +00:00
|
|
|
|
|
2013-07-10 23:07:14 +00:00
|
|
|
|
// panic response if it's not successful.
|
2014-01-12 07:40:55 +00:00
|
|
|
|
if !resp.Success() {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
return
|
|
|
|
|
}
|
2013-07-07 20:55:55 +00:00
|
|
|
|
|
2013-07-10 23:07:14 +00:00
|
|
|
|
// if one peer successfully append a log from the leader term,
|
|
|
|
|
// we add it to the synced list
|
|
|
|
|
if resp.append == true {
|
|
|
|
|
s.syncedPeer[resp.peer] = true
|
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Increment the commit count to make sure we have a quorum before committing.
|
2013-07-10 23:07:14 +00:00
|
|
|
|
if len(s.syncedPeer) < s.QuorumSize() {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Determine the committed index that a majority has.
|
|
|
|
|
var indices []uint64
|
|
|
|
|
indices = append(indices, s.log.currentIndex())
|
|
|
|
|
for _, peer := range s.peers {
|
2013-07-07 22:12:24 +00:00
|
|
|
|
indices = append(indices, peer.getPrevLogIndex())
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
2013-09-29 03:00:40 +00:00
|
|
|
|
sort.Sort(sort.Reverse(uint64Slice(indices)))
|
2013-07-07 20:21:04 +00:00
|
|
|
|
|
|
|
|
|
// We can commit up to the index which the majority of the members have appended.
|
|
|
|
|
commitIndex := indices[s.QuorumSize()-1]
|
|
|
|
|
committedIndex := s.log.commitIndex
|
|
|
|
|
|
|
|
|
|
if commitIndex > committedIndex {
|
2014-01-10 12:45:53 +00:00
|
|
|
|
// leader needs to do a fsync before committing log entries
|
|
|
|
|
s.log.sync()
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.log.setCommitIndex(commitIndex)
|
2013-07-08 05:26:36 +00:00
|
|
|
|
s.debugln("commit index ", commitIndex)
|
2013-06-25 21:41:42 +00:00
|
|
|
|
}
|
2013-05-05 19:36:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-13 22:05:42 +00:00
|
|
|
|
// processVoteReponse processes a vote request:
|
|
|
|
|
// 1. if the vote is granted for the current term of the candidate, return true
|
|
|
|
|
// 2. if the vote is denied due to smaller term, update the term of this server
|
|
|
|
|
// which will also cause the candidate to step-down, and return false.
|
|
|
|
|
// 3. if the vote is for a smaller term, ignore it and return false.
|
|
|
|
|
func (s *server) processVoteResponse(resp *RequestVoteResponse) bool {
|
|
|
|
|
if resp.VoteGranted && resp.Term == s.currentTerm {
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if resp.Term > s.currentTerm {
|
|
|
|
|
s.debugln("server.candidate.vote.failed")
|
2014-02-16 00:14:24 +00:00
|
|
|
|
s.updateCurrentTerm(resp.Term, "")
|
2014-02-13 22:05:42 +00:00
|
|
|
|
} else {
|
|
|
|
|
s.debugln("server.candidate.vote: denied")
|
|
|
|
|
}
|
|
|
|
|
return false
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-01 05:11:23 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Request Vote
|
2013-04-28 21:23:21 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
|
|
|
|
// Requests a vote from a server. A vote can be obtained if the vote's term is
|
|
|
|
|
// at the server's current term and the server has not made a vote yet. A vote
|
|
|
|
|
// can also be obtained if the term is greater than the server's current term.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse {
|
2013-07-07 20:21:04 +00:00
|
|
|
|
ret, _ := s.send(req)
|
|
|
|
|
resp, _ := ret.(*RequestVoteResponse)
|
|
|
|
|
return resp
|
|
|
|
|
}
|
2013-05-08 03:56:32 +00:00
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// Processes a "request vote" request.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) {
|
2013-07-11 19:56:34 +00:00
|
|
|
|
|
2013-04-28 21:23:21 +00:00
|
|
|
|
// If the request is coming from an old term then reject it.
|
2013-10-22 22:39:32 +00:00
|
|
|
|
if req.Term < s.Term() {
|
2013-11-27 05:50:03 +00:00
|
|
|
|
s.debugln("server.rv.deny.vote: cause stale term")
|
2013-07-08 04:31:58 +00:00
|
|
|
|
return newRequestVoteResponse(s.currentTerm, false), false
|
2013-04-28 21:23:21 +00:00
|
|
|
|
}
|
2013-07-03 01:22:37 +00:00
|
|
|
|
|
2014-02-16 00:14:24 +00:00
|
|
|
|
// If the term of the request peer is larger than this node, update the term
|
|
|
|
|
// If the term is equal and we've already voted for a different candidate then
|
|
|
|
|
// don't vote for this candidate.
|
|
|
|
|
if req.Term > s.Term() {
|
|
|
|
|
s.updateCurrentTerm(req.Term, "")
|
|
|
|
|
} else if s.votedFor != "" && s.votedFor != req.CandidateName {
|
2013-11-27 05:50:03 +00:00
|
|
|
|
s.debugln("server.deny.vote: cause duplicate vote: ", req.CandidateName,
|
2013-07-08 04:31:58 +00:00
|
|
|
|
" already vote for ", s.votedFor)
|
|
|
|
|
return newRequestVoteResponse(s.currentTerm, false), false
|
2013-04-28 21:23:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
// If the candidate's log is not at least as up-to-date as our last log then don't vote.
|
2013-07-06 04:49:47 +00:00
|
|
|
|
lastIndex, lastTerm := s.log.lastInfo()
|
2013-06-23 18:42:31 +00:00
|
|
|
|
if lastIndex > req.LastLogIndex || lastTerm > req.LastLogTerm {
|
2013-11-27 05:50:03 +00:00
|
|
|
|
s.debugln("server.deny.vote: cause out of date log: ", req.CandidateName,
|
2013-07-10 23:07:14 +00:00
|
|
|
|
"Index :[", lastIndex, "]", " [", req.LastLogIndex, "]",
|
|
|
|
|
"Term :[", lastTerm, "]", " [", req.LastLogTerm, "]")
|
2013-07-08 04:31:58 +00:00
|
|
|
|
return newRequestVoteResponse(s.currentTerm, false), false
|
2013-04-28 22:49:52 +00:00
|
|
|
|
}
|
2013-04-28 21:23:21 +00:00
|
|
|
|
|
2013-04-28 22:49:52 +00:00
|
|
|
|
// If we made it this far then cast a vote and reset our election time out.
|
2013-07-07 20:21:04 +00:00
|
|
|
|
s.debugln("server.rv.vote: ", s.name, " votes for", req.CandidateName, "at term", req.Term)
|
2013-04-28 22:49:52 +00:00
|
|
|
|
s.votedFor = req.CandidateName
|
2013-06-24 16:52:51 +00:00
|
|
|
|
|
2013-07-08 04:31:58 +00:00
|
|
|
|
return newRequestVoteResponse(s.currentTerm, true), true
|
2013-05-05 21:41:55 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-04-17 02:32:49 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Membership
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-07-09 02:55:00 +00:00
|
|
|
|
// Adds a peer to the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) AddPeer(name string, connectiongString string) error {
|
2013-07-09 02:55:00 +00:00
|
|
|
|
s.debugln("server.peer.add: ", name, len(s.peers))
|
2013-08-19 15:54:44 +00:00
|
|
|
|
|
2013-07-09 02:55:00 +00:00
|
|
|
|
// Do not allow peers to be added twice.
|
2013-05-28 18:46:27 +00:00
|
|
|
|
if s.peers[name] != nil {
|
2013-07-11 03:02:24 +00:00
|
|
|
|
return nil
|
2013-04-17 02:32:49 +00:00
|
|
|
|
}
|
2013-04-28 21:23:21 +00:00
|
|
|
|
|
2013-08-09 16:55:25 +00:00
|
|
|
|
// Skip the Peer if it has the same name as the Server
|
2013-08-19 15:54:44 +00:00
|
|
|
|
if s.name != name {
|
2014-01-23 20:50:29 +00:00
|
|
|
|
peer := newPeer(s, name, connectiongString, s.heartbeatInterval)
|
2013-08-09 16:55:25 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
if s.State() == Leader {
|
|
|
|
|
peer.startHeartbeat()
|
|
|
|
|
}
|
2013-08-13 21:31:19 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
s.peers[peer.Name] = peer
|
2013-12-19 23:39:37 +00:00
|
|
|
|
|
|
|
|
|
s.DispatchEvent(newEvent(AddPeerEventType, name, nil))
|
2013-08-09 16:55:25 +00:00
|
|
|
|
}
|
2013-08-13 21:31:19 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
// Write the configuration to file.
|
|
|
|
|
s.writeConf()
|
2013-08-13 21:31:19 +00:00
|
|
|
|
|
2013-05-28 18:46:27 +00:00
|
|
|
|
return nil
|
2013-04-17 02:32:49 +00:00
|
|
|
|
}
|
2013-06-05 17:57:31 +00:00
|
|
|
|
|
2013-07-09 02:55:00 +00:00
|
|
|
|
// Removes a peer from the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) RemovePeer(name string) error {
|
2013-07-09 02:55:00 +00:00
|
|
|
|
s.debugln("server.peer.remove: ", name, len(s.peers))
|
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
// Skip the Peer if it has the same name as the Server
|
|
|
|
|
if name != s.Name() {
|
|
|
|
|
// Return error if peer doesn't exist.
|
|
|
|
|
peer := s.peers[name]
|
|
|
|
|
if peer == nil {
|
|
|
|
|
return fmt.Errorf("raft: Peer not found: %s", name)
|
|
|
|
|
}
|
2013-08-15 19:25:00 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
// Stop peer and remove it.
|
|
|
|
|
if s.State() == Leader {
|
2014-03-10 19:03:44 +00:00
|
|
|
|
// We create a go routine here to avoid potential deadlock.
|
|
|
|
|
// We are holding log write lock when reach this line of code.
|
|
|
|
|
// Peer.stopHeartbeat can be blocked without go routine, if the
|
|
|
|
|
// target go routine (which we want to stop) is calling
|
|
|
|
|
// log.getEntriesAfter and waiting for log read lock.
|
|
|
|
|
// So we might be holding log lock and waiting for log lock,
|
|
|
|
|
// which lead to a deadlock.
|
|
|
|
|
// TODO(xiangli) refactor log lock
|
2014-04-01 17:21:31 +00:00
|
|
|
|
s.routineGroup.Add(1)
|
|
|
|
|
go func() {
|
|
|
|
|
defer s.routineGroup.Done()
|
|
|
|
|
peer.stopHeartbeat(true)
|
|
|
|
|
}()
|
2013-08-19 15:54:44 +00:00
|
|
|
|
}
|
2013-06-05 17:57:31 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
delete(s.peers, name)
|
2013-12-19 23:39:37 +00:00
|
|
|
|
|
|
|
|
|
s.DispatchEvent(newEvent(RemovePeerEventType, name, nil))
|
2013-08-13 21:31:19 +00:00
|
|
|
|
}
|
2013-07-23 22:30:14 +00:00
|
|
|
|
|
2013-08-19 15:54:44 +00:00
|
|
|
|
// Write the configuration to file.
|
|
|
|
|
s.writeConf()
|
2013-06-05 17:57:31 +00:00
|
|
|
|
|
|
|
|
|
return nil
|
|
|
|
|
}
|
2013-06-06 03:32:52 +00:00
|
|
|
|
|
2013-06-03 21:58:12 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Log compaction
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) TakeSnapshot() error {
|
2014-02-27 16:12:39 +00:00
|
|
|
|
if s.stateMachine == nil {
|
|
|
|
|
return errors.New("Snapshot: Cannot create snapshot. Missing state machine.")
|
|
|
|
|
}
|
2014-01-21 17:00:59 +00:00
|
|
|
|
|
2014-02-27 16:12:39 +00:00
|
|
|
|
// Shortcut without lock
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Exit if the server is currently creating a snapshot.
|
2014-02-21 16:38:12 +00:00
|
|
|
|
if s.pendingSnapshot != nil {
|
|
|
|
|
return errors.New("Snapshot: Last snapshot is not finished.")
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
2013-06-05 05:56:59 +00:00
|
|
|
|
|
2014-02-27 16:12:39 +00:00
|
|
|
|
// TODO: acquire the lock and no more committed is allowed
|
|
|
|
|
// This will be done after finishing refactoring heartbeat
|
|
|
|
|
s.debugln("take.snapshot")
|
|
|
|
|
|
2013-07-06 04:49:47 +00:00
|
|
|
|
lastIndex, lastTerm := s.log.commitInfo()
|
2014-02-27 16:12:39 +00:00
|
|
|
|
|
|
|
|
|
// check if there is log has been committed since the
|
|
|
|
|
// last snapshot.
|
|
|
|
|
if lastIndex == s.log.startIndex {
|
|
|
|
|
return nil
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-27 16:12:39 +00:00
|
|
|
|
path := s.SnapshotPath(lastIndex, lastTerm)
|
|
|
|
|
// Attach snapshot to pending snapshot and save it to disk.
|
|
|
|
|
s.pendingSnapshot = &Snapshot{lastIndex, lastTerm, nil, nil, path}
|
|
|
|
|
|
|
|
|
|
state, err := s.stateMachine.Save()
|
|
|
|
|
if err != nil {
|
2014-10-30 16:06:30 +00:00
|
|
|
|
s.pendingSnapshot = nil
|
2014-02-27 16:12:39 +00:00
|
|
|
|
return err
|
2013-06-06 04:14:07 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Clone the list of peers.
|
|
|
|
|
peers := make([]*Peer, 0, len(s.peers)+1)
|
2013-06-12 16:47:48 +00:00
|
|
|
|
for _, peer := range s.peers {
|
2014-01-21 17:00:59 +00:00
|
|
|
|
peers = append(peers, peer.clone())
|
2013-06-06 04:14:07 +00:00
|
|
|
|
}
|
2014-01-21 17:00:59 +00:00
|
|
|
|
peers = append(peers, &Peer{Name: s.Name(), ConnectionString: s.connectionString})
|
2013-06-06 04:14:07 +00:00
|
|
|
|
|
2014-02-27 16:12:39 +00:00
|
|
|
|
// Attach snapshot to pending snapshot and save it to disk.
|
|
|
|
|
s.pendingSnapshot.Peers = peers
|
|
|
|
|
s.pendingSnapshot.State = state
|
2013-06-05 00:02:45 +00:00
|
|
|
|
s.saveSnapshot()
|
2013-06-05 05:56:59 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// We keep some log entries after the snapshot.
|
|
|
|
|
// We do not want to send the whole snapshot to the slightly slow machines
|
2013-07-18 23:44:01 +00:00
|
|
|
|
if lastIndex-s.log.startIndex > NumberOfLogEntriesAfterSnapshot {
|
|
|
|
|
compactIndex := lastIndex - NumberOfLogEntriesAfterSnapshot
|
2014-01-11 14:41:12 +00:00
|
|
|
|
compactTerm := s.log.getEntry(compactIndex).Term()
|
2013-07-16 20:16:33 +00:00
|
|
|
|
s.log.compact(compactIndex, compactTerm)
|
|
|
|
|
}
|
2013-06-05 05:56:59 +00:00
|
|
|
|
|
2013-06-03 21:58:12 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Retrieves the log path for the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) saveSnapshot() error {
|
2014-02-21 16:38:12 +00:00
|
|
|
|
if s.pendingSnapshot == nil {
|
|
|
|
|
return errors.New("pendingSnapshot.is.nil")
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Write snapshot to disk.
|
2014-02-21 16:38:12 +00:00
|
|
|
|
if err := s.pendingSnapshot.save(); err != nil {
|
2014-10-30 16:06:30 +00:00
|
|
|
|
s.pendingSnapshot = nil
|
2013-06-03 21:58:12 +00:00
|
|
|
|
return err
|
|
|
|
|
}
|
2013-06-08 02:19:18 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Swap the current and last snapshots.
|
2014-02-21 16:38:12 +00:00
|
|
|
|
tmp := s.snapshot
|
|
|
|
|
s.snapshot = s.pendingSnapshot
|
2013-06-05 05:56:59 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Delete the previous snapshot if there is any change
|
2014-02-21 16:38:12 +00:00
|
|
|
|
if tmp != nil && !(tmp.LastIndex == s.snapshot.LastIndex && tmp.LastTerm == s.snapshot.LastTerm) {
|
2013-07-06 04:49:47 +00:00
|
|
|
|
tmp.remove()
|
2013-06-05 00:02:45 +00:00
|
|
|
|
}
|
2014-02-21 16:38:12 +00:00
|
|
|
|
s.pendingSnapshot = nil
|
2014-01-21 17:00:59 +00:00
|
|
|
|
|
2013-06-03 21:58:12 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-20 17:12:59 +00:00
|
|
|
|
// Returns a list of available snapshot names sorted newest to oldest
|
|
|
|
|
func (s *server) SnapshotList() ([]string, error) {
|
|
|
|
|
// Get FileInfo for everything in the snapshot dir
|
|
|
|
|
ssdir := path.Join(s.path, "snapshot")
|
|
|
|
|
finfos, err := ioutil.ReadDir(ssdir)
|
|
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
|
return nil, err
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Build a list of snapshot file names
|
|
|
|
|
var ssnames []string
|
|
|
|
|
for _, finfo := range finfos {
|
|
|
|
|
fname := finfo.Name()
|
|
|
|
|
if finfo.Mode().IsRegular() && filepath.Ext(fname) == ".ss" {
|
|
|
|
|
ssnames = append(ssnames, fname)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Sort snapshot names from newest to oldest
|
|
|
|
|
sort.Sort(sort.Reverse(sort.StringSlice(ssnames)))
|
|
|
|
|
|
|
|
|
|
return ssnames, nil
|
|
|
|
|
}
|
|
|
|
|
|
2013-06-05 00:02:45 +00:00
|
|
|
|
// Retrieves the log path for the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) SnapshotPath(lastIndex uint64, lastTerm uint64) string {
|
2013-06-05 17:38:49 +00:00
|
|
|
|
return path.Join(s.path, "snapshot", fmt.Sprintf("%v_%v.ss", lastTerm, lastIndex))
|
2013-06-05 00:02:45 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse {
|
2013-07-17 00:40:19 +00:00
|
|
|
|
ret, _ := s.send(req)
|
|
|
|
|
resp, _ := ret.(*SnapshotResponse)
|
|
|
|
|
return resp
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse {
|
2013-07-17 00:40:19 +00:00
|
|
|
|
// If the follower’s log contains an entry at the snapshot’s last index with a term
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// that matches the snapshot’s last term, then the follower already has all the
|
|
|
|
|
// information found in the snapshot and can reply false.
|
2013-07-18 23:29:06 +00:00
|
|
|
|
entry := s.log.getEntry(req.LastIndex)
|
2013-07-17 00:40:19 +00:00
|
|
|
|
|
2014-01-11 14:41:12 +00:00
|
|
|
|
if entry != nil && entry.Term() == req.LastTerm {
|
2013-07-17 00:40:19 +00:00
|
|
|
|
return newSnapshotResponse(false)
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Update state.
|
2013-07-17 00:40:19 +00:00
|
|
|
|
s.setState(Snapshotting)
|
|
|
|
|
|
|
|
|
|
return newSnapshotResponse(true)
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
2013-07-17 00:40:19 +00:00
|
|
|
|
ret, _ := s.send(req)
|
|
|
|
|
resp, _ := ret.(*SnapshotRecoveryResponse)
|
|
|
|
|
return resp
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse {
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Recover state sent from request.
|
|
|
|
|
if err := s.stateMachine.Recovery(req.State); err != nil {
|
2014-02-27 16:12:39 +00:00
|
|
|
|
panic("cannot recover from previous state")
|
2014-01-21 17:00:59 +00:00
|
|
|
|
}
|
2013-06-05 05:56:59 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Recover the cluster configuration.
|
2013-07-17 00:40:19 +00:00
|
|
|
|
s.peers = make(map[string]*Peer)
|
2013-08-15 23:35:01 +00:00
|
|
|
|
for _, peer := range req.Peers {
|
|
|
|
|
s.AddPeer(peer.Name, peer.ConnectionString)
|
2013-06-12 16:47:48 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Update log state.
|
2013-06-06 03:25:17 +00:00
|
|
|
|
s.currentTerm = req.LastTerm
|
2013-07-06 04:49:47 +00:00
|
|
|
|
s.log.updateCommitIndex(req.LastIndex)
|
2013-06-12 16:47:48 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Create local snapshot.
|
2014-02-21 16:38:12 +00:00
|
|
|
|
s.pendingSnapshot = &Snapshot{req.LastIndex, req.LastTerm, req.Peers, req.State, s.SnapshotPath(req.LastIndex, req.LastTerm)}
|
2013-06-08 02:19:18 +00:00
|
|
|
|
s.saveSnapshot()
|
2013-06-24 16:52:51 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Clear the previous log entries.
|
2013-07-06 04:49:47 +00:00
|
|
|
|
s.log.compact(req.LastIndex, req.LastTerm)
|
2013-06-05 00:02:45 +00:00
|
|
|
|
|
2013-07-17 00:40:19 +00:00
|
|
|
|
return newSnapshotRecoveryResponse(req.LastTerm, true, req.LastIndex)
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-06-05 05:56:59 +00:00
|
|
|
|
// Load a snapshot at restart
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) LoadSnapshot() error {
|
2014-10-20 17:12:59 +00:00
|
|
|
|
sslist, err := s.SnapshotList()
|
2014-10-20 23:04:13 +00:00
|
|
|
|
|
|
|
|
|
// if the snapshot directory doesn't exist or no snapshots exist in
|
|
|
|
|
// the directory, return immediately
|
|
|
|
|
if err != nil && os.IsNotExist(err) || err == nil && len(sslist) == 0 {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2013-06-03 21:58:12 +00:00
|
|
|
|
if err != nil {
|
2013-06-12 16:47:48 +00:00
|
|
|
|
return err
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-20 17:12:59 +00:00
|
|
|
|
// Load most recent snapshot (falling back to older snapshots if needed)
|
|
|
|
|
var ss *Snapshot
|
|
|
|
|
for _, ssname := range sslist {
|
|
|
|
|
ssFullPath := path.Join(s.path, "snapshot", ssname)
|
|
|
|
|
ss, err = loadSnapshot(ssFullPath)
|
|
|
|
|
if err == nil {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
s.debugln(err)
|
2013-06-03 21:58:12 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
if err != nil {
|
2014-10-20 17:12:59 +00:00
|
|
|
|
return err // couldn't load any of the snapshots
|
2014-01-21 17:00:59 +00:00
|
|
|
|
}
|
2013-06-12 16:47:48 +00:00
|
|
|
|
|
2014-10-20 17:12:59 +00:00
|
|
|
|
s.snapshot = ss
|
2013-06-06 04:14:07 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Recover snapshot into state machine.
|
2014-10-20 17:12:59 +00:00
|
|
|
|
if err = s.stateMachine.Recovery(ss.State); err != nil {
|
2014-02-27 16:12:39 +00:00
|
|
|
|
s.debugln("recovery.snapshot.error: ", err)
|
2013-07-01 02:14:02 +00:00
|
|
|
|
return err
|
2013-07-01 00:55:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Recover cluster configuration.
|
2014-10-20 17:12:59 +00:00
|
|
|
|
for _, peer := range ss.Peers {
|
2013-08-15 23:35:01 +00:00
|
|
|
|
s.AddPeer(peer.Name, peer.ConnectionString)
|
2013-06-12 16:47:48 +00:00
|
|
|
|
}
|
2013-06-06 04:14:07 +00:00
|
|
|
|
|
2014-01-21 17:00:59 +00:00
|
|
|
|
// Update log state.
|
2014-10-20 17:12:59 +00:00
|
|
|
|
s.log.startTerm = ss.LastTerm
|
|
|
|
|
s.log.startIndex = ss.LastIndex
|
|
|
|
|
s.log.updateCommitIndex(ss.LastIndex)
|
2013-06-05 00:02:45 +00:00
|
|
|
|
|
2013-06-03 21:58:12 +00:00
|
|
|
|
return err
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
|
2013-08-15 19:25:00 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Config File
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2014-03-20 23:29:36 +00:00
|
|
|
|
// Flushes commit index to the disk.
|
|
|
|
|
// So when the raft server restarts, it will commit upto the flushed commitIndex.
|
|
|
|
|
func (s *server) FlushCommitIndex() {
|
|
|
|
|
s.debugln("server.conf.update")
|
|
|
|
|
// Write the configuration to file.
|
|
|
|
|
s.writeConf()
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) writeConf() {
|
2013-08-15 19:25:00 +00:00
|
|
|
|
|
2013-08-15 23:35:01 +00:00
|
|
|
|
peers := make([]*Peer, len(s.peers))
|
2013-08-15 19:25:00 +00:00
|
|
|
|
|
|
|
|
|
i := 0
|
2013-08-15 23:35:01 +00:00
|
|
|
|
for _, peer := range s.peers {
|
|
|
|
|
peers[i] = peer.clone()
|
2013-08-15 19:25:00 +00:00
|
|
|
|
i++
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
r := &Config{
|
|
|
|
|
CommitIndex: s.log.commitIndex,
|
|
|
|
|
Peers: peers,
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
b, _ := json.Marshal(r)
|
|
|
|
|
|
|
|
|
|
confPath := path.Join(s.path, "conf")
|
|
|
|
|
tmpConfPath := path.Join(s.path, "conf.tmp")
|
|
|
|
|
|
2014-01-10 12:49:33 +00:00
|
|
|
|
err := writeFileSynced(tmpConfPath, b, 0600)
|
2013-08-15 19:25:00 +00:00
|
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
|
panic(err)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
os.Rename(tmpConfPath, confPath)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Read the configuration for the server.
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) readConf() error {
|
2013-08-15 19:25:00 +00:00
|
|
|
|
confPath := path.Join(s.path, "conf")
|
|
|
|
|
s.debugln("readConf.open ", confPath)
|
|
|
|
|
|
|
|
|
|
// open conf file
|
|
|
|
|
b, err := ioutil.ReadFile(confPath)
|
|
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
conf := &Config{}
|
|
|
|
|
|
|
|
|
|
if err = json.Unmarshal(b, conf); err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-18 04:19:46 +00:00
|
|
|
|
s.log.updateCommitIndex(conf.CommitIndex)
|
2013-08-15 19:25:00 +00:00
|
|
|
|
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2013-07-07 20:21:04 +00:00
|
|
|
|
//--------------------------------------
|
|
|
|
|
// Debugging
|
|
|
|
|
//--------------------------------------
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) debugln(v ...interface{}) {
|
2013-11-27 04:43:02 +00:00
|
|
|
|
if logLevel > Debug {
|
|
|
|
|
debugf("[%s Term:%d] %s", s.name, s.Term(), fmt.Sprintln(v...))
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 17:54:49 +00:00
|
|
|
|
func (s *server) traceln(v ...interface{}) {
|
2013-11-27 04:43:02 +00:00
|
|
|
|
if logLevel > Trace {
|
|
|
|
|
tracef("[%s] %s", s.name, fmt.Sprintln(v...))
|
|
|
|
|
}
|
2013-07-07 20:21:04 +00:00
|
|
|
|
}
|