node.go 9.6 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353
  1. package raft
  2. import (
  3. "errors"
  4. "log"
  5. "reflect"
  6. "github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
  7. pb "github.com/coreos/etcd/raft/raftpb"
  8. )
  9. var (
  10. emptyState = pb.HardState{}
  11. // ErrStopped is returned by methods on Nodes that have been stopped.
  12. ErrStopped = errors.New("raft: stopped")
  13. )
  14. // SoftState provides state that is useful for logging and debugging.
  15. // The state is volatile and does not need to be persisted to the WAL.
  16. type SoftState struct {
  17. Lead uint64
  18. RaftState StateType
  19. Nodes []uint64
  20. RemovedNodes []uint64
  21. ShouldStop bool
  22. }
  23. func (a *SoftState) equal(b *SoftState) bool {
  24. return reflect.DeepEqual(a, b)
  25. }
  26. // Ready encapsulates the entries and messages that are ready to read,
  27. // be saved to stable storage, committed or sent to other peers.
  28. // All fields in Ready are read-only.
  29. type Ready struct {
  30. // The current volatile state of a Node.
  31. // SoftState will be nil if there is no update.
  32. // It is not required to consume or store SoftState.
  33. *SoftState
  34. // The current state of a Node to be saved to stable storage BEFORE
  35. // Messages are sent.
  36. // HardState will be equal to empty state if there is no update.
  37. pb.HardState
  38. // Entries specifies entries to be saved to stable storage BEFORE
  39. // Messages are sent.
  40. Entries []pb.Entry
  41. // Snapshot specifies the snapshot to be saved to stable storage.
  42. Snapshot pb.Snapshot
  43. // CommittedEntries specifies entries to be committed to a
  44. // store/state-machine. These have previously been committed to stable
  45. // store.
  46. CommittedEntries []pb.Entry
  47. // Messages specifies outbound messages to be sent AFTER Entries are
  48. // committed to stable storage.
  49. Messages []pb.Message
  50. }
  51. type compact struct {
  52. index uint64
  53. nodes []uint64
  54. data []byte
  55. }
  56. func isHardStateEqual(a, b pb.HardState) bool {
  57. return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
  58. }
  59. // IsEmptyHardState returns true if the given HardState is empty.
  60. func IsEmptyHardState(st pb.HardState) bool {
  61. return isHardStateEqual(st, emptyState)
  62. }
  63. // IsEmptySnap returns true if the given Snapshot is empty.
  64. func IsEmptySnap(sp pb.Snapshot) bool {
  65. return sp.Index == 0
  66. }
  67. func (rd Ready) containsUpdates() bool {
  68. return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) || !IsEmptySnap(rd.Snapshot) ||
  69. len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
  70. }
  71. // Node represents a node in a raft cluster.
  72. type Node interface {
  73. // Tick increments the internal logical clock for the Node by a single tick. Election
  74. // timeouts and heartbeat timeouts are in units of ticks.
  75. Tick()
  76. // Campaign causes the Node to transition to candidate state and start campaigning to become leader.
  77. Campaign(ctx context.Context) error
  78. // Propose proposes that data be appended to the log.
  79. Propose(ctx context.Context, data []byte) error
  80. // ProposeConfChange proposes config change.
  81. // At most one ConfChange can be in the process of going through consensus.
  82. // Application needs to call ApplyConfChange when applying EntryConfChange type entry.
  83. ProposeConfChange(ctx context.Context, cc pb.ConfChange) error
  84. // Step advances the state machine using the given message. ctx.Err() will be returned, if any.
  85. Step(ctx context.Context, msg pb.Message) error
  86. // Ready returns a channel that returns the current point-in-time state
  87. Ready() <-chan Ready
  88. // ApplyConfChange applies config change to the local node.
  89. // TODO: reject existing node when add node
  90. // TODO: reject non-existant node when remove node
  91. ApplyConfChange(cc pb.ConfChange)
  92. // Stop performs any necessary termination of the Node
  93. Stop()
  94. // Compact discards the entrire log up to the given index. It also
  95. // generates a raft snapshot containing the given nodes configuration
  96. // and the given snapshot data.
  97. // It is the caller's responsibility to ensure the given configuration
  98. // and snapshot data match the actual point-in-time configuration and snapshot
  99. // at the given index.
  100. Compact(index uint64, nodes []uint64, d []byte)
  101. }
  102. type Peer struct {
  103. ID uint64
  104. Context []byte
  105. }
  106. // StartNode returns a new Node given a unique raft id, a list of raft peers, and
  107. // the election and heartbeat timeouts in units of ticks.
  108. // It also builds ConfChangeAddNode entry for each peer and puts them at the head of the log.
  109. func StartNode(id uint64, peers []Peer, election, heartbeat int) Node {
  110. n := newNode()
  111. r := newRaft(id, nil, election, heartbeat)
  112. ents := make([]pb.Entry, len(peers))
  113. for i, peer := range peers {
  114. cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
  115. data, err := cc.Marshal()
  116. if err != nil {
  117. panic("unexpected marshal error")
  118. }
  119. ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
  120. }
  121. r.raftLog.append(0, ents...)
  122. r.raftLog.committed = uint64(len(ents))
  123. go n.run(r)
  124. return &n
  125. }
  126. // RestartNode is identical to StartNode but takes an initial State and a slice
  127. // of entries. Generally this is used when restarting from a stable storage
  128. // log.
  129. func RestartNode(id uint64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
  130. n := newNode()
  131. r := newRaft(id, nil, election, heartbeat)
  132. if snapshot != nil {
  133. r.restore(*snapshot)
  134. }
  135. r.loadState(st)
  136. r.loadEnts(ents)
  137. go n.run(r)
  138. return &n
  139. }
  140. // node is the canonical implementation of the Node interface
  141. type node struct {
  142. propc chan pb.Message
  143. recvc chan pb.Message
  144. compactc chan compact
  145. confc chan pb.ConfChange
  146. readyc chan Ready
  147. tickc chan struct{}
  148. done chan struct{}
  149. }
  150. func newNode() node {
  151. return node{
  152. propc: make(chan pb.Message),
  153. recvc: make(chan pb.Message),
  154. compactc: make(chan compact),
  155. confc: make(chan pb.ConfChange),
  156. readyc: make(chan Ready),
  157. tickc: make(chan struct{}),
  158. done: make(chan struct{}),
  159. }
  160. }
  161. func (n *node) Stop() {
  162. close(n.done)
  163. }
  164. func (n *node) run(r *raft) {
  165. var propc chan pb.Message
  166. var readyc chan Ready
  167. lead := None
  168. prevSoftSt := r.softState()
  169. prevHardSt := r.HardState
  170. prevSnapi := r.raftLog.snapshot.Index
  171. for {
  172. rd := newReady(r, prevSoftSt, prevHardSt, prevSnapi)
  173. if rd.containsUpdates() {
  174. readyc = n.readyc
  175. } else {
  176. readyc = nil
  177. }
  178. if rd.SoftState != nil && lead != rd.SoftState.Lead {
  179. log.Printf("raft: leader changed from %#x to %#x", lead, rd.SoftState.Lead)
  180. lead = rd.SoftState.Lead
  181. if r.hasLeader() {
  182. propc = n.propc
  183. } else {
  184. propc = nil
  185. }
  186. }
  187. select {
  188. // TODO: maybe buffer the config propose if there exists one (the way
  189. // described in raft dissertation)
  190. // Currently it is dropped in Step silently.
  191. case m := <-propc:
  192. m.From = r.id
  193. r.Step(m)
  194. case m := <-n.recvc:
  195. r.Step(m) // raft never returns an error
  196. case c := <-n.compactc:
  197. r.compact(c.index, c.nodes, c.data)
  198. case cc := <-n.confc:
  199. if cc.NodeID == None {
  200. r.resetPendingConf()
  201. break
  202. }
  203. switch cc.Type {
  204. case pb.ConfChangeAddNode:
  205. r.addNode(cc.NodeID)
  206. case pb.ConfChangeRemoveNode:
  207. r.removeNode(cc.NodeID)
  208. default:
  209. panic("unexpected conf type")
  210. }
  211. case <-n.tickc:
  212. r.tick()
  213. case readyc <- rd:
  214. if rd.SoftState != nil {
  215. prevSoftSt = rd.SoftState
  216. }
  217. if !IsEmptyHardState(rd.HardState) {
  218. prevHardSt = rd.HardState
  219. }
  220. if !IsEmptySnap(rd.Snapshot) {
  221. prevSnapi = rd.Snapshot.Index
  222. }
  223. // TODO(yichengq): we assume that all committed config
  224. // entries will be applied to make things easy for now.
  225. // TODO(yichengq): it may have race because applied is set
  226. // before entries are applied.
  227. r.raftLog.resetNextEnts()
  228. r.raftLog.resetUnstable()
  229. r.msgs = nil
  230. case <-n.done:
  231. return
  232. }
  233. }
  234. }
  235. // Tick increments the internal logical clock for this Node. Election timeouts
  236. // and heartbeat timeouts are in units of ticks.
  237. func (n *node) Tick() {
  238. select {
  239. case n.tickc <- struct{}{}:
  240. case <-n.done:
  241. }
  242. }
  243. func (n *node) Campaign(ctx context.Context) error {
  244. return n.step(ctx, pb.Message{Type: pb.MsgHup})
  245. }
  246. func (n *node) Propose(ctx context.Context, data []byte) error {
  247. return n.step(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
  248. }
  249. func (n *node) Step(ctx context.Context, m pb.Message) error {
  250. // ignore unexpected local messages receiving over network
  251. if m.Type == pb.MsgHup || m.Type == pb.MsgBeat {
  252. // TODO: return an error?
  253. return nil
  254. }
  255. return n.step(ctx, m)
  256. }
  257. func (n *node) ProposeConfChange(ctx context.Context, cc pb.ConfChange) error {
  258. data, err := cc.Marshal()
  259. if err != nil {
  260. return err
  261. }
  262. return n.Step(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange, Data: data}}})
  263. }
  264. // Step advances the state machine using msgs. The ctx.Err() will be returned,
  265. // if any.
  266. func (n *node) step(ctx context.Context, m pb.Message) error {
  267. ch := n.recvc
  268. if m.Type == pb.MsgProp {
  269. ch = n.propc
  270. }
  271. select {
  272. case ch <- m:
  273. return nil
  274. case <-ctx.Done():
  275. return ctx.Err()
  276. case <-n.done:
  277. return ErrStopped
  278. }
  279. }
  280. func (n *node) Ready() <-chan Ready {
  281. return n.readyc
  282. }
  283. func (n *node) ApplyConfChange(cc pb.ConfChange) {
  284. select {
  285. case n.confc <- cc:
  286. case <-n.done:
  287. }
  288. }
  289. func (n *node) Compact(index uint64, nodes []uint64, d []byte) {
  290. select {
  291. case n.compactc <- compact{index, nodes, d}:
  292. case <-n.done:
  293. }
  294. }
  295. func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState, prevSnapi uint64) Ready {
  296. rd := Ready{
  297. Entries: r.raftLog.unstableEnts(),
  298. CommittedEntries: r.raftLog.nextEnts(),
  299. Messages: r.msgs,
  300. }
  301. if softSt := r.softState(); !softSt.equal(prevSoftSt) {
  302. rd.SoftState = softSt
  303. }
  304. if !isHardStateEqual(r.HardState, prevHardSt) {
  305. rd.HardState = r.HardState
  306. }
  307. if prevSnapi != r.raftLog.snapshot.Index {
  308. rd.Snapshot = r.raftLog.snapshot
  309. }
  310. return rd
  311. }