multinode.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481
  1. package raft
  2. import (
  3. pb "github.com/coreos/etcd/raft/raftpb"
  4. "golang.org/x/net/context"
  5. )
  6. // MultiNode represents a node that is participating in multiple consensus groups.
  7. // A MultiNode is more efficient than a collection of Nodes.
  8. // The methods of this interface correspond to the methods of Node and are described
  9. // more fully there.
  10. type MultiNode interface {
  11. // CreateGroup adds a new group to the MultiNode. The application must call CreateGroup
  12. // on each particpating node with the same group ID; it may create groups on demand as it
  13. // receives messages. If the given storage contains existing log entries the list of peers
  14. // may be empty. The Config.ID field will be ignored and replaced by the ID passed
  15. // to StartMultiNode.
  16. CreateGroup(group uint64, c *Config, peers []Peer) error
  17. // RemoveGroup removes a group from the MultiNode.
  18. RemoveGroup(group uint64) error
  19. // Tick advances the internal logical clock by a single tick.
  20. Tick()
  21. // Campaign causes this MultiNode to transition to candidate state in the given group.
  22. Campaign(ctx context.Context, group uint64) error
  23. // Propose proposes that data be appended to the given group's log.
  24. Propose(ctx context.Context, group uint64, data []byte) error
  25. // ProposeConfChange proposes a config change.
  26. ProposeConfChange(ctx context.Context, group uint64, cc pb.ConfChange) error
  27. // ApplyConfChange applies a config change to the local node.
  28. ApplyConfChange(group uint64, cc pb.ConfChange) *pb.ConfState
  29. // Step advances the state machine using the given message.
  30. Step(ctx context.Context, group uint64, msg pb.Message) error
  31. // Ready returns a channel that returns the current point-in-time state of any ready
  32. // groups. Only groups with something to report will appear in the map.
  33. Ready() <-chan map[uint64]Ready
  34. // Advance notifies the node that the application has applied and saved progress in the
  35. // last Ready results. It must be called with the last value returned from the Ready()
  36. // channel.
  37. Advance(map[uint64]Ready)
  38. // Status returns the current status of the given group. Returns nil if no such group
  39. // exists.
  40. Status(group uint64) *Status
  41. // Report reports the given node is not reachable for the last send.
  42. ReportUnreachable(id, groupID uint64)
  43. // ReportSnapshot reports the stutus of the sent snapshot.
  44. ReportSnapshot(id, groupID uint64, status SnapshotStatus)
  45. // Stop performs any necessary termination of the MultiNode.
  46. Stop()
  47. }
  48. // StartMultiNode creates a MultiNode and starts its background goroutine.
  49. // The id identifies this node and will be used as its node ID in all groups.
  50. // The election and heartbeat timers are in units of ticks.
  51. func StartMultiNode(id uint64) MultiNode {
  52. mn := newMultiNode(id)
  53. go mn.run()
  54. return &mn
  55. }
  56. // TODO(bdarnell): add group ID to the underlying protos?
  57. type multiMessage struct {
  58. group uint64
  59. msg pb.Message
  60. }
  61. type multiConfChange struct {
  62. group uint64
  63. msg pb.ConfChange
  64. ch chan pb.ConfState
  65. }
  66. type multiStatus struct {
  67. group uint64
  68. ch chan *Status
  69. }
  70. type groupCreation struct {
  71. id uint64
  72. config *Config
  73. peers []Peer
  74. // TODO(bdarnell): do we really need the done channel here? It's
  75. // unlike the rest of this package, but we need the group creation
  76. // to be complete before any Propose or other calls.
  77. done chan struct{}
  78. }
  79. type groupRemoval struct {
  80. id uint64
  81. // TODO(bdarnell): see comment on groupCreation.done
  82. done chan struct{}
  83. }
  84. type multiNode struct {
  85. id uint64
  86. groupc chan groupCreation
  87. rmgroupc chan groupRemoval
  88. propc chan multiMessage
  89. recvc chan multiMessage
  90. confc chan multiConfChange
  91. readyc chan map[uint64]Ready
  92. advancec chan map[uint64]Ready
  93. tickc chan struct{}
  94. stop chan struct{}
  95. done chan struct{}
  96. status chan multiStatus
  97. }
  98. func newMultiNode(id uint64) multiNode {
  99. return multiNode{
  100. id: id,
  101. groupc: make(chan groupCreation),
  102. rmgroupc: make(chan groupRemoval),
  103. propc: make(chan multiMessage),
  104. recvc: make(chan multiMessage),
  105. confc: make(chan multiConfChange),
  106. readyc: make(chan map[uint64]Ready),
  107. advancec: make(chan map[uint64]Ready),
  108. tickc: make(chan struct{}),
  109. stop: make(chan struct{}),
  110. done: make(chan struct{}),
  111. status: make(chan multiStatus),
  112. }
  113. }
  114. type groupState struct {
  115. id uint64
  116. raft *raft
  117. prevSoftSt *SoftState
  118. prevHardSt pb.HardState
  119. prevSnapi uint64
  120. }
  121. func (g *groupState) newReady() Ready {
  122. return newReady(g.raft, g.prevSoftSt, g.prevHardSt)
  123. }
  124. func (g *groupState) commitReady(rd Ready) {
  125. if rd.SoftState != nil {
  126. g.prevSoftSt = rd.SoftState
  127. }
  128. if !IsEmptyHardState(rd.HardState) {
  129. g.prevHardSt = rd.HardState
  130. }
  131. if g.prevHardSt.Commit != 0 {
  132. // In most cases, prevHardSt and rd.HardState will be the same
  133. // because when there are new entries to apply we just sent a
  134. // HardState with an updated Commit value. However, on initial
  135. // startup the two are different because we don't send a HardState
  136. // until something changes, but we do send any un-applied but
  137. // committed entries (and previously-committed entries may be
  138. // incorporated into the snapshot, even if rd.CommittedEntries is
  139. // empty). Therefore we mark all committed entries as applied
  140. // whether they were included in rd.HardState or not.
  141. g.raft.raftLog.appliedTo(g.prevHardSt.Commit)
  142. }
  143. if len(rd.Entries) > 0 {
  144. e := rd.Entries[len(rd.Entries)-1]
  145. g.raft.raftLog.stableTo(e.Index, e.Term)
  146. }
  147. if !IsEmptySnap(rd.Snapshot) {
  148. g.prevSnapi = rd.Snapshot.Metadata.Index
  149. g.raft.raftLog.stableSnapTo(g.prevSnapi)
  150. }
  151. }
  152. func (mn *multiNode) run() {
  153. groups := map[uint64]*groupState{}
  154. rds := map[uint64]Ready{}
  155. var advancec chan map[uint64]Ready
  156. for {
  157. // Only select readyc if we have something to report and we are not
  158. // currently waiting for an advance.
  159. readyc := mn.readyc
  160. if len(rds) == 0 || advancec != nil {
  161. readyc = nil
  162. }
  163. // group points to the group that was touched on this iteration (if any)
  164. var group *groupState
  165. select {
  166. case gc := <-mn.groupc:
  167. gc.config.ID = mn.id
  168. r := newRaft(gc.config)
  169. group = &groupState{
  170. id: gc.id,
  171. raft: r,
  172. }
  173. groups[gc.id] = group
  174. lastIndex, err := gc.config.Storage.LastIndex()
  175. if err != nil {
  176. panic(err) // TODO(bdarnell)
  177. }
  178. // If the log is empty, this is a new group (like StartNode); otherwise it's
  179. // restoring an existing group (like RestartNode).
  180. // TODO(bdarnell): rethink group initialization and whether the application needs
  181. // to be able to tell us when it expects the group to exist.
  182. if lastIndex == 0 {
  183. r.becomeFollower(1, None)
  184. ents := make([]pb.Entry, len(gc.peers))
  185. for i, peer := range gc.peers {
  186. cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
  187. data, err := cc.Marshal()
  188. if err != nil {
  189. panic("unexpected marshal error")
  190. }
  191. ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
  192. }
  193. r.raftLog.append(ents...)
  194. r.raftLog.committed = uint64(len(ents))
  195. for _, peer := range gc.peers {
  196. r.addNode(peer.ID)
  197. }
  198. }
  199. // Set the initial hard and soft states after performing all initialization.
  200. group.prevSoftSt = r.softState()
  201. group.prevHardSt = r.HardState
  202. close(gc.done)
  203. case gr := <-mn.rmgroupc:
  204. delete(groups, gr.id)
  205. delete(rds, gr.id)
  206. close(gr.done)
  207. case mm := <-mn.propc:
  208. // TODO(bdarnell): single-node impl doesn't read from propc unless the group
  209. // has a leader; we can't do that since we have one propc for many groups.
  210. // We'll have to buffer somewhere on a group-by-group basis, or just let
  211. // raft.Step drop any such proposals on the floor.
  212. mm.msg.From = mn.id
  213. group = groups[mm.group]
  214. group.raft.Step(mm.msg)
  215. case mm := <-mn.recvc:
  216. group = groups[mm.group]
  217. if _, ok := group.raft.prs[mm.msg.From]; ok || !IsResponseMsg(mm.msg) {
  218. group.raft.Step(mm.msg)
  219. }
  220. case mcc := <-mn.confc:
  221. group = groups[mcc.group]
  222. if mcc.msg.NodeID == None {
  223. group.raft.resetPendingConf()
  224. select {
  225. case mcc.ch <- pb.ConfState{Nodes: group.raft.nodes()}:
  226. case <-mn.done:
  227. }
  228. break
  229. }
  230. switch mcc.msg.Type {
  231. case pb.ConfChangeAddNode:
  232. group.raft.addNode(mcc.msg.NodeID)
  233. case pb.ConfChangeRemoveNode:
  234. group.raft.removeNode(mcc.msg.NodeID)
  235. case pb.ConfChangeUpdateNode:
  236. group.raft.resetPendingConf()
  237. default:
  238. panic("unexpected conf type")
  239. }
  240. select {
  241. case mcc.ch <- pb.ConfState{Nodes: group.raft.nodes()}:
  242. case <-mn.done:
  243. }
  244. case <-mn.tickc:
  245. // TODO(bdarnell): instead of calling every group on every tick,
  246. // we should have a priority queue of groups based on their next
  247. // time-based event.
  248. for _, g := range groups {
  249. g.raft.tick()
  250. rd := g.newReady()
  251. if rd.containsUpdates() {
  252. rds[g.id] = rd
  253. }
  254. }
  255. case readyc <- rds:
  256. // Clear outgoing messages as soon as we've passed them to the application.
  257. for g := range rds {
  258. groups[g].raft.msgs = nil
  259. }
  260. rds = map[uint64]Ready{}
  261. advancec = mn.advancec
  262. case advs := <-advancec:
  263. for groupID, rd := range advs {
  264. group, ok := groups[groupID]
  265. if !ok {
  266. continue
  267. }
  268. group.commitReady(rd)
  269. // We've been accumulating new entries in rds which may now be obsolete.
  270. // Drop the old Ready object and create a new one if needed.
  271. delete(rds, groupID)
  272. newRd := group.newReady()
  273. if newRd.containsUpdates() {
  274. rds[groupID] = newRd
  275. }
  276. }
  277. advancec = nil
  278. case ms := <-mn.status:
  279. if group, ok := groups[ms.group]; ok {
  280. s := getStatus(group.raft)
  281. ms.ch <- &s
  282. } else {
  283. ms.ch <- nil
  284. }
  285. case <-mn.stop:
  286. close(mn.done)
  287. return
  288. }
  289. if group != nil {
  290. rd := group.newReady()
  291. if rd.containsUpdates() {
  292. rds[group.id] = rd
  293. }
  294. }
  295. }
  296. }
  297. func (mn *multiNode) CreateGroup(id uint64, config *Config, peers []Peer) error {
  298. gc := groupCreation{
  299. id: id,
  300. config: config,
  301. peers: peers,
  302. done: make(chan struct{}),
  303. }
  304. mn.groupc <- gc
  305. select {
  306. case <-gc.done:
  307. return nil
  308. case <-mn.done:
  309. return ErrStopped
  310. }
  311. }
  312. func (mn *multiNode) RemoveGroup(id uint64) error {
  313. gr := groupRemoval{
  314. id: id,
  315. done: make(chan struct{}),
  316. }
  317. mn.rmgroupc <- gr
  318. select {
  319. case <-gr.done:
  320. return nil
  321. case <-mn.done:
  322. return ErrStopped
  323. }
  324. }
  325. func (mn *multiNode) Stop() {
  326. select {
  327. case mn.stop <- struct{}{}:
  328. case <-mn.done:
  329. }
  330. <-mn.done
  331. }
  332. func (mn *multiNode) Tick() {
  333. select {
  334. case mn.tickc <- struct{}{}:
  335. case <-mn.done:
  336. }
  337. }
  338. func (mn *multiNode) Campaign(ctx context.Context, group uint64) error {
  339. return mn.step(ctx, multiMessage{group,
  340. pb.Message{
  341. Type: pb.MsgHup,
  342. },
  343. })
  344. }
  345. func (mn *multiNode) Propose(ctx context.Context, group uint64, data []byte) error {
  346. return mn.step(ctx, multiMessage{group,
  347. pb.Message{
  348. Type: pb.MsgProp,
  349. Entries: []pb.Entry{
  350. {Data: data},
  351. },
  352. }})
  353. }
  354. func (mn *multiNode) ProposeConfChange(ctx context.Context, group uint64, cc pb.ConfChange) error {
  355. data, err := cc.Marshal()
  356. if err != nil {
  357. return err
  358. }
  359. return mn.Step(ctx, group,
  360. pb.Message{
  361. Type: pb.MsgProp,
  362. Entries: []pb.Entry{
  363. {Type: pb.EntryConfChange, Data: data},
  364. },
  365. })
  366. }
  367. func (mn *multiNode) step(ctx context.Context, m multiMessage) error {
  368. ch := mn.recvc
  369. if m.msg.Type == pb.MsgProp {
  370. ch = mn.propc
  371. }
  372. select {
  373. case ch <- m:
  374. return nil
  375. case <-ctx.Done():
  376. return ctx.Err()
  377. case <-mn.done:
  378. return ErrStopped
  379. }
  380. }
  381. func (mn *multiNode) ApplyConfChange(group uint64, cc pb.ConfChange) *pb.ConfState {
  382. mcc := multiConfChange{group, cc, make(chan pb.ConfState)}
  383. select {
  384. case mn.confc <- mcc:
  385. case <-mn.done:
  386. }
  387. select {
  388. case cs := <-mcc.ch:
  389. return &cs
  390. case <-mn.done:
  391. // Per comments on Node.ApplyConfChange, this method should never return nil.
  392. return &pb.ConfState{}
  393. }
  394. }
  395. func (mn *multiNode) Step(ctx context.Context, group uint64, m pb.Message) error {
  396. // ignore unexpected local messages receiving over network
  397. if IsLocalMsg(m) {
  398. // TODO: return an error?
  399. return nil
  400. }
  401. return mn.step(ctx, multiMessage{group, m})
  402. }
  403. func (mn *multiNode) Ready() <-chan map[uint64]Ready {
  404. return mn.readyc
  405. }
  406. func (mn *multiNode) Advance(rds map[uint64]Ready) {
  407. select {
  408. case mn.advancec <- rds:
  409. case <-mn.done:
  410. }
  411. }
  412. func (mn *multiNode) Status(group uint64) *Status {
  413. ms := multiStatus{
  414. group: group,
  415. ch: make(chan *Status),
  416. }
  417. mn.status <- ms
  418. return <-ms.ch
  419. }
  420. func (mn *multiNode) ReportUnreachable(id, groupID uint64) {
  421. select {
  422. case mn.recvc <- multiMessage{
  423. group: groupID,
  424. msg: pb.Message{Type: pb.MsgUnreachable, From: id},
  425. }:
  426. case <-mn.done:
  427. }
  428. }
  429. func (mn *multiNode) ReportSnapshot(id, groupID uint64, status SnapshotStatus) {
  430. rej := status == SnapshotFailure
  431. select {
  432. case mn.recvc <- multiMessage{
  433. group: groupID,
  434. msg: pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej},
  435. }:
  436. case <-mn.done:
  437. }
  438. }