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

     1  // Copyright 2020 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 orchestrator
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"strconv"
    20  	"time"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/failpoint"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tiflow/pkg/chdelay"
    26  	cerrors "github.com/pingcap/tiflow/pkg/errors"
    27  	"github.com/pingcap/tiflow/pkg/etcd"
    28  	"github.com/pingcap/tiflow/pkg/migrate"
    29  	"github.com/pingcap/tiflow/pkg/orchestrator/util"
    30  	pkgutil "github.com/pingcap/tiflow/pkg/util"
    31  	"github.com/prometheus/client_golang/prometheus"
    32  	"go.etcd.io/etcd/api/v3/etcdserverpb"
    33  	"go.etcd.io/etcd/api/v3/mvccpb"
    34  	"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
    35  	clientv3 "go.etcd.io/etcd/client/v3"
    36  	"go.etcd.io/etcd/client/v3/concurrency"
    37  	"go.uber.org/zap"
    38  	"go.uber.org/zap/zapcore"
    39  	"golang.org/x/time/rate"
    40  )
    41  
    42  const (
    43  	// When EtcdWorker commits a txn to etcd or ticks its reactor
    44  	// takes more than etcdWorkerLogsWarnDuration, it will print a log
    45  	etcdWorkerLogsWarnDuration = 1 * time.Second
    46  )
    47  
    48  // EtcdWorker handles all interactions with Etcd
    49  type EtcdWorker struct {
    50  	clusterID string
    51  	client    *etcd.Client
    52  	reactor   Reactor
    53  	state     ReactorState
    54  	// rawState is the local cache of the latest Etcd state.
    55  	rawState map[util.EtcdKey]rawStateEntry
    56  	// pendingUpdates stores Etcd updates that the Reactor has not been notified of.
    57  	pendingUpdates []*etcdUpdate
    58  	// revision is the Etcd revision of the latest event received from Etcd
    59  	// (which has not necessarily been applied to the ReactorState)
    60  	revision int64
    61  	// reactor.Tick() should not be called until revision >= barrierRev.
    62  	barrierRev int64
    63  	// prefix is the scope of Etcd watch
    64  	prefix util.EtcdPrefix
    65  	// deleteCounter maintains a local copy of a value stored in Etcd used to
    66  	// keep track of how many deletes have been committed by an EtcdWorker
    67  	// watching this key prefix.
    68  	// This mechanism is necessary as a workaround to correctly detect
    69  	// write-write conflicts when at least a transaction commits a delete,
    70  	// because deletes in Etcd reset the mod-revision of keys, making it
    71  	// difficult to use it as a unique version identifier to implement
    72  	// a `compare-and-swap` semantics, which is essential for implementing
    73  	// snapshot isolation for Reactor ticks.
    74  	deleteCounter int64
    75  	metrics       *etcdWorkerMetrics
    76  
    77  	migrator     migrate.Migrator
    78  	ownerMetaKey string
    79  	isOwner      bool
    80  }
    81  
    82  type etcdWorkerMetrics struct {
    83  	// kv events related metrics
    84  	metricEtcdTxnSize            prometheus.Observer
    85  	metricEtcdTxnDuration        prometheus.Observer
    86  	metricEtcdWorkerTickDuration prometheus.Observer
    87  }
    88  
    89  type etcdUpdate struct {
    90  	key      util.EtcdKey
    91  	value    []byte
    92  	revision int64
    93  }
    94  
    95  // rawStateEntry stores the latest version of a key as seen by the EtcdWorker.
    96  // modRevision is stored to implement `compare-and-swap` semantics more reliably.
    97  type rawStateEntry struct {
    98  	value       []byte
    99  	modRevision int64
   100  }
   101  
   102  // NewEtcdWorker returns a new EtcdWorker
   103  func NewEtcdWorker(
   104  	client etcd.CDCEtcdClient,
   105  	prefix string,
   106  	reactor Reactor,
   107  	initState ReactorState,
   108  	migrator migrate.Migrator,
   109  ) (*EtcdWorker, error) {
   110  	return &EtcdWorker{
   111  		clusterID:  client.GetClusterID(),
   112  		client:     client.GetEtcdClient(),
   113  		reactor:    reactor,
   114  		state:      initState,
   115  		rawState:   make(map[util.EtcdKey]rawStateEntry),
   116  		prefix:     util.NormalizePrefix(prefix),
   117  		barrierRev: -1, // -1 indicates no barrier
   118  		migrator:   migrator,
   119  	}, nil
   120  }
   121  
   122  func (worker *EtcdWorker) initMetrics() {
   123  	metrics := &etcdWorkerMetrics{}
   124  	metrics.metricEtcdTxnSize = etcdTxnSize
   125  	metrics.metricEtcdTxnDuration = etcdTxnExecDuration
   126  	metrics.metricEtcdWorkerTickDuration = etcdWorkerTickDuration
   127  	worker.metrics = metrics
   128  }
   129  
   130  // Run starts the EtcdWorker event loop.
   131  // A tick is generated either on a timer whose interval is timerInterval, or on an Etcd event.
   132  // If the specified etcd session is Done, this Run function will exit with cerrors.ErrEtcdSessionDone.
   133  // And the specified etcd session is nil-safety.
   134  func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, timerInterval time.Duration, role string) error {
   135  	defer worker.cleanUp()
   136  	worker.isOwner = role == pkgutil.RoleController.String()
   137  	// migrate data here
   138  	err := worker.checkAndMigrateMetaData(ctx, role)
   139  	if err != nil {
   140  		return errors.Trace(err)
   141  	}
   142  	// migration is done, cdc server can serve http now
   143  	worker.migrator.MarkMigrateDone()
   144  
   145  	worker.initMetrics()
   146  
   147  	err = worker.syncRawState(ctx)
   148  	if err != nil {
   149  		return errors.Trace(err)
   150  	}
   151  
   152  	ticker := time.NewTicker(timerInterval)
   153  	defer ticker.Stop()
   154  
   155  	watchCtx, cancel := context.WithCancel(ctx)
   156  	defer cancel()
   157  	watchCh := worker.client.Watch(watchCtx, worker.prefix.String(), role, clientv3.WithPrefix(), clientv3.WithRev(worker.revision+1))
   158  
   159  	if role == pkgutil.RoleProcessor.String() {
   160  		failpoint.Inject("ProcessorEtcdDelay", func() {
   161  			delayer := chdelay.NewChannelDelayer(time.Second*3, watchCh, 1024, 16)
   162  			defer delayer.Close()
   163  			watchCh = delayer.Out()
   164  		})
   165  	}
   166  
   167  	var (
   168  		pendingPatches   [][]DataPatch
   169  		committedChanges int
   170  		exiting          bool
   171  		retry            bool
   172  		sessionDone      <-chan struct{}
   173  	)
   174  	if session != nil {
   175  		sessionDone = session.Done()
   176  		worker.ownerMetaKey = fmt.Sprintf("%s/%x",
   177  			etcd.CaptureOwnerKey(worker.clusterID), session.Lease())
   178  	} else {
   179  		// should never be closed
   180  		sessionDone = make(chan struct{})
   181  	}
   182  
   183  	// limit the number of times EtcdWorker can tick
   184  	rl := rate.NewLimiter(rate.Every(timerInterval), 2)
   185  	for {
   186  		select {
   187  		case <-ctx.Done():
   188  			return ctx.Err()
   189  		case <-sessionDone:
   190  			return cerrors.ErrEtcdSessionDone.GenWithStackByArgs()
   191  		case <-ticker.C:
   192  			// There is no new event to handle on timer ticks, so we have nothing here.
   193  		case response := <-watchCh:
   194  			// In this select case, we receive new events from Etcd, and call handleEvent if appropriate.
   195  			if err := response.Err(); err != nil {
   196  				return errors.Trace(err)
   197  			}
   198  
   199  			// ProgressNotify implies no new events.
   200  			if response.IsProgressNotify() {
   201  				log.Debug("Etcd progress notification",
   202  					zap.Int64("revision", response.Header.GetRevision()))
   203  				// Note that we don't need to update the revision here, and we
   204  				// should not do so, because the revision of the progress notification
   205  				// may not satisfy the strict monotonicity we have expected.
   206  				//
   207  				// Updating `worker.revision` can cause a useful event with the
   208  				// same revision to be dropped erroneously.
   209  				//
   210  				// Refer to https://etcd.io/docs/v3.3/dev-guide/interacting_v3/#watch-progress
   211  				// "Note: The revision number in the progress notify response is the revision
   212  				// from the local etcd server node that the watch stream is connected to. [...]"
   213  				// This implies that the progress notification will NOT go through the raft
   214  				// consensus, thereby NOT affecting the revision (index).
   215  				continue
   216  			}
   217  
   218  			// Check whether the response is stale.
   219  			if worker.revision >= response.Header.GetRevision() {
   220  				log.Info("Stale Etcd event dropped",
   221  					zap.Int64("eventRevision", response.Header.GetRevision()),
   222  					zap.Int64("previousRevision", worker.revision),
   223  					zap.Any("events", response.Events),
   224  					zap.String("role", role))
   225  				continue
   226  			}
   227  			worker.revision = response.Header.GetRevision()
   228  
   229  			for _, event := range response.Events {
   230  				// handleEvent will apply the event to our internal `rawState`.
   231  				err = worker.handleEvent(ctx, event)
   232  				if err != nil {
   233  					// This error means owner is resigned by itself,
   234  					// and we should exit etcd worker and campaign owner again.
   235  					return err
   236  				}
   237  			}
   238  
   239  		}
   240  
   241  		tryCommitPendingPatches := func() (bool, error) {
   242  			if len(pendingPatches) > 0 {
   243  				// Here we have some patches yet to be uploaded to Etcd.
   244  				pendingPatches, committedChanges, err = worker.applyPatchGroups(ctx, pendingPatches)
   245  				if isRetryableError(err) {
   246  					return true, nil
   247  				}
   248  				if err != nil {
   249  					return false, errors.Trace(err)
   250  				}
   251  				// pendingPatches size may greater than 0, but nothing to commit to etcd
   252  				// if nothing is changed, we should not break this tick
   253  				if committedChanges > 0 {
   254  					return true, nil
   255  				}
   256  			}
   257  			return false, nil
   258  		}
   259  		if retry, err = tryCommitPendingPatches(); err != nil {
   260  			return err
   261  		}
   262  		if exiting {
   263  			// If exiting is true here, it means that the reactor returned `ErrReactorFinished` last tick,
   264  			// and all pending patches is applied.
   265  			return nil
   266  		}
   267  		if retry {
   268  			continue
   269  		}
   270  		if worker.revision < worker.barrierRev {
   271  			// We hold off notifying the Reactor because barrierRev has not been reached.
   272  			// This usually happens when a committed write Txn has not been received by Watch.
   273  			continue
   274  		}
   275  
   276  		// We are safe to update the ReactorState only if there is no pending patch.
   277  		if err := worker.applyUpdates(); err != nil {
   278  			return errors.Trace(err)
   279  		}
   280  
   281  		// If !rl.Allow(), skip this Tick to avoid etcd worker tick reactor too frequency.
   282  		// It makes etcdWorker to batch etcd changed event in worker.state.
   283  		// The semantics of `ReactorState` requires that any implementation
   284  		// can batch updates internally.
   285  		if !rl.Allow() {
   286  			continue
   287  		}
   288  		startTime := time.Now()
   289  		// it is safe that a batch of updates has been applied to worker.state before worker.reactor.Tick
   290  		nextState, err := worker.reactor.Tick(ctx, worker.state)
   291  		costTime := time.Since(startTime)
   292  		if costTime > etcdWorkerLogsWarnDuration {
   293  			log.Warn("EtcdWorker reactor tick took too long",
   294  				zap.Duration("duration", costTime),
   295  				zap.String("role", role))
   296  		}
   297  		worker.metrics.metricEtcdWorkerTickDuration.Observe(costTime.Seconds())
   298  		if err != nil {
   299  			if !cerrors.ErrReactorFinished.Equal(errors.Cause(err)) {
   300  				return errors.Trace(err)
   301  			}
   302  			// normal exit
   303  			exiting = true
   304  		}
   305  		worker.state = nextState
   306  		pendingPatches = append(pendingPatches, nextState.GetPatches()...)
   307  		// apply pending patches
   308  		if _, err = tryCommitPendingPatches(); err != nil {
   309  			return err
   310  		}
   311  	}
   312  }
   313  
   314  func isRetryableError(err error) bool {
   315  	err = errors.Cause(err)
   316  	if cerrors.ErrEtcdTryAgain.Equal(err) ||
   317  		context.DeadlineExceeded == err {
   318  		return true
   319  	}
   320  	// When encountering an abnormal connection with etcd, the worker will keep retrying
   321  	// until the session is done.
   322  	_, ok := err.(rpctypes.EtcdError)
   323  	return ok
   324  }
   325  
   326  func (worker *EtcdWorker) handleEvent(_ context.Context, event *clientv3.Event) error {
   327  	if worker.isDeleteCounterKey(event.Kv.Key) {
   328  		switch event.Type {
   329  		case mvccpb.PUT:
   330  			worker.handleDeleteCounter(event.Kv.Value)
   331  		case mvccpb.DELETE:
   332  			log.Warn("deletion counter key deleted", zap.Reflect("event", event))
   333  			worker.handleDeleteCounter(nil)
   334  		}
   335  		// We return here because the delete-counter is not used for business logic,
   336  		// and it should not be exposed further to the Reactor.
   337  		return nil
   338  	}
   339  
   340  	worker.pendingUpdates = append(worker.pendingUpdates, &etcdUpdate{
   341  		key:      util.NewEtcdKeyFromBytes(event.Kv.Key),
   342  		value:    event.Kv.Value,
   343  		revision: event.Kv.ModRevision,
   344  	})
   345  
   346  	switch event.Type {
   347  	case mvccpb.PUT:
   348  		value := event.Kv.Value
   349  		if value == nil {
   350  			value = []byte{}
   351  		}
   352  		worker.rawState[util.NewEtcdKeyFromBytes(event.Kv.Key)] = rawStateEntry{
   353  			value:       value,
   354  			modRevision: event.Kv.ModRevision,
   355  		}
   356  	case mvccpb.DELETE:
   357  		delete(worker.rawState, util.NewEtcdKeyFromBytes(event.Kv.Key))
   358  		if string(event.Kv.Key) == worker.ownerMetaKey {
   359  			if worker.isOwner {
   360  				err := cerrors.ErrNotOwner.GenWithStackByArgs()
   361  				log.Error("owner key is delete, it may causes by "+
   362  					"owner resign or externally delete operation"+
   363  					"exit etcd worker and campaign again",
   364  					zap.String("ownerMetaKey", worker.ownerMetaKey),
   365  					zap.String("value", string(event.Kv.Value)),
   366  					zap.Error(err))
   367  				return err
   368  			}
   369  		}
   370  	}
   371  	return nil
   372  }
   373  
   374  func (worker *EtcdWorker) syncRawState(ctx context.Context) error {
   375  	resp, err := worker.client.Get(ctx, worker.prefix.String(), clientv3.WithPrefix())
   376  	if err != nil {
   377  		return errors.Trace(err)
   378  	}
   379  
   380  	worker.rawState = make(map[util.EtcdKey]rawStateEntry)
   381  	for _, kv := range resp.Kvs {
   382  		if worker.isDeleteCounterKey(kv.Key) {
   383  			worker.handleDeleteCounter(kv.Value)
   384  			continue
   385  		}
   386  		key := util.NewEtcdKeyFromBytes(kv.Key)
   387  		worker.rawState[key] = rawStateEntry{
   388  			value:       kv.Value,
   389  			modRevision: kv.ModRevision,
   390  		}
   391  		err := worker.state.Update(key, kv.Value, true)
   392  		if err != nil {
   393  			return errors.Trace(err)
   394  		}
   395  	}
   396  
   397  	worker.revision = resp.Header.Revision
   398  	return nil
   399  }
   400  
   401  func (worker *EtcdWorker) cloneRawState() map[util.EtcdKey][]byte {
   402  	ret := make(map[util.EtcdKey][]byte)
   403  	for k, v := range worker.rawState {
   404  		vCloned := make([]byte, len(v.value))
   405  		copy(vCloned, v.value)
   406  		ret[util.NewEtcdKey(k.String())] = vCloned
   407  	}
   408  	return ret
   409  }
   410  
   411  func (worker *EtcdWorker) applyPatchGroups(ctx context.Context, patchGroups [][]DataPatch) ([][]DataPatch, int, error) {
   412  	state := worker.cloneRawState()
   413  	committedChanges := 0
   414  	for len(patchGroups) > 0 {
   415  		changeSate, n, size, err := getBatchChangedState(state, patchGroups)
   416  		if err != nil {
   417  			return patchGroups, committedChanges, err
   418  		}
   419  		committedChanges += len(changeSate)
   420  		err = worker.commitChangedState(ctx, changeSate, size)
   421  		if err != nil {
   422  			return patchGroups, committedChanges, err
   423  		}
   424  		patchGroups = patchGroups[n:]
   425  	}
   426  	return patchGroups, committedChanges, nil
   427  }
   428  
   429  func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState map[util.EtcdKey][]byte, size int) error {
   430  	if len(changedState) == 0 {
   431  		return nil
   432  	}
   433  
   434  	cmps := make([]clientv3.Cmp, 0, len(changedState))
   435  	opsThen := make([]clientv3.Op, 0, len(changedState))
   436  	hasDelete := false
   437  
   438  	for key, value := range changedState {
   439  		// make sure someone else has not updated the key after the last snapshot
   440  		var cmp clientv3.Cmp
   441  		if entry, ok := worker.rawState[key]; ok {
   442  			cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "=", entry.modRevision)
   443  		} else {
   444  			// if ok is false, it means that the key of this patch is not exist in a committed state
   445  			// this compare is equivalent to `patch.Key` is not exist
   446  			cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "=", 0)
   447  		}
   448  		cmps = append(cmps, cmp)
   449  
   450  		var op clientv3.Op
   451  		if value != nil {
   452  			op = clientv3.OpPut(key.String(), string(value))
   453  		} else {
   454  			op = clientv3.OpDelete(key.String())
   455  			hasDelete = true
   456  		}
   457  		opsThen = append(opsThen, op)
   458  	}
   459  
   460  	if hasDelete {
   461  		opsThen = append(opsThen, clientv3.OpPut(worker.prefix.String()+etcd.DeletionCounterKey,
   462  			fmt.Sprint(worker.deleteCounter+1)))
   463  	}
   464  	if worker.deleteCounter > 0 {
   465  		cmps = append(cmps, clientv3.Compare(clientv3.Value(worker.prefix.String()+
   466  			etcd.DeletionCounterKey),
   467  			"=", fmt.Sprint(worker.deleteCounter)))
   468  	} else if worker.deleteCounter == 0 {
   469  		cmps = append(cmps, clientv3.Compare(clientv3.CreateRevision(worker.prefix.String()+
   470  			etcd.DeletionCounterKey),
   471  			"=", 0))
   472  	} else {
   473  		panic("unreachable")
   474  	}
   475  
   476  	worker.metrics.metricEtcdTxnSize.Observe(float64(size))
   477  	startTime := time.Now()
   478  	resp, err := worker.client.Txn(ctx, cmps, opsThen, etcd.TxnEmptyOpsElse)
   479  
   480  	// For testing the situation where we have a progress notification that
   481  	// has the same revision as the committed Etcd transaction.
   482  	failpoint.Inject("InjectProgressRequestAfterCommit", func() {
   483  		if err := worker.client.RequestProgress(ctx); err != nil {
   484  			failpoint.Return(errors.Trace(err))
   485  		}
   486  	})
   487  
   488  	costTime := time.Since(startTime)
   489  	if costTime > etcdWorkerLogsWarnDuration {
   490  		log.Warn("Etcd transaction took too long", zap.Duration("duration", costTime))
   491  	}
   492  	worker.metrics.metricEtcdTxnDuration.Observe(costTime.Seconds())
   493  	if err != nil {
   494  		return errors.Trace(err)
   495  	}
   496  
   497  	logEtcdOps(opsThen, resp.Succeeded)
   498  	if resp.Succeeded {
   499  		worker.barrierRev = resp.Header.GetRevision()
   500  		return nil
   501  	}
   502  
   503  	// Logs the conditions for the failed Etcd transaction.
   504  	worker.logEtcdCmps(cmps)
   505  	return cerrors.ErrEtcdTryAgain.GenWithStackByArgs()
   506  }
   507  
   508  func (worker *EtcdWorker) applyUpdates() error {
   509  	for _, update := range worker.pendingUpdates {
   510  		err := worker.state.Update(update.key, update.value, false)
   511  		if err != nil {
   512  			return errors.Trace(err)
   513  		}
   514  	}
   515  	worker.state.UpdatePendingChange()
   516  
   517  	worker.pendingUpdates = worker.pendingUpdates[:0]
   518  	return nil
   519  }
   520  
   521  func logEtcdOps(ops []clientv3.Op, committed bool) {
   522  	if committed && (log.GetLevel() != zapcore.DebugLevel || len(ops) == 0) {
   523  		return
   524  	}
   525  	logFn := log.Debug
   526  	if !committed {
   527  		logFn = log.Info
   528  	}
   529  
   530  	logFn("[etcd worker] ==========Update State to ETCD==========")
   531  	for _, op := range ops {
   532  		if op.IsDelete() {
   533  			logFn("[etcd worker] delete key", zap.ByteString("key", op.KeyBytes()))
   534  		} else {
   535  			logFn("[etcd worker] put key", zap.ByteString("key", op.KeyBytes()), zap.ByteString("value", op.ValueBytes()))
   536  		}
   537  	}
   538  	logFn("[etcd worker] ============State Commit=============", zap.Bool("committed", committed))
   539  }
   540  
   541  func (worker *EtcdWorker) logEtcdCmps(cmps []clientv3.Cmp) {
   542  	log.Info("[etcd worker] ==========Failed Etcd Txn Cmps==========")
   543  	for _, cmp := range cmps {
   544  		cmp := etcdserverpb.Compare(cmp)
   545  		log.Info("[etcd worker] compare",
   546  			zap.String("cmp", cmp.String()))
   547  	}
   548  	log.Info("[etcd worker] ============End Failed Etcd Txn Cmps=============")
   549  }
   550  
   551  func (worker *EtcdWorker) cleanUp() {
   552  	worker.rawState = nil
   553  	worker.revision = 0
   554  	worker.pendingUpdates = worker.pendingUpdates[:0]
   555  }
   556  
   557  func (worker *EtcdWorker) isDeleteCounterKey(key []byte) bool {
   558  	return string(key) == worker.prefix.String()+etcd.DeletionCounterKey
   559  }
   560  
   561  func (worker *EtcdWorker) handleDeleteCounter(value []byte) {
   562  	if len(value) == 0 {
   563  		// The delete counter key has been deleted, resetting the internal counter
   564  		worker.deleteCounter = 0
   565  		return
   566  	}
   567  
   568  	var err error
   569  	worker.deleteCounter, err = strconv.ParseInt(string(value), 10, 64)
   570  	if err != nil {
   571  		// This should never happen unless Etcd server has been tampered with.
   572  		log.Panic("strconv failed. Unexpected Etcd state.", zap.Error(err))
   573  	}
   574  	if worker.deleteCounter <= 0 {
   575  		log.Panic("unexpected delete counter", zap.Int64("value", worker.deleteCounter))
   576  	}
   577  }
   578  
   579  // checkAndMigrateMetaData check if we should migrate meta, if true, it will block
   580  // until migrate done
   581  func (worker *EtcdWorker) checkAndMigrateMetaData(
   582  	ctx context.Context, role string,
   583  ) error {
   584  	shouldMigrate, err := worker.migrator.ShouldMigrate(ctx)
   585  	if err != nil {
   586  		return errors.Trace(err)
   587  	}
   588  	if !shouldMigrate {
   589  		return nil
   590  	}
   591  
   592  	if role != pkgutil.RoleController.String() {
   593  		err := worker.migrator.WaitMetaVersionMatched(ctx)
   594  		return errors.Trace(err)
   595  	}
   596  
   597  	err = worker.migrator.Migrate(ctx)
   598  	return err
   599  }