server.go 51 KB

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