github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/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 "fmt" 19 "os" 20 "path/filepath" 21 "sync" 22 "testing" 23 "time" 24 25 "github.com/pingcap/errors" 26 timodel "github.com/pingcap/tidb/pkg/parser/model" 27 "github.com/pingcap/tiflow/cdc/entry" 28 "github.com/pingcap/tiflow/cdc/model" 29 "github.com/pingcap/tiflow/cdc/puller" 30 credo "github.com/pingcap/tiflow/cdc/redo" 31 "github.com/pingcap/tiflow/cdc/scheduler" 32 "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" 33 "github.com/pingcap/tiflow/cdc/vars" 34 "github.com/pingcap/tiflow/pkg/config" 35 "github.com/pingcap/tiflow/pkg/etcd" 36 "github.com/pingcap/tiflow/pkg/filter" 37 "github.com/pingcap/tiflow/pkg/orchestrator" 38 "github.com/pingcap/tiflow/pkg/redo" 39 "github.com/pingcap/tiflow/pkg/sink/observer" 40 "github.com/pingcap/tiflow/pkg/txnutil/gc" 41 "github.com/pingcap/tiflow/pkg/upstream" 42 "github.com/pingcap/tiflow/pkg/util" 43 "github.com/stretchr/testify/require" 44 "github.com/tikv/client-go/v2/oracle" 45 ) 46 47 var _ puller.DDLPuller = (*mockDDLPuller)(nil) 48 49 type mockDDLPuller struct { 50 // DDLPuller 51 resolvedTs model.Ts 52 ddlQueue []*timodel.Job 53 schemaStorage entry.SchemaStorage 54 } 55 56 func (m *mockDDLPuller) PopFrontDDL() (uint64, *timodel.Job) { 57 if len(m.ddlQueue) > 0 { 58 job := m.ddlQueue[0] 59 m.ddlQueue = m.ddlQueue[1:] 60 err := m.schemaStorage.HandleDDLJob(job) 61 if err != nil { 62 panic(fmt.Sprintf("handle ddl job failed: %v", err)) 63 } 64 return job.BinlogInfo.FinishedTS, job 65 } 66 return m.resolvedTs, nil 67 } 68 69 func (m *mockDDLPuller) Close() {} 70 71 func (m *mockDDLPuller) Run(ctx context.Context) error { 72 <-ctx.Done() 73 return nil 74 } 75 76 func (m *mockDDLPuller) ResolvedTs() model.Ts { 77 if len(m.ddlQueue) > 0 { 78 return m.ddlQueue[0].BinlogInfo.FinishedTS 79 } 80 return m.resolvedTs 81 } 82 83 type mockDDLSink struct { 84 // DDLSink 85 ddlExecuting *model.DDLEvent 86 ddlDone bool 87 // whether to reset ddlDone flag, only for rename table 88 resetDDLDone bool 89 // whether to record the DDL history, only for rename table 90 recordDDLHistory bool 91 // a slice of DDL history, only for rename table 92 ddlHistory []string 93 mu struct { 94 sync.Mutex 95 checkpointTs model.Ts 96 currentTables []*model.TableInfo 97 } 98 syncPoint model.Ts 99 syncPointHis []model.Ts 100 101 wg sync.WaitGroup 102 } 103 104 func (m *mockDDLSink) run(ctx context.Context) { 105 m.wg.Add(1) 106 go func() { 107 <-ctx.Done() 108 m.wg.Done() 109 }() 110 } 111 112 func (m *mockDDLSink) emitDDLEvent(ctx context.Context, ddl *model.DDLEvent) (bool, error) { 113 m.ddlExecuting = ddl 114 defer func() { 115 if m.resetDDLDone { 116 m.ddlDone = false 117 } 118 }() 119 if m.recordDDLHistory { 120 m.ddlHistory = append(m.ddlHistory, ddl.Query) 121 } else { 122 m.ddlHistory = nil 123 } 124 return m.ddlDone, nil 125 } 126 127 func (m *mockDDLSink) emitSyncPoint(ctx context.Context, checkpointTs uint64) error { 128 if checkpointTs == m.syncPoint { 129 return nil 130 } 131 m.syncPoint = checkpointTs 132 m.syncPointHis = append(m.syncPointHis, checkpointTs) 133 return nil 134 } 135 136 func (m *mockDDLSink) emitCheckpointTs(ts uint64, tables []*model.TableInfo) { 137 m.mu.Lock() 138 defer m.mu.Unlock() 139 m.mu.checkpointTs = ts 140 m.mu.currentTables = tables 141 } 142 143 func (m *mockDDLSink) getCheckpointTsAndTableNames() (uint64, []*model.TableInfo) { 144 m.mu.Lock() 145 defer m.mu.Unlock() 146 return m.mu.checkpointTs, m.mu.currentTables 147 } 148 149 func (m *mockDDLSink) close(ctx context.Context) error { 150 m.wg.Wait() 151 return nil 152 } 153 154 func (m *mockDDLSink) Barrier(ctx context.Context) error { 155 return nil 156 } 157 158 type mockScheduler struct { 159 currentTables []model.TableID 160 } 161 162 func (m *mockScheduler) Tick( 163 ctx context.Context, 164 checkpointTs model.Ts, 165 currentTables []model.TableID, 166 captures map[model.CaptureID]*model.CaptureInfo, 167 barrier *schedulepb.BarrierWithMinTs, 168 ) (watermark schedulepb.Watermark, err error) { 169 m.currentTables = currentTables 170 return schedulepb.Watermark{ 171 CheckpointTs: barrier.MinTableBarrierTs, 172 ResolvedTs: barrier.GlobalBarrierTs, 173 LastSyncedTs: scheduler.CheckpointCannotProceed, 174 PullerResolvedTs: scheduler.CheckpointCannotProceed, 175 }, nil 176 } 177 178 // MoveTable is used to trigger manual table moves. 179 func (m *mockScheduler) MoveTable(tableID model.TableID, target model.CaptureID) {} 180 181 // Rebalance is used to trigger manual workload rebalances. 182 func (m *mockScheduler) Rebalance() {} 183 184 // DrainCapture implement scheduler interface 185 func (m *mockScheduler) DrainCapture(target model.CaptureID) (int, error) { 186 return 0, nil 187 } 188 189 // Close closes the scheduler and releases resources. 190 func (m *mockScheduler) Close(ctx context.Context) {} 191 192 func createChangefeed4Test(globalVars *vars.GlobalVars, 193 changefeedInfo *model.ChangeFeedInfo, 194 t *testing.T, 195 ) ( 196 *changefeed, map[model.CaptureID]*model.CaptureInfo, *orchestrator.ReactorStateTester, *orchestrator.ChangefeedReactorState, 197 ) { 198 up := upstream.NewUpstream4Test(&gc.MockPDClient{ 199 UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { 200 return safePoint - 1, nil 201 }, 202 }) 203 204 state := orchestrator.NewChangefeedReactorState(etcd.DefaultCDCClusterID, 205 model.DefaultChangeFeedID(changefeedInfo.ID)) 206 tester := orchestrator.NewReactorStateTester(t, state, nil) 207 state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 208 require.Nil(t, info) 209 info = changefeedInfo 210 return info, true, nil 211 }) 212 tester.MustApplyPatches() 213 cf := newChangefeed4Test(model.DefaultChangeFeedID(changefeedInfo.ID), 214 state.Info, state.Status, NewFeedStateManager(up, state), up, 215 // new ddl puller 216 func(ctx context.Context, 217 up *upstream.Upstream, 218 startTs uint64, 219 changefeed model.ChangeFeedID, 220 schemaStorage entry.SchemaStorage, 221 filter filter.Filter, 222 ) puller.DDLPuller { 223 return &mockDDLPuller{resolvedTs: startTs - 1, schemaStorage: schemaStorage} 224 }, 225 // new ddl ddlSink 226 func(_ model.ChangeFeedID, _ *model.ChangeFeedInfo, _ func(error), _ func(error)) DDLSink { 227 return &mockDDLSink{ 228 resetDDLDone: true, 229 recordDDLHistory: false, 230 } 231 }, 232 // new scheduler 233 func( 234 ctx context.Context, id model.ChangeFeedID, up *upstream.Upstream, epoch uint64, 235 cfg *config.SchedulerConfig, redoMetaManager credo.MetaManager, 236 globalVars *vars.GlobalVars, 237 ) (scheduler.Scheduler, error) { 238 return &mockScheduler{}, nil 239 }, 240 // new downstream observer 241 func( 242 ctx context.Context, id model.ChangeFeedID, 243 sinkURIStr string, replCfg *config.ReplicaConfig, 244 opts ...observer.NewObserverOption, 245 ) (observer.Observer, error) { 246 return observer.NewDummyObserver(), nil 247 }, 248 globalVars, 249 ) 250 251 cf.upstream = up 252 253 tester.MustUpdate(fmt.Sprintf("%s/capture/%s", 254 etcd.DefaultClusterAndMetaPrefix, globalVars.CaptureInfo.ID), 255 []byte(`{"id":"`+globalVars.CaptureInfo.ID+`","address":"127.0.0.1:8300"}`)) 256 tester.MustApplyPatches() 257 captures := map[model.CaptureID]*model.CaptureInfo{globalVars.CaptureInfo.ID: globalVars.CaptureInfo} 258 return cf, captures, tester, state 259 } 260 261 func TestPreCheck(t *testing.T) { 262 globalvars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test() 263 _, captures, tester, state := createChangefeed4Test(globalvars, changefeedVars, t) 264 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 265 preflightCheck(state, captures) 266 tester.MustApplyPatches() 267 require.NotNil(t, state.Status) 268 269 // test clean the meta data of offline capture 270 offlineCaputreID := "offline-capture" 271 state.PatchTaskPosition(offlineCaputreID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 272 return new(model.TaskPosition), true, nil 273 }) 274 tester.MustApplyPatches() 275 276 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 277 require.False(t, preflightCheck(state, captures)) 278 tester.MustApplyPatches() 279 require.NotNil(t, state.Status) 280 require.NotContains(t, state.TaskPositions, offlineCaputreID) 281 } 282 283 func TestInitialize(t *testing.T) { 284 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 285 ctx := context.Background() 286 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 287 defer cf.Close(ctx) 288 // pre check 289 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 290 require.False(t, preflightCheck(state, captures)) 291 tester.MustApplyPatches() 292 293 // initialize 294 globalvars.EtcdClient = &etcd.CDCEtcdClientImpl{} 295 cf.Tick(ctx, state.Info, state.Status, captures) 296 tester.MustApplyPatches() 297 require.Equal(t, state.Status.CheckpointTs, changefeedInfo.StartTs) 298 } 299 300 func TestChangefeedHandleError(t *testing.T) { 301 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 302 ctx := context.Background() 303 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 304 defer cf.Close(ctx) 305 // pre check 306 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 307 require.False(t, preflightCheck(state, captures)) 308 tester.MustApplyPatches() 309 310 // initialize 311 cf.Tick(ctx, state.Info, state.Status, captures) 312 tester.MustApplyPatches() 313 314 cf.errCh <- errors.New("fake error") 315 // handle error 316 cf.Tick(ctx, state.Info, state.Status, captures) 317 tester.MustApplyPatches() 318 require.Equal(t, state.Status.CheckpointTs, changefeedInfo.StartTs) 319 require.Equal(t, state.Info.Error.Message, "fake error") 320 } 321 322 func TestExecDDL(t *testing.T) { 323 helper := entry.NewSchemaTestHelper(t) 324 defer helper.Close() 325 // Creates a table, which will be deleted at the start-ts of the changefeed. 326 // It is expected that the changefeed DOES NOT replicate this table. 327 helper.DDL2Job("create database test0") 328 job := helper.DDL2Job("create table test0.table0(id int primary key)") 329 startTs := job.BinlogInfo.FinishedTS + 1000 330 331 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 332 changefeedInfo.StartTs = startTs 333 ctx := context.Background() 334 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 335 cf.upstream.KVStorage = helper.Storage() 336 defer cf.Close(ctx) 337 tickTwoTime := func() { 338 checkpointTs, minTableBarrierTs := cf.Tick(ctx, state.Info, state.Status, captures) 339 updateStatus(state, checkpointTs, minTableBarrierTs) 340 tester.MustApplyPatches() 341 checkpointTs, minTableBarrierTs = cf.Tick(ctx, state.Info, state.Status, captures) 342 updateStatus(state, checkpointTs, minTableBarrierTs) 343 tester.MustApplyPatches() 344 } 345 // pre check and initialize 346 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 347 require.False(t, preflightCheck(state, captures)) 348 tester.MustApplyPatches() 349 tickTwoTime() 350 tableIDs, err := cf.schema.AllPhysicalTables(ctx, startTs-1) 351 require.Nil(t, err) 352 require.Len(t, tableIDs, 1) 353 354 job = helper.DDL2Job("drop table test0.table0") 355 // ddl puller resolved ts grow up 356 mockDDLPuller := cf.ddlManager.ddlPuller.(*mockDDLPuller) 357 mockDDLPuller.resolvedTs = startTs 358 mockDDLSink := cf.ddlManager.ddlSink.(*mockDDLSink) 359 job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs 360 mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) 361 // three tick to make sure all barrier set in initialize is handled 362 tickTwoTime() 363 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 364 // The ephemeral table should have left no trace in the schema cache 365 tableIDs, err = cf.schema.AllPhysicalTables(ctx, mockDDLPuller.resolvedTs) 366 require.Nil(t, err) 367 require.Len(t, tableIDs, 0) 368 369 // executing the ddl finished 370 mockDDLSink.ddlDone = true 371 mockDDLPuller.resolvedTs += 1000 372 tickTwoTime() 373 require.Equal(t, mockDDLPuller.resolvedTs, state.Status.CheckpointTs) 374 375 // handle create database 376 job = helper.DDL2Job("create database test1") 377 mockDDLPuller.resolvedTs += 1000 378 job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs 379 mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) 380 tickTwoTime() 381 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 382 require.Equal(t, "create database test1", mockDDLSink.ddlExecuting.Query) 383 384 // executing the ddl finished 385 mockDDLSink.ddlDone = true 386 mockDDLPuller.resolvedTs += 1000 387 tickTwoTime() 388 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 389 390 // handle create table 391 job = helper.DDL2Job("create table test1.test1(id int primary key)") 392 mockDDLPuller.resolvedTs += 1000 393 job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs 394 mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) 395 tickTwoTime() 396 397 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 398 require.Equal(t, "create table test1.test1(id int primary key)", mockDDLSink.ddlExecuting.Query) 399 400 // executing the ddl finished 401 mockDDLSink.ddlDone = true 402 mockDDLPuller.resolvedTs += 1000 403 tickTwoTime() 404 require.Contains(t, cf.scheduler.(*mockScheduler).currentTables, job.TableID) 405 } 406 407 func TestEmitCheckpointTs(t *testing.T) { 408 helper := entry.NewSchemaTestHelper(t) 409 defer helper.Close() 410 // Creates a table, which will be deleted at the start-ts of the changefeed. 411 // It is expected that the changefeed DOES NOT replicate this table. 412 helper.DDL2Job("create database test0") 413 job := helper.DDL2Job("create table test0.table0(id int primary key)") 414 startTs := job.BinlogInfo.FinishedTS + 1000 415 416 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 417 changefeedInfo.StartTs = startTs 418 ctx := context.Background() 419 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 420 cf.upstream.KVStorage = helper.Storage() 421 422 defer cf.Close(ctx) 423 tickThreeTime := func() { 424 checkpointTs, minTableBarrierTs := cf.Tick(ctx, state.Info, state.Status, captures) 425 updateStatus(state, checkpointTs, minTableBarrierTs) 426 tester.MustApplyPatches() 427 checkpointTs, minTableBarrierTs = cf.Tick(ctx, state.Info, state.Status, captures) 428 updateStatus(state, checkpointTs, minTableBarrierTs) 429 tester.MustApplyPatches() 430 checkpointTs, minTableBarrierTs = cf.Tick(ctx, state.Info, state.Status, captures) 431 updateStatus(state, checkpointTs, minTableBarrierTs) 432 tester.MustApplyPatches() 433 } 434 // pre check and initialize 435 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 436 require.False(t, preflightCheck(state, captures)) 437 tester.MustApplyPatches() 438 tickThreeTime() 439 mockDDLSink := cf.ddlManager.ddlSink.(*mockDDLSink) 440 441 tables, err := cf.ddlManager.allTables(ctx) 442 require.Nil(t, err) 443 444 require.Len(t, tables, 1) 445 ts, names := mockDDLSink.getCheckpointTsAndTableNames() 446 require.Equal(t, ts, startTs) 447 require.Len(t, names, 1) 448 449 job = helper.DDL2Job("drop table test0.table0") 450 // ddl puller resolved ts grow up 451 mockDDLPuller := cf.ddlManager.ddlPuller.(*mockDDLPuller) 452 mockDDLPuller.resolvedTs = startTs + 1000 453 cf.ddlManager.schema.AdvanceResolvedTs(mockDDLPuller.resolvedTs) 454 state.Status.CheckpointTs = mockDDLPuller.resolvedTs 455 job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs 456 mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) 457 // three tick to make sure all barrier set in initialize is handled 458 tickThreeTime() 459 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 460 tables, err = cf.ddlManager.allTables(ctx) 461 require.Nil(t, err) 462 // The ephemeral table should only be deleted after the ddl is executed. 463 require.Len(t, tables, 1) 464 // We can't use the new schema because the ddl hasn't been executed yet. 465 ts, names = mockDDLSink.getCheckpointTsAndTableNames() 466 require.Equal(t, ts, mockDDLPuller.resolvedTs) 467 require.Len(t, names, 1) 468 469 // executing the ddl finished 470 mockDDLSink.ddlDone = true 471 mockDDLPuller.resolvedTs += 2000 472 tickThreeTime() 473 require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) 474 ts, names = mockDDLSink.getCheckpointTsAndTableNames() 475 require.Equal(t, ts, mockDDLPuller.resolvedTs) 476 require.Len(t, names, 0) 477 } 478 479 func TestSyncPoint(t *testing.T) { 480 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 481 ctx := context.Background() 482 changefeedInfo.Config.EnableSyncPoint = util.AddressOf(true) 483 changefeedInfo.Config.SyncPointInterval = util.AddressOf(1 * time.Second) 484 // SyncPoint option is only available for MySQL compatible database. 485 changefeedInfo.SinkURI = "mysql://" 486 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 487 defer cf.Close(ctx) 488 489 // pre check 490 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 491 require.False(t, preflightCheck(state, captures)) 492 tester.MustApplyPatches() 493 494 // initialize 495 cf.Tick(ctx, state.Info, state.Status, captures) 496 tester.MustApplyPatches() 497 498 mockDDLPuller := cf.ddlManager.ddlPuller.(*mockDDLPuller) 499 mockDDLSink := cf.ddlManager.ddlSink.(*mockDDLSink) 500 // add 5s to resolvedTs 501 mockDDLPuller.resolvedTs = oracle.GoTimeToTS(oracle.GetTimeFromTS(mockDDLPuller.resolvedTs).Add(5 * time.Second)) 502 // tick 20 times 503 for i := 0; i <= 20; i++ { 504 checkpointTs, minTableBarrierTs := cf.Tick(ctx, state.Info, state.Status, captures) 505 updateStatus(state, checkpointTs, minTableBarrierTs) 506 tester.MustApplyPatches() 507 } 508 for i := 1; i < len(mockDDLSink.syncPointHis); i++ { 509 // check the time interval between adjacent sync points is less or equal than one second 510 require.LessOrEqual(t, mockDDLSink.syncPointHis[i]-mockDDLSink.syncPointHis[i-1], uint64(1000<<18)) 511 } 512 require.GreaterOrEqual(t, len(mockDDLSink.syncPointHis), 5) 513 } 514 515 func TestFinished(t *testing.T) { 516 globalvars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 517 ctx := context.Background() 518 changefeedInfo.TargetTs = changefeedInfo.StartTs + 1000 519 cf, captures, tester, state := createChangefeed4Test(globalvars, changefeedInfo, t) 520 defer cf.Close(ctx) 521 522 // pre check 523 state.CheckCaptureAlive(globalvars.CaptureInfo.ID) 524 require.False(t, preflightCheck(state, captures)) 525 tester.MustApplyPatches() 526 527 // initialize 528 cf.Tick(ctx, state.Info, state.Status, captures) 529 tester.MustApplyPatches() 530 531 mockDDLPuller := cf.ddlManager.ddlPuller.(*mockDDLPuller) 532 mockDDLPuller.resolvedTs += 2000 533 // tick many times to make sure the change feed is stopped 534 for i := 0; i <= 10; i++ { 535 checkpointTs, minTableBarrierTs := cf.Tick(ctx, state.Info, state.Status, captures) 536 updateStatus(state, checkpointTs, minTableBarrierTs) 537 tester.MustApplyPatches() 538 } 539 fmt.Println("checkpoint ts", state.Status.CheckpointTs) 540 fmt.Println("target ts", state.Info.TargetTs) 541 require.Equal(t, state.Status.CheckpointTs, state.Info.TargetTs) 542 require.Equal(t, state.Info.State, model.StateFinished) 543 } 544 545 func TestRemoveChangefeed(t *testing.T) { 546 globalVars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 547 ctx := context.Background() 548 ctx, cancel := context.WithCancel(ctx) 549 info := changefeedInfo 550 dir := t.TempDir() 551 info.SinkURI = "mysql://" 552 info.Config.Consistent = &config.ConsistentConfig{ 553 Level: "eventual", 554 Storage: filepath.Join("nfs://", dir), 555 FlushIntervalInMs: redo.DefaultFlushIntervalInMs, 556 MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, 557 EncodingWorkerNum: redo.DefaultEncodingWorkerNum, 558 FlushWorkerNum: redo.DefaultFlushWorkerNum, 559 } 560 testChangefeedReleaseResource(ctx, t, globalVars, changefeedInfo, cancel, dir, true /*expectedInitialized*/) 561 } 562 563 func TestRemovePausedChangefeed(t *testing.T) { 564 globalVars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 565 ctx := context.Background() 566 ctx, cancel := context.WithCancel(ctx) 567 info := changefeedInfo 568 info.State = model.StateStopped 569 dir := t.TempDir() 570 // Field `Consistent` is valid only when the downstream 571 // is MySQL compatible Schema 572 info.SinkURI = "mysql://" 573 info.Config.Consistent = &config.ConsistentConfig{ 574 Level: "eventual", 575 Storage: filepath.Join("nfs://", dir), 576 FlushIntervalInMs: redo.DefaultFlushIntervalInMs, 577 } 578 testChangefeedReleaseResource(ctx, t, globalVars, changefeedInfo, cancel, dir, false /*expectedInitialized*/) 579 } 580 581 func testChangefeedReleaseResource( 582 ctx context.Context, 583 t *testing.T, 584 globalVars *vars.GlobalVars, 585 changefeedInfo *model.ChangeFeedInfo, 586 cancel context.CancelFunc, 587 redoLogDir string, 588 expectedInitialized bool, 589 ) { 590 var err error 591 cf, captures, tester, state := createChangefeed4Test(globalVars, changefeedInfo, t) 592 593 // pre check 594 state.CheckCaptureAlive(globalVars.CaptureInfo.ID) 595 require.False(t, preflightCheck(state, captures)) 596 tester.MustApplyPatches() 597 598 // initialize 599 cf.Tick(ctx, state.Info, state.Status, captures) 600 tester.MustApplyPatches() 601 require.Equal(t, cf.initialized.Load(), expectedInitialized) 602 603 // redo's metaManager:Run() will call preStart, which will clean up the redo log dir. 604 // Run() is another background goroutine called in tick() 605 // so it's not guaranteed to be started before the next tick() 606 // Thus, we need to wait for a while to make sure the redo log dir is cleaned up before remove changefeed 607 // Otherwise, it will delete the delete mark file after we remove changefeed, which will cause the test to fail 608 time.Sleep(5 * time.Second) 609 610 // remove changefeed from state manager by admin job 611 cf.feedStateManager.PushAdminJob(&model.AdminJob{ 612 CfID: cf.id, 613 Type: model.AdminRemove, 614 }) 615 cf.isReleased = false 616 // changefeed tick will release resources 617 cf.Tick(ctx, state.Info, state.Status, captures) 618 require.Nil(t, err) 619 cancel() 620 621 if state.Info.Config.Consistent.UseFileBackend { 622 // check redo log dir is deleted 623 _, err = os.Stat(redoLogDir) 624 require.True(t, os.IsNotExist(err)) 625 } else { 626 files, err := os.ReadDir(redoLogDir) 627 require.NoError(t, err) 628 require.Len(t, files, 1) // only delete mark 629 } 630 } 631 632 func TestBarrierAdvance(t *testing.T) { 633 for i := 0; i < 2; i++ { 634 globalVars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 635 ctx := context.Background() 636 if i == 1 { 637 changefeedInfo.Config.EnableSyncPoint = util.AddressOf(true) 638 changefeedInfo.Config.SyncPointInterval = util.AddressOf(100 * time.Second) 639 } 640 changefeedInfo.SinkURI = "mysql://" 641 642 cf, captures, tester, state := createChangefeed4Test(globalVars, changefeedInfo, t) 643 defer cf.Close(ctx) 644 645 // The changefeed load the info from etcd. 646 state.Status = &model.ChangeFeedStatus{ 647 CheckpointTs: state.Info.StartTs, 648 MinTableBarrierTs: state.Info.StartTs + 5, 649 } 650 // Do the preflightCheck and initialize the changefeed. 651 cf.Tick(ctx, state.Info, state.Status, captures) 652 tester.MustApplyPatches() 653 if i == 1 { 654 cf.ddlManager.ddlResolvedTs += 10 655 } 656 _, barrier, err := cf.ddlManager.tick(ctx, state.Status.CheckpointTs) 657 658 require.Nil(t, err) 659 660 err = cf.handleBarrier(ctx, state.Info, state.Status, barrier) 661 require.Nil(t, err) 662 663 if i == 0 { 664 require.Equal(t, state.Info.StartTs, barrier.GlobalBarrierTs) 665 } 666 667 // sync-point is enabled, sync point barrier is ticked 668 if i == 1 { 669 require.Equal(t, state.Info.StartTs+10, barrier.GlobalBarrierTs) 670 } 671 672 // Suppose tableCheckpoint has been advanced. 673 state.Status.CheckpointTs += 10 674 675 // Need more 1 tick to advance barrier if sync-point is enabled. 676 if i == 1 { 677 err = cf.handleBarrier(ctx, state.Info, state.Status, barrier) 678 require.Nil(t, err) 679 require.Equal(t, state.Info.StartTs+10, barrier.GlobalBarrierTs) 680 681 // Then the last tick barrier must be advanced correctly. 682 cf.ddlManager.ddlResolvedTs += 1000000000000 683 _, barrier, err = cf.ddlManager.tick(ctx, state.Status.CheckpointTs+10) 684 require.Nil(t, err) 685 err = cf.handleBarrier(ctx, state.Info, state.Status, barrier) 686 687 nextSyncPointTs := oracle.GoTimeToTS( 688 oracle.GetTimeFromTS(state.Status.CheckpointTs + 10). 689 Add(util.GetOrZero(changefeedInfo.Config.SyncPointInterval)), 690 ) 691 692 require.Nil(t, err) 693 require.Equal(t, nextSyncPointTs, barrier.GlobalBarrierTs) 694 require.Less(t, state.Status.CheckpointTs+10, barrier.GlobalBarrierTs) 695 require.Less(t, barrier.GlobalBarrierTs, cf.ddlManager.ddlResolvedTs) 696 } 697 } 698 }