rawnode.go 9.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package raft
  15. import (
  16. "errors"
  17. pb "go.etcd.io/etcd/raft/raftpb"
  18. "go.etcd.io/etcd/raft/tracker"
  19. )
  20. // ErrStepLocalMsg is returned when try to step a local raft message
  21. var ErrStepLocalMsg = errors.New("raft: cannot step raft local message")
  22. // ErrStepPeerNotFound is returned when try to step a response message
  23. // but there is no peer found in raft.prs for that node.
  24. var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found")
  25. // RawNode is a thread-unsafe Node.
  26. // The methods of this struct correspond to the methods of Node and are described
  27. // more fully there.
  28. type RawNode struct {
  29. raft *raft
  30. prevSoftSt *SoftState
  31. prevHardSt pb.HardState
  32. }
  33. func (rn *RawNode) newReady() Ready {
  34. return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
  35. }
  36. func (rn *RawNode) commitReady(rd Ready) {
  37. if rd.SoftState != nil {
  38. rn.prevSoftSt = rd.SoftState
  39. }
  40. if !IsEmptyHardState(rd.HardState) {
  41. rn.prevHardSt = rd.HardState
  42. }
  43. // If entries were applied (or a snapshot), update our cursor for
  44. // the next Ready. Note that if the current HardState contains a
  45. // new Commit index, this does not mean that we're also applying
  46. // all of the new entries due to commit pagination by size.
  47. if index := rd.appliedCursor(); index > 0 {
  48. rn.raft.raftLog.appliedTo(index)
  49. }
  50. if len(rd.Entries) > 0 {
  51. e := rd.Entries[len(rd.Entries)-1]
  52. rn.raft.raftLog.stableTo(e.Index, e.Term)
  53. }
  54. if !IsEmptySnap(rd.Snapshot) {
  55. rn.raft.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
  56. }
  57. if len(rd.ReadStates) != 0 {
  58. rn.raft.readStates = nil
  59. }
  60. }
  61. // NewRawNode returns a new RawNode given configuration and a list of raft peers.
  62. func NewRawNode(config *Config, peers []Peer) (*RawNode, error) {
  63. if config.ID == 0 {
  64. panic("config.ID must not be zero")
  65. }
  66. r := newRaft(config)
  67. rn := &RawNode{
  68. raft: r,
  69. }
  70. lastIndex, err := config.Storage.LastIndex()
  71. if err != nil {
  72. panic(err) // TODO(bdarnell)
  73. }
  74. // If the log is empty, this is a new RawNode (like StartNode); otherwise it's
  75. // restoring an existing RawNode (like RestartNode).
  76. // TODO(bdarnell): rethink RawNode initialization and whether the application needs
  77. // to be able to tell us when it expects the RawNode to exist.
  78. if lastIndex == 0 {
  79. r.becomeFollower(1, None)
  80. ents := make([]pb.Entry, len(peers))
  81. for i, peer := range peers {
  82. cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
  83. data, err := cc.Marshal()
  84. if err != nil {
  85. panic("unexpected marshal error")
  86. }
  87. ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
  88. }
  89. r.raftLog.append(ents...)
  90. r.raftLog.committed = uint64(len(ents))
  91. for _, peer := range peers {
  92. r.addNode(peer.ID)
  93. }
  94. }
  95. // Set the initial hard and soft states after performing all initialization.
  96. rn.prevSoftSt = r.softState()
  97. if lastIndex == 0 {
  98. rn.prevHardSt = emptyState
  99. } else {
  100. rn.prevHardSt = r.hardState()
  101. }
  102. return rn, nil
  103. }
  104. // Tick advances the internal logical clock by a single tick.
  105. func (rn *RawNode) Tick() {
  106. rn.raft.tick()
  107. }
  108. // TickQuiesced advances the internal logical clock by a single tick without
  109. // performing any other state machine processing. It allows the caller to avoid
  110. // periodic heartbeats and elections when all of the peers in a Raft group are
  111. // known to be at the same state. Expected usage is to periodically invoke Tick
  112. // or TickQuiesced depending on whether the group is "active" or "quiesced".
  113. //
  114. // WARNING: Be very careful about using this method as it subverts the Raft
  115. // state machine. You should probably be using Tick instead.
  116. func (rn *RawNode) TickQuiesced() {
  117. rn.raft.electionElapsed++
  118. }
  119. // Campaign causes this RawNode to transition to candidate state.
  120. func (rn *RawNode) Campaign() error {
  121. return rn.raft.Step(pb.Message{
  122. Type: pb.MsgHup,
  123. })
  124. }
  125. // Propose proposes data be appended to the raft log.
  126. func (rn *RawNode) Propose(data []byte) error {
  127. return rn.raft.Step(pb.Message{
  128. Type: pb.MsgProp,
  129. From: rn.raft.id,
  130. Entries: []pb.Entry{
  131. {Data: data},
  132. }})
  133. }
  134. // ProposeConfChange proposes a config change.
  135. func (rn *RawNode) ProposeConfChange(cc pb.ConfChange) error {
  136. data, err := cc.Marshal()
  137. if err != nil {
  138. return err
  139. }
  140. return rn.raft.Step(pb.Message{
  141. Type: pb.MsgProp,
  142. Entries: []pb.Entry{
  143. {Type: pb.EntryConfChange, Data: data},
  144. },
  145. })
  146. }
  147. // ApplyConfChange applies a config change to the local node.
  148. func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
  149. if cc.NodeID == None {
  150. return &pb.ConfState{Nodes: rn.raft.prs.VoterNodes(), Learners: rn.raft.prs.LearnerNodes()}
  151. }
  152. switch cc.Type {
  153. case pb.ConfChangeAddNode:
  154. rn.raft.addNode(cc.NodeID)
  155. case pb.ConfChangeAddLearnerNode:
  156. rn.raft.addLearner(cc.NodeID)
  157. case pb.ConfChangeRemoveNode:
  158. rn.raft.removeNode(cc.NodeID)
  159. case pb.ConfChangeUpdateNode:
  160. default:
  161. panic("unexpected conf type")
  162. }
  163. return &pb.ConfState{Nodes: rn.raft.prs.VoterNodes(), Learners: rn.raft.prs.LearnerNodes()}
  164. }
  165. // Step advances the state machine using the given message.
  166. func (rn *RawNode) Step(m pb.Message) error {
  167. // ignore unexpected local messages receiving over network
  168. if IsLocalMsg(m.Type) {
  169. return ErrStepLocalMsg
  170. }
  171. if pr := rn.raft.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) {
  172. return rn.raft.Step(m)
  173. }
  174. return ErrStepPeerNotFound
  175. }
  176. // Ready returns the current point-in-time state of this RawNode.
  177. func (rn *RawNode) Ready() Ready {
  178. rd := rn.newReady()
  179. rn.raft.msgs = nil
  180. rn.raft.reduceUncommittedSize(rd.CommittedEntries)
  181. return rd
  182. }
  183. // HasReady called when RawNode user need to check if any Ready pending.
  184. // Checking logic in this method should be consistent with Ready.containsUpdates().
  185. func (rn *RawNode) HasReady() bool {
  186. r := rn.raft
  187. if !r.softState().equal(rn.prevSoftSt) {
  188. return true
  189. }
  190. if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) {
  191. return true
  192. }
  193. if r.raftLog.unstable.snapshot != nil && !IsEmptySnap(*r.raftLog.unstable.snapshot) {
  194. return true
  195. }
  196. if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
  197. return true
  198. }
  199. if len(r.readStates) != 0 {
  200. return true
  201. }
  202. return false
  203. }
  204. // Advance notifies the RawNode that the application has applied and saved progress in the
  205. // last Ready results.
  206. func (rn *RawNode) Advance(rd Ready) {
  207. rn.commitReady(rd)
  208. }
  209. // Status returns the current status of the given group.
  210. func (rn *RawNode) Status() *Status {
  211. status := getStatus(rn.raft)
  212. return &status
  213. }
  214. // StatusWithoutProgress returns a Status without populating the Progress field
  215. // (and returns the Status as a value to avoid forcing it onto the heap). This
  216. // is more performant if the Progress is not required. See WithProgress for an
  217. // allocation-free way to introspect the Progress.
  218. func (rn *RawNode) StatusWithoutProgress() Status {
  219. return getStatusWithoutProgress(rn.raft)
  220. }
  221. // ProgressType indicates the type of replica a Progress corresponds to.
  222. type ProgressType byte
  223. const (
  224. // ProgressTypePeer accompanies a Progress for a regular peer replica.
  225. ProgressTypePeer ProgressType = iota
  226. // ProgressTypeLearner accompanies a Progress for a learner replica.
  227. ProgressTypeLearner
  228. )
  229. // WithProgress is a helper to introspect the Progress for this node and its
  230. // peers.
  231. func (rn *RawNode) WithProgress(visitor func(id uint64, typ ProgressType, pr tracker.Progress)) {
  232. rn.raft.prs.Visit(func(id uint64, pr *tracker.Progress) {
  233. typ := ProgressTypePeer
  234. if pr.IsLearner {
  235. typ = ProgressTypeLearner
  236. }
  237. p := *pr
  238. p.Inflights = nil
  239. visitor(id, typ, p)
  240. })
  241. }
  242. // ReportUnreachable reports the given node is not reachable for the last send.
  243. func (rn *RawNode) ReportUnreachable(id uint64) {
  244. _ = rn.raft.Step(pb.Message{Type: pb.MsgUnreachable, From: id})
  245. }
  246. // ReportSnapshot reports the status of the sent snapshot.
  247. func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
  248. rej := status == SnapshotFailure
  249. _ = rn.raft.Step(pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej})
  250. }
  251. // TransferLeader tries to transfer leadership to the given transferee.
  252. func (rn *RawNode) TransferLeader(transferee uint64) {
  253. _ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
  254. }
  255. // ReadIndex requests a read state. The read state will be set in ready.
  256. // Read State has a read index. Once the application advances further than the read
  257. // index, any linearizable read requests issued before the read request can be
  258. // processed safely. The read state will have the same rctx attached.
  259. func (rn *RawNode) ReadIndex(rctx []byte) {
  260. _ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
  261. }