Merge pull request #1586 from xiangli-cmu/fix_node
*: add Advance interface to raft.Node
This commit is contained in:
@ -343,6 +343,8 @@ func (s *EtcdServer) run() {
|
|||||||
appliedi = rd.Snapshot.Index
|
appliedi = rd.Snapshot.Index
|
||||||
}
|
}
|
||||||
|
|
||||||
|
s.node.Advance()
|
||||||
|
|
||||||
if appliedi-snapi > s.snapCount {
|
if appliedi-snapi > s.snapCount {
|
||||||
s.snapshot(appliedi, nodes)
|
s.snapshot(appliedi, nodes)
|
||||||
snapi = appliedi
|
snapi = appliedi
|
||||||
|
@ -828,6 +828,7 @@ func TestTriggerSnap(t *testing.T) {
|
|||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1)
|
n := raft.StartNode(0xBAD0, mustMakePeerSlice(t, 0xBAD0), 10, 1)
|
||||||
<-n.Ready()
|
<-n.Ready()
|
||||||
|
n.Advance()
|
||||||
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0})
|
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 0xBAD0})
|
||||||
n.Campaign(ctx)
|
n.Campaign(ctx)
|
||||||
st := &storeRecorder{}
|
st := &storeRecorder{}
|
||||||
@ -1267,6 +1268,7 @@ func (n *readyNode) ProposeConfChange(ctx context.Context, conf raftpb.ConfChang
|
|||||||
}
|
}
|
||||||
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
|
||||||
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
|
||||||
|
func (n *readyNode) Advance() {}
|
||||||
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) {}
|
func (n *readyNode) ApplyConfChange(conf raftpb.ConfChange) {}
|
||||||
func (n *readyNode) Stop() {}
|
func (n *readyNode) Stop() {}
|
||||||
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
func (n *readyNode) Compact(index uint64, nodes []uint64, d []byte) {}
|
||||||
@ -1275,9 +1277,8 @@ type nodeRecorder struct {
|
|||||||
recorder
|
recorder
|
||||||
}
|
}
|
||||||
|
|
||||||
func (n *nodeRecorder) Tick() {
|
func (n *nodeRecorder) Tick() { n.record(action{name: "Tick"}) }
|
||||||
n.record(action{name: "Tick"})
|
|
||||||
}
|
|
||||||
func (n *nodeRecorder) Campaign(ctx context.Context) error {
|
func (n *nodeRecorder) Campaign(ctx context.Context) error {
|
||||||
n.record(action{name: "Campaign"})
|
n.record(action{name: "Campaign"})
|
||||||
return nil
|
return nil
|
||||||
@ -1295,6 +1296,7 @@ func (n *nodeRecorder) Step(ctx context.Context, msg raftpb.Message) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
func (n *nodeRecorder) Ready() <-chan raft.Ready { return nil }
|
func (n *nodeRecorder) Ready() <-chan raft.Ready { return nil }
|
||||||
|
func (n *nodeRecorder) Advance() {}
|
||||||
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
func (n *nodeRecorder) ApplyConfChange(conf raftpb.ConfChange) {
|
||||||
n.record(action{name: "ApplyConfChange", params: []interface{}{conf}})
|
n.record(action{name: "ApplyConfChange", params: []interface{}{conf}})
|
||||||
}
|
}
|
||||||
|
@ -51,8 +51,9 @@ The total state machine handling loop will look something like this:
|
|||||||
n.Tick()
|
n.Tick()
|
||||||
case rd := <-s.Node.Ready():
|
case rd := <-s.Node.Ready():
|
||||||
saveToStable(rd.State, rd.Entries)
|
saveToStable(rd.State, rd.Entries)
|
||||||
process(rd.CommittedEntries)
|
|
||||||
send(rd.Messages)
|
send(rd.Messages)
|
||||||
|
process(rd.CommittedEntries)
|
||||||
|
s.Node.Advance()
|
||||||
case <-s.done:
|
case <-s.done:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
18
raft/log.go
18
raft/log.go
@ -18,6 +18,7 @@ package raft
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"log"
|
||||||
|
|
||||||
pb "github.com/coreos/etcd/raft/raftpb"
|
pb "github.com/coreos/etcd/raft/raftpb"
|
||||||
)
|
)
|
||||||
@ -132,6 +133,23 @@ func (l *raftLog) resetNextEnts() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (l *raftLog) appliedTo(i uint64) {
|
||||||
|
if i == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if l.committed < i || i < l.applied {
|
||||||
|
log.Panicf("applied[%d] is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed)
|
||||||
|
}
|
||||||
|
l.applied = i
|
||||||
|
}
|
||||||
|
|
||||||
|
func (l *raftLog) stableTo(i uint64) {
|
||||||
|
if i == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
l.unstable = i + 1
|
||||||
|
}
|
||||||
|
|
||||||
func (l *raftLog) lastIndex() uint64 { return uint64(len(l.ents)) - 1 + l.offset }
|
func (l *raftLog) lastIndex() uint64 { return uint64(len(l.ents)) - 1 + l.offset }
|
||||||
|
|
||||||
func (l *raftLog) lastTerm() uint64 { return l.term(l.lastIndex()) }
|
func (l *raftLog) lastTerm() uint64 { return l.term(l.lastIndex()) }
|
||||||
|
75
raft/node.go
75
raft/node.go
@ -116,7 +116,11 @@ type Node interface {
|
|||||||
// Step advances the state machine using the given message. ctx.Err() will be returned, if any.
|
// Step advances the state machine using the given message. ctx.Err() will be returned, if any.
|
||||||
Step(ctx context.Context, msg pb.Message) error
|
Step(ctx context.Context, msg pb.Message) error
|
||||||
// Ready returns a channel that returns the current point-in-time state
|
// Ready returns a channel that returns the current point-in-time state
|
||||||
|
// Users of the Node must call Advance after applying the state returned by Ready
|
||||||
Ready() <-chan Ready
|
Ready() <-chan Ready
|
||||||
|
// Advance notifies the Node that the application has applied and saved progress up to the last Ready.
|
||||||
|
// It prepares the node to return the next available Ready.
|
||||||
|
Advance()
|
||||||
// ApplyConfChange applies config change to the local node.
|
// ApplyConfChange applies config change to the local node.
|
||||||
// TODO: reject existing node when add node
|
// TODO: reject existing node when add node
|
||||||
// TODO: reject non-existant node when remove node
|
// TODO: reject non-existant node when remove node
|
||||||
@ -185,6 +189,7 @@ type node struct {
|
|||||||
compactc chan compact
|
compactc chan compact
|
||||||
confc chan pb.ConfChange
|
confc chan pb.ConfChange
|
||||||
readyc chan Ready
|
readyc chan Ready
|
||||||
|
advancec chan struct{}
|
||||||
tickc chan struct{}
|
tickc chan struct{}
|
||||||
done chan struct{}
|
done chan struct{}
|
||||||
}
|
}
|
||||||
@ -196,6 +201,7 @@ func newNode() node {
|
|||||||
compactc: make(chan compact),
|
compactc: make(chan compact),
|
||||||
confc: make(chan pb.ConfChange),
|
confc: make(chan pb.ConfChange),
|
||||||
readyc: make(chan Ready),
|
readyc: make(chan Ready),
|
||||||
|
advancec: make(chan struct{}),
|
||||||
tickc: make(chan struct{}),
|
tickc: make(chan struct{}),
|
||||||
done: make(chan struct{}),
|
done: make(chan struct{}),
|
||||||
}
|
}
|
||||||
@ -208,6 +214,9 @@ func (n *node) Stop() {
|
|||||||
func (n *node) run(r *raft) {
|
func (n *node) run(r *raft) {
|
||||||
var propc chan pb.Message
|
var propc chan pb.Message
|
||||||
var readyc chan Ready
|
var readyc chan Ready
|
||||||
|
var advancec chan struct{}
|
||||||
|
var prevLastUnstablei uint64
|
||||||
|
var rd Ready
|
||||||
|
|
||||||
lead := None
|
lead := None
|
||||||
prevSoftSt := r.softState()
|
prevSoftSt := r.softState()
|
||||||
@ -215,26 +224,30 @@ func (n *node) run(r *raft) {
|
|||||||
prevSnapi := r.raftLog.snapshot.Index
|
prevSnapi := r.raftLog.snapshot.Index
|
||||||
|
|
||||||
for {
|
for {
|
||||||
rd := newReady(r, prevSoftSt, prevHardSt, prevSnapi)
|
if advancec != nil {
|
||||||
if rd.containsUpdates() {
|
|
||||||
readyc = n.readyc
|
|
||||||
} else {
|
|
||||||
readyc = nil
|
readyc = nil
|
||||||
}
|
} else {
|
||||||
|
rd = newReady(r, prevSoftSt, prevHardSt, prevSnapi)
|
||||||
if rd.SoftState != nil && lead != rd.SoftState.Lead {
|
if rd.containsUpdates() {
|
||||||
if r.hasLeader() {
|
readyc = n.readyc
|
||||||
if lead == None {
|
|
||||||
log.Printf("raft: elected leader %x at term %d", rd.SoftState.Lead, r.Term)
|
|
||||||
} else {
|
|
||||||
log.Printf("raft: leader changed from %x to %x at term %d", lead, rd.SoftState.Lead, r.Term)
|
|
||||||
}
|
|
||||||
propc = n.propc
|
|
||||||
} else {
|
} else {
|
||||||
log.Printf("raft: lost leader %x at term %d", lead, r.Term)
|
readyc = nil
|
||||||
propc = nil
|
}
|
||||||
|
|
||||||
|
if rd.SoftState != nil && lead != rd.SoftState.Lead {
|
||||||
|
if r.hasLeader() {
|
||||||
|
if lead == None {
|
||||||
|
log.Printf("raft: elected leader %x at term %d", rd.SoftState.Lead, r.Term)
|
||||||
|
} else {
|
||||||
|
log.Printf("raft: leader changed from %x to %x at term %d", lead, rd.SoftState.Lead, r.Term)
|
||||||
|
}
|
||||||
|
propc = n.propc
|
||||||
|
} else {
|
||||||
|
log.Printf("raft: lost leader %x at term %d", lead, r.Term)
|
||||||
|
propc = nil
|
||||||
|
}
|
||||||
|
lead = rd.SoftState.Lead
|
||||||
}
|
}
|
||||||
lead = rd.SoftState.Lead
|
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
select {
|
||||||
@ -267,19 +280,28 @@ func (n *node) run(r *raft) {
|
|||||||
if rd.SoftState != nil {
|
if rd.SoftState != nil {
|
||||||
prevSoftSt = rd.SoftState
|
prevSoftSt = rd.SoftState
|
||||||
}
|
}
|
||||||
|
if len(rd.Entries) > 0 {
|
||||||
|
prevLastUnstablei = rd.Entries[len(rd.Entries)-1].Index
|
||||||
|
}
|
||||||
if !IsEmptyHardState(rd.HardState) {
|
if !IsEmptyHardState(rd.HardState) {
|
||||||
prevHardSt = rd.HardState
|
prevHardSt = rd.HardState
|
||||||
}
|
}
|
||||||
if !IsEmptySnap(rd.Snapshot) {
|
if !IsEmptySnap(rd.Snapshot) {
|
||||||
prevSnapi = rd.Snapshot.Index
|
prevSnapi = rd.Snapshot.Index
|
||||||
|
if prevSnapi > prevLastUnstablei {
|
||||||
|
prevLastUnstablei = prevSnapi
|
||||||
|
}
|
||||||
}
|
}
|
||||||
// TODO(yichengq): we assume that all committed config
|
|
||||||
// entries will be applied to make things easy for now.
|
|
||||||
// TODO(yichengq): it may have race because applied is set
|
|
||||||
// before entries are applied.
|
|
||||||
r.raftLog.resetNextEnts()
|
|
||||||
r.raftLog.resetUnstable()
|
|
||||||
r.msgs = nil
|
r.msgs = nil
|
||||||
|
advancec = n.advancec
|
||||||
|
case <-advancec:
|
||||||
|
if prevHardSt.Commit != 0 {
|
||||||
|
r.raftLog.appliedTo(prevHardSt.Commit)
|
||||||
|
}
|
||||||
|
if prevLastUnstablei != 0 {
|
||||||
|
r.raftLog.stableTo(prevLastUnstablei)
|
||||||
|
}
|
||||||
|
advancec = nil
|
||||||
case <-n.done:
|
case <-n.done:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -342,6 +364,13 @@ func (n *node) Ready() <-chan Ready {
|
|||||||
return n.readyc
|
return n.readyc
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (n *node) Advance() {
|
||||||
|
select {
|
||||||
|
case n.advancec <- struct{}{}:
|
||||||
|
case <-n.done:
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func (n *node) ApplyConfChange(cc pb.ConfChange) {
|
func (n *node) ApplyConfChange(cc pb.ConfChange) {
|
||||||
select {
|
select {
|
||||||
case n.confc <- cc:
|
case n.confc <- cc:
|
||||||
|
@ -26,12 +26,16 @@ func BenchmarkOneNode(b *testing.B) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
n := StartNode(1, []Peer{{ID: 1}}, 0, 0)
|
n := newNode()
|
||||||
|
r := newRaft(1, []uint64{1}, 10, 1)
|
||||||
|
go n.run(r)
|
||||||
|
|
||||||
defer n.Stop()
|
defer n.Stop()
|
||||||
|
|
||||||
n.Campaign(ctx)
|
n.Campaign(ctx)
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
<-n.Ready()
|
<-n.Ready()
|
||||||
|
n.Advance()
|
||||||
n.Propose(ctx, []byte("foo"))
|
n.Propose(ctx, []byte("foo"))
|
||||||
}
|
}
|
||||||
rd := <-n.Ready()
|
rd := <-n.Ready()
|
||||||
|
@ -195,11 +195,15 @@ func TestNode(t *testing.T) {
|
|||||||
n.Campaign(ctx)
|
n.Campaign(ctx)
|
||||||
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[0]) {
|
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[0]) {
|
||||||
t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0])
|
t.Errorf("#%d: g = %+v,\n w %+v", 1, g, wants[0])
|
||||||
|
} else {
|
||||||
|
n.Advance()
|
||||||
}
|
}
|
||||||
|
|
||||||
n.Propose(ctx, []byte("foo"))
|
n.Propose(ctx, []byte("foo"))
|
||||||
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) {
|
if g := <-n.Ready(); !reflect.DeepEqual(g, wants[1]) {
|
||||||
t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1])
|
t.Errorf("#%d: g = %+v,\n w %+v", 2, g, wants[1])
|
||||||
|
} else {
|
||||||
|
n.Advance()
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
select {
|
||||||
@ -226,6 +230,8 @@ func TestNodeRestart(t *testing.T) {
|
|||||||
n := RestartNode(1, 10, 1, nil, st, entries)
|
n := RestartNode(1, 10, 1, nil, st, entries)
|
||||||
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
if g := <-n.Ready(); !reflect.DeepEqual(g, want) {
|
||||||
t.Errorf("g = %+v,\n w %+v", g, want)
|
t.Errorf("g = %+v,\n w %+v", g, want)
|
||||||
|
} else {
|
||||||
|
n.Advance()
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
select {
|
||||||
@ -256,6 +262,7 @@ func TestNodeCompact(t *testing.T) {
|
|||||||
testutil.ForceGosched()
|
testutil.ForceGosched()
|
||||||
select {
|
select {
|
||||||
case <-n.Ready():
|
case <-n.Ready():
|
||||||
|
n.Advance()
|
||||||
default:
|
default:
|
||||||
t.Fatalf("unexpected proposal failure: unable to commit entry")
|
t.Fatalf("unexpected proposal failure: unable to commit entry")
|
||||||
}
|
}
|
||||||
@ -267,6 +274,7 @@ func TestNodeCompact(t *testing.T) {
|
|||||||
if !reflect.DeepEqual(rd.Snapshot, w) {
|
if !reflect.DeepEqual(rd.Snapshot, w) {
|
||||||
t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
|
t.Errorf("snap = %+v, want %+v", rd.Snapshot, w)
|
||||||
}
|
}
|
||||||
|
n.Advance()
|
||||||
default:
|
default:
|
||||||
t.Fatalf("unexpected compact failure: unable to create a snapshot")
|
t.Fatalf("unexpected compact failure: unable to create a snapshot")
|
||||||
}
|
}
|
||||||
@ -285,6 +293,28 @@ func TestNodeCompact(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestNodeAdvance(t *testing.T) {
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
n := StartNode(1, []Peer{{ID: 1}}, 10, 1)
|
||||||
|
n.ApplyConfChange(raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1})
|
||||||
|
n.Campaign(ctx)
|
||||||
|
<-n.Ready()
|
||||||
|
n.Propose(ctx, []byte("foo"))
|
||||||
|
select {
|
||||||
|
case rd := <-n.Ready():
|
||||||
|
t.Fatalf("unexpected Ready before Advance: %+v", rd)
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
n.Advance()
|
||||||
|
select {
|
||||||
|
case <-n.Ready():
|
||||||
|
default:
|
||||||
|
t.Errorf("expect Ready after Advance, but there is no Ready available")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestSoftStateEqual(t *testing.T) {
|
func TestSoftStateEqual(t *testing.T) {
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
st *SoftState
|
st *SoftState
|
||||||
|
Reference in New Issue
Block a user