github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/capture/capture.go (about)

     1  // Copyright 2021 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 capture
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"io"
    20  	"os"
    21  	"strings"
    22  	"sync"
    23  	"time"
    24  
    25  	"github.com/google/uuid"
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/failpoint"
    28  	"github.com/pingcap/log"
    29  	"github.com/pingcap/tiflow/cdc/controller"
    30  	"github.com/pingcap/tiflow/cdc/model"
    31  	"github.com/pingcap/tiflow/cdc/owner"
    32  	"github.com/pingcap/tiflow/cdc/processor"
    33  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter/factory"
    34  	"github.com/pingcap/tiflow/cdc/vars"
    35  	"github.com/pingcap/tiflow/pkg/config"
    36  	cerror "github.com/pingcap/tiflow/pkg/errors"
    37  	"github.com/pingcap/tiflow/pkg/etcd"
    38  	"github.com/pingcap/tiflow/pkg/migrate"
    39  	"github.com/pingcap/tiflow/pkg/orchestrator"
    40  	"github.com/pingcap/tiflow/pkg/p2p"
    41  	"github.com/pingcap/tiflow/pkg/upstream"
    42  	"github.com/pingcap/tiflow/pkg/util"
    43  	"github.com/pingcap/tiflow/pkg/version"
    44  	"github.com/pingcap/tiflow/pkg/workerpool"
    45  	pd "github.com/tikv/pd/client"
    46  	"go.etcd.io/etcd/client/v3/concurrency"
    47  	"go.etcd.io/etcd/server/v3/mvcc"
    48  	"go.uber.org/zap"
    49  	"golang.org/x/sync/errgroup"
    50  	"golang.org/x/time/rate"
    51  )
    52  
    53  const (
    54  	cleanMetaDuration = 10 * time.Second
    55  	// changefeedAsyncInitWorkerCount is the size of the worker pool for changefeed initialization processing.
    56  	changefeedAsyncInitWorkerCount = 8
    57  )
    58  
    59  // Capture represents a Capture server, it monitors the changefeed
    60  // information in etcd and schedules Task on it.
    61  type Capture interface {
    62  	Run(ctx context.Context) error
    63  	Close()
    64  	Drain() <-chan struct{}
    65  	Liveness() model.Liveness
    66  
    67  	GetOwner() (owner.Owner, error)
    68  	GetController() (controller.Controller, error)
    69  	GetControllerCaptureInfo(ctx context.Context) (*model.CaptureInfo, error)
    70  	IsController() bool
    71  
    72  	Info() (model.CaptureInfo, error)
    73  	StatusProvider() owner.StatusProvider
    74  	WriteDebugInfo(ctx context.Context, w io.Writer)
    75  
    76  	GetUpstreamManager() (*upstream.Manager, error)
    77  	GetEtcdClient() etcd.CDCEtcdClient
    78  	// IsReady returns if the cdc server is ready
    79  	// currently only check if ettcd data migration is done
    80  	IsReady() bool
    81  	GetUpstreamInfo(context.Context, model.UpstreamID, string) (*model.UpstreamInfo, error)
    82  }
    83  
    84  type captureImpl struct {
    85  	// captureMu is used to protect the capture info and processorManager.
    86  	captureMu        sync.Mutex
    87  	info             *model.CaptureInfo
    88  	processorManager processor.Manager
    89  	liveness         model.Liveness
    90  	config           *config.ServerConfig
    91  
    92  	pdClient        pd.Client
    93  	pdEndpoints     []string
    94  	ownerMu         sync.Mutex
    95  	owner           owner.Owner
    96  	controller      controller.Controller
    97  	upstreamManager *upstream.Manager
    98  
    99  	// session keeps alive between the capture and etcd
   100  	session  *concurrency.Session
   101  	election election
   102  
   103  	EtcdClient etcd.CDCEtcdClient
   104  
   105  	sortEngineFactory *factory.SortEngineFactory
   106  
   107  	// ChangefeedThreadPool is the thread pool for changefeed initialization
   108  	ChangefeedThreadPool workerpool.AsyncPool
   109  
   110  	// MessageServer is the receiver of the messages from the other nodes.
   111  	// It should be recreated each time the capture is restarted.
   112  	MessageServer *p2p.MessageServer
   113  
   114  	// MessageRouter manages the clients to send messages to all peers.
   115  	MessageRouter p2p.MessageRouter
   116  
   117  	// grpcService is a wrapper that can hold a MessageServer.
   118  	// The instance should last for the whole life of the server,
   119  	// regardless of server restarting.
   120  	// This design is to solve the problem that grpc-go cannot gracefully
   121  	// unregister a service.
   122  	grpcService *p2p.ServerWrapper
   123  
   124  	cancel context.CancelFunc
   125  
   126  	migrator migrate.Migrator
   127  
   128  	newProcessorManager func(
   129  		captureInfo *model.CaptureInfo,
   130  		upstreamManager *upstream.Manager,
   131  		liveness *model.Liveness,
   132  		cfg *config.SchedulerConfig,
   133  		globalVars *vars.GlobalVars,
   134  	) processor.Manager
   135  	newOwner func(upstreamManager *upstream.Manager, cfg *config.SchedulerConfig,
   136  		globalVars *vars.GlobalVars) owner.Owner
   137  	newController func(upstreamManager *upstream.Manager, captureInfo *model.CaptureInfo, client etcd.CDCEtcdClient) controller.Controller
   138  }
   139  
   140  func (c *captureImpl) GetUpstreamInfo(ctx context.Context, id model.UpstreamID, namespace string) (*model.UpstreamInfo, error) {
   141  	return c.GetEtcdClient().GetUpstreamInfo(ctx, id, namespace)
   142  }
   143  
   144  // NewCapture returns a new Capture instance
   145  func NewCapture(pdEndpoints []string,
   146  	etcdClient etcd.CDCEtcdClient,
   147  	grpcService *p2p.ServerWrapper,
   148  	sortEngineMangerFactory *factory.SortEngineFactory,
   149  	pdClient pd.Client,
   150  ) Capture {
   151  	conf := config.GetGlobalServerConfig()
   152  	return &captureImpl{
   153  		config:              config.GetGlobalServerConfig(),
   154  		liveness:            model.LivenessCaptureAlive,
   155  		EtcdClient:          etcdClient,
   156  		grpcService:         grpcService,
   157  		cancel:              func() {},
   158  		pdEndpoints:         pdEndpoints,
   159  		newProcessorManager: processor.NewManager,
   160  		newOwner:            owner.NewOwner,
   161  		newController:       controller.NewController,
   162  		info:                &model.CaptureInfo{},
   163  		sortEngineFactory:   sortEngineMangerFactory,
   164  		migrator:            migrate.NewMigrator(etcdClient, pdEndpoints, conf),
   165  		pdClient:            pdClient,
   166  	}
   167  }
   168  
   169  // NewCapture4Test returns a new Capture instance for test.
   170  func NewCapture4Test(o owner.Owner) *captureImpl {
   171  	res := &captureImpl{
   172  		info: &model.CaptureInfo{
   173  			ID:            "capture-for-test",
   174  			AdvertiseAddr: "127.0.0.1",
   175  			Version:       "test",
   176  		},
   177  		migrator: &migrate.NoOpMigrator{},
   178  		config:   config.GetGlobalServerConfig(),
   179  	}
   180  	res.owner = o
   181  	return res
   182  }
   183  
   184  // NewCaptureWithController4Test returns a new Capture instance for test.
   185  func NewCaptureWithController4Test(o owner.Owner,
   186  	manager controller.Controller,
   187  ) *captureImpl {
   188  	res := &captureImpl{
   189  		info: &model.CaptureInfo{
   190  			ID:            "capture-for-test",
   191  			AdvertiseAddr: "127.0.0.1",
   192  			Version:       "test",
   193  		},
   194  		migrator: &migrate.NoOpMigrator{},
   195  		config:   config.GetGlobalServerConfig(),
   196  	}
   197  	res.controller = manager
   198  	res.owner = o
   199  	return res
   200  }
   201  
   202  // NewCaptureWithManager4Test returns a new Capture instance for test.
   203  func NewCaptureWithManager4Test(o owner.Owner, m *upstream.Manager) *captureImpl {
   204  	res := &captureImpl{
   205  		upstreamManager: m,
   206  		migrator:        &migrate.NoOpMigrator{},
   207  	}
   208  	res.owner = o
   209  	return res
   210  }
   211  
   212  // GetUpstreamManager is a Getter of capture's upstream manager
   213  func (c *captureImpl) GetUpstreamManager() (*upstream.Manager, error) {
   214  	if c.upstreamManager == nil {
   215  		return nil, cerror.ErrUpstreamManagerNotReady
   216  	}
   217  	return c.upstreamManager, nil
   218  }
   219  
   220  func (c *captureImpl) GetEtcdClient() etcd.CDCEtcdClient {
   221  	return c.EtcdClient
   222  }
   223  
   224  // reset the capture before run it.
   225  func (c *captureImpl) reset(ctx context.Context) (*vars.GlobalVars, error) {
   226  	lease, err := c.EtcdClient.GetEtcdClient().Grant(ctx, int64(c.config.CaptureSessionTTL))
   227  	if err != nil {
   228  		return nil, errors.Trace(err)
   229  	}
   230  	sess, err := concurrency.NewSession(
   231  		c.EtcdClient.GetEtcdClient().Unwrap(), concurrency.WithLease(lease.ID))
   232  	if err != nil {
   233  		return nil, errors.Trace(err)
   234  	}
   235  	log.Info("reset session successfully", zap.Any("session", sess))
   236  
   237  	c.captureMu.Lock()
   238  	defer c.captureMu.Unlock()
   239  	deployPath, err := os.Executable()
   240  	if err != nil {
   241  		deployPath = ""
   242  	}
   243  	c.info = &model.CaptureInfo{
   244  		ID:             uuid.New().String(),
   245  		AdvertiseAddr:  c.config.AdvertiseAddr,
   246  		Version:        version.ReleaseVersion,
   247  		GitHash:        version.GitHash,
   248  		DeployPath:     deployPath,
   249  		StartTimestamp: time.Now().Unix(),
   250  	}
   251  
   252  	if c.upstreamManager != nil {
   253  		c.upstreamManager.Close()
   254  	}
   255  	c.upstreamManager = upstream.NewManager(ctx, upstream.CaptureTopologyCfg{
   256  		CaptureInfo: c.info,
   257  		GCServiceID: c.EtcdClient.GetGCServiceID(),
   258  		SessionTTL:  int64(c.config.CaptureSessionTTL),
   259  	})
   260  	_, err = c.upstreamManager.AddDefaultUpstream(c.pdEndpoints, c.config.Security, c.pdClient, c.EtcdClient.GetEtcdClient())
   261  	if err != nil {
   262  		return nil, errors.Trace(err)
   263  	}
   264  	if c.session != nil {
   265  		// It can't be handled even after it fails, so we ignore it.
   266  		_ = c.session.Close()
   267  	}
   268  	c.session = sess
   269  	c.election = newElection(sess, etcd.CaptureOwnerKey(c.EtcdClient.GetClusterID()))
   270  
   271  	c.grpcService.Reset(nil)
   272  
   273  	if c.MessageRouter != nil {
   274  		c.MessageRouter.Close()
   275  		c.MessageRouter = nil
   276  	}
   277  	messageServerConfig := c.config.Debug.Messages.ToMessageServerConfig()
   278  	c.MessageServer = p2p.NewMessageServer(c.info.ID, messageServerConfig)
   279  	c.grpcService.Reset(c.MessageServer)
   280  
   281  	messageClientConfig := c.config.Debug.Messages.ToMessageClientConfig()
   282  
   283  	// Puts the advertise-addr of the local node to the client config.
   284  	// This is for metrics purpose only, so that the receiver knows which
   285  	// node the connections are from.
   286  	advertiseAddr := c.config.AdvertiseAddr
   287  	messageClientConfig.AdvertisedAddr = advertiseAddr
   288  
   289  	c.MessageRouter = p2p.NewMessageRouterWithLocalClient(c.info.ID, c.config.Security, messageClientConfig)
   290  	c.ChangefeedThreadPool = workerpool.NewDefaultAsyncPool(changefeedAsyncInitWorkerCount)
   291  	globalVars := &vars.GlobalVars{
   292  		CaptureInfo:          c.info,
   293  		EtcdClient:           c.EtcdClient,
   294  		MessageServer:        c.MessageServer,
   295  		MessageRouter:        c.MessageRouter,
   296  		SortEngineFactory:    c.sortEngineFactory,
   297  		ChangefeedThreadPool: c.ChangefeedThreadPool,
   298  	}
   299  	c.processorManager = c.newProcessorManager(
   300  		c.info, c.upstreamManager, &c.liveness, c.config.Debug.Scheduler, globalVars)
   301  
   302  	log.Info("capture initialized", zap.Any("capture", c.info))
   303  	return globalVars, nil
   304  }
   305  
   306  // Run runs the capture
   307  func (c *captureImpl) Run(ctx context.Context) error {
   308  	defer log.Info("the capture routine has exited")
   309  	// Limit the frequency of reset capture to avoid frequent recreating of resources
   310  	rl := rate.NewLimiter(0.05, 2)
   311  	for {
   312  		select {
   313  		case <-ctx.Done():
   314  			return nil
   315  		default:
   316  		}
   317  		ctx, cancel := context.WithCancel(ctx)
   318  		c.cancel = cancel
   319  		err := rl.Wait(ctx)
   320  		if err != nil {
   321  			if errors.Cause(err) == context.Canceled {
   322  				return nil
   323  			}
   324  			return errors.Trace(err)
   325  		}
   326  		err = c.run(ctx)
   327  		// if capture suicided, reset the capture and run again.
   328  		// if the canceled error throw, there are two possible scenarios:
   329  		//   1. the internal context canceled, it means some error happened in
   330  		//      the internal, and the routine is exited, we should restart
   331  		//      the capture.
   332  		//   2. the parent context canceled, it means that the caller of
   333  		//      the capture hope the capture to exit, and this loop will return
   334  		//      in the above `select` block.
   335  		// if there are some **internal** context deadline exceeded (IO/network
   336  		// timeout), reset the capture and run again.
   337  		//
   338  		// TODO: make sure the internal cancel should return the real error
   339  		//       instead of context.Canceled.
   340  		if cerror.ErrCaptureSuicide.Equal(err) ||
   341  			context.Canceled == errors.Cause(err) ||
   342  			context.DeadlineExceeded == errors.Cause(err) {
   343  			log.Info("capture recovered", zap.String("captureID", c.info.ID))
   344  			continue
   345  		}
   346  		return errors.Trace(err)
   347  	}
   348  }
   349  
   350  func (c *captureImpl) run(stdCtx context.Context) error {
   351  	globalVars, err := c.reset(stdCtx)
   352  	if err != nil {
   353  		log.Error("reset capture failed", zap.Error(err))
   354  		return errors.Trace(err)
   355  	}
   356  
   357  	err = c.register(stdCtx)
   358  	if err != nil {
   359  		return errors.Trace(err)
   360  	}
   361  	defer func() {
   362  		timeoutCtx, cancel := context.WithTimeout(context.Background(), cleanMetaDuration)
   363  		if err := c.EtcdClient.DeleteCaptureInfo(timeoutCtx, c.info.ID); err != nil {
   364  			log.Warn("failed to delete capture info when capture exited",
   365  				zap.String("captureID", c.info.ID),
   366  				zap.Error(err))
   367  		}
   368  		cancel()
   369  	}()
   370  
   371  	defer func() {
   372  		c.Close()
   373  		c.grpcService.Reset(nil)
   374  	}()
   375  
   376  	g, stdCtx := errgroup.WithContext(stdCtx)
   377  	stdCtx, cancel := context.WithCancel(stdCtx)
   378  
   379  	g.Go(func() error {
   380  		// when the campaignOwner returns an error, it means that the owner throws
   381  		// an unrecoverable serious errors (recoverable errors are intercepted in the owner tick)
   382  		// so we should restart the capture.
   383  		err := c.campaignOwner(stdCtx, globalVars)
   384  		if err != nil || c.liveness.Load() != model.LivenessCaptureStopping {
   385  			log.Warn("campaign owner routine exited, restart the capture",
   386  				zap.String("captureID", c.info.ID), zap.Error(err))
   387  			// Throw ErrCaptureSuicide to restart capture.
   388  			return cerror.ErrCaptureSuicide.FastGenByArgs()
   389  		}
   390  		return nil
   391  	})
   392  
   393  	g.Go(func() error {
   394  		// Processor manager should be closed as soon as possible to prevent double write issue.
   395  		defer func() {
   396  			if cancel != nil {
   397  				// Propagate the cancel signal to the owner and other goroutines.
   398  				cancel()
   399  			}
   400  			if c.processorManager != nil {
   401  				c.processorManager.Close()
   402  			}
   403  			log.Info("processor manager closed", zap.String("captureID", c.info.ID))
   404  		}()
   405  		processorFlushInterval := time.Duration(c.config.ProcessorFlushInterval)
   406  
   407  		globalState := orchestrator.NewGlobalState(c.EtcdClient.GetClusterID(), c.config.CaptureSessionTTL)
   408  
   409  		globalState.SetOnCaptureAdded(func(captureID model.CaptureID, addr string) {
   410  			c.MessageRouter.AddPeer(captureID, addr)
   411  		})
   412  		globalState.SetOnCaptureRemoved(func(captureID model.CaptureID) {
   413  			c.MessageRouter.RemovePeer(captureID)
   414  		})
   415  
   416  		// when the etcd worker of processor returns an error, it means that the processor throws an unrecoverable serious errors
   417  		// (recoverable errors are intercepted in the processor tick)
   418  		// so we should also stop the processor and let capture restart or exit
   419  		err := c.runEtcdWorker(stdCtx, c.processorManager, globalState, processorFlushInterval, util.RoleProcessor.String())
   420  		log.Info("processor routine exited",
   421  			zap.String("captureID", c.info.ID), zap.Error(err))
   422  		return err
   423  	})
   424  
   425  	g.Go(func() error {
   426  		return c.MessageServer.Run(stdCtx, c.MessageRouter.GetLocalChannel())
   427  	})
   428  
   429  	poolCtx, cancelPool := context.WithCancel(stdCtx)
   430  	defer func() {
   431  		cancelPool()
   432  		log.Info("workerpool exited", zap.Error(err))
   433  	}()
   434  	g.Go(func() error {
   435  		return c.ChangefeedThreadPool.Run(poolCtx)
   436  	})
   437  	return errors.Trace(g.Wait())
   438  }
   439  
   440  // Info gets the capture info
   441  func (c *captureImpl) Info() (model.CaptureInfo, error) {
   442  	c.captureMu.Lock()
   443  	defer c.captureMu.Unlock()
   444  	// when c.reset has not been called yet, c.info is nil.
   445  	if c.info != nil {
   446  		return *c.info, nil
   447  	}
   448  	return model.CaptureInfo{}, cerror.ErrCaptureNotInitialized.GenWithStackByArgs()
   449  }
   450  
   451  func (c *captureImpl) campaignOwner(ctx context.Context, globalVars *vars.GlobalVars) error {
   452  	// In most failure cases, we don't return error directly, just run another
   453  	// campaign loop. We treat campaign loop as a special background routine.
   454  	ownerFlushInterval := time.Duration(c.config.OwnerFlushInterval)
   455  	failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) {
   456  		ownerFlushInterval = time.Millisecond * time.Duration(val.(int))
   457  	})
   458  	// Limit the frequency of elections to avoid putting too much pressure on the etcd server
   459  	rl := rate.NewLimiter(rate.Every(time.Second), 1 /* burst */)
   460  	for {
   461  		select {
   462  		case <-ctx.Done():
   463  			return nil
   464  		default:
   465  		}
   466  		err := rl.Wait(ctx)
   467  		if err != nil {
   468  			if errors.Cause(err) == context.Canceled {
   469  				return nil
   470  			}
   471  			return errors.Trace(err)
   472  		}
   473  		// Before campaign check liveness
   474  		if c.liveness.Load() == model.LivenessCaptureStopping {
   475  			log.Info("do not campaign owner, liveness is stopping",
   476  				zap.String("captureID", c.info.ID))
   477  			return nil
   478  		}
   479  		// Campaign to be the owner, it blocks until it been elected.
   480  		if err := c.campaign(ctx); err != nil {
   481  			rootErr := errors.Cause(err)
   482  			if rootErr == context.Canceled {
   483  				return nil
   484  			} else if rootErr == mvcc.ErrCompacted || isErrCompacted(rootErr) {
   485  				log.Warn("campaign owner failed due to etcd revision "+
   486  					"has been compacted, retry later", zap.Error(err))
   487  				continue
   488  			}
   489  			log.Warn("campaign owner failed",
   490  				zap.String("captureID", c.info.ID), zap.Error(err))
   491  			return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   492  		}
   493  		// After campaign check liveness again.
   494  		// It is possible it becomes the owner right after receiving SIGTERM.
   495  		if c.liveness.Load() == model.LivenessCaptureStopping {
   496  			// If the capture is stopping, resign actively.
   497  			log.Info("resign owner actively, liveness is stopping")
   498  			if resignErr := c.resign(ctx); resignErr != nil {
   499  				log.Warn("resign owner actively failed",
   500  					zap.String("captureID", c.info.ID), zap.Error(resignErr))
   501  				return errors.Trace(err)
   502  			}
   503  			return nil
   504  		}
   505  
   506  		ownerRev, err := c.EtcdClient.GetOwnerRevision(ctx, c.info.ID)
   507  		if err != nil {
   508  			if errors.Cause(err) == context.Canceled {
   509  				return nil
   510  			}
   511  			return errors.Trace(err)
   512  		}
   513  
   514  		// We do a copy of the globalVars here to avoid
   515  		// accidental modifications and potential race conditions.
   516  		globalVars := *globalVars
   517  		newGlobalVars := &globalVars
   518  		newGlobalVars.OwnerRevision = ownerRev
   519  
   520  		log.Info("campaign owner successfully",
   521  			zap.String("captureID", c.info.ID),
   522  			zap.Int64("ownerRev", ownerRev))
   523  
   524  		controller := c.newController(c.upstreamManager, c.info, c.EtcdClient)
   525  		owner := c.newOwner(c.upstreamManager, c.config.Debug.Scheduler, newGlobalVars)
   526  		c.setOwner(owner)
   527  		c.setController(controller)
   528  
   529  		globalState := orchestrator.NewGlobalState(c.EtcdClient.GetClusterID(), c.config.CaptureSessionTTL)
   530  
   531  		globalState.SetOnCaptureAdded(func(captureID model.CaptureID, addr string) {
   532  			c.MessageRouter.AddPeer(captureID, addr)
   533  		})
   534  		globalState.SetOnCaptureRemoved(func(captureID model.CaptureID) {
   535  			c.MessageRouter.RemovePeer(captureID)
   536  			// If an owner is killed by "kill -19", other CDC nodes will remove that capture,
   537  			// but the peer in the message server will not be removed, so the message server still sends
   538  			// ack message to that peer, until the write buffer is full. So we need to deregister the peer
   539  			// when the capture is removed.
   540  			if err := c.MessageServer.ScheduleDeregisterPeerTask(ctx, captureID); err != nil {
   541  				log.Warn("deregister peer failed",
   542  					zap.String("captureID", captureID),
   543  					zap.Error(err))
   544  			}
   545  		})
   546  
   547  		g, ctx := errgroup.WithContext(ctx)
   548  		g.Go(func() error {
   549  			return c.runEtcdWorker(ctx, owner.(orchestrator.Reactor),
   550  				orchestrator.NewGlobalState(c.EtcdClient.GetClusterID(), c.config.CaptureSessionTTL),
   551  				ownerFlushInterval, util.RoleOwner.String())
   552  		})
   553  		g.Go(func() error {
   554  			er := c.runEtcdWorker(ctx, controller.(orchestrator.Reactor),
   555  				globalState,
   556  				// todo: do not use owner flush interval
   557  				ownerFlushInterval, util.RoleController.String())
   558  			// controller has exited, stop owner.
   559  			c.owner.AsyncStop()
   560  			return er
   561  		})
   562  		err = g.Wait()
   563  		c.owner.AsyncStop()
   564  		c.controller.AsyncStop()
   565  		c.setController(nil)
   566  		c.setOwner(nil)
   567  
   568  		if !cerror.ErrNotOwner.Equal(err) {
   569  			// if owner exits, resign the owner key,
   570  			// use a new context to prevent the context from being cancelled.
   571  			resignCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
   572  			if resignErr := c.resign(resignCtx); resignErr != nil {
   573  				if errors.Cause(resignErr) != context.DeadlineExceeded {
   574  					log.Info("owner resign failed", zap.String("captureID", c.info.ID),
   575  						zap.Error(resignErr), zap.Int64("ownerRev", ownerRev))
   576  					cancel()
   577  					return errors.Trace(resignErr)
   578  				}
   579  
   580  				log.Warn("owner resign timeout", zap.String("captureID", c.info.ID),
   581  					zap.Error(resignErr), zap.Int64("ownerRev", ownerRev))
   582  			}
   583  			cancel()
   584  		}
   585  
   586  		log.Info("owner resigned successfully",
   587  			zap.String("captureID", c.info.ID), zap.Int64("ownerRev", ownerRev))
   588  		if err != nil {
   589  			log.Warn("run owner exited with error",
   590  				zap.String("captureID", c.info.ID), zap.Int64("ownerRev", ownerRev),
   591  				zap.Error(err))
   592  			// for errors, return error and let capture exits or restart
   593  			return errors.Trace(err)
   594  		}
   595  		// if owner exits normally, continue the campaign loop and try to election owner again
   596  		log.Info("run owner exited normally",
   597  			zap.String("captureID", c.info.ID), zap.Int64("ownerRev", ownerRev))
   598  	}
   599  }
   600  
   601  func (c *captureImpl) runEtcdWorker(
   602  	ctx context.Context,
   603  	reactor orchestrator.Reactor,
   604  	reactorState *orchestrator.GlobalReactorState,
   605  	timerInterval time.Duration,
   606  	role string,
   607  ) error {
   608  	reactorState.Role = role
   609  	etcdWorker, err := orchestrator.NewEtcdWorker(c.EtcdClient,
   610  		etcd.BaseKey(c.EtcdClient.GetClusterID()), reactor, reactorState, c.migrator)
   611  	if err != nil {
   612  		return errors.Trace(err)
   613  	}
   614  	if err := etcdWorker.Run(ctx, c.session, timerInterval, role); err != nil {
   615  		// We check ttl of lease instead of check `session.Done`, because
   616  		// `session.Done` is only notified when etcd client establish a
   617  		// new keepalive request, there could be a time window as long as
   618  		// 1/3 of session ttl that `session.Done` can't be triggered even
   619  		// the lease is already revoked.
   620  		switch {
   621  		case cerror.ErrEtcdSessionDone.Equal(err),
   622  			cerror.ErrLeaseExpired.Equal(err):
   623  			log.Warn("session is disconnected", zap.Error(err))
   624  			return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   625  		}
   626  		lease, inErr := c.EtcdClient.GetEtcdClient().TimeToLive(ctx, c.session.Lease())
   627  		if inErr != nil {
   628  			return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr)
   629  		}
   630  		if lease.TTL == int64(-1) {
   631  			log.Warn("session is disconnected", zap.Error(err))
   632  			return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   633  		}
   634  		return errors.Trace(err)
   635  	}
   636  	return nil
   637  }
   638  
   639  func (c *captureImpl) setOwner(owner owner.Owner) {
   640  	c.ownerMu.Lock()
   641  	defer c.ownerMu.Unlock()
   642  	c.owner = owner
   643  }
   644  
   645  func (c *captureImpl) setController(controller controller.Controller) {
   646  	c.ownerMu.Lock()
   647  	defer c.ownerMu.Unlock()
   648  	c.controller = controller
   649  }
   650  
   651  // GetOwner returns owner if it is the owner.
   652  func (c *captureImpl) GetOwner() (owner.Owner, error) {
   653  	c.ownerMu.Lock()
   654  	defer c.ownerMu.Unlock()
   655  	if c.owner == nil {
   656  		return nil, cerror.ErrNotOwner.GenWithStackByArgs()
   657  	}
   658  	return c.owner, nil
   659  }
   660  
   661  // GetController returns `controller.Controller` if not nil
   662  func (c *captureImpl) GetController() (controller.Controller, error) {
   663  	c.ownerMu.Lock()
   664  	defer c.ownerMu.Unlock()
   665  	if c.owner == nil {
   666  		return nil, cerror.ErrNotOwner.GenWithStackByArgs()
   667  	}
   668  	return c.controller, nil
   669  }
   670  
   671  // campaign to be an owner.
   672  func (c *captureImpl) campaign(ctx context.Context) error {
   673  	// TODO: `Campaign` will get stuck when send SIGSTOP to pd leader.
   674  	// For `Campaign`, when send SIGSTOP to pd leader, cdc maybe call `cancel`
   675  	// (cause by `processor routine` exit). And inside `Campaign`, the routine
   676  	// return from `waitDeletes`(https://github.com/etcd-io/etcd/blob/main/client/v3/concurrency/election.go#L93),
   677  	// then call `Resign`(note: use `client.Ctx`) to etcd server. But the etcd server
   678  	// (the client connects to) has entered the STOP state, which means that
   679  	// the server cannot process the request, but will still maintain the GRPC
   680  	// connection. So `routine` will block 'Resign'.
   681  	return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.campaign(ctx, c.info.ID))
   682  }
   683  
   684  // resign lets an owner start a new election.
   685  func (c *captureImpl) resign(ctx context.Context) error {
   686  	failpoint.Inject("capture-resign-failed", func() {
   687  		failpoint.Return(errors.New("capture resign failed"))
   688  	})
   689  	if c.election == nil {
   690  		return nil
   691  	}
   692  	return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.resign(ctx))
   693  }
   694  
   695  // register the capture by put the capture's information in etcd
   696  func (c *captureImpl) register(ctx context.Context) error {
   697  	err := c.EtcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease())
   698  	if err != nil {
   699  		return cerror.WrapError(cerror.ErrCaptureRegister, err)
   700  	}
   701  	return nil
   702  }
   703  
   704  // Close closes the capture by deregister it from etcd,
   705  // it also closes the owner and processorManager
   706  // Note: this function should be reentrant
   707  func (c *captureImpl) Close() {
   708  	defer c.cancel()
   709  	// Safety: Here we mainly want to stop the owner
   710  	// and ignore it if the owner does not exist or is not set.
   711  	o, _ := c.GetOwner()
   712  	if o != nil {
   713  		o.AsyncStop()
   714  		log.Info("owner closed", zap.String("captureID", c.info.ID))
   715  	}
   716  
   717  	c.captureMu.Lock()
   718  	defer c.captureMu.Unlock()
   719  
   720  	c.grpcService.Reset(nil)
   721  	if c.MessageRouter != nil {
   722  		c.MessageRouter.Close()
   723  		c.MessageRouter = nil
   724  	}
   725  	log.Info("message router closed", zap.String("captureID", c.info.ID))
   726  }
   727  
   728  // Drain removes tables in the current TiCDC instance.
   729  func (c *captureImpl) Drain() <-chan struct{} {
   730  	done := make(chan struct{})
   731  	go func() {
   732  		// Set liveness stopping first, no matter is the owner or not.
   733  		// this is triggered by user manually stop the TiCDC instance by sent signals.
   734  		// It may cost a few seconds before cdc server fully stop, set it to `stopping` to prevent
   735  		// the capture become the leader or tables dispatched to it.
   736  		c.liveness.Store(model.LivenessCaptureStopping)
   737  
   738  		// if the instance is the owner, resign the ownership
   739  		if o, _ := c.GetOwner(); o != nil {
   740  			o.AsyncStop()
   741  		}
   742  		close(done)
   743  	}()
   744  	return done
   745  }
   746  
   747  // Liveness returns liveness of the capture.
   748  func (c *captureImpl) Liveness() model.Liveness {
   749  	return c.liveness.Load()
   750  }
   751  
   752  // WriteDebugInfo writes the debug info into writer.
   753  func (c *captureImpl) WriteDebugInfo(ctx context.Context, w io.Writer) {
   754  	wait := func(done <-chan error) {
   755  		var err error
   756  		select {
   757  		case <-ctx.Done():
   758  			err = ctx.Err()
   759  		case err = <-done:
   760  		}
   761  		if err != nil {
   762  			log.Warn("write debug info failed", zap.Error(err))
   763  		}
   764  	}
   765  	// Safety: Because we are mainly outputting information about the owner here,
   766  	// if the owner does not exist or is not set, the information will not be output.
   767  	o, _ := c.GetOwner()
   768  	if o != nil {
   769  		doneOwner := make(chan error, 1)
   770  		fmt.Fprintf(w, "\n\n*** owner info ***:\n\n")
   771  		o.WriteDebugInfo(w, doneOwner)
   772  		// wait the debug info printed
   773  		wait(doneOwner)
   774  	}
   775  
   776  	doneM := make(chan error, 1)
   777  	c.captureMu.Lock()
   778  	if c.processorManager != nil {
   779  		fmt.Fprintf(w, "\n\n*** processors info ***:\n\n")
   780  		c.processorManager.WriteDebugInfo(ctx, w, doneM)
   781  	}
   782  	// NOTICE: we must release the lock before wait the debug info process down.
   783  	// Otherwise, the capture initialization and request response will compete
   784  	// for captureMu resulting in a deadlock.
   785  	c.captureMu.Unlock()
   786  	// wait the debug info printed
   787  	wait(doneM)
   788  }
   789  
   790  // IsController returns whether the capture is a controller
   791  func (c *captureImpl) IsController() bool {
   792  	c.ownerMu.Lock()
   793  	defer c.ownerMu.Unlock()
   794  	return c.controller != nil
   795  }
   796  
   797  // GetControllerCaptureInfo return the controller capture info of current TiCDC cluster
   798  func (c *captureImpl) GetControllerCaptureInfo(ctx context.Context) (*model.CaptureInfo, error) {
   799  	_, captureInfos, err := c.EtcdClient.GetCaptures(ctx)
   800  	if err != nil {
   801  		return nil, err
   802  	}
   803  
   804  	ownerID, err := c.EtcdClient.GetOwnerID(ctx)
   805  	if err != nil {
   806  		return nil, err
   807  	}
   808  
   809  	for _, captureInfo := range captureInfos {
   810  		if captureInfo.ID == ownerID {
   811  			return captureInfo, nil
   812  		}
   813  	}
   814  	return nil, cerror.ErrOwnerNotFound.FastGenByArgs()
   815  }
   816  
   817  // StatusProvider returns owner's StatusProvider.
   818  func (c *captureImpl) StatusProvider() owner.StatusProvider {
   819  	c.ownerMu.Lock()
   820  	defer c.ownerMu.Unlock()
   821  	if c.owner == nil {
   822  		return nil
   823  	}
   824  	return owner.NewStatusProvider(c.owner)
   825  }
   826  
   827  func (c *captureImpl) IsReady() bool {
   828  	return c.migrator.IsMigrateDone()
   829  }
   830  
   831  func isErrCompacted(err error) bool {
   832  	return strings.Contains(err.Error(), "required revision has been compacted")
   833  }