github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/election/election.go (about)

     1  // Copyright 2019 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  	"encoding/json"
    19  	"fmt"
    20  	"math"
    21  	"sync"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/failpoint"
    26  	"github.com/pingcap/tiflow/dm/pkg/log"
    27  	"github.com/pingcap/tiflow/dm/pkg/terror"
    28  	"go.etcd.io/etcd/api/v3/mvccpb"
    29  	clientv3 "go.etcd.io/etcd/client/v3"
    30  	"go.etcd.io/etcd/client/v3/concurrency"
    31  	"go.uber.org/atomic"
    32  	"go.uber.org/zap"
    33  )
    34  
    35  const (
    36  	// newSessionDefaultRetryCnt is the default retry times when creating new session.
    37  	newSessionDefaultRetryCnt = 3
    38  	// newSessionRetryUnlimited is the unlimited retry times when creating new session.
    39  	newSessionRetryUnlimited = math.MaxInt64
    40  	// newSessionRetryInterval is the interval time when retrying to create a new session.
    41  	newSessionRetryInterval = 200 * time.Millisecond
    42  
    43  	// IsLeader means current compaigner become leader.
    44  	IsLeader = "isLeader"
    45  	// RetireFromLeader means current compaigner is old leader, and retired.
    46  	RetireFromLeader = "retireFromLeader"
    47  	// IsNotLeader means current compaigner is not old leader and current leader.
    48  	IsNotLeader = "isNotLeader"
    49  )
    50  
    51  // CampaignerInfo is the campaigner's information.
    52  type CampaignerInfo struct {
    53  	ID string `json:"id"`
    54  	// addr is the campaigner's advertise address
    55  	Addr string `json:"addr"`
    56  }
    57  
    58  func (c *CampaignerInfo) String() string {
    59  	infoBytes, err := json.Marshal(c)
    60  	if err != nil {
    61  		// this should never happened
    62  		return fmt.Sprintf("id: %s, addr: %s", c.ID, c.Addr)
    63  	}
    64  
    65  	return string(infoBytes)
    66  }
    67  
    68  func getCampaignerInfo(infoBytes []byte) (*CampaignerInfo, error) {
    69  	info := &CampaignerInfo{}
    70  	err := json.Unmarshal(infoBytes, info)
    71  	if err != nil {
    72  		return nil, err
    73  	}
    74  	return info, nil
    75  }
    76  
    77  // Election implements the leader election based on etcd.
    78  type Election struct {
    79  	// the Election instance does not own the client instance,
    80  	// so do not close it in the methods of Election.
    81  	cli        *clientv3.Client
    82  	sessionTTL int
    83  	key        string
    84  
    85  	info    *CampaignerInfo
    86  	infoStr string
    87  
    88  	ech      chan error
    89  	leaderCh chan *CampaignerInfo
    90  	isLeader atomic.Bool
    91  
    92  	closed atomic.Bool
    93  	cancel context.CancelFunc
    94  
    95  	bgWg sync.WaitGroup
    96  
    97  	campaignMu sync.RWMutex
    98  
    99  	cancelCampaign func()
   100  
   101  	// notifyBlockTime is the max block time for notify leader
   102  	notifyBlockTime time.Duration
   103  
   104  	// set evictLeader to true if don't hope this member be leader
   105  	evictLeader atomic.Bool
   106  
   107  	resignCh chan struct{}
   108  
   109  	l log.Logger
   110  }
   111  
   112  // NewElection creates a new etcd leader Election instance and starts the campaign loop.
   113  func NewElection(ctx context.Context, cli *clientv3.Client, sessionTTL int, key, id, addr string, notifyBlockTime time.Duration) (*Election, error) {
   114  	ctx2, cancel2 := context.WithCancel(ctx)
   115  	e := &Election{
   116  		cli:        cli,
   117  		sessionTTL: sessionTTL,
   118  		key:        key,
   119  		info: &CampaignerInfo{
   120  			ID:   id,
   121  			Addr: addr,
   122  		},
   123  		notifyBlockTime: notifyBlockTime,
   124  		leaderCh:        make(chan *CampaignerInfo, 1),
   125  		ech:             make(chan error, 1), // size 1 is enough
   126  		cancel:          cancel2,
   127  		l:               log.With(zap.String("component", "election")),
   128  	}
   129  	e.infoStr = e.info.String()
   130  
   131  	// try create a session before enter the campaign loop.
   132  	// so we can detect potential error earlier.
   133  	session, err := e.newSession(ctx, newSessionDefaultRetryCnt)
   134  	if err != nil {
   135  		cancel2()
   136  		return nil, terror.ErrElectionCampaignFail.Delegate(err, "create the initial session")
   137  	}
   138  
   139  	e.bgWg.Add(1)
   140  	go func() {
   141  		defer e.bgWg.Done()
   142  		e.campaignLoop(ctx2, session)
   143  	}()
   144  	return e, nil
   145  }
   146  
   147  // IsLeader returns whether this member is the leader.
   148  func (e *Election) IsLeader() bool {
   149  	return e.isLeader.Load()
   150  }
   151  
   152  // ID returns the current member's ID.
   153  func (e *Election) ID() string {
   154  	return e.info.ID
   155  }
   156  
   157  // LeaderInfo returns the current leader's key, ID and address.
   158  // it's similar with https://github.com/etcd-io/etcd/blob/v3.4.3/clientv3/concurrency/election.go#L147.
   159  func (e *Election) LeaderInfo(ctx context.Context) (string, string, string, error) {
   160  	resp, err := e.cli.Get(ctx, e.key, clientv3.WithFirstCreate()...)
   161  	if err != nil {
   162  		return "", "", "", terror.ErrElectionGetLeaderIDFail.Delegate(err)
   163  	} else if len(resp.Kvs) == 0 {
   164  		// no leader currently elected
   165  		return "", "", "", terror.ErrElectionGetLeaderIDFail.Delegate(concurrency.ErrElectionNoLeader)
   166  	}
   167  
   168  	leaderInfo, err := getCampaignerInfo(resp.Kvs[0].Value)
   169  	if err != nil {
   170  		return "", "", "", terror.ErrElectionGetLeaderIDFail.Delegate(err)
   171  	}
   172  
   173  	return string(resp.Kvs[0].Key), leaderInfo.ID, leaderInfo.Addr, nil
   174  }
   175  
   176  // LeaderNotify returns a channel that can fetch the leader's information when the member become the leader or retire from the leader, or get a new leader.
   177  // leader's information can be nil which means this member is leader and retire.
   178  func (e *Election) LeaderNotify() <-chan *CampaignerInfo {
   179  	return e.leaderCh
   180  }
   181  
   182  // ErrorNotify returns a channel that can fetch errors occurred for campaign.
   183  func (e *Election) ErrorNotify() <-chan error {
   184  	return e.ech
   185  }
   186  
   187  // Close closes the election instance and release the resources.
   188  func (e *Election) Close() {
   189  	e.l.Info("election is closing", zap.Stringer("current member", e.info))
   190  	if !e.closed.CAS(false, true) {
   191  		e.l.Info("election was already closed", zap.Stringer("current member", e.info))
   192  		return
   193  	}
   194  
   195  	e.cancel()
   196  	e.bgWg.Wait()
   197  	e.l.Info("election is closed", zap.Stringer("current member", e.info))
   198  }
   199  
   200  func (e *Election) campaignLoop(ctx context.Context, session *concurrency.Session) {
   201  	closeSession := func(se *concurrency.Session) {
   202  		err2 := se.Close() // only log this error
   203  		if err2 != nil {
   204  			e.l.Error("fail to close etcd session", zap.Int64("lease", int64(se.Lease())), zap.Error(err2))
   205  		}
   206  	}
   207  	failpoint.Inject("mockCampaignLoopExitedAbnormally", func(_ failpoint.Value) {
   208  		closeSession = func(_ *concurrency.Session) {
   209  			e.l.Info("skip closeSession", zap.String("failpoint", "mockCampaignLoopExitedAbnormally"))
   210  		}
   211  	})
   212  
   213  	var err error
   214  	defer func() {
   215  		if session != nil {
   216  			closeSession(session) // close the latest session.
   217  		}
   218  		if err != nil && errors.Cause(err) != ctx.Err() { // only send non-ctx.Err() error
   219  			e.ech <- err
   220  		}
   221  	}()
   222  
   223  	var (
   224  		oldLeaderID string
   225  		campaignWg  sync.WaitGroup
   226  	)
   227  	for {
   228  		// check context canceled/timeout
   229  		select {
   230  		case <-session.Done():
   231  			e.l.Info("etcd session is done, will try to create a new one", zap.Int64("old lease", int64(session.Lease())))
   232  			closeSession(session)
   233  			session, err = e.newSession(ctx, newSessionRetryUnlimited) // retry until context is done
   234  			if err != nil {
   235  				err = terror.ErrElectionCampaignFail.Delegate(err, "create a new session")
   236  				return
   237  			}
   238  		case <-ctx.Done():
   239  			e.l.Info("break campaign loop, context is done", zap.Stringer("current member", e.info), zap.Error(ctx.Err()))
   240  			return
   241  		default:
   242  		}
   243  
   244  		// try to campaign
   245  		elec := concurrency.NewElection(session, e.key)
   246  		ctx2, cancel2 := context.WithCancel(ctx)
   247  
   248  		e.campaignMu.Lock()
   249  		campaignWg.Add(1)
   250  		e.cancelCampaign = func() {
   251  			cancel2()
   252  			campaignWg.Wait()
   253  		}
   254  		e.campaignMu.Unlock()
   255  
   256  		go func() {
   257  			defer campaignWg.Done()
   258  
   259  			if e.evictLeader.Load() {
   260  				// skip campaign
   261  				return
   262  			}
   263  
   264  			e.l.Debug("begin to campaign", zap.Stringer("current member", e.info))
   265  
   266  			err2 := elec.Campaign(ctx2, e.infoStr)
   267  			if err2 != nil {
   268  				// because inner commit may return undetermined error, we try to delete the election key manually
   269  				deleted, err3 := e.ClearSessionIfNeeded(ctx, e.ID())
   270  				if err3 != nil {
   271  					e.l.Warn("failed to clean election key", zap.Error(err3))
   272  				} else if deleted {
   273  					e.l.Info("successful manually clean election key",
   274  						zap.String("campaign error", err2.Error()))
   275  				}
   276  
   277  				// err may be ctx.Err(), but this can be handled in `case <-ctx.Done()`
   278  				e.l.Info("fail to campaign", zap.Stringer("current member", e.info), zap.Error(err2))
   279  			}
   280  		}()
   281  
   282  		var (
   283  			leaderKey  string
   284  			leaderInfo *CampaignerInfo
   285  		)
   286  		eleObserveCh := elec.Observe(ctx2)
   287  
   288  	observeElection:
   289  		for {
   290  			select {
   291  			case <-ctx.Done():
   292  				break observeElection
   293  			case <-session.Done():
   294  				break observeElection
   295  			case resp, ok := <-eleObserveCh:
   296  				if !ok {
   297  					break observeElection
   298  				}
   299  
   300  				e.l.Info("get response from election observe", zap.String("key", string(resp.Kvs[0].Key)), zap.String("value", string(resp.Kvs[0].Value)))
   301  				leaderKey = string(resp.Kvs[0].Key)
   302  				leaderInfo, err = getCampaignerInfo(resp.Kvs[0].Value)
   303  				if err != nil {
   304  					// this should never happened
   305  					e.l.Error("fail to get leader information", zap.String("value", string(resp.Kvs[0].Value)), zap.Error(err))
   306  					continue
   307  				}
   308  
   309  				if oldLeaderID == leaderInfo.ID {
   310  					continue
   311  				}
   312  				oldLeaderID = leaderInfo.ID
   313  				break observeElection
   314  			}
   315  		}
   316  
   317  		if leaderInfo == nil || len(leaderInfo.ID) == 0 {
   318  			cancel2()
   319  			campaignWg.Wait()
   320  			continue
   321  		}
   322  
   323  		if leaderInfo.ID != e.info.ID {
   324  			e.l.Info("current member is not the leader", zap.Stringer("current member", e.info), zap.Stringer("leader", leaderInfo))
   325  			e.notifyLeader(ctx, leaderInfo)
   326  			cancel2()
   327  			campaignWg.Wait()
   328  			continue
   329  		}
   330  
   331  		e.l.Info("become leader", zap.Stringer("current member", e.info))
   332  		e.notifyLeader(ctx, leaderInfo) // become the leader now
   333  		e.watchLeader(ctx, session, leaderKey, elec)
   334  		e.l.Info("retire from leader", zap.Stringer("current member", e.info))
   335  		e.notifyLeader(ctx, nil) // need to re-campaign
   336  		oldLeaderID = ""
   337  
   338  		cancel2()
   339  		campaignWg.Wait()
   340  	}
   341  }
   342  
   343  // notifyLeader notify the leader's information.
   344  // leader info can be nil, and this is used when retire from leader.
   345  func (e *Election) notifyLeader(ctx context.Context, leaderInfo *CampaignerInfo) {
   346  	if leaderInfo != nil && leaderInfo.ID == e.info.ID {
   347  		e.isLeader.Store(true)
   348  	} else {
   349  		e.isLeader.Store(false)
   350  	}
   351  
   352  	select {
   353  	case e.leaderCh <- leaderInfo:
   354  	case <-time.After(e.notifyBlockTime):
   355  		// this should not happened
   356  		e.l.Error("ignore notify the leader's information after block a period of time", zap.Stringer("current member", e.info), zap.Stringer("leader", leaderInfo))
   357  	case <-ctx.Done():
   358  		e.l.Warn("ignore notify the leader's information because context canceled", zap.Stringer("current member", e.info), zap.Stringer("leader", leaderInfo))
   359  	}
   360  }
   361  
   362  func (e *Election) watchLeader(ctx context.Context, session *concurrency.Session, key string, elec *concurrency.Election) {
   363  	e.l.Debug("watch leader key", zap.String("key", key))
   364  
   365  	e.campaignMu.Lock()
   366  	e.resignCh = make(chan struct{})
   367  	e.campaignMu.Unlock()
   368  
   369  	defer func() {
   370  		e.campaignMu.Lock()
   371  		e.resignCh = nil
   372  		e.campaignMu.Unlock()
   373  	}()
   374  
   375  	wCtx, cancel := context.WithCancel(ctx)
   376  	defer cancel()
   377  	wch := e.cli.Watch(wCtx, key)
   378  
   379  	for {
   380  		if e.evictLeader.Load() {
   381  			if err := elec.Resign(ctx); err != nil {
   382  				e.l.Info("fail to resign leader", zap.Stringer("current member", e.info), zap.Error(err))
   383  			}
   384  			return
   385  		}
   386  
   387  		select {
   388  		case resp, ok := <-wch:
   389  			if !ok {
   390  				e.l.Info("watch channel is closed")
   391  				return
   392  			}
   393  			if resp.Canceled {
   394  				e.l.Info("watch canceled")
   395  				return
   396  			}
   397  
   398  			for _, ev := range resp.Events {
   399  				// user may use some etcd client (like etcdctl) to delete the leader key and trigger a new campaign.
   400  				if ev.Type == mvccpb.DELETE {
   401  					e.l.Info("fail to watch, the leader is deleted", zap.ByteString("key", ev.Kv.Key))
   402  					return
   403  				}
   404  			}
   405  		case <-session.Done():
   406  			return
   407  		case <-ctx.Done():
   408  			return
   409  		case <-e.resignCh:
   410  			if err := elec.Resign(ctx); err != nil {
   411  				e.l.Info("fail to resign leader", zap.Stringer("current member", e.info), zap.Error(err))
   412  			}
   413  			return
   414  		}
   415  	}
   416  }
   417  
   418  // EvictLeader set evictLeader to true, and this member can't be leader.
   419  func (e *Election) EvictLeader() {
   420  	if !e.evictLeader.CAS(false, true) {
   421  		return
   422  	}
   423  
   424  	e.Resign()
   425  }
   426  
   427  // Resign resign the leader.
   428  func (e *Election) Resign() {
   429  	// cancel campaign or current member is leader and then resign
   430  	e.campaignMu.Lock()
   431  	if e.cancelCampaign != nil {
   432  		e.cancelCampaign()
   433  		e.cancelCampaign = nil
   434  	}
   435  
   436  	if e.resignCh != nil {
   437  		close(e.resignCh)
   438  	}
   439  	e.campaignMu.Unlock()
   440  }
   441  
   442  // CancelEvictLeader set evictLeader to false, and this member can campaign leader again.
   443  func (e *Election) CancelEvictLeader() {
   444  	if !e.evictLeader.CAS(true, false) {
   445  		return
   446  	}
   447  
   448  	e.campaignMu.Lock()
   449  	if e.cancelCampaign != nil {
   450  		e.cancelCampaign()
   451  		e.cancelCampaign = nil
   452  	}
   453  	e.campaignMu.Unlock()
   454  }
   455  
   456  func (e *Election) newSession(ctx context.Context, retryCnt int) (*concurrency.Session, error) {
   457  	var (
   458  		err     error
   459  		session *concurrency.Session
   460  	)
   461  
   462  forLoop:
   463  	for i := 0; i < retryCnt; i++ {
   464  		if i > 0 {
   465  			select {
   466  			case e.ech <- terror.ErrElectionCampaignFail.Delegate(err, "create a new session"):
   467  			default:
   468  			}
   469  
   470  			select {
   471  			case <-time.After(newSessionRetryInterval):
   472  			case <-ctx.Done():
   473  				break forLoop
   474  			}
   475  		}
   476  
   477  		// add more options if needed.
   478  		// NOTE: I think use the client's context is better than something like `concurrency.WithContext(ctx)`,
   479  		// so we can close the session when the client is still valid.
   480  		session, err = concurrency.NewSession(e.cli, concurrency.WithTTL(e.sessionTTL))
   481  		if err == nil || errors.Cause(err) == e.cli.Ctx().Err() {
   482  			break forLoop
   483  		}
   484  	}
   485  	return session, err
   486  }
   487  
   488  // ClearSessionIfNeeded will clear session when deleted master quited abnormally
   489  // returns (triggered deleting session, error).
   490  func (e *Election) ClearSessionIfNeeded(ctx context.Context, id string) (bool, error) {
   491  	resp, err := e.cli.Get(ctx, e.key, clientv3.WithPrefix())
   492  	if err != nil {
   493  		return false, err
   494  	}
   495  	deleteKey := ""
   496  	for _, kv := range resp.Kvs {
   497  		leaderInfo, err2 := getCampaignerInfo(kv.Value)
   498  		if err2 != nil {
   499  			return false, err2
   500  		}
   501  		if leaderInfo.ID == id {
   502  			deleteKey = string(kv.Key)
   503  			break
   504  		}
   505  	}
   506  	if deleteKey == "" {
   507  		// no campaign info left in etcd, no need to trigger re-campaign
   508  		return false, nil
   509  	}
   510  	delResp, err := e.cli.Delete(ctx, deleteKey)
   511  	if err != nil {
   512  		return false, err
   513  	}
   514  	return delResp.Deleted > 0, err
   515  }