raft.go 7.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273
  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. if ents[i].Type != raftpb.EntryNormal || len(ents[i].Data) == 0 {
  76. // ignore conf changes and empty messages
  77. continue
  78. }
  79. s := string(ents[i].Data)
  80. select {
  81. case rc.commitC <- &s:
  82. case <-rc.stopc:
  83. return false
  84. }
  85. }
  86. return true
  87. }
  88. // openWAL returns a WAL ready for reading.
  89. func (rc *raftNode) openWAL() *wal.WAL {
  90. if wal.Exist(rc.waldir) == false {
  91. if err := os.Mkdir(rc.waldir, 0750); err != nil {
  92. log.Fatalf("raftexample: cannot create dir for wal (%v)", err)
  93. }
  94. w, err := wal.Create(rc.waldir, nil)
  95. if err != nil {
  96. log.Fatalf("raftexample: create wal error (%v)", err)
  97. }
  98. w.Close()
  99. }
  100. w, err := wal.Open(rc.waldir, walpb.Snapshot{})
  101. if err != nil {
  102. log.Fatalf("raftexample: error loading wal (%v)", err)
  103. }
  104. return w
  105. }
  106. // replayWAL replays WAL entries into the raft instance and the commit
  107. // channel and returns an appendable WAL.
  108. func (rc *raftNode) replayWAL() *wal.WAL {
  109. w := rc.openWAL()
  110. _, _, ents, err := w.ReadAll()
  111. if err != nil {
  112. log.Fatalf("raftexample: failed to read WAL (%v)", err)
  113. }
  114. // append to storage so raft starts at the right place in log
  115. rc.raftStorage.Append(ents)
  116. rc.publishEntries(ents)
  117. // send nil value so client knows commit channel is current
  118. rc.commitC <- nil
  119. return w
  120. }
  121. func (rc *raftNode) writeError(err error) {
  122. rc.stopHTTP()
  123. close(rc.commitC)
  124. rc.errorC <- err
  125. close(rc.errorC)
  126. rc.node.Stop()
  127. }
  128. func (rc *raftNode) startRaft() {
  129. oldwal := wal.Exist(rc.waldir)
  130. rc.wal = rc.replayWAL()
  131. rpeers := make([]raft.Peer, len(rc.peers))
  132. for i := range rpeers {
  133. rpeers[i] = raft.Peer{ID: uint64(i + 1)}
  134. }
  135. c := &raft.Config{
  136. ID: uint64(rc.id),
  137. ElectionTick: 10,
  138. HeartbeatTick: 1,
  139. Storage: rc.raftStorage,
  140. MaxSizePerMsg: 1024 * 1024,
  141. MaxInflightMsgs: 256,
  142. }
  143. if oldwal {
  144. rc.node = raft.RestartNode(c)
  145. } else {
  146. rc.node = raft.StartNode(c, rpeers)
  147. }
  148. ss := &stats.ServerStats{}
  149. ss.Initialize()
  150. rc.transport = &rafthttp.Transport{
  151. ID: types.ID(rc.id),
  152. ClusterID: 0x1000,
  153. Raft: rc,
  154. ServerStats: ss,
  155. LeaderStats: stats.NewLeaderStats(strconv.Itoa(rc.id)),
  156. ErrorC: make(chan error),
  157. }
  158. rc.transport.Start()
  159. for i := range rc.peers {
  160. if i+1 != rc.id {
  161. rc.transport.AddPeer(types.ID(i+1), []string{rc.peers[i]})
  162. }
  163. }
  164. go rc.serveRaft()
  165. go rc.serveChannels()
  166. }
  167. // stop closes http, closes all channels, and stops raft.
  168. func (rc *raftNode) stop() {
  169. rc.stopHTTP()
  170. close(rc.commitC)
  171. close(rc.errorC)
  172. rc.node.Stop()
  173. }
  174. func (rc *raftNode) stopHTTP() {
  175. rc.transport.Stop()
  176. close(rc.httpstopc)
  177. <-rc.httpdonec
  178. }
  179. func (rc *raftNode) serveChannels() {
  180. defer rc.wal.Close()
  181. ticker := time.NewTicker(100 * time.Millisecond)
  182. defer ticker.Stop()
  183. // send proposals over raft
  184. go func() {
  185. for prop := range rc.proposeC {
  186. // blocks until accepted by raft state machine
  187. rc.node.Propose(context.TODO(), []byte(prop))
  188. }
  189. // client closed channel; shutdown raft if not already
  190. close(rc.stopc)
  191. }()
  192. // event loop on raft state machine updates
  193. for {
  194. select {
  195. case <-ticker.C:
  196. rc.node.Tick()
  197. // store raft entries to wal, then publish over commit channel
  198. case rd := <-rc.node.Ready():
  199. rc.wal.Save(rd.HardState, rd.Entries)
  200. rc.raftStorage.Append(rd.Entries)
  201. rc.transport.Send(rd.Messages)
  202. if ok := rc.publishEntries(rd.Entries); !ok {
  203. rc.stop()
  204. return
  205. }
  206. rc.node.Advance()
  207. case err := <-rc.transport.ErrorC:
  208. rc.writeError(err)
  209. return
  210. case <-rc.stopc:
  211. rc.stop()
  212. return
  213. }
  214. }
  215. }
  216. func (rc *raftNode) serveRaft() {
  217. url, err := url.Parse(rc.peers[rc.id-1])
  218. if err != nil {
  219. log.Fatalf("raftexample: Failed parsing URL (%v)", err)
  220. }
  221. ln, err := newStoppableListener(url.Host, rc.httpstopc)
  222. if err != nil {
  223. log.Fatalf("raftexample: Failed to listen rafthttp (%v)", err)
  224. }
  225. err = (&http.Server{Handler: rc.transport.Handler()}).Serve(ln)
  226. select {
  227. case <-rc.httpstopc:
  228. default:
  229. log.Fatalf("raftexample: Failed to serve rafthttp (%v)", err)
  230. }
  231. close(rc.httpdonec)
  232. }
  233. func (rc *raftNode) Process(ctx context.Context, m raftpb.Message) error {
  234. return rc.node.Step(ctx, m)
  235. }
  236. func (rc *raftNode) IsIDRemoved(id uint64) bool { return false }
  237. func (rc *raftNode) ReportUnreachable(id uint64) {}
  238. func (rc *raftNode) ReportSnapshot(id uint64, status raft.SnapshotStatus) {}