github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/master/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 master
    15  
    16  import (
    17  	"context"
    18  	"strings"
    19  	"time"
    20  
    21  	"github.com/pingcap/failpoint"
    22  	toolutils "github.com/pingcap/tidb-tools/pkg/utils"
    23  	"github.com/pingcap/tiflow/dm/master/metrics"
    24  	"github.com/pingcap/tiflow/dm/pb"
    25  	"github.com/pingcap/tiflow/dm/pkg/log"
    26  	"go.uber.org/zap"
    27  	"google.golang.org/grpc"
    28  )
    29  
    30  const (
    31  	oneselfLeader         = "oneself"
    32  	oneselfStartingLeader = "starting"
    33  )
    34  
    35  func (s *Server) electionNotify(ctx context.Context) {
    36  	for {
    37  		select {
    38  		case <-ctx.Done():
    39  			return
    40  		case leaderInfo := <-s.election.LeaderNotify():
    41  			// retire from leader
    42  			if leaderInfo == nil {
    43  				if s.leader.Load() == oneselfLeader {
    44  					s.retireLeader()
    45  					log.L().Info("current member retire from the leader", zap.String("current member", s.cfg.Name))
    46  				} else {
    47  					// leader retire before
    48  					log.L().Error("current member is not the leader, can't retire", zap.String("current member", s.cfg.Name))
    49  				}
    50  
    51  				continue
    52  			}
    53  
    54  			if leaderInfo.ID == s.cfg.Name {
    55  				// this member become leader
    56  				log.L().Info("current member become the leader", zap.String("current member", s.cfg.Name))
    57  				s.leader.Store(oneselfStartingLeader)
    58  
    59  				// try to upgrade the cluster before scheduler start
    60  				err := s.bootstrapBeforeSchedulerStart(ctx)
    61  				if err != nil {
    62  					log.L().Error("fail to bootstrap the cluster before scheduler start", zap.Error(err))
    63  					s.retireLeader()
    64  					s.election.Resign()
    65  					continue
    66  				}
    67  
    68  				// NOTE: for logic errors, we should return with `true`, so that the cluster can serve requests and the user can fix these errors.
    69  				// otherwise no member of DM-master can become the leader and the user can't fix them (the cluster may need to be fixed offline with some other tools like etcdctl).
    70  				ok := s.startLeaderComponent(ctx)
    71  
    72  				if !ok {
    73  					s.retireLeader()
    74  					s.election.Resign()
    75  					continue
    76  				}
    77  
    78  				s.Lock()
    79  				s.leader.Store(oneselfLeader)
    80  				s.closeLeaderClient()
    81  				s.Unlock()
    82  
    83  				// try to upgrade the cluster version if a member BECOME the leader.
    84  				// so if the old leader failed when upgrading, the new leader can try again.
    85  				// NOTE: if the cluster has been upgraded, calling this method again should have no side effects.
    86  				// NOTE: now, bootstrap relies on scheduler to handle DM-worker instances, sources, tasks, etcd.
    87  				err = s.bootstrap(ctx)
    88  				if err != nil {
    89  					log.L().Error("fail to bootstrap the cluster", zap.Error(err))
    90  					s.retireLeader()
    91  					s.election.Resign()
    92  					continue
    93  				}
    94  			} else {
    95  				// this member is not leader
    96  				log.L().Info("get new leader", zap.String("leader", leaderInfo.ID), zap.String("current member", s.cfg.Name))
    97  
    98  				s.Lock()
    99  				s.leader.Store(leaderInfo.ID)
   100  				s.createLeaderClient(leaderInfo.Addr)
   101  				s.Unlock()
   102  			}
   103  
   104  		case err := <-s.election.ErrorNotify():
   105  			// handle errors here, we do no meaningful things now.
   106  			// but maybe:
   107  			// 1. trigger an alert
   108  			// 2. shutdown the DM-master process
   109  			log.L().Error("receive error from election", zap.Error(err))
   110  		}
   111  	}
   112  }
   113  
   114  func (s *Server) createLeaderClient(leaderAddr string) {
   115  	s.closeLeaderClient()
   116  
   117  	tls, err := toolutils.NewTLS(s.cfg.SSLCA, s.cfg.SSLCert, s.cfg.SSLKey, s.cfg.AdvertiseAddr, s.cfg.CertAllowedCN)
   118  	if err != nil {
   119  		log.L().Error("can't create grpc connection with leader, can't forward request to leader", zap.String("leader", leaderAddr), zap.Error(err))
   120  		return
   121  	}
   122  
   123  	//nolint:staticcheck
   124  	conn, err := grpc.Dial(leaderAddr, tls.ToGRPCDialOption(), grpc.WithBackoffMaxDelay(3*time.Second))
   125  	if err != nil {
   126  		log.L().Error("can't create grpc connection with leader, can't forward request to leader", zap.String("leader", leaderAddr), zap.Error(err))
   127  		return
   128  	}
   129  	s.leaderGrpcConn = conn
   130  	s.leaderClient = pb.NewMasterClient(conn)
   131  }
   132  
   133  func (s *Server) closeLeaderClient() {
   134  	if s.leaderGrpcConn != nil {
   135  		s.leaderGrpcConn.Close()
   136  		s.leaderGrpcConn = nil
   137  	}
   138  }
   139  
   140  func (s *Server) isLeaderAndNeedForward(ctx context.Context) (isLeader bool, needForward bool) {
   141  	// maybe in `startLeaderComponent`, will wait for a short time
   142  	if s.leader.Load() == oneselfStartingLeader {
   143  		retry := 10
   144  		ticker := time.NewTicker(100 * time.Millisecond)
   145  		defer ticker.Stop()
   146  
   147  		for s.leader.Load() == oneselfStartingLeader {
   148  			if retry == 0 {
   149  				log.L().Error("leader didn't finish starting after retry, please manually retry later")
   150  				return false, false
   151  			}
   152  			select {
   153  			case <-ctx.Done():
   154  				return false, false
   155  			case <-ticker.C:
   156  				retry--
   157  			}
   158  		}
   159  	}
   160  
   161  	s.RLock()
   162  	defer s.RUnlock()
   163  
   164  	isLeader = s.leader.Load() == oneselfLeader
   165  	needForward = s.leaderGrpcConn != nil
   166  	return
   167  }
   168  
   169  func (s *Server) startLeaderComponent(ctx context.Context) bool {
   170  	metrics.ReportStartLeader()
   171  
   172  	err := s.scheduler.Start(ctx, s.etcdClient)
   173  	if err != nil {
   174  		log.L().Error("scheduler do not started", zap.Error(err))
   175  		return false
   176  	}
   177  
   178  	err = s.pessimist.Start(ctx, s.etcdClient)
   179  	if err != nil {
   180  		log.L().Error("pessimist do not started", zap.Error(err))
   181  		return false
   182  	}
   183  
   184  	err = s.optimist.Start(ctx, s.etcdClient)
   185  	if err != nil {
   186  		log.L().Error("optimist do not started", zap.Error(err))
   187  		return false
   188  	}
   189  
   190  	err = s.initClusterID(ctx)
   191  	if err != nil {
   192  		log.L().Error("init cluster id failed", zap.Error(err))
   193  		return false
   194  	}
   195  
   196  	failpoint.Inject("FailToStartLeader", func(val failpoint.Value) {
   197  		masterStrings := val.(string)
   198  		if strings.Contains(masterStrings, s.cfg.Name) {
   199  			log.L().Info("fail to start leader", zap.String("failpoint", "FailToStartLeader"))
   200  			failpoint.Return(false)
   201  		}
   202  	})
   203  
   204  	return true
   205  }
   206  
   207  func (s *Server) retireLeader() {
   208  	s.pessimist.Close()
   209  	s.optimist.Close()
   210  	s.scheduler.Close()
   211  
   212  	s.Lock()
   213  	s.leader.Store("")
   214  	s.closeLeaderClient()
   215  	s.Unlock()
   216  
   217  	metrics.OnRetireLeader()
   218  }