raft.go 6.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234
  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. }
  46. // newRaftNode initiates a raft instance and returns a committed log entry
  47. // channel and error channel. Proposals for log updates are sent over the
  48. // provided the proposal channel. All log entries are replayed over the
  49. // commit channel, followed by a nil message (to indicate the channel is
  50. // current), then new log entries.
  51. func newRaftNode(id int, peers []string, proposeC <-chan string) (<-chan *string, <-chan error) {
  52. rc := &raftNode{
  53. proposeC: proposeC,
  54. commitC: make(chan *string),
  55. errorC: make(chan error),
  56. id: id,
  57. peers: peers,
  58. waldir: fmt.Sprintf("raftexample-%d", id),
  59. raftStorage: raft.NewMemoryStorage(),
  60. // rest of structure populated after WAL replay
  61. }
  62. go rc.startRaft()
  63. return rc.commitC, rc.errorC
  64. }
  65. // publishEntries writes committed log entries to commit channel.
  66. func (rc *raftNode) publishEntries(ents []raftpb.Entry) {
  67. for i := range ents {
  68. if ents[i].Type != raftpb.EntryNormal || len(ents[i].Data) == 0 {
  69. // ignore conf changes and empty messages
  70. continue
  71. }
  72. s := string(ents[i].Data)
  73. rc.commitC <- &s
  74. }
  75. }
  76. // openWAL returns a WAL ready for reading.
  77. func (rc *raftNode) openWAL() *wal.WAL {
  78. if wal.Exist(rc.waldir) == false {
  79. if err := os.Mkdir(rc.waldir, 0750); err != nil {
  80. log.Fatalf("raftexample: cannot create dir for wal (%v)", err)
  81. }
  82. w, err := wal.Create(rc.waldir, nil)
  83. if err != nil {
  84. log.Fatalf("raftexample: create wal error (%v)", err)
  85. }
  86. w.Close()
  87. }
  88. w, err := wal.Open(rc.waldir, walpb.Snapshot{})
  89. if err != nil {
  90. log.Fatalf("raftexample: error loading wal (%v)", err)
  91. }
  92. return w
  93. }
  94. // replayWAL replays WAL entries into the raft instance and the commit
  95. // channel and returns an appendable WAL.
  96. func (rc *raftNode) replayWAL() *wal.WAL {
  97. w := rc.openWAL()
  98. _, _, ents, err := w.ReadAll()
  99. if err != nil {
  100. log.Fatalf("raftexample: failed to read WAL (%v)", err)
  101. }
  102. // append to storage so raft starts at the right place in log
  103. rc.raftStorage.Append(ents)
  104. rc.publishEntries(ents)
  105. // send nil value so client knows commit channel is current
  106. rc.commitC <- nil
  107. return w
  108. }
  109. func (rc *raftNode) writeError(err error) {
  110. rc.errorC <- err
  111. rc.stop()
  112. }
  113. func (rc *raftNode) stop() {
  114. close(rc.commitC)
  115. close(rc.errorC)
  116. rc.node.Stop()
  117. }
  118. func (rc *raftNode) startRaft() {
  119. oldwal := wal.Exist(rc.waldir)
  120. rc.wal = rc.replayWAL()
  121. rpeers := make([]raft.Peer, len(rc.peers))
  122. for i := range rpeers {
  123. rpeers[i] = raft.Peer{ID: uint64(i + 1)}
  124. }
  125. c := &raft.Config{
  126. ID: uint64(rc.id),
  127. ElectionTick: 10,
  128. HeartbeatTick: 1,
  129. Storage: rc.raftStorage,
  130. MaxSizePerMsg: 1024 * 1024,
  131. MaxInflightMsgs: 256,
  132. }
  133. if oldwal {
  134. rc.node = raft.RestartNode(c)
  135. } else {
  136. rc.node = raft.StartNode(c, rpeers)
  137. }
  138. ss := &stats.ServerStats{}
  139. ss.Initialize()
  140. rc.transport = &rafthttp.Transport{
  141. ID: types.ID(rc.id),
  142. ClusterID: 0x1000,
  143. Raft: rc,
  144. ServerStats: ss,
  145. LeaderStats: stats.NewLeaderStats(strconv.Itoa(rc.id)),
  146. ErrorC: make(chan error),
  147. }
  148. rc.transport.Start()
  149. for i := range rc.peers {
  150. if i+1 != rc.id {
  151. rc.transport.AddPeer(types.ID(i+1), []string{rc.peers[i]})
  152. }
  153. }
  154. go rc.serveRaft()
  155. go rc.serveChannels()
  156. }
  157. func (rc *raftNode) serveChannels() {
  158. defer rc.wal.Close()
  159. ticker := time.NewTicker(100 * time.Millisecond)
  160. defer ticker.Stop()
  161. // event loop on client proposals and raft updates
  162. for {
  163. select {
  164. case <-ticker.C:
  165. rc.node.Tick()
  166. // send proposals over raft
  167. case prop, ok := <-rc.proposeC:
  168. if !ok {
  169. // client closed channel; shut down
  170. rc.stop()
  171. return
  172. }
  173. rc.node.Propose(context.TODO(), []byte(prop))
  174. // store raft entries to wal, then publish over commit channel
  175. case rd := <-rc.node.Ready():
  176. rc.wal.Save(rd.HardState, rd.Entries)
  177. rc.raftStorage.Append(rd.Entries)
  178. rc.transport.Send(rd.Messages)
  179. rc.publishEntries(rd.Entries)
  180. rc.node.Advance()
  181. case err := <-rc.transport.ErrorC:
  182. rc.writeError(err)
  183. return
  184. }
  185. }
  186. }
  187. func (rc *raftNode) serveRaft() {
  188. url, err := url.Parse(rc.peers[rc.id-1])
  189. if err != nil {
  190. log.Fatalf("raftexample: Failed parsing URL (%v)", err)
  191. }
  192. srv := http.Server{Addr: url.Host, Handler: rc.transport.Handler()}
  193. if err := srv.ListenAndServe(); err != nil {
  194. log.Fatalf("raftexample: Failed serving rafthttp (%v)", err)
  195. }
  196. }
  197. func (rc *raftNode) Process(ctx context.Context, m raftpb.Message) error {
  198. return rc.node.Step(ctx, m)
  199. }
  200. func (rc *raftNode) IsIDRemoved(id uint64) bool { return false }
  201. func (rc *raftNode) ReportUnreachable(id uint64) {}
  202. func (rc *raftNode) ReportSnapshot(id uint64, status raft.SnapshotStatus) {}