server.go 16 KB

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