server.go 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588
  1. package etcdserver
  2. import (
  3. "encoding/json"
  4. "errors"
  5. "log"
  6. "math/rand"
  7. "os"
  8. "sync/atomic"
  9. "time"
  10. "github.com/coreos/etcd/discovery"
  11. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  12. "github.com/coreos/etcd/raft"
  13. "github.com/coreos/etcd/raft/raftpb"
  14. "github.com/coreos/etcd/snap"
  15. "github.com/coreos/etcd/store"
  16. "github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
  17. "github.com/coreos/etcd/wait"
  18. "github.com/coreos/etcd/wal"
  19. )
  20. const (
  21. // owner can make/remove files inside the directory
  22. privateDirMode = 0700
  23. defaultSyncTimeout = time.Second
  24. DefaultSnapCount = 10000
  25. // TODO: calculate based on heartbeat interval
  26. defaultPublishRetryInterval = 5 * time.Second
  27. )
  28. var (
  29. ErrUnknownMethod = errors.New("etcdserver: unknown method")
  30. ErrStopped = errors.New("etcdserver: server stopped")
  31. )
  32. func init() {
  33. rand.Seed(time.Now().UnixNano())
  34. }
  35. type sendFunc func(m []raftpb.Message)
  36. type Response struct {
  37. Event *store.Event
  38. Watcher store.Watcher
  39. err error
  40. }
  41. type Storage interface {
  42. // Save function saves ents and state to the underlying stable storage.
  43. // Save MUST block until st and ents are on stable storage.
  44. Save(st raftpb.HardState, ents []raftpb.Entry)
  45. // SaveSnap function saves snapshot to the underlying stable storage.
  46. SaveSnap(snap raftpb.Snapshot)
  47. // TODO: WAL should be able to control cut itself. After implement self-controled cut,
  48. // remove it in this interface.
  49. // Cut cuts out a new wal file for saving new state and entries.
  50. Cut() error
  51. }
  52. type Server interface {
  53. // Start performs any initialization of the Server necessary for it to
  54. // begin serving requests. It must be called before Do or Process.
  55. // Start must be non-blocking; any long-running server functionality
  56. // should be implemented in goroutines.
  57. Start()
  58. // Stop terminates the Server and performs any necessary finalization.
  59. // Do and Process cannot be called after Stop has been invoked.
  60. Stop()
  61. // Do takes a request and attempts to fulfil it, returning a Response.
  62. Do(ctx context.Context, r pb.Request) (Response, error)
  63. // Process takes a raft message and applies it to the server's raft state
  64. // machine, respecting any timeout of the given context.
  65. Process(ctx context.Context, m raftpb.Message) error
  66. }
  67. type RaftTimer interface {
  68. Index() uint64
  69. Term() uint64
  70. }
  71. // EtcdServer is the production implementation of the Server interface
  72. type EtcdServer struct {
  73. w wait.Wait
  74. done chan struct{}
  75. id uint64
  76. attributes Attributes
  77. ClusterStore ClusterStore
  78. node raft.Node
  79. store store.Store
  80. // send specifies the send function for sending msgs to members. send
  81. // MUST NOT block. It is okay to drop messages, since clients should
  82. // timeout and reissue their messages. If send is nil, server will
  83. // panic.
  84. send sendFunc
  85. storage Storage
  86. ticker <-chan time.Time
  87. syncTicker <-chan time.Time
  88. snapCount uint64 // number of entries to trigger a snapshot
  89. // Cache of the latest raft index and raft term the server has seen
  90. raftIndex uint64
  91. raftTerm uint64
  92. }
  93. // NewServer creates a new EtcdServer from the supplied configuration. The
  94. // configuration is considered static for the lifetime of the EtcdServer.
  95. func NewServer(cfg *ServerConfig) *EtcdServer {
  96. if err := cfg.Verify(); err != nil {
  97. log.Fatalln(err)
  98. }
  99. if err := os.MkdirAll(cfg.SnapDir(), privateDirMode); err != nil {
  100. log.Fatalf("etcdserver: cannot create snapshot directory: %v", err)
  101. }
  102. ss := snap.New(cfg.SnapDir())
  103. st := store.New()
  104. var w *wal.WAL
  105. var n raft.Node
  106. if !wal.Exist(cfg.WALDir()) {
  107. if !cfg.IsBootstrap() {
  108. log.Fatalf("etcd: initial cluster state unset and no wal or discovery URL found")
  109. }
  110. if cfg.DiscoveryURL != "" {
  111. d, err := discovery.New(cfg.DiscoveryURL, cfg.ID(), cfg.Cluster.String())
  112. if err != nil {
  113. log.Fatalf("etcd: cannot init discovery %v", err)
  114. }
  115. s, err := d.Discover()
  116. if err != nil {
  117. log.Fatalf("etcd: %v", err)
  118. }
  119. if err = cfg.Cluster.Set(s); err != nil {
  120. log.Fatalf("etcd: %v", err)
  121. }
  122. }
  123. n, w = startNode(cfg)
  124. } else {
  125. if cfg.DiscoveryURL != "" {
  126. log.Printf("etcd: warn: ignoring discovery URL: etcd has already been initialized and has a valid log in %q", cfg.WALDir())
  127. }
  128. var index uint64
  129. snapshot, err := ss.Load()
  130. if err != nil && err != snap.ErrNoSnapshot {
  131. log.Fatal(err)
  132. }
  133. if snapshot != nil {
  134. log.Printf("etcdserver: restart from snapshot at index %d", snapshot.Index)
  135. st.Recovery(snapshot.Data)
  136. index = snapshot.Index
  137. }
  138. n, w = restartNode(cfg, index, snapshot)
  139. }
  140. cls := &clusterStore{Store: st}
  141. s := &EtcdServer{
  142. store: st,
  143. node: n,
  144. id: cfg.ID(),
  145. attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
  146. storage: struct {
  147. *wal.WAL
  148. *snap.Snapshotter
  149. }{w, ss},
  150. send: Sender(cfg.Transport, cls),
  151. ticker: time.Tick(100 * time.Millisecond),
  152. syncTicker: time.Tick(500 * time.Millisecond),
  153. snapCount: cfg.SnapCount,
  154. ClusterStore: cls,
  155. }
  156. return s
  157. }
  158. // Start prepares and starts server in a new goroutine. It is no longer safe to
  159. // modify a server's fields after it has been sent to Start.
  160. // It also starts a goroutine to publish its server information.
  161. func (s *EtcdServer) Start() {
  162. s.start()
  163. go s.publish(defaultPublishRetryInterval)
  164. }
  165. // start prepares and starts server in a new goroutine. It is no longer safe to
  166. // modify a server's fields after it has been sent to Start.
  167. // This function is just used for testing.
  168. func (s *EtcdServer) start() {
  169. if s.snapCount == 0 {
  170. log.Printf("etcdserver: set snapshot count to default %d", DefaultSnapCount)
  171. s.snapCount = DefaultSnapCount
  172. }
  173. s.w = wait.New()
  174. s.done = make(chan struct{})
  175. // TODO: if this is an empty log, writes all peer infos
  176. // into the first entry
  177. go s.run()
  178. }
  179. func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
  180. return s.node.Step(ctx, m)
  181. }
  182. func (s *EtcdServer) run() {
  183. var syncC <-chan time.Time
  184. // snapi indicates the index of the last submitted snapshot request
  185. var snapi, appliedi uint64
  186. var nodes []uint64
  187. for {
  188. select {
  189. case <-s.ticker:
  190. s.node.Tick()
  191. case rd := <-s.node.Ready():
  192. s.storage.Save(rd.HardState, rd.Entries)
  193. s.storage.SaveSnap(rd.Snapshot)
  194. s.send(rd.Messages)
  195. // TODO(bmizerany): do this in the background, but take
  196. // care to apply entries in a single goroutine, and not
  197. // race them.
  198. // TODO: apply configuration change into ClusterStore.
  199. for _, e := range rd.CommittedEntries {
  200. switch e.Type {
  201. case raftpb.EntryNormal:
  202. var r pb.Request
  203. if err := r.Unmarshal(e.Data); err != nil {
  204. panic("TODO: this is bad, what do we do about it?")
  205. }
  206. s.w.Trigger(r.ID, s.applyRequest(r))
  207. case raftpb.EntryConfChange:
  208. var cc raftpb.ConfChange
  209. if err := cc.Unmarshal(e.Data); err != nil {
  210. panic("TODO: this is bad, what do we do about it?")
  211. }
  212. s.applyConfChange(cc)
  213. s.w.Trigger(cc.ID, nil)
  214. default:
  215. panic("unexpected entry type")
  216. }
  217. atomic.StoreUint64(&s.raftIndex, e.Index)
  218. atomic.StoreUint64(&s.raftTerm, e.Term)
  219. appliedi = e.Index
  220. }
  221. if rd.SoftState != nil {
  222. nodes = rd.SoftState.Nodes
  223. if rd.RaftState == raft.StateLeader {
  224. syncC = s.syncTicker
  225. } else {
  226. syncC = nil
  227. }
  228. if rd.SoftState.ShouldStop {
  229. s.Stop()
  230. return
  231. }
  232. }
  233. if rd.Snapshot.Index > snapi {
  234. snapi = rd.Snapshot.Index
  235. }
  236. // recover from snapshot if it is more updated than current applied
  237. if rd.Snapshot.Index > appliedi {
  238. if err := s.store.Recovery(rd.Snapshot.Data); err != nil {
  239. panic("TODO: this is bad, what do we do about it?")
  240. }
  241. appliedi = rd.Snapshot.Index
  242. }
  243. if appliedi-snapi > s.snapCount {
  244. s.snapshot(appliedi, nodes)
  245. snapi = appliedi
  246. }
  247. case <-syncC:
  248. s.sync(defaultSyncTimeout)
  249. case <-s.done:
  250. return
  251. }
  252. }
  253. }
  254. // Stop stops the server, and shuts down the running goroutine. Stop should be
  255. // called after a Start(s), otherwise it will block forever.
  256. func (s *EtcdServer) Stop() {
  257. s.node.Stop()
  258. close(s.done)
  259. }
  260. // Do interprets r and performs an operation on s.store according to r.Method
  261. // and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
  262. // Quorum == true, r will be sent through consensus before performing its
  263. // respective operation. Do will block until an action is performed or there is
  264. // an error.
  265. func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
  266. if r.ID == 0 {
  267. panic("r.Id cannot be 0")
  268. }
  269. if r.Method == "GET" && r.Quorum {
  270. r.Method = "QGET"
  271. }
  272. switch r.Method {
  273. case "POST", "PUT", "DELETE", "QGET":
  274. data, err := r.Marshal()
  275. if err != nil {
  276. return Response{}, err
  277. }
  278. ch := s.w.Register(r.ID)
  279. s.node.Propose(ctx, data)
  280. select {
  281. case x := <-ch:
  282. resp := x.(Response)
  283. return resp, resp.err
  284. case <-ctx.Done():
  285. s.w.Trigger(r.ID, nil) // GC wait
  286. return Response{}, ctx.Err()
  287. case <-s.done:
  288. return Response{}, ErrStopped
  289. }
  290. case "GET":
  291. switch {
  292. case r.Wait:
  293. wc, err := s.store.Watch(r.Path, r.Recursive, r.Stream, r.Since)
  294. if err != nil {
  295. return Response{}, err
  296. }
  297. return Response{Watcher: wc}, nil
  298. default:
  299. ev, err := s.store.Get(r.Path, r.Recursive, r.Sorted)
  300. if err != nil {
  301. return Response{}, err
  302. }
  303. return Response{Event: ev}, nil
  304. }
  305. default:
  306. return Response{}, ErrUnknownMethod
  307. }
  308. }
  309. func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
  310. // TODO: move Member to protobuf type
  311. b, err := json.Marshal(memb)
  312. if err != nil {
  313. return err
  314. }
  315. cc := raftpb.ConfChange{
  316. ID: GenID(),
  317. Type: raftpb.ConfChangeAddNode,
  318. NodeID: memb.ID,
  319. Context: b,
  320. }
  321. return s.configure(ctx, cc)
  322. }
  323. func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) error {
  324. cc := raftpb.ConfChange{
  325. ID: GenID(),
  326. Type: raftpb.ConfChangeRemoveNode,
  327. NodeID: id,
  328. }
  329. return s.configure(ctx, cc)
  330. }
  331. // Implement the RaftTimer interface
  332. func (s *EtcdServer) Index() uint64 {
  333. return atomic.LoadUint64(&s.raftIndex)
  334. }
  335. func (s *EtcdServer) Term() uint64 {
  336. return atomic.LoadUint64(&s.raftTerm)
  337. }
  338. // configure sends configuration change through consensus then performs it.
  339. // It will block until the change is performed or there is an error.
  340. func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) error {
  341. ch := s.w.Register(cc.ID)
  342. if err := s.node.ProposeConfChange(ctx, cc); err != nil {
  343. log.Printf("configure error: %v", err)
  344. s.w.Trigger(cc.ID, nil)
  345. return err
  346. }
  347. select {
  348. case <-ch:
  349. return nil
  350. case <-ctx.Done():
  351. s.w.Trigger(cc.ID, nil) // GC wait
  352. return ctx.Err()
  353. case <-s.done:
  354. return ErrStopped
  355. }
  356. }
  357. // sync proposes a SYNC request and is non-blocking.
  358. // This makes no guarantee that the request will be proposed or performed.
  359. // The request will be cancelled after the given timeout.
  360. func (s *EtcdServer) sync(timeout time.Duration) {
  361. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  362. req := pb.Request{
  363. Method: "SYNC",
  364. ID: GenID(),
  365. Time: time.Now().UnixNano(),
  366. }
  367. data, err := req.Marshal()
  368. if err != nil {
  369. log.Printf("marshal request %#v error: %v", req, err)
  370. return
  371. }
  372. // There is no promise that node has leader when do SYNC request,
  373. // so it uses goroutine to propose.
  374. go func() {
  375. s.node.Propose(ctx, data)
  376. cancel()
  377. }()
  378. }
  379. // publish registers server information into the cluster. The information
  380. // is the JSON representation of this server's member struct, updated with the
  381. // static clientURLs of the server.
  382. // The function keeps attempting to register until it succeeds,
  383. // or its server is stopped.
  384. func (s *EtcdServer) publish(retryInterval time.Duration) {
  385. b, err := json.Marshal(s.attributes)
  386. if err != nil {
  387. log.Printf("etcdserver: json marshal error: %v", err)
  388. return
  389. }
  390. req := pb.Request{
  391. ID: GenID(),
  392. Method: "PUT",
  393. Path: Member{ID: s.id}.storeKey() + attributesSuffix,
  394. Val: string(b),
  395. }
  396. for {
  397. ctx, cancel := context.WithTimeout(context.Background(), retryInterval)
  398. _, err := s.Do(ctx, req)
  399. cancel()
  400. switch err {
  401. case nil:
  402. log.Printf("etcdserver: published %+v to the cluster", s.attributes)
  403. return
  404. case ErrStopped:
  405. log.Printf("etcdserver: aborting publish because server is stopped")
  406. return
  407. default:
  408. log.Printf("etcdserver: publish error: %v", err)
  409. }
  410. }
  411. }
  412. func getExpirationTime(r *pb.Request) time.Time {
  413. var t time.Time
  414. if r.Expiration != 0 {
  415. t = time.Unix(0, r.Expiration)
  416. }
  417. return t
  418. }
  419. // applyRequest interprets r as a call to store.X and returns a Response interpreted
  420. // from store.Event
  421. func (s *EtcdServer) applyRequest(r pb.Request) Response {
  422. f := func(ev *store.Event, err error) Response {
  423. return Response{Event: ev, err: err}
  424. }
  425. expr := getExpirationTime(&r)
  426. switch r.Method {
  427. case "POST":
  428. return f(s.store.Create(r.Path, r.Dir, r.Val, true, expr))
  429. case "PUT":
  430. exists, existsSet := getBool(r.PrevExist)
  431. switch {
  432. case existsSet:
  433. if exists {
  434. return f(s.store.Update(r.Path, r.Val, expr))
  435. }
  436. return f(s.store.Create(r.Path, r.Dir, r.Val, false, expr))
  437. case r.PrevIndex > 0 || r.PrevValue != "":
  438. return f(s.store.CompareAndSwap(r.Path, r.PrevValue, r.PrevIndex, r.Val, expr))
  439. default:
  440. return f(s.store.Set(r.Path, r.Dir, r.Val, expr))
  441. }
  442. case "DELETE":
  443. switch {
  444. case r.PrevIndex > 0 || r.PrevValue != "":
  445. return f(s.store.CompareAndDelete(r.Path, r.PrevValue, r.PrevIndex))
  446. default:
  447. return f(s.store.Delete(r.Path, r.Dir, r.Recursive))
  448. }
  449. case "QGET":
  450. return f(s.store.Get(r.Path, r.Recursive, r.Sorted))
  451. case "SYNC":
  452. s.store.DeleteExpiredKeys(time.Unix(0, r.Time))
  453. return Response{}
  454. default:
  455. // This should never be reached, but just in case:
  456. return Response{err: ErrUnknownMethod}
  457. }
  458. }
  459. func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange) {
  460. s.node.ApplyConfChange(cc)
  461. switch cc.Type {
  462. case raftpb.ConfChangeAddNode:
  463. var m Member
  464. if err := json.Unmarshal(cc.Context, &m); err != nil {
  465. panic("unexpected unmarshal error")
  466. }
  467. if cc.NodeID != m.ID {
  468. panic("unexpected nodeID mismatch")
  469. }
  470. s.ClusterStore.Add(m)
  471. case raftpb.ConfChangeRemoveNode:
  472. s.ClusterStore.Remove(cc.NodeID)
  473. default:
  474. panic("unexpected ConfChange type")
  475. }
  476. }
  477. // TODO: non-blocking snapshot
  478. func (s *EtcdServer) snapshot(snapi uint64, snapnodes []uint64) {
  479. d, err := s.store.Save()
  480. // TODO: current store will never fail to do a snapshot
  481. // what should we do if the store might fail?
  482. if err != nil {
  483. panic("TODO: this is bad, what do we do about it?")
  484. }
  485. s.node.Compact(snapi, snapnodes, d)
  486. s.storage.Cut()
  487. }
  488. func startNode(cfg *ServerConfig) (n raft.Node, w *wal.WAL) {
  489. i := pb.Info{ID: cfg.ID()}
  490. b, err := i.Marshal()
  491. if err != nil {
  492. log.Fatal(err)
  493. }
  494. if w, err = wal.Create(cfg.WALDir(), b); err != nil {
  495. log.Fatal(err)
  496. }
  497. ids := cfg.Cluster.IDs()
  498. peers := make([]raft.Peer, len(ids))
  499. for i, id := range ids {
  500. ctx, err := json.Marshal((*cfg.Cluster)[id])
  501. if err != nil {
  502. log.Fatal(err)
  503. }
  504. peers[i] = raft.Peer{ID: id, Context: ctx}
  505. }
  506. n = raft.StartNode(cfg.ID(), peers, 10, 1)
  507. return
  508. }
  509. func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (n raft.Node, w *wal.WAL) {
  510. var err error
  511. // restart a node from previous wal
  512. if w, err = wal.OpenAtIndex(cfg.WALDir(), index); err != nil {
  513. log.Fatal(err)
  514. }
  515. md, st, ents, err := w.ReadAll()
  516. if err != nil {
  517. log.Fatal(err)
  518. }
  519. var info pb.Info
  520. if err := info.Unmarshal(md); err != nil {
  521. log.Fatal(err)
  522. }
  523. n = raft.RestartNode(info.ID, 10, 1, snapshot, st, ents)
  524. return
  525. }
  526. // TODO: move the function to /id pkg maybe?
  527. // GenID generates a random id that is not equal to 0.
  528. func GenID() (n uint64) {
  529. for n == 0 {
  530. n = uint64(rand.Int63())
  531. }
  532. return
  533. }
  534. func getBool(v *bool) (vv bool, set bool) {
  535. if v == nil {
  536. return false, false
  537. }
  538. return *v, true
  539. }