server.go 15 KB

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