github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdcv2/metadata/sql/observation.go (about)

     1  // Copyright 2023 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 sql
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"fmt"
    20  	"hash/fnv"
    21  	"sort"
    22  	"strings"
    23  	"sync"
    24  	"time"
    25  
    26  	"github.com/pingcap/log"
    27  	"github.com/pingcap/tiflow/cdc/model"
    28  	"github.com/pingcap/tiflow/cdcv2/metadata"
    29  	ormUtil "github.com/pingcap/tiflow/engine/pkg/orm"
    30  	"github.com/pingcap/tiflow/pkg/chann"
    31  	"github.com/pingcap/tiflow/pkg/election"
    32  	"github.com/pingcap/tiflow/pkg/errors"
    33  	"github.com/pingcap/tiflow/pkg/security"
    34  	"go.uber.org/zap"
    35  	"golang.org/x/sync/errgroup"
    36  	"gorm.io/gorm"
    37  )
    38  
    39  var (
    40  	_ metadata.Querier               = &CaptureOb[*gorm.DB]{}
    41  	_ metadata.CaptureObservation    = &CaptureOb[*gorm.DB]{}
    42  	_ metadata.ControllerObservation = &ControllerOb[*gorm.DB]{}
    43  	_ metadata.OwnerObservation      = &OwnerOb[*gorm.DB]{}
    44  )
    45  
    46  // CaptureOb is an implement for metadata.CaptureObservation.
    47  type CaptureOb[T TxnContext] struct {
    48  	// election related fields.
    49  	metadata.Elector
    50  	selfInfo *model.CaptureInfo
    51  
    52  	// TODO(CharlesCheung): handle ctx properly.
    53  	egCtx         context.Context
    54  	client        client[T]
    55  	uuidGenerator uuidGenerator
    56  
    57  	tasks *entity[metadata.ChangefeedUUID, *ScheduleDO]
    58  
    59  	// TODO: remove processorChanges.
    60  	ownerChanges     *chann.DrainableChann[metadata.ScheduledChangefeed]
    61  	processorChanges *chann.DrainableChann[metadata.ScheduledChangefeed]
    62  }
    63  
    64  // NewCaptureObservation creates a capture observation.
    65  func NewCaptureObservation(
    66  	backendDB *sql.DB, selfInfo *model.CaptureInfo, opts ...ClientOptionFunc,
    67  ) (*CaptureOb[*gorm.DB], error) {
    68  	db, err := ormUtil.NewGormDB(backendDB, "mysql")
    69  	if err != nil {
    70  		return nil, err
    71  	}
    72  	electionStorage, err := election.NewORMStorage(db, "election")
    73  	if err != nil {
    74  		return nil, errors.Trace(err)
    75  	}
    76  	ormClient := NewORMClient(selfInfo.ID, db)
    77  
    78  	if err := AutoMigrate(db); err != nil {
    79  		return nil, errors.Trace(err)
    80  	}
    81  	return &CaptureOb[*gorm.DB]{
    82  		selfInfo:         selfInfo,
    83  		client:           newClient(electionStorage, ormClient, opts...),
    84  		uuidGenerator:    NewUUIDGenerator("orm", db),
    85  		tasks:            newEntity[metadata.ChangefeedUUID, *ScheduleDO](defaultMaxExecTime),
    86  		Elector:          metadata.NewElector(selfInfo, electionStorage),
    87  		ownerChanges:     chann.NewAutoDrainChann[metadata.ScheduledChangefeed](),
    88  		processorChanges: chann.NewAutoDrainChann[metadata.ScheduledChangefeed](),
    89  	}, nil
    90  }
    91  
    92  // Run runs the given CaptureOb.
    93  func (c *CaptureOb[T]) Run(
    94  	egCtx context.Context,
    95  	controllerCallback func(context.Context, metadata.ControllerObservation) error,
    96  ) (err error) {
    97  	eg, egCtx := errgroup.WithContext(egCtx)
    98  	c.egCtx = egCtx
    99  
   100  	err = c.client.Txn(egCtx, func(tx T) error {
   101  		return c.client.createProgress(tx, &ProgressDO{
   102  			CaptureID: c.selfInfo.ID,
   103  			Progress:  nil,
   104  			Version:   1,
   105  		})
   106  	})
   107  	if err != nil {
   108  		return errors.Trace(err)
   109  	}
   110  
   111  	eg.Go(func() error {
   112  		return c.Elector.RunElection(egCtx, c.onTakeControl(controllerCallback))
   113  	})
   114  
   115  	eg.Go(func() error {
   116  		ticker := time.NewTicker(500 * time.Millisecond)
   117  		defer ticker.Stop()
   118  		for {
   119  			select {
   120  			case <-egCtx.Done():
   121  				err := egCtx.Err()
   122  				log.Info("capture stops handle task changes", zap.String("capture", c.selfInfo.ID), zap.Error(err))
   123  				return err
   124  			case <-ticker.C:
   125  				if err := c.handleTaskChanges(egCtx); err != nil {
   126  					log.Warn("capture handle task changes fail", zap.String("capture", c.selfInfo.ID), zap.Error(err))
   127  					return err
   128  				}
   129  			}
   130  		}
   131  	})
   132  
   133  	// TODO: add runWithEg function to reduce the wait goroutine.
   134  	return eg.Wait()
   135  }
   136  
   137  func (c *CaptureOb[T]) onTakeControl(
   138  	controllerCallback func(context.Context, metadata.ControllerObservation) error,
   139  ) func(context.Context) error {
   140  	return func(ctx context.Context) error {
   141  		controllerOb := newControllerObservation(c.client, c.uuidGenerator, c.selfInfo, c.getAllCaptures)
   142  
   143  		eg, egCtx := errgroup.WithContext(ctx)
   144  		eg.Go(func() error {
   145  			return controllerOb.run(egCtx)
   146  		})
   147  		eg.Go(func() error {
   148  			return controllerCallback(egCtx, controllerOb)
   149  		})
   150  		return eg.Wait()
   151  	}
   152  }
   153  
   154  func (c *CaptureOb[T]) handleTaskChanges(ctx context.Context) error {
   155  	var err error
   156  	var schedItems []*ScheduleDO
   157  
   158  	err = c.client.Txn(ctx, func(tx T) error {
   159  		lastSafePoint := c.tasks.getSafePoint()
   160  		schedItems, err = c.client.querySchedulesByOwnerIDAndUpdateAt(tx, c.Self().ID, lastSafePoint)
   161  		return err
   162  	})
   163  	if err != nil {
   164  		return errors.Trace(err)
   165  	}
   166  
   167  	if len(schedItems) == 0 {
   168  		// No scheudle information for the capture.
   169  		return nil
   170  	}
   171  
   172  	c.tasks.doUpsert(schedItems, func(newV *ScheduleDO) (skip bool) {
   173  		if newV.OwnerState == metadata.SchedRemoved {
   174  			return true
   175  		}
   176  		c.ownerChanges.In() <- newV.ScheduledChangefeed
   177  		c.processorChanges.In() <- newV.ScheduledChangefeed
   178  		return false
   179  	})
   180  
   181  	return nil
   182  }
   183  
   184  // Advance updates the progress of the capture.
   185  func (c *CaptureOb[T]) Advance(cp metadata.CaptureProgress) error {
   186  	return c.client.Txn(c.egCtx, func(tx T) error {
   187  		pr, err := c.client.queryProgressByCaptureID(tx, c.selfInfo.ID)
   188  		if err != nil {
   189  			return errors.Trace(err)
   190  		}
   191  		if pr == nil {
   192  			erMsg := fmt.Sprintf("expect non-empty progress for capture %s", c.selfInfo.ID)
   193  			return errors.ErrMetaInvalidState.GenWithStackByArgs(erMsg)
   194  		}
   195  		return c.client.updateProgress(tx, &ProgressDO{
   196  			CaptureID: c.selfInfo.ID,
   197  			Progress:  &cp,
   198  			Version:   pr.Version,
   199  		})
   200  	})
   201  }
   202  
   203  // OwnerChanges returns a channel that receives changefeeds when the owner of the changefeed changes.
   204  func (c *CaptureOb[T]) OwnerChanges() <-chan metadata.ScheduledChangefeed {
   205  	return c.ownerChanges.Out()
   206  }
   207  
   208  // OnOwnerLaunched is called when the owner of a changefeed is launched.
   209  func (c *CaptureOb[T]) OnOwnerLaunched(cf metadata.ChangefeedUUID) metadata.OwnerObservation {
   210  	return newOwnerObservation[T](c, cf)
   211  }
   212  
   213  // PostOwnerRemoved is called when the owner of a changefeed is removed.
   214  func (c *CaptureOb[T]) PostOwnerRemoved(cf metadata.ChangefeedUUID, taskPosition metadata.ChangefeedProgress) error {
   215  	sc := c.tasks.get(cf)
   216  	if sc == nil {
   217  		errMsg := fmt.Sprintf("remove owner for a changefeed %d that is not owned by the capture", cf)
   218  		return errors.ErrInconsistentMetaCache.GenWithStackByArgs(errMsg)
   219  	}
   220  
   221  	sc.TaskPosition = taskPosition
   222  	return c.client.TxnWithOwnerLock(c.egCtx, cf, func(tx T) error {
   223  		return c.client.updateSchedule(tx, sc)
   224  	})
   225  }
   226  
   227  // ProcessorChanges returns a channel that receives changefeeds when the changefeed changes.
   228  func (c *CaptureOb[T]) ProcessorChanges() <-chan metadata.ScheduledChangefeed {
   229  	return c.processorChanges.Out()
   230  }
   231  
   232  // GetChangefeed returns the changefeeds with the given UUIDs.
   233  func (c *CaptureOb[T]) GetChangefeed(cfs ...metadata.ChangefeedUUID) (infos []*metadata.ChangefeedInfo, err error) {
   234  	var cfDOs []*ChangefeedInfoDO
   235  	err = c.client.Txn(c.egCtx, func(tx T) error {
   236  		if len(cfs) == 0 {
   237  			cfDOs, err = c.client.queryChangefeedInfos(tx)
   238  			return err
   239  		}
   240  		cfDOs, err = c.client.queryChangefeedInfosByUUIDs(tx, cfs...)
   241  		return err
   242  	})
   243  	if err != nil {
   244  		return nil, errors.Trace(err)
   245  	}
   246  
   247  	for _, cfDO := range cfDOs {
   248  		infos = append(infos, &cfDO.ChangefeedInfo)
   249  	}
   250  	return
   251  }
   252  
   253  // GetChangefeedState returns the state of the changefeed with the given UUID.
   254  func (c *CaptureOb[T]) GetChangefeedState(cfs ...metadata.ChangefeedUUID) (states []*metadata.ChangefeedState, err error) {
   255  	var cfDOs []*ChangefeedStateDO
   256  	err = c.client.Txn(c.egCtx, func(tx T) error {
   257  		if len(cfs) == 0 {
   258  			cfDOs, err = c.client.queryChangefeedStates(tx)
   259  			return err
   260  		}
   261  		cfDOs, err = c.client.queryChangefeedStateByUUIDs(tx, cfs...)
   262  		return err
   263  	})
   264  	if err != nil {
   265  		return nil, errors.Trace(err)
   266  	}
   267  
   268  	for _, cfDO := range cfDOs {
   269  		states = append(states, &cfDO.ChangefeedState)
   270  	}
   271  	return
   272  }
   273  
   274  // GetChangefeedProgress returns the progress of the changefeed with the given UUID.
   275  func (c *CaptureOb[T]) GetChangefeedProgress(
   276  	cfs ...metadata.ChangefeedUUID,
   277  ) (progresses map[metadata.ChangefeedUUID]metadata.ChangefeedProgress, err error) {
   278  	var prDOs []*ProgressDO
   279  	var scDOs []*ScheduleDO
   280  	err = c.client.Txn(c.egCtx, func(tx T) error {
   281  		prDOs, err = c.client.queryProgresses(tx)
   282  		if err != nil {
   283  			return err
   284  		}
   285  
   286  		scDOs, err = c.client.querySchedules(tx)
   287  		return err
   288  	})
   289  	if err != nil {
   290  		return nil, errors.Trace(err)
   291  	}
   292  
   293  	cfMap := make(map[metadata.ChangefeedUUID]struct{})
   294  	for _, cf := range cfs {
   295  		cfMap[cf] = struct{}{}
   296  	}
   297  	queryAll := len(cfMap) == 0
   298  
   299  	progresses = make(map[metadata.ChangefeedUUID]metadata.ChangefeedProgress)
   300  	for _, prDO := range prDOs {
   301  		if prDO.Progress != nil {
   302  			for cf, pos := range *prDO.Progress {
   303  				if _, ok := cfMap[cf]; ok || queryAll {
   304  					progresses[cf] = pos
   305  				}
   306  			}
   307  		}
   308  	}
   309  
   310  	if queryAll || len(progresses) < len(cfMap) {
   311  		for _, scDO := range scDOs {
   312  			if _, alreadyFound := progresses[scDO.ChangefeedUUID]; alreadyFound {
   313  				continue
   314  			}
   315  			if _, ok := cfMap[scDO.ChangefeedUUID]; ok || queryAll {
   316  				progresses[scDO.ChangefeedUUID] = scDO.TaskPosition
   317  			}
   318  		}
   319  	}
   320  
   321  	return
   322  }
   323  
   324  func (c *CaptureOb[T]) getAllCaptures() []*model.CaptureInfo {
   325  	infos, _ := c.GetCaptures()
   326  	return infos
   327  }
   328  
   329  // ControllerOb is an implement for metadata.ControllerObservation.
   330  type ControllerOb[T TxnContext] struct {
   331  	selfInfo *model.CaptureInfo
   332  	client   client[T]
   333  
   334  	// TODO(CharlesCheung): handle ctx properly.
   335  	// egCtx is the inner ctx of elector.
   336  	egCtx         context.Context
   337  	uuidGenerator uuidGenerator
   338  
   339  	aliveCaptures struct {
   340  		sync.Mutex
   341  		outgoing     []*model.CaptureInfo
   342  		incoming     []*model.CaptureInfo
   343  		outgoingHash uint64
   344  		incomingHash uint64
   345  	}
   346  
   347  	getAllCaptures func() []*model.CaptureInfo
   348  }
   349  
   350  func newControllerObservation[T TxnContext](
   351  	client client[T],
   352  	uuidGenerator uuidGenerator,
   353  	selfInfo *model.CaptureInfo,
   354  	getAllCaptures func() []*model.CaptureInfo,
   355  ) *ControllerOb[T] {
   356  	return &ControllerOb[T]{
   357  		client:         client,
   358  		selfInfo:       selfInfo,
   359  		getAllCaptures: getAllCaptures,
   360  		uuidGenerator:  uuidGenerator,
   361  	}
   362  }
   363  
   364  func (c *ControllerOb[T]) run(ctx context.Context) error {
   365  	c.egCtx = ctx
   366  	if err := c.init(); err != nil {
   367  		return errors.Trace(err)
   368  	}
   369  
   370  	ticker := time.NewTicker(500 * time.Millisecond)
   371  	defer ticker.Stop()
   372  	for {
   373  		select {
   374  		case <-ctx.Done():
   375  			err := ctx.Err()
   376  			log.Info("controller stops handle alive captures ", zap.String("capture", c.selfInfo.ID), zap.Error(err))
   377  			return err
   378  		case <-ticker.C:
   379  		}
   380  
   381  		if err := c.handleAliveCaptures(); err != nil {
   382  			log.Warn("controller handle alive captures fail", zap.String("capture", c.selfInfo.ID), zap.Error(err))
   383  			return err
   384  		}
   385  	}
   386  }
   387  
   388  func (c *ControllerOb[T]) init() error {
   389  	var (
   390  		captureOfflined   []model.CaptureID
   391  		captureOnline     = make(map[model.CaptureID]struct{})
   392  		capturesScheduled []model.CaptureID
   393  		err               error
   394  	)
   395  	err = c.client.Txn(c.egCtx, func(tx T) error {
   396  		capturesScheduled, err = c.client.querySchedulesUinqueOwnerIDs(tx)
   397  		return err
   398  	})
   399  	if err != nil {
   400  		return errors.Trace(err)
   401  	}
   402  
   403  	currentCaptures := c.getAllCaptures()
   404  	for _, capture := range currentCaptures {
   405  		captureOnline[capture.ID] = struct{}{}
   406  	}
   407  
   408  	for _, captureID := range capturesScheduled {
   409  		if _, ok := captureOnline[captureID]; !ok {
   410  			captureOfflined = append(captureOfflined, captureID)
   411  		}
   412  	}
   413  	return c.onCaptureOffline(captureOfflined...)
   414  }
   415  
   416  func (c *ControllerOb[T]) handleAliveCaptures() error {
   417  	alives := c.getAllCaptures()
   418  	hash := sortAndHashCaptureList(alives)
   419  
   420  	c.aliveCaptures.Lock()
   421  	defer c.aliveCaptures.Unlock()
   422  	c.aliveCaptures.incomingHash = hash
   423  	c.aliveCaptures.incoming = alives
   424  	return nil
   425  }
   426  
   427  func (c *ControllerOb[T]) upsertUpstream(tx T, up *model.UpstreamInfo) error {
   428  	newUp := &UpstreamDO{
   429  		ID:        up.ID,
   430  		Endpoints: up.PDEndpoints,
   431  		Config: &security.Credential{
   432  			CAPath:        up.CAPath,
   433  			CertPath:      up.CertPath,
   434  			KeyPath:       up.KeyPath,
   435  			CertAllowedCN: up.CertAllowedCN,
   436  		},
   437  		Version: 1,
   438  	}
   439  
   440  	// Create or update the upstream info.
   441  	oldUp, err := c.client.queryUpstreamByID(tx, up.ID)
   442  	if errors.Is(errors.Cause(err), gorm.ErrRecordNotFound) {
   443  		return c.client.createUpstream(tx, newUp)
   444  	} else if err != nil {
   445  		return errors.Trace(err)
   446  	}
   447  
   448  	if oldUp == nil {
   449  		errMsg := fmt.Sprintf("expect non-empty upstream info for id %d", up.ID)
   450  		return errors.Trace(errors.ErrMetaInvalidState.GenWithStackByArgs(errMsg))
   451  	}
   452  	if !oldUp.equal(newUp) {
   453  		newUp.Version = oldUp.Version
   454  		if err := c.client.updateUpstream(tx, newUp); err != nil {
   455  			return errors.Trace(err)
   456  		}
   457  	}
   458  	return nil
   459  }
   460  
   461  func (c *ControllerOb[T]) txnWithLeaderLock(fn func(T) error) error {
   462  	return c.client.TxnWithLeaderLock(c.egCtx, c.selfInfo.ID, fn)
   463  }
   464  
   465  // CreateChangefeed initializes the changefeed info, schedule info and state info of the given changefeed. It also
   466  // updates or creates the upstream info depending on whether the upstream info exists.
   467  func (c *ControllerOb[T]) CreateChangefeed(cf *metadata.ChangefeedInfo, up *model.UpstreamInfo) (metadata.ChangefeedIdent, error) {
   468  	if cf.UpstreamID != up.ID {
   469  		errMsg := fmt.Sprintf("changefeed %s has different upstream id %d from the given upstream id %d",
   470  			cf.ChangefeedIdent, cf.UpstreamID, up.ID)
   471  		return cf.ChangefeedIdent, errors.ErrMetaInvalidState.GenWithStackByArgs(errMsg)
   472  	}
   473  	uuid, err := c.uuidGenerator.GenChangefeedUUID(c.egCtx)
   474  	if err != nil {
   475  		return cf.ChangefeedIdent, errors.Trace(err)
   476  	}
   477  
   478  	cf.ChangefeedIdent.UUID = uuid
   479  	err = c.txnWithLeaderLock(func(tx T) error {
   480  		if err := c.upsertUpstream(tx, up); err != nil {
   481  			return errors.Trace(err)
   482  		}
   483  
   484  		if err := c.client.createChangefeedInfo(tx, &ChangefeedInfoDO{
   485  			ChangefeedInfo: *cf,
   486  			Version:        1,
   487  		}); err != nil {
   488  			return errors.Trace(err)
   489  		}
   490  
   491  		if err := c.client.createSchedule(tx, &ScheduleDO{
   492  			ScheduledChangefeed: metadata.ScheduledChangefeed{
   493  				ChangefeedUUID: cf.UUID,
   494  				Owner:          nil,
   495  				OwnerState:     metadata.SchedRemoved,
   496  				Processors:     nil,
   497  				TaskPosition: metadata.ChangefeedProgress{
   498  					CheckpointTs:      cf.StartTs,
   499  					MinTableBarrierTs: cf.StartTs,
   500  				},
   501  			},
   502  			Version: 1,
   503  		}); err != nil {
   504  			return errors.Trace(err)
   505  		}
   506  
   507  		return c.client.createChangefeedState(tx, &ChangefeedStateDO{
   508  			ChangefeedState: metadata.ChangefeedState{
   509  				ChangefeedUUID: cf.UUID,
   510  				State:          model.StateUnInitialized,
   511  				Error:          nil,
   512  				Warning:        nil,
   513  			},
   514  			Version: 1,
   515  		})
   516  	})
   517  	return cf.ChangefeedIdent, err
   518  }
   519  
   520  // RemoveChangefeed removes the changefeed info
   521  func (c *ControllerOb[T]) RemoveChangefeed(cf metadata.ChangefeedUUID) error {
   522  	return c.txnWithLeaderLock(func(tx T) error {
   523  		oldInfo, err := c.client.queryChangefeedInfoByUUID(tx, cf)
   524  		if err != nil {
   525  			return errors.Trace(err)
   526  		}
   527  		err = c.client.markChangefeedRemoved(tx, &ChangefeedInfoDO{
   528  			ChangefeedInfo: metadata.ChangefeedInfo{
   529  				ChangefeedIdent: metadata.ChangefeedIdent{
   530  					UUID: cf,
   531  				},
   532  			},
   533  			Version: oldInfo.Version,
   534  		})
   535  		if err != nil {
   536  			return errors.Trace(err)
   537  		}
   538  
   539  		sc, err := c.client.queryScheduleByUUID(tx, cf)
   540  		if err != nil {
   541  			return errors.Trace(err)
   542  		}
   543  		if sc.OwnerState == metadata.SchedLaunched {
   544  			err = c.client.updateScheduleOwnerState(tx, &ScheduleDO{
   545  				ScheduledChangefeed: metadata.ScheduledChangefeed{
   546  					ChangefeedUUID: cf,
   547  					OwnerState:     metadata.SchedRemoving,
   548  				},
   549  				Version: sc.Version,
   550  			})
   551  			if err != nil {
   552  				return errors.Trace(err)
   553  			}
   554  		}
   555  		return nil
   556  	})
   557  }
   558  
   559  // CleanupChangefeed removes the changefeed info, schedule info and state info of the given changefeed.
   560  // Note that this function should only be called when the owner is removed and changefeed is marked as removed.
   561  func (c *ControllerOb[T]) CleanupChangefeed(cf metadata.ChangefeedUUID) error {
   562  	return c.txnWithLeaderLock(func(tx T) error {
   563  		err := c.client.deleteChangefeedInfo(tx, &ChangefeedInfoDO{
   564  			ChangefeedInfo: metadata.ChangefeedInfo{
   565  				ChangefeedIdent: metadata.ChangefeedIdent{
   566  					UUID: cf,
   567  				},
   568  			},
   569  		})
   570  		if err != nil {
   571  			return errors.Trace(err)
   572  		}
   573  
   574  		err = c.client.deleteChangefeedState(tx, &ChangefeedStateDO{
   575  			ChangefeedState: metadata.ChangefeedState{
   576  				ChangefeedUUID: cf,
   577  			},
   578  		})
   579  		if err != nil {
   580  			return errors.Trace(err)
   581  		}
   582  
   583  		err = c.client.deleteSchedule(tx, &ScheduleDO{
   584  			ScheduledChangefeed: metadata.ScheduledChangefeed{
   585  				ChangefeedUUID: cf,
   586  			},
   587  		})
   588  		if err != nil {
   589  			return errors.Trace(err)
   590  		}
   591  
   592  		return nil
   593  	})
   594  }
   595  
   596  // RefreshCaptures Fetch the latest capture list in the TiCDC cluster.
   597  func (c *ControllerOb[T]) RefreshCaptures() (captures []*model.CaptureInfo, changed bool) {
   598  	c.aliveCaptures.Lock()
   599  	defer c.aliveCaptures.Unlock()
   600  	if c.aliveCaptures.outgoingHash != c.aliveCaptures.incomingHash {
   601  		c.aliveCaptures.outgoingHash = c.aliveCaptures.incomingHash
   602  		c.aliveCaptures.outgoing = c.aliveCaptures.incoming
   603  	}
   604  	captures = make([]*model.CaptureInfo, len(c.aliveCaptures.outgoing))
   605  	copy(captures, c.aliveCaptures.outgoing)
   606  	return
   607  }
   608  
   609  // onCaptureOffline is called when a capture is offline.
   610  func (c *ControllerOb[T]) onCaptureOffline(ids ...model.CaptureID) error {
   611  	// TODO(CharlesCheung): use multiple statements to reduce the number of round trips.
   612  	// Note currently we only handle single capture offline, so it is not a big deal.
   613  	for _, id := range ids {
   614  		err := c.txnWithLeaderLock(func(tx T) error {
   615  			prs, err := c.client.queryProgressByCaptureIDsWithLock(tx, []model.CaptureID{id})
   616  			if err != nil {
   617  				return errors.Trace(err)
   618  			}
   619  			prMap := prs[0]
   620  			if prMap.Progress == nil || len(*prMap.Progress) == 0 {
   621  				log.Info("no progress is updated by the capture", zap.String("capture", id))
   622  			} else {
   623  				for cf, taskPosition := range *prMap.Progress {
   624  					// TODO(CharlesCheung): maybe such operation should be done in background.
   625  					oldSc, err := c.client.queryScheduleByUUID(tx, cf)
   626  					if err != nil {
   627  						return errors.Trace(err)
   628  					}
   629  					if *oldSc.Owner != id || oldSc.OwnerState == metadata.SchedRemoved {
   630  						continue
   631  					}
   632  					newSc := &ScheduleDO{
   633  						ScheduledChangefeed: metadata.ScheduledChangefeed{
   634  							ChangefeedUUID: oldSc.ChangefeedUUID,
   635  							Owner:          nil,
   636  							OwnerState:     metadata.SchedRemoved,
   637  							Processors:     nil,
   638  							TaskPosition:   taskPosition,
   639  						},
   640  						Version: oldSc.Version,
   641  					}
   642  					// TODO: use Model to prevent nil value from being ignored.
   643  					err = c.client.updateSchedule(tx, newSc)
   644  					if err != nil {
   645  						return errors.Trace(err)
   646  					}
   647  				}
   648  			}
   649  
   650  			// If capture fails before a owner progress is updated for the first time, the corresponding
   651  			// taskPosition will not stored in Progress. In this case, we also need to set owner removed.
   652  			// Version would not be checked here because multiple rows may be updated.
   653  			err = c.client.updateScheduleOwnerStateByOwnerID(tx, metadata.SchedRemoved, id)
   654  			if err != nil {
   655  				return errors.Trace(err)
   656  			}
   657  
   658  			return c.client.deleteProgress(tx, prMap)
   659  		})
   660  		if err != nil {
   661  			return errors.Trace(err)
   662  		}
   663  	}
   664  	return nil
   665  }
   666  
   667  // SetOwner Schedule a changefeed owner to a given target.
   668  func (c *ControllerOb[T]) SetOwner(target metadata.ScheduledChangefeed) error {
   669  	return c.txnWithLeaderLock(func(tx T) error {
   670  		old, err := c.client.queryScheduleByUUID(tx, target.ChangefeedUUID)
   671  		if err != nil {
   672  			return errors.Trace(err)
   673  		}
   674  		if err := metadata.CheckScheduleState(old.ScheduledChangefeed, target); err != nil {
   675  			return errors.Trace(err)
   676  		}
   677  		return c.client.updateSchedule(tx, &ScheduleDO{
   678  			ScheduledChangefeed: target,
   679  			Version:             old.Version,
   680  		})
   681  	})
   682  }
   683  
   684  // GetChangefeedSchedule Get current schedule of the given changefeed.
   685  func (c *ControllerOb[T]) GetChangefeedSchedule(cf metadata.ChangefeedUUID) (metadata.ScheduledChangefeed, error) {
   686  	var ret metadata.ScheduledChangefeed
   687  	err := c.client.Txn(c.egCtx, func(tx T) error {
   688  		sc, inErr := c.client.queryScheduleByUUID(tx, cf)
   689  		if inErr != nil {
   690  			return errors.Trace(inErr)
   691  		}
   692  		ret = sc.ScheduledChangefeed
   693  		return nil
   694  	})
   695  	return ret, err
   696  }
   697  
   698  // ScheduleSnapshot Get a snapshot of all changefeeds current schedule.
   699  func (c *ControllerOb[T]) ScheduleSnapshot() (ss []metadata.ScheduledChangefeed, cs []*model.CaptureInfo, err error) {
   700  	err = c.client.Txn(c.egCtx, func(tx T) error {
   701  		scs, inErr := c.client.querySchedules(tx)
   702  		if inErr != nil {
   703  			return errors.Trace(inErr)
   704  		}
   705  		for _, sc := range scs {
   706  			ss = append(ss, sc.ScheduledChangefeed)
   707  		}
   708  		return err
   709  	})
   710  
   711  	cs = c.getAllCaptures()
   712  	hash := sortAndHashCaptureList(cs)
   713  	c.aliveCaptures.Lock()
   714  	defer c.aliveCaptures.Unlock()
   715  	c.aliveCaptures.outgoingHash = hash
   716  	c.aliveCaptures.outgoing = cs
   717  	return
   718  }
   719  
   720  // OwnerOb is an implement for metadata.OwnerObservation.
   721  type OwnerOb[T TxnContext] struct {
   722  	egCtx  context.Context
   723  	client client[T]
   724  	cfUUID metadata.ChangefeedUUID
   725  }
   726  
   727  func newOwnerObservation[T TxnContext](c *CaptureOb[T], cf metadata.ChangefeedUUID) *OwnerOb[T] {
   728  	return &OwnerOb[T]{
   729  		egCtx:  c.egCtx,
   730  		client: c.client,
   731  		cfUUID: cf,
   732  	}
   733  }
   734  
   735  // Self returns the changefeed info of the owner.
   736  // nolint:unused
   737  func (o *OwnerOb[T]) Self() metadata.ChangefeedUUID {
   738  	return o.cfUUID
   739  }
   740  
   741  func (o *OwnerOb[T]) updateChangefeedState(
   742  	state model.FeedState,
   743  	cfErr *model.RunningError,
   744  	cfWarn *model.RunningError,
   745  ) error {
   746  	return o.client.TxnWithOwnerLock(o.egCtx, o.cfUUID, func(tx T) error {
   747  		oldState, err := o.client.queryChangefeedStateByUUID(tx, o.cfUUID)
   748  		if err != nil {
   749  			return errors.Trace(err)
   750  		}
   751  
   752  		newState := &ChangefeedStateDO{
   753  			ChangefeedState: metadata.ChangefeedState{
   754  				ChangefeedUUID: o.cfUUID,
   755  				State:          state,
   756  				Error:          oldState.Error,
   757  				Warning:        oldState.Warning,
   758  			},
   759  			Version: oldState.Version,
   760  		}
   761  		if cfErr != nil {
   762  			newState.Error = cfErr
   763  		}
   764  		if cfWarn != nil {
   765  			newState.Warning = cfWarn
   766  		}
   767  
   768  		return o.client.updateChangefeedState(tx, newState)
   769  	})
   770  }
   771  
   772  // UpdateChangefeed updates changefeed metadata, must be called on a paused one.
   773  // nolint:unused
   774  func (o *OwnerOb[T]) UpdateChangefeed(info *metadata.ChangefeedInfo) error {
   775  	return o.client.TxnWithOwnerLock(o.egCtx, o.cfUUID, func(tx T) error {
   776  		state, err := o.client.queryChangefeedStateByUUIDWithLock(tx, o.cfUUID)
   777  		if err != nil {
   778  			return errors.Trace(err)
   779  		}
   780  		if state.State != model.StateStopped && state.State != model.StateFailed {
   781  			return errors.ErrChangefeedUpdateRefused.GenWithStackByArgs(
   782  				"can only update changefeed config when it is stopped or failed",
   783  			)
   784  		}
   785  
   786  		oldInfo, err := o.client.queryChangefeedInfoByUUID(tx, o.cfUUID)
   787  		if err != nil {
   788  			return errors.Trace(err)
   789  		}
   790  		return o.client.updateChangefeedInfo(tx, &ChangefeedInfoDO{
   791  			ChangefeedInfo: *info,
   792  			Version:        oldInfo.Version,
   793  		})
   794  	})
   795  }
   796  
   797  // ResumeChangefeed resumes a changefeed.
   798  // nolint:unused
   799  func (o *OwnerOb[T]) ResumeChangefeed() error {
   800  	return o.updateChangefeedState(model.StateNormal, nil, nil)
   801  }
   802  
   803  // SetChangefeedPending sets the changefeed to state pending.
   804  // nolint:unused
   805  func (o *OwnerOb[T]) SetChangefeedPending(err *model.RunningError) error {
   806  	return o.updateChangefeedState(model.StatePending, err, nil)
   807  }
   808  
   809  // SetChangefeedFailed set the changefeed to state failed.
   810  // nolint:unused
   811  func (o *OwnerOb[T]) SetChangefeedFailed(err *model.RunningError) error {
   812  	return o.updateChangefeedState(model.StateFailed, err, nil)
   813  }
   814  
   815  // PauseChangefeed pauses a changefeed.
   816  // nolint:unused
   817  func (o *OwnerOb[T]) PauseChangefeed() error {
   818  	return o.updateChangefeedState(model.StateStopped, nil, nil)
   819  }
   820  
   821  // SetChangefeedRemoved set the changefeed to state removed.
   822  // nolint:unused
   823  func (o *OwnerOb[T]) SetChangefeedRemoved() error {
   824  	return o.updateChangefeedState(model.StateRemoved, nil, nil)
   825  }
   826  
   827  // SetChangefeedFinished set the changefeed to state finished.
   828  // nolint:unused
   829  func (o *OwnerOb[T]) SetChangefeedFinished() error {
   830  	return o.updateChangefeedState(model.StateFinished, nil, nil)
   831  }
   832  
   833  // SetChangefeedWarning set the changefeed to state warning.
   834  // nolint:unused
   835  func (o *OwnerOb[T]) SetChangefeedWarning(warn *model.RunningError) error {
   836  	return o.updateChangefeedState(model.StateWarning, nil, warn)
   837  }
   838  
   839  func sortAndHashCaptureList(cs []*model.CaptureInfo) uint64 {
   840  	hasher := fnv.New64()
   841  	sort.Slice(cs, func(i, j int) bool { return strings.Compare(cs[i].ID, cs[j].ID) < 0 })
   842  	for _, info := range cs {
   843  		hasher.Write([]byte(info.ID))
   844  	}
   845  	return hasher.Sum64()
   846  }