raft.go 8.3 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324
  1. // Copyright 2015 CoreOS, Inc.
  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. "fmt"
  17. "log"
  18. "os"
  19. "strconv"
  20. "time"
  21. "net/http"
  22. "net/url"
  23. "github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
  24. "github.com/coreos/etcd/etcdserver/stats"
  25. "github.com/coreos/etcd/pkg/types"
  26. "github.com/coreos/etcd/raft"
  27. "github.com/coreos/etcd/raft/raftpb"
  28. "github.com/coreos/etcd/rafthttp"
  29. "github.com/coreos/etcd/wal"
  30. "github.com/coreos/etcd/wal/walpb"
  31. )
  32. // A key-value stream backed by raft
  33. type raftNode struct {
  34. proposeC <-chan string // proposed messages (k,v)
  35. confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
  36. commitC chan *string // entries committed to log (k,v)
  37. errorC chan error // errors from raft session
  38. id int // client ID for raft session
  39. peers []string // raft peer URLs
  40. join bool // node is joining an existing cluster
  41. waldir string // path to WAL directory
  42. // raft backing for the commit/error channel
  43. node raft.Node
  44. raftStorage *raft.MemoryStorage
  45. wal *wal.WAL
  46. transport *rafthttp.Transport
  47. stopc chan struct{} // signals proposal channel closed
  48. httpstopc chan struct{} // signals http server to shutdown
  49. httpdonec chan struct{} // signals http server shutdown complete
  50. }
  51. // newRaftNode initiates a raft instance and returns a committed log entry
  52. // channel and error channel. Proposals for log updates are sent over the
  53. // provided the proposal channel. All log entries are replayed over the
  54. // commit channel, followed by a nil message (to indicate the channel is
  55. // current), then new log entries. To shutdown, close proposeC and read errorC.
  56. func newRaftNode(id int, peers []string, join bool, proposeC <-chan string,
  57. confChangeC <-chan raftpb.ConfChange) (<-chan *string, <-chan error) {
  58. rc := &raftNode{
  59. proposeC: proposeC,
  60. confChangeC: confChangeC,
  61. commitC: make(chan *string),
  62. errorC: make(chan error),
  63. id: id,
  64. peers: peers,
  65. join: join,
  66. waldir: fmt.Sprintf("raftexample-%d", id),
  67. raftStorage: raft.NewMemoryStorage(),
  68. stopc: make(chan struct{}),
  69. httpstopc: make(chan struct{}),
  70. httpdonec: make(chan struct{}),
  71. // rest of structure populated after WAL replay
  72. }
  73. go rc.startRaft()
  74. return rc.commitC, rc.errorC
  75. }
  76. // publishEntries writes committed log entries to commit channel and returns
  77. // whether all entries could be published.
  78. func (rc *raftNode) publishEntries(ents []raftpb.Entry) bool {
  79. for i := range ents {
  80. switch ents[i].Type {
  81. case raftpb.EntryNormal:
  82. if len(ents[i].Data) == 0 {
  83. // ignore conf changes and empty messages
  84. continue
  85. }
  86. s := string(ents[i].Data)
  87. select {
  88. case rc.commitC <- &s:
  89. case <-rc.stopc:
  90. return false
  91. }
  92. case raftpb.EntryConfChange:
  93. var cc raftpb.ConfChange
  94. cc.Unmarshal(ents[i].Data)
  95. rc.node.ApplyConfChange(cc)
  96. switch cc.Type {
  97. case raftpb.ConfChangeAddNode:
  98. if len(cc.Context) > 0 {
  99. rc.transport.AddPeer(types.ID(cc.NodeID), []string{string(cc.Context)})
  100. }
  101. case raftpb.ConfChangeRemoveNode:
  102. if cc.NodeID == uint64(rc.id) {
  103. log.Println("I've been removed from the cluster! Shutting down.")
  104. return false
  105. }
  106. rc.transport.RemovePeer(types.ID(cc.NodeID))
  107. }
  108. }
  109. }
  110. return true
  111. }
  112. // openWAL returns a WAL ready for reading.
  113. func (rc *raftNode) openWAL() *wal.WAL {
  114. if wal.Exist(rc.waldir) == false {
  115. if err := os.Mkdir(rc.waldir, 0750); err != nil {
  116. log.Fatalf("raftexample: cannot create dir for wal (%v)", err)
  117. }
  118. w, err := wal.Create(rc.waldir, nil)
  119. if err != nil {
  120. log.Fatalf("raftexample: create wal error (%v)", err)
  121. }
  122. w.Close()
  123. }
  124. w, err := wal.Open(rc.waldir, walpb.Snapshot{})
  125. if err != nil {
  126. log.Fatalf("raftexample: error loading wal (%v)", err)
  127. }
  128. return w
  129. }
  130. // replayWAL replays WAL entries into the raft instance and the commit
  131. // channel and returns an appendable WAL.
  132. func (rc *raftNode) replayWAL() *wal.WAL {
  133. w := rc.openWAL()
  134. _, _, ents, err := w.ReadAll()
  135. if err != nil {
  136. log.Fatalf("raftexample: failed to read WAL (%v)", err)
  137. }
  138. // append to storage so raft starts at the right place in log
  139. rc.raftStorage.Append(ents)
  140. rc.publishEntries(ents)
  141. // send nil value so client knows commit channel is current
  142. rc.commitC <- nil
  143. return w
  144. }
  145. func (rc *raftNode) writeError(err error) {
  146. rc.stopHTTP()
  147. close(rc.commitC)
  148. rc.errorC <- err
  149. close(rc.errorC)
  150. rc.node.Stop()
  151. }
  152. func (rc *raftNode) startRaft() {
  153. oldwal := wal.Exist(rc.waldir)
  154. rc.wal = rc.replayWAL()
  155. rpeers := make([]raft.Peer, len(rc.peers))
  156. for i := range rpeers {
  157. rpeers[i] = raft.Peer{ID: uint64(i + 1)}
  158. }
  159. c := &raft.Config{
  160. ID: uint64(rc.id),
  161. ElectionTick: 10,
  162. HeartbeatTick: 1,
  163. Storage: rc.raftStorage,
  164. MaxSizePerMsg: 1024 * 1024,
  165. MaxInflightMsgs: 256,
  166. }
  167. if oldwal {
  168. rc.node = raft.RestartNode(c)
  169. } else {
  170. startPeers := rpeers
  171. if rc.join {
  172. startPeers = nil
  173. }
  174. rc.node = raft.StartNode(c, startPeers)
  175. }
  176. ss := &stats.ServerStats{}
  177. ss.Initialize()
  178. rc.transport = &rafthttp.Transport{
  179. ID: types.ID(rc.id),
  180. ClusterID: 0x1000,
  181. Raft: rc,
  182. ServerStats: ss,
  183. LeaderStats: stats.NewLeaderStats(strconv.Itoa(rc.id)),
  184. ErrorC: make(chan error),
  185. }
  186. rc.transport.Start()
  187. for i := range rc.peers {
  188. if i+1 != rc.id {
  189. rc.transport.AddPeer(types.ID(i+1), []string{rc.peers[i]})
  190. }
  191. }
  192. go rc.serveRaft()
  193. go rc.serveChannels()
  194. }
  195. // stop closes http, closes all channels, and stops raft.
  196. func (rc *raftNode) stop() {
  197. rc.stopHTTP()
  198. close(rc.commitC)
  199. close(rc.errorC)
  200. rc.node.Stop()
  201. os.Exit(0)
  202. }
  203. func (rc *raftNode) stopHTTP() {
  204. rc.transport.Stop()
  205. close(rc.httpstopc)
  206. <-rc.httpdonec
  207. }
  208. func (rc *raftNode) serveChannels() {
  209. defer rc.wal.Close()
  210. ticker := time.NewTicker(100 * time.Millisecond)
  211. defer ticker.Stop()
  212. // send proposals over raft
  213. go func() {
  214. var confChangeCount uint64 = 0
  215. for rc.proposeC != nil && rc.confChangeC != nil {
  216. select {
  217. case prop, ok := <-rc.proposeC:
  218. if !ok {
  219. rc.proposeC = nil
  220. } else {
  221. // blocks until accepted by raft state machine
  222. rc.node.Propose(context.TODO(), []byte(prop))
  223. }
  224. case cc, ok := <-rc.confChangeC:
  225. if !ok {
  226. rc.confChangeC = nil
  227. } else {
  228. confChangeCount += 1
  229. cc.ID = confChangeCount
  230. rc.node.ProposeConfChange(context.TODO(), cc)
  231. }
  232. }
  233. }
  234. // client closed channel; shutdown raft if not already
  235. close(rc.stopc)
  236. }()
  237. // event loop on raft state machine updates
  238. for {
  239. select {
  240. case <-ticker.C:
  241. rc.node.Tick()
  242. // store raft entries to wal, then publish over commit channel
  243. case rd := <-rc.node.Ready():
  244. rc.wal.Save(rd.HardState, rd.Entries)
  245. rc.raftStorage.Append(rd.Entries)
  246. rc.transport.Send(rd.Messages)
  247. if ok := rc.publishEntries(rd.CommittedEntries); !ok {
  248. rc.stop()
  249. return
  250. }
  251. rc.node.Advance()
  252. case err := <-rc.transport.ErrorC:
  253. rc.writeError(err)
  254. return
  255. case <-rc.stopc:
  256. rc.stop()
  257. return
  258. }
  259. }
  260. }
  261. func (rc *raftNode) serveRaft() {
  262. url, err := url.Parse(rc.peers[rc.id-1])
  263. if err != nil {
  264. log.Fatalf("raftexample: Failed parsing URL (%v)", err)
  265. }
  266. ln, err := newStoppableListener(url.Host, rc.httpstopc)
  267. if err != nil {
  268. log.Fatalf("raftexample: Failed to listen rafthttp (%v)", err)
  269. }
  270. err = (&http.Server{Handler: rc.transport.Handler()}).Serve(ln)
  271. select {
  272. case <-rc.httpstopc:
  273. default:
  274. log.Fatalf("raftexample: Failed to serve rafthttp (%v)", err)
  275. }
  276. close(rc.httpdonec)
  277. }
  278. func (rc *raftNode) Process(ctx context.Context, m raftpb.Message) error {
  279. return rc.node.Step(ctx, m)
  280. }
  281. func (rc *raftNode) IsIDRemoved(id uint64) bool { return false }
  282. func (rc *raftNode) ReportUnreachable(id uint64) {}
  283. func (rc *raftNode) ReportSnapshot(id uint64, status raft.SnapshotStatus) {}