etcd/raft/node.go

215 lines
5.1 KiB
Go
Raw Normal View History

package raft
import (
2014-09-01 05:49:07 +04:00
"errors"
2014-09-03 07:21:58 +04:00
"log"
2014-09-01 05:49:07 +04:00
2014-08-28 05:53:18 +04:00
pb "github.com/coreos/etcd/raft/raftpb"
2014-09-04 07:06:16 +04:00
"github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
)
2014-06-05 21:49:34 +04:00
2014-09-09 08:15:35 +04:00
var (
2014-09-16 04:35:02 +04:00
emptyState = pb.HardState{}
2014-09-09 08:15:35 +04:00
ErrStopped = errors.New("raft: stopped")
)
2014-09-01 05:49:07 +04:00
2014-09-16 04:35:02 +04:00
// SoftState provides state that is useful for logging and debugging.
// The state is volatile and does not need to be persisted to the WAL.
type SoftState struct {
Lead int64
RaftState StateType
}
func (a *SoftState) equal(b *SoftState) bool {
return a.Lead == b.Lead && a.RaftState == b.RaftState
}
// Ready encapsulates the entries and messages that are ready to read,
// be saved to stable storage, committed or sent to other peers.
// All fields in Ready are read-only.
type Ready struct {
2014-09-16 04:35:02 +04:00
// The current volatile state of a Node.
// SoftState will be nil if there is no update.
// It is not required to consume or store SoftState.
*SoftState
// The current state of a Node to be saved to stable storage BEFORE
// Messages are sent.
// HardState will be equal to empty state if there is no update.
pb.HardState
// Entries specifies entries to be saved to stable storage BEFORE
// Messages are sent.
2014-08-28 05:53:18 +04:00
Entries []pb.Entry
// CommittedEntries specifies entries to be committed to a
// store/state-machine. These have previously been committed to stable
// store.
2014-08-28 05:53:18 +04:00
CommittedEntries []pb.Entry
// Messages specifies outbound messages to be sent AFTER Entries are
// committed to stable storage.
2014-08-28 05:53:18 +04:00
Messages []pb.Message
}
2014-09-16 04:35:02 +04:00
func isStateEqual(a, b pb.HardState) bool {
2014-09-16 01:34:23 +04:00
return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
2014-08-25 07:09:06 +04:00
}
2014-05-29 00:53:26 +04:00
2014-09-16 04:35:02 +04:00
func IsEmptyState(st pb.HardState) bool {
2014-09-09 08:45:10 +04:00
return isStateEqual(st, emptyState)
}
func (rd Ready) containsUpdates() bool {
2014-09-16 04:35:02 +04:00
return rd.SoftState != nil || !IsEmptyState(rd.HardState) || len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
2014-06-05 21:49:34 +04:00
}
type Node struct {
2014-09-09 09:14:41 +04:00
propc chan pb.Message
recvc chan pb.Message
readyc chan Ready
tickc chan struct{}
done chan struct{}
}
// Start returns a new Node given a unique raft id, a list of raft peers, and
// the election and heartbeat timeouts in units of ticks.
2014-09-03 03:59:29 +04:00
func Start(id int64, peers []int64, election, heartbeat int) Node {
2014-09-05 08:15:39 +04:00
n := newNode()
r := newRaft(id, peers, election, heartbeat)
go n.run(r)
return n
}
// Restart is identical to Start but takes an initial State and a slice of
// entries. Generally this is used when restarting from a stable storage
// log.
2014-09-16 04:35:02 +04:00
func Restart(id int64, peers []int64, election, heartbeat int, st pb.HardState, ents []pb.Entry) Node {
2014-09-05 08:15:39 +04:00
n := newNode()
r := newRaft(id, peers, election, heartbeat)
r.loadState(st)
r.loadEnts(ents)
go n.run(r)
return n
}
func newNode() Node {
return Node{
2014-09-09 09:14:41 +04:00
propc: make(chan pb.Message),
recvc: make(chan pb.Message),
readyc: make(chan Ready),
tickc: make(chan struct{}),
done: make(chan struct{}),
2014-08-25 07:09:06 +04:00
}
}
2014-09-01 05:49:07 +04:00
func (n *Node) Stop() {
close(n.done)
}
2014-08-25 07:09:06 +04:00
func (n *Node) run(r *raft) {
var propc chan pb.Message
var readyc chan Ready
2014-06-20 01:39:17 +04:00
2014-09-16 04:35:02 +04:00
lead := None
prevSoftSt := r.softState()
prevHardSt := r.HardState
2014-09-05 08:15:39 +04:00
2014-08-25 07:09:06 +04:00
for {
2014-09-16 04:35:02 +04:00
rd := newReady(r, prevSoftSt, prevHardSt)
if rd.containsUpdates() {
readyc = n.readyc
} else {
readyc = nil
}
if rd.SoftState != nil && lead != rd.SoftState.Lead {
log.Printf("raft: leader changed from %#x to %#x", lead, rd.SoftState.Lead)
lead = rd.SoftState.Lead
2014-09-03 07:21:58 +04:00
if r.hasLeader() {
propc = n.propc
} else {
propc = nil
}
}
2014-08-25 07:09:06 +04:00
select {
case m := <-propc:
m.From = r.id
r.Step(m)
2014-08-25 07:09:06 +04:00
case m := <-n.recvc:
r.Step(m) // raft never returns an error
case <-n.tickc:
2014-09-03 03:59:29 +04:00
r.tick()
case readyc <- rd:
2014-09-16 04:35:02 +04:00
if rd.SoftState != nil {
prevSoftSt = rd.SoftState
}
if !IsEmptyState(rd.HardState) {
prevHardSt = rd.HardState
}
2014-08-25 07:09:06 +04:00
r.raftLog.resetNextEnts()
r.raftLog.resetUnstable()
r.msgs = nil
2014-09-01 05:49:07 +04:00
case <-n.done:
2014-08-25 07:09:06 +04:00
return
2014-07-15 10:41:19 +04:00
}
2014-05-29 00:53:26 +04:00
}
}
2014-06-05 21:49:34 +04:00
// Tick increments the internal logical clock for this Node. Election timeouts
// and heartbeat timeouts are in units of ticks.
2014-09-16 00:22:21 +04:00
func (n *Node) Tick() {
2014-08-25 07:09:06 +04:00
select {
case n.tickc <- struct{}{}:
2014-09-01 05:49:07 +04:00
case <-n.done:
2014-06-08 13:50:39 +04:00
}
2014-06-05 21:49:34 +04:00
}
2014-08-29 03:41:42 +04:00
func (n *Node) Campaign(ctx context.Context) error {
return n.Step(ctx, pb.Message{Type: msgHup})
}
2014-08-25 07:09:06 +04:00
// Propose proposes data be appended to the log.
2014-08-29 03:41:42 +04:00
func (n *Node) Propose(ctx context.Context, data []byte) error {
return n.Step(ctx, pb.Message{Type: msgProp, Entries: []pb.Entry{{Data: data}}})
2014-07-25 01:11:53 +04:00
}
2014-07-31 04:21:27 +04:00
2014-08-28 04:22:34 +04:00
// Step advances the state machine using msgs. The ctx.Err() will be returned,
2014-08-25 23:49:14 +04:00
// if any.
2014-08-28 05:53:18 +04:00
func (n *Node) Step(ctx context.Context, m pb.Message) error {
2014-08-28 04:22:34 +04:00
ch := n.recvc
if m.Type == msgProp {
ch = n.propc
}
2014-08-28 04:22:34 +04:00
select {
case ch <- m:
return nil
case <-ctx.Done():
return ctx.Err()
2014-09-01 05:49:07 +04:00
case <-n.done:
return ErrStopped
2014-08-01 02:18:44 +04:00
}
2014-07-31 04:21:27 +04:00
}
2014-08-25 07:09:06 +04:00
// ReadState returns the current point-in-time state.
func (n *Node) Ready() <-chan Ready {
return n.readyc
2014-07-31 04:21:27 +04:00
}
2014-09-09 08:50:04 +04:00
2014-09-16 04:35:02 +04:00
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
2014-09-09 08:50:04 +04:00
rd := Ready{
Entries: r.raftLog.unstableEnts(),
CommittedEntries: r.raftLog.nextEnts(),
Messages: r.msgs,
}
2014-09-16 04:35:02 +04:00
if softSt := r.softState(); !softSt.equal(prevSoftSt) {
rd.SoftState = softSt
}
if !isStateEqual(r.HardState, prevHardSt) {
rd.HardState = r.HardState
2014-09-09 08:50:04 +04:00
}
return rd
}