server.go 10 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391
  1. package etcdserver
  2. import (
  3. "errors"
  4. "log"
  5. "math/rand"
  6. "sync/atomic"
  7. "time"
  8. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  9. "github.com/coreos/etcd/raft"
  10. "github.com/coreos/etcd/raft/raftpb"
  11. "github.com/coreos/etcd/store"
  12. "github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
  13. "github.com/coreos/etcd/wait"
  14. )
  15. const (
  16. defaultSyncTimeout = time.Second
  17. DefaultSnapCount = 10000
  18. )
  19. var (
  20. ErrUnknownMethod = errors.New("etcdserver: unknown method")
  21. ErrStopped = errors.New("etcdserver: server stopped")
  22. )
  23. func init() {
  24. rand.Seed(time.Now().UnixNano())
  25. }
  26. type SendFunc func(m []raftpb.Message)
  27. type SaveFunc func(st raftpb.HardState, ents []raftpb.Entry)
  28. type Response struct {
  29. Event *store.Event
  30. Watcher store.Watcher
  31. err error
  32. }
  33. type Storage interface {
  34. // Save function saves ents and state to the underlying stable storage.
  35. // Save MUST block until st and ents are on stable storage.
  36. Save(st raftpb.HardState, ents []raftpb.Entry)
  37. // SaveSnap function saves snapshot to the underlying stable storage.
  38. SaveSnap(snap raftpb.Snapshot)
  39. // TODO: WAL should be able to control cut itself. After implement self-controled cut,
  40. // remove it in this interface.
  41. // Cut cuts out a new wal file for saving new state and entries.
  42. Cut() error
  43. }
  44. type Server interface {
  45. // Start performs any initialization of the Server necessary for it to
  46. // begin serving requests. It must be called before Do or Process.
  47. // Start must be non-blocking; any long-running server functionality
  48. // should be implemented in goroutines.
  49. Start()
  50. // Stop terminates the Server and performs any necessary finalization.
  51. // Do and Process cannot be called after Stop has been invoked.
  52. Stop()
  53. // Do takes a request and attempts to fulfil it, returning a Response.
  54. Do(ctx context.Context, r pb.Request) (Response, error)
  55. // Process takes a raft message and applies it to the server's raft state
  56. // machine, respecting any timeout of the given context.
  57. Process(ctx context.Context, m raftpb.Message) error
  58. }
  59. type RaftTimer interface {
  60. Index() int64
  61. Term() int64
  62. }
  63. // EtcdServer is the production implementation of the Server interface
  64. type EtcdServer struct {
  65. w wait.Wait
  66. done chan struct{}
  67. Node raft.Node
  68. Store store.Store
  69. // Send specifies the send function for sending msgs to peers. Send
  70. // MUST NOT block. It is okay to drop messages, since clients should
  71. // timeout and reissue their messages. If Send is nil, server will
  72. // panic.
  73. Send SendFunc
  74. Storage Storage
  75. Ticker <-chan time.Time
  76. SyncTicker <-chan time.Time
  77. SnapCount int64 // number of entries to trigger a snapshot
  78. // Cache of the latest raft index and raft term the server has seen
  79. raftIndex int64
  80. raftTerm int64
  81. }
  82. // Start prepares and starts server in a new goroutine. It is no longer safe to
  83. // modify a server's fields after it has been sent to Start.
  84. func (s *EtcdServer) Start() {
  85. if s.SnapCount == 0 {
  86. log.Printf("etcdserver: set snapshot count to default %d", DefaultSnapCount)
  87. s.SnapCount = DefaultSnapCount
  88. }
  89. s.w = wait.New()
  90. s.done = make(chan struct{})
  91. go s.run()
  92. }
  93. func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
  94. return s.Node.Step(ctx, m)
  95. }
  96. func (s *EtcdServer) run() {
  97. var syncC <-chan time.Time
  98. // snapi indicates the index of the last submitted snapshot request
  99. var snapi, appliedi int64
  100. for {
  101. select {
  102. case <-s.Ticker:
  103. s.Node.Tick()
  104. case rd := <-s.Node.Ready():
  105. s.Storage.Save(rd.HardState, rd.Entries)
  106. s.Storage.SaveSnap(rd.Snapshot)
  107. s.Send(rd.Messages)
  108. // TODO(bmizerany): do this in the background, but take
  109. // care to apply entries in a single goroutine, and not
  110. // race them.
  111. for _, e := range rd.CommittedEntries {
  112. switch e.Type {
  113. case raftpb.EntryNormal:
  114. var r pb.Request
  115. if err := r.Unmarshal(e.Data); err != nil {
  116. panic("TODO: this is bad, what do we do about it?")
  117. }
  118. s.w.Trigger(r.Id, s.apply(r))
  119. case raftpb.EntryConfChange:
  120. var cc raftpb.ConfChange
  121. if err := cc.Unmarshal(e.Data); err != nil {
  122. panic("TODO: this is bad, what do we do about it?")
  123. }
  124. s.Node.ApplyConfChange(cc)
  125. s.w.Trigger(cc.ID, nil)
  126. default:
  127. panic("unexpected entry type")
  128. }
  129. atomic.StoreInt64(&s.raftIndex, e.Index)
  130. atomic.StoreInt64(&s.raftTerm, e.Term)
  131. appliedi = e.Index
  132. }
  133. if rd.Snapshot.Index > snapi {
  134. snapi = rd.Snapshot.Index
  135. }
  136. // recover from snapshot if it is more updated than current applied
  137. if rd.Snapshot.Index > appliedi {
  138. if err := s.Store.Recovery(rd.Snapshot.Data); err != nil {
  139. panic("TODO: this is bad, what do we do about it?")
  140. }
  141. appliedi = rd.Snapshot.Index
  142. }
  143. if appliedi-snapi > s.SnapCount {
  144. s.snapshot()
  145. snapi = appliedi
  146. }
  147. if rd.SoftState != nil {
  148. if rd.RaftState == raft.StateLeader {
  149. syncC = s.SyncTicker
  150. } else {
  151. syncC = nil
  152. }
  153. }
  154. case <-syncC:
  155. s.sync(defaultSyncTimeout)
  156. case <-s.done:
  157. return
  158. }
  159. }
  160. }
  161. // Stop stops the server, and shuts down the running goroutine. Stop should be
  162. // called after a Start(s), otherwise it will block forever.
  163. func (s *EtcdServer) Stop() {
  164. s.Node.Stop()
  165. close(s.done)
  166. }
  167. // Do interprets r and performs an operation on s.Store according to r.Method
  168. // and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
  169. // Quorum == true, r will be sent through consensus before performing its
  170. // respective operation. Do will block until an action is performed or there is
  171. // an error.
  172. func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
  173. if r.Id == 0 {
  174. panic("r.Id cannot be 0")
  175. }
  176. if r.Method == "GET" && r.Quorum {
  177. r.Method = "QGET"
  178. }
  179. switch r.Method {
  180. case "POST", "PUT", "DELETE", "QGET":
  181. data, err := r.Marshal()
  182. if err != nil {
  183. return Response{}, err
  184. }
  185. ch := s.w.Register(r.Id)
  186. s.Node.Propose(ctx, data)
  187. select {
  188. case x := <-ch:
  189. resp := x.(Response)
  190. return resp, resp.err
  191. case <-ctx.Done():
  192. s.w.Trigger(r.Id, nil) // GC wait
  193. return Response{}, ctx.Err()
  194. case <-s.done:
  195. return Response{}, ErrStopped
  196. }
  197. case "GET":
  198. switch {
  199. case r.Wait:
  200. wc, err := s.Store.Watch(r.Path, r.Recursive, r.Stream, r.Since)
  201. if err != nil {
  202. return Response{}, err
  203. }
  204. return Response{Watcher: wc}, nil
  205. default:
  206. ev, err := s.Store.Get(r.Path, r.Recursive, r.Sorted)
  207. if err != nil {
  208. return Response{}, err
  209. }
  210. return Response{Event: ev}, nil
  211. }
  212. default:
  213. return Response{}, ErrUnknownMethod
  214. }
  215. }
  216. func (s *EtcdServer) AddNode(ctx context.Context, id int64, context []byte) error {
  217. cc := raftpb.ConfChange{
  218. ID: GenID(),
  219. Type: raftpb.ConfChangeAddNode,
  220. NodeID: id,
  221. Context: context,
  222. }
  223. return s.configure(ctx, cc)
  224. }
  225. func (s *EtcdServer) RemoveNode(ctx context.Context, id int64) error {
  226. cc := raftpb.ConfChange{
  227. ID: GenID(),
  228. Type: raftpb.ConfChangeRemoveNode,
  229. NodeID: id,
  230. }
  231. return s.configure(ctx, cc)
  232. }
  233. // Implement the RaftTimer interface
  234. func (s *EtcdServer) Index() int64 {
  235. return atomic.LoadInt64(&s.raftIndex)
  236. }
  237. func (s *EtcdServer) Term() int64 {
  238. return atomic.LoadInt64(&s.raftTerm)
  239. }
  240. // configure sends configuration change through consensus then performs it.
  241. // It will block until the change is performed or there is an error.
  242. func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) error {
  243. ch := s.w.Register(cc.ID)
  244. if err := s.Node.ProposeConfChange(ctx, cc); err != nil {
  245. log.Printf("configure error: %v", err)
  246. s.w.Trigger(cc.ID, nil)
  247. return err
  248. }
  249. select {
  250. case <-ch:
  251. return nil
  252. case <-ctx.Done():
  253. s.w.Trigger(cc.ID, nil) // GC wait
  254. return ctx.Err()
  255. case <-s.done:
  256. return ErrStopped
  257. }
  258. }
  259. // sync proposes a SYNC request and is non-blocking.
  260. // This makes no guarantee that the request will be proposed or performed.
  261. // The request will be cancelled after the given timeout.
  262. func (s *EtcdServer) sync(timeout time.Duration) {
  263. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  264. req := pb.Request{
  265. Method: "SYNC",
  266. Id: GenID(),
  267. Time: time.Now().UnixNano(),
  268. }
  269. data, err := req.Marshal()
  270. if err != nil {
  271. log.Printf("marshal request %#v error: %v", req, err)
  272. return
  273. }
  274. // There is no promise that node has leader when do SYNC request,
  275. // so it uses goroutine to propose.
  276. go func() {
  277. s.Node.Propose(ctx, data)
  278. cancel()
  279. }()
  280. }
  281. func getExpirationTime(r *pb.Request) time.Time {
  282. var t time.Time
  283. if r.Expiration != 0 {
  284. t = time.Unix(0, r.Expiration)
  285. }
  286. return t
  287. }
  288. // apply interprets r as a call to store.X and returns a Response interpreted
  289. // from store.Event
  290. func (s *EtcdServer) apply(r pb.Request) Response {
  291. f := func(ev *store.Event, err error) Response {
  292. return Response{Event: ev, err: err}
  293. }
  294. expr := getExpirationTime(&r)
  295. switch r.Method {
  296. case "POST":
  297. return f(s.Store.Create(r.Path, r.Dir, r.Val, true, expr))
  298. case "PUT":
  299. exists, existsSet := getBool(r.PrevExist)
  300. switch {
  301. case existsSet:
  302. if exists {
  303. return f(s.Store.Update(r.Path, r.Val, expr))
  304. }
  305. return f(s.Store.Create(r.Path, r.Dir, r.Val, false, expr))
  306. case r.PrevIndex > 0 || r.PrevValue != "":
  307. return f(s.Store.CompareAndSwap(r.Path, r.PrevValue, r.PrevIndex, r.Val, expr))
  308. default:
  309. return f(s.Store.Set(r.Path, r.Dir, r.Val, expr))
  310. }
  311. case "DELETE":
  312. switch {
  313. case r.PrevIndex > 0 || r.PrevValue != "":
  314. return f(s.Store.CompareAndDelete(r.Path, r.PrevValue, r.PrevIndex))
  315. default:
  316. return f(s.Store.Delete(r.Path, r.Dir, r.Recursive))
  317. }
  318. case "QGET":
  319. return f(s.Store.Get(r.Path, r.Recursive, r.Sorted))
  320. case "SYNC":
  321. s.Store.DeleteExpiredKeys(time.Unix(0, r.Time))
  322. return Response{}
  323. default:
  324. // This should never be reached, but just in case:
  325. return Response{err: ErrUnknownMethod}
  326. }
  327. }
  328. // TODO: non-blocking snapshot
  329. func (s *EtcdServer) snapshot() {
  330. d, err := s.Store.Save()
  331. // TODO: current store will never fail to do a snapshot
  332. // what should we do if the store might fail?
  333. if err != nil {
  334. panic("TODO: this is bad, what do we do about it?")
  335. }
  336. s.Node.Compact(d)
  337. s.Storage.Cut()
  338. }
  339. // TODO: move the function to /id pkg maybe?
  340. // GenID generates a random id that is not equal to 0.
  341. func GenID() (n int64) {
  342. for n == 0 {
  343. n = rand.Int63()
  344. }
  345. return
  346. }
  347. func getBool(v *bool) (vv bool, set bool) {
  348. if v == nil {
  349. return false, false
  350. }
  351. return *v, true
  352. }