multinode.go 13 KB

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