github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/master/scheduler/scheduler_test.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 scheduler
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"sync"
    20  	"testing"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/failpoint"
    25  	"github.com/pingcap/tiflow/dm/config"
    26  	"github.com/pingcap/tiflow/dm/config/dbconfig"
    27  	"github.com/pingcap/tiflow/dm/config/security"
    28  	"github.com/pingcap/tiflow/dm/master/workerrpc"
    29  	"github.com/pingcap/tiflow/dm/pb"
    30  	"github.com/pingcap/tiflow/dm/pkg/ha"
    31  	"github.com/pingcap/tiflow/dm/pkg/log"
    32  	"github.com/pingcap/tiflow/dm/pkg/terror"
    33  	"github.com/stretchr/testify/require"
    34  	"github.com/stretchr/testify/suite"
    35  	v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
    36  	clientv3 "go.etcd.io/etcd/client/v3"
    37  	"go.etcd.io/etcd/tests/v3/integration"
    38  )
    39  
    40  const (
    41  	noRestart          = iota // do nothing in rebuildPessimist, just keep testing
    42  	restartOnly               // restart without building new instance. mock leader role transfer
    43  	restartNewInstance        // restart with build a new instance. mock progress restore from failure
    44  )
    45  
    46  var (
    47  	etcdErrCompacted = v3rpc.ErrCompacted
    48  	testRelayDir     = "./test_relay_dir"
    49  )
    50  
    51  func TestSchedulerSuite(t *testing.T) {
    52  	suite.Run(t, new(testSchedulerSuite))
    53  }
    54  
    55  // clear keys in etcd test cluster.
    56  func (t *testSchedulerSuite) clearTestInfoOperation() {
    57  	t.T().Helper()
    58  	require.NoError(t.T(), ha.ClearTestInfoOperation(t.etcdTestCli))
    59  }
    60  
    61  type testSchedulerSuite struct {
    62  	suite.Suite
    63  	mockCluster *integration.ClusterV3
    64  	etcdTestCli *clientv3.Client
    65  }
    66  
    67  func (t *testSchedulerSuite) SetupSuite() {
    68  	require.NoError(t.T(), log.InitLogger(&log.Config{}))
    69  
    70  	integration.BeforeTestExternal(t.T())
    71  	t.mockCluster = integration.NewClusterV3(t.T(), &integration.ClusterConfig{Size: 1})
    72  	t.etcdTestCli = t.mockCluster.RandClient()
    73  }
    74  
    75  func (t *testSchedulerSuite) TearDownSuite() {
    76  	t.mockCluster.Terminate(t.T())
    77  }
    78  
    79  func (t *testSchedulerSuite) TearDownTest() {
    80  	t.clearTestInfoOperation()
    81  }
    82  
    83  var stageEmpty ha.Stage
    84  
    85  func (t *testSchedulerSuite) TestScheduler() {
    86  	t.testSchedulerProgress(noRestart)
    87  	t.testSchedulerProgress(restartOnly)
    88  	t.testSchedulerProgress(restartNewInstance)
    89  }
    90  
    91  func (t *testSchedulerSuite) testSchedulerProgress(restart int) {
    92  	defer t.clearTestInfoOperation()
    93  
    94  	var (
    95  		logger       = log.L()
    96  		s            = NewScheduler(&logger, security.Security{})
    97  		sourceID1    = "mysql-replica-1"
    98  		sourceID2    = "mysql-replica-2"
    99  		workerName1  = "dm-worker-1"
   100  		workerName2  = "dm-worker-2"
   101  		workerAddr1  = "127.0.0.1:8262"
   102  		workerAddr2  = "127.0.0.1:18262"
   103  		taskName1    = "task-1"
   104  		taskName2    = "task-2"
   105  		workerInfo1  = ha.NewWorkerInfo(workerName1, workerAddr1)
   106  		workerInfo2  = ha.NewWorkerInfo(workerName2, workerAddr2)
   107  		subtaskCfg1  config.SubTaskConfig
   108  		keepAliveTTL = int64(5) // NOTE: this should be >= minLeaseTTL, in second.
   109  
   110  		rebuildScheduler = func(ctx context.Context) {
   111  			switch restart {
   112  			case restartOnly:
   113  				s.Close()
   114  				require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   115  			case restartNewInstance:
   116  				s.Close()
   117  				s = NewScheduler(&logger, security.Security{})
   118  				require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   119  			}
   120  		}
   121  	)
   122  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
   123  	require.NoError(t.T(), err)
   124  	sourceCfg1.SourceID = sourceID1
   125  	sourceCfg2 := *sourceCfg1
   126  	sourceCfg2.SourceID = sourceID2
   127  
   128  	require.NoError(t.T(), subtaskCfg1.Decode(config.SampleSubtaskConfig, true))
   129  	subtaskCfg1.SourceID = sourceID1
   130  	subtaskCfg1.Name = taskName1
   131  	subtaskCfg1.LoaderConfig.ImportMode = config.LoadModePhysical
   132  	require.NoError(t.T(), subtaskCfg1.Adjust(true))
   133  	subtaskCfg21 := subtaskCfg1
   134  	subtaskCfg21.Name = taskName2
   135  	require.NoError(t.T(), subtaskCfg21.Adjust(true))
   136  	subtaskCfg22 := subtaskCfg21
   137  	subtaskCfg22.SourceID = sourceID2
   138  	subtaskCfg22.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationFast}
   139  	require.NoError(t.T(), subtaskCfg22.Adjust(true))
   140  
   141  	// not started scheduler can't do anything.
   142  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfg(sourceCfg1)))
   143  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfgWithWorker(sourceCfg1, workerName1)))
   144  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.UpdateSourceCfg(sourceCfg1)))
   145  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.RemoveSourceCfg(sourceID1)))
   146  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.AddSubTasks(false, pb.Stage_Running, subtaskCfg1)))
   147  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.RemoveSubTasks(taskName1, sourceID1)))
   148  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.AddWorker(workerName1, workerAddr1)))
   149  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.RemoveWorker(workerName1)))
   150  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.UpdateExpectRelayStage(pb.Stage_Running, sourceID1)))
   151  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.UpdateExpectSubTaskStage(pb.Stage_Running, taskName1, sourceID1)))
   152  	require.True(t.T(), terror.ErrSchedulerNotStarted.Equal(s.OperateValidationTask(nil, nil)))
   153  
   154  	ctx, cancel := context.WithCancel(context.Background())
   155  	defer cancel()
   156  
   157  	// CASE 1: start without any previous info.
   158  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   159  	require.True(t.T(), terror.ErrSchedulerStarted.Equal(s.Start(ctx, t.etcdTestCli))) // start multiple times.
   160  	s.Close()
   161  	s.Close() // close multiple times.
   162  
   163  	// CASE 2: start again without any previous info.
   164  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   165  
   166  	// CASE 2.1: add the first source config.
   167  	// no source config exist before added.
   168  	t.sourceCfgNotExist(s, sourceID1)
   169  	// add source config1.
   170  	require.NoError(t.T(), s.AddSourceCfg(sourceCfg1))
   171  	require.True(t.T(), terror.ErrSchedulerSourceCfgExist.Equal(s.AddSourceCfg(sourceCfg1))) // can't add multiple times.
   172  	// the source config added.
   173  	t.sourceCfgExist(s, sourceCfg1)
   174  
   175  	// update source cfg
   176  	sourceCfg1.RelayDir = testRelayDir
   177  	require.NoError(t.T(), s.UpdateSourceCfg(sourceCfg1))
   178  	newCfg := s.GetSourceCfgByID(sourceID1)
   179  	require.Equal(t.T(), testRelayDir, newCfg.RelayDir)
   180  
   181  	// update with invalid SourceID
   182  	fake := newCfg.Clone()
   183  	fake.SourceID = "not a source id"
   184  	require.True(t.T(), terror.ErrSchedulerSourceCfgNotExist.Equal(s.UpdateSourceCfg(fake)))
   185  
   186  	// one unbound source exist (because no free worker).
   187  	t.sourceBounds(s, []string{}, []string{sourceID1})
   188  	rebuildScheduler(ctx)
   189  
   190  	// CASE 2.2: add the first worker.
   191  	// no worker exist before added.
   192  	t.workerNotExist(s, workerName1)
   193  	// add worker1.
   194  	require.NoError(t.T(), s.AddWorker(workerName1, workerAddr1))
   195  	require.True(t.T(), terror.ErrSchedulerWorkerExist.Equal(s.AddWorker(workerName1, workerAddr2))) // can't add with different address now.
   196  	require.NoError(t.T(), s.AddWorker(workerName1, workerAddr1))                                    // but can add the worker multiple times (like restart the worker).
   197  	// the worker added.
   198  	t.workerExist(s, workerInfo1)
   199  	t.workerOffline(s, workerName1)
   200  	// still no bounds (because the worker is offline).
   201  	t.sourceBounds(s, []string{}, []string{sourceID1})
   202  	// no expect relay stage exist (because the source has never been bound).
   203  	t.relayStageMatch(s, sourceID1, pb.Stage_InvalidStage)
   204  	rebuildScheduler(ctx)
   205  
   206  	// CASE 2.3: the worker become online.
   207  	// do keep-alive for worker1.
   208  	ctx1, cancel1 := context.WithCancel(ctx)
   209  	var wg sync.WaitGroup
   210  	wg.Add(1)
   211  	go func() {
   212  		defer wg.Done()
   213  		require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL))
   214  	}()
   215  	// wait for source1 being bound to worker1.
   216  	require.Eventually(t.T(), func() bool {
   217  		bounds := s.BoundSources()
   218  		return len(bounds) == 1 && bounds[0] == sourceID1
   219  	}, 3*time.Second, 100*time.Millisecond)
   220  	t.sourceBounds(s, []string{sourceID1}, []string{})
   221  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName1))
   222  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName1}))
   223  	// expect relay stage become Running after the start relay.
   224  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   225  	rebuildScheduler(ctx)
   226  
   227  	// CASE 2.4: pause the relay.
   228  	require.NoError(t.T(), s.UpdateExpectRelayStage(pb.Stage_Paused, sourceID1))
   229  	t.relayStageMatch(s, sourceID1, pb.Stage_Paused)
   230  	// update relay stage without source take no effect now (and return without error).
   231  	require.NoError(t.T(), s.UpdateExpectRelayStage(pb.Stage_Running))
   232  	t.relayStageMatch(s, sourceID1, pb.Stage_Paused)
   233  	// update to non-(Running, Paused) stage is invalid.
   234  	require.True(t.T(), terror.ErrSchedulerRelayStageInvalidUpdate.Equal(s.UpdateExpectRelayStage(pb.Stage_InvalidStage, sourceID1)))
   235  	require.True(t.T(), terror.ErrSchedulerRelayStageInvalidUpdate.Equal(s.UpdateExpectRelayStage(pb.Stage_New, sourceID1)))
   236  	require.True(t.T(), terror.ErrSchedulerRelayStageInvalidUpdate.Equal(s.UpdateExpectRelayStage(pb.Stage_Stopped, sourceID1)))
   237  	require.True(t.T(), terror.ErrSchedulerRelayStageInvalidUpdate.Equal(s.UpdateExpectRelayStage(pb.Stage_Finished, sourceID1)))
   238  	// can't update stage with not existing sources now.
   239  	require.True(t.T(), terror.ErrSchedulerRelayStageSourceNotExist.Equal(s.UpdateExpectRelayStage(pb.Stage_Running, sourceID1, sourceID2)))
   240  	t.relayStageMatch(s, sourceID1, pb.Stage_Paused)
   241  	rebuildScheduler(ctx)
   242  
   243  	// CASE 2.5: resume the relay.
   244  	require.NoError(t.T(), s.UpdateExpectRelayStage(pb.Stage_Running, sourceID1))
   245  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   246  	rebuildScheduler(ctx)
   247  
   248  	// CASE 2.6: start a task with only one source.
   249  	// wait for source bound recovered
   250  	require.Eventually(t.T(), func() bool {
   251  		bounds := s.BoundSources()
   252  		return len(bounds) == 1 && bounds[0] == sourceID1
   253  	}, 3*time.Second, 100*time.Millisecond)
   254  	// no subtask config exists before start.
   255  	require.NoError(t.T(), s.AddSubTasks(false, pb.Stage_Running)) // can call without configs, return without error, but take no effect.
   256  	t.subTaskCfgNotExist(s, taskName1, sourceID1)
   257  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_InvalidStage)
   258  	t.downstreamMetaNotExist(s, taskName1)
   259  	// start the task.
   260  	require.NoError(t.T(), s.AddSubTasks(false, pb.Stage_Running, subtaskCfg1))
   261  	require.True(t.T(), terror.ErrSchedulerSubTaskExist.Equal(s.AddSubTasks(false, pb.Stage_Running, subtaskCfg1))) // add again.
   262  	// subtask config and stage exist.
   263  	t.subTaskCfgExist(s, subtaskCfg1)
   264  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   265  	t.downstreamMetaExist(s, taskName1, subtaskCfg1.To, subtaskCfg1.MetaSchema)
   266  	t.downstreamMetaNotExist(s, taskName2)
   267  	// check lightning status is written to etcd
   268  	status, err := ha.GetAllLightningStatus(t.etcdTestCli, taskName1)
   269  	require.NoError(t.T(), err)
   270  	require.Equal(t.T(), []string{ha.LightningNotReady}, status)
   271  
   272  	// try start a task with two sources, some sources not bound.
   273  	require.True(t.T(), terror.ErrSchedulerSourcesUnbound.Equal(s.AddSubTasks(false, pb.Stage_Running, subtaskCfg21, subtaskCfg22)))
   274  	t.subTaskCfgNotExist(s, taskName2, sourceID1)
   275  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_InvalidStage)
   276  	t.subTaskCfgNotExist(s, taskName2, sourceID2)
   277  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_InvalidStage)
   278  	rebuildScheduler(ctx)
   279  
   280  	// CASE 2.7: pause/resume task1.
   281  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Paused, taskName1, sourceID1))
   282  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Paused)
   283  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Running, taskName1, sourceID1))
   284  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   285  	// update subtask stage without source or task take no effect now (and return without error).
   286  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Paused, "", sourceID1))
   287  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Paused, taskName1))
   288  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   289  	// update to non-(New, Finished) stage is invalid.
   290  	require.True(t.T(), terror.ErrSchedulerSubTaskStageInvalidUpdate.Equal(s.UpdateExpectSubTaskStage(pb.Stage_InvalidStage, taskName1, sourceID1)))
   291  	require.True(t.T(), terror.ErrSchedulerSubTaskStageInvalidUpdate.Equal(s.UpdateExpectSubTaskStage(pb.Stage_New, taskName1, sourceID1)))
   292  	require.True(t.T(), terror.ErrSchedulerSubTaskStageInvalidUpdate.Equal(s.UpdateExpectSubTaskStage(pb.Stage_Finished, taskName1, sourceID1)))
   293  	// can't update stage with not existing sources now.
   294  	require.True(t.T(), terror.ErrSchedulerSubTaskOpSourceNotExist.Equal(s.UpdateExpectSubTaskStage(pb.Stage_Paused, taskName1, sourceID1, sourceID2)))
   295  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   296  	rebuildScheduler(ctx)
   297  
   298  	// CASE 2.8: worker1 become offline.
   299  	// cancel keep-alive.
   300  	cancel1()
   301  	wg.Wait()
   302  	// wait for source1 unbound from worker1.
   303  	require.Eventually(t.T(), func() bool {
   304  		unbounds := s.UnboundSources()
   305  		return len(unbounds) == 1 && unbounds[0] == sourceID1
   306  	}, 3*time.Duration(keepAliveTTL)*time.Second, time.Second)
   307  	t.sourceBounds(s, []string{}, []string{sourceID1})
   308  	// static information are still there.
   309  	t.sourceCfgExist(s, sourceCfg1)
   310  	t.subTaskCfgExist(s, subtaskCfg1)
   311  	t.workerExist(s, workerInfo1)
   312  	// worker1 still exists, but it's offline.
   313  	t.workerOffline(s, workerName1)
   314  	// expect relay stage keep Running.
   315  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   316  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   317  	rebuildScheduler(ctx)
   318  
   319  	// CASE 3: start again with previous `Offline` worker, relay stage, subtask stage.
   320  	// CASE 3.1: previous information should recover.
   321  	// source1 is still unbound.
   322  	t.sourceBounds(s, []string{}, []string{sourceID1})
   323  	// worker1 still exists, but it's offline.
   324  	t.workerOffline(s, workerName1)
   325  	// static information are still there.
   326  	t.sourceCfgExist(s, sourceCfg1)
   327  	t.subTaskCfgExist(s, subtaskCfg1)
   328  	t.workerExist(s, workerInfo1)
   329  	// expect relay stage keep Running.
   330  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   331  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   332  	rebuildScheduler(ctx)
   333  
   334  	// CASE 3.2: start worker1 again.
   335  	// do keep-alive for worker1 again.
   336  	ctx1, cancel1 = context.WithCancel(ctx)
   337  	wg.Add(1)
   338  	go func() {
   339  		defer wg.Done()
   340  		require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL))
   341  	}()
   342  	// wait for source1 bound to worker1.
   343  	require.Eventually(t.T(), func() bool {
   344  		bounds := s.BoundSources()
   345  		return len(bounds) == 1 && bounds[0] == sourceID1
   346  	}, 3*time.Second, 100*time.Millisecond)
   347  	// source1 bound to worker1.
   348  	t.sourceBounds(s, []string{sourceID1}, []string{})
   349  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName1))
   350  	// expect stages keep Running.
   351  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   352  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   353  	rebuildScheduler(ctx)
   354  
   355  	// CASE 4.1: previous information should recover.
   356  	// source1 is still bound.
   357  	t.sourceBounds(s, []string{sourceID1}, []string{})
   358  	// worker1 still exists, and it's bound.
   359  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName1))
   360  	// static information are still there.
   361  	t.sourceCfgExist(s, sourceCfg1)
   362  	t.subTaskCfgExist(s, subtaskCfg1)
   363  	t.workerExist(s, workerInfo1)
   364  	// expect stages keep Running.
   365  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   366  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   367  	rebuildScheduler(ctx)
   368  
   369  	// CASE 4.2: add another worker into the cluster.
   370  	// worker2 not exists before added.
   371  	t.workerNotExist(s, workerName2)
   372  	// add worker2.
   373  	require.NoError(t.T(), s.AddWorker(workerName2, workerAddr2))
   374  	// the worker added, but is offline.
   375  	t.workerExist(s, workerInfo2)
   376  	t.workerOffline(s, workerName2)
   377  	rebuildScheduler(ctx)
   378  
   379  	// CASE 4.3: the worker2 become online.
   380  	// do keep-alive for worker2.
   381  	ctx2, cancel2 := context.WithCancel(ctx)
   382  	wg.Add(1)
   383  	go func() {
   384  		defer wg.Done()
   385  		require.NoError(t.T(), ha.KeepAlive(ctx2, t.etcdTestCli, workerName2, keepAliveTTL))
   386  	}()
   387  	// wait for worker2 become Free.
   388  	require.Eventually(t.T(), func() bool {
   389  		w := s.GetWorkerByName(workerName2)
   390  		return w.Stage() == WorkerFree
   391  	}, 3*time.Second, 100*time.Millisecond)
   392  	t.workerFree(s, workerName2)
   393  	rebuildScheduler(ctx)
   394  
   395  	// CASE 4.4: add source config2.
   396  	// wait for source bound recovered
   397  	require.Eventually(t.T(), func() bool {
   398  		bounds := s.BoundSources()
   399  		return len(bounds) == 1 && bounds[0] == sourceID1
   400  	}, 3*time.Second, 100*time.Millisecond)
   401  	// source2 not exists before.
   402  	t.sourceCfgNotExist(s, sourceID2)
   403  	// add source2.
   404  	require.NoError(t.T(), s.AddSourceCfg(&sourceCfg2))
   405  	// source2 added.
   406  	t.sourceCfgExist(s, &sourceCfg2)
   407  	// source2 should bound to worker2.
   408  	t.workerBound(s, ha.NewSourceBound(sourceID2, workerName2))
   409  	t.sourceBounds(s, []string{sourceID1, sourceID2}, []string{})
   410  	require.NoError(t.T(), s.StartRelay(sourceID2, []string{workerName2}))
   411  	t.relayStageMatch(s, sourceID2, pb.Stage_Running)
   412  	rebuildScheduler(ctx)
   413  
   414  	// CASE 4.4.1: start a task with two sources.
   415  	// can't add more than one tasks at a time now.
   416  	require.True(t.T(), terror.ErrSchedulerMultiTask.Equal(s.AddSubTasks(false, pb.Stage_Running, subtaskCfg1, subtaskCfg21)))
   417  	// task2' config and stage not exists before.
   418  	t.subTaskCfgNotExist(s, taskName2, sourceID1)
   419  	t.subTaskCfgNotExist(s, taskName2, sourceID2)
   420  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_InvalidStage)
   421  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_InvalidStage)
   422  	// start task2.
   423  	require.NoError(t.T(), s.AddSubTasks(false, pb.Stage_Running, subtaskCfg21, subtaskCfg22))
   424  	// config added, stage become Running.
   425  	t.subTaskCfgExist(s, subtaskCfg21)
   426  	t.subTaskCfgExist(s, subtaskCfg22)
   427  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_Running)
   428  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_Running)
   429  	t.validatorStageMatch(s, taskName2, sourceID2, pb.Stage_Running)
   430  	// check lightning status is written to etcd
   431  	status, err = ha.GetAllLightningStatus(t.etcdTestCli, taskName2)
   432  	require.NoError(t.T(), err)
   433  	require.Equal(t.T(), []string{ha.LightningNotReady, ha.LightningNotReady}, status)
   434  	rebuildScheduler(ctx)
   435  
   436  	// CASE 4.4.2 fail to stop any task.
   437  	// can call without tasks or sources, return without error, but take no effect.
   438  	require.NoError(t.T(), s.RemoveSubTasks("", sourceID1))
   439  	require.NoError(t.T(), s.RemoveSubTasks(taskName1))
   440  	// stop not exist task.
   441  	require.True(t.T(), terror.ErrSchedulerSubTaskOpTaskNotExist.Equal(s.RemoveSubTasks("not-exist", sourceID1)))
   442  	// config and stage not changed.
   443  	t.subTaskCfgExist(s, subtaskCfg21)
   444  	t.subTaskCfgExist(s, subtaskCfg22)
   445  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_Running)
   446  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_Running)
   447  
   448  	// CASE 4.5: update subtasks stage from different current stage.
   449  	// pause <task2, source1>.
   450  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Paused, taskName2, sourceID1))
   451  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_Paused)
   452  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_Running)
   453  	// resume <task2, source1 and source2>.
   454  	require.NoError(t.T(), s.UpdateExpectSubTaskStage(pb.Stage_Running, taskName2, sourceID1, sourceID2))
   455  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_Running)
   456  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_Running)
   457  	rebuildScheduler(ctx)
   458  
   459  	// CASE 4.6: try remove source when subtasks exist.
   460  	// wait for source bound recovered
   461  	require.Eventually(t.T(), func() bool {
   462  		bounds := s.BoundSources()
   463  		return len(bounds) == 2 && bounds[0] == sourceID1 && bounds[1] == sourceID2
   464  	}, 3*time.Second, 100*time.Millisecond)
   465  	require.True(t.T(), terror.ErrSchedulerSourceOpTaskExist.Equal(s.RemoveSourceCfg(sourceID2)))
   466  	// source2 keep there.
   467  	t.sourceCfgExist(s, &sourceCfg2)
   468  	// source2 still bound to worker2.
   469  	t.workerBound(s, ha.NewSourceBound(sourceID2, workerName2))
   470  	t.sourceBounds(s, []string{sourceID1, sourceID2}, []string{})
   471  	t.relayStageMatch(s, sourceID2, pb.Stage_Running)
   472  	rebuildScheduler(ctx)
   473  
   474  	// CASE 4.7: stop task2.
   475  	require.NoError(t.T(), s.RemoveSubTasks(taskName2, sourceID1, sourceID2))
   476  	t.subTaskCfgNotExist(s, taskName2, sourceID1)
   477  	t.subTaskCfgNotExist(s, taskName2, sourceID2)
   478  	t.subTaskStageMatch(s, taskName2, sourceID1, pb.Stage_InvalidStage)
   479  	t.subTaskStageMatch(s, taskName2, sourceID2, pb.Stage_InvalidStage)
   480  	t.validatorStageMatch(s, taskName2, sourceID2, pb.Stage_InvalidStage)
   481  	rebuildScheduler(ctx)
   482  
   483  	// CASE 4.7: remove source2.
   484  	require.NoError(t.T(), s.StopRelay(sourceID2, []string{workerName2}))
   485  	require.NoError(t.T(), s.RemoveSourceCfg(sourceID2))
   486  	require.True(t.T(), terror.ErrSchedulerSourceCfgNotExist.Equal(s.RemoveSourceCfg(sourceID2))) // already removed.
   487  	// source2 removed.
   488  	t.sourceCfgNotExist(s, sourceID2)
   489  	// worker2 become Free now.
   490  	t.workerFree(s, workerName2)
   491  	t.sourceBounds(s, []string{sourceID1}, []string{})
   492  	t.relayStageMatch(s, sourceID2, pb.Stage_InvalidStage)
   493  	rebuildScheduler(ctx)
   494  
   495  	// CASE 4.7.1: add source2 with specify worker1
   496  	// source2 not exist, worker1 is bound
   497  	t.sourceCfgNotExist(s, sourceID2)
   498  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName1))
   499  	require.True(t.T(), terror.ErrSchedulerWorkerNotFree.Equal(s.AddSourceCfgWithWorker(&sourceCfg2, workerName1)))
   500  	// source2 is not created because expected worker1 is already bound
   501  	t.sourceCfgNotExist(s, sourceID2)
   502  	rebuildScheduler(ctx)
   503  
   504  	// CASE 4.7.2: add source2 with specify worker2
   505  	// source2 not exist, worker2 should be free
   506  	t.sourceCfgNotExist(s, sourceID2)
   507  	t.workerFree(s, workerName2)
   508  	require.NoError(t.T(), s.AddSourceCfgWithWorker(&sourceCfg2, workerName2))
   509  	t.workerBound(s, ha.NewSourceBound(sourceID2, workerName2))
   510  	t.sourceBounds(s, []string{sourceID1, sourceID2}, []string{})
   511  	require.NoError(t.T(), s.StartRelay(sourceID2, []string{workerName2}))
   512  	t.relayStageMatch(s, sourceID2, pb.Stage_Running)
   513  	rebuildScheduler(ctx)
   514  
   515  	// CASE 4.7.3: remove source2 again.
   516  	require.NoError(t.T(), s.StopRelay(sourceID2, []string{workerName2}))
   517  	require.NoError(t.T(), s.RemoveSourceCfg(sourceID2))
   518  	require.True(t.T(), terror.ErrSchedulerSourceCfgNotExist.Equal(s.RemoveSourceCfg(sourceID2))) // already removed.
   519  	// source2 removed.
   520  	t.sourceCfgNotExist(s, sourceID2)
   521  	// worker2 become Free now.
   522  	t.workerFree(s, workerName2)
   523  	t.sourceBounds(s, []string{sourceID1}, []string{})
   524  	t.relayStageMatch(s, sourceID2, pb.Stage_InvalidStage)
   525  	rebuildScheduler(ctx)
   526  
   527  	// CASE 4.8: worker1 become offline.
   528  	// before shutdown, worker1 bound source
   529  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName1))
   530  	// cancel keep-alive.
   531  	cancel1()
   532  	// wait for worker1 become offline.
   533  	require.Eventually(t.T(), func() bool {
   534  		w := s.GetWorkerByName(workerName1)
   535  		require.NotNil(t.T(), w)
   536  		return w.Stage() == WorkerOffline
   537  	}, 3*time.Duration(keepAliveTTL)*time.Second, time.Second)
   538  	t.workerOffline(s, workerName1)
   539  	// source1 should bound to worker2.
   540  	t.sourceBounds(s, []string{sourceID1}, []string{})
   541  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName2))
   542  	// expect stages keep Running.
   543  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   544  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_Running)
   545  	rebuildScheduler(ctx)
   546  
   547  	// CASE 4.9: remove worker1.
   548  	require.NoError(t.T(), s.RemoveWorker(workerName1))
   549  	require.True(t.T(), terror.ErrSchedulerWorkerNotExist.Equal(s.RemoveWorker(workerName1))) // can't remove multiple times.
   550  	// worker1 not exists now.
   551  	t.workerNotExist(s, workerName1)
   552  	rebuildScheduler(ctx)
   553  
   554  	// CASE 4.10: stop task1.
   555  	// wait for worker2 become bouned.
   556  	require.Eventually(t.T(), func() bool {
   557  		w := s.GetWorkerByName(workerName2)
   558  		require.NotNil(t.T(), w)
   559  		return w.Stage() == WorkerBound
   560  	}, 3*time.Duration(keepAliveTTL)*time.Second, time.Second)
   561  	require.NoError(t.T(), s.RemoveSubTasks(taskName1, sourceID1))
   562  	t.subTaskCfgNotExist(s, taskName1, sourceID1)
   563  	t.subTaskStageMatch(s, taskName1, sourceID1, pb.Stage_InvalidStage)
   564  	rebuildScheduler(ctx)
   565  
   566  	// CASE 4.11: remove worker not supported when the worker is online.
   567  	// wait for worker2 become bouned.
   568  	require.Eventually(t.T(), func() bool {
   569  		w := s.GetWorkerByName(workerName2)
   570  		require.NotNil(t.T(), w)
   571  		return w.Stage() == WorkerBound
   572  	}, 3*time.Duration(keepAliveTTL)*time.Second, time.Second)
   573  	require.True(t.T(), terror.ErrSchedulerWorkerOnline.Equal(s.RemoveWorker(workerName2)))
   574  	t.sourceBounds(s, []string{sourceID1}, []string{})
   575  	t.workerBound(s, ha.NewSourceBound(sourceID1, workerName2))
   576  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   577  	rebuildScheduler(ctx)
   578  
   579  	// CASE 4.12: worker2 become offline.
   580  	cancel2()
   581  	wg.Wait()
   582  	// wait for worker2 become offline.
   583  	require.Eventually(t.T(), func() bool {
   584  		w := s.GetWorkerByName(workerName2)
   585  		require.NotNil(t.T(), w)
   586  		return w.Stage() == WorkerOffline
   587  	}, 3*time.Duration(keepAliveTTL)*time.Second, time.Second)
   588  	t.workerOffline(s, workerName2)
   589  	// source1 should unbound
   590  	t.sourceBounds(s, []string{}, []string{sourceID1})
   591  	// expect stages keep Running.
   592  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   593  	rebuildScheduler(ctx)
   594  
   595  	// CASE 4.13: remove worker2.
   596  	require.NoError(t.T(), s.RemoveWorker(workerName2))
   597  	t.workerNotExist(s, workerName2)
   598  	// relay stage still there.
   599  	t.sourceBounds(s, []string{}, []string{sourceID1})
   600  	t.relayStageMatch(s, sourceID1, pb.Stage_Running)
   601  	rebuildScheduler(ctx)
   602  
   603  	// CASE 4.14: remove source1.
   604  	require.NoError(t.T(), s.RemoveSourceCfg(sourceID1))
   605  	t.sourceCfgNotExist(s, sourceID1)
   606  	t.sourceBounds(s, []string{}, []string{})
   607  	t.relayStageMatch(s, sourceID1, pb.Stage_InvalidStage)
   608  }
   609  
   610  func (t *testSchedulerSuite) sourceCfgNotExist(s *Scheduler, source string) {
   611  	t.T().Helper()
   612  	require.Nil(t.T(), s.GetSourceCfgByID(source))
   613  	scm, _, err := ha.GetSourceCfg(t.etcdTestCli, source, 0)
   614  	require.NoError(t.T(), err)
   615  	require.Len(t.T(), scm, 0)
   616  }
   617  
   618  func (t *testSchedulerSuite) sourceCfgExist(s *Scheduler, expectCfg *config.SourceConfig) {
   619  	t.T().Helper()
   620  	cfgP := s.GetSourceCfgByID(expectCfg.SourceID)
   621  	require.Equal(t.T(), expectCfg, cfgP)
   622  	scm, _, err := ha.GetSourceCfg(t.etcdTestCli, expectCfg.SourceID, 0)
   623  	require.NoError(t.T(), err)
   624  	cfgV := scm[expectCfg.SourceID]
   625  	require.Equal(t.T(), expectCfg, cfgV)
   626  }
   627  
   628  func (t *testSchedulerSuite) subTaskCfgNotExist(s *Scheduler, task, source string) {
   629  	t.T().Helper()
   630  	require.Nil(t.T(), s.getSubTaskCfgByTaskSource(task, source))
   631  	cfgM, _, err := ha.GetSubTaskCfg(t.etcdTestCli, source, task, 0)
   632  	require.NoError(t.T(), err)
   633  	require.Len(t.T(), cfgM, 0)
   634  }
   635  
   636  func (t *testSchedulerSuite) subTaskCfgExist(s *Scheduler, expectCfg config.SubTaskConfig) {
   637  	t.T().Helper()
   638  	cfgP := s.getSubTaskCfgByTaskSource(expectCfg.Name, expectCfg.SourceID)
   639  	require.Equal(t.T(), expectCfg, *cfgP)
   640  	cfgM, _, err := ha.GetSubTaskCfg(t.etcdTestCli, expectCfg.SourceID, expectCfg.Name, 0)
   641  	require.NoError(t.T(), err)
   642  	require.Len(t.T(), cfgM, 1)
   643  	require.Equal(t.T(), expectCfg, cfgM[expectCfg.Name])
   644  }
   645  
   646  func (t *testSchedulerSuite) downstreamMetaNotExist(s *Scheduler, task string) {
   647  	t.T().Helper()
   648  	dbConfig, metaConfig := s.GetDownstreamMetaByTask(task)
   649  	require.Nil(t.T(), dbConfig)
   650  	require.Equal(t.T(), "", metaConfig)
   651  }
   652  
   653  func (t *testSchedulerSuite) downstreamMetaExist(s *Scheduler, task string, expectDBCfg dbconfig.DBConfig, expectMetaConfig string) {
   654  	t.T().Helper()
   655  	dbConfig, metaConfig := s.GetDownstreamMetaByTask(task)
   656  	require.NotNil(t.T(), dbConfig)
   657  	require.Equal(t.T(), expectDBCfg, *dbConfig)
   658  	require.Equal(t.T(), expectMetaConfig, metaConfig)
   659  }
   660  
   661  func (t *testSchedulerSuite) workerNotExist(s *Scheduler, worker string) {
   662  	t.T().Helper()
   663  	require.Nil(t.T(), s.GetWorkerByName(worker))
   664  	wm, _, err := ha.GetAllWorkerInfo(t.etcdTestCli)
   665  	require.NoError(t.T(), err)
   666  	_, ok := wm[worker]
   667  	require.False(t.T(), ok)
   668  }
   669  
   670  func (t *testSchedulerSuite) workerExist(s *Scheduler, info ha.WorkerInfo) {
   671  	t.T().Helper()
   672  	require.NotNil(t.T(), s.GetWorkerByName(info.Name))
   673  	require.Equal(t.T(), info, s.GetWorkerByName(info.Name).BaseInfo())
   674  	wm, _, err := ha.GetAllWorkerInfo(t.etcdTestCli)
   675  	require.NoError(t.T(), err)
   676  	require.Equal(t.T(), info, wm[info.Name])
   677  }
   678  
   679  func (t *testSchedulerSuite) workerOffline(s *Scheduler, worker string) {
   680  	t.T().Helper()
   681  	w := s.GetWorkerByName(worker)
   682  	require.NotNil(t.T(), w)
   683  	require.Equal(t.T(), nullBound, w.Bound())
   684  	require.Equal(t.T(), WorkerOffline, w.Stage())
   685  	wm, _, err := ha.GetAllWorkerInfo(t.etcdTestCli)
   686  	require.NoError(t.T(), err)
   687  	_, ok := wm[worker]
   688  	require.True(t.T(), ok)
   689  	sbm, _, err := ha.GetSourceBound(t.etcdTestCli, worker)
   690  	require.NoError(t.T(), err)
   691  	_, ok = sbm[worker]
   692  	require.False(t.T(), ok)
   693  }
   694  
   695  func (t *testSchedulerSuite) workerFree(s *Scheduler, worker string) {
   696  	t.T().Helper()
   697  	w := s.GetWorkerByName(worker)
   698  	require.NotNil(t.T(), w)
   699  	require.Equal(t.T(), nullBound, w.Bound())
   700  	require.Equal(t.T(), WorkerFree, w.Stage())
   701  	wm, _, err := ha.GetAllWorkerInfo(t.etcdTestCli)
   702  	require.NoError(t.T(), err)
   703  	_, ok := wm[worker]
   704  	require.True(t.T(), ok)
   705  	sbm, _, err := ha.GetSourceBound(t.etcdTestCli, worker)
   706  	require.NoError(t.T(), err)
   707  	_, ok = sbm[worker]
   708  	require.False(t.T(), ok)
   709  }
   710  
   711  func (t *testSchedulerSuite) workerBound(s *Scheduler, bound ha.SourceBound) {
   712  	t.T().Helper()
   713  	w := s.GetWorkerByName(bound.Worker)
   714  	require.NotNil(t.T(), w)
   715  	boundDeepEqualExcludeRev(t.T(), w.Bound(), bound)
   716  	require.Equal(t.T(), WorkerBound, w.Stage())
   717  	wm, _, err := ha.GetAllWorkerInfo(t.etcdTestCli)
   718  	require.NoError(t.T(), err)
   719  	_, ok := wm[bound.Worker]
   720  	require.True(t.T(), ok)
   721  	sbm, _, err := ha.GetSourceBound(t.etcdTestCli, bound.Worker)
   722  	require.NoError(t.T(), err)
   723  	boundDeepEqualExcludeRev(t.T(), sbm[bound.Worker], bound)
   724  }
   725  
   726  func (t *testSchedulerSuite) sourceBounds(s *Scheduler, expectBounds, expectUnbounds []string) {
   727  	t.T().Helper()
   728  	require.Equal(t.T(), expectBounds, s.BoundSources())
   729  	require.Equal(t.T(), expectUnbounds, s.UnboundSources())
   730  
   731  	wToB, _, err := ha.GetSourceBound(t.etcdTestCli, "")
   732  	require.NoError(t.T(), err)
   733  	require.Len(t.T(), wToB, len(expectBounds))
   734  
   735  	sToB := make(map[string]ha.SourceBound, len(wToB))
   736  	for _, b := range wToB {
   737  		sToB[b.Source] = b
   738  	}
   739  	for _, source := range expectBounds {
   740  		require.NotNil(t.T(), sToB[source])
   741  		require.NotNil(t.T(), s.GetWorkerBySource(source))
   742  		require.Equal(t.T(), WorkerBound, s.GetWorkerBySource(source).Stage())
   743  		boundDeepEqualExcludeRev(t.T(), sToB[source], s.GetWorkerBySource(source).Bound())
   744  	}
   745  
   746  	for _, source := range expectUnbounds {
   747  		require.Nil(t.T(), s.GetWorkerBySource(source))
   748  	}
   749  }
   750  
   751  func boundDeepEqualExcludeRev(t *testing.T, bound, expectBound ha.SourceBound) {
   752  	t.Helper()
   753  	expectBound.Revision = bound.Revision
   754  	require.Equal(t, expectBound, bound)
   755  }
   756  
   757  func stageDeepEqualExcludeRev(t *testing.T, stage, expectStage ha.Stage) {
   758  	t.Helper()
   759  	expectStage.Revision = stage.Revision
   760  	require.Equal(t, expectStage, stage)
   761  }
   762  
   763  func (t *testSchedulerSuite) relayStageMatch(s *Scheduler, source string, expectStage pb.Stage) {
   764  	t.T().Helper()
   765  	stage := ha.NewRelayStage(expectStage, source)
   766  	stageDeepEqualExcludeRev(t.T(), s.GetExpectRelayStage(source), stage)
   767  
   768  	eStage, _, err := ha.GetRelayStage(t.etcdTestCli, source)
   769  	require.NoError(t.T(), err)
   770  	switch expectStage {
   771  	case pb.Stage_Running, pb.Stage_Paused:
   772  		stageDeepEqualExcludeRev(t.T(), eStage, stage)
   773  	default:
   774  		require.Equal(t.T(), stageEmpty, eStage)
   775  	}
   776  }
   777  
   778  func (t *testSchedulerSuite) subTaskStageMatch(s *Scheduler, task, source string, expectStage pb.Stage) {
   779  	t.T().Helper()
   780  	stage := ha.NewSubTaskStage(expectStage, source, task)
   781  	stageDeepEqualExcludeRev(t.T(), s.GetExpectSubTaskStage(task, source), stage)
   782  
   783  	eStageM, _, err := ha.GetSubTaskStage(t.etcdTestCli, source, task)
   784  	require.NoError(t.T(), err)
   785  	switch expectStage {
   786  	case pb.Stage_Running, pb.Stage_Paused:
   787  		require.Len(t.T(), eStageM, 1)
   788  		stageDeepEqualExcludeRev(t.T(), eStageM[task], stage)
   789  	default:
   790  		require.Len(t.T(), eStageM, 0)
   791  	}
   792  }
   793  
   794  func (t *testSchedulerSuite) validatorStageMatch(s *Scheduler, task, source string, expectStage pb.Stage) {
   795  	t.T().Helper()
   796  	stage := ha.NewValidatorStage(expectStage, source, task)
   797  	var m map[string]ha.Stage
   798  	if v, ok := s.expectValidatorStages.Load(task); ok {
   799  		m = v.(map[string]ha.Stage)
   800  	}
   801  	if expectStage == pb.Stage_InvalidStage {
   802  		_, ok := m[source]
   803  		require.False(t.T(), ok)
   804  	} else {
   805  		stageDeepEqualExcludeRev(t.T(), m[source], stage)
   806  	}
   807  	stageM, _, err := ha.GetValidatorStage(t.etcdTestCli, source, task, 0)
   808  	require.NoError(t.T(), err)
   809  	switch expectStage {
   810  	case pb.Stage_Running, pb.Stage_Stopped:
   811  		require.Len(t.T(), stageM, 1)
   812  		stageDeepEqualExcludeRev(t.T(), stageM[task], stage)
   813  	default:
   814  		require.Len(t.T(), stageM, 0)
   815  	}
   816  }
   817  
   818  func (t *testSchedulerSuite) validatorModeMatch(s *Scheduler, task, source string, expectMode string) {
   819  	t.T().Helper()
   820  	cfg := s.getSubTaskCfgByTaskSource(task, source)
   821  	require.NotNil(t.T(), cfg)
   822  	require.Equal(t.T(), expectMode, cfg.ValidatorCfg.Mode)
   823  }
   824  
   825  func (t *testSchedulerSuite) TestRestartScheduler() {
   826  	var (
   827  		logger       = log.L()
   828  		sourceID1    = "mysql-replica-1"
   829  		workerName1  = "dm-worker-1"
   830  		workerAddr1  = "127.0.0.1:8262"
   831  		workerInfo1  = ha.NewWorkerInfo(workerName1, workerAddr1)
   832  		sourceBound1 = ha.NewSourceBound(sourceID1, workerName1)
   833  		wg           sync.WaitGroup
   834  		keepAliveTTL = int64(2) // NOTE: this should be >= minLeaseTTL, in second.
   835  	)
   836  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
   837  	require.NoError(t.T(), err)
   838  	sourceCfg1.SourceID = sourceID1
   839  
   840  	s := NewScheduler(&logger, security.Security{})
   841  	ctx, cancel := context.WithCancel(context.Background())
   842  	defer cancel()
   843  	// step 1: start scheduler
   844  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   845  	// step 1.1: add sourceCfg and worker
   846  	require.NoError(t.T(), s.AddSourceCfg(sourceCfg1))
   847  	t.sourceCfgExist(s, sourceCfg1)
   848  	require.NoError(t.T(), s.AddWorker(workerName1, workerAddr1))
   849  	t.workerExist(s, workerInfo1)
   850  	// step 2: start a worker
   851  	// step 2.1: worker start watching source bound
   852  	bsm, revBound, err := ha.GetSourceBound(t.etcdTestCli, workerName1)
   853  	require.NoError(t.T(), err)
   854  	require.Len(t.T(), bsm, 0)
   855  	sourceBoundCh := make(chan ha.SourceBound, 10)
   856  	sourceBoundErrCh := make(chan error, 10)
   857  	go func() {
   858  		ha.WatchSourceBound(ctx, t.etcdTestCli, workerName1, revBound+1, sourceBoundCh, sourceBoundErrCh)
   859  	}()
   860  	// step 2.2: worker start keepAlive
   861  	ctx1, cancel1 := context.WithCancel(ctx)
   862  	wg.Add(1)
   863  	go func() {
   864  		defer wg.Done()
   865  		require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL))
   866  	}()
   867  	// step 2.3: scheduler should bound source to worker
   868  	// wait for source1 bound to worker1.
   869  	require.Eventually(t.T(), func() bool {
   870  		bounds := s.BoundSources()
   871  		return len(bounds) == 1 && bounds[0] == sourceID1
   872  	}, 3*time.Second, 100*time.Millisecond)
   873  	checkSourceBoundCh := func() {
   874  		require.Eventually(t.T(), func() bool {
   875  			return len(sourceBoundCh) == 1
   876  		}, 5*time.Second, 500*time.Millisecond)
   877  		sourceBound := <-sourceBoundCh
   878  		sourceBound.Revision = 0
   879  		require.Equal(t.T(), sourceBound1, sourceBound)
   880  		require.Len(t.T(), sourceBoundErrCh, 0)
   881  	}
   882  	// worker should receive a put sourceBound event
   883  	checkSourceBoundCh()
   884  	// case 1: scheduler restarted, and worker keepalive brock but re-setup before scheduler is started
   885  	// step 3: restart scheduler, but don't stop worker keepalive, which can simulate two situations:
   886  	//			a. worker keepalive breaks but re-setup again before scheduler is started
   887  	//			b. worker is restarted but re-setup keepalive before scheduler is started
   888  	// dm-worker will close its source when keepalive is broken, so scheduler will send an event
   889  	// to trigger it to restart the source again
   890  	s.Close()
   891  	require.Len(t.T(), sourceBoundCh, 0)
   892  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
   893  	// worker should receive the trigger event again
   894  	checkSourceBoundCh()
   895  	// case 2: worker is restarted, and worker keepalive broke but scheduler didn't catch the delete event
   896  	// step 4: restart worker keepalive, which can simulator one situation:
   897  	//			a. worker keepalive breaks but re-setup again before keepaliveTTL is timeout
   898  	require.Len(t.T(), sourceBoundCh, 0)
   899  	ctx2, cancel2 := context.WithCancel(ctx)
   900  	// trigger same keepalive event again, just for test
   901  	wg.Add(1)
   902  	go func() {
   903  		defer wg.Done()
   904  		require.NoError(t.T(), ha.KeepAlive(ctx2, t.etcdTestCli, workerName1, keepAliveTTL))
   905  	}()
   906  	checkSourceBoundCh()
   907  	// case 3: scheduler is restarted, but worker also broke after scheduler is down
   908  	// step 5: stop scheduler -> stop worker keepalive -> restart scheduler
   909  	//		   scheduler should unbound the source and update the bound info in etcd
   910  	s.Close() // stop scheduler
   911  	cancel1()
   912  	cancel2() // stop worker keepalive
   913  	wg.Wait()
   914  	// check whether keepalive lease is out of date
   915  	time.Sleep(time.Duration(keepAliveTTL) * time.Second)
   916  	require.Eventually(t.T(), func() bool {
   917  		kam, _, err := ha.GetKeepAliveWorkers(t.etcdTestCli)
   918  		return err == nil && len(kam) == 0
   919  	}, 3*time.Second, 100*time.Millisecond)
   920  	require.Len(t.T(), sourceBoundCh, 0)
   921  	require.Len(t.T(), sourceBoundCh, 0)
   922  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli)) // restart scheduler
   923  	require.Len(t.T(), s.BoundSources(), 0)
   924  	unbounds := s.UnboundSources()
   925  	require.Len(t.T(), unbounds, 1)
   926  	require.Equal(t.T(), sourceID1, unbounds[0])
   927  	sourceBound1.Source = ""
   928  	sourceBound1.IsDeleted = true
   929  	checkSourceBoundCh()
   930  
   931  	// case 4: scheduler is restarted, but worker also broke after scheduler is down, then start another worker
   932  	// step 6: add another worker -> stop scheduler -> stop worker keepalive -> restart scheduler
   933  	//		   scheduler should unbound the source and rebound it to the newly alive worker
   934  
   935  	// first let the source bound again
   936  	ctx4, cancel4 := context.WithCancel(ctx)
   937  	wg.Add(1)
   938  	go func() {
   939  		defer wg.Done()
   940  		require.NoError(t.T(), ha.KeepAlive(ctx4, t.etcdTestCli, workerName1, keepAliveTTL))
   941  	}()
   942  	sourceBound1.Source = sourceID1
   943  	sourceBound1.IsDeleted = false
   944  	checkSourceBoundCh()
   945  
   946  	var (
   947  		workerName2 = "dm-worker-2"
   948  		workerAddr2 = "127.0.0.1:8263"
   949  		workerInfo2 = ha.NewWorkerInfo(workerName2, workerAddr2)
   950  	)
   951  
   952  	// add another worker
   953  	require.NoError(t.T(), s.AddWorker(workerName2, workerAddr2))
   954  	t.workerExist(s, workerInfo2)
   955  
   956  	// step 2.2: worker start keepAlive
   957  	go func() {
   958  		require.NoError(t.T(), ha.KeepAlive(ctx, t.etcdTestCli, workerName2, keepAliveTTL))
   959  	}()
   960  
   961  	s.Close() // stop scheduler
   962  	cancel4() // stop worker keepalive
   963  	wg.Wait()
   964  	// check whether keepalive lease is out of date
   965  	time.Sleep(time.Duration(keepAliveTTL) * time.Second)
   966  	require.Eventually(t.T(), func() bool {
   967  		kam, _, err := ha.GetKeepAliveWorkers(t.etcdTestCli)
   968  		return err == nil && len(kam) == 1
   969  	}, 3*time.Second, 100*time.Millisecond)
   970  	require.Len(t.T(), sourceBoundCh, 0)
   971  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli)) // restart scheduler
   972  	require.Len(t.T(), s.BoundSources(), 1)
   973  	w := s.workers[workerName2]
   974  	require.Equal(t.T(), WorkerBound, w.stage)
   975  	require.Equal(t.T(), sourceID1, w.bound.Source)
   976  	unbounds = s.UnboundSources()
   977  	require.Len(t.T(), unbounds, 0)
   978  }
   979  
   980  func (t *testSchedulerSuite) TestWatchWorkerEventEtcdCompact() {
   981  	var (
   982  		logger       = log.L()
   983  		s            = NewScheduler(&logger, security.Security{})
   984  		sourceID1    = "mysql-replica-1"
   985  		sourceID2    = "mysql-replica-2"
   986  		workerName1  = "dm-worker-1"
   987  		workerName2  = "dm-worker-2"
   988  		workerName3  = "dm-worker-3"
   989  		workerName4  = "dm-worker-4"
   990  		workerAddr1  = "127.0.0.1:8262"
   991  		workerAddr2  = "127.0.0.1:18262"
   992  		workerAddr3  = "127.0.0.1:18362"
   993  		workerAddr4  = "127.0.0.1:18462"
   994  		keepAliveTTL = int64(2) // NOTE: this should be >= minLeaseTTL, in second.
   995  	)
   996  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
   997  	require.NoError(t.T(), err)
   998  	sourceCfg1.SourceID = sourceID1
   999  	sourceCfg2 := *sourceCfg1
  1000  	sourceCfg2.SourceID = sourceID2
  1001  	ctx, cancel := context.WithCancel(context.Background())
  1002  	defer cancel()
  1003  
  1004  	// step 1: start an empty scheduler without listening the worker event
  1005  	s.started.Store(true)
  1006  	s.cancel = cancel
  1007  	s.etcdCli = t.etcdTestCli
  1008  
  1009  	// step 2: add two sources and register four workers
  1010  	require.NoError(t.T(), s.AddSourceCfg(sourceCfg1))
  1011  	require.NoError(t.T(), s.AddSourceCfg(&sourceCfg2))
  1012  	require.Len(t.T(), s.unbounds, 2)
  1013  	require.Contains(t.T(), s.unbounds, sourceID1)
  1014  	require.Contains(t.T(), s.unbounds, sourceID2)
  1015  
  1016  	require.NoError(t.T(), s.AddWorker(workerName1, workerAddr1))
  1017  	require.NoError(t.T(), s.AddWorker(workerName2, workerAddr2))
  1018  	require.NoError(t.T(), s.AddWorker(workerName3, workerAddr3))
  1019  	require.NoError(t.T(), s.AddWorker(workerName4, workerAddr4))
  1020  	require.Len(t.T(), s.workers, 4)
  1021  	require.Contains(t.T(), s.workers, workerName1)
  1022  	require.Contains(t.T(), s.workers, workerName2)
  1023  	require.Contains(t.T(), s.workers, workerName3)
  1024  	require.Contains(t.T(), s.workers, workerName4)
  1025  
  1026  	// step 3: add two workers, and then cancel them to simulate they have lost connection
  1027  	var wg sync.WaitGroup
  1028  	ctx1, cancel1 := context.WithCancel(ctx)
  1029  	wg.Add(1)
  1030  	go func() {
  1031  		defer wg.Done()
  1032  		require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL))
  1033  	}()
  1034  	wg.Add(1)
  1035  	go func() {
  1036  		defer wg.Done()
  1037  		require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName2, keepAliveTTL))
  1038  	}()
  1039  	require.Eventually(t.T(), func() bool {
  1040  		kam, _, e := ha.GetKeepAliveWorkers(t.etcdTestCli)
  1041  		return e == nil && len(kam) == 2
  1042  	}, 3*time.Second, 100*time.Millisecond)
  1043  	cancel1()
  1044  	wg.Wait()
  1045  	// check whether keepalive lease is out of date
  1046  	time.Sleep(time.Duration(keepAliveTTL) * time.Second)
  1047  	var rev int64
  1048  	require.Eventually(t.T(), func() bool {
  1049  		kam, rev1, e := ha.GetKeepAliveWorkers(t.etcdTestCli)
  1050  		rev = rev1
  1051  		return e == nil && len(kam) == 0
  1052  	}, 3*time.Second, 100*time.Millisecond)
  1053  
  1054  	// step 4: trigger etcd compaction and check whether we can receive it through watcher
  1055  	var startRev int64 = 1
  1056  	_, err = t.etcdTestCli.Compact(ctx, rev)
  1057  	require.NoError(t.T(), err)
  1058  	workerEvCh := make(chan ha.WorkerEvent, 10)
  1059  	workerErrCh := make(chan error, 10)
  1060  	ha.WatchWorkerEvent(ctx, t.etcdTestCli, startRev, workerEvCh, workerErrCh)
  1061  	select {
  1062  	case err := <-workerErrCh:
  1063  		require.Equal(t.T(), etcdErrCompacted, errors.Cause(err))
  1064  	case <-time.After(time.Second):
  1065  		t.T().Fatal("fail to get etcd error compacted")
  1066  	}
  1067  
  1068  	// step 5: scheduler start to handle workerEvent from compact revision, should handle worker keepalive events correctly
  1069  	ctx2, cancel2 := context.WithCancel(ctx)
  1070  	// step 5.1: start one worker before scheduler start to handle workerEvent
  1071  	wg.Add(1)
  1072  	go func() {
  1073  		defer wg.Done()
  1074  		require.NoError(t.T(), ha.KeepAlive(ctx2, t.etcdTestCli, workerName3, keepAliveTTL))
  1075  	}()
  1076  	require.Eventually(t.T(), func() bool {
  1077  		kam, _, err := ha.GetKeepAliveWorkers(t.etcdTestCli)
  1078  		if err == nil {
  1079  			if _, ok := kam[workerName3]; ok {
  1080  				return len(kam) == 1
  1081  			}
  1082  		}
  1083  		return false
  1084  	}, 3*time.Second, 100*time.Millisecond)
  1085  	// step 5.2: scheduler start to handle workerEvent
  1086  	wg.Add(1)
  1087  	go func() {
  1088  		defer wg.Done()
  1089  		require.NoError(t.T(), s.observeWorkerEvent(ctx2, startRev))
  1090  	}()
  1091  	// step 5.3: wait for scheduler to restart handleWorkerEvent, then start a new worker
  1092  	time.Sleep(time.Second)
  1093  	wg.Add(1)
  1094  	go func() {
  1095  		defer wg.Done()
  1096  		require.NoError(t.T(), ha.KeepAlive(ctx2, t.etcdTestCli, workerName4, keepAliveTTL))
  1097  	}()
  1098  	require.Eventually(t.T(), func() bool {
  1099  		unbounds := s.UnboundSources()
  1100  		return len(unbounds) == 0
  1101  	}, 3*time.Second, 100*time.Millisecond)
  1102  	require.Equal(t.T(), []string{sourceID1, sourceID2}, s.BoundSources())
  1103  	cancel2()
  1104  	wg.Wait()
  1105  
  1106  	// step 6: restart to observe workerEvents, should unbound all sources
  1107  	ctx3, cancel3 := context.WithCancel(ctx)
  1108  	wg.Add(1)
  1109  	go func() {
  1110  		defer wg.Done()
  1111  		require.NoError(t.T(), s.observeWorkerEvent(ctx3, startRev))
  1112  	}()
  1113  	require.Eventually(t.T(), func() bool {
  1114  		bounds := s.BoundSources()
  1115  		return len(bounds) == 0
  1116  	}, 3*time.Second, 100*time.Millisecond)
  1117  	require.Equal(t.T(), []string{sourceID1, sourceID2}, s.UnboundSources())
  1118  	cancel3()
  1119  	wg.Wait()
  1120  }
  1121  
  1122  func (t *testSchedulerSuite) TestLastBound() {
  1123  	var (
  1124  		logger      = log.L()
  1125  		s           = NewScheduler(&logger, security.Security{})
  1126  		sourceID1   = "mysql-replica-1"
  1127  		sourceID2   = "mysql-replica-2"
  1128  		workerName1 = "dm-worker-1"
  1129  		workerName2 = "dm-worker-2"
  1130  		workerName3 = "dm-worker-3"
  1131  		workerName4 = "dm-worker-4"
  1132  	)
  1133  
  1134  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  1135  	require.NoError(t.T(), err)
  1136  	sourceCfg1.SourceID = sourceID1
  1137  	sourceCfg2 := sourceCfg1
  1138  	sourceCfg2.SourceID = sourceID2
  1139  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1140  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1141  	worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}}
  1142  	worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}}
  1143  
  1144  	// step 1: start an empty scheduler without listening the worker event
  1145  	s.started.Store(true)
  1146  	s.etcdCli = t.etcdTestCli
  1147  	s.workers[workerName1] = worker1
  1148  	s.workers[workerName2] = worker2
  1149  	s.workers[workerName3] = worker3
  1150  	s.workers[workerName4] = worker4
  1151  	s.sourceCfgs[sourceID1] = sourceCfg1
  1152  	s.sourceCfgs[sourceID2] = sourceCfg2
  1153  
  1154  	s.lastBound[workerName1] = ha.SourceBound{Source: sourceID1}
  1155  	s.lastBound[workerName2] = ha.SourceBound{Source: sourceID2}
  1156  	s.unbounds[sourceID1] = struct{}{}
  1157  	s.unbounds[sourceID2] = struct{}{}
  1158  
  1159  	// worker1 goes to last bound source
  1160  	worker1.ToFree()
  1161  	bound, err := s.tryBoundForWorker(worker1)
  1162  	require.NoError(t.T(), err)
  1163  	require.True(t.T(), bound)
  1164  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1165  
  1166  	// worker3 has to bind source2
  1167  	worker3.ToFree()
  1168  	bound, err = s.tryBoundForWorker(worker3)
  1169  	require.NoError(t.T(), err)
  1170  	require.True(t.T(), bound)
  1171  	require.Equal(t.T(), worker3, s.bounds[sourceID2])
  1172  
  1173  	// though worker2 has a previous source, that source is not available, so not bound
  1174  	worker2.ToFree()
  1175  	bound, err = s.tryBoundForWorker(worker2)
  1176  	require.NoError(t.T(), err)
  1177  	require.False(t.T(), bound)
  1178  
  1179  	// worker4 is used to test whether source2 should be bound to worker2 rather than a new worker
  1180  	worker4.ToFree()
  1181  	bound, err = s.tryBoundForWorker(worker4)
  1182  	require.NoError(t.T(), err)
  1183  	require.False(t.T(), bound)
  1184  
  1185  	// after worker3 become offline, source2 should be bound to worker2
  1186  	s.updateStatusToUnbound(sourceID2)
  1187  	_, ok := s.bounds[sourceID2]
  1188  	require.False(t.T(), ok)
  1189  	worker3.ToOffline()
  1190  	bound, err = s.tryBoundForSource(sourceID2)
  1191  	require.NoError(t.T(), err)
  1192  	require.True(t.T(), bound)
  1193  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1194  }
  1195  
  1196  func (t *testSchedulerSuite) TestInvalidLastBound() {
  1197  	var (
  1198  		logger      = log.L()
  1199  		s           = NewScheduler(&logger, security.Security{})
  1200  		sourceID1   = "mysql-replica-1"
  1201  		sourceID2   = "invalid-replica-1"
  1202  		workerName1 = "dm-worker-1"
  1203  	)
  1204  
  1205  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  1206  	require.NoError(t.T(), err)
  1207  	sourceCfg1.SourceID = sourceID1
  1208  	sourceCfg2 := sourceCfg1
  1209  	sourceCfg2.SourceID = sourceID2
  1210  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1211  
  1212  	// step 1: start an empty scheduler without listening the worker event
  1213  	s.started.Store(true)
  1214  	s.etcdCli = t.etcdTestCli
  1215  	s.workers[workerName1] = worker1
  1216  	// sourceID2 doesn't have a source config and not in unbound
  1217  	s.sourceCfgs[sourceID1] = sourceCfg1
  1218  	s.lastBound[workerName1] = ha.SourceBound{Source: sourceID2}
  1219  	s.unbounds[sourceID1] = struct{}{}
  1220  	// step2: worker1 doesn't go to last bound source, because last source doesn't have a source config (might be removed)
  1221  	worker1.ToFree()
  1222  	bound, err := s.tryBoundForWorker(worker1)
  1223  	require.NoError(t.T(), err)
  1224  	require.True(t.T(), bound)
  1225  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1226  }
  1227  
  1228  func (t *testSchedulerSuite) TestTransferSource() {
  1229  	var (
  1230  		logger      = log.L()
  1231  		s           = NewScheduler(&logger, security.Security{})
  1232  		sourceID1   = "mysql-replica-1"
  1233  		sourceID2   = "mysql-replica-2"
  1234  		sourceID3   = "mysql-replica-3"
  1235  		sourceID4   = "mysql-replica-4"
  1236  		workerName1 = "dm-worker-1"
  1237  		workerName2 = "dm-worker-2"
  1238  		workerName3 = "dm-worker-3"
  1239  		workerName4 = "dm-worker-4"
  1240  	)
  1241  
  1242  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1243  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1244  	worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}}
  1245  	worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}}
  1246  
  1247  	// step 1: start an empty scheduler
  1248  	s.started.Store(true)
  1249  	s.etcdCli = t.etcdTestCli
  1250  	s.workers[workerName1] = worker1
  1251  	s.workers[workerName2] = worker2
  1252  	s.workers[workerName3] = worker3
  1253  	s.workers[workerName4] = worker4
  1254  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1255  	s.sourceCfgs[sourceID2] = &config.SourceConfig{}
  1256  
  1257  	worker1.ToFree()
  1258  	require.NoError(t.T(), s.boundSourceToWorker(sourceID1, worker1))
  1259  	worker2.ToFree()
  1260  	require.NoError(t.T(), s.boundSourceToWorker(sourceID2, worker2))
  1261  
  1262  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1263  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1264  
  1265  	worker3.ToFree()
  1266  	worker4.ToFree()
  1267  
  1268  	ctx := context.Background()
  1269  	// test invalid transfer: source not exists
  1270  	require.Error(t.T(), s.TransferSource(ctx, "not-exist", workerName3))
  1271  
  1272  	// test valid transfer: source -> worker = bound -> free
  1273  	require.NoError(t.T(), s.TransferSource(ctx, sourceID1, workerName4))
  1274  	require.Equal(t.T(), worker4, s.bounds[sourceID1])
  1275  	require.Equal(t.T(), WorkerFree, worker1.Stage())
  1276  
  1277  	// test valid transfer: source -> worker = unbound -> free
  1278  	s.sourceCfgs[sourceID3] = &config.SourceConfig{}
  1279  	s.unbounds[sourceID3] = struct{}{}
  1280  	require.NoError(t.T(), s.TransferSource(ctx, sourceID3, workerName3))
  1281  	require.Equal(t.T(), worker3, s.bounds[sourceID3])
  1282  
  1283  	// test valid transfer: self
  1284  	require.NoError(t.T(), s.TransferSource(ctx, sourceID3, workerName3))
  1285  	require.Equal(t.T(), worker3, s.bounds[sourceID3])
  1286  
  1287  	// test invalid transfer: source -> worker = bound -> bound
  1288  	require.Error(t.T(), s.TransferSource(ctx, sourceID1, workerName3))
  1289  	require.Equal(t.T(), worker4, s.bounds[sourceID1])
  1290  	require.Equal(t.T(), worker3, s.bounds[sourceID3])
  1291  
  1292  	// test invalid transfer: source -> worker = bound -> offline
  1293  	worker1.ToOffline()
  1294  	require.Error(t.T(), s.TransferSource(ctx, sourceID1, workerName1))
  1295  	require.Equal(t.T(), worker4, s.bounds[sourceID1])
  1296  
  1297  	// test invalid transfer: source -> worker = unbound -> bound
  1298  	s.sourceCfgs[sourceID4] = &config.SourceConfig{}
  1299  	s.unbounds[sourceID4] = struct{}{}
  1300  	require.Error(t.T(), s.TransferSource(ctx, sourceID4, workerName3))
  1301  	require.Equal(t.T(), worker3, s.bounds[sourceID3])
  1302  	delete(s.unbounds, sourceID4)
  1303  	delete(s.sourceCfgs, sourceID4)
  1304  
  1305  	worker1.ToFree()
  1306  	// now we have (worker1, nil) (worker2, source2) (worker3, source3) (worker4, source1)
  1307  
  1308  	// test fail halfway won't left old worker unbound
  1309  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/failToReplaceSourceBound", `return()`))
  1310  	require.Error(t.T(), s.TransferSource(ctx, sourceID1, workerName1))
  1311  	require.Equal(t.T(), worker4, s.bounds[sourceID1])
  1312  	require.Equal(t.T(), WorkerFree, worker1.Stage())
  1313  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/failToReplaceSourceBound"))
  1314  
  1315  	// set running tasks
  1316  	s.expectSubTaskStages.Store("test", map[string]ha.Stage{sourceID1: {Expect: pb.Stage_Running}})
  1317  
  1318  	// test can't transfer when running tasks not in sync unit
  1319  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus", `return("notInSyncUnit")`))
  1320  	defer failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus") //nolint:errcheck
  1321  	require.True(t.T(), terror.ErrSchedulerRequireRunningTaskInSyncUnit.Equal(s.TransferSource(ctx, sourceID1, workerName1)))
  1322  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus"))
  1323  
  1324  	// test can't transfer when query status met error
  1325  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus", `return("error")`))
  1326  	require.Contains(t.T(), s.TransferSource(ctx, sourceID1, workerName1).Error(), "failed to query worker")
  1327  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus"))
  1328  
  1329  	// test can transfer when all running task is in sync unit
  1330  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/skipBatchOperateTaskOnWorkerSleep", `return()`))
  1331  	defer failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/skipBatchOperateTaskOnWorkerSleep") //nolint:errcheck
  1332  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus", `return("allTaskIsPaused")`))
  1333  
  1334  	// we only retry 10 times, open a failpoint to make need retry more than 10 times, so this transfer will fail
  1335  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/batchOperateTaskOnWorkerMustRetry", `return(11)`))
  1336  	require.True(t.T(), terror.ErrSchedulerPauseTaskForTransferSource.Equal(s.TransferSource(ctx, sourceID1, workerName1)))
  1337  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/batchOperateTaskOnWorkerMustRetry"))
  1338  
  1339  	// now we can transfer successfully after 2 times retry
  1340  	s.expectSubTaskStages.Store("test", map[string]ha.Stage{sourceID1: {Expect: pb.Stage_Running}})
  1341  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/batchOperateTaskOnWorkerMustRetry", `return(2)`))
  1342  	require.NoError(t.T(), s.TransferSource(ctx, sourceID1, workerName1))
  1343  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/batchOperateTaskOnWorkerMustRetry"))
  1344  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1345  	require.Equal(t.T(), WorkerBound, worker1.Stage())
  1346  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateWorkerQueryStatus"))
  1347  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/skipBatchOperateTaskOnWorkerSleep"))
  1348  }
  1349  
  1350  func (t *testSchedulerSuite) TestStartStopRelay() {
  1351  	var (
  1352  		logger      = log.L()
  1353  		s           = NewScheduler(&logger, security.Security{})
  1354  		sourceID1   = "mysql-replica-1"
  1355  		sourceID2   = "mysql-replica-2"
  1356  		sourceID3   = "mysql-replica-3"
  1357  		sourceID4   = "mysql-replica-4"
  1358  		workerName1 = "dm-worker-1"
  1359  		workerName2 = "dm-worker-2"
  1360  		workerName3 = "dm-worker-3"
  1361  		workerName4 = "dm-worker-4"
  1362  	)
  1363  
  1364  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1365  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1366  	worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}}
  1367  	worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}}
  1368  
  1369  	// step 1: start an empty scheduler
  1370  	s.started.Store(true)
  1371  	s.etcdCli = t.etcdTestCli
  1372  	s.workers[workerName1] = worker1
  1373  	s.workers[workerName2] = worker2
  1374  	s.workers[workerName3] = worker3
  1375  	s.workers[workerName4] = worker4
  1376  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1377  	s.sourceCfgs[sourceID2] = &config.SourceConfig{}
  1378  
  1379  	worker1.ToFree()
  1380  	require.NoError(t.T(), s.boundSourceToWorker(sourceID1, worker1))
  1381  	worker2.ToFree()
  1382  	require.NoError(t.T(), s.boundSourceToWorker(sourceID2, worker2))
  1383  
  1384  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1385  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1386  
  1387  	worker3.ToFree()
  1388  	worker4.ToFree()
  1389  
  1390  	// test not exist source
  1391  	require.True(t.T(), terror.ErrSchedulerSourceCfgNotExist.Equal(s.StartRelay(sourceID3, []string{workerName1})))
  1392  	require.True(t.T(), terror.ErrSchedulerSourceCfgNotExist.Equal(s.StopRelay(sourceID4, []string{workerName1})))
  1393  	noWorkerSources := []string{sourceID1, sourceID2, sourceID3, sourceID4}
  1394  	for _, source := range noWorkerSources {
  1395  		workers, err := s.GetRelayWorkers(source)
  1396  		require.NoError(t.T(), err)
  1397  		require.Len(t.T(), workers, 0)
  1398  	}
  1399  
  1400  	// start-relay success on bound-same-source and free worker
  1401  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName1}))
  1402  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName1}))
  1403  	require.Len(t.T(), s.expectRelayStages, 1)
  1404  	require.Contains(t.T(), s.expectRelayStages, sourceID1)
  1405  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName3}))
  1406  	require.Len(t.T(), s.relayWorkers, 1)
  1407  	require.Len(t.T(), s.relayWorkers[sourceID1], 2)
  1408  	require.Contains(t.T(), s.relayWorkers[sourceID1], workerName1)
  1409  	require.Contains(t.T(), s.relayWorkers[sourceID1], workerName3)
  1410  	workers, err := s.GetRelayWorkers(sourceID1)
  1411  	require.NoError(t.T(), err)
  1412  	require.Equal(t.T(), []*Worker{worker1, worker3}, workers)
  1413  
  1414  	// failed on bound-not-same-source worker and not exist worker
  1415  	require.True(t.T(), terror.ErrSchedulerRelayWorkersWrongBound.Equal(s.StartRelay(sourceID1, []string{workerName2})))
  1416  	require.True(t.T(), terror.ErrSchedulerWorkerNotExist.Equal(s.StartRelay(sourceID1, []string{"not-exist"})))
  1417  
  1418  	// failed on one worker multiple relay source
  1419  	require.True(t.T(), terror.ErrSchedulerRelayWorkersBusy.Equal(s.StartRelay(sourceID2, []string{workerName3})))
  1420  
  1421  	// start another relay worker
  1422  	require.NoError(t.T(), s.StartRelay(sourceID2, []string{workerName2}))
  1423  	require.Len(t.T(), s.expectRelayStages, 2)
  1424  	require.Contains(t.T(), s.expectRelayStages, sourceID2)
  1425  	require.Len(t.T(), s.relayWorkers[sourceID2], 1)
  1426  	require.Contains(t.T(), s.relayWorkers[sourceID2], workerName2)
  1427  	workers, err = s.GetRelayWorkers(sourceID2)
  1428  	require.NoError(t.T(), err)
  1429  	require.Equal(t.T(), []*Worker{worker2}, workers)
  1430  
  1431  	// failed on not-same-source worker and not exist worker
  1432  	require.True(t.T(), terror.ErrSchedulerRelayWorkersWrongRelay.Equal(s.StopRelay(sourceID1, []string{workerName2})))
  1433  	require.True(t.T(), terror.ErrSchedulerWorkerNotExist.Equal(s.StopRelay(sourceID1, []string{"not-exist"})))
  1434  
  1435  	// nothing changed
  1436  	workers, err = s.GetRelayWorkers(sourceID1)
  1437  	require.NoError(t.T(), err)
  1438  	require.Equal(t.T(), []*Worker{worker1, worker3}, workers)
  1439  	workers, err = s.GetRelayWorkers(sourceID2)
  1440  	require.NoError(t.T(), err)
  1441  	require.Equal(t.T(), []*Worker{worker2}, workers)
  1442  
  1443  	// stop-relay success
  1444  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{workerName1}))
  1445  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{workerName1}))
  1446  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{workerName3}))
  1447  	require.Len(t.T(), s.expectRelayStages, 1)
  1448  	require.Contains(t.T(), s.expectRelayStages, sourceID2)
  1449  	require.Len(t.T(), s.relayWorkers, 1)
  1450  	require.Contains(t.T(), s.relayWorkers, sourceID2)
  1451  	workers, err = s.GetRelayWorkers(sourceID1)
  1452  	require.NoError(t.T(), err)
  1453  	require.Len(t.T(), workers, 0)
  1454  
  1455  	// can't bind source to a worker which has different relay
  1456  	// currently source1 -> worker1, source2 -> worker2
  1457  	require.Equal(t.T(), workerName1, s.bounds[sourceID1].baseInfo.Name)
  1458  	require.Equal(t.T(), workerName2, s.bounds[sourceID2].baseInfo.Name)
  1459  
  1460  	s.updateStatusToUnbound(sourceID2)
  1461  	require.Equal(t.T(), WorkerRelay, worker2.Stage())
  1462  	require.NoError(t.T(), s.StopRelay(sourceID2, []string{workerName2}))
  1463  	require.Equal(t.T(), WorkerFree, worker2.Stage())
  1464  
  1465  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName2}))
  1466  	require.Equal(t.T(), WorkerRelay, worker2.Stage())
  1467  	require.Equal(t.T(), sourceID1, worker2.RelaySourceID())
  1468  
  1469  	worker3.ToOffline()
  1470  	worker4.ToOffline()
  1471  
  1472  	bound, err := s.tryBoundForSource(sourceID2)
  1473  	require.NoError(t.T(), err)
  1474  	require.False(t.T(), bound)
  1475  }
  1476  
  1477  func (t *testSchedulerSuite) TestRelayWithWithoutWorker() {
  1478  	var (
  1479  		logger      = log.L()
  1480  		s           = NewScheduler(&logger, security.Security{})
  1481  		sourceID1   = "mysql-replica-1"
  1482  		workerName1 = "dm-worker-1"
  1483  		workerName2 = "dm-worker-2"
  1484  	)
  1485  
  1486  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1487  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1488  
  1489  	// step 1: start an empty scheduler
  1490  	s.started.Store(true)
  1491  	s.etcdCli = t.etcdTestCli
  1492  	s.workers[workerName1] = worker1
  1493  	s.workers[workerName2] = worker2
  1494  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1495  
  1496  	worker1.ToFree()
  1497  	require.NoError(t.T(), s.boundSourceToWorker(sourceID1, worker1))
  1498  	worker2.ToFree()
  1499  
  1500  	// step 2: check when enable-relay = false, can start/stop relay without worker name
  1501  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{}))
  1502  	require.True(t.T(), s.sourceCfgs[sourceID1].EnableRelay)
  1503  
  1504  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{}))
  1505  	require.True(t.T(), s.sourceCfgs[sourceID1].EnableRelay)
  1506  
  1507  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{}))
  1508  	require.False(t.T(), s.sourceCfgs[sourceID1].EnableRelay)
  1509  
  1510  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{}))
  1511  	require.False(t.T(), s.sourceCfgs[sourceID1].EnableRelay)
  1512  
  1513  	// step 3: check when enable-relay = false, can start/stop relay with worker name
  1514  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName1, workerName2}))
  1515  	require.False(t.T(), s.sourceCfgs[sourceID1].EnableRelay)
  1516  	require.Equal(t.T(), WorkerBound, worker1.Stage())
  1517  	require.Equal(t.T(), WorkerRelay, worker2.Stage())
  1518  
  1519  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{workerName1}))
  1520  	require.Equal(t.T(), WorkerBound, worker1.Stage())
  1521  	require.Equal(t.T(), WorkerRelay, worker2.Stage())
  1522  
  1523  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{workerName2}))
  1524  	require.Equal(t.T(), WorkerBound, worker1.Stage())
  1525  	require.Equal(t.T(), WorkerFree, worker2.Stage())
  1526  
  1527  	// step 4: check when enable-relay = true, can't start/stop relay with worker name
  1528  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{}))
  1529  
  1530  	err := s.StartRelay(sourceID1, []string{workerName1})
  1531  	require.True(t.T(), terror.ErrSchedulerStartRelayOnBound.Equal(err))
  1532  	err = s.StartRelay(sourceID1, []string{workerName2})
  1533  	require.True(t.T(), terror.ErrSchedulerStartRelayOnBound.Equal(err))
  1534  
  1535  	err = s.StopRelay(sourceID1, []string{workerName1})
  1536  	require.True(t.T(), terror.ErrSchedulerStopRelayOnBound.Equal(err))
  1537  	err = s.StopRelay(sourceID1, []string{workerName2})
  1538  	require.True(t.T(), terror.ErrSchedulerStopRelayOnBound.Equal(err))
  1539  
  1540  	require.NoError(t.T(), s.StopRelay(sourceID1, []string{}))
  1541  
  1542  	// step5. check when started relay with workerName, can't turn on enable-relay
  1543  	require.NoError(t.T(), s.StartRelay(sourceID1, []string{workerName1}))
  1544  
  1545  	err = s.StartRelay(sourceID1, []string{})
  1546  	require.True(t.T(), terror.ErrSchedulerStartRelayOnSpecified.Equal(err))
  1547  	err = s.StopRelay(sourceID1, []string{})
  1548  	require.True(t.T(), terror.ErrSchedulerStopRelayOnSpecified.Equal(err))
  1549  }
  1550  
  1551  func checkAllWorkersClosed(t *testing.T, s *Scheduler, closed bool) {
  1552  	t.Helper()
  1553  	for _, worker := range s.workers {
  1554  		cli, ok := worker.cli.(*workerrpc.GRPCClient)
  1555  		require.True(t, ok)
  1556  		require.Equal(t, closed, cli.Closed())
  1557  	}
  1558  }
  1559  
  1560  func (t *testSchedulerSuite) TestCloseAllWorkers() {
  1561  	var (
  1562  		logger = log.L()
  1563  		s      = NewScheduler(&logger, security.Security{})
  1564  		names  []string
  1565  	)
  1566  	ctx, cancel := context.WithCancel(context.Background())
  1567  	defer cancel()
  1568  	for i := 1; i < 4; i++ {
  1569  		names = append(names, fmt.Sprintf("worker%d", i))
  1570  	}
  1571  
  1572  	for i, name := range names {
  1573  		info := ha.NewWorkerInfo(name, fmt.Sprintf("127.0.0.1:%d", 50801+i))
  1574  		_, err := ha.PutWorkerInfo(t.etcdTestCli, info)
  1575  		require.NoError(t.T(), err)
  1576  	}
  1577  
  1578  	require.NoError(t.T(), failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/failToRecoverWorkersBounds", "return"))
  1579  	// Test closed when fail to start
  1580  	require.Errorf(t.T(), s.Start(ctx, t.etcdTestCli), "failToRecoverWorkersBounds")
  1581  	require.Len(t.T(), s.workers, 3)
  1582  	checkAllWorkersClosed(t.T(), s, true)
  1583  	require.NoError(t.T(), failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/failToRecoverWorkersBounds"))
  1584  
  1585  	s.workers = map[string]*Worker{}
  1586  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
  1587  	checkAllWorkersClosed(t.T(), s, false)
  1588  	s.Close()
  1589  	require.Len(t.T(), s.workers, 3)
  1590  	checkAllWorkersClosed(t.T(), s, true)
  1591  }
  1592  
  1593  func (t *testSchedulerSuite) TestStartSourcesWithoutSourceConfigsInEtcd() {
  1594  	var (
  1595  		logger       = log.L()
  1596  		s            = NewScheduler(&logger, security.Security{})
  1597  		sourceID1    = "mysql-replica-1"
  1598  		sourceID2    = "mysql-replica-2"
  1599  		workerName1  = "dm-worker-1"
  1600  		workerName2  = "dm-worker-2"
  1601  		workerAddr1  = "127.0.0.1:28362"
  1602  		workerAddr2  = "127.0.0.1:28363"
  1603  		wg           sync.WaitGroup
  1604  		keepaliveTTL = int64(60)
  1605  	)
  1606  
  1607  	ctx, cancel := context.WithCancel(context.Background())
  1608  	defer cancel()
  1609  
  1610  	s.started.Store(true)
  1611  	s.etcdCli = t.etcdTestCli
  1612  	// found source configs before bound
  1613  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1614  	s.sourceCfgs[sourceID2] = &config.SourceConfig{}
  1615  	s.unbounds[sourceID1] = struct{}{}
  1616  	s.unbounds[sourceID2] = struct{}{}
  1617  	require.NoError(t.T(), s.AddWorker(workerName1, workerAddr1))
  1618  	require.NoError(t.T(), s.AddWorker(workerName2, workerAddr2))
  1619  
  1620  	wg.Add(2)
  1621  	go func() {
  1622  		require.NoError(t.T(), ha.KeepAlive(ctx, t.etcdTestCli, workerName1, keepaliveTTL))
  1623  		wg.Done()
  1624  	}()
  1625  	go func() {
  1626  		require.NoError(t.T(), ha.KeepAlive(ctx, t.etcdTestCli, workerName2, keepaliveTTL))
  1627  		wg.Done()
  1628  	}()
  1629  
  1630  	s.workers[workerName1].stage = WorkerFree
  1631  	s.workers[workerName2].stage = WorkerFree
  1632  	bound, err := s.tryBoundForSource(sourceID1)
  1633  	require.NoError(t.T(), err)
  1634  	require.True(t.T(), bound)
  1635  	bound, err = s.tryBoundForSource(sourceID2)
  1636  	require.NoError(t.T(), err)
  1637  	require.True(t.T(), bound)
  1638  
  1639  	s.started.Store(false)
  1640  	sbm, _, err := ha.GetSourceBound(t.etcdTestCli, "")
  1641  	require.NoError(t.T(), err)
  1642  	require.Len(t.T(), sbm, 2)
  1643  	require.Eventually(t.T(), func() bool {
  1644  		kam, _, err2 := ha.GetKeepAliveWorkers(t.etcdTestCli)
  1645  		if err2 != nil {
  1646  			return false
  1647  		}
  1648  		return len(kam) == 2
  1649  	}, 3*time.Second, 100*time.Millisecond)
  1650  	// there isn't any source config in etcd
  1651  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
  1652  	require.Len(t.T(), s.bounds, 0)
  1653  	sbm, _, err = ha.GetSourceBound(t.etcdTestCli, "")
  1654  	require.NoError(t.T(), err)
  1655  	require.Len(t.T(), sbm, 0)
  1656  	cancel()
  1657  	wg.Wait()
  1658  }
  1659  
  1660  func (t *testSchedulerSuite) TestTransferWorkerAndSource() {
  1661  	var (
  1662  		logger      = log.L()
  1663  		s           = NewScheduler(&logger, security.Security{})
  1664  		sourceID1   = "mysql-replica-1"
  1665  		sourceID2   = "mysql-replica-2"
  1666  		sourceID3   = "mysql-replica-3"
  1667  		sourceID4   = "mysql-replica-4"
  1668  		workerName1 = "dm-worker-1"
  1669  		workerName2 = "dm-worker-2"
  1670  		workerName3 = "dm-worker-3"
  1671  		workerName4 = "dm-worker-4"
  1672  	)
  1673  
  1674  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1675  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1676  	worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}}
  1677  	worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}}
  1678  
  1679  	// step 1: start an empty scheduler
  1680  	s.started.Store(true)
  1681  	s.etcdCli = t.etcdTestCli
  1682  	s.workers[workerName1] = worker1
  1683  	s.workers[workerName2] = worker2
  1684  	s.workers[workerName3] = worker3
  1685  	s.workers[workerName4] = worker4
  1686  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1687  	s.sourceCfgs[sourceID2] = &config.SourceConfig{}
  1688  
  1689  	worker1.ToFree()
  1690  	worker2.ToFree()
  1691  	worker3.ToFree()
  1692  	worker4.ToFree()
  1693  	s.unbounds[sourceID1] = struct{}{}
  1694  	s.unbounds[sourceID2] = struct{}{}
  1695  
  1696  	// test free worker and unbound source
  1697  	require.NoError(t.T(), s.transferWorkerAndSource(workerName1, "", "", sourceID1))
  1698  	require.NoError(t.T(), s.transferWorkerAndSource("", sourceID2, workerName2, ""))
  1699  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1700  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1701  	require.Equal(t.T(), 0, len(s.unbounds))
  1702  
  1703  	// test transfer bound source to free worker
  1704  	require.NoError(t.T(), s.transferWorkerAndSource(workerName1, sourceID1, workerName4, ""))
  1705  	require.Equal(t.T(), worker4, s.bounds[sourceID1])
  1706  	require.Equal(t.T(), WorkerFree, worker1.Stage())
  1707  	require.Equal(t.T(), WorkerBound, worker4.Stage())
  1708  
  1709  	require.NoError(t.T(), s.transferWorkerAndSource(workerName3, "", workerName2, sourceID2))
  1710  	require.Equal(t.T(), worker3, s.bounds[sourceID2])
  1711  	require.Equal(t.T(), WorkerFree, worker2.Stage())
  1712  	require.Equal(t.T(), WorkerBound, worker3.Stage())
  1713  
  1714  	// test transfer bound worker to unbound source
  1715  	s.unbounds[sourceID3] = struct{}{}
  1716  	s.unbounds[sourceID4] = struct{}{}
  1717  	require.NoError(t.T(), s.transferWorkerAndSource("", sourceID3, workerName3, sourceID2))
  1718  	require.Equal(t.T(), worker3, s.bounds[sourceID3])
  1719  	// sourceID2 bound to last bound worker
  1720  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1721  
  1722  	require.NoError(t.T(), s.transferWorkerAndSource(workerName4, sourceID1, "", sourceID4))
  1723  	require.Equal(t.T(), worker4, s.bounds[sourceID4])
  1724  	// sourceID1 bound to last bound worker
  1725  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1726  
  1727  	require.Equal(t.T(), 0, len(s.unbounds))
  1728  
  1729  	// test transfer two bound sources
  1730  	require.NoError(t.T(), s.transferWorkerAndSource(workerName1, sourceID1, workerName2, sourceID2))
  1731  	require.NoError(t.T(), s.transferWorkerAndSource(workerName4, sourceID4, workerName3, sourceID3))
  1732  	require.Equal(t.T(), worker2, s.bounds[sourceID1])
  1733  	require.Equal(t.T(), worker1, s.bounds[sourceID2])
  1734  	require.Equal(t.T(), worker4, s.bounds[sourceID3])
  1735  	require.Equal(t.T(), worker3, s.bounds[sourceID4])
  1736  
  1737  	require.NoError(t.T(), worker1.StartRelay(sourceID2))
  1738  	err := s.transferWorkerAndSource(workerName1, sourceID2, workerName2, sourceID1)
  1739  	require.True(t.T(), terror.ErrSchedulerBoundDiffWithStartedRelay.Equal(err))
  1740  }
  1741  
  1742  func (t *testSchedulerSuite) TestWatchLoadTask() {
  1743  	var (
  1744  		logger      = log.L()
  1745  		s           = NewScheduler(&logger, security.Security{})
  1746  		task1       = "task1"
  1747  		task2       = "task2"
  1748  		sourceID1   = "mysql-replica-1"
  1749  		sourceID2   = "mysql-replica-2"
  1750  		workerName1 = "dm-worker-1"
  1751  		workerName2 = "dm-worker-2"
  1752  		workerName3 = "dm-worker-3"
  1753  		workerName4 = "dm-worker-4"
  1754  	)
  1755  
  1756  	// step 1: start an empty scheduler
  1757  	s.started.Store(true)
  1758  	s.etcdCli = t.etcdTestCli
  1759  
  1760  	worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}}
  1761  	worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}}
  1762  	worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}}
  1763  	worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}}
  1764  	s.workers[workerName1] = worker1
  1765  	s.workers[workerName2] = worker2
  1766  	s.workers[workerName3] = worker3
  1767  	s.workers[workerName4] = worker4
  1768  	s.sourceCfgs[sourceID1] = &config.SourceConfig{}
  1769  	s.sourceCfgs[sourceID2] = &config.SourceConfig{}
  1770  	s.subTaskCfgs.Store(task1, map[string]config.SubTaskConfig{
  1771  		sourceID1: {},
  1772  	})
  1773  	s.subTaskCfgs.Store(task2, map[string]config.SubTaskConfig{
  1774  		sourceID1: {},
  1775  		sourceID2: {},
  1776  	})
  1777  
  1778  	worker1.ToFree()
  1779  	require.NoError(t.T(), s.boundSourceToWorker(sourceID1, worker1))
  1780  	worker2.ToFree()
  1781  	require.NoError(t.T(), s.boundSourceToWorker(sourceID2, worker2))
  1782  	require.Equal(t.T(), worker1, s.bounds[sourceID1])
  1783  	require.Equal(t.T(), worker2, s.bounds[sourceID2])
  1784  
  1785  	worker3.ToFree()
  1786  	worker4.ToOffline()
  1787  
  1788  	// put task1, source1, worker3
  1789  	_, err := ha.PutLoadTask(t.etcdTestCli, task1, sourceID1, workerName3)
  1790  	require.NoError(t.T(), err)
  1791  	// put task2, source2, worker4
  1792  	_, err = ha.PutLoadTask(t.etcdTestCli, task2, sourceID2, workerName4)
  1793  	require.NoError(t.T(), err)
  1794  
  1795  	// get all load tasks
  1796  	var wg sync.WaitGroup
  1797  	ctx1, cancel1 := context.WithTimeout(context.Background(), 30*time.Second)
  1798  	defer cancel1()
  1799  	loadTasks, startRev, err := ha.GetAllLoadTask(t.etcdTestCli)
  1800  	require.NoError(t.T(), err)
  1801  	s.loadTasks = loadTasks
  1802  
  1803  	require.True(t.T(), s.hasLoadTaskByWorkerAndSource(workerName3, sourceID1))
  1804  	require.True(t.T(), s.hasLoadTaskByWorkerAndSource(workerName4, sourceID2))
  1805  
  1806  	// observer load tasks
  1807  	wg.Add(1)
  1808  	go func() {
  1809  		defer wg.Done()
  1810  		require.NoError(t.T(), s.observeLoadTask(ctx1, startRev))
  1811  	}()
  1812  
  1813  	// put task2, source1, worker1
  1814  	_, err = ha.PutLoadTask(t.etcdTestCli, task2, sourceID1, workerName1)
  1815  	require.NoError(t.T(), err)
  1816  	require.Eventually(t.T(), func() bool {
  1817  		s.mu.Lock()
  1818  		defer s.mu.Unlock()
  1819  		return s.hasLoadTaskByWorkerAndSource(workerName1, sourceID1)
  1820  	}, 3*time.Second, 100*time.Millisecond)
  1821  
  1822  	// del task2, source1, worker1
  1823  	_, _, err = ha.DelLoadTask(t.etcdTestCli, task2, sourceID1)
  1824  	require.NoError(t.T(), err)
  1825  	require.Eventually(t.T(), func() bool {
  1826  		s.mu.Lock()
  1827  		defer s.mu.Unlock()
  1828  		return !s.hasLoadTaskByWorkerAndSource(workerName1, sourceID1)
  1829  	}, 3*time.Second, 100*time.Millisecond)
  1830  
  1831  	// source1 transfer to worker3
  1832  	require.Eventually(t.T(), func() bool {
  1833  		w, ok := s.bounds[sourceID1]
  1834  		return ok && w.baseInfo.Name == workerName3
  1835  	}, 3*time.Second, 100*time.Millisecond)
  1836  
  1837  	require.Equal(t.T(), worker3, s.bounds[sourceID1])
  1838  	require.Equal(t.T(), WorkerFree, worker1.stage)
  1839  
  1840  	// worker4 online
  1841  	// source2 transfer to worker4
  1842  	require.NoError(t.T(), s.handleWorkerOnline(ha.WorkerEvent{WorkerName: workerName4}, true))
  1843  	require.Eventually(t.T(), func() bool {
  1844  		w, ok := s.bounds[sourceID2]
  1845  		return ok && w.baseInfo.Name == workerName4
  1846  	}, 3*time.Second, 100*time.Millisecond)
  1847  	require.Equal(t.T(), worker4, s.bounds[sourceID2])
  1848  	require.Equal(t.T(), WorkerFree, worker2.stage)
  1849  
  1850  	// after stop-task, hasLoadTaskByWorkerAndSource is no longer valid
  1851  	require.True(t.T(), s.hasLoadTaskByWorkerAndSource(workerName4, sourceID2))
  1852  	s.subTaskCfgs.Delete(task2)
  1853  	require.False(t.T(), s.hasLoadTaskByWorkerAndSource(workerName4, sourceID2))
  1854  
  1855  	cancel1()
  1856  	wg.Wait()
  1857  }
  1858  
  1859  func (t *testSchedulerSuite) TestWorkerHasDiffRelayAndBound() {
  1860  	var (
  1861  		logger      = log.L()
  1862  		s           = NewScheduler(&logger, security.Security{})
  1863  		sourceID1   = "mysql-replica-1"
  1864  		sourceID2   = "mysql-replica-2"
  1865  		workerName1 = "dm-worker-1"
  1866  		keepAlive   = int64(3)
  1867  	)
  1868  
  1869  	workerInfo := ha.WorkerInfo{
  1870  		Name: workerName1,
  1871  		Addr: "workerinfo.addr",
  1872  	}
  1873  	bound := ha.SourceBound{
  1874  		Source: sourceID1,
  1875  		Worker: workerName1,
  1876  	}
  1877  
  1878  	sourceCfg, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  1879  	require.NoError(t.T(), err)
  1880  	sourceCfg.Checker.BackoffMax = config.Duration{Duration: 5 * time.Second}
  1881  
  1882  	// prepare etcd data
  1883  	s.etcdCli = t.etcdTestCli
  1884  	sourceCfg.SourceID = sourceID1
  1885  	_, err = ha.PutSourceCfg(t.etcdTestCli, sourceCfg)
  1886  	require.NoError(t.T(), err)
  1887  	sourceCfg.SourceID = sourceID2
  1888  	_, err = ha.PutSourceCfg(t.etcdTestCli, sourceCfg)
  1889  	require.NoError(t.T(), err)
  1890  	_, err = ha.PutRelayConfig(t.etcdTestCli, sourceID2, workerName1)
  1891  	require.NoError(t.T(), err)
  1892  	_, err = ha.PutWorkerInfo(t.etcdTestCli, workerInfo)
  1893  	require.NoError(t.T(), err)
  1894  	_, err = ha.PutSourceBound(t.etcdTestCli, bound)
  1895  	require.NoError(t.T(), err)
  1896  	ctx, cancel := context.WithCancel(context.Background())
  1897  	defer cancel()
  1898  	//nolint:errcheck
  1899  	go ha.KeepAlive(ctx, t.etcdTestCli, workerName1, keepAlive)
  1900  
  1901  	// bootstrap
  1902  	require.NoError(t.T(), s.recoverSources())
  1903  	require.NoError(t.T(), s.recoverRelayConfigs())
  1904  	_, err = s.recoverWorkersBounds()
  1905  	require.NoError(t.T(), err)
  1906  
  1907  	// check
  1908  	require.Len(t.T(), s.relayWorkers[sourceID2], 1)
  1909  	_, ok := s.relayWorkers[sourceID2][workerName1]
  1910  	require.True(t.T(), ok)
  1911  	worker := s.workers[workerName1]
  1912  	require.Equal(t.T(), WorkerRelay, worker.Stage())
  1913  	require.Equal(t.T(), sourceID2, worker.RelaySourceID())
  1914  	_, ok = s.unbounds[sourceID1]
  1915  	require.True(t.T(), ok)
  1916  }
  1917  
  1918  func (t *testSchedulerSuite) TestUpgradeCauseConflictRelayType() {
  1919  	var (
  1920  		logger      = log.L()
  1921  		s           = NewScheduler(&logger, security.Security{})
  1922  		sourceID1   = "mysql-replica-1"
  1923  		workerName1 = "dm-worker-1"
  1924  		workerName2 = "dm-worker-2"
  1925  		keepAlive   = int64(3)
  1926  	)
  1927  
  1928  	workerInfo1 := ha.WorkerInfo{
  1929  		Name: workerName1,
  1930  		Addr: "workerinfo.addr",
  1931  	}
  1932  	workerInfo2 := ha.WorkerInfo{
  1933  		Name: workerName2,
  1934  		Addr: "workerinfo.addr",
  1935  	}
  1936  	bound := ha.SourceBound{
  1937  		Source: sourceID1,
  1938  		Worker: workerName1,
  1939  	}
  1940  
  1941  	sourceCfg, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  1942  	require.NoError(t.T(), err)
  1943  	sourceCfg.Checker.BackoffMax = config.Duration{Duration: 5 * time.Second}
  1944  
  1945  	// prepare etcd data
  1946  	s.etcdCli = t.etcdTestCli
  1947  	sourceCfg.EnableRelay = true
  1948  	sourceCfg.SourceID = sourceID1
  1949  	_, err = ha.PutSourceCfg(t.etcdTestCli, sourceCfg)
  1950  	require.NoError(t.T(), err)
  1951  	_, err = ha.PutRelayConfig(t.etcdTestCli, sourceID1, workerName1)
  1952  	require.NoError(t.T(), err)
  1953  	_, err = ha.PutRelayConfig(t.etcdTestCli, sourceID1, workerName2)
  1954  	require.NoError(t.T(), err)
  1955  	_, err = ha.PutWorkerInfo(t.etcdTestCli, workerInfo1)
  1956  	require.NoError(t.T(), err)
  1957  	_, err = ha.PutWorkerInfo(t.etcdTestCli, workerInfo2)
  1958  	require.NoError(t.T(), err)
  1959  	_, err = ha.PutSourceBound(t.etcdTestCli, bound)
  1960  	require.NoError(t.T(), err)
  1961  	ctx, cancel := context.WithCancel(context.Background())
  1962  	defer cancel()
  1963  	//nolint:errcheck
  1964  	go ha.KeepAlive(ctx, t.etcdTestCli, workerName1, keepAlive)
  1965  	//nolint:errcheck
  1966  	go ha.KeepAlive(ctx, t.etcdTestCli, workerName2, keepAlive)
  1967  
  1968  	// bootstrap
  1969  	require.NoError(t.T(), s.recoverSources())
  1970  	require.NoError(t.T(), s.recoverRelayConfigs())
  1971  	_, err = s.recoverWorkersBounds()
  1972  	require.NoError(t.T(), err)
  1973  
  1974  	// check when the relay config is conflicting with source config, relay config should be deleted
  1975  	require.Len(t.T(), s.relayWorkers[sourceID1], 0)
  1976  	result, _, err := ha.GetAllRelayConfig(t.etcdTestCli)
  1977  	require.NoError(t.T(), err)
  1978  	require.Len(t.T(), result, 0)
  1979  
  1980  	worker := s.workers[workerName1]
  1981  	require.Equal(t.T(), WorkerBound, worker.Stage())
  1982  	require.Len(t.T(), worker.RelaySourceID(), 0)
  1983  	require.Equal(t.T(), WorkerFree, s.workers[workerName2].Stage())
  1984  }
  1985  
  1986  func (t *testSchedulerSuite) TestOperateValidatorTask() {
  1987  	var (
  1988  		logger      = log.L()
  1989  		s           = NewScheduler(&logger, security.Security{})
  1990  		sourceID1   = "mysql-replica-1"
  1991  		workerName1 = "dm-worker-1"
  1992  		taskName    = "task-1"
  1993  		keepAlive   = int64(2)
  1994  		subtaskCfg  config.SubTaskConfig
  1995  	)
  1996  	require.NoError(t.T(), subtaskCfg.Decode(config.SampleSubtaskConfig, true))
  1997  	subtaskCfg.SourceID = sourceID1
  1998  	subtaskCfg.Name = taskName
  1999  	subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationNone}
  2000  	require.NoError(t.T(), subtaskCfg.Adjust(true))
  2001  
  2002  	workerInfo1 := ha.WorkerInfo{
  2003  		Name: workerName1,
  2004  		Addr: "workerinfo.addr",
  2005  	}
  2006  	bound := ha.SourceBound{
  2007  		Source: sourceID1,
  2008  		Worker: workerName1,
  2009  	}
  2010  	sourceCfg, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  2011  	require.NoError(t.T(), err)
  2012  	s.etcdCli = t.etcdTestCli
  2013  	sourceCfg.SourceID = sourceID1
  2014  	_, err = ha.PutSourceCfg(t.etcdTestCli, sourceCfg)
  2015  	require.NoError(t.T(), err)
  2016  	_, err = ha.PutWorkerInfo(t.etcdTestCli, workerInfo1)
  2017  	require.NoError(t.T(), err)
  2018  	_, err = ha.PutSourceBound(t.etcdTestCli, bound)
  2019  	require.NoError(t.T(), err)
  2020  	ctx, cancel := context.WithCancel(context.Background())
  2021  	defer cancel()
  2022  	//nolint:errcheck
  2023  	go ha.KeepAlive(ctx, t.etcdTestCli, workerName1, keepAlive)
  2024  	require.NoError(t.T(), s.recoverSources())
  2025  	_, err = s.recoverWorkersBounds()
  2026  	require.NoError(t.T(), err)
  2027  	require.NoError(t.T(), s.Start(ctx, t.etcdTestCli))
  2028  	// CASE 1: start subtask without starting validation
  2029  	require.NoError(t.T(), s.AddSubTasks(false, pb.Stage_Running, subtaskCfg)) // create new subtask without validation
  2030  	t.subTaskCfgExist(s, subtaskCfg)
  2031  	subtaskCfg.ValidatorCfg.Mode = config.ValidationFull // set mode
  2032  	validatorStages := []ha.Stage{ha.NewValidatorStage(pb.Stage_Running, subtaskCfg.SourceID, subtaskCfg.Name)}
  2033  	changedCfgs := []config.SubTaskConfig{subtaskCfg}
  2034  	require.NoError(t.T(), s.OperateValidationTask(validatorStages, changedCfgs))        // create validator task
  2035  	t.validatorStageMatch(s, subtaskCfg.Name, subtaskCfg.SourceID, pb.Stage_Running)     // task running
  2036  	t.validatorModeMatch(s, subtaskCfg.Name, subtaskCfg.SourceID, config.ValidationFull) // succeed to change mode
  2037  
  2038  	// CASE 2: stop running subtask
  2039  	validatorStages = []ha.Stage{ha.NewValidatorStage(pb.Stage_Stopped, subtaskCfg.SourceID, subtaskCfg.Name)}
  2040  	changedCfgs = []config.SubTaskConfig{}
  2041  	require.NoError(t.T(), s.OperateValidationTask(validatorStages, changedCfgs))
  2042  	t.validatorStageMatch(s, subtaskCfg.Name, subtaskCfg.SourceID, pb.Stage_Stopped) // task stopped
  2043  	require.NoError(t.T(), s.OperateValidationTask(validatorStages, changedCfgs))    // stop stopped validator task with no error
  2044  	t.validatorStageMatch(s, subtaskCfg.Name, subtaskCfg.SourceID, pb.Stage_Stopped) // stage not changed
  2045  }
  2046  
  2047  func (t *testSchedulerSuite) TestUpdateSubTasksAndSourceCfg() {
  2048  	defer t.clearTestInfoOperation()
  2049  
  2050  	var (
  2051  		logger       = log.L()
  2052  		s            = NewScheduler(&logger, security.Security{})
  2053  		sourceID1    = "mysql-replica-1"
  2054  		taskName1    = "task-1"
  2055  		workerName1  = "dm-worker-1"
  2056  		workerAddr1  = "127.0.0.1:8262"
  2057  		subtaskCfg1  config.SubTaskConfig
  2058  		keepAliveTTL = int64(5)
  2059  		ctx          = context.Background()
  2060  	)
  2061  	sourceCfg1, err := config.SourceCfgFromYamlAndVerify(config.SampleSourceConfig)
  2062  	require.NoError(t.T(), err)
  2063  	sourceCfg1.SourceID = sourceID1
  2064  
  2065  	require.NoError(t.T(), subtaskCfg1.Decode(config.SampleSubtaskConfig, true))
  2066  	subtaskCfg1.SourceID = sourceID1
  2067  	subtaskCfg1.Name = taskName1
  2068  	require.NoError(t.T(), subtaskCfg1.Adjust(true))
  2069  
  2070  	// not started scheduler can't update
  2071  	t.True(terror.ErrSchedulerNotStarted.Equal(s.UpdateSubTasks(ctx, subtaskCfg1)))
  2072  	t.True(terror.ErrSchedulerNotStarted.Equal(s.UpdateSourceCfg(sourceCfg1)))
  2073  
  2074  	// start the scheduler
  2075  	ctx, cancel := context.WithCancel(context.Background())
  2076  	defer cancel()
  2077  	t.NoError(s.Start(ctx, t.etcdTestCli))
  2078  
  2079  	// can't update source when source not added
  2080  	t.True(terror.ErrSchedulerSourceCfgNotExist.Equal(s.UpdateSourceCfg(sourceCfg1)))
  2081  
  2082  	subtaskCfg2 := subtaskCfg1
  2083  	subtaskCfg2.Name = "fake name"
  2084  	// can't update subtask with different task name
  2085  	t.True(terror.ErrSchedulerMultiTask.Equal(s.UpdateSubTasks(ctx, subtaskCfg1, subtaskCfg2)))
  2086  
  2087  	// can't update not added subtask
  2088  	t.True(terror.ErrSchedulerTaskNotExist.Equal(s.UpdateSubTasks(ctx, subtaskCfg1)))
  2089  
  2090  	// start worker,add source and subtask
  2091  	t.NoError(s.AddSourceCfg(sourceCfg1))
  2092  	ctx1, cancel1 := context.WithCancel(ctx)
  2093  	defer cancel1()
  2094  	t.NoError(s.AddWorker(workerName1, workerAddr1))
  2095  	go func() {
  2096  		t.NoError(ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL))
  2097  	}()
  2098  	// wait for source1 bound to worker1.
  2099  	t.Eventually(func() bool {
  2100  		bounds := s.BoundSources()
  2101  		return len(bounds) == 1 && bounds[0] == sourceID1
  2102  	}, 100*30*time.Millisecond, 30*time.Millisecond)
  2103  	t.NoError(s.AddSubTasks(false, pb.Stage_Running, subtaskCfg1))
  2104  
  2105  	// can't update subtask not in scheduler
  2106  	subtaskCfg2.Name = subtaskCfg1.Name
  2107  	subtaskCfg2.SourceID = "fake source name"
  2108  	t.True(terror.ErrSchedulerSubTaskNotExist.Equal(s.UpdateSubTasks(ctx, subtaskCfg2)))
  2109  
  2110  	// can't update subtask in running stage
  2111  	t.True(terror.ErrSchedulerSubTaskCfgUpdate.Equal(s.UpdateSubTasks(ctx, subtaskCfg1)))
  2112  	// can't update source when there is running tasks
  2113  	t.True(terror.ErrSchedulerSourceCfgUpdate.Equal(s.UpdateSourceCfg(sourceCfg1)))
  2114  
  2115  	// pause task
  2116  	t.NoError(s.UpdateExpectSubTaskStage(pb.Stage_Paused, taskName1, sourceID1))
  2117  
  2118  	// can't update source when there is a relay worker for this source
  2119  	t.NoError(s.StartRelay(sourceID1, []string{workerName1}))
  2120  	t.True(terror.ErrSchedulerSourceCfgUpdate.Equal(s.UpdateSourceCfg(sourceCfg1)))
  2121  	t.NoError(s.StopRelay(sourceID1, []string{workerName1}))
  2122  
  2123  	// can't updated when worker rpc error
  2124  	t.NoError(failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate", `return("error")`))
  2125  	t.Regexp("query error", s.UpdateSubTasks(ctx, subtaskCfg1))
  2126  	t.NoError(failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate"))
  2127  
  2128  	// can't updated when worker rpc check not pass
  2129  	t.NoError(failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate", `return("failed")`))
  2130  	t.True(terror.ErrSchedulerSubTaskCfgUpdate.Equal(s.UpdateSubTasks(ctx, subtaskCfg1)))
  2131  	t.NoError(failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate"))
  2132  
  2133  	// update success
  2134  	subtaskCfg1.Batch = 1000
  2135  	t.NoError(failpoint.Enable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate", `return("success")`))
  2136  	t.NoError(s.UpdateSubTasks(ctx, subtaskCfg1))
  2137  	t.NoError(failpoint.Disable("github.com/pingcap/tiflow/dm/master/scheduler/operateCheckSubtasksCanUpdate"))
  2138  	t.Equal(s.getSubTaskCfgByTaskSource(taskName1, sourceID1).Batch, subtaskCfg1.Batch)
  2139  
  2140  	sourceCfg1.MetaDir = "new meta"
  2141  	t.NoError(s.UpdateSourceCfg(sourceCfg1))
  2142  	t.Equal(s.GetSourceCfgByID(sourceID1).MetaDir, sourceCfg1.MetaDir)
  2143  }
  2144  
  2145  func (t *testSchedulerSuite) TestValidatorEnabledAndGetValidatorStage() {
  2146  	logger := log.L()
  2147  	s := NewScheduler(&logger, security.Security{})
  2148  	task := "test"
  2149  	source := "source"
  2150  	m, _ := s.expectValidatorStages.LoadOrStore(task, map[string]ha.Stage{})
  2151  	m.(map[string]ha.Stage)[source] = ha.Stage{Expect: pb.Stage_Running}
  2152  
  2153  	t.True(s.ValidatorEnabled(task, source))
  2154  	t.False(s.ValidatorEnabled(task, "not-exist"))
  2155  	t.False(s.ValidatorEnabled("not-exist", source))
  2156  	t.False(s.ValidatorEnabled("not-exist", "not-exist"))
  2157  
  2158  	stage := s.GetValidatorStage(task, source)
  2159  	t.NotNil(stage)
  2160  	t.Equal(pb.Stage_Running, stage.Expect)
  2161  	t.Nil(s.GetValidatorStage(task, "not-exist"))
  2162  	t.Nil(s.GetValidatorStage("not-exist", source))
  2163  	t.Nil(s.GetValidatorStage("not-exist", "not-exist"))
  2164  }