etcdserver: separate EtcdServer from raftNode

This commit is contained in:
Gyu-Ho Lee
2016-10-07 13:18:39 -07:00
parent e1e16d9b28
commit e011ea25ca
3 changed files with 73 additions and 56 deletions

View File

@ -40,6 +40,7 @@ import (
"github.com/coreos/etcd/lease"
"github.com/coreos/etcd/mvcc"
"github.com/coreos/etcd/mvcc/backend"
"github.com/coreos/etcd/pkg/contention"
"github.com/coreos/etcd/pkg/fileutil"
"github.com/coreos/etcd/pkg/idutil"
"github.com/coreos/etcd/pkg/pbutil"
@ -176,7 +177,8 @@ type EtcdServer struct {
snapCount uint64
w wait.Wait
w wait.Wait
td *contention.TimeoutDetector
readMu sync.RWMutex
// read routine notifies etcd server that it waits for reading by sending an empty struct to
@ -390,15 +392,19 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
sstats.Initialize()
lstats := stats.NewLeaderStats(id.String())
heartbeat := time.Duration(cfg.TickMs) * time.Millisecond
srv = &EtcdServer{
readych: make(chan struct{}),
Cfg: cfg,
snapCount: cfg.SnapCount,
errorc: make(chan error, 1),
store: st,
// set up contention detectors for raft heartbeat message.
// expect to send a heartbeat within 2 heartbeat intervals.
td: contention.NewTimeoutDetector(2 * heartbeat),
errorc: make(chan error, 1),
store: st,
r: raftNode{
Node: n,
ticker: time.Tick(time.Duration(cfg.TickMs) * time.Millisecond),
ticker: time.Tick(heartbeat),
raftStorage: s,
storage: NewStorage(w, ss),
readStateC: make(chan raft.ReadState, 1),
@ -418,7 +424,7 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
srv.be = be
minTTL := time.Duration((3*cfg.ElectionTicks)/2) * time.Duration(cfg.TickMs) * time.Millisecond
minTTL := time.Duration((3*cfg.ElectionTicks)/2) * heartbeat
// always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
// If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
@ -570,12 +576,64 @@ type etcdProgress struct {
appliedi uint64
}
// raftReadyHandler contains a set of EtcdServer operations to be called by raftNode,
// and helps decouple state machine logic from Raft algorithms.
// TODO: add a state machine interface to apply the commit entries and do snapshot/recover
type raftReadyHandler struct {
leadershipUpdate func()
sendMessage func(msgs []raftpb.Message)
}
func (s *EtcdServer) run() {
snap, err := s.r.raftStorage.Snapshot()
if err != nil {
plog.Panicf("get snapshot from raft storage error: %v", err)
}
s.r.start(s)
var (
smu sync.RWMutex
syncC <-chan time.Time
)
setSyncC := func(ch <-chan time.Time) {
smu.Lock()
syncC = ch
smu.Unlock()
}
getSyncC := func() (ch <-chan time.Time) {
smu.RLock()
ch = syncC
smu.RUnlock()
return
}
rh := &raftReadyHandler{
leadershipUpdate: func() {
if !s.isLeader() {
if s.lessor != nil {
s.lessor.Demote()
}
if s.compactor != nil {
s.compactor.Pause()
}
setSyncC(nil)
} else {
setSyncC(s.SyncTicker)
}
// TODO: remove the nil checking
// current test utility does not provide the stats
if s.stats != nil {
s.stats.BecomeLeader()
}
if s.compactor != nil {
s.compactor.Resume()
}
if s.td != nil {
s.td.Reset()
}
},
sendMessage: func(msgs []raftpb.Message) { s.send(msgs) },
}
s.r.start(rh)
// asynchronously accept apply packets, dispatch progress in-order
sched := schedule.NewFIFOScheduler()
@ -655,6 +713,8 @@ func (s *EtcdServer) run() {
plog.Errorf("%s", err)
plog.Infof("the data-dir used by this member must be removed.")
return
case <-getSyncC():
s.sync(s.Cfg.ReqTimeout())
case <-s.stop:
return
}
@ -1130,7 +1190,7 @@ func (s *EtcdServer) send(ms []raftpb.Message) {
ms[i].To = 0
}
if ms[i].Type == raftpb.MsgHeartbeat {
ok, exceed := s.r.td.Observe(ms[i].To)
ok, exceed := s.td.Observe(ms[i].To)
if !ok {
// TODO: limit request rate.
plog.Warningf("failed to send out heartbeat on time (exceeded the %dms timeout for %v)", s.Cfg.TickMs, exceed)