raft.go 13 KB

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