node.go 3.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165
  1. // Package raft implements raft.
  2. package raft
  3. import (
  4. "errors"
  5. "log"
  6. pb "github.com/coreos/etcd/raft/raftpb"
  7. "github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
  8. )
  9. var ErrStopped = errors.New("raft: stopped")
  10. type Ready struct {
  11. // The current state of a Node
  12. pb.State
  13. // Entries specifies entries to be saved to stable storage BEFORE
  14. // Messages are sent.
  15. Entries []pb.Entry
  16. // CommittedEntries specifies entries to be committed to a
  17. // store/state-machine. These have previously been committed to stable
  18. // store.
  19. CommittedEntries []pb.Entry
  20. // Messages specifies outbound messages to be sent AFTER Entries are
  21. // committed to stable storage.
  22. Messages []pb.Message
  23. }
  24. func isStateEqual(a, b pb.State) bool {
  25. return a.Term == b.Term && a.Vote == b.Vote && a.LastIndex == b.LastIndex
  26. }
  27. func (rd Ready) containsUpdates(prev Ready) bool {
  28. return !isStateEqual(prev.State, rd.State) || len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
  29. }
  30. type Node struct {
  31. ctx context.Context
  32. propc chan pb.Message
  33. recvc chan pb.Message
  34. readyc chan Ready
  35. tickc chan struct{}
  36. alwaysreadyc chan Ready
  37. done chan struct{}
  38. }
  39. func Start(id int64, peers []int64, election, heartbeat int) Node {
  40. n := Node{
  41. propc: make(chan pb.Message),
  42. recvc: make(chan pb.Message),
  43. readyc: make(chan Ready),
  44. tickc: make(chan struct{}),
  45. alwaysreadyc: make(chan Ready),
  46. done: make(chan struct{}),
  47. }
  48. r := newRaft(id, peers, election, heartbeat)
  49. go n.run(r)
  50. return n
  51. }
  52. func (n *Node) Stop() {
  53. close(n.done)
  54. }
  55. func (n *Node) run(r *raft) {
  56. propc := n.propc
  57. readyc := n.readyc
  58. var lead int64
  59. var prev Ready
  60. prev.Vote = none
  61. for {
  62. if lead != r.lead {
  63. log.Printf("raft: leader changed from %#x to %#x", lead, r.lead)
  64. lead = r.lead
  65. if r.hasLeader() {
  66. propc = n.propc
  67. } else {
  68. propc = nil
  69. }
  70. }
  71. rd := Ready{
  72. r.State,
  73. r.raftLog.unstableEnts(),
  74. r.raftLog.nextEnts(),
  75. r.msgs,
  76. }
  77. if rd.containsUpdates(prev) {
  78. readyc = n.readyc
  79. prev = rd
  80. } else {
  81. readyc = nil
  82. }
  83. select {
  84. case m := <-propc:
  85. m.From = r.id
  86. r.Step(m)
  87. case m := <-n.recvc:
  88. r.Step(m) // raft never returns an error
  89. case <-n.tickc:
  90. r.tick()
  91. case readyc <- rd:
  92. r.raftLog.resetNextEnts()
  93. r.raftLog.resetUnstable()
  94. r.msgs = nil
  95. case n.alwaysreadyc <- rd:
  96. // this is for testing only
  97. case <-n.done:
  98. return
  99. }
  100. }
  101. }
  102. func (n *Node) Tick() error {
  103. select {
  104. case n.tickc <- struct{}{}:
  105. return nil
  106. case <-n.done:
  107. return n.ctx.Err()
  108. }
  109. }
  110. func (n *Node) Campaign(ctx context.Context) error {
  111. return n.Step(ctx, pb.Message{Type: msgHup})
  112. }
  113. // Propose proposes data be appended to the log.
  114. func (n *Node) Propose(ctx context.Context, data []byte) error {
  115. return n.Step(ctx, pb.Message{Type: msgProp, Entries: []pb.Entry{{Data: data}}})
  116. }
  117. // Step advances the state machine using msgs. The ctx.Err() will be returned,
  118. // if any.
  119. func (n *Node) Step(ctx context.Context, m pb.Message) error {
  120. ch := n.recvc
  121. if m.Type == msgProp {
  122. ch = n.propc
  123. }
  124. select {
  125. case ch <- m:
  126. return nil
  127. case <-ctx.Done():
  128. return ctx.Err()
  129. case <-n.done:
  130. return ErrStopped
  131. }
  132. }
  133. // ReadState returns the current point-in-time state.
  134. func (n *Node) Ready() <-chan Ready {
  135. return n.readyc
  136. }
  137. // RecvReadyNow returns the state of n without blocking. It is primarly for
  138. // testing purposes only.
  139. func RecvReadyNow(n Node) Ready {
  140. return <-n.alwaysreadyc
  141. }