raft.go 7.6 KB

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