github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/checker/checker.go (about) 1 // Copyright 2019 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package checker 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/json" 20 "errors" 21 "fmt" 22 "strings" 23 "sync" 24 "time" 25 26 _ "github.com/go-sql-driver/mysql" // for mysql 27 "github.com/pingcap/tidb/dumpling/export" 28 "github.com/pingcap/tidb/lightning/pkg/importer" 29 "github.com/pingcap/tidb/lightning/pkg/importer/opts" 30 "github.com/pingcap/tidb/lightning/pkg/precheck" 31 "github.com/pingcap/tidb/pkg/lightning/checkpoints" 32 "github.com/pingcap/tidb/pkg/lightning/common" 33 "github.com/pingcap/tidb/pkg/lightning/mydump" 34 "github.com/pingcap/tidb/pkg/parser/mysql" 35 "github.com/pingcap/tidb/pkg/types" 36 "github.com/pingcap/tidb/pkg/util/dbutil" 37 "github.com/pingcap/tidb/pkg/util/filter" 38 regexprrouter "github.com/pingcap/tidb/pkg/util/regexpr-router" 39 "github.com/pingcap/tiflow/dm/config" 40 "github.com/pingcap/tiflow/dm/config/dbconfig" 41 "github.com/pingcap/tiflow/dm/loader" 42 "github.com/pingcap/tiflow/dm/pb" 43 "github.com/pingcap/tiflow/dm/pkg/binlog" 44 "github.com/pingcap/tiflow/dm/pkg/checker" 45 "github.com/pingcap/tiflow/dm/pkg/conn" 46 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 47 "github.com/pingcap/tiflow/dm/pkg/cputil" 48 "github.com/pingcap/tiflow/dm/pkg/dumpling" 49 fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" 50 "github.com/pingcap/tiflow/dm/pkg/log" 51 "github.com/pingcap/tiflow/dm/pkg/terror" 52 onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" 53 "github.com/pingcap/tiflow/dm/unit" 54 pdhttp "github.com/tikv/pd/client/http" 55 "go.uber.org/atomic" 56 "go.uber.org/zap" 57 "golang.org/x/sync/errgroup" 58 ) 59 60 const ( 61 // the total time needed to complete the check depends on the number of instances, databases and tables, 62 // now increase the total timeout to 30min, but set `readTimeout` to 30s for source/target DB. 63 // if we can not complete the check in 30min, then we must need to refactor the implementation of the function. 64 checkTimeout = 30 * time.Minute 65 readTimeout = "30s" 66 ) 67 68 type mysqlInstance struct { 69 cfg *config.SubTaskConfig 70 71 sourceDB *conn.BaseDB 72 sourceDBinfo *dbutil.DBConfig 73 74 targetDB *conn.BaseDB 75 targetDBInfo *dbutil.DBConfig 76 77 baList *filter.Filter 78 } 79 80 // Checker performs pre-check of data synchronization. 81 type Checker struct { 82 closed atomic.Bool 83 84 tctx *tcontext.Context 85 86 instances []*mysqlInstance 87 88 checkList []checker.RealChecker 89 checkingItems map[string]string 90 dumpWholeInstance bool 91 result struct { 92 sync.RWMutex 93 detail *checker.Results 94 } 95 errCnt int64 96 warnCnt int64 97 98 onlineDDL onlineddl.OnlinePlugin 99 100 stCfgs []*config.SubTaskConfig 101 } 102 103 // NewChecker returns a checker. 104 func NewChecker(cfgs []*config.SubTaskConfig, checkingItems map[string]string, errCnt, warnCnt int64) *Checker { 105 c := &Checker{ 106 instances: make([]*mysqlInstance, 0, len(cfgs)), 107 checkingItems: checkingItems, 108 errCnt: errCnt, 109 warnCnt: warnCnt, 110 stCfgs: cfgs, 111 } 112 113 for _, cfg := range cfgs { 114 // we have verified it in SubTaskConfig.Adjust 115 replica, _ := cfg.DecryptedClone() 116 c.instances = append(c.instances, &mysqlInstance{ 117 cfg: replica, 118 }) 119 } 120 121 return c 122 } 123 124 // tablePairInfo records information about a upstream-downstream(source-target) table pair. 125 // Members may have repeated meanings but they have different data structure to satisfy different usages. 126 type tablePairInfo struct { 127 // target table -> sourceID -> source tables 128 targetTable2SourceTablesMap map[filter.Table]map[string][]filter.Table 129 // target database -> target tables under this database 130 db2TargetTables map[string][]filter.Table 131 // number of sharding tables (source tables) of a target table among all upstreams. 132 targetTableShardNum map[filter.Table]int 133 // sourceID -> tables of this source in allow-list 134 sourceID2SourceTables map[string][]filter.Table 135 // sourceID -> databases that contain allow-list tables 136 sourceID2InterestedDB []map[string]struct{} 137 // sourceID -> target table -> source tables 138 sourceID2TableMap map[string]map[filter.Table][]filter.Table 139 // target table -> extended columns 140 targetTable2ExtendedColumns map[filter.Table][]string 141 // byte size of all upstream tables, counting both data and index 142 totalDataSize atomic.Int64 143 } 144 145 func (c *Checker) getTablePairInfo(ctx context.Context) (info *tablePairInfo, err error) { 146 info = &tablePairInfo{} 147 eg, ctx2 := errgroup.WithContext(ctx) 148 149 // do network things concurrently 150 tableMapPerUpstream := make([]map[filter.Table][]filter.Table, len(c.instances)) 151 extendedColumnPerTable := map[filter.Table][]string{} 152 extendedColumnPerTableMu := sync.Mutex{} 153 for idx := range c.instances { 154 i := idx 155 eg.Go(func() error { 156 tableMapping, extendedColumnM, fetchErr := c.fetchSourceTargetDB(ctx2, c.instances[i]) 157 if fetchErr != nil { 158 return fetchErr 159 } 160 tableMapPerUpstream[i] = tableMapping 161 for table, cols := range extendedColumnM { 162 // same target table may come from different upstream instances 163 // though they are duplicated they should be the same 164 extendedColumnPerTableMu.Lock() 165 extendedColumnPerTable[table] = cols 166 extendedColumnPerTableMu.Unlock() 167 } 168 return nil 169 }) 170 } 171 if egErr := eg.Wait(); egErr != nil { 172 return nil, egErr 173 } 174 175 info.targetTable2ExtendedColumns = extendedColumnPerTable 176 info.targetTable2SourceTablesMap = make(map[filter.Table]map[string][]filter.Table) 177 info.targetTableShardNum = make(map[filter.Table]int) 178 info.db2TargetTables = make(map[string][]filter.Table) 179 180 for i, inst := range c.instances { 181 mapping := tableMapPerUpstream[i] 182 err = sameTableNameDetection(mapping) 183 if err != nil { 184 return nil, err 185 } 186 187 sourceID := inst.cfg.SourceID 188 for targetTable, sourceTables := range mapping { 189 tablesPerSource, ok := info.targetTable2SourceTablesMap[targetTable] 190 if !ok { 191 tablesPerSource = make(map[string][]filter.Table) 192 info.targetTable2SourceTablesMap[targetTable] = tablesPerSource 193 } 194 tablesPerSource[sourceID] = append(tablesPerSource[sourceID], sourceTables...) 195 info.targetTableShardNum[targetTable] += len(sourceTables) 196 info.db2TargetTables[targetTable.Schema] = append(info.db2TargetTables[targetTable.Schema], targetTable) 197 } 198 } 199 200 info.sourceID2SourceTables = make(map[string][]filter.Table, len(c.instances)) 201 info.sourceID2InterestedDB = make([]map[string]struct{}, len(c.instances)) 202 info.sourceID2TableMap = make(map[string]map[filter.Table][]filter.Table, len(c.instances)) 203 sourceIDs := make([]string, 0, len(c.instances)) 204 dbs := make(map[string]*conn.BaseDB, len(c.instances)) 205 for i, inst := range c.instances { 206 sourceID := inst.cfg.SourceID 207 info.sourceID2InterestedDB[i] = make(map[string]struct{}) 208 mapping := tableMapPerUpstream[i] 209 info.sourceID2TableMap[sourceID] = mapping 210 for _, tables := range mapping { 211 info.sourceID2SourceTables[sourceID] = append(info.sourceID2SourceTables[sourceID], tables...) 212 for _, table := range tables { 213 info.sourceID2InterestedDB[i][table.Schema] = struct{}{} 214 } 215 } 216 sourceIDs = append(sourceIDs, sourceID) 217 dbs[sourceID] = inst.sourceDB 218 } 219 220 if _, ok := c.checkingItems[config.LightningFreeSpaceChecking]; ok && 221 c.stCfgs[0].LoaderConfig.ImportMode == config.LoadModePhysical && 222 config.HasLoad(c.stCfgs[0].Mode) { 223 concurrency, err := checker.GetConcurrency(ctx, sourceIDs, dbs, c.stCfgs[0].MydumperConfig.Threads) 224 if err != nil { 225 return nil, err 226 } 227 228 type job struct { 229 db *conn.BaseDB 230 schema string 231 table string 232 } 233 234 pool := checker.NewWorkerPoolWithContext[job, int64](ctx, func(result int64) { 235 info.totalDataSize.Add(result) 236 }) 237 for i := 0; i < concurrency; i++ { 238 pool.Go(func(ctx context.Context, job job) (int64, error) { 239 return conn.FetchTableEstimatedBytes( 240 ctx, 241 job.db, 242 job.schema, 243 job.table, 244 ) 245 }) 246 } 247 248 for idx := range c.instances { 249 for _, sourceTables := range tableMapPerUpstream[idx] { 250 for _, sourceTable := range sourceTables { 251 pool.PutJob(job{ 252 db: c.instances[idx].sourceDB, 253 schema: sourceTable.Schema, 254 table: sourceTable.Name, 255 }) 256 } 257 } 258 } 259 err2 := pool.Wait() 260 if err2 != nil { 261 return nil, err2 262 } 263 } 264 265 return info, nil 266 } 267 268 // Init implements Unit interface. 269 func (c *Checker) Init(ctx context.Context) (err error) { 270 rollbackHolder := fr.NewRollbackHolder("checker") 271 defer func() { 272 if err != nil { 273 rollbackHolder.RollbackReverseOrder() 274 } 275 }() 276 277 rollbackHolder.Add(fr.FuncRollback{Name: "close-DBs", Fn: c.closeDBs}) 278 279 c.tctx = tcontext.NewContext(ctx, log.With(zap.String("unit", "task check"))) 280 info, err := c.getTablePairInfo(ctx) 281 if err != nil { 282 return err 283 } 284 285 if _, ok := c.checkingItems[config.ConnNumberChecking]; ok { 286 if len(c.stCfgs) > 0 { 287 // only check the first subtask's config 288 // because the Mode is the same across all the subtasks 289 // as long as they are derived from the same task config. 290 // TODO: check the connections for syncer 291 // TODO: check for incremental mode 292 if config.HasDump(c.stCfgs[0].Mode) { 293 for i, inst := range c.instances { 294 c.checkList = append(c.checkList, checker.NewDumperConnNumberChecker(inst.sourceDB, c.stCfgs[i].MydumperConfig.Threads)) 295 } 296 } 297 if config.HasLoad(c.stCfgs[0].Mode) { 298 c.checkList = append(c.checkList, checker.NewLoaderConnNumberChecker(c.instances[0].targetDB, c.stCfgs)) 299 } 300 } 301 } 302 // check target DB's privilege 303 if _, ok := c.checkingItems[config.TargetDBPrivilegeChecking]; ok { 304 c.checkList = append(c.checkList, checker.NewTargetPrivilegeChecker( 305 c.instances[0].targetDB.DB, 306 c.instances[0].targetDBInfo, 307 )) 308 } 309 // sourceID -> DB 310 upstreamDBs := make(map[string]*conn.BaseDB) 311 for i, instance := range c.instances { 312 sourceID := instance.cfg.SourceID 313 // init online ddl for checker 314 if instance.cfg.OnlineDDL && c.onlineDDL == nil { 315 c.onlineDDL, err = onlineddl.NewRealOnlinePlugin(c.tctx, instance.cfg, nil) 316 if err != nil { 317 return err 318 } 319 rollbackHolder.Add(fr.FuncRollback{Name: "close-onlineDDL", Fn: c.closeOnlineDDL}) 320 } 321 if _, ok := c.checkingItems[config.VersionChecking]; ok { 322 c.checkList = append(c.checkList, checker.NewMySQLVersionChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 323 } 324 325 upstreamDBs[sourceID] = instance.sourceDB 326 if config.HasDump(instance.cfg.Mode) { 327 // increment mode needn't check dump privilege 328 if _, ok := c.checkingItems[config.DumpPrivilegeChecking]; ok { 329 exportCfg := export.DefaultConfig() 330 err := dumpling.ParseExtraArgs(&c.tctx.Logger, exportCfg, strings.Fields(instance.cfg.ExtraArgs)) 331 if err != nil { 332 return err 333 } 334 c.checkList = append(c.checkList, checker.NewSourceDumpPrivilegeChecker( 335 instance.sourceDB.DB, 336 instance.sourceDBinfo, 337 info.sourceID2SourceTables[sourceID], 338 exportCfg.Consistency, 339 c.dumpWholeInstance, 340 )) 341 } 342 } else if !instance.cfg.UseRelay && instance.cfg.Meta != nil { 343 checkMetaPos := len(instance.cfg.Meta.BinLogName) > 0 || 344 (instance.cfg.EnableGTID && len(instance.cfg.Meta.BinLogGTID) > 0) 345 if _, ok := c.checkingItems[config.MetaPositionChecking]; checkMetaPos && ok { 346 c.checkList = append(c.checkList, checker.NewMetaPositionChecker(instance.sourceDB, 347 instance.cfg.From, 348 instance.cfg.EnableGTID, 349 instance.cfg.Meta)) 350 } 351 } 352 if config.HasSync(instance.cfg.Mode) { 353 // full mode needn't check follows 354 if _, ok := c.checkingItems[config.ServerIDChecking]; ok { 355 c.checkList = append(c.checkList, checker.NewMySQLServerIDChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 356 } 357 if _, ok := c.checkingItems[config.BinlogEnableChecking]; ok { 358 c.checkList = append(c.checkList, checker.NewMySQLBinlogEnableChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 359 } 360 if _, ok := c.checkingItems[config.BinlogFormatChecking]; ok { 361 c.checkList = append(c.checkList, checker.NewMySQLBinlogFormatChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 362 } 363 if _, ok := c.checkingItems[config.BinlogRowImageChecking]; ok { 364 c.checkList = append(c.checkList, checker.NewMySQLBinlogRowImageChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 365 } 366 if _, ok := c.checkingItems[config.ReplicationPrivilegeChecking]; ok { 367 c.checkList = append(c.checkList, checker.NewSourceReplicationPrivilegeChecker(instance.sourceDB.DB, instance.sourceDBinfo)) 368 } 369 if _, ok := c.checkingItems[config.OnlineDDLChecking]; c.onlineDDL != nil && ok { 370 c.checkList = append(c.checkList, checker.NewOnlineDDLChecker(instance.sourceDB.DB, info.sourceID2InterestedDB[i], c.onlineDDL, instance.baList)) 371 } 372 if _, ok := c.checkingItems[config.BinlogDBChecking]; ok { 373 c.checkList = append(c.checkList, checker.NewBinlogDBChecker(instance.sourceDB, instance.sourceDBinfo, info.sourceID2InterestedDB[i], instance.cfg.CaseSensitive)) 374 } 375 } 376 } 377 378 dumpThreads := c.instances[0].cfg.MydumperConfig.Threads 379 if _, ok := c.checkingItems[config.TableSchemaChecking]; ok { 380 c.checkList = append(c.checkList, checker.NewTablesChecker( 381 upstreamDBs, 382 c.instances[0].targetDB, 383 info.sourceID2TableMap, 384 info.targetTable2ExtendedColumns, 385 dumpThreads, 386 )) 387 } 388 389 instance := c.instances[0] 390 // Not check the sharding tables’ schema when the mode is increment. 391 // Because the table schema obtained from `show create table` is not the schema at the point of binlog. 392 _, checkingShardID := c.checkingItems[config.ShardAutoIncrementIDChecking] 393 _, checkingShard := c.checkingItems[config.ShardTableSchemaChecking] 394 if checkingShard && instance.cfg.ShardMode != "" && config.HasDump(instance.cfg.Mode) { 395 isFresh, err := c.IsFreshTask() 396 if err != nil { 397 return err 398 } 399 if isFresh { 400 for targetTable, shardingSet := range info.targetTable2SourceTablesMap { 401 if info.targetTableShardNum[targetTable] <= 1 { 402 continue 403 } 404 if instance.cfg.ShardMode == config.ShardPessimistic { 405 c.checkList = append(c.checkList, checker.NewShardingTablesChecker( 406 targetTable.String(), 407 upstreamDBs, 408 shardingSet, 409 checkingShardID, 410 dumpThreads, 411 )) 412 } else { 413 c.checkList = append(c.checkList, checker.NewOptimisticShardingTablesChecker( 414 targetTable.String(), 415 upstreamDBs, 416 shardingSet, 417 dumpThreads, 418 )) 419 } 420 } 421 } 422 } 423 424 hasLightningPrecheck := false 425 for _, item := range config.LightningPrechecks { 426 if _, ok := c.checkingItems[item]; ok { 427 hasLightningPrecheck = true 428 break 429 } 430 } 431 432 if config.HasLoad(instance.cfg.Mode) && 433 instance.cfg.LoaderConfig.ImportMode == config.LoadModePhysical && 434 hasLightningPrecheck { 435 lCfg, err := loader.GetLightningConfig(loader.MakeGlobalConfig(instance.cfg), instance.cfg) 436 if err != nil { 437 return err 438 } 439 // Adjust will raise error when this field is empty, so we set any non empty value here. 440 lCfg.Mydumper.SourceDir = "noop://" 441 if lightningCheckGroupOnlyTableEmpty(c.checkingItems) { 442 lCfg.TiDB.PdAddr = "noop:2379" 443 } 444 err = lCfg.Adjust(ctx) 445 if err != nil { 446 return err 447 } 448 449 cpdb, err := checkpoints.OpenCheckpointsDB(ctx, lCfg) 450 if err != nil { 451 return err 452 } 453 targetDB, err := importer.DBFromConfig(ctx, lCfg.TiDB) 454 if err != nil { 455 return err 456 } 457 458 var opts []pdhttp.ClientOption 459 tls, err := common.NewTLS( 460 lCfg.Security.CAPath, 461 lCfg.Security.CertPath, 462 lCfg.Security.KeyPath, 463 "", 464 lCfg.Security.CABytes, 465 lCfg.Security.CertBytes, 466 lCfg.Security.KeyBytes, 467 ) 468 if err != nil { 469 log.L().Fatal("failed to load TLS certificates", zap.Error(err)) 470 } 471 if o := tls.TLSConfig(); o != nil { 472 opts = append(opts, pdhttp.WithTLSConfig(o)) 473 } 474 pdClient := pdhttp.NewClient( 475 "dm-check", []string{lCfg.TiDB.PdAddr}, opts...) 476 477 targetInfoGetter, err := importer.NewTargetInfoGetterImpl(lCfg, targetDB, pdClient) 478 if err != nil { 479 return err 480 } 481 482 var dbMetas []*mydump.MDDatabaseMeta 483 484 // use downstream table for shard merging 485 for db, tables := range info.db2TargetTables { 486 mdTables := make([]*mydump.MDTableMeta, 0, len(tables)) 487 for _, table := range tables { 488 mdTables = append(mdTables, &mydump.MDTableMeta{ 489 DB: db, 490 Name: table.Name, 491 }) 492 } 493 dbMetas = append(dbMetas, &mydump.MDDatabaseMeta{ 494 Name: db, 495 Tables: mdTables, 496 }) 497 } 498 499 builder := importer.NewPrecheckItemBuilder( 500 lCfg, 501 dbMetas, 502 newLightningPrecheckAdaptor(targetInfoGetter, info), 503 cpdb, 504 pdClient, 505 ) 506 507 if _, ok := c.checkingItems[config.LightningFreeSpaceChecking]; ok { 508 c.checkList = append(c.checkList, checker.NewLightningFreeSpaceChecker( 509 info.totalDataSize.Load(), targetInfoGetter)) 510 } 511 if _, ok := c.checkingItems[config.LightningEmptyRegionChecking]; ok { 512 lChecker, err := builder.BuildPrecheckItem(precheck.CheckTargetClusterEmptyRegion) 513 if err != nil { 514 return err 515 } 516 c.checkList = append(c.checkList, checker.NewLightningEmptyRegionChecker(lChecker)) 517 } 518 if _, ok := c.checkingItems[config.LightningRegionDistributionChecking]; ok { 519 lChecker, err := builder.BuildPrecheckItem(precheck.CheckTargetClusterRegionDist) 520 if err != nil { 521 return err 522 } 523 c.checkList = append(c.checkList, checker.NewLightningRegionDistributionChecker(lChecker)) 524 } 525 if _, ok := c.checkingItems[config.LightningDownstreamVersionChecking]; ok { 526 lChecker, err := builder.BuildPrecheckItem(precheck.CheckTargetClusterVersion) 527 if err != nil { 528 return err 529 } 530 c.checkList = append(c.checkList, checker.NewLightningClusterVersionChecker(lChecker)) 531 } 532 if _, ok := c.checkingItems[config.LightningMutexFeatureChecking]; ok { 533 lChecker, err := builder.BuildPrecheckItem(precheck.CheckTargetUsingCDCPITR) 534 if err != nil { 535 return err 536 } 537 c.checkList = append(c.checkList, checker.NewLightningCDCPiTRChecker(lChecker)) 538 } 539 if _, ok := c.checkingItems[config.LightningTableEmptyChecking]; ok { 540 lChecker, err := builder.BuildPrecheckItem(precheck.CheckTargetTableEmpty) 541 if err != nil { 542 return err 543 } 544 c.checkList = append(c.checkList, checker.NewLightningEmptyTableChecker(lChecker)) 545 } 546 } 547 548 c.tctx.Logger.Info(c.displayCheckingItems()) 549 return nil 550 } 551 552 func lightningCheckGroupOnlyTableEmpty(checkingItems map[string]string) bool { 553 for _, item := range config.LightningPrechecks { 554 if _, ok := checkingItems[item]; ok && item != config.LightningTableEmptyChecking { 555 return false 556 } 557 } 558 _, ok := checkingItems[config.LightningTableEmptyChecking] 559 return ok 560 } 561 562 func (c *Checker) fetchSourceTargetDB( 563 ctx context.Context, 564 instance *mysqlInstance, 565 ) (map[filter.Table][]filter.Table, map[filter.Table][]string, error) { 566 bAList, err := filter.New(instance.cfg.CaseSensitive, instance.cfg.BAList) 567 if err != nil { 568 return nil, nil, terror.ErrTaskCheckGenBAList.Delegate(err) 569 } 570 instance.baList = bAList 571 r, err := regexprrouter.NewRegExprRouter(instance.cfg.CaseSensitive, instance.cfg.RouteRules) 572 if err != nil { 573 return nil, nil, terror.ErrTaskCheckGenTableRouter.Delegate(err) 574 } 575 576 if err != nil { 577 return nil, nil, terror.ErrTaskCheckGenColumnMapping.Delegate(err) 578 } 579 580 instance.sourceDBinfo = &dbutil.DBConfig{ 581 Host: instance.cfg.From.Host, 582 Port: instance.cfg.From.Port, 583 User: instance.cfg.From.User, 584 Password: instance.cfg.From.Password, 585 } 586 dbCfg := instance.cfg.From 587 dbCfg.RawDBCfg = dbconfig.DefaultRawDBConfig().SetReadTimeout(readTimeout) 588 instance.sourceDB, err = conn.GetUpstreamDB(&dbCfg) 589 if err != nil { 590 return nil, nil, terror.WithScope(terror.ErrTaskCheckFailedOpenDB.Delegate(err, instance.cfg.From.User, instance.cfg.From.Host, instance.cfg.From.Port), terror.ScopeUpstream) 591 } 592 instance.targetDBInfo = &dbutil.DBConfig{ 593 Host: instance.cfg.To.Host, 594 Port: instance.cfg.To.Port, 595 User: instance.cfg.To.User, 596 Password: instance.cfg.To.Password, 597 } 598 dbCfg = instance.cfg.To 599 dbCfg.RawDBCfg = dbconfig.DefaultRawDBConfig().SetReadTimeout(readTimeout) 600 instance.targetDB, err = conn.GetDownstreamDB(&dbCfg) 601 if err != nil { 602 return nil, nil, terror.WithScope(terror.ErrTaskCheckFailedOpenDB.Delegate(err, instance.cfg.To.User, instance.cfg.To.Host, instance.cfg.To.Port), terror.ScopeDownstream) 603 } 604 return conn.FetchTargetDoTables(ctx, instance.cfg.SourceID, instance.sourceDB, instance.baList, r) 605 } 606 607 func (c *Checker) displayCheckingItems() string { 608 if len(c.checkList) == 0 { 609 return "not found any checking items\n" 610 } 611 612 var buf bytes.Buffer 613 fmt.Fprintf(&buf, "\n************ task %s checking items ************\n", c.instances[0].cfg.Name) 614 for _, checkFunc := range c.checkList { 615 fmt.Fprintf(&buf, "%s\n", checkFunc.Name()) 616 } 617 fmt.Fprintf(&buf, "************ task %s checking items ************", c.instances[0].cfg.Name) 618 return buf.String() 619 } 620 621 // Process implements Unit interface. 622 func (c *Checker) Process(ctx context.Context, pr chan pb.ProcessResult) { 623 cctx, cancel := context.WithTimeout(ctx, checkTimeout) 624 defer cancel() 625 626 isCanceled := false 627 errs := make([]*pb.ProcessError, 0, 1) 628 result, err := checker.Do(cctx, c.checkList) 629 if err != nil { 630 errs = append(errs, unit.NewProcessError(err)) 631 } else if !result.Summary.Passed { 632 errs = append(errs, unit.NewProcessError(errors.New("check was failed, please see detail"))) 633 } 634 635 filterResults(result, c.warnCnt, c.errCnt) 636 637 c.updateInstruction(result) 638 639 select { 640 case <-cctx.Done(): 641 isCanceled = true 642 default: 643 } 644 645 var rawResult []byte 646 if result.Summary.Successful != result.Summary.Total { 647 rawResult, err = json.MarshalIndent(result, "\t", "\t") 648 if err != nil { 649 rawResult = []byte(fmt.Sprintf("marshal error %v", err)) 650 } 651 } 652 c.result.Lock() 653 c.result.detail = result 654 c.result.Unlock() 655 656 pr <- pb.ProcessResult{ 657 IsCanceled: isCanceled, 658 Errors: errs, 659 Detail: rawResult, 660 } 661 } 662 663 func filterResults(result *checker.Results, warnCnt, errCnt int64) { 664 // remove success result if not pass 665 results := result.Results[:0] 666 for _, r := range result.Results { 667 if r.State == checker.StateSuccess { 668 continue 669 } 670 671 // handle results without r.Errors 672 if len(r.Errors) == 0 { 673 switch r.State { 674 case checker.StateWarning: 675 if warnCnt == 0 { 676 continue 677 } 678 warnCnt-- 679 results = append(results, r) 680 case checker.StateFailure: 681 if errCnt == 0 { 682 continue 683 } 684 errCnt-- 685 results = append(results, r) 686 } 687 continue 688 } 689 690 subErrors := make([]*checker.Error, 0, len(r.Errors)) 691 for _, e := range r.Errors { 692 switch e.Severity { 693 case checker.StateWarning: 694 if warnCnt == 0 { 695 continue 696 } 697 warnCnt-- 698 subErrors = append(subErrors, e) 699 case checker.StateFailure: 700 if errCnt == 0 { 701 continue 702 } 703 errCnt-- 704 subErrors = append(subErrors, e) 705 } 706 } 707 // skip display an empty Result 708 if len(subErrors) > 0 { 709 r.Errors = subErrors 710 results = append(results, r) 711 } 712 } 713 result.Results = results 714 } 715 716 // updateInstruction updates the check result's Instruction. 717 func (c *Checker) updateInstruction(result *checker.Results) { 718 for _, r := range result.Results { 719 if r.State == checker.StateSuccess { 720 continue 721 } 722 723 // can't judge by other field, maybe update it later 724 if r.Extra == checker.AutoIncrementKeyChecking { 725 if strings.HasPrefix(r.Instruction, "please handle it by yourself") { 726 r.Instruction += ", refer to https://docs.pingcap.com/tidb-data-migration/stable/shard-merge-best-practices#handle-conflicts-of-auto-increment-primary-key) for details." 727 } 728 } 729 } 730 } 731 732 // Close implements Unit interface. 733 func (c *Checker) Close() { 734 if c.closed.Load() { 735 return 736 } 737 738 c.closeDBs() 739 c.closeOnlineDDL() 740 741 c.closed.Store(true) 742 } 743 744 func (c *Checker) closeDBs() { 745 for _, instance := range c.instances { 746 if instance.sourceDB != nil { 747 if err := instance.sourceDB.Close(); err != nil { 748 c.tctx.Logger.Error("close source db", zap.Stringer("db", instance.sourceDBinfo), log.ShortError(err)) 749 } 750 instance.sourceDB = nil 751 } 752 753 if instance.targetDB != nil { 754 if err := instance.targetDB.Close(); err != nil { 755 c.tctx.Logger.Error("close target db", zap.Stringer("db", instance.targetDBInfo), log.ShortError(err)) 756 } 757 instance.targetDB = nil 758 } 759 } 760 } 761 762 func (c *Checker) closeOnlineDDL() { 763 if c.onlineDDL != nil { 764 c.onlineDDL.Close() 765 c.onlineDDL = nil 766 } 767 } 768 769 // Pause implements Unit interface. 770 func (c *Checker) Pause() { 771 if c.closed.Load() { 772 c.tctx.Logger.Warn("try to pause, but already closed") 773 return 774 } 775 } 776 777 // Resume resumes the paused process. 778 func (c *Checker) Resume(ctx context.Context, pr chan pb.ProcessResult) { 779 if c.closed.Load() { 780 c.tctx.Logger.Warn("try to resume, but already closed") 781 return 782 } 783 784 c.Process(ctx, pr) 785 } 786 787 // Update implements Unit.Update. 788 func (c *Checker) Update(ctx context.Context, cfg *config.SubTaskConfig) error { 789 // not support update configuration now 790 return nil 791 } 792 793 // Type implements Unit interface. 794 func (c *Checker) Type() pb.UnitType { 795 return pb.UnitType_Check 796 } 797 798 // IsFreshTask implements Unit.IsFreshTask. 799 func (c *Checker) IsFreshTask() (bool, error) { 800 instance := c.instances[0] 801 checkpointSQLs := []string{ 802 fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.LoaderCheckpoint(instance.cfg.Name))), 803 fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.LightningCheckpoint(instance.cfg.Name))), 804 fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.SyncerCheckpoint(instance.cfg.Name))), 805 } 806 var existCheckpoint bool 807 for _, sql := range checkpointSQLs { 808 c.tctx.Logger.Info("exec query", zap.String("sql", sql)) 809 rows, err := instance.targetDB.DB.QueryContext(c.tctx.Ctx, sql) 810 if err != nil { 811 if conn.IsMySQLError(err, mysql.ErrNoSuchTable) { 812 continue 813 } 814 return false, err 815 } 816 defer rows.Close() 817 if rows.Err() != nil { 818 return false, rows.Err() 819 } 820 existCheckpoint = true 821 c.tctx.Logger.Info("exist checkpoint, so don't check sharding tables") 822 break 823 } 824 return !existCheckpoint, nil 825 } 826 827 // Status implements Unit interface. 828 func (c *Checker) Status(_ *binlog.SourceStatus) interface{} { 829 c.result.RLock() 830 res := c.result.detail 831 c.result.RUnlock() 832 833 rawResult, err := json.Marshal(res) 834 if err != nil { 835 rawResult = []byte(fmt.Sprintf("marshal %+v error %v", res, err)) 836 } 837 838 return &pb.CheckStatus{ 839 Passed: res.Summary.Passed, 840 Total: int32(res.Summary.Total), 841 Failed: int32(res.Summary.Failed), 842 Successful: int32(res.Summary.Successful), 843 Warning: int32(res.Summary.Warning), 844 Detail: rawResult, 845 } 846 } 847 848 // Error implements Unit interface. 849 func (c *Checker) Error() interface{} { 850 return &pb.CheckError{} 851 } 852 853 func sameTableNameDetection(tables map[filter.Table][]filter.Table) error { 854 tableNameSets := make(map[string]string) 855 var messages []string 856 857 for tbl := range tables { 858 name := tbl.String() 859 nameL := strings.ToLower(name) 860 if nameO, ok := tableNameSets[nameL]; !ok { 861 tableNameSets[nameL] = name 862 } else { 863 messages = append(messages, fmt.Sprintf("same target table %v vs %s", nameO, name)) 864 } 865 } 866 867 if len(messages) > 0 { 868 return terror.ErrTaskCheckSameTableName.Generate(messages) 869 } 870 871 return nil 872 } 873 874 // lightningPrecheckAdaptor implements the importer.PreRestoreInfoGetter interface. 875 type lightningPrecheckAdaptor struct { 876 importer.TargetInfoGetter 877 allTables map[string]*checkpoints.TidbDBInfo 878 sourceDataResult importer.EstimateSourceDataSizeResult 879 } 880 881 func newLightningPrecheckAdaptor( 882 targetInfoGetter importer.TargetInfoGetter, 883 info *tablePairInfo, 884 ) *lightningPrecheckAdaptor { 885 var ( 886 sourceDataResult importer.EstimateSourceDataSizeResult 887 allTables = make(map[string]*checkpoints.TidbDBInfo) 888 ) 889 if info != nil { 890 sourceDataResult.SizeWithIndex = info.totalDataSize.Load() 891 } 892 for db, tables := range info.db2TargetTables { 893 allTables[db] = &checkpoints.TidbDBInfo{ 894 Name: db, 895 Tables: make(map[string]*checkpoints.TidbTableInfo), 896 } 897 for _, table := range tables { 898 allTables[db].Tables[table.Name] = &checkpoints.TidbTableInfo{ 899 DB: db, 900 Name: table.Name, 901 } 902 } 903 } 904 return &lightningPrecheckAdaptor{ 905 TargetInfoGetter: targetInfoGetter, 906 allTables: allTables, 907 sourceDataResult: sourceDataResult, 908 } 909 } 910 911 func (l *lightningPrecheckAdaptor) GetAllTableStructures(ctx context.Context, opts ...opts.GetPreInfoOption) (map[string]*checkpoints.TidbDBInfo, error) { 912 // re-use with other checker? or in fact we only use other information than structure? 913 return l.allTables, nil 914 } 915 916 func (l *lightningPrecheckAdaptor) ReadFirstNRowsByTableName(ctx context.Context, schemaName string, tableName string, n int) (cols []string, rows [][]types.Datum, err error) { 917 return nil, nil, errors.New("not implemented") 918 } 919 920 func (l *lightningPrecheckAdaptor) ReadFirstNRowsByFileMeta(ctx context.Context, dataFileMeta mydump.SourceFileMeta, n int) (cols []string, rows [][]types.Datum, err error) { 921 return nil, nil, errors.New("not implemented") 922 } 923 924 func (l *lightningPrecheckAdaptor) EstimateSourceDataSize(ctx context.Context, opts ...opts.GetPreInfoOption) (*importer.EstimateSourceDataSizeResult, error) { 925 return &l.sourceDataResult, nil 926 }