github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/worker/source_worker.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 worker
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"path/filepath"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/golang/protobuf/proto"
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/failpoint"
    26  	"github.com/pingcap/tiflow/dm/config"
    27  	"github.com/pingcap/tiflow/dm/pb"
    28  	"github.com/pingcap/tiflow/dm/pkg/binlog"
    29  	"github.com/pingcap/tiflow/dm/pkg/conn"
    30  	tcontext "github.com/pingcap/tiflow/dm/pkg/context"
    31  	"github.com/pingcap/tiflow/dm/pkg/etcdutil"
    32  	"github.com/pingcap/tiflow/dm/pkg/ha"
    33  	"github.com/pingcap/tiflow/dm/pkg/log"
    34  	"github.com/pingcap/tiflow/dm/pkg/streamer"
    35  	"github.com/pingcap/tiflow/dm/pkg/terror"
    36  	"github.com/pingcap/tiflow/dm/relay"
    37  	bf "github.com/pingcap/tiflow/pkg/binlog-filter"
    38  	clientv3 "go.etcd.io/etcd/client/v3"
    39  	"go.uber.org/atomic"
    40  	"go.uber.org/zap"
    41  )
    42  
    43  // SourceWorker manages a source(upstream) which is mainly related to subtasks and relay.
    44  type SourceWorker struct {
    45  	// ensure no other operation can be done when closing (we can use `WaitGroup`/`Context` to archive this)
    46  	// TODO: check what does it guards. Now it's used to guard relayHolder and relayPurger (maybe subTaskHolder?) since
    47  	// query-status maybe access them when closing/disable functionalities
    48  	// This lock is used to guards source worker's source config and subtask holder(subtask configs)
    49  	sync.RWMutex
    50  
    51  	wg     sync.WaitGroup
    52  	closed atomic.Bool
    53  
    54  	// context created when SourceWorker created, and canceled when closing
    55  	ctx    context.Context
    56  	cancel context.CancelFunc
    57  
    58  	cfg        *config.SourceConfig
    59  	sourceDB   *conn.BaseDB
    60  	sourceDBMu sync.Mutex // if the sourceDB can't be connected at start time, we try to re-connect before using it.
    61  
    62  	l log.Logger
    63  
    64  	sourceStatus atomic.Value // stores a pointer to SourceStatus
    65  
    66  	// subtask functionality
    67  	subTaskEnabled atomic.Bool
    68  	subTaskCtx     context.Context
    69  	subTaskCancel  context.CancelFunc
    70  	subTaskWg      sync.WaitGroup
    71  	subTaskHolder  *subTaskHolder
    72  
    73  	// relay functionality
    74  	// during relayEnabled == true, relayHolder and relayPurger should not be nil
    75  	relayEnabled atomic.Bool
    76  	relayCtx     context.Context
    77  	relayCancel  context.CancelFunc
    78  	relayWg      sync.WaitGroup
    79  	relayHolder  RelayHolder
    80  	relayPurger  relay.Purger
    81  	relayDir     string
    82  
    83  	startedRelayBySourceCfg bool
    84  
    85  	taskStatusChecker TaskStatusChecker
    86  
    87  	etcdClient *clientv3.Client
    88  
    89  	name string
    90  }
    91  
    92  // NewSourceWorker creates a new SourceWorker. The functionality of relay and subtask is disabled by default, need call EnableRelay
    93  // and EnableSubtask later.
    94  func NewSourceWorker(
    95  	cfg *config.SourceConfig,
    96  	etcdClient *clientv3.Client,
    97  	name string,
    98  	relayDir string,
    99  ) (w *SourceWorker, err error) {
   100  	w = &SourceWorker{
   101  		cfg:           cfg,
   102  		subTaskHolder: newSubTaskHolder(),
   103  		l:             log.With(zap.String("component", "worker controller")),
   104  		etcdClient:    etcdClient,
   105  		name:          name,
   106  		relayDir:      relayDir,
   107  	}
   108  	// keep running until canceled in `Close`.
   109  	w.ctx, w.cancel = context.WithCancel(context.Background())
   110  	w.closed.Store(true)
   111  	w.subTaskEnabled.Store(false)
   112  	w.relayEnabled.Store(false)
   113  
   114  	defer func(w2 *SourceWorker) {
   115  		if err != nil { // when err != nil, `w` will become nil in this func, so we pass `w` in defer.
   116  			// release resources, NOTE: we need to refactor New/Init/Start/Close for components later.
   117  			w2.cancel()
   118  			w2.subTaskHolder.closeAllSubTasks()
   119  		}
   120  	}(w)
   121  
   122  	// initial task status checker
   123  	if w.cfg.Checker.CheckEnable {
   124  		tsc := NewTaskStatusChecker(w.cfg.Checker, w)
   125  		err = tsc.Init()
   126  		if err != nil {
   127  			return nil, err
   128  		}
   129  		w.taskStatusChecker = tsc
   130  	}
   131  
   132  	InitConditionHub(w)
   133  
   134  	w.l.Info("initialized", zap.Stringer("cfg", cfg))
   135  
   136  	return w, nil
   137  }
   138  
   139  // Start starts working, but the functionalities should be turned on separately.
   140  func (w *SourceWorker) Start() {
   141  	// start task status checker
   142  	if w.cfg.Checker.CheckEnable {
   143  		w.taskStatusChecker.Start()
   144  	}
   145  
   146  	var err error
   147  	w.sourceDB, err = conn.GetUpstreamDB(&w.cfg.GetDecryptedClone().From)
   148  	if err != nil {
   149  		w.l.Error("can't connected to upstream", zap.Error(err))
   150  	}
   151  
   152  	w.wg.Add(1)
   153  	defer w.wg.Done()
   154  
   155  	w.l.Info("start running")
   156  
   157  	printTaskInterval := 30 * time.Second
   158  	failpoint.Inject("PrintStatusCheckSeconds", func(val failpoint.Value) {
   159  		if seconds, ok := val.(int); ok {
   160  			printTaskInterval = time.Duration(seconds) * time.Second
   161  			log.L().Info("set printStatusInterval",
   162  				zap.String("failpoint", "PrintStatusCheckSeconds"),
   163  				zap.Int("value", seconds))
   164  		}
   165  	})
   166  
   167  	ticker := time.NewTicker(printTaskInterval)
   168  	w.closed.Store(false)
   169  	defer ticker.Stop()
   170  	for {
   171  		select {
   172  		case <-w.ctx.Done():
   173  			w.l.Info("status print process exits!")
   174  			return
   175  		case <-ticker.C:
   176  			old := w.sourceStatus.Load()
   177  			if old != nil {
   178  				status := old.(*binlog.SourceStatus)
   179  				if time.Since(status.UpdateTime) < printTaskInterval/2 {
   180  					w.l.Info("we just updated the source status, skip once",
   181  						zap.Time("last update time", status.UpdateTime))
   182  					continue
   183  				}
   184  			}
   185  			if err2 := w.updateSourceStatus(w.ctx, true); err2 != nil {
   186  				if terror.ErrNoMasterStatus.Equal(err2) {
   187  					w.l.Warn("This source's bin_log is OFF, so it only supports full_mode.", zap.String("sourceID", w.cfg.SourceID), zap.Error(err2))
   188  				} else {
   189  					w.l.Error("failed to update source status", zap.Error(err2))
   190  				}
   191  				continue
   192  			}
   193  
   194  			sourceStatus := w.sourceStatus.Load().(*binlog.SourceStatus)
   195  			if w.l.Core().Enabled(zap.DebugLevel) {
   196  				w.l.Debug("runtime status", zap.String("status", w.GetUnitAndSourceStatusJSON("", sourceStatus)))
   197  			}
   198  
   199  			// periodically print the status and update metrics
   200  			w.Status("", sourceStatus)
   201  		}
   202  	}
   203  }
   204  
   205  // Stop stops working and releases resources.
   206  func (w *SourceWorker) Stop(graceful bool) {
   207  	if w.closed.Load() {
   208  		w.l.Warn("already closed")
   209  		return
   210  	}
   211  
   212  	// cancel status output ticker and wait for return
   213  	w.cancel()
   214  	w.wg.Wait()
   215  	w.relayWg.Wait()
   216  	w.subTaskWg.Wait()
   217  
   218  	w.Lock()
   219  	defer w.Unlock()
   220  
   221  	// close or kill all subtasks
   222  	if graceful {
   223  		w.subTaskHolder.closeAllSubTasks()
   224  	} else {
   225  		w.subTaskHolder.killAllSubTasks()
   226  	}
   227  
   228  	if w.relayHolder != nil {
   229  		w.relayHolder.Close()
   230  	}
   231  
   232  	if w.relayPurger != nil {
   233  		w.relayPurger.Close()
   234  	}
   235  
   236  	// close task status checker
   237  	if w.cfg.Checker.CheckEnable {
   238  		w.taskStatusChecker.Close()
   239  	}
   240  
   241  	w.sourceDB.Close()
   242  	w.sourceDB = nil
   243  
   244  	w.closed.Store(true)
   245  
   246  	w.l.Info("Stop worker")
   247  }
   248  
   249  // updateSourceStatus updates w.sourceStatus.
   250  func (w *SourceWorker) updateSourceStatus(ctx context.Context, needLock bool) error {
   251  	var cfg *config.SourceConfig
   252  	if needLock {
   253  		w.RLock()
   254  		cfg = w.cfg
   255  		w.RUnlock()
   256  	} else {
   257  		cfg = w.cfg
   258  	}
   259  	w.sourceDBMu.Lock()
   260  	if w.sourceDB == nil {
   261  		var err error
   262  		w.sourceDB, err = conn.GetUpstreamDB(&cfg.GetDecryptedClone().From)
   263  		if err != nil {
   264  			w.sourceDBMu.Unlock()
   265  			return err
   266  		}
   267  	}
   268  	w.sourceDBMu.Unlock()
   269  
   270  	status, err := binlog.GetSourceStatus(tcontext.NewContext(ctx, w.l), w.sourceDB, cfg.Flavor)
   271  	if err != nil {
   272  		return err
   273  	}
   274  
   275  	w.sourceStatus.Store(status)
   276  	return nil
   277  }
   278  
   279  // EnableRelay enables the functionality of start/watch/handle relay.
   280  // According to relay schedule of DM-master, a source worker will enable relay in two scenarios: its bound source has
   281  // `enable-relay: true` in config, or it has a UpstreamRelayWorkerKeyAdapter etcd KV.
   282  // The paths to EnableRelay are:
   283  // - source config `enable-relay: true`, which is checked in enableHandleSubtasks
   284  //   - when DM-worker Server.Start
   285  //   - when DM-worker Server watches a SourceBound change, which is to turn a free source worker to bound or notify a
   286  //     bound worker that source config has changed
   287  //   - when DM-worker Server fails watching and recovers from a snapshot
   288  //
   289  // - UpstreamRelayWorkerKeyAdapter
   290  //   - when DM-worker Server.Start
   291  //   - when DM-worker Server watches a UpstreamRelayWorkerKeyAdapter change
   292  //   - when DM-worker Server fails watching and recovers from a snapshot
   293  func (w *SourceWorker) EnableRelay(startBySourceCfg bool) (err error) {
   294  	w.l.Info("enter EnableRelay")
   295  	w.Lock()
   296  	defer w.Unlock()
   297  	if w.relayEnabled.Load() {
   298  		w.l.Warn("already enabled relay")
   299  		return nil
   300  	}
   301  
   302  	w.startedRelayBySourceCfg = startBySourceCfg
   303  
   304  	failpoint.Inject("MockGetSourceCfgFromETCD", func(_ failpoint.Value) {
   305  		failpoint.Goto("bypass")
   306  	})
   307  	// we need update config from etcd first in case this cfg is updated by master
   308  	if refreshErr := w.refreshSourceCfg(); refreshErr != nil {
   309  		return refreshErr
   310  	}
   311  	failpoint.Label("bypass")
   312  
   313  	w.relayCtx, w.relayCancel = context.WithCancel(w.ctx)
   314  	// 1. adjust relay starting position, to the earliest of subtasks
   315  	var subTaskCfgs map[string]config.SubTaskConfig
   316  	//nolint:dogsled
   317  	_, _, subTaskCfgs, _, err = w.fetchSubTasksAndAdjust()
   318  	if err != nil {
   319  		return err
   320  	}
   321  
   322  	dctx, dcancel := context.WithTimeout(w.etcdClient.Ctx(), time.Duration(len(subTaskCfgs)*3)*time.Second)
   323  	defer dcancel()
   324  	minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs)
   325  	if err1 != nil {
   326  		w.l.Error("meet error when EnableRelay", zap.Error(err1))
   327  	}
   328  
   329  	if minLoc != nil {
   330  		w.l.Info("get min location in all subtasks", zap.Stringer("location", *minLoc))
   331  		w.cfg.RelayBinLogName = binlog.RemoveRelaySubDirSuffix(minLoc.Position).Name
   332  		w.cfg.RelayBinlogGTID = minLoc.GTIDSetStr()
   333  		// set UUIDSuffix when bound to a source
   334  		w.cfg.UUIDSuffix, err = binlog.ExtractSuffix(minLoc.Position.Name)
   335  		if err != nil {
   336  			return err
   337  		}
   338  	} else {
   339  		// set UUIDSuffix even not checkpoint exist
   340  		// so we will still remove relay dir
   341  		w.cfg.UUIDSuffix = binlog.MinRelaySubDirSuffix
   342  	}
   343  
   344  	// 2. initial relay holder, the cfg's password need decrypt
   345  	// worker's relay-dir has higher priority than source's relay-dir
   346  	if w.relayDir != "" {
   347  		workerRelayDir := filepath.Join(w.relayDir, w.name)
   348  		log.L().Info("use worker's relay-dir", zap.String("RelayDir", workerRelayDir))
   349  		w.cfg.RelayDir = workerRelayDir
   350  	}
   351  
   352  	w.relayHolder = NewRelayHolder(w.cfg)
   353  	relayPurger, err := w.relayHolder.Init(w.relayCtx, []relay.PurgeInterceptor{
   354  		w,
   355  	})
   356  	if err != nil {
   357  		return err
   358  	}
   359  	w.relayPurger = relayPurger
   360  
   361  	// 3. get relay stage from etcd and check if need starting
   362  	// we get the newest relay stages directly which will omit the relay stage PUT/DELETE event
   363  	// because triggering these events is useless now
   364  	relayStage, revRelay, err := ha.GetRelayStage(w.etcdClient, w.cfg.SourceID)
   365  	if err != nil {
   366  		// TODO: need retry
   367  		return err
   368  	}
   369  	startImmediately := !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running
   370  	if startImmediately {
   371  		w.l.Info("start relay for existing relay stage")
   372  		w.relayHolder.Start()
   373  		w.relayPurger.Start()
   374  	}
   375  
   376  	// 4. watch relay stage
   377  	w.relayWg.Add(1)
   378  	go func() {
   379  		defer w.relayWg.Done()
   380  		// TODO: handle fatal error from observeRelayStage
   381  		//nolint:errcheck
   382  		w.observeRelayStage(w.relayCtx, w.etcdClient, revRelay)
   383  	}()
   384  
   385  	w.relayEnabled.Store(true)
   386  	w.l.Info("relay enabled")
   387  	w.subTaskHolder.resetAllSubTasks(w.getRelayWithoutLock())
   388  	return nil
   389  }
   390  
   391  // DisableRelay disables the functionality of start/watch/handle relay.
   392  // a source worker will disable relay by the reason of EnableRelay is no longer valid.
   393  // The paths to DisableRelay are:
   394  // - source config `enable-relay: true` no longer valid
   395  //   - when DM-worker Server watches a SourceBound change, which is to notify that source config has changed, and the
   396  //     worker has started relay by that bound
   397  //   - when the source worker is unbound and has started relay by that bound
   398  //
   399  // - UpstreamRelayWorkerKeyAdapter no longer valid
   400  //   - when DM-worker Server watches a UpstreamRelayWorkerKeyAdapter change
   401  //   - when DM-worker Server fails watching and recovers from a snapshot
   402  func (w *SourceWorker) DisableRelay() {
   403  	w.l.Info("enter DisableRelay")
   404  	w.Lock()
   405  	defer w.Unlock()
   406  
   407  	w.startedRelayBySourceCfg = false
   408  	if !w.relayEnabled.CAS(true, false) {
   409  		w.l.Warn("already disabled relay")
   410  		return
   411  	}
   412  
   413  	w.relayCancel()
   414  	w.relayWg.Wait()
   415  
   416  	// refresh task checker know latest relayEnabled, to avoid accessing relayHolder
   417  	if w.cfg.Checker.CheckEnable {
   418  		w.l.Info("refresh task checker")
   419  		w.taskStatusChecker.Close()
   420  		w.taskStatusChecker.Start()
   421  		w.l.Info("finish refreshing task checker")
   422  	}
   423  
   424  	w.subTaskHolder.resetAllSubTasks(nil)
   425  
   426  	if w.relayHolder != nil {
   427  		r := w.relayHolder
   428  		w.relayHolder = nil
   429  		r.Close()
   430  	}
   431  	if w.relayPurger != nil {
   432  		r := w.relayPurger
   433  		w.relayPurger = nil
   434  		r.Close()
   435  	}
   436  	w.l.Info("relay disabled")
   437  }
   438  
   439  // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks.
   440  func (w *SourceWorker) EnableHandleSubtasks() error {
   441  	w.l.Info("enter EnableHandleSubtasks")
   442  	w.Lock()
   443  	defer w.Unlock()
   444  	if w.subTaskEnabled.Load() {
   445  		w.l.Warn("already enabled handling subtasks")
   446  		return nil
   447  	}
   448  	w.subTaskCtx, w.subTaskCancel = context.WithCancel(w.ctx)
   449  
   450  	// we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event
   451  	// because triggering these events is useless now
   452  	subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust()
   453  	if err != nil {
   454  		return err
   455  	}
   456  
   457  	w.l.Info("starting to handle mysql source", zap.String("sourceCfg", w.cfg.String()), zap.Any("subTasks", subTaskCfgM))
   458  
   459  	for _, subTaskCfg := range subTaskCfgM {
   460  		expectStage := subTaskStages[subTaskCfg.Name]
   461  		if expectStage.IsDeleted {
   462  			continue
   463  		}
   464  		validatorStage := pb.Stage_InvalidStage
   465  		if s, ok := validatorStages[subTaskCfg.Name]; ok {
   466  			validatorStage = s.Expect
   467  		}
   468  		w.l.Info("start to create subtask in EnableHandleSubtasks", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name))
   469  		// "for range" of a map will use same value address, so we'd better not pass value address to other function
   470  		clone := subTaskCfg
   471  		if err2 := w.StartSubTask(&clone, expectStage.Expect, validatorStage, false); err2 != nil {
   472  			w.subTaskHolder.closeAllSubTasks()
   473  			return err2
   474  		}
   475  	}
   476  
   477  	w.subTaskWg.Add(1)
   478  	go func() {
   479  		defer w.subTaskWg.Done()
   480  		// TODO: handle fatal error from observeSubtaskStage
   481  		//nolint:errcheck
   482  		w.observeSubtaskStage(w.subTaskCtx, w.etcdClient, revSubTask)
   483  	}()
   484  	w.subTaskWg.Add(1)
   485  	go func() {
   486  		defer w.subTaskWg.Done()
   487  		// TODO: handle fatal error from observeValidatorStage
   488  		//nolint:errcheck
   489  		w.observeValidatorStage(w.subTaskCtx, revSubTask)
   490  	}()
   491  
   492  	w.subTaskEnabled.Store(true)
   493  	w.l.Info("handling subtask enabled")
   494  	return nil
   495  }
   496  
   497  // DisableHandleSubtasks disables the functionality of start/watch/handle subtasks.
   498  func (w *SourceWorker) DisableHandleSubtasks() {
   499  	w.l.Info("enter DisableHandleSubtasks")
   500  	if !w.subTaskEnabled.CAS(true, false) {
   501  		w.l.Warn("already disabled handling subtasks")
   502  		return
   503  	}
   504  
   505  	w.subTaskCancel()
   506  	w.subTaskWg.Wait()
   507  
   508  	w.Lock()
   509  	defer w.Unlock()
   510  
   511  	// close all sub tasks
   512  	w.subTaskHolder.closeAllSubTasks()
   513  	w.l.Info("handling subtask disabled")
   514  }
   515  
   516  // fetchSubTasksAndAdjust gets source's subtask stages and configs, adjust some values by worker's config and status
   517  // source **must not be empty**
   518  // return map{task name -> subtask stage}, map{task name -> validator stage}, map{task name -> subtask config}, revision, error.
   519  func (w *SourceWorker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) {
   520  	// we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event
   521  	// because triggering these events is useless now
   522  	subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID)
   523  	if err != nil {
   524  		return nil, nil, nil, 0, err
   525  	}
   526  
   527  	if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg, w.relayEnabled.Load()); err != nil {
   528  		return nil, nil, nil, 0, err
   529  	}
   530  	return subTaskStages, validatorStages, subTaskCfgM, revSubTask, nil
   531  }
   532  
   533  // StartSubTask creates a subtask and run it.
   534  // TODO(ehco) rename this func.
   535  func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage, validatorStage pb.Stage, needLock bool) error {
   536  	if needLock {
   537  		w.Lock()
   538  		defer w.Unlock()
   539  	}
   540  
   541  	// copy some config item from dm-worker's source config
   542  	err := copyConfigFromSource(cfg, w.cfg, w.relayEnabled.Load())
   543  	if err != nil {
   544  		return err
   545  	}
   546  
   547  	// directly put cfg into subTaskHolder
   548  	// the uniqueness of subtask should be assured by etcd
   549  	st := NewSubTask(cfg, w.etcdClient, w.name)
   550  	w.subTaskHolder.recordSubTask(st)
   551  	if w.closed.Load() {
   552  		st.fail(terror.ErrWorkerAlreadyClosed.Generate())
   553  		return nil
   554  	}
   555  
   556  	cfg2, err := cfg.DecryptedClone()
   557  	if err != nil {
   558  		st.fail(errors.Annotate(err, "start sub task"))
   559  		return nil
   560  	}
   561  	st.cfg = cfg2
   562  	// inject worker name to this subtask config
   563  	st.cfg.WorkerName = w.name
   564  
   565  	if w.relayEnabled.Load() && w.relayPurger.Purging() {
   566  		// TODO: retry until purged finished
   567  		st.fail(terror.ErrWorkerRelayIsPurging.Generate(cfg.Name))
   568  		return nil
   569  	}
   570  
   571  	w.l.Info("subtask created", zap.Stringer("config", cfg2))
   572  	st.Run(expectStage, validatorStage, w.getRelayWithoutLock())
   573  	return nil
   574  }
   575  
   576  // caller should make sure w.Lock is locked before calling this method.
   577  func (w *SourceWorker) getRelayWithoutLock() relay.Process {
   578  	if w.relayHolder != nil {
   579  		return w.relayHolder.Relay()
   580  	}
   581  	return nil
   582  }
   583  
   584  // UpdateSubTask update config for a sub task.
   585  func (w *SourceWorker) UpdateSubTask(ctx context.Context, cfg *config.SubTaskConfig, needLock bool) error {
   586  	if needLock {
   587  		w.Lock()
   588  		defer w.Unlock()
   589  	}
   590  
   591  	if w.closed.Load() {
   592  		return terror.ErrWorkerAlreadyClosed.Generate()
   593  	}
   594  
   595  	st := w.subTaskHolder.findSubTask(cfg.Name)
   596  	if st == nil {
   597  		return terror.ErrWorkerSubTaskNotFound.Generate(cfg.Name)
   598  	}
   599  
   600  	w.l.Info("update sub task", zap.String("task", cfg.Name))
   601  	return st.Update(ctx, cfg)
   602  }
   603  
   604  // OperateSubTask stop/resume/pause sub task.
   605  func (w *SourceWorker) OperateSubTask(name string, op pb.TaskOp) error {
   606  	w.Lock()
   607  	defer w.Unlock()
   608  
   609  	if w.closed.Load() {
   610  		return terror.ErrWorkerAlreadyClosed.Generate()
   611  	}
   612  
   613  	st := w.subTaskHolder.findSubTask(name)
   614  	if st == nil {
   615  		return terror.ErrWorkerSubTaskNotFound.Generate(name)
   616  	}
   617  
   618  	w.l.Info("OperateSubTask start", zap.Stringer("op", op), zap.String("task", name))
   619  	var err error
   620  	switch op {
   621  	case pb.TaskOp_Delete:
   622  		w.l.Info("delete subtask", zap.String("task", name))
   623  		st.Close()
   624  		w.subTaskHolder.removeSubTask(name)
   625  	case pb.TaskOp_Pause, pb.TaskOp_Stop:
   626  		w.l.Info("pause subtask", zap.String("task", name))
   627  		err = st.Pause()
   628  	case pb.TaskOp_Resume:
   629  		failpoint.Inject("SkipRefreshFromETCDInUT", func(_ failpoint.Value) {
   630  			failpoint.Goto("bypassRefresh")
   631  		})
   632  		if refreshErr := w.tryRefreshSubTaskAndSourceConfig(st); refreshErr != nil {
   633  			// NOTE: for current unit is not syncer unit or is in shard merge.
   634  			w.l.Warn("can not update subtask config now", zap.Error(refreshErr))
   635  		}
   636  		failpoint.Label("bypassRefresh")
   637  		w.l.Info("resume subtask", zap.String("task", name))
   638  		err = st.Resume(w.getRelayWithoutLock())
   639  	case pb.TaskOp_AutoResume:
   640  		// TODO(ehco) change to auto_restart
   641  		w.l.Info("auto_resume subtask", zap.String("task", name))
   642  		err = st.Resume(w.getRelayWithoutLock())
   643  	default:
   644  		err = terror.ErrWorkerUpdateTaskStage.Generatef("invalid operate %s on subtask %v", op, name)
   645  	}
   646  	w.l.Info("OperateSubTask finished", zap.Stringer("op", op), zap.String("task", name))
   647  	return err
   648  }
   649  
   650  // QueryStatus query worker's sub tasks' status. If relay enabled, also return source status.
   651  func (w *SourceWorker) QueryStatus(ctx context.Context, name string) ([]*pb.SubTaskStatus, *pb.RelayStatus, error) {
   652  	w.RLock()
   653  	defer w.RUnlock()
   654  
   655  	if w.closed.Load() {
   656  		w.l.Warn("querying status from a closed worker")
   657  		return nil, nil, nil
   658  	}
   659  
   660  	var (
   661  		sourceStatus *binlog.SourceStatus
   662  		relayStatus  *pb.RelayStatus
   663  	)
   664  
   665  	if err := w.updateSourceStatus(ctx, false); err != nil {
   666  		if terror.ErrNoMasterStatus.Equal(err) {
   667  			w.l.Warn("This source's bin_log is OFF, so it only supports full_mode.", zap.String("sourceID", w.cfg.SourceID), zap.Error(err))
   668  		} else {
   669  			w.l.Error("failed to update source status", zap.Error(err))
   670  		}
   671  	} else {
   672  		sourceStatus = w.sourceStatus.Load().(*binlog.SourceStatus)
   673  	}
   674  
   675  	subtaskStatus := w.Status(name, sourceStatus)
   676  	if w.relayEnabled.Load() {
   677  		relayStatus = w.relayHolder.Status(sourceStatus)
   678  	}
   679  	return subtaskStatus, relayStatus, nil
   680  }
   681  
   682  func (w *SourceWorker) resetSubtaskStage() (int64, error) {
   683  	subTaskStages, _, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust()
   684  	if err != nil {
   685  		return 0, err
   686  	}
   687  	// use sts to check which subtask has no subtaskCfg or subtaskStage now
   688  	sts := w.subTaskHolder.getAllSubTasks()
   689  	for name, subtaskCfg := range subTaskCfgm {
   690  		stage, ok := subTaskStages[name]
   691  		if ok {
   692  			// TODO: right operation sequences may get error when we get etcdErrCompact, need to handle it later
   693  			// For example, Expect: Running -(pause)-> Paused -(resume)-> Running
   694  			// we get an etcd compact error at the first running. If we try to "resume" it now, we will get an error
   695  			opType, err2 := w.operateSubTaskStage(stage, subtaskCfg)
   696  			if err2 != nil {
   697  				opErrCounter.WithLabelValues(w.name, opType).Inc()
   698  				log.L().Error("fail to operate subtask stage", zap.Stringer("stage", stage),
   699  					zap.String("task", subtaskCfg.Name), zap.Error(err2))
   700  			}
   701  			delete(sts, name)
   702  		}
   703  	}
   704  	// remove subtasks without subtask config or subtask stage
   705  	for name := range sts {
   706  		err = w.OperateSubTask(name, pb.TaskOp_Delete)
   707  		if err != nil {
   708  			opErrCounter.WithLabelValues(w.name, pb.TaskOp_Delete.String()).Inc()
   709  			log.L().Error("fail to stop subtask", zap.String("task", name), zap.Error(err))
   710  		}
   711  	}
   712  	return revSubTask, nil
   713  }
   714  
   715  func (w *SourceWorker) observeSubtaskStage(ctx context.Context, etcdCli *clientv3.Client, rev int64) error {
   716  	var wg sync.WaitGroup
   717  
   718  	for {
   719  		subTaskStageCh := make(chan ha.Stage, 10)
   720  		subTaskErrCh := make(chan error, 10)
   721  		wg.Add(1)
   722  		// use ctx1, cancel1 to make sure old watcher has been released
   723  		ctx1, cancel1 := context.WithCancel(ctx)
   724  		go func() {
   725  			defer func() {
   726  				close(subTaskStageCh)
   727  				close(subTaskErrCh)
   728  				wg.Done()
   729  			}()
   730  			ha.WatchSubTaskStage(ctx1, etcdCli, w.cfg.SourceID, rev+1, subTaskStageCh, subTaskErrCh)
   731  		}()
   732  		err := w.handleSubTaskStage(ctx1, subTaskStageCh, subTaskErrCh)
   733  		cancel1()
   734  		wg.Wait()
   735  
   736  		if etcdutil.IsRetryableError(err) {
   737  			rev = 0
   738  			retryNum := 1
   739  			for rev == 0 {
   740  				select {
   741  				case <-ctx.Done():
   742  					return nil
   743  				case <-time.After(500 * time.Millisecond):
   744  					rev, err = w.resetSubtaskStage()
   745  					if err != nil {
   746  						log.L().Error("resetSubtaskStage is failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum))
   747  					}
   748  				}
   749  				retryNum++
   750  			}
   751  		} else {
   752  			if err != nil {
   753  				log.L().Error("observeSubtaskStage is failed and will quit now", zap.Error(err))
   754  			} else {
   755  				log.L().Info("observeSubtaskStage will quit now")
   756  			}
   757  			return err
   758  		}
   759  	}
   760  }
   761  
   762  func (w *SourceWorker) handleSubTaskStage(ctx context.Context, stageCh chan ha.Stage, errCh chan error) error {
   763  	closed := false
   764  	for {
   765  		select {
   766  		case <-ctx.Done():
   767  			closed = true
   768  		case stage, ok := <-stageCh:
   769  			if !ok {
   770  				closed = true
   771  				break
   772  			}
   773  			log.L().Info("receive subtask stage change", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted))
   774  			opType, err := w.operateSubTaskStageWithoutConfig(stage)
   775  			if err != nil {
   776  				opErrCounter.WithLabelValues(w.name, opType).Inc()
   777  				log.L().Error("fail to operate subtask stage", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err))
   778  				if etcdutil.IsRetryableError(err) {
   779  					return err
   780  				}
   781  			}
   782  		case err, ok := <-errCh:
   783  			if !ok {
   784  				closed = true
   785  				break
   786  			}
   787  			// TODO: deal with err
   788  			log.L().Error("WatchSubTaskStage received an error", zap.Error(err))
   789  			if etcdutil.IsRetryableError(err) {
   790  				return err
   791  			}
   792  		}
   793  		if closed {
   794  			log.L().Info("worker is closed, handleSubTaskStage will quit now")
   795  			return nil
   796  		}
   797  	}
   798  }
   799  
   800  // operateSubTaskStage returns TaskOp.String() additionally to record metrics.
   801  func (w *SourceWorker) operateSubTaskStage(stage ha.Stage, subTaskCfg config.SubTaskConfig) (string, error) {
   802  	var op pb.TaskOp
   803  	log.L().Info("operateSubTaskStage",
   804  		zap.String("sourceID", subTaskCfg.SourceID),
   805  		zap.String("task", subTaskCfg.Name),
   806  		zap.Stringer("stage", stage))
   807  
   808  	// for new added subtask
   809  	if st := w.subTaskHolder.findSubTask(stage.Task); st == nil {
   810  		switch stage.Expect {
   811  		case pb.Stage_Running, pb.Stage_Paused, pb.Stage_Stopped:
   812  			// todo refactor here deciding if the expected stage is valid should be put inside StartSubTask and OperateSubTask
   813  			log.L().Info("start to create subtask in operateSubTaskStage", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name))
   814  			expectValidatorStage, err := getExpectValidatorStage(subTaskCfg.ValidatorCfg, w.etcdClient, stage.Source, stage.Task, stage.Revision)
   815  			if err != nil {
   816  				return opErrTypeBeforeOp, terror.Annotate(err, "fail to get validator stage from etcd")
   817  			}
   818  			return opErrTypeBeforeOp, w.StartSubTask(&subTaskCfg, stage.Expect, expectValidatorStage, true)
   819  		default:
   820  			// not valid stage
   821  			return op.String(), w.OperateSubTask(stage.Task, op)
   822  		}
   823  	}
   824  	// todo(ehco) remove pause and resume after using openapi to impl dmctl
   825  	switch stage.Expect {
   826  	case pb.Stage_Stopped, pb.Stage_Paused:
   827  		op = pb.TaskOp_Pause
   828  	case pb.Stage_Running:
   829  		op = pb.TaskOp_Resume
   830  	}
   831  	if stage.IsDeleted {
   832  		op = pb.TaskOp_Delete
   833  	}
   834  	return op.String(), w.OperateSubTask(stage.Task, op)
   835  }
   836  
   837  // operateSubTaskStageWithoutConfig returns TaskOp additionally to record metrics.
   838  func (w *SourceWorker) operateSubTaskStageWithoutConfig(stage ha.Stage) (string, error) {
   839  	var subTaskCfg config.SubTaskConfig
   840  	if stage.Expect == pb.Stage_Running || stage.Expect == pb.Stage_Stopped {
   841  		if st := w.subTaskHolder.findSubTask(stage.Task); st == nil {
   842  			tsm, _, err := ha.GetSubTaskCfg(w.etcdClient, stage.Source, stage.Task, stage.Revision)
   843  			if err != nil {
   844  				// TODO: need retry
   845  				return opErrTypeBeforeOp, terror.Annotate(err, "fail to get subtask config from etcd")
   846  			}
   847  			var ok bool
   848  			if subTaskCfg, ok = tsm[stage.Task]; !ok {
   849  				return opErrTypeBeforeOp, terror.ErrWorkerFailToGetSubtaskConfigFromEtcd.Generate(stage.Task)
   850  			}
   851  		}
   852  	}
   853  	return w.operateSubTaskStage(stage, subTaskCfg)
   854  }
   855  
   856  func (w *SourceWorker) observeRelayStage(ctx context.Context, etcdCli *clientv3.Client, rev int64) error {
   857  	var wg sync.WaitGroup
   858  	for {
   859  		relayStageCh := make(chan ha.Stage, 10)
   860  		relayErrCh := make(chan error, 10)
   861  		wg.Add(1)
   862  		// use ctx1, cancel1 to make sure old watcher has been released
   863  		ctx1, cancel1 := context.WithCancel(ctx)
   864  		go func() {
   865  			defer func() {
   866  				close(relayStageCh)
   867  				close(relayErrCh)
   868  				wg.Done()
   869  			}()
   870  			ha.WatchRelayStage(ctx1, etcdCli, w.cfg.SourceID, rev+1, relayStageCh, relayErrCh)
   871  		}()
   872  		err := w.handleRelayStage(ctx1, relayStageCh, relayErrCh)
   873  		cancel1()
   874  		wg.Wait()
   875  
   876  		if etcdutil.IsRetryableError(err) {
   877  			rev = 0
   878  			retryNum := 1
   879  			for rev == 0 {
   880  				select {
   881  				case <-ctx.Done():
   882  					return nil
   883  				case <-time.After(500 * time.Millisecond):
   884  					stage, rev1, err1 := ha.GetRelayStage(etcdCli, w.cfg.SourceID)
   885  					if err1 != nil {
   886  						log.L().Error("get source bound from etcd failed, will retry later", zap.Error(err1), zap.Int("retryNum", retryNum))
   887  						break
   888  					}
   889  					rev = rev1
   890  					if stage.IsEmpty() {
   891  						stage.IsDeleted = true
   892  					}
   893  					opType, err1 := w.operateRelayStage(ctx, stage)
   894  					if err1 != nil {
   895  						opErrCounter.WithLabelValues(w.name, opType).Inc()
   896  						log.L().Error("fail to operate relay", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err1))
   897  					}
   898  				}
   899  				retryNum++
   900  			}
   901  		} else {
   902  			if err != nil {
   903  				log.L().Error("observeRelayStage is failed and will quit now", zap.Error(err))
   904  			} else {
   905  				log.L().Info("observeRelayStage will quit now")
   906  			}
   907  			return err
   908  		}
   909  	}
   910  }
   911  
   912  func (w *SourceWorker) handleRelayStage(ctx context.Context, stageCh chan ha.Stage, errCh chan error) error {
   913  OUTER:
   914  	for {
   915  		select {
   916  		case <-ctx.Done():
   917  			break OUTER
   918  		case stage, ok := <-stageCh:
   919  			if !ok {
   920  				break OUTER
   921  			}
   922  			log.L().Info("receive relay stage change", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted))
   923  			opType, err := w.operateRelayStage(ctx, stage)
   924  			if err != nil {
   925  				opErrCounter.WithLabelValues(w.name, opType).Inc()
   926  				log.L().Error("fail to operate relay", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err))
   927  			}
   928  		case err, ok := <-errCh:
   929  			if !ok {
   930  				break OUTER
   931  			}
   932  			log.L().Error("WatchRelayStage received an error", zap.Error(err))
   933  			if etcdutil.IsRetryableError(err) {
   934  				return err
   935  			}
   936  		}
   937  	}
   938  	log.L().Info("worker is closed, handleRelayStage will quit now")
   939  	return nil
   940  }
   941  
   942  // operateRelayStage returns RelayOp.String() additionally to record metrics
   943  // *RelayOp is nil only when error is nil, so record on error will not meet nil-pointer deference.
   944  func (w *SourceWorker) operateRelayStage(ctx context.Context, stage ha.Stage) (string, error) {
   945  	var op pb.RelayOp
   946  	switch {
   947  	case stage.Expect == pb.Stage_Running:
   948  		if w.relayHolder.Stage() == pb.Stage_New {
   949  			w.relayHolder.Start()
   950  			w.relayPurger.Start()
   951  			return opErrTypeBeforeOp, nil
   952  		}
   953  		op = pb.RelayOp_ResumeRelay
   954  	case stage.Expect == pb.Stage_Paused:
   955  		op = pb.RelayOp_PauseRelay
   956  	case stage.IsDeleted:
   957  		op = pb.RelayOp_StopRelay
   958  	}
   959  	return op.String(), w.operateRelay(ctx, op)
   960  }
   961  
   962  // OperateRelay operates relay unit.
   963  func (w *SourceWorker) operateRelay(ctx context.Context, op pb.RelayOp) error {
   964  	if w.closed.Load() {
   965  		return terror.ErrWorkerAlreadyClosed.Generate()
   966  	}
   967  
   968  	if w.relayEnabled.Load() {
   969  		// TODO: lock the worker?
   970  		return w.relayHolder.Operate(ctx, op)
   971  	}
   972  
   973  	w.l.Warn("enable-relay is false, ignore operate relay")
   974  	return nil
   975  }
   976  
   977  // PurgeRelay purges relay log files.
   978  func (w *SourceWorker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) error {
   979  	if w.closed.Load() {
   980  		return terror.ErrWorkerAlreadyClosed.Generate()
   981  	}
   982  
   983  	if !w.relayEnabled.Load() {
   984  		w.l.Warn("enable-relay is false, ignore purge relay")
   985  		return nil
   986  	}
   987  
   988  	if !w.subTaskEnabled.Load() {
   989  		w.l.Info("worker received purge-relay but didn't handling subtasks, read global checkpoint to decided active relay log")
   990  
   991  		subDir := w.relayHolder.Status(nil).RelaySubDir
   992  
   993  		_, _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust()
   994  		if err != nil {
   995  			return err
   996  		}
   997  		for _, subTaskCfg := range subTaskCfgs {
   998  			loc, err2 := getMinLocForSubTaskFunc(ctx, subTaskCfg)
   999  			if err2 != nil {
  1000  				return err2
  1001  			}
  1002  			w.l.Info("update active relay log with",
  1003  				zap.String("task name", subTaskCfg.Name),
  1004  				zap.String("subDir", subDir),
  1005  				zap.String("binlog name", loc.Position.Name))
  1006  			if err3 := streamer.GetReaderHub().UpdateActiveRelayLog(subTaskCfg.Name, subDir, loc.Position.Name); err3 != nil {
  1007  				w.l.Error("Error when update active relay log", zap.Error(err3))
  1008  			}
  1009  		}
  1010  	}
  1011  	return w.relayPurger.Do(ctx, req)
  1012  }
  1013  
  1014  // ForbidPurge implements PurgeInterceptor.ForbidPurge.
  1015  func (w *SourceWorker) ForbidPurge() (bool, string) {
  1016  	if w.closed.Load() {
  1017  		return false, ""
  1018  	}
  1019  
  1020  	// forbid purging if some sub tasks are paused, so we can debug the system easily
  1021  	// This function is not protected by `w.RWMutex`, which may lead to sub tasks information
  1022  	// not up to date, but do not affect correctness.
  1023  	for _, st := range w.subTaskHolder.getAllSubTasks() {
  1024  		stage := st.Stage()
  1025  		if stage == pb.Stage_New || stage == pb.Stage_Paused {
  1026  			return true, fmt.Sprintf("sub task %s current stage is %s", st.cfg.Name, stage.String())
  1027  		}
  1028  	}
  1029  	return false, ""
  1030  }
  1031  
  1032  // OperateSchema operates schema for an upstream table.
  1033  func (w *SourceWorker) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaRequest) (schema string, err error) {
  1034  	w.Lock()
  1035  	defer w.Unlock()
  1036  
  1037  	if w.closed.Load() {
  1038  		return "", terror.ErrWorkerAlreadyClosed.Generate()
  1039  	}
  1040  
  1041  	st := w.subTaskHolder.findSubTask(req.Task)
  1042  	if st == nil {
  1043  		return "", terror.ErrWorkerSubTaskNotFound.Generate(req.Task)
  1044  	}
  1045  
  1046  	return st.OperateSchema(ctx, req)
  1047  }
  1048  
  1049  // copyConfigFromSource copies config items from source config and worker's relayEnabled to sub task.
  1050  func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceConfig, enableRelay bool) error {
  1051  	cfg.From = sourceCfg.From
  1052  
  1053  	cfg.Flavor = sourceCfg.Flavor
  1054  	cfg.ServerID = sourceCfg.ServerID
  1055  	cfg.RelayDir = sourceCfg.RelayDir
  1056  	cfg.EnableGTID = sourceCfg.EnableGTID
  1057  	cfg.UseRelay = enableRelay
  1058  
  1059  	if cfg.CaseSensitive != sourceCfg.CaseSensitive {
  1060  		log.L().Warn("different case-sensitive config between task config and source config, use `true` for it.")
  1061  	}
  1062  	cfg.CaseSensitive = cfg.CaseSensitive || sourceCfg.CaseSensitive
  1063  	filter, err := bf.NewBinlogEvent(cfg.CaseSensitive, cfg.FilterRules)
  1064  	if err != nil {
  1065  		return err
  1066  	}
  1067  
  1068  	for _, filterRule := range sourceCfg.Filters {
  1069  		if err = filter.AddRule(filterRule); err != nil {
  1070  			// task level config has higher priority
  1071  			if errors.IsAlreadyExists(errors.Cause(err)) {
  1072  				log.L().Warn("filter config already exist in source config, overwrite it", log.ShortError(err))
  1073  				continue
  1074  			}
  1075  			return err
  1076  		}
  1077  		cfg.FilterRules = append(cfg.FilterRules, filterRule)
  1078  	}
  1079  	return nil
  1080  }
  1081  
  1082  // copyConfigFromSourceForEach do copyConfigFromSource for each value in subTaskCfgM and change subTaskCfgM in-place.
  1083  func copyConfigFromSourceForEach(
  1084  	subTaskCfgM map[string]config.SubTaskConfig,
  1085  	sourceCfg *config.SourceConfig,
  1086  	enableRelay bool,
  1087  ) error {
  1088  	for k, subTaskCfg := range subTaskCfgM {
  1089  		if err2 := copyConfigFromSource(&subTaskCfg, sourceCfg, enableRelay); err2 != nil {
  1090  			return err2
  1091  		}
  1092  		subTaskCfgM[k] = subTaskCfg
  1093  	}
  1094  	return nil
  1095  }
  1096  
  1097  // getAllSubTaskStatus returns all subtask status of this worker, note the field
  1098  // in subtask status is not completed, only includes `Name`, `Stage` and `Result` now.
  1099  func (w *SourceWorker) getAllSubTaskStatus() map[string]*pb.SubTaskStatus {
  1100  	sts := w.subTaskHolder.getAllSubTasks()
  1101  	result := make(map[string]*pb.SubTaskStatus, len(sts))
  1102  	for name, st := range sts {
  1103  		st.RLock()
  1104  		result[name] = &pb.SubTaskStatus{
  1105  			Name:   name,
  1106  			Stage:  st.stage,
  1107  			Result: proto.Clone(st.result).(*pb.ProcessResult),
  1108  		}
  1109  		st.RUnlock()
  1110  	}
  1111  	return result
  1112  }
  1113  
  1114  // HandleError handle worker error.
  1115  func (w *SourceWorker) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) {
  1116  	w.Lock()
  1117  	defer w.Unlock()
  1118  
  1119  	if w.closed.Load() {
  1120  		return "", terror.ErrWorkerAlreadyClosed.Generate()
  1121  	}
  1122  
  1123  	st := w.subTaskHolder.findSubTask(req.Task)
  1124  	if st == nil {
  1125  		return "", terror.ErrWorkerSubTaskNotFound.Generate(req.Task)
  1126  	}
  1127  
  1128  	return st.HandleError(ctx, req, w.getRelayWithoutLock())
  1129  }
  1130  
  1131  func (w *SourceWorker) observeValidatorStage(ctx context.Context, lastUsedRev int64) error {
  1132  	var wg sync.WaitGroup
  1133  
  1134  	startRevision := lastUsedRev + 1
  1135  	for {
  1136  		stageCh := make(chan ha.Stage, 10)
  1137  		errCh := make(chan error, 10)
  1138  		wg.Add(1)
  1139  		watchCtx, watchCancel := context.WithCancel(ctx)
  1140  		go func() {
  1141  			defer func() {
  1142  				close(stageCh)
  1143  				close(errCh)
  1144  				wg.Done()
  1145  			}()
  1146  			ha.WatchValidatorStage(watchCtx, w.etcdClient, w.cfg.SourceID, startRevision, stageCh, errCh)
  1147  		}()
  1148  		err := w.handleValidatorStage(watchCtx, stageCh, errCh)
  1149  		watchCancel()
  1150  		wg.Wait()
  1151  
  1152  		if etcdutil.IsRetryableError(err) {
  1153  			startRevision = 0
  1154  			retryNum := 1
  1155  			for startRevision == 0 {
  1156  				select {
  1157  				case <-ctx.Done():
  1158  					return nil
  1159  				case <-time.After(500 * time.Millisecond):
  1160  					w.RLock()
  1161  					sourceID := w.cfg.SourceID
  1162  					w.RUnlock()
  1163  					startRevision, err = w.getCurrentValidatorRevision(sourceID)
  1164  					if err != nil {
  1165  						log.L().Error("reset validator stage failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum))
  1166  					}
  1167  				}
  1168  				retryNum++
  1169  			}
  1170  		} else {
  1171  			if err != nil {
  1172  				log.L().Error("observe validator stage failed, quit now", zap.Error(err))
  1173  			} else {
  1174  				log.L().Info("observe validator stage will quit now")
  1175  			}
  1176  			return err
  1177  		}
  1178  	}
  1179  }
  1180  
  1181  func (w *SourceWorker) handleValidatorStage(ctx context.Context, stageCh chan ha.Stage, errCh chan error) error {
  1182  	closed := false
  1183  	for {
  1184  		select {
  1185  		case <-ctx.Done():
  1186  			closed = true
  1187  		case stage, ok := <-stageCh:
  1188  			if !ok {
  1189  				closed = true
  1190  				break
  1191  			}
  1192  			log.L().Info("receive validator stage change", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted))
  1193  			err := w.operateValidatorStage(stage)
  1194  			if err != nil {
  1195  				opType := w.getValidatorOp(stage)
  1196  				opErrCounter.WithLabelValues(w.name, opType).Inc()
  1197  				log.L().Error("fail to operate validator stage", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err))
  1198  				if etcdutil.IsRetryableError(err) {
  1199  					return err
  1200  				}
  1201  			}
  1202  		case err, ok := <-errCh:
  1203  			if !ok {
  1204  				closed = true
  1205  				break
  1206  			}
  1207  			// TODO: deal with err
  1208  			log.L().Error("watch validator stage received an error", zap.Error(err))
  1209  			if etcdutil.IsRetryableError(err) {
  1210  				return err
  1211  			}
  1212  		}
  1213  		if closed {
  1214  			log.L().Info("worker is closed, handle validator stage will quit now")
  1215  			return nil
  1216  		}
  1217  	}
  1218  }
  1219  
  1220  func (w *SourceWorker) getCurrentValidatorRevision(source string) (int64, error) {
  1221  	_, rev, err := ha.GetValidatorStage(w.etcdClient, source, "", 0)
  1222  	if err != nil {
  1223  		return 0, err
  1224  	}
  1225  	return rev, nil
  1226  }
  1227  
  1228  func (w *SourceWorker) getValidatorOp(stage ha.Stage) string {
  1229  	if stage.IsDeleted {
  1230  		return "validator-delete"
  1231  	}
  1232  	if stage.Expect == pb.Stage_Running {
  1233  		return pb.ValidatorOp_StartValidator.String()
  1234  	} else if stage.Expect == pb.Stage_Stopped {
  1235  		return pb.ValidatorOp_StopValidator.String()
  1236  	}
  1237  	// should not happen
  1238  	return ""
  1239  }
  1240  
  1241  func (w *SourceWorker) operateValidatorStage(stage ha.Stage) error {
  1242  	// if the key it's deleted, the subtask is deleted too, let subtask clean it up.
  1243  	if stage.IsDeleted {
  1244  		return nil
  1245  	}
  1246  
  1247  	subtask := w.subTaskHolder.findSubTask(stage.Task)
  1248  	if subtask == nil {
  1249  		// when a new subtask start with validator, both subtask and validator stage observer will observe it,
  1250  		// if validator observe it first, we may not have the subtask.
  1251  		log.L().Info("cannot find subtask. maybe it's a new task, let subtask stage observer handles it")
  1252  		return nil
  1253  	}
  1254  
  1255  	// stage of validator can only be Running or Stopped
  1256  	switch stage.Expect {
  1257  	case pb.Stage_Stopped:
  1258  		subtask.StopValidator()
  1259  	case pb.Stage_Running:
  1260  		// validator's config is stored with subtask config, we need to update subtask config as validator may start
  1261  		// on the fly.
  1262  		subTaskCfg, _, err := ha.GetSubTaskCfg(w.etcdClient, stage.Source, stage.Task, stage.Revision)
  1263  		if err != nil {
  1264  			return err
  1265  		}
  1266  		targetCfg, ok := subTaskCfg[stage.Task]
  1267  		if !ok {
  1268  			log.L().Error("failed to get subtask config", zap.Reflect("stage", stage))
  1269  			return errors.New("failed to get subtask config")
  1270  		}
  1271  		subtask.UpdateValidatorCfg(targetCfg.ValidatorCfg)
  1272  		subtask.StartValidator(stage.Expect, false)
  1273  	default:
  1274  		// should not happen
  1275  		log.L().Warn("invalid validator stage", zap.Reflect("stage", stage))
  1276  	}
  1277  	return nil
  1278  }
  1279  
  1280  func (w *SourceWorker) refreshSourceCfg() error {
  1281  	oldCfg := w.cfg
  1282  	sourceCfgM, _, err := ha.GetSourceCfg(w.etcdClient, oldCfg.SourceID, 0)
  1283  	if err != nil {
  1284  		return err
  1285  	}
  1286  	w.cfg = sourceCfgM[oldCfg.SourceID]
  1287  	return nil
  1288  }
  1289  
  1290  func (w *SourceWorker) tryRefreshSubTaskAndSourceConfig(subTask *SubTask) error {
  1291  	// try refresh source config first
  1292  	if err := w.refreshSourceCfg(); err != nil {
  1293  		return err
  1294  	}
  1295  	sourceName := subTask.cfg.SourceID
  1296  	taskName := subTask.cfg.Name
  1297  	tsm, _, err := ha.GetSubTaskCfg(w.etcdClient, sourceName, taskName, 0)
  1298  	if err != nil {
  1299  		return terror.Annotate(err, "fail to get subtask config from etcd")
  1300  	}
  1301  
  1302  	var cfg config.SubTaskConfig
  1303  	var ok bool
  1304  	if cfg, ok = tsm[taskName]; !ok {
  1305  		return terror.ErrWorkerFailToGetSubtaskConfigFromEtcd.Generate(taskName)
  1306  	}
  1307  
  1308  	// copy some config item from dm-worker's source config
  1309  	if err := copyConfigFromSource(&cfg, w.cfg, w.relayEnabled.Load()); err != nil {
  1310  		return err
  1311  	}
  1312  	if checkErr := subTask.CheckUnitCfgCanUpdate(&cfg); checkErr != nil {
  1313  		return checkErr
  1314  	}
  1315  	return w.UpdateSubTask(w.ctx, &cfg, false)
  1316  }
  1317  
  1318  // CheckCfgCanUpdated check if current subtask config can be updated.
  1319  func (w *SourceWorker) CheckCfgCanUpdated(cfg *config.SubTaskConfig) error {
  1320  	w.RLock()
  1321  	defer w.RUnlock()
  1322  
  1323  	subTask := w.subTaskHolder.findSubTask(cfg.Name)
  1324  	if subTask == nil {
  1325  		return terror.ErrWorkerSubTaskNotFound.Generate(cfg.Name)
  1326  	}
  1327  	// copy some config item from dm-worker's source config
  1328  	if err := copyConfigFromSource(cfg, w.cfg, w.relayEnabled.Load()); err != nil {
  1329  		return err
  1330  	}
  1331  	return subTask.CheckUnitCfgCanUpdate(cfg)
  1332  }
  1333  
  1334  func (w *SourceWorker) GetWorkerValidatorErr(taskName string, errState pb.ValidateErrorState) ([]*pb.ValidationError, error) {
  1335  	st := w.subTaskHolder.findSubTask(taskName)
  1336  	if st != nil {
  1337  		return st.GetValidatorError(errState)
  1338  	}
  1339  	return nil, terror.ErrWorkerSubTaskNotFound.Generate(taskName)
  1340  }
  1341  
  1342  func (w *SourceWorker) OperateWorkerValidatorErr(taskName string, op pb.ValidationErrOp, errID uint64, isAll bool) error {
  1343  	st := w.subTaskHolder.findSubTask(taskName)
  1344  	if st != nil {
  1345  		return st.OperateValidatorError(op, errID, isAll)
  1346  	}
  1347  	return terror.ErrWorkerSubTaskNotFound.Generate(taskName)
  1348  }
  1349  
  1350  func (w *SourceWorker) GetValidatorStatus(taskName string) (*pb.ValidationStatus, error) {
  1351  	st := w.subTaskHolder.findSubTask(taskName)
  1352  	if st == nil {
  1353  		return nil, terror.ErrWorkerSubTaskNotFound.Generate(taskName)
  1354  	}
  1355  	return st.GetValidatorStatus()
  1356  }
  1357  
  1358  func (w *SourceWorker) GetValidatorTableStatus(taskName string, filterStatus pb.Stage) ([]*pb.ValidationTableStatus, error) {
  1359  	st := w.subTaskHolder.findSubTask(taskName)
  1360  	if st == nil {
  1361  		return nil, terror.ErrWorkerSubTaskNotFound.Generate(taskName)
  1362  	}
  1363  	return st.GetValidatorTableStatus(filterStatus)
  1364  }
  1365  
  1366  func (w *SourceWorker) UpdateWorkerValidator(req *pb.UpdateValidationWorkerRequest) error {
  1367  	st := w.subTaskHolder.findSubTask(req.TaskName)
  1368  	if st == nil {
  1369  		return terror.ErrWorkerSubTaskNotFound.Generate(req.TaskName)
  1370  	}
  1371  	return st.UpdateValidator(req)
  1372  }