github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner/changefeed_test.go (about)

     1  // Copyright 2021 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package owner
    15  
    16  import (
    17  	"context"
    18  	"sync/atomic"
    19  	"time"
    20  
    21  	"github.com/pingcap/check"
    22  	"github.com/pingcap/errors"
    23  	timodel "github.com/pingcap/parser/model"
    24  	"github.com/pingcap/ticdc/cdc/entry"
    25  	"github.com/pingcap/ticdc/cdc/model"
    26  	cdcContext "github.com/pingcap/ticdc/pkg/context"
    27  	"github.com/pingcap/ticdc/pkg/orchestrator"
    28  	"github.com/pingcap/ticdc/pkg/util/testleak"
    29  	"github.com/pingcap/tidb/store/tikv/oracle"
    30  )
    31  
    32  type mockDDLPuller struct {
    33  	// DDLPuller
    34  	resolvedTs model.Ts
    35  	ddlQueue   []*timodel.Job
    36  }
    37  
    38  func (m *mockDDLPuller) FrontDDL() (uint64, *timodel.Job) {
    39  	if len(m.ddlQueue) > 0 {
    40  		return m.ddlQueue[0].BinlogInfo.FinishedTS, m.ddlQueue[0]
    41  	}
    42  	return m.resolvedTs, nil
    43  }
    44  
    45  func (m *mockDDLPuller) PopFrontDDL() (uint64, *timodel.Job) {
    46  	if len(m.ddlQueue) > 0 {
    47  		job := m.ddlQueue[0]
    48  		m.ddlQueue = m.ddlQueue[1:]
    49  		return job.BinlogInfo.FinishedTS, job
    50  	}
    51  	return m.resolvedTs, nil
    52  }
    53  
    54  func (m *mockDDLPuller) Close() {}
    55  
    56  func (m *mockDDLPuller) Run(ctx cdcContext.Context) error {
    57  	<-ctx.Done()
    58  	return nil
    59  }
    60  
    61  type mockAsyncSink struct {
    62  	// AsyncSink
    63  	ddlExecuting *model.DDLEvent
    64  	ddlDone      bool
    65  	checkpointTs model.Ts
    66  	syncPoint    model.Ts
    67  	syncPointHis []model.Ts
    68  }
    69  
    70  func (m *mockAsyncSink) EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) {
    71  	m.ddlExecuting = ddl
    72  	defer func() { m.ddlDone = false }()
    73  	return m.ddlDone, nil
    74  }
    75  
    76  func (m *mockAsyncSink) SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error {
    77  	if checkpointTs == m.syncPoint {
    78  		return nil
    79  	}
    80  	m.syncPoint = checkpointTs
    81  	m.syncPointHis = append(m.syncPointHis, checkpointTs)
    82  	return nil
    83  }
    84  
    85  func (m *mockAsyncSink) Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error {
    86  	return nil
    87  }
    88  
    89  func (m *mockAsyncSink) EmitCheckpointTs(ctx cdcContext.Context, ts uint64) {
    90  	atomic.StoreUint64(&m.checkpointTs, ts)
    91  }
    92  
    93  func (m *mockAsyncSink) Close(ctx context.Context) error {
    94  	return nil
    95  }
    96  
    97  func (m *mockAsyncSink) Barrier(ctx context.Context) error {
    98  	return nil
    99  }
   100  
   101  var _ = check.Suite(&changefeedSuite{})
   102  
   103  type changefeedSuite struct {
   104  }
   105  
   106  func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *model.ChangefeedReactorState,
   107  	map[model.CaptureID]*model.CaptureInfo, *orchestrator.ReactorStateTester) {
   108  	ctx.GlobalVars().PDClient = &mockPDClient{updateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) {
   109  		return safePoint, nil
   110  	}}
   111  	gcManager := newGCManager()
   112  	cf := newChangefeed4Test(ctx.ChangefeedVars().ID, gcManager, func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) {
   113  		return &mockDDLPuller{resolvedTs: startTs - 1}, nil
   114  	}, func(ctx cdcContext.Context) (AsyncSink, error) {
   115  		return &mockAsyncSink{}, nil
   116  	})
   117  	state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID)
   118  	tester := orchestrator.NewReactorStateTester(c, state, nil)
   119  	state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) {
   120  		c.Assert(info, check.IsNil)
   121  		info = ctx.ChangefeedVars().Info
   122  		return info, true, nil
   123  	})
   124  	tester.MustUpdate("/tidb/cdc/capture/"+ctx.GlobalVars().CaptureInfo.ID, []byte(`{"id":"`+ctx.GlobalVars().CaptureInfo.ID+`","address":"127.0.0.1:8300"}`))
   125  	tester.MustApplyPatches()
   126  	captures := map[model.CaptureID]*model.CaptureInfo{ctx.GlobalVars().CaptureInfo.ID: ctx.GlobalVars().CaptureInfo}
   127  	return cf, state, captures, tester
   128  }
   129  
   130  func (s *changefeedSuite) TestPreCheck(c *check.C) {
   131  	defer testleak.AfterTest(c)()
   132  	ctx := cdcContext.NewBackendContext4Test(true)
   133  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   134  	cf.Tick(ctx, state, captures)
   135  	tester.MustApplyPatches()
   136  	c.Assert(state.Status, check.NotNil)
   137  	c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID)
   138  
   139  	// test clean the meta data of offline capture
   140  	offlineCaputreID := "offline-capture"
   141  	state.PatchTaskStatus(offlineCaputreID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) {
   142  		return new(model.TaskStatus), true, nil
   143  	})
   144  	state.PatchTaskPosition(offlineCaputreID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) {
   145  		return new(model.TaskPosition), true, nil
   146  	})
   147  	state.PatchTaskWorkload(offlineCaputreID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) {
   148  		return make(model.TaskWorkload), true, nil
   149  	})
   150  	tester.MustApplyPatches()
   151  
   152  	cf.Tick(ctx, state, captures)
   153  	tester.MustApplyPatches()
   154  	c.Assert(state.Status, check.NotNil)
   155  	c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID)
   156  	c.Assert(state.TaskStatuses, check.Not(check.HasKey), offlineCaputreID)
   157  	c.Assert(state.TaskPositions, check.Not(check.HasKey), offlineCaputreID)
   158  	c.Assert(state.Workloads, check.Not(check.HasKey), offlineCaputreID)
   159  }
   160  
   161  func (s *changefeedSuite) TestInitialize(c *check.C) {
   162  	defer testleak.AfterTest(c)()
   163  	ctx := cdcContext.NewBackendContext4Test(true)
   164  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   165  	defer cf.Close()
   166  	// pre check
   167  	cf.Tick(ctx, state, captures)
   168  	tester.MustApplyPatches()
   169  
   170  	// initialize
   171  	cf.Tick(ctx, state, captures)
   172  	tester.MustApplyPatches()
   173  	c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs)
   174  }
   175  
   176  func (s *changefeedSuite) TestHandleError(c *check.C) {
   177  	defer testleak.AfterTest(c)()
   178  	ctx := cdcContext.NewBackendContext4Test(true)
   179  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   180  	defer cf.Close()
   181  	// pre check
   182  	cf.Tick(ctx, state, captures)
   183  	tester.MustApplyPatches()
   184  
   185  	// initialize
   186  	cf.Tick(ctx, state, captures)
   187  	tester.MustApplyPatches()
   188  
   189  	cf.errCh <- errors.New("fake error")
   190  	// handle error
   191  	cf.Tick(ctx, state, captures)
   192  	tester.MustApplyPatches()
   193  	c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs)
   194  	c.Assert(state.Info.Error.Message, check.Equals, "fake error")
   195  }
   196  
   197  func (s *changefeedSuite) TestExecDDL(c *check.C) {
   198  	defer testleak.AfterTest(c)()
   199  	ctx := cdcContext.NewBackendContext4Test(true)
   200  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   201  	defer cf.Close()
   202  	helper := entry.NewSchemaTestHelper(c)
   203  	defer helper.Close()
   204  	tickThreeTime := func() {
   205  		cf.Tick(ctx, state, captures)
   206  		tester.MustApplyPatches()
   207  		cf.Tick(ctx, state, captures)
   208  		tester.MustApplyPatches()
   209  		cf.Tick(ctx, state, captures)
   210  		tester.MustApplyPatches()
   211  	}
   212  	// pre check and initialize
   213  	tickThreeTime()
   214  
   215  	// ddl puller resolved ts grow uo
   216  	mockDDLPuller := cf.ddlPuller.(*mockDDLPuller)
   217  	mockDDLPuller.resolvedTs += 1000
   218  	mockAsyncSink := cf.sink.(*mockAsyncSink)
   219  	// three tick to make sure all barriers set in initialize is handled
   220  	tickThreeTime()
   221  	c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs)
   222  
   223  	// handle create database
   224  	job := helper.DDL2Job("create database test1")
   225  	mockDDLPuller.resolvedTs += 1000
   226  	job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs
   227  	mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job)
   228  	tickThreeTime()
   229  	c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs)
   230  	c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create database test1")
   231  
   232  	// executing the ddl finished
   233  	mockAsyncSink.ddlDone = true
   234  	mockDDLPuller.resolvedTs += 1000
   235  	tickThreeTime()
   236  	c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs)
   237  
   238  	// handle create table
   239  	job = helper.DDL2Job("create table test1.test1(id int primary key)")
   240  	mockDDLPuller.resolvedTs += 1000
   241  	job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs
   242  	mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job)
   243  	tickThreeTime()
   244  	c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs)
   245  	c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create table test1.test1(id int primary key)")
   246  
   247  	// executing the ddl finished
   248  	mockAsyncSink.ddlDone = true
   249  	mockDDLPuller.resolvedTs += 1000
   250  	tickThreeTime()
   251  	c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasKey, job.TableID)
   252  }
   253  
   254  func (s *changefeedSuite) TestSyncPoint(c *check.C) {
   255  	defer testleak.AfterTest(c)()
   256  	ctx := cdcContext.NewBackendContext4Test(true)
   257  	ctx.ChangefeedVars().Info.SyncPointEnabled = true
   258  	ctx.ChangefeedVars().Info.SyncPointInterval = 1 * time.Second
   259  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   260  	defer cf.Close()
   261  
   262  	// pre check
   263  	cf.Tick(ctx, state, captures)
   264  	tester.MustApplyPatches()
   265  
   266  	// initialize
   267  	cf.Tick(ctx, state, captures)
   268  	tester.MustApplyPatches()
   269  
   270  	mockDDLPuller := cf.ddlPuller.(*mockDDLPuller)
   271  	mockAsyncSink := cf.sink.(*mockAsyncSink)
   272  	// add 5s to resolvedTs
   273  	mockDDLPuller.resolvedTs = oracle.GoTimeToTS(oracle.GetTimeFromTS(mockDDLPuller.resolvedTs).Add(5 * time.Second))
   274  	// tick 20 times
   275  	for i := 0; i <= 20; i++ {
   276  		cf.Tick(ctx, state, captures)
   277  		tester.MustApplyPatches()
   278  	}
   279  	for i := 1; i < len(mockAsyncSink.syncPointHis); i++ {
   280  		// check the time interval between adjacent sync points is less or equal than one second
   281  		c.Assert(mockAsyncSink.syncPointHis[i]-mockAsyncSink.syncPointHis[i-1], check.LessEqual, uint64(1000<<18))
   282  	}
   283  	c.Assert(len(mockAsyncSink.syncPointHis), check.GreaterEqual, 5)
   284  }
   285  
   286  func (s *changefeedSuite) TestFinished(c *check.C) {
   287  	defer testleak.AfterTest(c)()
   288  	ctx := cdcContext.NewBackendContext4Test(true)
   289  	ctx.ChangefeedVars().Info.TargetTs = ctx.ChangefeedVars().Info.StartTs + 1000
   290  	cf, state, captures, tester := createChangefeed4Test(ctx, c)
   291  	defer cf.Close()
   292  
   293  	// pre check
   294  	cf.Tick(ctx, state, captures)
   295  	tester.MustApplyPatches()
   296  
   297  	// initialize
   298  	cf.Tick(ctx, state, captures)
   299  	tester.MustApplyPatches()
   300  
   301  	mockDDLPuller := cf.ddlPuller.(*mockDDLPuller)
   302  	mockDDLPuller.resolvedTs += 2000
   303  	// tick many times to make sure the change feed is stopped
   304  	for i := 0; i <= 10; i++ {
   305  		cf.Tick(ctx, state, captures)
   306  		tester.MustApplyPatches()
   307  	}
   308  
   309  	c.Assert(state.Status.CheckpointTs, check.Equals, state.Info.TargetTs)
   310  	c.Assert(state.Info.State, check.Equals, model.StateFinished)
   311  }