server.go 44 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package etcdserver
  15. import (
  16. "encoding/json"
  17. "expvar"
  18. "fmt"
  19. "math"
  20. "math/rand"
  21. "net/http"
  22. "os"
  23. "path"
  24. "regexp"
  25. "sync"
  26. "sync/atomic"
  27. "time"
  28. "github.com/coreos/etcd/alarm"
  29. "github.com/coreos/etcd/auth"
  30. "github.com/coreos/etcd/compactor"
  31. "github.com/coreos/etcd/discovery"
  32. "github.com/coreos/etcd/etcdserver/api"
  33. "github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
  34. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  35. "github.com/coreos/etcd/etcdserver/membership"
  36. "github.com/coreos/etcd/etcdserver/stats"
  37. "github.com/coreos/etcd/lease"
  38. "github.com/coreos/etcd/mvcc"
  39. "github.com/coreos/etcd/mvcc/backend"
  40. "github.com/coreos/etcd/pkg/fileutil"
  41. "github.com/coreos/etcd/pkg/idutil"
  42. "github.com/coreos/etcd/pkg/pbutil"
  43. "github.com/coreos/etcd/pkg/runtime"
  44. "github.com/coreos/etcd/pkg/schedule"
  45. "github.com/coreos/etcd/pkg/types"
  46. "github.com/coreos/etcd/pkg/wait"
  47. "github.com/coreos/etcd/raft"
  48. "github.com/coreos/etcd/raft/raftpb"
  49. "github.com/coreos/etcd/rafthttp"
  50. "github.com/coreos/etcd/snap"
  51. "github.com/coreos/etcd/store"
  52. "github.com/coreos/etcd/version"
  53. "github.com/coreos/etcd/wal"
  54. "github.com/coreos/go-semver/semver"
  55. "github.com/coreos/pkg/capnslog"
  56. "golang.org/x/net/context"
  57. )
  58. const (
  59. DefaultSnapCount = 10000
  60. StoreClusterPrefix = "/0"
  61. StoreKeysPrefix = "/1"
  62. // HealthInterval is the minimum time the cluster should be healthy
  63. // before accepting add member requests.
  64. HealthInterval = 5 * time.Second
  65. purgeFileInterval = 30 * time.Second
  66. // monitorVersionInterval should be smaller than the timeout
  67. // on the connection. Or we will not be able to reuse the connection
  68. // (since it will timeout).
  69. monitorVersionInterval = rafthttp.ConnWriteTimeout - time.Second
  70. databaseFilename = "db"
  71. // max number of in-flight snapshot messages etcdserver allows to have
  72. // This number is more than enough for most clusters with 5 machines.
  73. maxInFlightMsgSnap = 16
  74. releaseDelayAfterSnapshot = 30 * time.Second
  75. )
  76. var (
  77. plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "etcdserver")
  78. storeMemberAttributeRegexp = regexp.MustCompile(path.Join(membership.StoreMembersPrefix, "[[:xdigit:]]{1,16}", "attributes"))
  79. )
  80. func init() {
  81. rand.Seed(time.Now().UnixNano())
  82. expvar.Publish(
  83. "file_descriptor_limit",
  84. expvar.Func(
  85. func() interface{} {
  86. n, _ := runtime.FDLimit()
  87. return n
  88. },
  89. ),
  90. )
  91. }
  92. type Response struct {
  93. Event *store.Event
  94. Watcher store.Watcher
  95. err error
  96. }
  97. type Server interface {
  98. // Start performs any initialization of the Server necessary for it to
  99. // begin serving requests. It must be called before Do or Process.
  100. // Start must be non-blocking; any long-running server functionality
  101. // should be implemented in goroutines.
  102. Start()
  103. // Stop terminates the Server and performs any necessary finalization.
  104. // Do and Process cannot be called after Stop has been invoked.
  105. Stop()
  106. // ID returns the ID of the Server.
  107. ID() types.ID
  108. // Leader returns the ID of the leader Server.
  109. Leader() types.ID
  110. // Do takes a request and attempts to fulfill it, returning a Response.
  111. Do(ctx context.Context, r pb.Request) (Response, error)
  112. // Process takes a raft message and applies it to the server's raft state
  113. // machine, respecting any timeout of the given context.
  114. Process(ctx context.Context, m raftpb.Message) error
  115. // AddMember attempts to add a member into the cluster. It will return
  116. // ErrIDRemoved if member ID is removed from the cluster, or return
  117. // ErrIDExists if member ID exists in the cluster.
  118. AddMember(ctx context.Context, memb membership.Member) error
  119. // RemoveMember attempts to remove a member from the cluster. It will
  120. // return ErrIDRemoved if member ID is removed from the cluster, or return
  121. // ErrIDNotFound if member ID is not in the cluster.
  122. RemoveMember(ctx context.Context, id uint64) error
  123. // UpdateMember attempts to update an existing member in the cluster. It will
  124. // return ErrIDNotFound if the member ID does not exist.
  125. UpdateMember(ctx context.Context, updateMemb membership.Member) error
  126. // ClusterVersion is the cluster-wide minimum major.minor version.
  127. // Cluster version is set to the min version that an etcd member is
  128. // compatible with when first bootstrap.
  129. //
  130. // ClusterVersion is nil until the cluster is bootstrapped (has a quorum).
  131. //
  132. // During a rolling upgrades, the ClusterVersion will be updated
  133. // automatically after a sync. (5 second by default)
  134. //
  135. // The API/raft component can utilize ClusterVersion to determine if
  136. // it can accept a client request or a raft RPC.
  137. // NOTE: ClusterVersion might be nil when etcd 2.1 works with etcd 2.0 and
  138. // the leader is etcd 2.0. etcd 2.0 leader will not update clusterVersion since
  139. // this feature is introduced post 2.0.
  140. ClusterVersion() *semver.Version
  141. }
  142. // EtcdServer is the production implementation of the Server interface
  143. type EtcdServer struct {
  144. // inflightSnapshots holds count the number of snapshots currently inflight.
  145. inflightSnapshots int64 // must use atomic operations to access; keep 64-bit aligned.
  146. appliedIndex uint64 // must use atomic operations to access; keep 64-bit aligned.
  147. committedIndex uint64 // must use atomic operations to access; keep 64-bit aligned.
  148. // consistIndex used to hold the offset of current executing entry
  149. // It is initialized to 0 before executing any entry.
  150. consistIndex consistentIndex // must use atomic operations to access; keep 64-bit aligned.
  151. Cfg *ServerConfig
  152. readych chan struct{}
  153. r raftNode
  154. snapCount uint64
  155. w wait.Wait
  156. stop chan struct{}
  157. done chan struct{}
  158. errorc chan error
  159. id types.ID
  160. attributes membership.Attributes
  161. cluster *membership.RaftCluster
  162. store store.Store
  163. applyV2 ApplierV2
  164. // applyV3 is the applier with auth and quotas
  165. applyV3 applierV3
  166. // applyV3Base is the core applier without auth or quotas
  167. applyV3Base applierV3
  168. applyWait wait.WaitTime
  169. kv mvcc.ConsistentWatchableKV
  170. lessor lease.Lessor
  171. bemu sync.Mutex
  172. be backend.Backend
  173. authStore auth.AuthStore
  174. alarmStore *alarm.AlarmStore
  175. stats *stats.ServerStats
  176. lstats *stats.LeaderStats
  177. SyncTicker <-chan time.Time
  178. // compactor is used to auto-compact the KV.
  179. compactor *compactor.Periodic
  180. // peerRt used to send requests (version, lease) to peers.
  181. peerRt http.RoundTripper
  182. reqIDGen *idutil.Generator
  183. // forceVersionC is used to force the version monitor loop
  184. // to detect the cluster version immediately.
  185. forceVersionC chan struct{}
  186. msgSnapC chan raftpb.Message
  187. // wg is used to wait for the go routines that depends on the server state
  188. // to exit when stopping the server.
  189. wg sync.WaitGroup
  190. }
  191. // NewServer creates a new EtcdServer from the supplied configuration. The
  192. // configuration is considered static for the lifetime of the EtcdServer.
  193. func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
  194. st := store.New(StoreClusterPrefix, StoreKeysPrefix)
  195. var (
  196. w *wal.WAL
  197. n raft.Node
  198. s *raft.MemoryStorage
  199. id types.ID
  200. cl *membership.RaftCluster
  201. )
  202. if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
  203. return nil, fmt.Errorf("cannot access data directory: %v", terr)
  204. }
  205. haveWAL := wal.Exist(cfg.WALDir())
  206. if err = fileutil.TouchDirAll(cfg.SnapDir()); err != nil {
  207. plog.Fatalf("create snapshot directory error: %v", err)
  208. }
  209. ss := snap.New(cfg.SnapDir())
  210. bepath := path.Join(cfg.SnapDir(), databaseFilename)
  211. beExist := fileutil.Exist(bepath)
  212. be := backend.NewDefaultBackend(bepath)
  213. defer func() {
  214. if err != nil {
  215. be.Close()
  216. }
  217. }()
  218. prt, err := rafthttp.NewRoundTripper(cfg.PeerTLSInfo, cfg.peerDialTimeout())
  219. if err != nil {
  220. return nil, err
  221. }
  222. var (
  223. remotes []*membership.Member
  224. snapshot *raftpb.Snapshot
  225. )
  226. switch {
  227. case !haveWAL && !cfg.NewCluster:
  228. if err = cfg.VerifyJoinExisting(); err != nil {
  229. return nil, err
  230. }
  231. cl, err = membership.NewClusterFromURLsMap(cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
  232. if err != nil {
  233. return nil, err
  234. }
  235. existingCluster, gerr := GetClusterFromRemotePeers(getRemotePeerURLs(cl, cfg.Name), prt)
  236. if gerr != nil {
  237. return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", gerr)
  238. }
  239. if err = membership.ValidateClusterAndAssignIDs(cl, existingCluster); err != nil {
  240. return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
  241. }
  242. if !isCompatibleWithCluster(cl, cl.MemberByName(cfg.Name).ID, prt) {
  243. return nil, fmt.Errorf("incompatible with current running cluster")
  244. }
  245. remotes = existingCluster.Members()
  246. cl.SetID(existingCluster.ID())
  247. cl.SetStore(st)
  248. cl.SetBackend(be)
  249. cfg.Print()
  250. id, n, s, w = startNode(cfg, cl, nil)
  251. case !haveWAL && cfg.NewCluster:
  252. if err = cfg.VerifyBootstrap(); err != nil {
  253. return nil, err
  254. }
  255. cl, err = membership.NewClusterFromURLsMap(cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
  256. if err != nil {
  257. return nil, err
  258. }
  259. m := cl.MemberByName(cfg.Name)
  260. if isMemberBootstrapped(cl, cfg.Name, prt, cfg.bootstrapTimeout()) {
  261. return nil, fmt.Errorf("member %s has already been bootstrapped", m.ID)
  262. }
  263. if cfg.ShouldDiscover() {
  264. var str string
  265. str, err = discovery.JoinCluster(cfg.DiscoveryURL, cfg.DiscoveryProxy, m.ID, cfg.InitialPeerURLsMap.String())
  266. if err != nil {
  267. return nil, &DiscoveryError{Op: "join", Err: err}
  268. }
  269. var urlsmap types.URLsMap
  270. urlsmap, err = types.NewURLsMap(str)
  271. if err != nil {
  272. return nil, err
  273. }
  274. if checkDuplicateURL(urlsmap) {
  275. return nil, fmt.Errorf("discovery cluster %s has duplicate url", urlsmap)
  276. }
  277. if cl, err = membership.NewClusterFromURLsMap(cfg.InitialClusterToken, urlsmap); err != nil {
  278. return nil, err
  279. }
  280. }
  281. cl.SetStore(st)
  282. cl.SetBackend(be)
  283. cfg.PrintWithInitial()
  284. id, n, s, w = startNode(cfg, cl, cl.MemberIDs())
  285. case haveWAL:
  286. if err = fileutil.IsDirWriteable(cfg.MemberDir()); err != nil {
  287. return nil, fmt.Errorf("cannot write to member directory: %v", err)
  288. }
  289. if err = fileutil.IsDirWriteable(cfg.WALDir()); err != nil {
  290. return nil, fmt.Errorf("cannot write to WAL directory: %v", err)
  291. }
  292. if cfg.ShouldDiscover() {
  293. plog.Warningf("discovery token ignored since a cluster has already been initialized. Valid log found at %q", cfg.WALDir())
  294. }
  295. snapshot, err = ss.Load()
  296. if err != nil && err != snap.ErrNoSnapshot {
  297. return nil, err
  298. }
  299. if snapshot != nil {
  300. if err = st.Recovery(snapshot.Data); err != nil {
  301. plog.Panicf("recovered store from snapshot error: %v", err)
  302. }
  303. plog.Infof("recovered store from snapshot at index %d", snapshot.Metadata.Index)
  304. }
  305. cfg.Print()
  306. if !cfg.ForceNewCluster {
  307. id, cl, n, s, w = restartNode(cfg, snapshot)
  308. } else {
  309. id, cl, n, s, w = restartAsStandaloneNode(cfg, snapshot)
  310. }
  311. cl.SetStore(st)
  312. cl.SetBackend(be)
  313. cl.Recover(api.UpdateCapability)
  314. if cl.Version() != nil && !cl.Version().LessThan(semver.Version{Major: 3}) && !beExist {
  315. os.RemoveAll(bepath)
  316. return nil, fmt.Errorf("database file (%v) of the backend is missing", bepath)
  317. }
  318. default:
  319. return nil, fmt.Errorf("unsupported bootstrap config")
  320. }
  321. if terr := fileutil.TouchDirAll(cfg.MemberDir()); terr != nil {
  322. return nil, fmt.Errorf("cannot access member directory: %v", terr)
  323. }
  324. sstats := &stats.ServerStats{
  325. Name: cfg.Name,
  326. ID: id.String(),
  327. }
  328. sstats.Initialize()
  329. lstats := stats.NewLeaderStats(id.String())
  330. srv = &EtcdServer{
  331. readych: make(chan struct{}),
  332. Cfg: cfg,
  333. snapCount: cfg.SnapCount,
  334. errorc: make(chan error, 1),
  335. store: st,
  336. r: raftNode{
  337. Node: n,
  338. ticker: time.Tick(time.Duration(cfg.TickMs) * time.Millisecond),
  339. raftStorage: s,
  340. storage: NewStorage(w, ss),
  341. },
  342. id: id,
  343. attributes: membership.Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
  344. cluster: cl,
  345. stats: sstats,
  346. lstats: lstats,
  347. SyncTicker: time.Tick(500 * time.Millisecond),
  348. peerRt: prt,
  349. reqIDGen: idutil.NewGenerator(uint16(id), time.Now()),
  350. forceVersionC: make(chan struct{}),
  351. msgSnapC: make(chan raftpb.Message, maxInFlightMsgSnap),
  352. }
  353. srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
  354. srv.be = be
  355. minTTL := time.Duration((3*cfg.ElectionTicks)/2) * time.Duration(cfg.TickMs) * time.Millisecond
  356. // always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
  357. // If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
  358. srv.lessor = lease.NewLessor(srv.be, int64(math.Ceil(minTTL.Seconds())))
  359. srv.kv = mvcc.New(srv.be, srv.lessor, &srv.consistIndex)
  360. if beExist {
  361. kvindex := srv.kv.ConsistentIndex()
  362. // TODO: remove kvindex != 0 checking when we do not expect users to upgrade
  363. // etcd from pre-3.0 release.
  364. if snapshot != nil && kvindex < snapshot.Metadata.Index {
  365. if kvindex != 0 {
  366. return nil, fmt.Errorf("database file (%v index %d) does not match with snapshot (index %d).", bepath, kvindex, snapshot.Metadata.Index)
  367. }
  368. plog.Warningf("consistent index never saved (snapshot index=%d)", snapshot.Metadata.Index)
  369. }
  370. }
  371. srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex())
  372. srv.authStore = auth.NewAuthStore(srv.be)
  373. if h := cfg.AutoCompactionRetention; h != 0 {
  374. srv.compactor = compactor.NewPeriodic(h, srv.kv, srv)
  375. srv.compactor.Run()
  376. }
  377. srv.applyV3Base = &applierV3backend{srv}
  378. if err = srv.restoreAlarms(); err != nil {
  379. return nil, err
  380. }
  381. // TODO: move transport initialization near the definition of remote
  382. tr := &rafthttp.Transport{
  383. TLSInfo: cfg.PeerTLSInfo,
  384. DialTimeout: cfg.peerDialTimeout(),
  385. ID: id,
  386. URLs: cfg.PeerURLs,
  387. ClusterID: cl.ID(),
  388. Raft: srv,
  389. Snapshotter: ss,
  390. ServerStats: sstats,
  391. LeaderStats: lstats,
  392. ErrorC: srv.errorc,
  393. }
  394. if err = tr.Start(); err != nil {
  395. return nil, err
  396. }
  397. // add all remotes into transport
  398. for _, m := range remotes {
  399. if m.ID != id {
  400. tr.AddRemote(m.ID, m.PeerURLs)
  401. }
  402. }
  403. for _, m := range cl.Members() {
  404. if m.ID != id {
  405. tr.AddPeer(m.ID, m.PeerURLs)
  406. }
  407. }
  408. srv.r.transport = tr
  409. return srv, nil
  410. }
  411. // Start prepares and starts server in a new goroutine. It is no longer safe to
  412. // modify a server's fields after it has been sent to Start.
  413. // It also starts a goroutine to publish its server information.
  414. func (s *EtcdServer) Start() {
  415. s.start()
  416. go s.publish(s.Cfg.ReqTimeout())
  417. go s.purgeFile()
  418. go monitorFileDescriptor(s.done)
  419. go s.monitorVersions()
  420. }
  421. // start prepares and starts server in a new goroutine. It is no longer safe to
  422. // modify a server's fields after it has been sent to Start.
  423. // This function is just used for testing.
  424. func (s *EtcdServer) start() {
  425. if s.snapCount == 0 {
  426. plog.Infof("set snapshot count to default %d", DefaultSnapCount)
  427. s.snapCount = DefaultSnapCount
  428. }
  429. s.w = wait.New()
  430. s.applyWait = wait.NewTimeList()
  431. s.done = make(chan struct{})
  432. s.stop = make(chan struct{})
  433. if s.ClusterVersion() != nil {
  434. plog.Infof("starting server... [version: %v, cluster version: %v]", version.Version, version.Cluster(s.ClusterVersion().String()))
  435. } else {
  436. plog.Infof("starting server... [version: %v, cluster version: to_be_decided]", version.Version)
  437. }
  438. // TODO: if this is an empty log, writes all peer infos
  439. // into the first entry
  440. go s.run()
  441. }
  442. func (s *EtcdServer) purgeFile() {
  443. var serrc, werrc <-chan error
  444. if s.Cfg.MaxSnapFiles > 0 {
  445. serrc = fileutil.PurgeFile(s.Cfg.SnapDir(), "snap", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
  446. }
  447. if s.Cfg.MaxWALFiles > 0 {
  448. werrc = fileutil.PurgeFile(s.Cfg.WALDir(), "wal", s.Cfg.MaxWALFiles, purgeFileInterval, s.done)
  449. }
  450. select {
  451. case e := <-werrc:
  452. plog.Fatalf("failed to purge wal file %v", e)
  453. case e := <-serrc:
  454. plog.Fatalf("failed to purge snap file %v", e)
  455. case <-s.done:
  456. return
  457. }
  458. }
  459. func (s *EtcdServer) ID() types.ID { return s.id }
  460. func (s *EtcdServer) Cluster() *membership.RaftCluster { return s.cluster }
  461. func (s *EtcdServer) RaftHandler() http.Handler { return s.r.transport.Handler() }
  462. func (s *EtcdServer) Lessor() lease.Lessor { return s.lessor }
  463. func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
  464. if s.cluster.IsIDRemoved(types.ID(m.From)) {
  465. plog.Warningf("reject message from removed member %s", types.ID(m.From).String())
  466. return httptypes.NewHTTPError(http.StatusForbidden, "cannot process message from removed member")
  467. }
  468. if m.Type == raftpb.MsgApp {
  469. s.stats.RecvAppendReq(types.ID(m.From).String(), m.Size())
  470. }
  471. return s.r.Step(ctx, m)
  472. }
  473. func (s *EtcdServer) IsIDRemoved(id uint64) bool { return s.cluster.IsIDRemoved(types.ID(id)) }
  474. func (s *EtcdServer) ReportUnreachable(id uint64) { s.r.ReportUnreachable(id) }
  475. // ReportSnapshot reports snapshot sent status to the raft state machine,
  476. // and clears the used snapshot from the snapshot store.
  477. func (s *EtcdServer) ReportSnapshot(id uint64, status raft.SnapshotStatus) {
  478. s.r.ReportSnapshot(id, status)
  479. }
  480. type etcdProgress struct {
  481. confState raftpb.ConfState
  482. snapi uint64
  483. appliedi uint64
  484. }
  485. func (s *EtcdServer) run() {
  486. snap, err := s.r.raftStorage.Snapshot()
  487. if err != nil {
  488. plog.Panicf("get snapshot from raft storage error: %v", err)
  489. }
  490. s.r.start(s)
  491. // asynchronously accept apply packets, dispatch progress in-order
  492. sched := schedule.NewFIFOScheduler()
  493. ep := etcdProgress{
  494. confState: snap.Metadata.ConfState,
  495. snapi: snap.Metadata.Index,
  496. appliedi: snap.Metadata.Index,
  497. }
  498. defer func() {
  499. sched.Stop()
  500. // wait for snapshots before closing raft so wal stays open
  501. s.wg.Wait()
  502. // must stop raft after scheduler-- etcdserver can leak rafthttp pipelines
  503. // by adding a peer after raft stops the transport
  504. s.r.stop()
  505. // kv, lessor and backend can be nil if running without v3 enabled
  506. // or running unit tests.
  507. if s.lessor != nil {
  508. s.lessor.Stop()
  509. }
  510. if s.kv != nil {
  511. s.kv.Close()
  512. }
  513. if s.be != nil {
  514. s.be.Close()
  515. }
  516. if s.compactor != nil {
  517. s.compactor.Stop()
  518. }
  519. close(s.done)
  520. }()
  521. var expiredLeaseC <-chan []*lease.Lease
  522. if s.lessor != nil {
  523. expiredLeaseC = s.lessor.ExpiredLeasesC()
  524. }
  525. for {
  526. select {
  527. case ap := <-s.r.apply():
  528. var ci uint64
  529. if len(ap.entries) != 0 {
  530. ci = ap.entries[len(ap.entries)-1].Index
  531. }
  532. if ap.snapshot.Metadata.Index > ci {
  533. ci = ap.snapshot.Metadata.Index
  534. }
  535. if ci != 0 {
  536. s.setCommittedIndex(ci)
  537. }
  538. f := func(context.Context) { s.applyAll(&ep, &ap) }
  539. sched.Schedule(f)
  540. case leases := <-expiredLeaseC:
  541. go func() {
  542. for _, l := range leases {
  543. s.LeaseRevoke(context.TODO(), &pb.LeaseRevokeRequest{ID: int64(l.ID)})
  544. }
  545. }()
  546. case err := <-s.errorc:
  547. plog.Errorf("%s", err)
  548. plog.Infof("the data-dir used by this member must be removed.")
  549. return
  550. case <-s.stop:
  551. return
  552. }
  553. }
  554. }
  555. func (s *EtcdServer) applyAll(ep *etcdProgress, apply *apply) {
  556. s.applySnapshot(ep, apply)
  557. st := time.Now()
  558. s.applyEntries(ep, apply)
  559. d := time.Since(st)
  560. entriesNum := len(apply.entries)
  561. if entriesNum != 0 && d > time.Duration(entriesNum)*warnApplyDuration {
  562. plog.Warningf("apply entries took too long [%v for %d entries]", d, len(apply.entries))
  563. plog.Warningf("avoid queries with large range/delete range!")
  564. }
  565. proposalsApplied.Set(float64(ep.appliedi))
  566. s.applyWait.Trigger(ep.appliedi)
  567. // wait for the raft routine to finish the disk writes before triggering a
  568. // snapshot. or applied index might be greater than the last index in raft
  569. // storage, since the raft routine might be slower than apply routine.
  570. <-apply.raftDone
  571. s.triggerSnapshot(ep)
  572. select {
  573. // snapshot requested via send()
  574. case m := <-s.msgSnapC:
  575. merged := s.createMergedSnapshotMessage(m, ep.appliedi, ep.confState)
  576. s.sendMergedSnap(merged)
  577. default:
  578. }
  579. }
  580. func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
  581. if raft.IsEmptySnap(apply.snapshot) {
  582. return
  583. }
  584. plog.Infof("applying snapshot at index %d...", ep.snapi)
  585. defer plog.Infof("finished applying incoming snapshot at index %d", ep.snapi)
  586. if apply.snapshot.Metadata.Index <= ep.appliedi {
  587. plog.Panicf("snapshot index [%d] should > appliedi[%d] + 1",
  588. apply.snapshot.Metadata.Index, ep.appliedi)
  589. }
  590. snapfn, err := s.r.storage.DBFilePath(apply.snapshot.Metadata.Index)
  591. if err != nil {
  592. plog.Panicf("get database snapshot file path error: %v", err)
  593. }
  594. fn := path.Join(s.Cfg.SnapDir(), databaseFilename)
  595. if err := os.Rename(snapfn, fn); err != nil {
  596. plog.Panicf("rename snapshot file error: %v", err)
  597. }
  598. newbe := backend.NewDefaultBackend(fn)
  599. // always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
  600. // If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
  601. if s.lessor != nil {
  602. plog.Info("recovering lessor...")
  603. s.lessor.Recover(newbe, s.kv)
  604. plog.Info("finished recovering lessor")
  605. }
  606. plog.Info("restoring mvcc store...")
  607. if err := s.kv.Restore(newbe); err != nil {
  608. plog.Panicf("restore KV error: %v", err)
  609. }
  610. s.consistIndex.setConsistentIndex(s.kv.ConsistentIndex())
  611. plog.Info("finished restoring mvcc store")
  612. // Closing old backend might block until all the txns
  613. // on the backend are finished.
  614. // We do not want to wait on closing the old backend.
  615. s.bemu.Lock()
  616. oldbe := s.be
  617. go func() {
  618. plog.Info("closing old backend...")
  619. defer plog.Info("finished closing old backend")
  620. if err := oldbe.Close(); err != nil {
  621. plog.Panicf("close backend error: %v", err)
  622. }
  623. }()
  624. s.be = newbe
  625. s.bemu.Unlock()
  626. plog.Info("recovering alarms...")
  627. if err := s.restoreAlarms(); err != nil {
  628. plog.Panicf("restore alarms error: %v", err)
  629. }
  630. plog.Info("finished recovering alarms")
  631. if s.authStore != nil {
  632. plog.Info("recovering auth store...")
  633. s.authStore.Recover(newbe)
  634. plog.Info("finished recovering auth store")
  635. }
  636. plog.Info("recovering store v2...")
  637. if err := s.store.Recovery(apply.snapshot.Data); err != nil {
  638. plog.Panicf("recovery store error: %v", err)
  639. }
  640. plog.Info("finished recovering store v2")
  641. s.cluster.SetBackend(s.be)
  642. plog.Info("recovering cluster configuration...")
  643. s.cluster.Recover(api.UpdateCapability)
  644. plog.Info("finished recovering cluster configuration")
  645. plog.Info("removing old peers from network...")
  646. // recover raft transport
  647. s.r.transport.RemoveAllPeers()
  648. plog.Info("finished removing old peers from network")
  649. plog.Info("adding peers from new cluster configuration into network...")
  650. for _, m := range s.cluster.Members() {
  651. if m.ID == s.ID() {
  652. continue
  653. }
  654. s.r.transport.AddPeer(m.ID, m.PeerURLs)
  655. }
  656. plog.Info("finished adding peers from new cluster configuration into network...")
  657. ep.appliedi = apply.snapshot.Metadata.Index
  658. ep.snapi = ep.appliedi
  659. ep.confState = apply.snapshot.Metadata.ConfState
  660. }
  661. func (s *EtcdServer) applyEntries(ep *etcdProgress, apply *apply) {
  662. if len(apply.entries) == 0 {
  663. return
  664. }
  665. firsti := apply.entries[0].Index
  666. if firsti > ep.appliedi+1 {
  667. plog.Panicf("first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, ep.appliedi)
  668. }
  669. var ents []raftpb.Entry
  670. if ep.appliedi+1-firsti < uint64(len(apply.entries)) {
  671. ents = apply.entries[ep.appliedi+1-firsti:]
  672. }
  673. if len(ents) == 0 {
  674. return
  675. }
  676. var shouldstop bool
  677. if ep.appliedi, shouldstop = s.apply(ents, &ep.confState); shouldstop {
  678. go s.stopWithDelay(10*100*time.Millisecond, fmt.Errorf("the member has been permanently removed from the cluster"))
  679. }
  680. }
  681. func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
  682. if ep.appliedi-ep.snapi <= s.snapCount {
  683. return
  684. }
  685. plog.Infof("start to snapshot (applied: %d, lastsnap: %d)", ep.appliedi, ep.snapi)
  686. s.snapshot(ep.appliedi, ep.confState)
  687. ep.snapi = ep.appliedi
  688. }
  689. func (s *EtcdServer) isMultiNode() bool {
  690. return s.cluster != nil && len(s.cluster.MemberIDs()) > 1
  691. }
  692. func (s *EtcdServer) isLeader() bool {
  693. return uint64(s.ID()) == s.Lead()
  694. }
  695. // transferLeadership transfers the leader to the given transferee.
  696. // TODO: maybe expose to client?
  697. func (s *EtcdServer) transferLeadership(ctx context.Context, lead, transferee uint64) error {
  698. now := time.Now()
  699. interval := time.Duration(s.Cfg.TickMs) * time.Millisecond
  700. plog.Infof("%s starts leadership transfer from %s to %s", s.ID(), types.ID(lead), types.ID(transferee))
  701. s.r.TransferLeadership(ctx, lead, transferee)
  702. for s.Lead() != transferee {
  703. select {
  704. case <-ctx.Done(): // time out
  705. return ErrTimeoutLeaderTransfer
  706. case <-time.After(interval):
  707. }
  708. }
  709. // TODO: drain all requests, or drop all messages to the old leader
  710. plog.Infof("%s finished leadership transfer from %s to %s (took %v)", s.ID(), types.ID(lead), types.ID(transferee), time.Since(now))
  711. return nil
  712. }
  713. // TransferLeadership transfers the leader to the chosen transferee.
  714. func (s *EtcdServer) TransferLeadership() error {
  715. if !s.isLeader() {
  716. plog.Printf("skipped leadership transfer for stopping non-leader member")
  717. return nil
  718. }
  719. if !s.isMultiNode() {
  720. plog.Printf("skipped leadership transfer for single member cluster")
  721. return nil
  722. }
  723. transferee, ok := longestConnected(s.r.transport, s.cluster.MemberIDs())
  724. if !ok {
  725. return ErrUnhealthy
  726. }
  727. tm := s.Cfg.ReqTimeout()
  728. ctx, cancel := context.WithTimeout(context.TODO(), tm)
  729. err := s.transferLeadership(ctx, s.Lead(), uint64(transferee))
  730. cancel()
  731. return err
  732. }
  733. // HardStop stops the server without coordination with other members in the cluster.
  734. func (s *EtcdServer) HardStop() {
  735. select {
  736. case s.stop <- struct{}{}:
  737. case <-s.done:
  738. return
  739. }
  740. <-s.done
  741. }
  742. // Stop stops the server gracefully, and shuts down the running goroutine.
  743. // Stop should be called after a Start(s), otherwise it will block forever.
  744. // When stopping leader, Stop transfers its leadership to one of its peers
  745. // before stopping the server.
  746. func (s *EtcdServer) Stop() {
  747. if err := s.TransferLeadership(); err != nil {
  748. plog.Warningf("%s failed to transfer leadership (%v)", s.ID(), err)
  749. }
  750. s.HardStop()
  751. }
  752. // ReadyNotify returns a channel that will be closed when the server
  753. // is ready to serve client requests
  754. func (s *EtcdServer) ReadyNotify() <-chan struct{} { return s.readych }
  755. func (s *EtcdServer) stopWithDelay(d time.Duration, err error) {
  756. select {
  757. case <-time.After(d):
  758. case <-s.done:
  759. }
  760. select {
  761. case s.errorc <- err:
  762. default:
  763. }
  764. }
  765. // StopNotify returns a channel that receives a empty struct
  766. // when the server is stopped.
  767. func (s *EtcdServer) StopNotify() <-chan struct{} { return s.done }
  768. func (s *EtcdServer) SelfStats() []byte { return s.stats.JSON() }
  769. func (s *EtcdServer) LeaderStats() []byte {
  770. lead := atomic.LoadUint64(&s.r.lead)
  771. if lead != uint64(s.id) {
  772. return nil
  773. }
  774. return s.lstats.JSON()
  775. }
  776. func (s *EtcdServer) StoreStats() []byte { return s.store.JsonStats() }
  777. func (s *EtcdServer) AddMember(ctx context.Context, memb membership.Member) error {
  778. if s.Cfg.StrictReconfigCheck {
  779. // by default StrictReconfigCheck is enabled; reject new members if unhealthy
  780. if !s.cluster.IsReadyToAddNewMember() {
  781. plog.Warningf("not enough started members, rejecting member add %+v", memb)
  782. return ErrNotEnoughStartedMembers
  783. }
  784. if !isConnectedFullySince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), s.cluster.Members()) {
  785. plog.Warningf("not healthy for reconfigure, rejecting member add %+v", memb)
  786. return ErrUnhealthy
  787. }
  788. }
  789. // TODO: move Member to protobuf type
  790. b, err := json.Marshal(memb)
  791. if err != nil {
  792. return err
  793. }
  794. cc := raftpb.ConfChange{
  795. Type: raftpb.ConfChangeAddNode,
  796. NodeID: uint64(memb.ID),
  797. Context: b,
  798. }
  799. return s.configure(ctx, cc)
  800. }
  801. func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) error {
  802. // by default StrictReconfigCheck is enabled; reject removal if leads to quorum loss
  803. if err := s.mayRemoveMember(types.ID(id)); err != nil {
  804. return err
  805. }
  806. cc := raftpb.ConfChange{
  807. Type: raftpb.ConfChangeRemoveNode,
  808. NodeID: id,
  809. }
  810. return s.configure(ctx, cc)
  811. }
  812. func (s *EtcdServer) mayRemoveMember(id types.ID) error {
  813. if !s.Cfg.StrictReconfigCheck {
  814. return nil
  815. }
  816. if !s.cluster.IsReadyToRemoveMember(uint64(id)) {
  817. plog.Warningf("not enough started members, rejecting remove member %s", id)
  818. return ErrNotEnoughStartedMembers
  819. }
  820. // downed member is safe to remove since it's not part of the active quorum
  821. if t := s.r.transport.ActiveSince(id); id != s.ID() && t.IsZero() {
  822. return nil
  823. }
  824. // protect quorum if some members are down
  825. m := s.cluster.Members()
  826. active := numConnectedSince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), m)
  827. if (active - 1) < 1+((len(m)-1)/2) {
  828. plog.Warningf("reconfigure breaks active quorum, rejecting remove member %s", id)
  829. return ErrUnhealthy
  830. }
  831. return nil
  832. }
  833. func (s *EtcdServer) UpdateMember(ctx context.Context, memb membership.Member) error {
  834. b, err := json.Marshal(memb)
  835. if err != nil {
  836. return err
  837. }
  838. cc := raftpb.ConfChange{
  839. Type: raftpb.ConfChangeUpdateNode,
  840. NodeID: uint64(memb.ID),
  841. Context: b,
  842. }
  843. return s.configure(ctx, cc)
  844. }
  845. // Implement the RaftTimer interface
  846. func (s *EtcdServer) Index() uint64 { return atomic.LoadUint64(&s.r.index) }
  847. func (s *EtcdServer) Term() uint64 { return atomic.LoadUint64(&s.r.term) }
  848. // Lead is only for testing purposes.
  849. // TODO: add Raft server interface to expose raft related info:
  850. // Index, Term, Lead, Committed, Applied, LastIndex, etc.
  851. func (s *EtcdServer) Lead() uint64 { return atomic.LoadUint64(&s.r.lead) }
  852. func (s *EtcdServer) Leader() types.ID { return types.ID(s.Lead()) }
  853. func (s *EtcdServer) IsPprofEnabled() bool { return s.Cfg.EnablePprof }
  854. // configure sends a configuration change through consensus and
  855. // then waits for it to be applied to the server. It
  856. // will block until the change is performed or there is an error.
  857. func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) error {
  858. cc.ID = s.reqIDGen.Next()
  859. ch := s.w.Register(cc.ID)
  860. start := time.Now()
  861. if err := s.r.ProposeConfChange(ctx, cc); err != nil {
  862. s.w.Trigger(cc.ID, nil)
  863. return err
  864. }
  865. select {
  866. case x := <-ch:
  867. if err, ok := x.(error); ok {
  868. return err
  869. }
  870. if x != nil {
  871. plog.Panicf("return type should always be error")
  872. }
  873. return nil
  874. case <-ctx.Done():
  875. s.w.Trigger(cc.ID, nil) // GC wait
  876. return s.parseProposeCtxErr(ctx.Err(), start)
  877. case <-s.done:
  878. return ErrStopped
  879. }
  880. }
  881. // sync proposes a SYNC request and is non-blocking.
  882. // This makes no guarantee that the request will be proposed or performed.
  883. // The request will be canceled after the given timeout.
  884. func (s *EtcdServer) sync(timeout time.Duration) {
  885. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  886. req := pb.Request{
  887. Method: "SYNC",
  888. ID: s.reqIDGen.Next(),
  889. Time: time.Now().UnixNano(),
  890. }
  891. data := pbutil.MustMarshal(&req)
  892. // There is no promise that node has leader when do SYNC request,
  893. // so it uses goroutine to propose.
  894. go func() {
  895. s.r.Propose(ctx, data)
  896. cancel()
  897. }()
  898. }
  899. // publish registers server information into the cluster. The information
  900. // is the JSON representation of this server's member struct, updated with the
  901. // static clientURLs of the server.
  902. // The function keeps attempting to register until it succeeds,
  903. // or its server is stopped.
  904. func (s *EtcdServer) publish(timeout time.Duration) {
  905. b, err := json.Marshal(s.attributes)
  906. if err != nil {
  907. plog.Panicf("json marshal error: %v", err)
  908. return
  909. }
  910. req := pb.Request{
  911. Method: "PUT",
  912. Path: membership.MemberAttributesStorePath(s.id),
  913. Val: string(b),
  914. }
  915. for {
  916. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  917. _, err := s.Do(ctx, req)
  918. cancel()
  919. switch err {
  920. case nil:
  921. close(s.readych)
  922. plog.Infof("published %+v to cluster %s", s.attributes, s.cluster.ID())
  923. return
  924. case ErrStopped:
  925. plog.Infof("aborting publish because server is stopped")
  926. return
  927. default:
  928. plog.Errorf("publish error: %v", err)
  929. }
  930. }
  931. }
  932. // TODO: move this function into raft.go
  933. func (s *EtcdServer) send(ms []raftpb.Message) {
  934. sentAppResp := false
  935. for i := len(ms) - 1; i >= 0; i-- {
  936. if s.cluster.IsIDRemoved(types.ID(ms[i].To)) {
  937. ms[i].To = 0
  938. }
  939. if ms[i].Type == raftpb.MsgAppResp {
  940. if sentAppResp {
  941. ms[i].To = 0
  942. } else {
  943. sentAppResp = true
  944. }
  945. }
  946. if ms[i].Type == raftpb.MsgSnap {
  947. // There are two separate data store: the store for v2, and the KV for v3.
  948. // The msgSnap only contains the most recent snapshot of store without KV.
  949. // So we need to redirect the msgSnap to etcd server main loop for merging in the
  950. // current store snapshot and KV snapshot.
  951. select {
  952. case s.msgSnapC <- ms[i]:
  953. default:
  954. // drop msgSnap if the inflight chan if full.
  955. }
  956. ms[i].To = 0
  957. }
  958. if ms[i].Type == raftpb.MsgHeartbeat {
  959. ok, exceed := s.r.td.Observe(ms[i].To)
  960. if !ok {
  961. // TODO: limit request rate.
  962. plog.Warningf("failed to send out heartbeat on time (exceeded the %dms timeout for %v)", s.Cfg.TickMs, exceed)
  963. plog.Warningf("server is likely overloaded")
  964. }
  965. }
  966. }
  967. s.r.transport.Send(ms)
  968. }
  969. func (s *EtcdServer) sendMergedSnap(merged snap.Message) {
  970. atomic.AddInt64(&s.inflightSnapshots, 1)
  971. s.r.transport.SendSnapshot(merged)
  972. go func() {
  973. select {
  974. case ok := <-merged.CloseNotify():
  975. // delay releasing inflight snapshot for another 30 seconds to
  976. // block log compaction.
  977. // If the follower still fails to catch up, it is probably just too slow
  978. // to catch up. We cannot avoid the snapshot cycle anyway.
  979. if ok {
  980. select {
  981. case <-time.After(releaseDelayAfterSnapshot):
  982. case <-s.done:
  983. }
  984. }
  985. atomic.AddInt64(&s.inflightSnapshots, -1)
  986. case <-s.done:
  987. return
  988. }
  989. }()
  990. }
  991. // apply takes entries received from Raft (after it has been committed) and
  992. // applies them to the current state of the EtcdServer.
  993. // The given entries should not be empty.
  994. func (s *EtcdServer) apply(es []raftpb.Entry, confState *raftpb.ConfState) (uint64, bool) {
  995. var applied uint64
  996. var shouldstop bool
  997. for i := range es {
  998. e := es[i]
  999. switch e.Type {
  1000. case raftpb.EntryNormal:
  1001. s.applyEntryNormal(&e)
  1002. case raftpb.EntryConfChange:
  1003. var cc raftpb.ConfChange
  1004. pbutil.MustUnmarshal(&cc, e.Data)
  1005. removedSelf, err := s.applyConfChange(cc, confState)
  1006. shouldstop = shouldstop || removedSelf
  1007. s.w.Trigger(cc.ID, err)
  1008. default:
  1009. plog.Panicf("entry type should be either EntryNormal or EntryConfChange")
  1010. }
  1011. atomic.StoreUint64(&s.r.index, e.Index)
  1012. atomic.StoreUint64(&s.r.term, e.Term)
  1013. applied = e.Index
  1014. }
  1015. return applied, shouldstop
  1016. }
  1017. // applyEntryNormal apples an EntryNormal type raftpb request to the EtcdServer
  1018. func (s *EtcdServer) applyEntryNormal(e *raftpb.Entry) {
  1019. shouldApplyV3 := false
  1020. if e.Index > s.consistIndex.ConsistentIndex() {
  1021. // set the consistent index of current executing entry
  1022. s.consistIndex.setConsistentIndex(e.Index)
  1023. shouldApplyV3 = true
  1024. }
  1025. defer s.setAppliedIndex(e.Index)
  1026. // raft state machine may generate noop entry when leader confirmation.
  1027. // skip it in advance to avoid some potential bug in the future
  1028. if len(e.Data) == 0 {
  1029. select {
  1030. case s.forceVersionC <- struct{}{}:
  1031. default:
  1032. }
  1033. return
  1034. }
  1035. var raftReq pb.InternalRaftRequest
  1036. if !pbutil.MaybeUnmarshal(&raftReq, e.Data) { // backward compatible
  1037. var r pb.Request
  1038. pbutil.MustUnmarshal(&r, e.Data)
  1039. s.w.Trigger(r.ID, s.applyV2Request(&r))
  1040. return
  1041. }
  1042. if raftReq.V2 != nil {
  1043. req := raftReq.V2
  1044. s.w.Trigger(req.ID, s.applyV2Request(req))
  1045. return
  1046. }
  1047. // do not re-apply applied entries.
  1048. if !shouldApplyV3 {
  1049. return
  1050. }
  1051. id := raftReq.ID
  1052. if id == 0 {
  1053. id = raftReq.Header.ID
  1054. }
  1055. var ar *applyResult
  1056. needResult := s.w.IsRegistered(id)
  1057. if needResult || !noSideEffect(&raftReq) {
  1058. if !needResult && raftReq.Txn != nil {
  1059. removeNeedlessRangeReqs(raftReq.Txn)
  1060. }
  1061. ar = s.applyV3.Apply(&raftReq)
  1062. }
  1063. if ar == nil {
  1064. return
  1065. }
  1066. if ar.err != ErrNoSpace || len(s.alarmStore.Get(pb.AlarmType_NOSPACE)) > 0 {
  1067. s.w.Trigger(id, ar)
  1068. return
  1069. }
  1070. plog.Errorf("applying raft message exceeded backend quota")
  1071. go func() {
  1072. a := &pb.AlarmRequest{
  1073. MemberID: uint64(s.ID()),
  1074. Action: pb.AlarmRequest_ACTIVATE,
  1075. Alarm: pb.AlarmType_NOSPACE,
  1076. }
  1077. r := pb.InternalRaftRequest{Alarm: a}
  1078. s.processInternalRaftRequest(context.TODO(), r)
  1079. s.w.Trigger(id, ar)
  1080. }()
  1081. }
  1082. // applyConfChange applies a ConfChange to the server. It is only
  1083. // invoked with a ConfChange that has already passed through Raft
  1084. func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.ConfState) (bool, error) {
  1085. if err := s.cluster.ValidateConfigurationChange(cc); err != nil {
  1086. cc.NodeID = raft.None
  1087. s.r.ApplyConfChange(cc)
  1088. return false, err
  1089. }
  1090. *confState = *s.r.ApplyConfChange(cc)
  1091. switch cc.Type {
  1092. case raftpb.ConfChangeAddNode:
  1093. m := new(membership.Member)
  1094. if err := json.Unmarshal(cc.Context, m); err != nil {
  1095. plog.Panicf("unmarshal member should never fail: %v", err)
  1096. }
  1097. if cc.NodeID != uint64(m.ID) {
  1098. plog.Panicf("nodeID should always be equal to member ID")
  1099. }
  1100. s.cluster.AddMember(m)
  1101. if m.ID != s.id {
  1102. s.r.transport.AddPeer(m.ID, m.PeerURLs)
  1103. }
  1104. case raftpb.ConfChangeRemoveNode:
  1105. id := types.ID(cc.NodeID)
  1106. s.cluster.RemoveMember(id)
  1107. if id == s.id {
  1108. return true, nil
  1109. }
  1110. s.r.transport.RemovePeer(id)
  1111. case raftpb.ConfChangeUpdateNode:
  1112. m := new(membership.Member)
  1113. if err := json.Unmarshal(cc.Context, m); err != nil {
  1114. plog.Panicf("unmarshal member should never fail: %v", err)
  1115. }
  1116. if cc.NodeID != uint64(m.ID) {
  1117. plog.Panicf("nodeID should always be equal to member ID")
  1118. }
  1119. s.cluster.UpdateRaftAttributes(m.ID, m.RaftAttributes)
  1120. if m.ID != s.id {
  1121. s.r.transport.UpdatePeer(m.ID, m.PeerURLs)
  1122. }
  1123. }
  1124. return false, nil
  1125. }
  1126. // TODO: non-blocking snapshot
  1127. func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
  1128. clone := s.store.Clone()
  1129. // commit kv to write metadata (for example: consistent index) to disk.
  1130. // KV().commit() updates the consistent index in backend.
  1131. // All operations that update consistent index must be called sequentially
  1132. // from applyAll function.
  1133. // So KV().Commit() cannot run in parallel with apply. It has to be called outside
  1134. // the go routine created below.
  1135. s.KV().Commit()
  1136. s.wg.Add(1)
  1137. go func() {
  1138. defer s.wg.Done()
  1139. d, err := clone.SaveNoCopy()
  1140. // TODO: current store will never fail to do a snapshot
  1141. // what should we do if the store might fail?
  1142. if err != nil {
  1143. plog.Panicf("store save should never fail: %v", err)
  1144. }
  1145. snap, err := s.r.raftStorage.CreateSnapshot(snapi, &confState, d)
  1146. if err != nil {
  1147. // the snapshot was done asynchronously with the progress of raft.
  1148. // raft might have already got a newer snapshot.
  1149. if err == raft.ErrSnapOutOfDate {
  1150. return
  1151. }
  1152. plog.Panicf("unexpected create snapshot error %v", err)
  1153. }
  1154. // SaveSnap saves the snapshot and releases the locked wal files
  1155. // to the snapshot index.
  1156. if err = s.r.storage.SaveSnap(snap); err != nil {
  1157. plog.Fatalf("save snapshot error: %v", err)
  1158. }
  1159. plog.Infof("saved snapshot at index %d", snap.Metadata.Index)
  1160. // When sending a snapshot, etcd will pause compaction.
  1161. // After receives a snapshot, the slow follower needs to get all the entries right after
  1162. // the snapshot sent to catch up. If we do not pause compaction, the log entries right after
  1163. // the snapshot sent might already be compacted. It happens when the snapshot takes long time
  1164. // to send and save. Pausing compaction avoids triggering a snapshot sending cycle.
  1165. if atomic.LoadInt64(&s.inflightSnapshots) != 0 {
  1166. plog.Infof("skip compaction since there is an inflight snapshot")
  1167. return
  1168. }
  1169. // keep some in memory log entries for slow followers.
  1170. compacti := uint64(1)
  1171. if snapi > numberOfCatchUpEntries {
  1172. compacti = snapi - numberOfCatchUpEntries
  1173. }
  1174. err = s.r.raftStorage.Compact(compacti)
  1175. if err != nil {
  1176. // the compaction was done asynchronously with the progress of raft.
  1177. // raft log might already been compact.
  1178. if err == raft.ErrCompacted {
  1179. return
  1180. }
  1181. plog.Panicf("unexpected compaction error %v", err)
  1182. }
  1183. plog.Infof("compacted raft log at %d", compacti)
  1184. }()
  1185. }
  1186. // CutPeer drops messages to the specified peer.
  1187. func (s *EtcdServer) CutPeer(id types.ID) {
  1188. tr, ok := s.r.transport.(*rafthttp.Transport)
  1189. if ok {
  1190. tr.CutPeer(id)
  1191. }
  1192. }
  1193. // MendPeer recovers the message dropping behavior of the given peer.
  1194. func (s *EtcdServer) MendPeer(id types.ID) {
  1195. tr, ok := s.r.transport.(*rafthttp.Transport)
  1196. if ok {
  1197. tr.MendPeer(id)
  1198. }
  1199. }
  1200. func (s *EtcdServer) PauseSending() { s.r.pauseSending() }
  1201. func (s *EtcdServer) ResumeSending() { s.r.resumeSending() }
  1202. func (s *EtcdServer) ClusterVersion() *semver.Version {
  1203. if s.cluster == nil {
  1204. return nil
  1205. }
  1206. return s.cluster.Version()
  1207. }
  1208. // monitorVersions checks the member's version every monitorVersionInterval.
  1209. // It updates the cluster version if all members agrees on a higher one.
  1210. // It prints out log if there is a member with a higher version than the
  1211. // local version.
  1212. func (s *EtcdServer) monitorVersions() {
  1213. for {
  1214. select {
  1215. case <-s.forceVersionC:
  1216. case <-time.After(monitorVersionInterval):
  1217. case <-s.done:
  1218. return
  1219. }
  1220. if s.Leader() != s.ID() {
  1221. continue
  1222. }
  1223. v := decideClusterVersion(getVersions(s.cluster, s.id, s.peerRt))
  1224. if v != nil {
  1225. // only keep major.minor version for comparison
  1226. v = &semver.Version{
  1227. Major: v.Major,
  1228. Minor: v.Minor,
  1229. }
  1230. }
  1231. // if the current version is nil:
  1232. // 1. use the decided version if possible
  1233. // 2. or use the min cluster version
  1234. if s.cluster.Version() == nil {
  1235. if v != nil {
  1236. go s.updateClusterVersion(v.String())
  1237. } else {
  1238. go s.updateClusterVersion(version.MinClusterVersion)
  1239. }
  1240. continue
  1241. }
  1242. // update cluster version only if the decided version is greater than
  1243. // the current cluster version
  1244. if v != nil && s.cluster.Version().LessThan(*v) {
  1245. go s.updateClusterVersion(v.String())
  1246. }
  1247. }
  1248. }
  1249. func (s *EtcdServer) updateClusterVersion(ver string) {
  1250. if s.cluster.Version() == nil {
  1251. plog.Infof("setting up the initial cluster version to %s", version.Cluster(ver))
  1252. } else {
  1253. plog.Infof("updating the cluster version from %s to %s", version.Cluster(s.cluster.Version().String()), version.Cluster(ver))
  1254. }
  1255. req := pb.Request{
  1256. Method: "PUT",
  1257. Path: membership.StoreClusterVersionKey(),
  1258. Val: ver,
  1259. }
  1260. ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout())
  1261. _, err := s.Do(ctx, req)
  1262. cancel()
  1263. switch err {
  1264. case nil:
  1265. return
  1266. case ErrStopped:
  1267. plog.Infof("aborting update cluster version because server is stopped")
  1268. return
  1269. default:
  1270. plog.Errorf("error updating cluster version (%v)", err)
  1271. }
  1272. }
  1273. func (s *EtcdServer) parseProposeCtxErr(err error, start time.Time) error {
  1274. switch err {
  1275. case context.Canceled:
  1276. return ErrCanceled
  1277. case context.DeadlineExceeded:
  1278. curLeadElected := s.r.leadElectedTime()
  1279. prevLeadLost := curLeadElected.Add(-2 * time.Duration(s.Cfg.ElectionTicks) * time.Duration(s.Cfg.TickMs) * time.Millisecond)
  1280. if start.After(prevLeadLost) && start.Before(curLeadElected) {
  1281. return ErrTimeoutDueToLeaderFail
  1282. }
  1283. lead := types.ID(atomic.LoadUint64(&s.r.lead))
  1284. switch lead {
  1285. case types.ID(raft.None):
  1286. // TODO: return error to specify it happens because the cluster does not have leader now
  1287. case s.ID():
  1288. if !isConnectedToQuorumSince(s.r.transport, start, s.ID(), s.cluster.Members()) {
  1289. return ErrTimeoutDueToConnectionLost
  1290. }
  1291. default:
  1292. if !isConnectedSince(s.r.transport, start, lead) {
  1293. return ErrTimeoutDueToConnectionLost
  1294. }
  1295. }
  1296. return ErrTimeout
  1297. default:
  1298. return err
  1299. }
  1300. }
  1301. func (s *EtcdServer) KV() mvcc.ConsistentWatchableKV { return s.kv }
  1302. func (s *EtcdServer) Backend() backend.Backend {
  1303. s.bemu.Lock()
  1304. defer s.bemu.Unlock()
  1305. return s.be
  1306. }
  1307. func (s *EtcdServer) AuthStore() auth.AuthStore { return s.authStore }
  1308. func (s *EtcdServer) restoreAlarms() error {
  1309. s.applyV3 = s.newApplierV3()
  1310. as, err := alarm.NewAlarmStore(s)
  1311. if err != nil {
  1312. return err
  1313. }
  1314. s.alarmStore = as
  1315. if len(as.Get(pb.AlarmType_NOSPACE)) > 0 {
  1316. s.applyV3 = newApplierV3Capped(s.applyV3)
  1317. }
  1318. return nil
  1319. }
  1320. func (s *EtcdServer) getAppliedIndex() uint64 {
  1321. return atomic.LoadUint64(&s.appliedIndex)
  1322. }
  1323. func (s *EtcdServer) setAppliedIndex(v uint64) {
  1324. atomic.StoreUint64(&s.appliedIndex, v)
  1325. }
  1326. func (s *EtcdServer) getCommittedIndex() uint64 {
  1327. return atomic.LoadUint64(&s.committedIndex)
  1328. }
  1329. func (s *EtcdServer) setCommittedIndex(v uint64) {
  1330. atomic.StoreUint64(&s.committedIndex, v)
  1331. }