github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/election/elector.go (about) 1 // Copyright 2022 PingCAP, Inc. 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 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package election 15 16 import ( 17 "context" 18 "math/rand" 19 "sync" 20 "time" 21 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/pkg/errors" 24 "github.com/pingcap/tiflow/pkg/retry" 25 "go.uber.org/atomic" 26 "go.uber.org/zap" 27 ) 28 29 func init() { 30 rand.Seed(time.Now().UnixNano()) 31 } 32 33 // Elector is a leader election client. 34 type Elector interface { 35 // RunElection runs the elector to continuously campaign for leadership 36 // until the context is canceled. 37 RunElection(ctx context.Context) error 38 // IsLeader returns true if the current member is the leader 39 // and its lease is still valid. 40 IsLeader() bool 41 // GetLeader returns the last observed leader whose lease is still valid. 42 GetLeader() (*Member, bool) 43 // GetMembers returns all members. 44 GetMembers() []*Member 45 // ResignLeader resigns the leadership and let the elector 46 // not to try to campaign for leadership during the duration. 47 ResignLeader(ctx context.Context, duration time.Duration) error 48 } 49 50 // NewElector creates a new Elector. 51 func NewElector(config Config) (Elector, error) { 52 if err := config.AdjustAndValidate(); err != nil { 53 return nil, err 54 } 55 return &electorImpl{ 56 config: config, 57 observedRenews: make(map[string]time.Time), 58 resignCh: make(chan *resignReq), 59 }, nil 60 } 61 62 // electorImpl is the default implementation of Elector. 63 type electorImpl struct { 64 config Config 65 66 observeLock sync.RWMutex 67 observedRecord Record 68 // observedRenews is a map of renew time of each member. 69 // Note that the time is not RenewTime recorded in the record, 70 // but the time when we observed the renewal. This is because 71 // absolute time is not reliable across different machines. 72 observedRenews map[string]time.Time 73 74 // resignCh is used to notify the elector to resign leadership. 75 resignCh chan *resignReq 76 // elector will not be leader until this time. 77 resignUntil time.Time 78 79 callbackWg sync.WaitGroup 80 callbackIsRunning atomic.Bool 81 callbackCancelFn context.CancelFunc 82 } 83 84 type resignReq struct { 85 ctx context.Context 86 duration time.Duration 87 errCh chan error 88 } 89 90 // RunElection implements Elector.RunElection. 91 func (e *electorImpl) RunElection(ctx context.Context) error { 92 ctx, cancel := context.WithCancel(ctx) 93 defer cancel() 94 for { 95 if err := e.renew(ctx); err != nil { 96 log.Warn("failed to renew lease after renew deadline", zap.Error(err), 97 zap.Duration("renewDeadline", e.config.RenewDeadline)) 98 e.cancelCallback("renew lease failed") 99 if e.config.ExitOnRenewFail { 100 cancel() 101 } 102 } else if e.IsLeader() { 103 e.ensureCallbackIsRunning(ctx) 104 } else { 105 e.cancelCallback("not leader") 106 } 107 108 select { 109 case <-ctx.Done(): 110 if err := e.release(context.Background(), true /* removeSelf */); err != nil { 111 log.Warn("failed to release member lease", zap.Error(err)) 112 } 113 e.cancelCallback(ctx.Err().Error()) 114 return ctx.Err() 115 case req := <-e.resignCh: 116 if e.IsLeader() { 117 log.Info("try to resign leadership") 118 if err := e.release(req.ctx, false /* removeSelf */); err != nil { 119 log.Warn("failed to resign leadership", zap.Error(err)) 120 req.errCh <- err 121 } else { 122 req.errCh <- nil 123 e.resignUntil = time.Now().Add(req.duration) 124 e.cancelCallback("leader resigned") 125 } 126 } else { 127 req.errCh <- nil 128 e.resignUntil = time.Now().Add(req.duration) 129 } 130 case <-time.After(e.config.RenewInterval): 131 } 132 } 133 } 134 135 func (e *electorImpl) renew(ctx context.Context) (err error) { 136 start := time.Now() 137 defer func() { 138 log.Debug("renew", zap.Duration("cost", time.Since(start)), zap.Error(err)) 139 }() 140 141 ctx, cancel := context.WithTimeout(ctx, e.config.RenewDeadline) 142 defer cancel() 143 144 return e.updateRecord(ctx, func(record *Record) (error, bool) { 145 var activeMembers []*Member 146 var isLeaderChanged bool 147 for _, m := range record.Members { 148 if e.isLeaseExpired(m.ID) { 149 if m.ID == record.LeaderID { 150 record.LeaderID = "" 151 isLeaderChanged = true 152 log.Info( 153 "leader lease expired", 154 zap.String("leaderID", m.ID), 155 zap.String("leaderName", m.Name), 156 zap.String("leaderAddress", m.Address), 157 ) 158 } else { 159 log.Info( 160 "member lease expired", 161 zap.String("memberID", m.ID), 162 zap.String("memberName", m.Name), 163 zap.String("memberAddress", m.Address), 164 ) 165 } 166 } else { 167 activeMembers = append(activeMembers, m) 168 } 169 } 170 record.Members = activeMembers 171 172 // Add self to the record if not exists. 173 if m, ok := record.FindMember(e.config.ID); !ok { 174 record.Members = append(record.Members, &Member{ 175 ID: e.config.ID, 176 Name: e.config.Name, 177 Address: e.config.Address, 178 LeaseDuration: e.config.LeaseDuration, 179 RenewTime: time.Now(), 180 }) 181 } else { 182 m.RenewTime = time.Now() 183 } 184 185 if time.Now().Before(e.resignUntil) { 186 if record.LeaderID == e.config.ID { 187 record.LeaderID = "" 188 isLeaderChanged = true 189 log.Info("try to resign leadership") 190 } 191 } else if record.LeaderID == "" { 192 // Elect a new leader if no leader exists. 193 record.LeaderID = e.config.ID 194 isLeaderChanged = true 195 log.Info( 196 "try to elect self as leader", 197 zap.String("id", e.config.ID), 198 zap.String("name", e.config.Name), 199 zap.String("address", e.config.Address), 200 ) 201 } 202 return nil, isLeaderChanged 203 }) 204 } 205 206 func (e *electorImpl) ensureCallbackIsRunning(ctx context.Context) { 207 if !e.callbackIsRunning.Load() { 208 leaderCallback := e.config.LeaderCallback 209 leaderCtx, leaderCancel := context.WithCancel(ctx) 210 e.callbackWg.Add(1) 211 e.callbackIsRunning.Store(true) 212 go func() { 213 defer func() { 214 e.callbackIsRunning.Store(false) 215 e.callbackWg.Done() 216 leaderCancel() 217 }() 218 log.Info("leader callback is called") 219 err := leaderCallback(leaderCtx) 220 if errors.Cause(err) != context.Canceled { 221 log.Warn("leader callback is unexpectedly exited", zap.Error(err)) 222 if e.IsLeader() { 223 log.Info("try to resign leadership") 224 if err := e.release(context.Background(), false /* removeSelf */); err != nil { 225 log.Warn("failed to resign leadership", zap.Error(err)) 226 } 227 } 228 } 229 }() 230 e.callbackCancelFn = leaderCancel 231 } 232 } 233 234 func (e *electorImpl) cancelCallback(reason string) { 235 if e.callbackIsRunning.Load() { 236 log.Info("cancel leader callback", zap.String("reason", reason)) 237 start := time.Now() 238 e.callbackCancelFn() 239 e.callbackWg.Wait() 240 log.Info("leader callback is canceled", zap.Duration("took", time.Since(start))) 241 } 242 } 243 244 func (e *electorImpl) release(ctx context.Context, removeSelf bool) error { 245 ctx, cancel := context.WithTimeout(ctx, defaultReleaseTimeout) 246 defer cancel() 247 248 return e.updateRecord(ctx, func(record *Record) (error, bool) { 249 var isLeaderChanged bool 250 if record.LeaderID == e.config.ID { 251 record.LeaderID = "" 252 isLeaderChanged = true 253 } 254 if removeSelf { 255 for i, m := range record.Members { 256 if m.ID == e.config.ID { 257 record.Members = append(record.Members[:i], record.Members[i+1:]...) 258 break 259 } 260 } 261 } 262 return nil, isLeaderChanged 263 }) 264 } 265 266 func (e *electorImpl) updateRecord( 267 ctx context.Context, 268 f func(*Record) (err error, isLeaderChanged bool), 269 ) error { 270 // Divide 2 is for more retries. 271 backoffBaseDelayInMs := int64(e.config.RenewInterval/time.Millisecond) / 2 272 // Make sure the retry delay is less than the deadline, otherwise the retry has no chance to execute. 273 backoffMaxDelayInMs := int64(e.config.RenewDeadline/time.Millisecond) / 2 274 if deadline, ok := ctx.Deadline(); ok { 275 maxDelayForCtx := int64(time.Until(deadline)/time.Millisecond) / 2 276 if maxDelayForCtx < backoffMaxDelayInMs { 277 backoffMaxDelayInMs = maxDelayForCtx 278 } 279 } 280 281 return retry.Do(ctx, func() error { 282 s := e.config.Storage 283 record, err := s.Get(ctx) 284 if err != nil { 285 return errors.Trace(err) 286 } 287 e.setObservedRecord(record) 288 289 var isLeaderChanged bool 290 if err, isLeaderChanged = f(record); err != nil { 291 return errors.Trace(err) 292 } 293 294 if err := s.Update(ctx, record, isLeaderChanged); err != nil { 295 return errors.Trace(err) 296 } 297 e.setObservedRecord(record) 298 return nil 299 }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), 300 retry.WithBackoffMaxDelay(backoffMaxDelayInMs), 301 retry.WithIsRetryableErr(func(err error) bool { 302 if errors.Is(err, errors.ErrElectionRecordConflict) { 303 log.Info("conflict encountered while updating record, retrying") 304 } else { 305 log.Warn("failed to update record, retrying", zap.Error(err)) 306 } 307 return true // For log only, retry doesn't rely on it. 308 }), 309 ) 310 } 311 312 func (e *electorImpl) setObservedRecord(record *Record) { 313 e.observeLock.Lock() 314 defer e.observeLock.Unlock() 315 316 // Remove members that are not in the new record. 317 for id := range e.observedRenews { 318 if _, ok := record.FindMember(id); !ok { 319 delete(e.observedRenews, id) 320 } 321 } 322 323 // Update observedRenews for members in the new record. 324 for _, m := range record.Members { 325 oldMember, ok := e.observedRecord.FindMember(m.ID) 326 // If the member is not in the old record, or the RenewTime is 327 // changed, update the local observedRenews. 328 if !ok || !oldMember.RenewTime.Equal(m.RenewTime) { 329 e.observedRenews[m.ID] = time.Now() 330 } 331 } 332 333 // New leader is elected. 334 if record.LeaderID != "" && record.LeaderID != e.observedRecord.LeaderID { 335 leader, ok := record.FindMember(record.LeaderID) 336 if ok { 337 log.Info( 338 "new leader elected", 339 zap.String("leaderID", leader.ID), 340 zap.String("leaderName", leader.Name), 341 zap.String("leaderAddress", leader.Address), 342 ) 343 } 344 } 345 346 e.observedRecord = *record.Clone() 347 } 348 349 func (e *electorImpl) isLeaseExpired(memberID string) bool { 350 e.observeLock.RLock() 351 defer e.observeLock.RUnlock() 352 353 return e.isLeaseExpiredLocked(memberID) 354 } 355 356 func (e *electorImpl) isLeaseExpiredLocked(memberID string) bool { 357 member, ok := e.observedRecord.FindMember(memberID) 358 if !ok { 359 return true 360 } 361 renewTime := e.observedRenews[memberID] 362 return renewTime.Add(member.LeaseDuration).Before(time.Now()) 363 } 364 365 // IsLeader implements the Elector.IsLeader. 366 func (e *electorImpl) IsLeader() bool { 367 e.observeLock.RLock() 368 defer e.observeLock.RUnlock() 369 370 if e.isLeaseExpiredLocked(e.config.ID) { 371 return false 372 } 373 return e.observedRecord.LeaderID == e.config.ID 374 } 375 376 // GetLeader implements the Elector.GetLeader. 377 func (e *electorImpl) GetLeader() (*Member, bool) { 378 e.observeLock.RLock() 379 defer e.observeLock.RUnlock() 380 381 leader, ok := e.observedRecord.FindMember(e.observedRecord.LeaderID) 382 if ok && !e.isLeaseExpiredLocked(leader.ID) { 383 return leader.Clone(), true 384 } 385 return nil, false 386 } 387 388 // GetMembers implements the Elector.GetMembers. 389 func (e *electorImpl) GetMembers() []*Member { 390 e.observeLock.RLock() 391 defer e.observeLock.RUnlock() 392 393 members := make([]*Member, 0, len(e.observedRecord.Members)) 394 for _, m := range e.observedRecord.Members { 395 members = append(members, m.Clone()) 396 } 397 return members 398 } 399 400 // ResignLeader implements the Elector.ResignLeader. 401 func (e *electorImpl) ResignLeader(ctx context.Context, duration time.Duration) error { 402 ctx, cancel := context.WithTimeout(ctx, defaultResignTimeout) 403 defer cancel() 404 405 req := &resignReq{ 406 ctx: ctx, 407 duration: duration, 408 errCh: make(chan error, 1), 409 } 410 411 select { 412 case <-ctx.Done(): 413 return ctx.Err() 414 case e.resignCh <- req: 415 return <-req.errCh 416 } 417 }