raft.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482
  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 main
  15. import (
  16. "context"
  17. "fmt"
  18. "log"
  19. "net/http"
  20. "net/url"
  21. "os"
  22. "strconv"
  23. "time"
  24. "go.etcd.io/etcd/etcdserver/api/rafthttp"
  25. "go.etcd.io/etcd/etcdserver/api/snap"
  26. stats "go.etcd.io/etcd/etcdserver/api/v2stats"
  27. "go.etcd.io/etcd/pkg/fileutil"
  28. "go.etcd.io/etcd/pkg/types"
  29. "go.etcd.io/etcd/raft"
  30. "go.etcd.io/etcd/raft/raftpb"
  31. "go.etcd.io/etcd/wal"
  32. "go.etcd.io/etcd/wal/walpb"
  33. "go.uber.org/zap"
  34. )
  35. // A key-value stream backed by raft
  36. type raftNode struct {
  37. proposeC <-chan string // proposed messages (k,v)
  38. confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
  39. commitC chan<- *string // entries committed to log (k,v)
  40. errorC chan<- error // errors from raft session
  41. id int // client ID for raft session
  42. peers []string // raft peer URLs
  43. join bool // node is joining an existing cluster
  44. waldir string // path to WAL directory
  45. snapdir string // path to snapshot directory
  46. getSnapshot func() ([]byte, error)
  47. lastIndex uint64 // index of log at start
  48. confState raftpb.ConfState
  49. snapshotIndex uint64
  50. appliedIndex uint64
  51. // raft backing for the commit/error channel
  52. node raft.Node
  53. raftStorage *raft.MemoryStorage
  54. wal *wal.WAL
  55. snapshotter *snap.Snapshotter
  56. snapshotterReady chan *snap.Snapshotter // signals when snapshotter is ready
  57. snapCount uint64
  58. transport *rafthttp.Transport
  59. stopc chan struct{} // signals proposal channel closed
  60. httpstopc chan struct{} // signals http server to shutdown
  61. httpdonec chan struct{} // signals http server shutdown complete
  62. }
  63. var defaultSnapshotCount uint64 = 10000
  64. // newRaftNode initiates a raft instance and returns a committed log entry
  65. // channel and error channel. Proposals for log updates are sent over the
  66. // provided the proposal channel. All log entries are replayed over the
  67. // commit channel, followed by a nil message (to indicate the channel is
  68. // current), then new log entries. To shutdown, close proposeC and read errorC.
  69. func newRaftNode(id int, peers []string, join bool, getSnapshot func() ([]byte, error), proposeC <-chan string,
  70. confChangeC <-chan raftpb.ConfChange) (<-chan *string, <-chan error, <-chan *snap.Snapshotter) {
  71. commitC := make(chan *string)
  72. errorC := make(chan error)
  73. rc := &raftNode{
  74. proposeC: proposeC,
  75. confChangeC: confChangeC,
  76. commitC: commitC,
  77. errorC: errorC,
  78. id: id,
  79. peers: peers,
  80. join: join,
  81. waldir: fmt.Sprintf("raftexample-%d", id),
  82. snapdir: fmt.Sprintf("raftexample-%d-snap", id),
  83. getSnapshot: getSnapshot,
  84. snapCount: defaultSnapshotCount,
  85. stopc: make(chan struct{}),
  86. httpstopc: make(chan struct{}),
  87. httpdonec: make(chan struct{}),
  88. snapshotterReady: make(chan *snap.Snapshotter, 1),
  89. // rest of structure populated after WAL replay
  90. }
  91. go rc.startRaft()
  92. return commitC, errorC, rc.snapshotterReady
  93. }
  94. func (rc *raftNode) saveSnap(snap raftpb.Snapshot) error {
  95. // must save the snapshot index to the WAL before saving the
  96. // snapshot to maintain the invariant that we only Open the
  97. // wal at previously-saved snapshot indexes.
  98. walSnap := walpb.Snapshot{
  99. Index: snap.Metadata.Index,
  100. Term: snap.Metadata.Term,
  101. }
  102. if err := rc.wal.SaveSnapshot(walSnap); err != nil {
  103. return err
  104. }
  105. if err := rc.snapshotter.SaveSnap(snap); err != nil {
  106. return err
  107. }
  108. return rc.wal.ReleaseLockTo(snap.Metadata.Index)
  109. }
  110. func (rc *raftNode) entriesToApply(ents []raftpb.Entry) (nents []raftpb.Entry) {
  111. if len(ents) == 0 {
  112. return ents
  113. }
  114. firstIdx := ents[0].Index
  115. if firstIdx > rc.appliedIndex+1 {
  116. log.Fatalf("first index of committed entry[%d] should <= progress.appliedIndex[%d]+1", firstIdx, rc.appliedIndex)
  117. }
  118. if rc.appliedIndex-firstIdx+1 < uint64(len(ents)) {
  119. nents = ents[rc.appliedIndex-firstIdx+1:]
  120. }
  121. return nents
  122. }
  123. // publishEntries writes committed log entries to commit channel and returns
  124. // whether all entries could be published.
  125. func (rc *raftNode) publishEntries(ents []raftpb.Entry) bool {
  126. for i := range ents {
  127. switch ents[i].Type {
  128. case raftpb.EntryNormal:
  129. if len(ents[i].Data) == 0 {
  130. // ignore empty messages
  131. break
  132. }
  133. s := string(ents[i].Data)
  134. select {
  135. case rc.commitC <- &s:
  136. case <-rc.stopc:
  137. return false
  138. }
  139. case raftpb.EntryConfChange:
  140. var cc raftpb.ConfChange
  141. cc.Unmarshal(ents[i].Data)
  142. rc.confState = *rc.node.ApplyConfChange(cc)
  143. switch cc.Type {
  144. case raftpb.ConfChangeAddNode:
  145. if len(cc.Context) > 0 {
  146. rc.transport.AddPeer(types.ID(cc.NodeID), []string{string(cc.Context)})
  147. }
  148. case raftpb.ConfChangeRemoveNode:
  149. if cc.NodeID == uint64(rc.id) {
  150. log.Println("I've been removed from the cluster! Shutting down.")
  151. return false
  152. }
  153. rc.transport.RemovePeer(types.ID(cc.NodeID))
  154. }
  155. }
  156. // after commit, update appliedIndex
  157. rc.appliedIndex = ents[i].Index
  158. // special nil commit to signal replay has finished
  159. if ents[i].Index == rc.lastIndex {
  160. select {
  161. case rc.commitC <- nil:
  162. case <-rc.stopc:
  163. return false
  164. }
  165. }
  166. }
  167. return true
  168. }
  169. func (rc *raftNode) loadSnapshot() *raftpb.Snapshot {
  170. snapshot, err := rc.snapshotter.Load()
  171. if err != nil && err != snap.ErrNoSnapshot {
  172. log.Fatalf("raftexample: error loading snapshot (%v)", err)
  173. }
  174. return snapshot
  175. }
  176. // openWAL returns a WAL ready for reading.
  177. func (rc *raftNode) openWAL(snapshot *raftpb.Snapshot) *wal.WAL {
  178. if !wal.Exist(rc.waldir) {
  179. if err := os.Mkdir(rc.waldir, 0750); err != nil {
  180. log.Fatalf("raftexample: cannot create dir for wal (%v)", err)
  181. }
  182. w, err := wal.Create(zap.NewExample(), rc.waldir, nil)
  183. if err != nil {
  184. log.Fatalf("raftexample: create wal error (%v)", err)
  185. }
  186. w.Close()
  187. }
  188. walsnap := walpb.Snapshot{}
  189. if snapshot != nil {
  190. walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
  191. }
  192. log.Printf("loading WAL at term %d and index %d", walsnap.Term, walsnap.Index)
  193. w, err := wal.Open(zap.NewExample(), rc.waldir, walsnap)
  194. if err != nil {
  195. log.Fatalf("raftexample: error loading wal (%v)", err)
  196. }
  197. return w
  198. }
  199. // replayWAL replays WAL entries into the raft instance.
  200. func (rc *raftNode) replayWAL() *wal.WAL {
  201. log.Printf("replaying WAL of member %d", rc.id)
  202. snapshot := rc.loadSnapshot()
  203. w := rc.openWAL(snapshot)
  204. _, st, ents, err := w.ReadAll()
  205. if err != nil {
  206. log.Fatalf("raftexample: failed to read WAL (%v)", err)
  207. }
  208. rc.raftStorage = raft.NewMemoryStorage()
  209. if snapshot != nil {
  210. rc.raftStorage.ApplySnapshot(*snapshot)
  211. }
  212. rc.raftStorage.SetHardState(st)
  213. // append to storage so raft starts at the right place in log
  214. rc.raftStorage.Append(ents)
  215. // send nil once lastIndex is published so client knows commit channel is current
  216. if len(ents) > 0 {
  217. rc.lastIndex = ents[len(ents)-1].Index
  218. } else {
  219. rc.commitC <- nil
  220. }
  221. return w
  222. }
  223. func (rc *raftNode) writeError(err error) {
  224. rc.stopHTTP()
  225. close(rc.commitC)
  226. rc.errorC <- err
  227. close(rc.errorC)
  228. rc.node.Stop()
  229. }
  230. func (rc *raftNode) startRaft() {
  231. if !fileutil.Exist(rc.snapdir) {
  232. if err := os.Mkdir(rc.snapdir, 0750); err != nil {
  233. log.Fatalf("raftexample: cannot create dir for snapshot (%v)", err)
  234. }
  235. }
  236. rc.snapshotter = snap.New(zap.NewExample(), rc.snapdir)
  237. rc.snapshotterReady <- rc.snapshotter
  238. oldwal := wal.Exist(rc.waldir)
  239. rc.wal = rc.replayWAL()
  240. rpeers := make([]raft.Peer, len(rc.peers))
  241. for i := range rpeers {
  242. rpeers[i] = raft.Peer{ID: uint64(i + 1)}
  243. }
  244. c := &raft.Config{
  245. ID: uint64(rc.id),
  246. ElectionTick: 10,
  247. HeartbeatTick: 1,
  248. Storage: rc.raftStorage,
  249. MaxSizePerMsg: 1024 * 1024,
  250. MaxInflightMsgs: 256,
  251. MaxUncommittedEntriesSize: 1 << 30,
  252. }
  253. if oldwal {
  254. rc.node = raft.RestartNode(c)
  255. } else {
  256. startPeers := rpeers
  257. if rc.join {
  258. startPeers = nil
  259. }
  260. rc.node = raft.StartNode(c, startPeers)
  261. }
  262. rc.transport = &rafthttp.Transport{
  263. Logger: zap.NewExample(),
  264. ID: types.ID(rc.id),
  265. ClusterID: 0x1000,
  266. Raft: rc,
  267. ServerStats: stats.NewServerStats("", ""),
  268. LeaderStats: stats.NewLeaderStats(strconv.Itoa(rc.id)),
  269. ErrorC: make(chan error),
  270. }
  271. rc.transport.Start()
  272. for i := range rc.peers {
  273. if i+1 != rc.id {
  274. rc.transport.AddPeer(types.ID(i+1), []string{rc.peers[i]})
  275. }
  276. }
  277. go rc.serveRaft()
  278. go rc.serveChannels()
  279. }
  280. // stop closes http, closes all channels, and stops raft.
  281. func (rc *raftNode) stop() {
  282. rc.stopHTTP()
  283. close(rc.commitC)
  284. close(rc.errorC)
  285. rc.node.Stop()
  286. }
  287. func (rc *raftNode) stopHTTP() {
  288. rc.transport.Stop()
  289. close(rc.httpstopc)
  290. <-rc.httpdonec
  291. }
  292. func (rc *raftNode) publishSnapshot(snapshotToSave raftpb.Snapshot) {
  293. if raft.IsEmptySnap(snapshotToSave) {
  294. return
  295. }
  296. log.Printf("publishing snapshot at index %d", rc.snapshotIndex)
  297. defer log.Printf("finished publishing snapshot at index %d", rc.snapshotIndex)
  298. if snapshotToSave.Metadata.Index <= rc.appliedIndex {
  299. log.Fatalf("snapshot index [%d] should > progress.appliedIndex [%d]", snapshotToSave.Metadata.Index, rc.appliedIndex)
  300. }
  301. rc.commitC <- nil // trigger kvstore to load snapshot
  302. rc.confState = snapshotToSave.Metadata.ConfState
  303. rc.snapshotIndex = snapshotToSave.Metadata.Index
  304. rc.appliedIndex = snapshotToSave.Metadata.Index
  305. }
  306. var snapshotCatchUpEntriesN uint64 = 10000
  307. func (rc *raftNode) maybeTriggerSnapshot() {
  308. if rc.appliedIndex-rc.snapshotIndex <= rc.snapCount {
  309. return
  310. }
  311. log.Printf("start snapshot [applied index: %d | last snapshot index: %d]", rc.appliedIndex, rc.snapshotIndex)
  312. data, err := rc.getSnapshot()
  313. if err != nil {
  314. log.Panic(err)
  315. }
  316. snap, err := rc.raftStorage.CreateSnapshot(rc.appliedIndex, &rc.confState, data)
  317. if err != nil {
  318. panic(err)
  319. }
  320. if err := rc.saveSnap(snap); err != nil {
  321. panic(err)
  322. }
  323. compactIndex := uint64(1)
  324. if rc.appliedIndex > snapshotCatchUpEntriesN {
  325. compactIndex = rc.appliedIndex - snapshotCatchUpEntriesN
  326. }
  327. if err := rc.raftStorage.Compact(compactIndex); err != nil {
  328. panic(err)
  329. }
  330. log.Printf("compacted log at index %d", compactIndex)
  331. rc.snapshotIndex = rc.appliedIndex
  332. }
  333. func (rc *raftNode) serveChannels() {
  334. snap, err := rc.raftStorage.Snapshot()
  335. if err != nil {
  336. panic(err)
  337. }
  338. rc.confState = snap.Metadata.ConfState
  339. rc.snapshotIndex = snap.Metadata.Index
  340. rc.appliedIndex = snap.Metadata.Index
  341. defer rc.wal.Close()
  342. ticker := time.NewTicker(100 * time.Millisecond)
  343. defer ticker.Stop()
  344. // send proposals over raft
  345. go func() {
  346. confChangeCount := uint64(0)
  347. for rc.proposeC != nil && rc.confChangeC != nil {
  348. select {
  349. case prop, ok := <-rc.proposeC:
  350. if !ok {
  351. rc.proposeC = nil
  352. } else {
  353. // blocks until accepted by raft state machine
  354. rc.node.Propose(context.TODO(), []byte(prop))
  355. }
  356. case cc, ok := <-rc.confChangeC:
  357. if !ok {
  358. rc.confChangeC = nil
  359. } else {
  360. confChangeCount++
  361. cc.ID = confChangeCount
  362. rc.node.ProposeConfChange(context.TODO(), cc)
  363. }
  364. }
  365. }
  366. // client closed channel; shutdown raft if not already
  367. close(rc.stopc)
  368. }()
  369. // event loop on raft state machine updates
  370. for {
  371. select {
  372. case <-ticker.C:
  373. rc.node.Tick()
  374. // store raft entries to wal, then publish over commit channel
  375. case rd := <-rc.node.Ready():
  376. rc.wal.Save(rd.HardState, rd.Entries)
  377. if !raft.IsEmptySnap(rd.Snapshot) {
  378. rc.saveSnap(rd.Snapshot)
  379. rc.raftStorage.ApplySnapshot(rd.Snapshot)
  380. rc.publishSnapshot(rd.Snapshot)
  381. }
  382. rc.raftStorage.Append(rd.Entries)
  383. rc.transport.Send(rd.Messages)
  384. if ok := rc.publishEntries(rc.entriesToApply(rd.CommittedEntries)); !ok {
  385. rc.stop()
  386. return
  387. }
  388. rc.maybeTriggerSnapshot()
  389. rc.node.Advance()
  390. case err := <-rc.transport.ErrorC:
  391. rc.writeError(err)
  392. return
  393. case <-rc.stopc:
  394. rc.stop()
  395. return
  396. }
  397. }
  398. }
  399. func (rc *raftNode) serveRaft() {
  400. url, err := url.Parse(rc.peers[rc.id-1])
  401. if err != nil {
  402. log.Fatalf("raftexample: Failed parsing URL (%v)", err)
  403. }
  404. ln, err := newStoppableListener(url.Host, rc.httpstopc)
  405. if err != nil {
  406. log.Fatalf("raftexample: Failed to listen rafthttp (%v)", err)
  407. }
  408. err = (&http.Server{Handler: rc.transport.Handler()}).Serve(ln)
  409. select {
  410. case <-rc.httpstopc:
  411. default:
  412. log.Fatalf("raftexample: Failed to serve rafthttp (%v)", err)
  413. }
  414. close(rc.httpdonec)
  415. }
  416. func (rc *raftNode) Process(ctx context.Context, m raftpb.Message) error {
  417. return rc.node.Step(ctx, m)
  418. }
  419. func (rc *raftNode) IsIDRemoved(id uint64) bool { return false }
  420. func (rc *raftNode) ReportUnreachable(id uint64) {}
  421. func (rc *raftNode) ReportSnapshot(id uint64, status raft.SnapshotStatus) {}