server.go 10.0 KB

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