github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/puller/ddl_puller_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 puller 15 16 import ( 17 "context" 18 "encoding/json" 19 "fmt" 20 "sync" 21 "testing" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/log" 25 timodel "github.com/pingcap/tidb/pkg/parser/model" 26 "github.com/pingcap/tidb/pkg/sessionctx/variable" 27 "github.com/pingcap/tidb/pkg/util/codec" 28 "github.com/pingcap/tiflow/cdc/entry" 29 "github.com/pingcap/tiflow/cdc/model" 30 "github.com/pingcap/tiflow/cdc/processor/tablepb" 31 "github.com/pingcap/tiflow/cdc/puller/memorysorter" 32 "github.com/pingcap/tiflow/cdc/vars" 33 "github.com/pingcap/tiflow/pkg/config" 34 "github.com/pingcap/tiflow/pkg/filter" 35 "github.com/pingcap/tiflow/pkg/retry" 36 "github.com/pingcap/tiflow/pkg/upstream" 37 "github.com/pingcap/tiflow/pkg/util" 38 "github.com/stretchr/testify/require" 39 "go.uber.org/zap" 40 "go.uber.org/zap/zaptest/observer" 41 ) 42 43 func jonToRawKVEntry(t *testing.T, job *timodel.Job) *model.RawKVEntry { 44 b, err := json.Marshal(job) 45 require.Nil(t, err) 46 ek := []byte("m") 47 ek = codec.EncodeBytes(ek, []byte("DDLJobList")) 48 ek = codec.EncodeUint(ek, uint64('l')) 49 ek = codec.EncodeInt(ek, 1) 50 return &model.RawKVEntry{ 51 OpType: model.OpTypePut, 52 Key: ek, 53 Value: b, 54 StartTs: job.StartTS, 55 CRTs: job.BinlogInfo.FinishedTS, 56 } 57 } 58 59 func tsToRawKVEntry(_ *testing.T, ts model.Ts) *model.RawKVEntry { 60 return &model.RawKVEntry{ 61 OpType: model.OpTypeResolved, 62 CRTs: ts, 63 StartTs: ts, 64 } 65 } 66 67 func inputDDL(t *testing.T, puller *ddlJobPullerImpl, job *timodel.Job) { 68 rawJob := jonToRawKVEntry(t, job) 69 puller.Input(context.Background(), rawJob, []tablepb.Span{}) 70 } 71 72 func inputTs(t *testing.T, puller *ddlJobPullerImpl, ts model.Ts) { 73 rawTs := tsToRawKVEntry(t, ts) 74 puller.Input(context.Background(), rawTs, []tablepb.Span{}) 75 } 76 77 func waitResolvedTs(t *testing.T, p DDLJobPuller, targetTs model.Ts) { 78 err := retry.Do(context.Background(), func() error { 79 if p.(*ddlJobPullerImpl).getResolvedTs() < targetTs { 80 return fmt.Errorf("resolvedTs %d < targetTs %d", p.(*ddlJobPullerImpl).getResolvedTs(), targetTs) 81 } 82 return nil 83 }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(200)) 84 require.Nil(t, err) 85 } 86 87 func newMockDDLJobPuller( 88 t *testing.T, 89 needSchemaStorage bool, 90 ) (DDLJobPuller, *entry.SchemaTestHelper) { 91 res := &ddlJobPullerImpl{ 92 outputCh: make( 93 chan *model.DDLJobEntry, 94 defaultPullerOutputChanSize), 95 } 96 res.sorter = memorysorter.NewEntrySorter(model.ChangeFeedID4Test("puller", "test")) 97 98 var helper *entry.SchemaTestHelper 99 if needSchemaStorage { 100 helper = entry.NewSchemaTestHelper(t) 101 kvStorage := helper.Storage() 102 ts := helper.GetCurrentMeta().StartTS 103 f, err := filter.NewFilter(config.GetDefaultReplicaConfig(), "") 104 require.Nil(t, err) 105 schemaStorage, err := entry.NewSchemaStorage( 106 kvStorage, 107 ts, 108 false, 109 model.DefaultChangeFeedID("test"), 110 util.RoleTester, 111 f) 112 require.Nil(t, err) 113 res.schemaStorage = schemaStorage 114 res.kvStorage = kvStorage 115 } 116 return res, helper 117 } 118 119 func TestHandleRenameTable(t *testing.T) { 120 ddlJobPuller, helper := newMockDDLJobPuller(t, true) 121 defer helper.Close() 122 123 startTs := uint64(10) 124 ddlJobPullerImpl := ddlJobPuller.(*ddlJobPullerImpl) 125 ddlJobPullerImpl.setResolvedTs(startTs) 126 127 cfg := config.GetDefaultReplicaConfig() 128 cfg.Filter.Rules = []string{ 129 "test1.t1", 130 "test1.t2", 131 "test1.t4", 132 "test1.t66", 133 "test1.t99", 134 "test1.t100", 135 "test1.t20230808", 136 "test1.t202308081", 137 "test1.t202308082", 138 139 "test2.t4", 140 141 "Test3.t1", 142 "Test3.t2", 143 } 144 f, err := filter.NewFilter(cfg, "") 145 require.NoError(t, err) 146 ddlJobPullerImpl.filter = f 147 148 ctx, cancel := context.WithCancel(context.Background()) 149 defer cancel() 150 151 go ddlJobPuller.Run(ctx) 152 go func() { 153 select { 154 case <-ctx.Done(): 155 return 156 case <-ddlJobPuller.Output(): 157 } 158 }() 159 160 // table t3, t5 not found in snapshot, skip it. 161 // only table t1 remain. 162 { 163 remainTables := make([]int64, 1) 164 job := helper.DDL2Job("create database test1") 165 inputDDL(t, ddlJobPullerImpl, job) 166 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 167 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 168 169 job = helper.DDL2Job("create table test1.t1(id int primary key)") 170 remainTables[0] = job.TableID 171 inputDDL(t, ddlJobPullerImpl, job) 172 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 173 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 174 175 job = helper.DDL2Job("create table test1.t2(id int primary key)") 176 inputDDL(t, ddlJobPullerImpl, job) 177 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 178 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 179 180 job = helper.DDL2Job("create table test1.t3(id int primary key)") 181 inputDDL(t, ddlJobPullerImpl, job) 182 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 183 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 184 185 job = helper.DDL2Job("create table test1.t5(id int primary key)") 186 inputDDL(t, ddlJobPullerImpl, job) 187 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 188 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 189 190 job = helper.DDL2Job("create database ignore1") 191 inputDDL(t, ddlJobPullerImpl, job) 192 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 193 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 194 195 job = helper.DDL2Job("create table ignore1.a(id int primary key)") 196 inputDDL(t, ddlJobPullerImpl, job) 197 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 198 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 199 200 job = helper.DDL2Job("rename table test1.t1 to test1.t11, test1.t3 to test1.t33, test1.t5 to test1.t55, ignore1.a to ignore1.b") 201 202 skip, err := ddlJobPullerImpl.handleRenameTables(job) 203 require.NoError(t, err) 204 require.False(t, skip) 205 require.Len(t, job.BinlogInfo.MultipleTableInfos, 1) 206 require.Equal(t, remainTables[0], job.BinlogInfo.MultipleTableInfos[0].ID) 207 } 208 209 { 210 _ = helper.DDL2Job("create table test1.t6(id int primary key)") 211 job := helper.DDL2Job("rename table test1.t2 to test1.t22, test1.t6 to test1.t66") 212 skip, err := ddlJobPullerImpl.handleRenameTables(job) 213 require.Error(t, err) 214 require.True(t, skip) 215 require.Contains(t, err.Error(), fmt.Sprintf("table's old name is not in filter rule, and its new name in filter rule "+ 216 "table id '%d', ddl query: [%s], it's an unexpected behavior, "+ 217 "if you want to replicate this table, please add its old name to filter rule.", job.TableID, job.Query)) 218 } 219 220 // all tables are filtered out 221 { 222 job := helper.DDL2Job("create database test2") 223 inputDDL(t, ddlJobPullerImpl, job) 224 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 225 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 226 227 job = helper.DDL2Job("create table test2.t1(id int primary key)") 228 inputDDL(t, ddlJobPullerImpl, job) 229 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 230 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 231 232 job = helper.DDL2Job("create table test2.t2(id int primary key)") 233 inputDDL(t, ddlJobPullerImpl, job) 234 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 235 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 236 237 job = helper.DDL2Job("create table test2.t3(id int primary key)") 238 inputDDL(t, ddlJobPullerImpl, job) 239 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 240 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 241 242 job = helper.DDL2Job("rename table test2.t1 to test2.t11, test2.t2 to test2.t22, test2.t3 to test2.t33") 243 skip, err := ddlJobPullerImpl.handleRenameTables(job) 244 require.NoError(t, err) 245 require.True(t, skip) 246 } 247 248 // test uppercase db name 249 { 250 job := helper.DDL2Job("create database Test3") 251 inputDDL(t, ddlJobPullerImpl, job) 252 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 253 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 254 255 job = helper.DDL2Job("create table Test3.t1(id int primary key)") 256 inputDDL(t, ddlJobPullerImpl, job) 257 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 258 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 259 260 // skip this table 261 job = helper.DDL2Job("create table Test3.t2(id int primary key)") 262 inputDDL(t, ddlJobPullerImpl, job) 263 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 264 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 265 266 job = helper.DDL2Job("rename table Test3.t1 to Test3.t11, Test3.t2 to Test3.t22") 267 skip, err := ddlJobPullerImpl.handleRenameTables(job) 268 require.NoError(t, err) 269 require.False(t, skip) 270 require.Equal(t, 2, len(job.BinlogInfo.MultipleTableInfos)) 271 require.Equal(t, "t11", job.BinlogInfo.MultipleTableInfos[0].Name.O) 272 require.Equal(t, "t22", job.BinlogInfo.MultipleTableInfos[1].Name.O) 273 } 274 275 // test rename table 276 { 277 job := helper.DDL2Job("create table test1.t99 (id int primary key)") 278 inputDDL(t, ddlJobPullerImpl, job) 279 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 280 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 281 282 // this ddl should be skipped 283 job = helper.DDL2Job("create table test1.t1000 (id int primary key)") 284 inputDDL(t, ddlJobPullerImpl, job) 285 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 286 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 287 288 // this ddl should be skipped 289 job = helper.DDL2Job("create table test1.t888 (id int primary key)") 290 inputDDL(t, ddlJobPullerImpl, job) 291 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 292 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 293 294 job = helper.DDL2Job("create table test1.t20230808 (id int primary key)") 295 inputDDL(t, ddlJobPullerImpl, job) 296 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 297 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 298 299 job = helper.DDL2Job("create table test1.t202308081 (id int primary key)") 300 inputDDL(t, ddlJobPullerImpl, job) 301 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 302 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 303 304 job = helper.DDL2Job("create table test1.t202308082 (id int primary key)") 305 inputDDL(t, ddlJobPullerImpl, job) 306 inputTs(t, ddlJobPullerImpl, job.BinlogInfo.FinishedTS+1) 307 waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) 308 // since test1.99 in filter rule, we replicate it 309 job = helper.DDL2Job("rename table test1.t99 to test1.t999") 310 skip, err := ddlJobPullerImpl.handleJob(job) 311 require.NoError(t, err) 312 require.False(t, skip) 313 314 // since test1.t100 is in filter rule, replicate it 315 job = helper.DDL2Job("rename table test1.t1000 to test1.t100") 316 _, err = ddlJobPullerImpl.handleJob(job) 317 require.Error(t, err) 318 require.Contains(t, err.Error(), fmt.Sprintf("table's old name is not in filter rule, and its new name in filter rule "+ 319 "table id '%d', ddl query: [%s], it's an unexpected behavior, "+ 320 "if you want to replicate this table, please add its old name to filter rule.", job.TableID, job.Query)) 321 322 // since test1.t888 and test1.t777 are not in filter rule, skip it 323 job = helper.DDL2Job("rename table test1.t888 to test1.t777") 324 skip, err = ddlJobPullerImpl.handleJob(job) 325 require.NoError(t, err) 326 require.True(t, skip) 327 328 // since test1.t20230808 is in filter rule, replicate it 329 // ref: https://github.com/pingcap/tiflow/issues/9488 330 job = helper.DDL2Job("rename table test1.t20230808 to ignore1.ignore") 331 skip, err = ddlJobPullerImpl.handleJob(job) 332 require.NoError(t, err) 333 require.False(t, skip) 334 335 // FIXME(dongmen): since test1.t202308081 and test1.t202308082 are in filter rule, it should be replicated 336 // but now it will throw an error since schema ignore1 are not in schemaStorage 337 // ref: https://github.com/pingcap/tiflow/issues/9488 338 job = helper.DDL2Job("rename table test1.t202308081 to ignore1.ignore1, test1.t202308082 to ignore1.dongmen") 339 _, err = ddlJobPullerImpl.handleJob(job) 340 require.NotNil(t, err) 341 require.Contains(t, err.Error(), "ErrSnapshotSchemaNotFound") 342 } 343 } 344 345 func TestHandleJob(t *testing.T) { 346 ddlJobPuller, helper := newMockDDLJobPuller(t, true) 347 defer helper.Close() 348 startTs := uint64(10) 349 ddlJobPullerImpl := ddlJobPuller.(*ddlJobPullerImpl) 350 ddlJobPullerImpl.setResolvedTs(startTs) 351 cfg := config.GetDefaultReplicaConfig() 352 cfg.Filter.Rules = []string{ 353 "test1.t1", 354 "test1.t2", 355 "test1.testStartTs", 356 } 357 // test start ts filter 358 cfg.Filter.IgnoreTxnStartTs = []uint64{1} 359 // test event filter 360 cfg.Filter.EventFilters = []*config.EventFilterRule{ 361 { 362 Matcher: []string{"test1.*"}, 363 IgnoreSQL: []string{"alter table test1.t1 add column c1 int"}, 364 }, 365 } 366 367 f, err := filter.NewFilter(cfg, "") 368 require.NoError(t, err) 369 ddlJobPullerImpl.filter = f 370 371 // test create database 372 { 373 job := helper.DDL2Job("create database test1") 374 skip, err := ddlJobPullerImpl.handleJob(job) 375 require.NoError(t, err) 376 require.False(t, skip) 377 378 job = helper.DDL2Job("create database test2") 379 skip, err = ddlJobPullerImpl.handleJob(job) 380 require.NoError(t, err) 381 require.True(t, skip) 382 383 job = helper.DDL2Job("create database test3") 384 skip, err = ddlJobPullerImpl.handleJob(job) 385 require.NoError(t, err) 386 require.True(t, skip) 387 } 388 389 // test drop databases 390 { 391 job := helper.DDL2Job("drop database test2") 392 skip, err := ddlJobPullerImpl.handleJob(job) 393 require.NoError(t, err) 394 require.True(t, skip) 395 } 396 397 // test create table 398 { 399 job := helper.DDL2Job("create table test1.t1(id int primary key) partition by range(id) (partition p0 values less than (10))") 400 skip, err := ddlJobPullerImpl.handleJob(job) 401 require.NoError(t, err) 402 require.False(t, skip) 403 404 job = helper.DDL2Job("alter table test1.t1 add column c1 int") 405 skip, err = ddlJobPullerImpl.handleJob(job) 406 require.NoError(t, err) 407 require.False(t, skip) 408 409 job = helper.DDL2Job("create table test1.testStartTs(id int primary key)") 410 skip, err = ddlJobPullerImpl.handleJob(job) 411 require.NoError(t, err) 412 require.False(t, skip) 413 414 job = helper.DDL2Job("alter table test1.testStartTs add column c1 int") 415 job.StartTS = 1 416 skip, err = ddlJobPullerImpl.handleJob(job) 417 require.NoError(t, err) 418 require.False(t, skip) 419 420 job = helper.DDL2Job("create table test1.t2(id int primary key)") 421 skip, err = ddlJobPullerImpl.handleJob(job) 422 require.NoError(t, err) 423 require.False(t, skip) 424 425 job = helper.DDL2Job("create table test1.t3(id int primary key)") 426 skip, err = ddlJobPullerImpl.handleJob(job) 427 require.NoError(t, err) 428 require.True(t, skip) 429 430 job = helper.DDL2Job("create table test1.t4(id int primary key) partition by range(id) (partition p0 values less than (10))") 431 skip, err = ddlJobPullerImpl.handleJob(job) 432 require.NoError(t, err) 433 require.True(t, skip) 434 435 // make sure no schema not found error 436 job = helper.DDL2Job("create table test3.t1(id int primary key) partition by range(id) (partition p0 values less than (10))") 437 skip, err = ddlJobPullerImpl.handleJob(job) 438 require.NoError(t, err) 439 require.True(t, skip) 440 } 441 442 // test drop table 443 { 444 job := helper.DDL2Job("drop table test1.t2") 445 skip, err := ddlJobPullerImpl.handleJob(job) 446 require.NoError(t, err) 447 require.False(t, skip) 448 449 job = helper.DDL2Job("drop table test1.t3") 450 skip, err = ddlJobPullerImpl.handleJob(job) 451 require.NoError(t, err) 452 require.True(t, skip) 453 } 454 455 // test add column and drop column 456 { 457 job := helper.DDL2Job("alter table test1.t1 add column age int") 458 skip, err := ddlJobPullerImpl.handleJob(job) 459 require.NoError(t, err) 460 require.False(t, skip) 461 462 job = helper.DDL2Job("alter table test1.t4 add column age int") 463 skip, err = ddlJobPullerImpl.handleJob(job) 464 require.NoError(t, err) 465 require.True(t, skip) 466 } 467 468 // test add index and drop index 469 { 470 job := helper.DDL2Job("alter table test1.t1 add index idx_age(age)") 471 skip, err := ddlJobPullerImpl.handleJob(job) 472 require.NoError(t, err) 473 require.False(t, skip) 474 475 job = helper.DDL2Job("alter table test1.t4 add index idx_age(age)") 476 skip, err = ddlJobPullerImpl.handleJob(job) 477 require.NoError(t, err) 478 require.True(t, skip) 479 480 job = helper.DDL2Job("alter table test1.t1 drop index idx_age") 481 skip, err = ddlJobPullerImpl.handleJob(job) 482 require.NoError(t, err) 483 require.False(t, skip) 484 485 job = helper.DDL2Job("alter table test1.t4 drop index idx_age") 486 skip, err = ddlJobPullerImpl.handleJob(job) 487 require.NoError(t, err) 488 require.True(t, skip) 489 } 490 491 // test drop column 492 { 493 job := helper.DDL2Job("alter table test1.t1 drop column age") 494 skip, err := ddlJobPullerImpl.handleJob(job) 495 require.NoError(t, err) 496 require.False(t, skip) 497 498 job = helper.DDL2Job("alter table test1.t4 drop column age") 499 skip, err = ddlJobPullerImpl.handleJob(job) 500 require.NoError(t, err) 501 require.True(t, skip) 502 } 503 504 // test truncate table 505 { 506 job := helper.DDL2Job("truncate table test1.t1") 507 skip, err := ddlJobPullerImpl.handleJob(job) 508 require.NoError(t, err) 509 require.False(t, skip) 510 511 job = helper.DDL2Job("truncate table test1.t4") 512 skip, err = ddlJobPullerImpl.handleJob(job) 513 require.NoError(t, err) 514 require.True(t, skip) 515 } 516 517 // test add table partition 518 { 519 job := helper.DDL2Job("alter table test1.t1 add partition (partition p1 values less than (100))") 520 skip, err := ddlJobPullerImpl.handleJob(job) 521 require.NoError(t, err) 522 require.False(t, skip) 523 524 job = helper.DDL2Job("alter table test1.t4 add partition (partition p1 values less than (100))") 525 skip, err = ddlJobPullerImpl.handleJob(job) 526 require.NoError(t, err) 527 require.True(t, skip) 528 } 529 530 // test flashback cluster 531 { 532 // mock a flashback job 533 job := &timodel.Job{ 534 Type: timodel.ActionFlashbackCluster, 535 BinlogInfo: &timodel.HistoryInfo{}, 536 Args: []interface{}{ 537 998, 538 map[string]interface{}{}, 539 true, /* tidb_gc_enable */ 540 variable.On, /* tidb_enable_auto_analyze */ 541 variable.Off, /* tidb_super_read_only */ 542 0, /* totalRegions */ 543 0, /* startTS */ 544 0, /* commitTS */ 545 }, 546 } 547 skip, err := ddlJobPullerImpl.handleJob(job) 548 require.NoError(t, err) 549 require.True(t, skip) 550 } 551 } 552 553 func TestDDLPuller(t *testing.T) { 554 startTs := uint64(10) 555 556 _, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 557 ctx := context.Background() 558 up := upstream.NewUpstream4Test(nil) 559 f, err := filter.NewFilter(changefeedInfo.Config, "") 560 require.Nil(t, err) 561 schemaStorage, err := entry.NewSchemaStorage(nil, 562 startTs, 563 changefeedInfo.Config.ForceReplicate, 564 model.DefaultChangeFeedID(changefeedInfo.ID), 565 util.RoleTester, 566 f, 567 ) 568 require.Nil(t, err) 569 p := NewDDLPuller(ctx, up, startTs, model.DefaultChangeFeedID(changefeedInfo.ID), schemaStorage, f) 570 p.(*ddlPullerImpl).ddlJobPuller, _ = newMockDDLJobPuller(t, false) 571 ddlJobPullerImpl := p.(*ddlPullerImpl).ddlJobPuller.(*ddlJobPullerImpl) 572 ddlJobPullerImpl.setResolvedTs(startTs) 573 574 var wg sync.WaitGroup 575 wg.Add(1) 576 go func() { 577 defer wg.Done() 578 err := p.Run(ctx) 579 require.True(t, errors.ErrorEqual(err, context.Canceled)) 580 }() 581 defer wg.Wait() 582 defer p.Close() 583 584 resolvedTs, ddl := p.PopFrontDDL() 585 require.Equal(t, resolvedTs, startTs) 586 require.Nil(t, ddl) 587 588 // test send resolvedTs 589 inputTs(t, ddlJobPullerImpl, 15) 590 waitResolvedTsGrowing(t, p, 15) 591 592 // test send ddl job out of order 593 inputDDL(t, ddlJobPullerImpl, &timodel.Job{ 594 ID: 2, 595 Type: timodel.ActionCreateTable, 596 StartTS: 5, 597 State: timodel.JobStateDone, 598 BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 2, FinishedTS: 18}, 599 Query: "create table test.t1(id int primary key)", 600 }) 601 inputDDL(t, ddlJobPullerImpl, &timodel.Job{ 602 ID: 1, 603 Type: timodel.ActionCreateTable, 604 StartTS: 5, 605 State: timodel.JobStateDone, 606 BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 1, FinishedTS: 16}, 607 Query: "create table t2(id int primary key)", 608 }) 609 resolvedTs, ddl = p.PopFrontDDL() 610 require.Equal(t, resolvedTs, uint64(15)) 611 require.Nil(t, ddl) 612 613 inputTs(t, ddlJobPullerImpl, 20) 614 waitResolvedTsGrowing(t, p, 16) 615 resolvedTs, ddl = p.PopFrontDDL() 616 require.Equal(t, resolvedTs, uint64(16)) 617 require.Equal(t, ddl.ID, int64(1)) 618 619 // DDL could be processed with a delay, wait here for a pending DDL job is added 620 waitResolvedTsGrowing(t, p, 18) 621 resolvedTs, ddl = p.PopFrontDDL() 622 require.Equal(t, resolvedTs, uint64(18)) 623 require.Equal(t, ddl.ID, int64(2)) 624 625 // test add ddl job repeated 626 inputDDL(t, ddlJobPullerImpl, &timodel.Job{ 627 ID: 3, 628 Type: timodel.ActionCreateTable, 629 StartTS: 20, 630 State: timodel.JobStateDone, 631 BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 4, FinishedTS: 25}, 632 Query: "create table t3(id int primary key)", 633 }) 634 635 inputDDL(t, ddlJobPullerImpl, &timodel.Job{ 636 ID: 3, 637 Type: timodel.ActionCreateTable, 638 StartTS: 20, 639 State: timodel.JobStateDone, 640 BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 4, FinishedTS: 25}, 641 Query: "create table t3(id int primary key)", 642 }) 643 644 inputTs(t, ddlJobPullerImpl, 30) 645 waitResolvedTsGrowing(t, p, 25) 646 647 resolvedTs, ddl = p.PopFrontDDL() 648 require.Equal(t, resolvedTs, uint64(25)) 649 require.Equal(t, ddl.ID, int64(3)) 650 _, ddl = p.PopFrontDDL() 651 require.Nil(t, ddl) 652 653 waitResolvedTsGrowing(t, p, 30) 654 resolvedTs, ddl = p.PopFrontDDL() 655 require.Equal(t, resolvedTs, uint64(30)) 656 require.Nil(t, ddl) 657 658 inputDDL(t, ddlJobPullerImpl, &timodel.Job{ 659 ID: 5, 660 Type: timodel.ActionCreateTable, 661 StartTS: 20, 662 State: timodel.JobStateCancelled, 663 BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 6, FinishedTS: 36}, 664 Query: "create table t4(id int primary key)", 665 }) 666 667 inputTs(t, ddlJobPullerImpl, 40) 668 waitResolvedTsGrowing(t, p, 40) 669 resolvedTs, ddl = p.PopFrontDDL() 670 // no ddl should be received 671 require.Equal(t, resolvedTs, uint64(40)) 672 require.Nil(t, ddl) 673 } 674 675 func TestResolvedTsStuck(t *testing.T) { 676 // For observing the logs 677 zapcore, logs := observer.New(zap.WarnLevel) 678 conf := &log.Config{Level: "warn", File: log.FileLogConfig{}} 679 _, r, _ := log.InitLogger(conf) 680 logger := zap.New(zapcore) 681 restoreFn := log.ReplaceGlobals(logger, r) 682 defer restoreFn() 683 684 startTs := uint64(10) 685 686 _, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test() 687 ctx := context.Background() 688 up := upstream.NewUpstream4Test(nil) 689 f, err := filter.NewFilter(config.GetDefaultReplicaConfig(), "") 690 require.Nil(t, err) 691 schemaStorage, err := entry.NewSchemaStorage(nil, 692 startTs, 693 changefeedInfo.Config.ForceReplicate, 694 model.DefaultChangeFeedID(changefeedInfo.ID), 695 util.RoleTester, 696 f, 697 ) 698 require.Nil(t, err) 699 p := NewDDLPuller(ctx, up, startTs, model.DefaultChangeFeedID(changefeedInfo.ID), schemaStorage, f) 700 701 p.(*ddlPullerImpl).ddlJobPuller, _ = newMockDDLJobPuller(t, false) 702 ddlJobPullerImpl := p.(*ddlPullerImpl).ddlJobPuller.(*ddlJobPullerImpl) 703 ddlJobPullerImpl.setResolvedTs(startTs) 704 705 var wg sync.WaitGroup 706 wg.Add(1) 707 go func() { 708 defer wg.Done() 709 err := p.Run(ctx) 710 if errors.Cause(err) == context.Canceled { 711 err = nil 712 } 713 require.Nil(t, err) 714 }() 715 defer wg.Wait() 716 defer p.Close() 717 718 // test initialize state 719 resolvedTs, ddl := p.PopFrontDDL() 720 require.Equal(t, resolvedTs, startTs) 721 require.Nil(t, ddl) 722 723 inputTs(t, ddlJobPullerImpl, 30) 724 waitResolvedTsGrowing(t, p, 30) 725 require.Equal(t, 0, logs.Len()) 726 727 inputTs(t, ddlJobPullerImpl, 40) 728 waitResolvedTsGrowing(t, p, 40) 729 } 730 731 // waitResolvedTsGrowing can wait the first DDL reaches targetTs or if no pending 732 // DDL, DDL resolved ts reaches targetTs. 733 func waitResolvedTsGrowing(t *testing.T, p DDLPuller, targetTs model.Ts) { 734 err := retry.Do(context.Background(), func() error { 735 resolvedTs := p.ResolvedTs() 736 if resolvedTs < targetTs { 737 return errors.New("resolvedTs < targetTs") 738 } 739 return nil 740 }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(200)) 741 require.Nil(t, err) 742 } 743 744 func TestCcheckIneligibleTableDDL(t *testing.T) { 745 ddlJobPuller, helper := newMockDDLJobPuller(t, true) 746 defer helper.Close() 747 748 startTs := uint64(10) 749 ddlJobPullerImpl := ddlJobPuller.(*ddlJobPullerImpl) 750 ddlJobPullerImpl.setResolvedTs(startTs) 751 752 cfg := config.GetDefaultReplicaConfig() 753 f, err := filter.NewFilter(cfg, "") 754 require.NoError(t, err) 755 ddlJobPullerImpl.filter = f 756 757 ddl := helper.DDL2Job("CREATE DATABASE test1") 758 skip, err := ddlJobPullerImpl.handleJob(ddl) 759 require.NoError(t, err) 760 require.False(t, skip) 761 762 // case 1: create a table only has a primary key and drop it, expect an error. 763 // It is because the table is not eligible after the drop primary key DDL. 764 ddl = helper.DDL2Job(`CREATE TABLE test1.t1 ( 765 id INT PRIMARY KEY /*T![clustered_index] NONCLUSTERED */, 766 name VARCHAR(255), 767 email VARCHAR(255) UNIQUE 768 );`) 769 skip, err = ddlJobPullerImpl.handleJob(ddl) 770 require.NoError(t, err) 771 require.False(t, skip) 772 773 ddl = helper.DDL2Job("ALTER TABLE test1.t1 DROP PRIMARY KEY;") 774 skip, err = ddlJobPullerImpl.handleJob(ddl) 775 require.Error(t, err) 776 require.False(t, skip) 777 require.Contains(t, err.Error(), "An eligible table become ineligible after DDL") 778 779 // case 2: create a table has a primary key and another not null unique key, 780 // and drop the primary key, expect no error. 781 // It is because the table is still eligible after the drop primary key DDL. 782 ddl = helper.DDL2Job(`CREATE TABLE test1.t2 ( 783 id INT PRIMARY KEY /*T![clustered_index] NONCLUSTERED */, 784 name VARCHAR(255), 785 email VARCHAR(255) NOT NULL UNIQUE 786 );`) 787 skip, err = ddlJobPullerImpl.handleJob(ddl) 788 require.NoError(t, err) 789 require.False(t, skip) 790 791 ddl = helper.DDL2Job("ALTER TABLE test1.t2 DROP PRIMARY KEY;") 792 skip, err = ddlJobPullerImpl.handleJob(ddl) 793 require.NoError(t, err) 794 require.False(t, skip) 795 796 // case 3: continue to drop the unique key, expect an error. 797 // It is because the table is not eligible after the drop unique key DDL. 798 ddl = helper.DDL2Job("ALTER TABLE test1.t2 DROP INDEX email;") 799 skip, err = ddlJobPullerImpl.handleJob(ddl) 800 require.Error(t, err) 801 require.False(t, skip) 802 require.Contains(t, err.Error(), "An eligible table become ineligible after DDL") 803 }