server.go 68 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409
  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. "context"
  17. "encoding/json"
  18. "expvar"
  19. "fmt"
  20. "math"
  21. "math/rand"
  22. "net/http"
  23. "os"
  24. "path"
  25. "regexp"
  26. "sync"
  27. "sync/atomic"
  28. "time"
  29. "github.com/coreos/etcd/auth"
  30. "github.com/coreos/etcd/etcdserver/api"
  31. "github.com/coreos/etcd/etcdserver/api/membership"
  32. "github.com/coreos/etcd/etcdserver/api/rafthttp"
  33. "github.com/coreos/etcd/etcdserver/api/snap"
  34. "github.com/coreos/etcd/etcdserver/api/v2discovery"
  35. "github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
  36. stats "github.com/coreos/etcd/etcdserver/api/v2stats"
  37. "github.com/coreos/etcd/etcdserver/api/v2store"
  38. "github.com/coreos/etcd/etcdserver/api/v3alarm"
  39. "github.com/coreos/etcd/etcdserver/api/v3compactor"
  40. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  41. "github.com/coreos/etcd/lease"
  42. "github.com/coreos/etcd/lease/leasehttp"
  43. "github.com/coreos/etcd/mvcc"
  44. "github.com/coreos/etcd/mvcc/backend"
  45. "github.com/coreos/etcd/pkg/fileutil"
  46. "github.com/coreos/etcd/pkg/idutil"
  47. "github.com/coreos/etcd/pkg/pbutil"
  48. "github.com/coreos/etcd/pkg/runtime"
  49. "github.com/coreos/etcd/pkg/schedule"
  50. "github.com/coreos/etcd/pkg/types"
  51. "github.com/coreos/etcd/pkg/wait"
  52. "github.com/coreos/etcd/raft"
  53. "github.com/coreos/etcd/raft/raftpb"
  54. "github.com/coreos/etcd/version"
  55. "github.com/coreos/etcd/wal"
  56. "github.com/coreos/go-semver/semver"
  57. "github.com/coreos/pkg/capnslog"
  58. humanize "github.com/dustin/go-humanize"
  59. "go.uber.org/zap"
  60. )
  61. const (
  62. DefaultSnapshotCount = 100000
  63. // DefaultSnapshotCatchUpEntries is the number of entries for a slow follower
  64. // to catch-up after compacting the raft storage entries.
  65. // We expect the follower has a millisecond level latency with the leader.
  66. // The max throughput is around 10K. Keep a 5K entries is enough for helping
  67. // follower to catch up.
  68. DefaultSnapshotCatchUpEntries uint64 = 5000
  69. StoreClusterPrefix = "/0"
  70. StoreKeysPrefix = "/1"
  71. // HealthInterval is the minimum time the cluster should be healthy
  72. // before accepting add member requests.
  73. HealthInterval = 5 * time.Second
  74. purgeFileInterval = 30 * time.Second
  75. // monitorVersionInterval should be smaller than the timeout
  76. // on the connection. Or we will not be able to reuse the connection
  77. // (since it will timeout).
  78. monitorVersionInterval = rafthttp.ConnWriteTimeout - time.Second
  79. // max number of in-flight snapshot messages etcdserver allows to have
  80. // This number is more than enough for most clusters with 5 machines.
  81. maxInFlightMsgSnap = 16
  82. releaseDelayAfterSnapshot = 30 * time.Second
  83. // maxPendingRevokes is the maximum number of outstanding expired lease revocations.
  84. maxPendingRevokes = 16
  85. recommendedMaxRequestBytes = 10 * 1024 * 1024
  86. )
  87. var (
  88. plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "etcdserver")
  89. storeMemberAttributeRegexp = regexp.MustCompile(path.Join(membership.StoreMembersPrefix, "[[:xdigit:]]{1,16}", "attributes"))
  90. )
  91. func init() {
  92. rand.Seed(time.Now().UnixNano())
  93. expvar.Publish(
  94. "file_descriptor_limit",
  95. expvar.Func(
  96. func() interface{} {
  97. n, _ := runtime.FDLimit()
  98. return n
  99. },
  100. ),
  101. )
  102. }
  103. type Response struct {
  104. Term uint64
  105. Index uint64
  106. Event *v2store.Event
  107. Watcher v2store.Watcher
  108. Err error
  109. }
  110. type ServerV2 interface {
  111. Server
  112. Leader() types.ID
  113. // Do takes a V2 request and attempts to fulfill it, returning a Response.
  114. Do(ctx context.Context, r pb.Request) (Response, error)
  115. stats.Stats
  116. ClientCertAuthEnabled() bool
  117. }
  118. type ServerV3 interface {
  119. Server
  120. RaftStatusGetter
  121. }
  122. func (s *EtcdServer) ClientCertAuthEnabled() bool { return s.Cfg.ClientCertAuthEnabled }
  123. type Server interface {
  124. // AddMember attempts to add a member into the cluster. It will return
  125. // ErrIDRemoved if member ID is removed from the cluster, or return
  126. // ErrIDExists if member ID exists in the cluster.
  127. AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error)
  128. // RemoveMember attempts to remove a member from the cluster. It will
  129. // return ErrIDRemoved if member ID is removed from the cluster, or return
  130. // ErrIDNotFound if member ID is not in the cluster.
  131. RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error)
  132. // UpdateMember attempts to update an existing member in the cluster. It will
  133. // return ErrIDNotFound if the member ID does not exist.
  134. UpdateMember(ctx context.Context, updateMemb membership.Member) ([]*membership.Member, error)
  135. // ClusterVersion is the cluster-wide minimum major.minor version.
  136. // Cluster version is set to the min version that an etcd member is
  137. // compatible with when first bootstrap.
  138. //
  139. // ClusterVersion is nil until the cluster is bootstrapped (has a quorum).
  140. //
  141. // During a rolling upgrades, the ClusterVersion will be updated
  142. // automatically after a sync. (5 second by default)
  143. //
  144. // The API/raft component can utilize ClusterVersion to determine if
  145. // it can accept a client request or a raft RPC.
  146. // NOTE: ClusterVersion might be nil when etcd 2.1 works with etcd 2.0 and
  147. // the leader is etcd 2.0. etcd 2.0 leader will not update clusterVersion since
  148. // this feature is introduced post 2.0.
  149. ClusterVersion() *semver.Version
  150. Cluster() api.Cluster
  151. Alarms() []*pb.AlarmMember
  152. }
  153. // EtcdServer is the production implementation of the Server interface
  154. type EtcdServer struct {
  155. // inflightSnapshots holds count the number of snapshots currently inflight.
  156. inflightSnapshots int64 // must use atomic operations to access; keep 64-bit aligned.
  157. appliedIndex uint64 // must use atomic operations to access; keep 64-bit aligned.
  158. committedIndex uint64 // must use atomic operations to access; keep 64-bit aligned.
  159. term uint64 // must use atomic operations to access; keep 64-bit aligned.
  160. lead uint64 // must use atomic operations to access; keep 64-bit aligned.
  161. // consistIndex used to hold the offset of current executing entry
  162. // It is initialized to 0 before executing any entry.
  163. consistIndex consistentIndex // must use atomic operations to access; keep 64-bit aligned.
  164. r raftNode // uses 64-bit atomics; keep 64-bit aligned.
  165. readych chan struct{}
  166. Cfg ServerConfig
  167. lgMu *sync.RWMutex
  168. lg *zap.Logger
  169. w wait.Wait
  170. readMu sync.RWMutex
  171. // read routine notifies etcd server that it waits for reading by sending an empty struct to
  172. // readwaitC
  173. readwaitc chan struct{}
  174. // readNotifier is used to notify the read routine that it can process the request
  175. // when there is no error
  176. readNotifier *notifier
  177. // stop signals the run goroutine should shutdown.
  178. stop chan struct{}
  179. // stopping is closed by run goroutine on shutdown.
  180. stopping chan struct{}
  181. // done is closed when all goroutines from start() complete.
  182. done chan struct{}
  183. errorc chan error
  184. id types.ID
  185. attributes membership.Attributes
  186. cluster *membership.RaftCluster
  187. v2store v2store.Store
  188. snapshotter *snap.Snapshotter
  189. applyV2 ApplierV2
  190. // applyV3 is the applier with auth and quotas
  191. applyV3 applierV3
  192. // applyV3Base is the core applier without auth or quotas
  193. applyV3Base applierV3
  194. applyWait wait.WaitTime
  195. kv mvcc.ConsistentWatchableKV
  196. lessor lease.Lessor
  197. bemu sync.Mutex
  198. be backend.Backend
  199. authStore auth.AuthStore
  200. alarmStore *v3alarm.AlarmStore
  201. stats *stats.ServerStats
  202. lstats *stats.LeaderStats
  203. SyncTicker *time.Ticker
  204. // compactor is used to auto-compact the KV.
  205. compactor v3compactor.Compactor
  206. // peerRt used to send requests (version, lease) to peers.
  207. peerRt http.RoundTripper
  208. reqIDGen *idutil.Generator
  209. // forceVersionC is used to force the version monitor loop
  210. // to detect the cluster version immediately.
  211. forceVersionC chan struct{}
  212. // wgMu blocks concurrent waitgroup mutation while server stopping
  213. wgMu sync.RWMutex
  214. // wg is used to wait for the go routines that depends on the server state
  215. // to exit when stopping the server.
  216. wg sync.WaitGroup
  217. // ctx is used for etcd-initiated requests that may need to be canceled
  218. // on etcd server shutdown.
  219. ctx context.Context
  220. cancel context.CancelFunc
  221. leadTimeMu sync.RWMutex
  222. leadElectedTime time.Time
  223. *AccessController
  224. }
  225. // NewServer creates a new EtcdServer from the supplied configuration. The
  226. // configuration is considered static for the lifetime of the EtcdServer.
  227. func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
  228. st := v2store.New(StoreClusterPrefix, StoreKeysPrefix)
  229. var (
  230. w *wal.WAL
  231. n raft.Node
  232. s *raft.MemoryStorage
  233. id types.ID
  234. cl *membership.RaftCluster
  235. )
  236. if cfg.MaxRequestBytes > recommendedMaxRequestBytes {
  237. if cfg.Logger != nil {
  238. cfg.Logger.Warn(
  239. "exceeded recommended requet limit",
  240. zap.Uint("max-request-bytes", cfg.MaxRequestBytes),
  241. zap.String("max-request-size", humanize.Bytes(uint64(cfg.MaxRequestBytes))),
  242. zap.Int("recommended-request-bytes", recommendedMaxRequestBytes),
  243. zap.String("recommended-request-size", humanize.Bytes(uint64(recommendedMaxRequestBytes))),
  244. )
  245. } else {
  246. plog.Warningf("MaxRequestBytes %v exceeds maximum recommended size %v", cfg.MaxRequestBytes, recommendedMaxRequestBytes)
  247. }
  248. }
  249. if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
  250. return nil, fmt.Errorf("cannot access data directory: %v", terr)
  251. }
  252. haveWAL := wal.Exist(cfg.WALDir())
  253. if err = fileutil.TouchDirAll(cfg.SnapDir()); err != nil {
  254. if cfg.Logger != nil {
  255. cfg.Logger.Fatal(
  256. "failed to create snapshot directory",
  257. zap.String("path", cfg.SnapDir()),
  258. zap.Error(err),
  259. )
  260. } else {
  261. plog.Fatalf("create snapshot directory error: %v", err)
  262. }
  263. }
  264. ss := snap.New(cfg.Logger, cfg.SnapDir())
  265. bepath := cfg.backendPath()
  266. beExist := fileutil.Exist(bepath)
  267. be := openBackend(cfg)
  268. defer func() {
  269. if err != nil {
  270. be.Close()
  271. }
  272. }()
  273. prt, err := rafthttp.NewRoundTripper(cfg.PeerTLSInfo, cfg.peerDialTimeout())
  274. if err != nil {
  275. return nil, err
  276. }
  277. var (
  278. remotes []*membership.Member
  279. snapshot *raftpb.Snapshot
  280. )
  281. switch {
  282. case !haveWAL && !cfg.NewCluster:
  283. if err = cfg.VerifyJoinExisting(); err != nil {
  284. return nil, err
  285. }
  286. cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
  287. if err != nil {
  288. return nil, err
  289. }
  290. existingCluster, gerr := GetClusterFromRemotePeers(cfg.Logger, getRemotePeerURLs(cl, cfg.Name), prt)
  291. if gerr != nil {
  292. return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", gerr)
  293. }
  294. if err = membership.ValidateClusterAndAssignIDs(cfg.Logger, cl, existingCluster); err != nil {
  295. return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
  296. }
  297. if !isCompatibleWithCluster(cfg.Logger, cl, cl.MemberByName(cfg.Name).ID, prt) {
  298. return nil, fmt.Errorf("incompatible with current running cluster")
  299. }
  300. remotes = existingCluster.Members()
  301. cl.SetID(types.ID(0), existingCluster.ID())
  302. cl.SetStore(st)
  303. cl.SetBackend(be)
  304. id, n, s, w = startNode(cfg, cl, nil)
  305. cl.SetID(id, existingCluster.ID())
  306. case !haveWAL && cfg.NewCluster:
  307. if err = cfg.VerifyBootstrap(); err != nil {
  308. return nil, err
  309. }
  310. cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
  311. if err != nil {
  312. return nil, err
  313. }
  314. m := cl.MemberByName(cfg.Name)
  315. if isMemberBootstrapped(cfg.Logger, cl, cfg.Name, prt, cfg.bootstrapTimeout()) {
  316. return nil, fmt.Errorf("member %s has already been bootstrapped", m.ID)
  317. }
  318. if cfg.ShouldDiscover() {
  319. var str string
  320. str, err = v2discovery.JoinCluster(cfg.Logger, cfg.DiscoveryURL, cfg.DiscoveryProxy, m.ID, cfg.InitialPeerURLsMap.String())
  321. if err != nil {
  322. return nil, &DiscoveryError{Op: "join", Err: err}
  323. }
  324. var urlsmap types.URLsMap
  325. urlsmap, err = types.NewURLsMap(str)
  326. if err != nil {
  327. return nil, err
  328. }
  329. if checkDuplicateURL(urlsmap) {
  330. return nil, fmt.Errorf("discovery cluster %s has duplicate url", urlsmap)
  331. }
  332. if cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, urlsmap); err != nil {
  333. return nil, err
  334. }
  335. }
  336. cl.SetStore(st)
  337. cl.SetBackend(be)
  338. id, n, s, w = startNode(cfg, cl, cl.MemberIDs())
  339. cl.SetID(id, cl.ID())
  340. case haveWAL:
  341. if err = fileutil.IsDirWriteable(cfg.MemberDir()); err != nil {
  342. return nil, fmt.Errorf("cannot write to member directory: %v", err)
  343. }
  344. if err = fileutil.IsDirWriteable(cfg.WALDir()); err != nil {
  345. return nil, fmt.Errorf("cannot write to WAL directory: %v", err)
  346. }
  347. if cfg.ShouldDiscover() {
  348. if cfg.Logger != nil {
  349. cfg.Logger.Warn(
  350. "discovery token is ignored since cluster already initialized; valid logs are found",
  351. zap.String("wal-dir", cfg.WALDir()),
  352. )
  353. } else {
  354. plog.Warningf("discovery token ignored since a cluster has already been initialized. Valid log found at %q", cfg.WALDir())
  355. }
  356. }
  357. snapshot, err = ss.Load()
  358. if err != nil && err != snap.ErrNoSnapshot {
  359. return nil, err
  360. }
  361. if snapshot != nil {
  362. if err = st.Recovery(snapshot.Data); err != nil {
  363. if cfg.Logger != nil {
  364. cfg.Logger.Panic("failed to recover from snapshot")
  365. } else {
  366. plog.Panicf("recovered store from snapshot error: %v", err)
  367. }
  368. }
  369. if cfg.Logger != nil {
  370. cfg.Logger.Info(
  371. "recovered v2 store from snapshot",
  372. zap.Uint64("snapshot-index", snapshot.Metadata.Index),
  373. zap.String("snapshot-size", humanize.Bytes(uint64(snapshot.Size()))),
  374. )
  375. } else {
  376. plog.Infof("recovered store from snapshot at index %d", snapshot.Metadata.Index)
  377. }
  378. if be, err = recoverSnapshotBackend(cfg, be, *snapshot); err != nil {
  379. if cfg.Logger != nil {
  380. cfg.Logger.Panic("failed to recover v3 backend from snapshot", zap.Error(err))
  381. } else {
  382. plog.Panicf("recovering backend from snapshot error: %v", err)
  383. }
  384. }
  385. if cfg.Logger != nil {
  386. s1, s2 := be.Size(), be.SizeInUse()
  387. cfg.Logger.Info(
  388. "recovered v3 backend from snapshot",
  389. zap.Int64("backend-size-bytes", s1),
  390. zap.String("backend-size", humanize.Bytes(uint64(s1))),
  391. zap.Int64("backend-size-in-use-bytes", s2),
  392. zap.String("backend-size-in-use", humanize.Bytes(uint64(s2))),
  393. )
  394. }
  395. }
  396. if !cfg.ForceNewCluster {
  397. id, cl, n, s, w = restartNode(cfg, snapshot)
  398. } else {
  399. id, cl, n, s, w = restartAsStandaloneNode(cfg, snapshot)
  400. }
  401. cl.SetStore(st)
  402. cl.SetBackend(be)
  403. cl.Recover(api.UpdateCapability)
  404. if cl.Version() != nil && !cl.Version().LessThan(semver.Version{Major: 3}) && !beExist {
  405. os.RemoveAll(bepath)
  406. return nil, fmt.Errorf("database file (%v) of the backend is missing", bepath)
  407. }
  408. default:
  409. return nil, fmt.Errorf("unsupported bootstrap config")
  410. }
  411. if terr := fileutil.TouchDirAll(cfg.MemberDir()); terr != nil {
  412. return nil, fmt.Errorf("cannot access member directory: %v", terr)
  413. }
  414. sstats := stats.NewServerStats(cfg.Name, id.String())
  415. lstats := stats.NewLeaderStats(id.String())
  416. heartbeat := time.Duration(cfg.TickMs) * time.Millisecond
  417. srv = &EtcdServer{
  418. readych: make(chan struct{}),
  419. Cfg: cfg,
  420. lgMu: new(sync.RWMutex),
  421. lg: cfg.Logger,
  422. errorc: make(chan error, 1),
  423. v2store: st,
  424. snapshotter: ss,
  425. r: *newRaftNode(
  426. raftNodeConfig{
  427. lg: cfg.Logger,
  428. isIDRemoved: func(id uint64) bool { return cl.IsIDRemoved(types.ID(id)) },
  429. Node: n,
  430. heartbeat: heartbeat,
  431. raftStorage: s,
  432. storage: NewStorage(w, ss),
  433. },
  434. ),
  435. id: id,
  436. attributes: membership.Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
  437. cluster: cl,
  438. stats: sstats,
  439. lstats: lstats,
  440. SyncTicker: time.NewTicker(500 * time.Millisecond),
  441. peerRt: prt,
  442. reqIDGen: idutil.NewGenerator(uint16(id), time.Now()),
  443. forceVersionC: make(chan struct{}),
  444. AccessController: &AccessController{CORS: cfg.CORS, HostWhitelist: cfg.HostWhitelist},
  445. }
  446. srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
  447. srv.be = be
  448. minTTL := time.Duration((3*cfg.ElectionTicks)/2) * heartbeat
  449. // always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
  450. // If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
  451. srv.lessor = lease.NewLessor(srv.be, int64(math.Ceil(minTTL.Seconds())))
  452. srv.kv = mvcc.New(srv.getLogger(), srv.be, srv.lessor, &srv.consistIndex)
  453. if beExist {
  454. kvindex := srv.kv.ConsistentIndex()
  455. // TODO: remove kvindex != 0 checking when we do not expect users to upgrade
  456. // etcd from pre-3.0 release.
  457. if snapshot != nil && kvindex < snapshot.Metadata.Index {
  458. if kvindex != 0 {
  459. return nil, fmt.Errorf("database file (%v index %d) does not match with snapshot (index %d)", bepath, kvindex, snapshot.Metadata.Index)
  460. }
  461. if cfg.Logger != nil {
  462. cfg.Logger.Warn(
  463. "consistent index was never saved",
  464. zap.Uint64("snapshot-index", snapshot.Metadata.Index),
  465. )
  466. } else {
  467. plog.Warningf("consistent index never saved (snapshot index=%d)", snapshot.Metadata.Index)
  468. }
  469. }
  470. }
  471. newSrv := srv // since srv == nil in defer if srv is returned as nil
  472. defer func() {
  473. // closing backend without first closing kv can cause
  474. // resumed compactions to fail with closed tx errors
  475. if err != nil {
  476. newSrv.kv.Close()
  477. }
  478. }()
  479. srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex())
  480. tp, err := auth.NewTokenProvider(cfg.Logger, cfg.AuthToken,
  481. func(index uint64) <-chan struct{} {
  482. return srv.applyWait.Wait(index)
  483. },
  484. )
  485. if err != nil {
  486. if cfg.Logger != nil {
  487. cfg.Logger.Warn("failed to create token provider", zap.Error(err))
  488. } else {
  489. plog.Errorf("failed to create token provider: %s", err)
  490. }
  491. return nil, err
  492. }
  493. srv.authStore = auth.NewAuthStore(srv.getLogger(), srv.be, tp, int(cfg.BcryptCost))
  494. if num := cfg.AutoCompactionRetention; num != 0 {
  495. srv.compactor, err = v3compactor.New(cfg.Logger, cfg.AutoCompactionMode, num, srv.kv, srv)
  496. if err != nil {
  497. return nil, err
  498. }
  499. srv.compactor.Run()
  500. }
  501. srv.applyV3Base = srv.newApplierV3Backend()
  502. if err = srv.restoreAlarms(); err != nil {
  503. return nil, err
  504. }
  505. // TODO: move transport initialization near the definition of remote
  506. tr := &rafthttp.Transport{
  507. Logger: cfg.Logger,
  508. TLSInfo: cfg.PeerTLSInfo,
  509. DialTimeout: cfg.peerDialTimeout(),
  510. ID: id,
  511. URLs: cfg.PeerURLs,
  512. ClusterID: cl.ID(),
  513. Raft: srv,
  514. Snapshotter: ss,
  515. ServerStats: sstats,
  516. LeaderStats: lstats,
  517. ErrorC: srv.errorc,
  518. }
  519. if err = tr.Start(); err != nil {
  520. return nil, err
  521. }
  522. // add all remotes into transport
  523. for _, m := range remotes {
  524. if m.ID != id {
  525. tr.AddRemote(m.ID, m.PeerURLs)
  526. }
  527. }
  528. for _, m := range cl.Members() {
  529. if m.ID != id {
  530. tr.AddPeer(m.ID, m.PeerURLs)
  531. }
  532. }
  533. srv.r.transport = tr
  534. return srv, nil
  535. }
  536. func (s *EtcdServer) getLogger() *zap.Logger {
  537. s.lgMu.RLock()
  538. l := s.lg
  539. s.lgMu.RUnlock()
  540. return l
  541. }
  542. func tickToDur(ticks int, tickMs uint) string {
  543. return fmt.Sprintf("%v", time.Duration(ticks)*time.Duration(tickMs)*time.Millisecond)
  544. }
  545. func (s *EtcdServer) adjustTicks() {
  546. lg := s.getLogger()
  547. clusterN := len(s.cluster.Members())
  548. // single-node fresh start, or single-node recovers from snapshot
  549. if clusterN == 1 {
  550. ticks := s.Cfg.ElectionTicks - 1
  551. if lg != nil {
  552. lg.Info(
  553. "started as single-node; fast-forwarding election ticks",
  554. zap.String("local-member-id", s.ID().String()),
  555. zap.Int("forward-ticks", ticks),
  556. zap.String("forward-duration", tickToDur(ticks, s.Cfg.TickMs)),
  557. zap.Int("election-ticks", s.Cfg.ElectionTicks),
  558. zap.String("election-timeout", tickToDur(s.Cfg.ElectionTicks, s.Cfg.TickMs)),
  559. )
  560. } else {
  561. plog.Infof("%s as single-node; fast-forwarding %d ticks (election ticks %d)", s.ID(), ticks, s.Cfg.ElectionTicks)
  562. }
  563. s.r.advanceTicks(ticks)
  564. return
  565. }
  566. if !s.Cfg.InitialElectionTickAdvance {
  567. if lg != nil {
  568. lg.Info("skipping initial election tick advance", zap.Int("election-ticks", s.Cfg.ElectionTicks))
  569. }
  570. return
  571. }
  572. if lg != nil {
  573. lg.Info("starting initial election tick advance", zap.Int("election-ticks", s.Cfg.ElectionTicks))
  574. }
  575. // retry up to "rafthttp.ConnReadTimeout", which is 5-sec
  576. // until peer connection reports; otherwise:
  577. // 1. all connections failed, or
  578. // 2. no active peers, or
  579. // 3. restarted single-node with no snapshot
  580. // then, do nothing, because advancing ticks would have no effect
  581. waitTime := rafthttp.ConnReadTimeout
  582. itv := 50 * time.Millisecond
  583. for i := int64(0); i < int64(waitTime/itv); i++ {
  584. select {
  585. case <-time.After(itv):
  586. case <-s.stopping:
  587. return
  588. }
  589. peerN := s.r.transport.ActivePeers()
  590. if peerN > 1 {
  591. // multi-node received peer connection reports
  592. // adjust ticks, in case slow leader message receive
  593. ticks := s.Cfg.ElectionTicks - 2
  594. if lg != nil {
  595. lg.Info(
  596. "initialized peer connections; fast-forwarding election ticks",
  597. zap.String("local-member-id", s.ID().String()),
  598. zap.Int("forward-ticks", ticks),
  599. zap.String("forward-duration", tickToDur(ticks, s.Cfg.TickMs)),
  600. zap.Int("election-ticks", s.Cfg.ElectionTicks),
  601. zap.String("election-timeout", tickToDur(s.Cfg.ElectionTicks, s.Cfg.TickMs)),
  602. zap.Int("active-remote-members", peerN),
  603. )
  604. } else {
  605. plog.Infof("%s initialized peer connection; fast-forwarding %d ticks (election ticks %d) with %d active peer(s)", s.ID(), ticks, s.Cfg.ElectionTicks, peerN)
  606. }
  607. s.r.advanceTicks(ticks)
  608. return
  609. }
  610. }
  611. }
  612. // Start performs any initialization of the Server necessary for it to
  613. // begin serving requests. It must be called before Do or Process.
  614. // Start must be non-blocking; any long-running server functionality
  615. // should be implemented in goroutines.
  616. func (s *EtcdServer) Start() {
  617. s.start()
  618. s.goAttach(func() { s.adjustTicks() })
  619. s.goAttach(func() { s.publish(s.Cfg.ReqTimeout()) })
  620. s.goAttach(s.purgeFile)
  621. s.goAttach(func() { monitorFileDescriptor(s.stopping) })
  622. s.goAttach(s.monitorVersions)
  623. s.goAttach(s.linearizableReadLoop)
  624. s.goAttach(s.monitorKVHash)
  625. }
  626. // start prepares and starts server in a new goroutine. It is no longer safe to
  627. // modify a server's fields after it has been sent to Start.
  628. // This function is just used for testing.
  629. func (s *EtcdServer) start() {
  630. lg := s.getLogger()
  631. if s.Cfg.SnapshotCount == 0 {
  632. if lg != nil {
  633. lg.Info(
  634. "updating snapshot-count to default",
  635. zap.Uint64("given-snapshot-count", s.Cfg.SnapshotCount),
  636. zap.Uint64("updated-snapshot-count", DefaultSnapshotCount),
  637. )
  638. } else {
  639. plog.Infof("set snapshot count to default %d", DefaultSnapshotCount)
  640. }
  641. s.Cfg.SnapshotCount = DefaultSnapshotCount
  642. }
  643. if s.Cfg.SnapshotCatchUpEntries == 0 {
  644. if lg != nil {
  645. lg.Info(
  646. "updating snapshot catch-up entries to default",
  647. zap.Uint64("given-snapshot-catchup-entries", s.Cfg.SnapshotCatchUpEntries),
  648. zap.Uint64("updated-snapshot-catchup-entries", DefaultSnapshotCatchUpEntries),
  649. )
  650. }
  651. s.Cfg.SnapshotCatchUpEntries = DefaultSnapshotCatchUpEntries
  652. }
  653. s.w = wait.New()
  654. s.applyWait = wait.NewTimeList()
  655. s.done = make(chan struct{})
  656. s.stop = make(chan struct{})
  657. s.stopping = make(chan struct{})
  658. s.ctx, s.cancel = context.WithCancel(context.Background())
  659. s.readwaitc = make(chan struct{}, 1)
  660. s.readNotifier = newNotifier()
  661. if s.ClusterVersion() != nil {
  662. if lg != nil {
  663. lg.Info(
  664. "starting etcd server",
  665. zap.String("local-member-id", s.ID().String()),
  666. zap.String("local-server-version", version.Version),
  667. zap.String("cluster-id", s.Cluster().ID().String()),
  668. zap.String("cluster-version", version.Cluster(s.ClusterVersion().String())),
  669. )
  670. } else {
  671. plog.Infof("starting server... [version: %v, cluster version: %v]", version.Version, version.Cluster(s.ClusterVersion().String()))
  672. }
  673. } else {
  674. if lg != nil {
  675. lg.Info(
  676. "starting etcd server",
  677. zap.String("local-member-id", s.ID().String()),
  678. zap.String("local-server-version", version.Version),
  679. zap.String("cluster-version", "to_be_decided"),
  680. )
  681. } else {
  682. plog.Infof("starting server... [version: %v, cluster version: to_be_decided]", version.Version)
  683. }
  684. }
  685. // TODO: if this is an empty log, writes all peer infos
  686. // into the first entry
  687. go s.run()
  688. }
  689. func (s *EtcdServer) purgeFile() {
  690. var dberrc, serrc, werrc <-chan error
  691. if s.Cfg.MaxSnapFiles > 0 {
  692. dberrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap.db", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
  693. serrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
  694. }
  695. if s.Cfg.MaxWALFiles > 0 {
  696. werrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.WALDir(), "wal", s.Cfg.MaxWALFiles, purgeFileInterval, s.done)
  697. }
  698. lg := s.getLogger()
  699. select {
  700. case e := <-dberrc:
  701. if lg != nil {
  702. lg.Fatal("failed to purge snap db file", zap.Error(e))
  703. } else {
  704. plog.Fatalf("failed to purge snap db file %v", e)
  705. }
  706. case e := <-serrc:
  707. if lg != nil {
  708. lg.Fatal("failed to purge snap file", zap.Error(e))
  709. } else {
  710. plog.Fatalf("failed to purge snap file %v", e)
  711. }
  712. case e := <-werrc:
  713. if lg != nil {
  714. lg.Fatal("failed to purge wal file", zap.Error(e))
  715. } else {
  716. plog.Fatalf("failed to purge wal file %v", e)
  717. }
  718. case <-s.stopping:
  719. return
  720. }
  721. }
  722. func (s *EtcdServer) Cluster() api.Cluster { return s.cluster }
  723. func (s *EtcdServer) ApplyWait() <-chan struct{} { return s.applyWait.Wait(s.getCommittedIndex()) }
  724. type ServerPeer interface {
  725. ServerV2
  726. RaftHandler() http.Handler
  727. LeaseHandler() http.Handler
  728. }
  729. func (s *EtcdServer) LeaseHandler() http.Handler {
  730. if s.lessor == nil {
  731. return nil
  732. }
  733. return leasehttp.NewHandler(s.lessor, s.ApplyWait)
  734. }
  735. func (s *EtcdServer) RaftHandler() http.Handler { return s.r.transport.Handler() }
  736. // Process takes a raft message and applies it to the server's raft state
  737. // machine, respecting any timeout of the given context.
  738. func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
  739. if s.cluster.IsIDRemoved(types.ID(m.From)) {
  740. if lg := s.getLogger(); lg != nil {
  741. lg.Warn(
  742. "rejected Raft message from removed member",
  743. zap.String("local-member-id", s.ID().String()),
  744. zap.String("removed-member-id", types.ID(m.From).String()),
  745. )
  746. } else {
  747. plog.Warningf("reject message from removed member %s", types.ID(m.From).String())
  748. }
  749. return httptypes.NewHTTPError(http.StatusForbidden, "cannot process message from removed member")
  750. }
  751. if m.Type == raftpb.MsgApp {
  752. s.stats.RecvAppendReq(types.ID(m.From).String(), m.Size())
  753. }
  754. return s.r.Step(ctx, m)
  755. }
  756. func (s *EtcdServer) IsIDRemoved(id uint64) bool { return s.cluster.IsIDRemoved(types.ID(id)) }
  757. func (s *EtcdServer) ReportUnreachable(id uint64) { s.r.ReportUnreachable(id) }
  758. // ReportSnapshot reports snapshot sent status to the raft state machine,
  759. // and clears the used snapshot from the snapshot store.
  760. func (s *EtcdServer) ReportSnapshot(id uint64, status raft.SnapshotStatus) {
  761. s.r.ReportSnapshot(id, status)
  762. }
  763. type etcdProgress struct {
  764. confState raftpb.ConfState
  765. snapi uint64
  766. appliedt uint64
  767. appliedi uint64
  768. }
  769. // raftReadyHandler contains a set of EtcdServer operations to be called by raftNode,
  770. // and helps decouple state machine logic from Raft algorithms.
  771. // TODO: add a state machine interface to apply the commit entries and do snapshot/recover
  772. type raftReadyHandler struct {
  773. getLead func() (lead uint64)
  774. updateLead func(lead uint64)
  775. updateLeadership func(newLeader bool)
  776. updateCommittedIndex func(uint64)
  777. }
  778. func (s *EtcdServer) run() {
  779. lg := s.getLogger()
  780. sn, err := s.r.raftStorage.Snapshot()
  781. if err != nil {
  782. if lg != nil {
  783. lg.Panic("failed to get snapshot from Raft storage", zap.Error(err))
  784. } else {
  785. plog.Panicf("get snapshot from raft storage error: %v", err)
  786. }
  787. }
  788. // asynchronously accept apply packets, dispatch progress in-order
  789. sched := schedule.NewFIFOScheduler()
  790. var (
  791. smu sync.RWMutex
  792. syncC <-chan time.Time
  793. )
  794. setSyncC := func(ch <-chan time.Time) {
  795. smu.Lock()
  796. syncC = ch
  797. smu.Unlock()
  798. }
  799. getSyncC := func() (ch <-chan time.Time) {
  800. smu.RLock()
  801. ch = syncC
  802. smu.RUnlock()
  803. return
  804. }
  805. rh := &raftReadyHandler{
  806. getLead: func() (lead uint64) { return s.getLead() },
  807. updateLead: func(lead uint64) { s.setLead(lead) },
  808. updateLeadership: func(newLeader bool) {
  809. if !s.isLeader() {
  810. if s.lessor != nil {
  811. s.lessor.Demote()
  812. }
  813. if s.compactor != nil {
  814. s.compactor.Pause()
  815. }
  816. setSyncC(nil)
  817. } else {
  818. if newLeader {
  819. t := time.Now()
  820. s.leadTimeMu.Lock()
  821. s.leadElectedTime = t
  822. s.leadTimeMu.Unlock()
  823. }
  824. setSyncC(s.SyncTicker.C)
  825. if s.compactor != nil {
  826. s.compactor.Resume()
  827. }
  828. }
  829. // TODO: remove the nil checking
  830. // current test utility does not provide the stats
  831. if s.stats != nil {
  832. s.stats.BecomeLeader()
  833. }
  834. },
  835. updateCommittedIndex: func(ci uint64) {
  836. cci := s.getCommittedIndex()
  837. if ci > cci {
  838. s.setCommittedIndex(ci)
  839. }
  840. },
  841. }
  842. s.r.start(rh)
  843. ep := etcdProgress{
  844. confState: sn.Metadata.ConfState,
  845. snapi: sn.Metadata.Index,
  846. appliedt: sn.Metadata.Term,
  847. appliedi: sn.Metadata.Index,
  848. }
  849. defer func() {
  850. s.wgMu.Lock() // block concurrent waitgroup adds in goAttach while stopping
  851. close(s.stopping)
  852. s.wgMu.Unlock()
  853. s.cancel()
  854. sched.Stop()
  855. // wait for gouroutines before closing raft so wal stays open
  856. s.wg.Wait()
  857. s.SyncTicker.Stop()
  858. // must stop raft after scheduler-- etcdserver can leak rafthttp pipelines
  859. // by adding a peer after raft stops the transport
  860. s.r.stop()
  861. // kv, lessor and backend can be nil if running without v3 enabled
  862. // or running unit tests.
  863. if s.lessor != nil {
  864. s.lessor.Stop()
  865. }
  866. if s.kv != nil {
  867. s.kv.Close()
  868. }
  869. if s.authStore != nil {
  870. s.authStore.Close()
  871. }
  872. if s.be != nil {
  873. s.be.Close()
  874. }
  875. if s.compactor != nil {
  876. s.compactor.Stop()
  877. }
  878. close(s.done)
  879. }()
  880. var expiredLeaseC <-chan []*lease.Lease
  881. if s.lessor != nil {
  882. expiredLeaseC = s.lessor.ExpiredLeasesC()
  883. }
  884. for {
  885. select {
  886. case ap := <-s.r.apply():
  887. f := func(context.Context) { s.applyAll(&ep, &ap) }
  888. sched.Schedule(f)
  889. case leases := <-expiredLeaseC:
  890. s.goAttach(func() {
  891. // Increases throughput of expired leases deletion process through parallelization
  892. c := make(chan struct{}, maxPendingRevokes)
  893. for _, lease := range leases {
  894. select {
  895. case c <- struct{}{}:
  896. case <-s.stopping:
  897. return
  898. }
  899. lid := lease.ID
  900. s.goAttach(func() {
  901. ctx := s.authStore.WithRoot(s.ctx)
  902. _, lerr := s.LeaseRevoke(ctx, &pb.LeaseRevokeRequest{ID: int64(lid)})
  903. if lerr == nil {
  904. leaseExpired.Inc()
  905. } else {
  906. if lg != nil {
  907. lg.Warn(
  908. "failed to revoke lease",
  909. zap.String("lease-id", fmt.Sprintf("%016x", lid)),
  910. zap.Error(lerr),
  911. )
  912. } else {
  913. plog.Warningf("failed to revoke %016x (%q)", lid, lerr.Error())
  914. }
  915. }
  916. <-c
  917. })
  918. }
  919. })
  920. case err := <-s.errorc:
  921. if lg != nil {
  922. lg.Warn("server error", zap.Error(err))
  923. lg.Warn("data-dir used by this member must be removed")
  924. } else {
  925. plog.Errorf("%s", err)
  926. plog.Infof("the data-dir used by this member must be removed.")
  927. }
  928. return
  929. case <-getSyncC():
  930. if s.v2store.HasTTLKeys() {
  931. s.sync(s.Cfg.ReqTimeout())
  932. }
  933. case <-s.stop:
  934. return
  935. }
  936. }
  937. }
  938. func (s *EtcdServer) applyAll(ep *etcdProgress, apply *apply) {
  939. s.applySnapshot(ep, apply)
  940. s.applyEntries(ep, apply)
  941. proposalsApplied.Set(float64(ep.appliedi))
  942. s.applyWait.Trigger(ep.appliedi)
  943. // wait for the raft routine to finish the disk writes before triggering a
  944. // snapshot. or applied index might be greater than the last index in raft
  945. // storage, since the raft routine might be slower than apply routine.
  946. <-apply.notifyc
  947. s.triggerSnapshot(ep)
  948. select {
  949. // snapshot requested via send()
  950. case m := <-s.r.msgSnapC:
  951. merged := s.createMergedSnapshotMessage(m, ep.appliedt, ep.appliedi, ep.confState)
  952. s.sendMergedSnap(merged)
  953. default:
  954. }
  955. }
  956. func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
  957. if raft.IsEmptySnap(apply.snapshot) {
  958. return
  959. }
  960. lg := s.getLogger()
  961. if lg != nil {
  962. lg.Info(
  963. "applying snapshot",
  964. zap.Uint64("current-snapshot-index", ep.snapi),
  965. zap.Uint64("current-applied-index", ep.appliedi),
  966. zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
  967. zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
  968. )
  969. } else {
  970. plog.Infof("applying snapshot at index %d...", ep.snapi)
  971. }
  972. defer func() {
  973. if lg != nil {
  974. lg.Info(
  975. "applied snapshot",
  976. zap.Uint64("current-snapshot-index", ep.snapi),
  977. zap.Uint64("current-applied-index", ep.appliedi),
  978. zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
  979. zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
  980. )
  981. } else {
  982. plog.Infof("finished applying incoming snapshot at index %d", ep.snapi)
  983. }
  984. }()
  985. if apply.snapshot.Metadata.Index <= ep.appliedi {
  986. if lg != nil {
  987. lg.Panic(
  988. "unexpected leader snapshot from outdated index",
  989. zap.Uint64("current-snapshot-index", ep.snapi),
  990. zap.Uint64("current-applied-index", ep.appliedi),
  991. zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
  992. zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
  993. )
  994. } else {
  995. plog.Panicf("snapshot index [%d] should > appliedi[%d] + 1",
  996. apply.snapshot.Metadata.Index, ep.appliedi)
  997. }
  998. }
  999. // wait for raftNode to persist snapshot onto the disk
  1000. <-apply.notifyc
  1001. newbe, err := openSnapshotBackend(s.Cfg, s.snapshotter, apply.snapshot)
  1002. if err != nil {
  1003. if lg != nil {
  1004. lg.Panic("failed to open snapshot backend", zap.Error(err))
  1005. } else {
  1006. plog.Panic(err)
  1007. }
  1008. }
  1009. // always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
  1010. // If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
  1011. if s.lessor != nil {
  1012. if lg != nil {
  1013. lg.Info("restoring lease store")
  1014. } else {
  1015. plog.Info("recovering lessor...")
  1016. }
  1017. s.lessor.Recover(newbe, func() lease.TxnDelete { return s.kv.Write() })
  1018. if lg != nil {
  1019. lg.Info("restored lease store")
  1020. } else {
  1021. plog.Info("finished recovering lessor")
  1022. }
  1023. }
  1024. if lg != nil {
  1025. lg.Info("restoring mvcc store")
  1026. } else {
  1027. plog.Info("restoring mvcc store...")
  1028. }
  1029. if err := s.kv.Restore(newbe); err != nil {
  1030. if lg != nil {
  1031. lg.Panic("failed to restore mvcc store", zap.Error(err))
  1032. } else {
  1033. plog.Panicf("restore KV error: %v", err)
  1034. }
  1035. }
  1036. s.consistIndex.setConsistentIndex(s.kv.ConsistentIndex())
  1037. if lg != nil {
  1038. lg.Info("restored mvcc store")
  1039. } else {
  1040. plog.Info("finished restoring mvcc store")
  1041. }
  1042. // Closing old backend might block until all the txns
  1043. // on the backend are finished.
  1044. // We do not want to wait on closing the old backend.
  1045. s.bemu.Lock()
  1046. oldbe := s.be
  1047. go func() {
  1048. if lg != nil {
  1049. lg.Info("closing old backend file")
  1050. } else {
  1051. plog.Info("closing old backend...")
  1052. }
  1053. defer func() {
  1054. if lg != nil {
  1055. lg.Info("closed old backend file")
  1056. } else {
  1057. plog.Info("finished closing old backend")
  1058. }
  1059. }()
  1060. if err := oldbe.Close(); err != nil {
  1061. if lg != nil {
  1062. lg.Panic("failed to close old backend", zap.Error(err))
  1063. } else {
  1064. plog.Panicf("close backend error: %v", err)
  1065. }
  1066. }
  1067. }()
  1068. s.be = newbe
  1069. s.bemu.Unlock()
  1070. if lg != nil {
  1071. lg.Info("restoring alarm store")
  1072. } else {
  1073. plog.Info("recovering alarms...")
  1074. }
  1075. if err := s.restoreAlarms(); err != nil {
  1076. if lg != nil {
  1077. lg.Panic("failed to restore alarm store", zap.Error(err))
  1078. } else {
  1079. plog.Panicf("restore alarms error: %v", err)
  1080. }
  1081. }
  1082. if lg != nil {
  1083. lg.Info("restored alarm store")
  1084. } else {
  1085. plog.Info("finished recovering alarms")
  1086. }
  1087. if s.authStore != nil {
  1088. if lg != nil {
  1089. lg.Info("restoring auth store")
  1090. } else {
  1091. plog.Info("recovering auth store...")
  1092. }
  1093. s.authStore.Recover(newbe)
  1094. if lg != nil {
  1095. lg.Info("restored auth store")
  1096. } else {
  1097. plog.Info("finished recovering auth store")
  1098. }
  1099. }
  1100. if lg != nil {
  1101. lg.Info("restoring v2 store")
  1102. } else {
  1103. plog.Info("recovering store v2...")
  1104. }
  1105. if err := s.v2store.Recovery(apply.snapshot.Data); err != nil {
  1106. if lg != nil {
  1107. lg.Panic("failed to restore v2 store", zap.Error(err))
  1108. } else {
  1109. plog.Panicf("recovery store error: %v", err)
  1110. }
  1111. }
  1112. if lg != nil {
  1113. lg.Info("restored v2 store")
  1114. } else {
  1115. plog.Info("finished recovering store v2")
  1116. }
  1117. s.cluster.SetBackend(s.be)
  1118. if lg != nil {
  1119. lg.Info("restoring cluster configuration")
  1120. } else {
  1121. plog.Info("recovering cluster configuration...")
  1122. }
  1123. s.cluster.Recover(api.UpdateCapability)
  1124. if lg != nil {
  1125. lg.Info("restored cluster configuration")
  1126. lg.Info("removing old peers from network")
  1127. } else {
  1128. plog.Info("finished recovering cluster configuration")
  1129. plog.Info("removing old peers from network...")
  1130. }
  1131. // recover raft transport
  1132. s.r.transport.RemoveAllPeers()
  1133. if lg != nil {
  1134. lg.Info("removed old peers from network")
  1135. lg.Info("adding peers from new cluster configuration")
  1136. } else {
  1137. plog.Info("finished removing old peers from network")
  1138. plog.Info("adding peers from new cluster configuration into network...")
  1139. }
  1140. for _, m := range s.cluster.Members() {
  1141. if m.ID == s.ID() {
  1142. continue
  1143. }
  1144. s.r.transport.AddPeer(m.ID, m.PeerURLs)
  1145. }
  1146. if lg != nil {
  1147. lg.Info("added peers from new cluster configuration")
  1148. } else {
  1149. plog.Info("finished adding peers from new cluster configuration into network...")
  1150. }
  1151. ep.appliedt = apply.snapshot.Metadata.Term
  1152. ep.appliedi = apply.snapshot.Metadata.Index
  1153. ep.snapi = ep.appliedi
  1154. ep.confState = apply.snapshot.Metadata.ConfState
  1155. }
  1156. func (s *EtcdServer) applyEntries(ep *etcdProgress, apply *apply) {
  1157. if len(apply.entries) == 0 {
  1158. return
  1159. }
  1160. firsti := apply.entries[0].Index
  1161. if firsti > ep.appliedi+1 {
  1162. if lg := s.getLogger(); lg != nil {
  1163. lg.Panic(
  1164. "unexpected committed entry index",
  1165. zap.Uint64("current-applied-index", ep.appliedi),
  1166. zap.Uint64("first-committed-entry-index", firsti),
  1167. )
  1168. } else {
  1169. plog.Panicf("first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, ep.appliedi)
  1170. }
  1171. }
  1172. var ents []raftpb.Entry
  1173. if ep.appliedi+1-firsti < uint64(len(apply.entries)) {
  1174. ents = apply.entries[ep.appliedi+1-firsti:]
  1175. }
  1176. if len(ents) == 0 {
  1177. return
  1178. }
  1179. var shouldstop bool
  1180. if ep.appliedt, ep.appliedi, shouldstop = s.apply(ents, &ep.confState); shouldstop {
  1181. go s.stopWithDelay(10*100*time.Millisecond, fmt.Errorf("the member has been permanently removed from the cluster"))
  1182. }
  1183. }
  1184. func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
  1185. if ep.appliedi-ep.snapi <= s.Cfg.SnapshotCount {
  1186. return
  1187. }
  1188. if lg := s.getLogger(); lg != nil {
  1189. lg.Info(
  1190. "triggering snapshot",
  1191. zap.String("local-member-id", s.ID().String()),
  1192. zap.Uint64("local-member-applied-index", ep.appliedi),
  1193. zap.Uint64("local-member-snapshot-index", ep.snapi),
  1194. zap.Uint64("local-member-snapshot-count", s.Cfg.SnapshotCount),
  1195. )
  1196. } else {
  1197. plog.Infof("start to snapshot (applied: %d, lastsnap: %d)", ep.appliedi, ep.snapi)
  1198. }
  1199. s.snapshot(ep.appliedi, ep.confState)
  1200. ep.snapi = ep.appliedi
  1201. }
  1202. func (s *EtcdServer) isMultiNode() bool {
  1203. return s.cluster != nil && len(s.cluster.MemberIDs()) > 1
  1204. }
  1205. func (s *EtcdServer) isLeader() bool {
  1206. return uint64(s.ID()) == s.Lead()
  1207. }
  1208. // MoveLeader transfers the leader to the given transferee.
  1209. func (s *EtcdServer) MoveLeader(ctx context.Context, lead, transferee uint64) error {
  1210. now := time.Now()
  1211. interval := time.Duration(s.Cfg.TickMs) * time.Millisecond
  1212. if lg := s.getLogger(); lg != nil {
  1213. lg.Info(
  1214. "leadership transfer starting",
  1215. zap.String("local-member-id", s.ID().String()),
  1216. zap.String("current-leader-member-id", types.ID(lead).String()),
  1217. zap.String("transferee-member-id", types.ID(transferee).String()),
  1218. )
  1219. } else {
  1220. plog.Infof("%s starts leadership transfer from %s to %s", s.ID(), types.ID(lead), types.ID(transferee))
  1221. }
  1222. s.r.TransferLeadership(ctx, lead, transferee)
  1223. for s.Lead() != transferee {
  1224. select {
  1225. case <-ctx.Done(): // time out
  1226. return ErrTimeoutLeaderTransfer
  1227. case <-time.After(interval):
  1228. }
  1229. }
  1230. // TODO: drain all requests, or drop all messages to the old leader
  1231. if lg := s.getLogger(); lg != nil {
  1232. lg.Info(
  1233. "leadership transfer finished",
  1234. zap.String("local-member-id", s.ID().String()),
  1235. zap.String("old-leader-member-id", types.ID(lead).String()),
  1236. zap.String("new-leader-member-id", types.ID(transferee).String()),
  1237. zap.Duration("took", time.Since(now)),
  1238. )
  1239. } else {
  1240. plog.Infof("%s finished leadership transfer from %s to %s (took %v)", s.ID(), types.ID(lead), types.ID(transferee), time.Since(now))
  1241. }
  1242. return nil
  1243. }
  1244. // TransferLeadership transfers the leader to the chosen transferee.
  1245. func (s *EtcdServer) TransferLeadership() error {
  1246. if !s.isLeader() {
  1247. if lg := s.getLogger(); lg != nil {
  1248. lg.Info(
  1249. "skipped leadership transfer; local server is not leader",
  1250. zap.String("local-member-id", s.ID().String()),
  1251. zap.String("current-leader-member-id", types.ID(s.Lead()).String()),
  1252. )
  1253. } else {
  1254. plog.Printf("skipped leadership transfer for stopping non-leader member")
  1255. }
  1256. return nil
  1257. }
  1258. if !s.isMultiNode() {
  1259. if lg := s.getLogger(); lg != nil {
  1260. lg.Info(
  1261. "skipped leadership transfer; it's a single-node cluster",
  1262. zap.String("local-member-id", s.ID().String()),
  1263. zap.String("current-leader-member-id", types.ID(s.Lead()).String()),
  1264. )
  1265. } else {
  1266. plog.Printf("skipped leadership transfer for single member cluster")
  1267. }
  1268. return nil
  1269. }
  1270. transferee, ok := longestConnected(s.r.transport, s.cluster.MemberIDs())
  1271. if !ok {
  1272. return ErrUnhealthy
  1273. }
  1274. tm := s.Cfg.ReqTimeout()
  1275. ctx, cancel := context.WithTimeout(s.ctx, tm)
  1276. err := s.MoveLeader(ctx, s.Lead(), uint64(transferee))
  1277. cancel()
  1278. return err
  1279. }
  1280. // HardStop stops the server without coordination with other members in the cluster.
  1281. func (s *EtcdServer) HardStop() {
  1282. select {
  1283. case s.stop <- struct{}{}:
  1284. case <-s.done:
  1285. return
  1286. }
  1287. <-s.done
  1288. }
  1289. // Stop stops the server gracefully, and shuts down the running goroutine.
  1290. // Stop should be called after a Start(s), otherwise it will block forever.
  1291. // When stopping leader, Stop transfers its leadership to one of its peers
  1292. // before stopping the server.
  1293. // Stop terminates the Server and performs any necessary finalization.
  1294. // Do and Process cannot be called after Stop has been invoked.
  1295. func (s *EtcdServer) Stop() {
  1296. if err := s.TransferLeadership(); err != nil {
  1297. if lg := s.getLogger(); lg != nil {
  1298. lg.Warn("leadership transfer failed", zap.String("local-member-id", s.ID().String()), zap.Error(err))
  1299. } else {
  1300. plog.Warningf("%s failed to transfer leadership (%v)", s.ID(), err)
  1301. }
  1302. }
  1303. s.HardStop()
  1304. }
  1305. // ReadyNotify returns a channel that will be closed when the server
  1306. // is ready to serve client requests
  1307. func (s *EtcdServer) ReadyNotify() <-chan struct{} { return s.readych }
  1308. func (s *EtcdServer) stopWithDelay(d time.Duration, err error) {
  1309. select {
  1310. case <-time.After(d):
  1311. case <-s.done:
  1312. }
  1313. select {
  1314. case s.errorc <- err:
  1315. default:
  1316. }
  1317. }
  1318. // StopNotify returns a channel that receives a empty struct
  1319. // when the server is stopped.
  1320. func (s *EtcdServer) StopNotify() <-chan struct{} { return s.done }
  1321. func (s *EtcdServer) SelfStats() []byte { return s.stats.JSON() }
  1322. func (s *EtcdServer) LeaderStats() []byte {
  1323. lead := s.getLead()
  1324. if lead != uint64(s.id) {
  1325. return nil
  1326. }
  1327. return s.lstats.JSON()
  1328. }
  1329. func (s *EtcdServer) StoreStats() []byte { return s.v2store.JsonStats() }
  1330. func (s *EtcdServer) checkMembershipOperationPermission(ctx context.Context) error {
  1331. if s.authStore == nil {
  1332. // In the context of ordinary etcd process, s.authStore will never be nil.
  1333. // This branch is for handling cases in server_test.go
  1334. return nil
  1335. }
  1336. // Note that this permission check is done in the API layer,
  1337. // so TOCTOU problem can be caused potentially in a schedule like this:
  1338. // update membership with user A -> revoke root role of A -> apply membership change
  1339. // in the state machine layer
  1340. // However, both of membership change and role management requires the root privilege.
  1341. // So careful operation by admins can prevent the problem.
  1342. authInfo, err := s.AuthInfoFromCtx(ctx)
  1343. if err != nil {
  1344. return err
  1345. }
  1346. return s.AuthStore().IsAdminPermitted(authInfo)
  1347. }
  1348. func (s *EtcdServer) AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) {
  1349. if err := s.checkMembershipOperationPermission(ctx); err != nil {
  1350. return nil, err
  1351. }
  1352. if s.Cfg.StrictReconfigCheck {
  1353. // by default StrictReconfigCheck is enabled; reject new members if unhealthy
  1354. if !s.cluster.IsReadyToAddNewMember() {
  1355. if lg := s.getLogger(); lg != nil {
  1356. lg.Warn(
  1357. "rejecting member add request; not enough healthy members",
  1358. zap.String("local-member-id", s.ID().String()),
  1359. zap.String("requested-member-add", fmt.Sprintf("%+v", memb)),
  1360. zap.Error(ErrNotEnoughStartedMembers),
  1361. )
  1362. } else {
  1363. plog.Warningf("not enough started members, rejecting member add %+v", memb)
  1364. }
  1365. return nil, ErrNotEnoughStartedMembers
  1366. }
  1367. if !isConnectedFullySince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), s.cluster.Members()) {
  1368. if lg := s.getLogger(); lg != nil {
  1369. lg.Warn(
  1370. "rejecting member add request; local member has not been connected to all peers, reconfigure breaks active quorum",
  1371. zap.String("local-member-id", s.ID().String()),
  1372. zap.String("requested-member-add", fmt.Sprintf("%+v", memb)),
  1373. zap.Error(ErrUnhealthy),
  1374. )
  1375. } else {
  1376. plog.Warningf("not healthy for reconfigure, rejecting member add %+v", memb)
  1377. }
  1378. return nil, ErrUnhealthy
  1379. }
  1380. }
  1381. // TODO: move Member to protobuf type
  1382. b, err := json.Marshal(memb)
  1383. if err != nil {
  1384. return nil, err
  1385. }
  1386. cc := raftpb.ConfChange{
  1387. Type: raftpb.ConfChangeAddNode,
  1388. NodeID: uint64(memb.ID),
  1389. Context: b,
  1390. }
  1391. return s.configure(ctx, cc)
  1392. }
  1393. func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
  1394. if err := s.checkMembershipOperationPermission(ctx); err != nil {
  1395. return nil, err
  1396. }
  1397. // by default StrictReconfigCheck is enabled; reject removal if leads to quorum loss
  1398. if err := s.mayRemoveMember(types.ID(id)); err != nil {
  1399. return nil, err
  1400. }
  1401. cc := raftpb.ConfChange{
  1402. Type: raftpb.ConfChangeRemoveNode,
  1403. NodeID: id,
  1404. }
  1405. return s.configure(ctx, cc)
  1406. }
  1407. func (s *EtcdServer) mayRemoveMember(id types.ID) error {
  1408. if !s.Cfg.StrictReconfigCheck {
  1409. return nil
  1410. }
  1411. if !s.cluster.IsReadyToRemoveMember(uint64(id)) {
  1412. if lg := s.getLogger(); lg != nil {
  1413. lg.Warn(
  1414. "rejecting member remove request; not enough healthy members",
  1415. zap.String("local-member-id", s.ID().String()),
  1416. zap.String("requested-member-remove-id", id.String()),
  1417. zap.Error(ErrNotEnoughStartedMembers),
  1418. )
  1419. } else {
  1420. plog.Warningf("not enough started members, rejecting remove member %s", id)
  1421. }
  1422. return ErrNotEnoughStartedMembers
  1423. }
  1424. // downed member is safe to remove since it's not part of the active quorum
  1425. if t := s.r.transport.ActiveSince(id); id != s.ID() && t.IsZero() {
  1426. return nil
  1427. }
  1428. // protect quorum if some members are down
  1429. m := s.cluster.Members()
  1430. active := numConnectedSince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), m)
  1431. if (active - 1) < 1+((len(m)-1)/2) {
  1432. if lg := s.getLogger(); lg != nil {
  1433. lg.Warn(
  1434. "rejecting member remove request; local member has not been connected to all peers, reconfigure breaks active quorum",
  1435. zap.String("local-member-id", s.ID().String()),
  1436. zap.String("requested-member-remove", id.String()),
  1437. zap.Int("active-peers", active),
  1438. zap.Error(ErrUnhealthy),
  1439. )
  1440. } else {
  1441. plog.Warningf("reconfigure breaks active quorum, rejecting remove member %s", id)
  1442. }
  1443. return ErrUnhealthy
  1444. }
  1445. return nil
  1446. }
  1447. func (s *EtcdServer) UpdateMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) {
  1448. b, merr := json.Marshal(memb)
  1449. if merr != nil {
  1450. return nil, merr
  1451. }
  1452. if err := s.checkMembershipOperationPermission(ctx); err != nil {
  1453. return nil, err
  1454. }
  1455. cc := raftpb.ConfChange{
  1456. Type: raftpb.ConfChangeUpdateNode,
  1457. NodeID: uint64(memb.ID),
  1458. Context: b,
  1459. }
  1460. return s.configure(ctx, cc)
  1461. }
  1462. func (s *EtcdServer) setCommittedIndex(v uint64) {
  1463. atomic.StoreUint64(&s.committedIndex, v)
  1464. }
  1465. func (s *EtcdServer) getCommittedIndex() uint64 {
  1466. return atomic.LoadUint64(&s.committedIndex)
  1467. }
  1468. func (s *EtcdServer) setAppliedIndex(v uint64) {
  1469. atomic.StoreUint64(&s.appliedIndex, v)
  1470. }
  1471. func (s *EtcdServer) getAppliedIndex() uint64 {
  1472. return atomic.LoadUint64(&s.appliedIndex)
  1473. }
  1474. func (s *EtcdServer) setTerm(v uint64) {
  1475. atomic.StoreUint64(&s.term, v)
  1476. }
  1477. func (s *EtcdServer) getTerm() uint64 {
  1478. return atomic.LoadUint64(&s.term)
  1479. }
  1480. func (s *EtcdServer) setLead(v uint64) {
  1481. atomic.StoreUint64(&s.lead, v)
  1482. }
  1483. func (s *EtcdServer) getLead() uint64 {
  1484. return atomic.LoadUint64(&s.lead)
  1485. }
  1486. // RaftStatusGetter represents etcd server and Raft progress.
  1487. type RaftStatusGetter interface {
  1488. ID() types.ID
  1489. Leader() types.ID
  1490. CommittedIndex() uint64
  1491. AppliedIndex() uint64
  1492. Term() uint64
  1493. }
  1494. func (s *EtcdServer) ID() types.ID { return s.id }
  1495. func (s *EtcdServer) Leader() types.ID { return types.ID(s.getLead()) }
  1496. func (s *EtcdServer) Lead() uint64 { return s.getLead() }
  1497. func (s *EtcdServer) CommittedIndex() uint64 { return s.getCommittedIndex() }
  1498. func (s *EtcdServer) AppliedIndex() uint64 { return s.getAppliedIndex() }
  1499. func (s *EtcdServer) Term() uint64 { return s.getTerm() }
  1500. type confChangeResponse struct {
  1501. membs []*membership.Member
  1502. err error
  1503. }
  1504. // configure sends a configuration change through consensus and
  1505. // then waits for it to be applied to the server. It
  1506. // will block until the change is performed or there is an error.
  1507. func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) ([]*membership.Member, error) {
  1508. cc.ID = s.reqIDGen.Next()
  1509. ch := s.w.Register(cc.ID)
  1510. start := time.Now()
  1511. if err := s.r.ProposeConfChange(ctx, cc); err != nil {
  1512. s.w.Trigger(cc.ID, nil)
  1513. return nil, err
  1514. }
  1515. select {
  1516. case x := <-ch:
  1517. if x == nil {
  1518. if lg := s.getLogger(); lg != nil {
  1519. lg.Panic("failed to configure")
  1520. } else {
  1521. plog.Panicf("configure trigger value should never be nil")
  1522. }
  1523. }
  1524. resp := x.(*confChangeResponse)
  1525. if lg := s.getLogger(); lg != nil {
  1526. lg.Info(
  1527. "applied a configuration change through raft",
  1528. zap.String("local-member-id", s.ID().String()),
  1529. zap.String("raft-conf-change", cc.Type.String()),
  1530. zap.String("raft-conf-change-node-id", types.ID(cc.NodeID).String()),
  1531. )
  1532. }
  1533. return resp.membs, resp.err
  1534. case <-ctx.Done():
  1535. s.w.Trigger(cc.ID, nil) // GC wait
  1536. return nil, s.parseProposeCtxErr(ctx.Err(), start)
  1537. case <-s.stopping:
  1538. return nil, ErrStopped
  1539. }
  1540. }
  1541. // sync proposes a SYNC request and is non-blocking.
  1542. // This makes no guarantee that the request will be proposed or performed.
  1543. // The request will be canceled after the given timeout.
  1544. func (s *EtcdServer) sync(timeout time.Duration) {
  1545. req := pb.Request{
  1546. Method: "SYNC",
  1547. ID: s.reqIDGen.Next(),
  1548. Time: time.Now().UnixNano(),
  1549. }
  1550. data := pbutil.MustMarshal(&req)
  1551. // There is no promise that node has leader when do SYNC request,
  1552. // so it uses goroutine to propose.
  1553. ctx, cancel := context.WithTimeout(s.ctx, timeout)
  1554. s.goAttach(func() {
  1555. s.r.Propose(ctx, data)
  1556. cancel()
  1557. })
  1558. }
  1559. // publish registers server information into the cluster. The information
  1560. // is the JSON representation of this server's member struct, updated with the
  1561. // static clientURLs of the server.
  1562. // The function keeps attempting to register until it succeeds,
  1563. // or its server is stopped.
  1564. func (s *EtcdServer) publish(timeout time.Duration) {
  1565. b, err := json.Marshal(s.attributes)
  1566. if err != nil {
  1567. if lg := s.getLogger(); lg != nil {
  1568. lg.Panic("failed to marshal JSON", zap.Error(err))
  1569. } else {
  1570. plog.Panicf("json marshal error: %v", err)
  1571. }
  1572. return
  1573. }
  1574. req := pb.Request{
  1575. Method: "PUT",
  1576. Path: membership.MemberAttributesStorePath(s.id),
  1577. Val: string(b),
  1578. }
  1579. for {
  1580. ctx, cancel := context.WithTimeout(s.ctx, timeout)
  1581. _, err := s.Do(ctx, req)
  1582. cancel()
  1583. switch err {
  1584. case nil:
  1585. close(s.readych)
  1586. if lg := s.getLogger(); lg != nil {
  1587. lg.Info(
  1588. "published local member to cluster through raft",
  1589. zap.String("local-member-id", s.ID().String()),
  1590. zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
  1591. zap.String("request-path", req.Path),
  1592. zap.String("cluster-id", s.cluster.ID().String()),
  1593. zap.Duration("publish-timeout", timeout),
  1594. )
  1595. } else {
  1596. plog.Infof("published %+v to cluster %s", s.attributes, s.cluster.ID())
  1597. }
  1598. return
  1599. case ErrStopped:
  1600. if lg := s.getLogger(); lg != nil {
  1601. lg.Warn(
  1602. "stopped publish because server is stopped",
  1603. zap.String("local-member-id", s.ID().String()),
  1604. zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
  1605. zap.Duration("publish-timeout", timeout),
  1606. zap.Error(err),
  1607. )
  1608. } else {
  1609. plog.Infof("aborting publish because server is stopped")
  1610. }
  1611. return
  1612. default:
  1613. if lg := s.getLogger(); lg != nil {
  1614. lg.Warn(
  1615. "failed to publish local member to cluster through raft",
  1616. zap.String("local-member-id", s.ID().String()),
  1617. zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
  1618. zap.String("request-path", req.Path),
  1619. zap.Duration("publish-timeout", timeout),
  1620. zap.Error(err),
  1621. )
  1622. } else {
  1623. plog.Errorf("publish error: %v", err)
  1624. }
  1625. }
  1626. }
  1627. }
  1628. func (s *EtcdServer) sendMergedSnap(merged snap.Message) {
  1629. atomic.AddInt64(&s.inflightSnapshots, 1)
  1630. lg := s.getLogger()
  1631. fields := []zap.Field{
  1632. zap.String("from", s.ID().String()),
  1633. zap.String("to", types.ID(merged.To).String()),
  1634. zap.Int64("bytes", merged.TotalSize),
  1635. zap.String("size", humanize.Bytes(uint64(merged.TotalSize))),
  1636. }
  1637. now := time.Now()
  1638. s.r.transport.SendSnapshot(merged)
  1639. if lg != nil {
  1640. lg.Info("sending merged snapshot", fields...)
  1641. }
  1642. s.goAttach(func() {
  1643. select {
  1644. case ok := <-merged.CloseNotify():
  1645. // delay releasing inflight snapshot for another 30 seconds to
  1646. // block log compaction.
  1647. // If the follower still fails to catch up, it is probably just too slow
  1648. // to catch up. We cannot avoid the snapshot cycle anyway.
  1649. if ok {
  1650. select {
  1651. case <-time.After(releaseDelayAfterSnapshot):
  1652. case <-s.stopping:
  1653. }
  1654. }
  1655. atomic.AddInt64(&s.inflightSnapshots, -1)
  1656. if lg != nil {
  1657. lg.Info("sent merged snapshot", append(fields, zap.Duration("took", time.Since(now)))...)
  1658. }
  1659. case <-s.stopping:
  1660. if lg != nil {
  1661. lg.Warn("canceled sending merged snapshot; server stopping", fields...)
  1662. }
  1663. return
  1664. }
  1665. })
  1666. }
  1667. // apply takes entries received from Raft (after it has been committed) and
  1668. // applies them to the current state of the EtcdServer.
  1669. // The given entries should not be empty.
  1670. func (s *EtcdServer) apply(
  1671. es []raftpb.Entry,
  1672. confState *raftpb.ConfState,
  1673. ) (appliedt uint64, appliedi uint64, shouldStop bool) {
  1674. for i := range es {
  1675. e := es[i]
  1676. switch e.Type {
  1677. case raftpb.EntryNormal:
  1678. s.applyEntryNormal(&e)
  1679. s.setAppliedIndex(e.Index)
  1680. s.setTerm(e.Term)
  1681. case raftpb.EntryConfChange:
  1682. // set the consistent index of current executing entry
  1683. if e.Index > s.consistIndex.ConsistentIndex() {
  1684. s.consistIndex.setConsistentIndex(e.Index)
  1685. }
  1686. var cc raftpb.ConfChange
  1687. pbutil.MustUnmarshal(&cc, e.Data)
  1688. removedSelf, err := s.applyConfChange(cc, confState)
  1689. s.setAppliedIndex(e.Index)
  1690. s.setTerm(e.Term)
  1691. shouldStop = shouldStop || removedSelf
  1692. s.w.Trigger(cc.ID, &confChangeResponse{s.cluster.Members(), err})
  1693. default:
  1694. if lg := s.getLogger(); lg != nil {
  1695. lg.Panic(
  1696. "unknown entry type; must be either EntryNormal or EntryConfChange",
  1697. zap.String("type", e.Type.String()),
  1698. )
  1699. } else {
  1700. plog.Panicf("entry type should be either EntryNormal or EntryConfChange")
  1701. }
  1702. }
  1703. appliedi, appliedt = e.Index, e.Term
  1704. }
  1705. return appliedt, appliedi, shouldStop
  1706. }
  1707. // applyEntryNormal apples an EntryNormal type raftpb request to the EtcdServer
  1708. func (s *EtcdServer) applyEntryNormal(e *raftpb.Entry) {
  1709. shouldApplyV3 := false
  1710. if e.Index > s.consistIndex.ConsistentIndex() {
  1711. // set the consistent index of current executing entry
  1712. s.consistIndex.setConsistentIndex(e.Index)
  1713. shouldApplyV3 = true
  1714. }
  1715. // raft state machine may generate noop entry when leader confirmation.
  1716. // skip it in advance to avoid some potential bug in the future
  1717. if len(e.Data) == 0 {
  1718. select {
  1719. case s.forceVersionC <- struct{}{}:
  1720. default:
  1721. }
  1722. // promote lessor when the local member is leader and finished
  1723. // applying all entries from the last term.
  1724. if s.isLeader() {
  1725. s.lessor.Promote(s.Cfg.electionTimeout())
  1726. }
  1727. return
  1728. }
  1729. var raftReq pb.InternalRaftRequest
  1730. if !pbutil.MaybeUnmarshal(&raftReq, e.Data) { // backward compatible
  1731. var r pb.Request
  1732. rp := &r
  1733. pbutil.MustUnmarshal(rp, e.Data)
  1734. s.w.Trigger(r.ID, s.applyV2Request((*RequestV2)(rp)))
  1735. return
  1736. }
  1737. if raftReq.V2 != nil {
  1738. req := (*RequestV2)(raftReq.V2)
  1739. s.w.Trigger(req.ID, s.applyV2Request(req))
  1740. return
  1741. }
  1742. // do not re-apply applied entries.
  1743. if !shouldApplyV3 {
  1744. return
  1745. }
  1746. id := raftReq.ID
  1747. if id == 0 {
  1748. id = raftReq.Header.ID
  1749. }
  1750. var ar *applyResult
  1751. needResult := s.w.IsRegistered(id)
  1752. if needResult || !noSideEffect(&raftReq) {
  1753. if !needResult && raftReq.Txn != nil {
  1754. removeNeedlessRangeReqs(raftReq.Txn)
  1755. }
  1756. ar = s.applyV3.Apply(&raftReq)
  1757. }
  1758. if ar == nil {
  1759. return
  1760. }
  1761. if ar.err != ErrNoSpace || len(s.alarmStore.Get(pb.AlarmType_NOSPACE)) > 0 {
  1762. s.w.Trigger(id, ar)
  1763. return
  1764. }
  1765. if lg := s.getLogger(); lg != nil {
  1766. lg.Warn(
  1767. "message exceeded backend quota; raising alarm",
  1768. zap.Int64("quota-size-bytes", s.Cfg.QuotaBackendBytes),
  1769. zap.String("quota-size", humanize.Bytes(uint64(s.Cfg.QuotaBackendBytes))),
  1770. zap.Error(ar.err),
  1771. )
  1772. } else {
  1773. plog.Errorf("applying raft message exceeded backend quota")
  1774. }
  1775. s.goAttach(func() {
  1776. a := &pb.AlarmRequest{
  1777. MemberID: uint64(s.ID()),
  1778. Action: pb.AlarmRequest_ACTIVATE,
  1779. Alarm: pb.AlarmType_NOSPACE,
  1780. }
  1781. s.raftRequest(s.ctx, pb.InternalRaftRequest{Alarm: a})
  1782. s.w.Trigger(id, ar)
  1783. })
  1784. }
  1785. // applyConfChange applies a ConfChange to the server. It is only
  1786. // invoked with a ConfChange that has already passed through Raft
  1787. func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.ConfState) (bool, error) {
  1788. if err := s.cluster.ValidateConfigurationChange(cc); err != nil {
  1789. cc.NodeID = raft.None
  1790. s.r.ApplyConfChange(cc)
  1791. return false, err
  1792. }
  1793. lg := s.getLogger()
  1794. *confState = *s.r.ApplyConfChange(cc)
  1795. switch cc.Type {
  1796. case raftpb.ConfChangeAddNode:
  1797. m := new(membership.Member)
  1798. if err := json.Unmarshal(cc.Context, m); err != nil {
  1799. if lg != nil {
  1800. lg.Panic("failed to unmarshal member", zap.Error(err))
  1801. } else {
  1802. plog.Panicf("unmarshal member should never fail: %v", err)
  1803. }
  1804. }
  1805. if cc.NodeID != uint64(m.ID) {
  1806. if lg != nil {
  1807. lg.Panic(
  1808. "got different member ID",
  1809. zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()),
  1810. zap.String("member-id-from-message", m.ID.String()),
  1811. )
  1812. } else {
  1813. plog.Panicf("nodeID should always be equal to member ID")
  1814. }
  1815. }
  1816. s.cluster.AddMember(m)
  1817. if m.ID != s.id {
  1818. s.r.transport.AddPeer(m.ID, m.PeerURLs)
  1819. }
  1820. case raftpb.ConfChangeRemoveNode:
  1821. id := types.ID(cc.NodeID)
  1822. s.cluster.RemoveMember(id)
  1823. if id == s.id {
  1824. return true, nil
  1825. }
  1826. s.r.transport.RemovePeer(id)
  1827. case raftpb.ConfChangeUpdateNode:
  1828. m := new(membership.Member)
  1829. if err := json.Unmarshal(cc.Context, m); err != nil {
  1830. if lg != nil {
  1831. lg.Panic("failed to unmarshal member", zap.Error(err))
  1832. } else {
  1833. plog.Panicf("unmarshal member should never fail: %v", err)
  1834. }
  1835. }
  1836. if cc.NodeID != uint64(m.ID) {
  1837. if lg != nil {
  1838. lg.Panic(
  1839. "got different member ID",
  1840. zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()),
  1841. zap.String("member-id-from-message", m.ID.String()),
  1842. )
  1843. } else {
  1844. plog.Panicf("nodeID should always be equal to member ID")
  1845. }
  1846. }
  1847. s.cluster.UpdateRaftAttributes(m.ID, m.RaftAttributes)
  1848. if m.ID != s.id {
  1849. s.r.transport.UpdatePeer(m.ID, m.PeerURLs)
  1850. }
  1851. }
  1852. return false, nil
  1853. }
  1854. // TODO: non-blocking snapshot
  1855. func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
  1856. clone := s.v2store.Clone()
  1857. // commit kv to write metadata (for example: consistent index) to disk.
  1858. // KV().commit() updates the consistent index in backend.
  1859. // All operations that update consistent index must be called sequentially
  1860. // from applyAll function.
  1861. // So KV().Commit() cannot run in parallel with apply. It has to be called outside
  1862. // the go routine created below.
  1863. s.KV().Commit()
  1864. s.goAttach(func() {
  1865. lg := s.getLogger()
  1866. d, err := clone.SaveNoCopy()
  1867. // TODO: current store will never fail to do a snapshot
  1868. // what should we do if the store might fail?
  1869. if err != nil {
  1870. if lg != nil {
  1871. lg.Panic("failed to save v2 store", zap.Error(err))
  1872. } else {
  1873. plog.Panicf("store save should never fail: %v", err)
  1874. }
  1875. }
  1876. snap, err := s.r.raftStorage.CreateSnapshot(snapi, &confState, d)
  1877. if err != nil {
  1878. // the snapshot was done asynchronously with the progress of raft.
  1879. // raft might have already got a newer snapshot.
  1880. if err == raft.ErrSnapOutOfDate {
  1881. return
  1882. }
  1883. if lg != nil {
  1884. lg.Panic("failed to create snapshot", zap.Error(err))
  1885. } else {
  1886. plog.Panicf("unexpected create snapshot error %v", err)
  1887. }
  1888. }
  1889. // SaveSnap saves the snapshot and releases the locked wal files
  1890. // to the snapshot index.
  1891. if err = s.r.storage.SaveSnap(snap); err != nil {
  1892. if lg != nil {
  1893. lg.Panic("failed to save snapshot", zap.Error(err))
  1894. } else {
  1895. plog.Fatalf("save snapshot error: %v", err)
  1896. }
  1897. }
  1898. if lg != nil {
  1899. lg.Info(
  1900. "saved snapshot",
  1901. zap.Uint64("snapshot-index", snap.Metadata.Index),
  1902. )
  1903. } else {
  1904. plog.Infof("saved snapshot at index %d", snap.Metadata.Index)
  1905. }
  1906. // When sending a snapshot, etcd will pause compaction.
  1907. // After receives a snapshot, the slow follower needs to get all the entries right after
  1908. // the snapshot sent to catch up. If we do not pause compaction, the log entries right after
  1909. // the snapshot sent might already be compacted. It happens when the snapshot takes long time
  1910. // to send and save. Pausing compaction avoids triggering a snapshot sending cycle.
  1911. if atomic.LoadInt64(&s.inflightSnapshots) != 0 {
  1912. if lg != nil {
  1913. lg.Info("skip compaction since there is an inflight snapshot")
  1914. } else {
  1915. plog.Infof("skip compaction since there is an inflight snapshot")
  1916. }
  1917. return
  1918. }
  1919. // keep some in memory log entries for slow followers.
  1920. compacti := uint64(1)
  1921. if snapi > s.Cfg.SnapshotCatchUpEntries {
  1922. compacti = snapi - s.Cfg.SnapshotCatchUpEntries
  1923. }
  1924. err = s.r.raftStorage.Compact(compacti)
  1925. if err != nil {
  1926. // the compaction was done asynchronously with the progress of raft.
  1927. // raft log might already been compact.
  1928. if err == raft.ErrCompacted {
  1929. return
  1930. }
  1931. if lg != nil {
  1932. lg.Panic("failed to compact", zap.Error(err))
  1933. } else {
  1934. plog.Panicf("unexpected compaction error %v", err)
  1935. }
  1936. }
  1937. if lg != nil {
  1938. lg.Info(
  1939. "compacted Raft logs",
  1940. zap.Uint64("compact-index", compacti),
  1941. )
  1942. } else {
  1943. plog.Infof("compacted raft log at %d", compacti)
  1944. }
  1945. })
  1946. }
  1947. // CutPeer drops messages to the specified peer.
  1948. func (s *EtcdServer) CutPeer(id types.ID) {
  1949. tr, ok := s.r.transport.(*rafthttp.Transport)
  1950. if ok {
  1951. tr.CutPeer(id)
  1952. }
  1953. }
  1954. // MendPeer recovers the message dropping behavior of the given peer.
  1955. func (s *EtcdServer) MendPeer(id types.ID) {
  1956. tr, ok := s.r.transport.(*rafthttp.Transport)
  1957. if ok {
  1958. tr.MendPeer(id)
  1959. }
  1960. }
  1961. func (s *EtcdServer) PauseSending() { s.r.pauseSending() }
  1962. func (s *EtcdServer) ResumeSending() { s.r.resumeSending() }
  1963. func (s *EtcdServer) ClusterVersion() *semver.Version {
  1964. if s.cluster == nil {
  1965. return nil
  1966. }
  1967. return s.cluster.Version()
  1968. }
  1969. // monitorVersions checks the member's version every monitorVersionInterval.
  1970. // It updates the cluster version if all members agrees on a higher one.
  1971. // It prints out log if there is a member with a higher version than the
  1972. // local version.
  1973. func (s *EtcdServer) monitorVersions() {
  1974. for {
  1975. select {
  1976. case <-s.forceVersionC:
  1977. case <-time.After(monitorVersionInterval):
  1978. case <-s.stopping:
  1979. return
  1980. }
  1981. if s.Leader() != s.ID() {
  1982. continue
  1983. }
  1984. v := decideClusterVersion(s.getLogger(), getVersions(s.getLogger(), s.cluster, s.id, s.peerRt))
  1985. if v != nil {
  1986. // only keep major.minor version for comparison
  1987. v = &semver.Version{
  1988. Major: v.Major,
  1989. Minor: v.Minor,
  1990. }
  1991. }
  1992. // if the current version is nil:
  1993. // 1. use the decided version if possible
  1994. // 2. or use the min cluster version
  1995. if s.cluster.Version() == nil {
  1996. verStr := version.MinClusterVersion
  1997. if v != nil {
  1998. verStr = v.String()
  1999. }
  2000. s.goAttach(func() { s.updateClusterVersion(verStr) })
  2001. continue
  2002. }
  2003. // update cluster version only if the decided version is greater than
  2004. // the current cluster version
  2005. if v != nil && s.cluster.Version().LessThan(*v) {
  2006. s.goAttach(func() { s.updateClusterVersion(v.String()) })
  2007. }
  2008. }
  2009. }
  2010. func (s *EtcdServer) updateClusterVersion(ver string) {
  2011. lg := s.getLogger()
  2012. if s.cluster.Version() == nil {
  2013. if lg != nil {
  2014. lg.Info(
  2015. "setting up initial cluster version",
  2016. zap.String("cluster-version", version.Cluster(ver)),
  2017. )
  2018. } else {
  2019. plog.Infof("setting up the initial cluster version to %s", version.Cluster(ver))
  2020. }
  2021. } else {
  2022. if lg != nil {
  2023. lg.Info(
  2024. "updating cluster version",
  2025. zap.String("from", version.Cluster(s.cluster.Version().String())),
  2026. zap.String("to", version.Cluster(ver)),
  2027. )
  2028. } else {
  2029. plog.Infof("updating the cluster version from %s to %s", version.Cluster(s.cluster.Version().String()), version.Cluster(ver))
  2030. }
  2031. }
  2032. req := pb.Request{
  2033. Method: "PUT",
  2034. Path: membership.StoreClusterVersionKey(),
  2035. Val: ver,
  2036. }
  2037. ctx, cancel := context.WithTimeout(s.ctx, s.Cfg.ReqTimeout())
  2038. _, err := s.Do(ctx, req)
  2039. cancel()
  2040. switch err {
  2041. case nil:
  2042. if lg != nil {
  2043. lg.Info("cluster version is updated", zap.String("cluster-version", version.Cluster(ver)))
  2044. }
  2045. return
  2046. case ErrStopped:
  2047. if lg != nil {
  2048. lg.Warn("aborting cluster version update; server is stopped", zap.Error(err))
  2049. } else {
  2050. plog.Infof("aborting update cluster version because server is stopped")
  2051. }
  2052. return
  2053. default:
  2054. if lg != nil {
  2055. lg.Warn("failed to update cluster version", zap.Error(err))
  2056. } else {
  2057. plog.Errorf("error updating cluster version (%v)", err)
  2058. }
  2059. }
  2060. }
  2061. func (s *EtcdServer) parseProposeCtxErr(err error, start time.Time) error {
  2062. switch err {
  2063. case context.Canceled:
  2064. return ErrCanceled
  2065. case context.DeadlineExceeded:
  2066. s.leadTimeMu.RLock()
  2067. curLeadElected := s.leadElectedTime
  2068. s.leadTimeMu.RUnlock()
  2069. prevLeadLost := curLeadElected.Add(-2 * time.Duration(s.Cfg.ElectionTicks) * time.Duration(s.Cfg.TickMs) * time.Millisecond)
  2070. if start.After(prevLeadLost) && start.Before(curLeadElected) {
  2071. return ErrTimeoutDueToLeaderFail
  2072. }
  2073. lead := types.ID(s.getLead())
  2074. switch lead {
  2075. case types.ID(raft.None):
  2076. // TODO: return error to specify it happens because the cluster does not have leader now
  2077. case s.ID():
  2078. if !isConnectedToQuorumSince(s.r.transport, start, s.ID(), s.cluster.Members()) {
  2079. return ErrTimeoutDueToConnectionLost
  2080. }
  2081. default:
  2082. if !isConnectedSince(s.r.transport, start, lead) {
  2083. return ErrTimeoutDueToConnectionLost
  2084. }
  2085. }
  2086. return ErrTimeout
  2087. default:
  2088. return err
  2089. }
  2090. }
  2091. func (s *EtcdServer) KV() mvcc.ConsistentWatchableKV { return s.kv }
  2092. func (s *EtcdServer) Backend() backend.Backend {
  2093. s.bemu.Lock()
  2094. defer s.bemu.Unlock()
  2095. return s.be
  2096. }
  2097. func (s *EtcdServer) AuthStore() auth.AuthStore { return s.authStore }
  2098. func (s *EtcdServer) restoreAlarms() error {
  2099. s.applyV3 = s.newApplierV3()
  2100. as, err := v3alarm.NewAlarmStore(s)
  2101. if err != nil {
  2102. return err
  2103. }
  2104. s.alarmStore = as
  2105. if len(as.Get(pb.AlarmType_NOSPACE)) > 0 {
  2106. s.applyV3 = newApplierV3Capped(s.applyV3)
  2107. }
  2108. if len(as.Get(pb.AlarmType_CORRUPT)) > 0 {
  2109. s.applyV3 = newApplierV3Corrupt(s.applyV3)
  2110. }
  2111. return nil
  2112. }
  2113. // goAttach creates a goroutine on a given function and tracks it using
  2114. // the etcdserver waitgroup.
  2115. func (s *EtcdServer) goAttach(f func()) {
  2116. s.wgMu.RLock() // this blocks with ongoing close(s.stopping)
  2117. defer s.wgMu.RUnlock()
  2118. select {
  2119. case <-s.stopping:
  2120. if lg := s.getLogger(); lg != nil {
  2121. lg.Warn("server has stopped; skipping goAttach")
  2122. } else {
  2123. plog.Warning("server has stopped (skipping goAttach)")
  2124. }
  2125. return
  2126. default:
  2127. }
  2128. // now safe to add since waitgroup wait has not started yet
  2129. s.wg.Add(1)
  2130. go func() {
  2131. defer s.wg.Done()
  2132. f()
  2133. }()
  2134. }
  2135. func (s *EtcdServer) Alarms() []*pb.AlarmMember {
  2136. return s.alarmStore.Get(pb.AlarmType_NONE)
  2137. }