2014-08-25 07:09:06 +04:00
|
|
|
// Package raft implements raft.
|
2014-05-19 10:34:55 +04:00
|
|
|
package raft
|
|
|
|
|
2014-08-25 23:30:44 +04:00
|
|
|
import (
|
|
|
|
"sort"
|
|
|
|
|
|
|
|
"code.google.com/p/go.net/context"
|
|
|
|
)
|
2014-06-05 21:49:34 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
type stateResp struct {
|
|
|
|
st State
|
|
|
|
ents, cents []Entry
|
|
|
|
msgs []Message
|
2014-05-19 10:34:55 +04:00
|
|
|
}
|
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
func (a State) Equal(b State) bool {
|
|
|
|
return a.Term == b.Term && a.Vote == b.Vote && a.LastIndex == b.LastIndex
|
|
|
|
}
|
2014-05-29 00:53:26 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
func (sr stateResp) containsUpdates(prev stateResp) bool {
|
|
|
|
return !prev.st.Equal(sr.st) || len(sr.ents) > 0 || len(sr.cents) > 0 || len(sr.msgs) > 0
|
2014-06-05 21:49:34 +04:00
|
|
|
}
|
|
|
|
|
2014-05-19 10:34:55 +04:00
|
|
|
type Node struct {
|
2014-08-25 07:09:06 +04:00
|
|
|
ctx context.Context
|
2014-08-25 23:10:10 +04:00
|
|
|
propc chan Message
|
2014-08-25 07:09:06 +04:00
|
|
|
recvc chan Message
|
|
|
|
statec chan stateResp
|
|
|
|
tickc chan struct{}
|
2014-05-19 10:34:55 +04:00
|
|
|
}
|
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
func Start(ctx context.Context, id int64, peers []int64) *Node {
|
2014-05-19 10:34:55 +04:00
|
|
|
n := &Node{
|
2014-08-25 07:09:06 +04:00
|
|
|
ctx: ctx,
|
2014-08-25 23:10:10 +04:00
|
|
|
propc: make(chan Message),
|
2014-08-25 07:09:06 +04:00
|
|
|
recvc: make(chan Message),
|
|
|
|
statec: make(chan stateResp),
|
|
|
|
tickc: make(chan struct{}),
|
|
|
|
}
|
|
|
|
r := newRaft(id, peers)
|
|
|
|
go n.run(r)
|
2014-05-19 10:34:55 +04:00
|
|
|
return n
|
|
|
|
}
|
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
func (n *Node) run(r *raft) {
|
|
|
|
propc := n.propc
|
|
|
|
statec := n.statec
|
2014-06-20 01:39:17 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
var prev stateResp
|
|
|
|
for {
|
|
|
|
if r.hasLeader() {
|
|
|
|
propc = n.propc
|
|
|
|
} else {
|
|
|
|
// We cannot accept proposals because we don't know who
|
|
|
|
// to send them to, so we'll apply back-pressure and
|
|
|
|
// block senders.
|
|
|
|
propc = nil
|
2014-07-11 13:35:31 +04:00
|
|
|
}
|
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
sr := stateResp{
|
|
|
|
r.State,
|
|
|
|
r.raftLog.unstableEnts(),
|
|
|
|
r.raftLog.nextEnts(),
|
|
|
|
r.msgs,
|
2014-05-29 00:53:26 +04:00
|
|
|
}
|
2014-05-19 11:24:02 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
if sr.containsUpdates(prev) {
|
|
|
|
statec = n.statec
|
|
|
|
} else {
|
|
|
|
statec = nil
|
2014-06-05 21:49:34 +04:00
|
|
|
}
|
2014-06-24 02:45:45 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
select {
|
2014-08-25 23:10:10 +04:00
|
|
|
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:
|
|
|
|
// r.tick()
|
|
|
|
case statec <- sr:
|
|
|
|
r.raftLog.resetNextEnts()
|
|
|
|
r.raftLog.resetUnstable()
|
|
|
|
r.msgs = nil
|
|
|
|
case <-n.ctx.Done():
|
|
|
|
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
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
func (n *Node) Tick() error {
|
|
|
|
select {
|
|
|
|
case n.tickc <- struct{}{}:
|
|
|
|
return nil
|
|
|
|
case <-n.ctx.Done():
|
|
|
|
return n.ctx.Err()
|
2014-06-08 13:50:39 +04:00
|
|
|
}
|
2014-06-05 21:49:34 +04:00
|
|
|
}
|
2014-07-24 08:45:57 +04:00
|
|
|
|
2014-08-25 07:09:06 +04:00
|
|
|
// Propose proposes data be appended to the log.
|
|
|
|
func (n *Node) Propose(ctx context.Context, data []byte) error {
|
2014-08-25 23:30:44 +04:00
|
|
|
return n.Step(ctx, []Message{{Type: msgProp, Entries: []Entry{{Data: data}}}})
|
2014-07-25 01:11:53 +04:00
|
|
|
}
|
2014-07-31 04:21:27 +04:00
|
|
|
|
2014-08-25 23:30:44 +04:00
|
|
|
// Step advances the state machine using msgs. Proposals are priotized last so
|
|
|
|
// that any votes and vote requests will not be wedged behind proposals and
|
|
|
|
// prevent this cluster from making progress.
|
|
|
|
func (n *Node) Step(ctx context.Context, msgs []Message) error {
|
|
|
|
sort.Sort(sort.Reverse(messages(msgs)))
|
|
|
|
for _, m := range msgs {
|
|
|
|
ch := n.recvc
|
|
|
|
if m.Type == msgProp {
|
|
|
|
ch = n.propc
|
|
|
|
}
|
2014-08-25 23:10:10 +04:00
|
|
|
|
2014-08-25 23:30:44 +04:00
|
|
|
select {
|
|
|
|
case ch <- m:
|
|
|
|
return nil
|
|
|
|
case <-ctx.Done():
|
|
|
|
return ctx.Err()
|
|
|
|
case <-n.ctx.Done():
|
|
|
|
return n.ctx.Err()
|
|
|
|
}
|
2014-08-01 02:18:44 +04:00
|
|
|
}
|
2014-08-25 23:30:44 +04:00
|
|
|
return nil
|
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) ReadState(ctx context.Context) (st State, ents, cents []Entry, msgs []Message, err error) {
|
|
|
|
select {
|
|
|
|
case sr := <-n.statec:
|
|
|
|
return sr.st, sr.ents, sr.cents, sr.msgs, nil
|
|
|
|
case <-ctx.Done():
|
|
|
|
return State{}, nil, nil, nil, ctx.Err()
|
|
|
|
case <-n.ctx.Done():
|
|
|
|
return State{}, nil, nil, nil, n.ctx.Err()
|
|
|
|
}
|
2014-07-31 04:21:27 +04:00
|
|
|
}
|
2014-08-25 23:30:44 +04:00
|
|
|
|
|
|
|
type messages []Message
|
|
|
|
|
|
|
|
func (msgs messages) Len() int { return len(msgs) }
|
|
|
|
func (msgs messages) Less(i, j int) bool { return msgs[i].Type == msgProp }
|
|
|
|
func (msgs messages) Swap(i, j int) { msgs[i], msgs[j] = msgs[i], msgs[j] }
|