|
@@ -177,8 +177,7 @@ type EtcdServer struct {
|
|
|
|
|
|
|
|
snapCount uint64
|
|
snapCount uint64
|
|
|
|
|
|
|
|
- w wait.Wait
|
|
|
|
|
- td *contention.TimeoutDetector
|
|
|
|
|
|
|
+ w wait.Wait
|
|
|
|
|
|
|
|
readMu sync.RWMutex
|
|
readMu sync.RWMutex
|
|
|
// read routine notifies etcd server that it waits for reading by sending an empty struct to
|
|
// read routine notifies etcd server that it waits for reading by sending an empty struct to
|
|
@@ -233,8 +232,6 @@ type EtcdServer struct {
|
|
|
// to detect the cluster version immediately.
|
|
// to detect the cluster version immediately.
|
|
|
forceVersionC chan struct{}
|
|
forceVersionC chan struct{}
|
|
|
|
|
|
|
|
- msgSnapC chan raftpb.Message
|
|
|
|
|
-
|
|
|
|
|
// wgMu blocks concurrent waitgroup mutation while server stopping
|
|
// wgMu blocks concurrent waitgroup mutation while server stopping
|
|
|
wgMu sync.RWMutex
|
|
wgMu sync.RWMutex
|
|
|
// wg is used to wait for the go routines that depends on the server state
|
|
// wg is used to wait for the go routines that depends on the server state
|
|
@@ -399,16 +396,19 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
|
|
|
readych: make(chan struct{}),
|
|
readych: make(chan struct{}),
|
|
|
Cfg: cfg,
|
|
Cfg: cfg,
|
|
|
snapCount: cfg.SnapCount,
|
|
snapCount: cfg.SnapCount,
|
|
|
- // 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,
|
|
|
|
|
|
|
+ errorc: make(chan error, 1),
|
|
|
|
|
+ store: st,
|
|
|
r: raftNode{
|
|
r: raftNode{
|
|
|
|
|
+ isIDRemoved: func(id uint64) bool { return cl.IsIDRemoved(types.ID(id)) },
|
|
|
Node: n,
|
|
Node: n,
|
|
|
ticker: time.Tick(heartbeat),
|
|
ticker: time.Tick(heartbeat),
|
|
|
|
|
+ // set up contention detectors for raft heartbeat message.
|
|
|
|
|
+ // expect to send a heartbeat within 2 heartbeat intervals.
|
|
|
|
|
+ td: contention.NewTimeoutDetector(2 * heartbeat),
|
|
|
|
|
+ heartbeat: heartbeat,
|
|
|
raftStorage: s,
|
|
raftStorage: s,
|
|
|
storage: NewStorage(w, ss),
|
|
storage: NewStorage(w, ss),
|
|
|
|
|
+ msgSnapC: make(chan raftpb.Message, maxInFlightMsgSnap),
|
|
|
readStateC: make(chan raft.ReadState, 1),
|
|
readStateC: make(chan raft.ReadState, 1),
|
|
|
},
|
|
},
|
|
|
id: id,
|
|
id: id,
|
|
@@ -420,7 +420,6 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
|
|
|
peerRt: prt,
|
|
peerRt: prt,
|
|
|
reqIDGen: idutil.NewGenerator(uint16(id), time.Now()),
|
|
reqIDGen: idutil.NewGenerator(uint16(id), time.Now()),
|
|
|
forceVersionC: make(chan struct{}),
|
|
forceVersionC: make(chan struct{}),
|
|
|
- msgSnapC: make(chan raftpb.Message, maxInFlightMsgSnap),
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
|
|
srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
|
|
@@ -583,7 +582,6 @@ type etcdProgress struct {
|
|
|
// TODO: add a state machine interface to apply the commit entries and do snapshot/recover
|
|
// TODO: add a state machine interface to apply the commit entries and do snapshot/recover
|
|
|
type raftReadyHandler struct {
|
|
type raftReadyHandler struct {
|
|
|
leadershipUpdate func()
|
|
leadershipUpdate func()
|
|
|
- sendMessage func(msgs []raftpb.Message)
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (s *EtcdServer) run() {
|
|
func (s *EtcdServer) run() {
|
|
@@ -629,11 +627,10 @@ func (s *EtcdServer) run() {
|
|
|
if s.compactor != nil {
|
|
if s.compactor != nil {
|
|
|
s.compactor.Resume()
|
|
s.compactor.Resume()
|
|
|
}
|
|
}
|
|
|
- if s.td != nil {
|
|
|
|
|
- s.td.Reset()
|
|
|
|
|
|
|
+ if s.r.td != nil {
|
|
|
|
|
+ s.r.td.Reset()
|
|
|
}
|
|
}
|
|
|
},
|
|
},
|
|
|
- sendMessage: func(msgs []raftpb.Message) { s.send(msgs) },
|
|
|
|
|
}
|
|
}
|
|
|
s.r.start(rh)
|
|
s.r.start(rh)
|
|
|
|
|
|
|
@@ -745,7 +742,7 @@ func (s *EtcdServer) applyAll(ep *etcdProgress, apply *apply) {
|
|
|
s.triggerSnapshot(ep)
|
|
s.triggerSnapshot(ep)
|
|
|
select {
|
|
select {
|
|
|
// snapshot requested via send()
|
|
// snapshot requested via send()
|
|
|
- case m := <-s.msgSnapC:
|
|
|
|
|
|
|
+ case m := <-s.r.msgSnapC:
|
|
|
merged := s.createMergedSnapshotMessage(m, ep.appliedi, ep.confState)
|
|
merged := s.createMergedSnapshotMessage(m, ep.appliedi, ep.confState)
|
|
|
s.sendMergedSnap(merged)
|
|
s.sendMergedSnap(merged)
|
|
|
default:
|
|
default:
|
|
@@ -1165,47 +1162,6 @@ func (s *EtcdServer) publish(timeout time.Duration) {
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
-// TODO: move this function into raft.go
|
|
|
|
|
-func (s *EtcdServer) send(ms []raftpb.Message) {
|
|
|
|
|
- sentAppResp := false
|
|
|
|
|
- for i := len(ms) - 1; i >= 0; i-- {
|
|
|
|
|
- if s.cluster.IsIDRemoved(types.ID(ms[i].To)) {
|
|
|
|
|
- ms[i].To = 0
|
|
|
|
|
- }
|
|
|
|
|
-
|
|
|
|
|
- if ms[i].Type == raftpb.MsgAppResp {
|
|
|
|
|
- if sentAppResp {
|
|
|
|
|
- ms[i].To = 0
|
|
|
|
|
- } else {
|
|
|
|
|
- sentAppResp = true
|
|
|
|
|
- }
|
|
|
|
|
- }
|
|
|
|
|
-
|
|
|
|
|
- if ms[i].Type == raftpb.MsgSnap {
|
|
|
|
|
- // There are two separate data store: the store for v2, and the KV for v3.
|
|
|
|
|
- // The msgSnap only contains the most recent snapshot of store without KV.
|
|
|
|
|
- // So we need to redirect the msgSnap to etcd server main loop for merging in the
|
|
|
|
|
- // current store snapshot and KV snapshot.
|
|
|
|
|
- select {
|
|
|
|
|
- case s.msgSnapC <- ms[i]:
|
|
|
|
|
- default:
|
|
|
|
|
- // drop msgSnap if the inflight chan if full.
|
|
|
|
|
- }
|
|
|
|
|
- ms[i].To = 0
|
|
|
|
|
- }
|
|
|
|
|
- if ms[i].Type == raftpb.MsgHeartbeat {
|
|
|
|
|
- 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)
|
|
|
|
|
- plog.Warningf("server is likely overloaded")
|
|
|
|
|
- }
|
|
|
|
|
- }
|
|
|
|
|
- }
|
|
|
|
|
-
|
|
|
|
|
- s.r.transport.Send(ms)
|
|
|
|
|
-}
|
|
|
|
|
-
|
|
|
|
|
func (s *EtcdServer) sendMergedSnap(merged snap.Message) {
|
|
func (s *EtcdServer) sendMergedSnap(merged snap.Message) {
|
|
|
atomic.AddInt64(&s.inflightSnapshots, 1)
|
|
atomic.AddInt64(&s.inflightSnapshots, 1)
|
|
|
|
|
|