github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/restore/restore.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 restore 15 16 import ( 17 "context" 18 "database/sql" 19 "fmt" 20 "io" 21 "math" 22 "os" 23 "strings" 24 "sync" 25 "time" 26 27 "github.com/docker/go-units" 28 "github.com/pingcap/errors" 29 "github.com/pingcap/failpoint" 30 sstpb "github.com/pingcap/kvproto/pkg/import_sstpb" 31 "github.com/pingcap/parser/model" 32 "github.com/pingcap/tidb/meta/autoid" 33 "github.com/pingcap/tidb/util/collate" 34 "go.uber.org/atomic" 35 "go.uber.org/multierr" 36 "go.uber.org/zap" 37 "modernc.org/mathutil" 38 39 "github.com/pingcap/br/pkg/lightning/backend" 40 "github.com/pingcap/br/pkg/lightning/backend/importer" 41 "github.com/pingcap/br/pkg/lightning/backend/kv" 42 "github.com/pingcap/br/pkg/lightning/backend/local" 43 "github.com/pingcap/br/pkg/lightning/backend/tidb" 44 "github.com/pingcap/br/pkg/lightning/checkpoints" 45 "github.com/pingcap/br/pkg/lightning/common" 46 "github.com/pingcap/br/pkg/lightning/config" 47 "github.com/pingcap/br/pkg/lightning/glue" 48 "github.com/pingcap/br/pkg/lightning/log" 49 "github.com/pingcap/br/pkg/lightning/metric" 50 "github.com/pingcap/br/pkg/lightning/mydump" 51 "github.com/pingcap/br/pkg/lightning/tikv" 52 verify "github.com/pingcap/br/pkg/lightning/verification" 53 "github.com/pingcap/br/pkg/lightning/web" 54 "github.com/pingcap/br/pkg/lightning/worker" 55 "github.com/pingcap/br/pkg/pdutil" 56 "github.com/pingcap/br/pkg/storage" 57 "github.com/pingcap/br/pkg/utils" 58 "github.com/pingcap/br/pkg/version" 59 "github.com/pingcap/br/pkg/version/build" 60 ) 61 62 const ( 63 FullLevelCompact = -1 64 Level1Compact = 1 65 ) 66 67 const ( 68 defaultGCLifeTime = 100 * time.Hour 69 ) 70 71 const ( 72 indexEngineID = -1 73 ) 74 75 const ( 76 compactStateIdle int32 = iota 77 compactStateDoing 78 ) 79 80 const ( 81 taskMetaTableName = "task_meta" 82 tableMetaTableName = "table_meta" 83 // CreateTableMetadataTable stores the per-table sub jobs information used by TiDB Lightning 84 CreateTableMetadataTable = `CREATE TABLE IF NOT EXISTS %s ( 85 task_id BIGINT(20) UNSIGNED, 86 table_id BIGINT(64) NOT NULL, 87 table_name VARCHAR(64) NOT NULL, 88 row_id_base BIGINT(20) NOT NULL DEFAULT 0, 89 row_id_max BIGINT(20) NOT NULL DEFAULT 0, 90 total_kvs_base BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 91 total_bytes_base BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 92 checksum_base BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 93 total_kvs BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 94 total_bytes BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 95 checksum BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 96 status VARCHAR(32) NOT NULL, 97 PRIMARY KEY (table_id, task_id) 98 );` 99 // CreateTaskMetaTable stores the pre-lightning metadata used by TiDB Lightning 100 CreateTaskMetaTable = `CREATE TABLE IF NOT EXISTS %s ( 101 task_id BIGINT(20) UNSIGNED NOT NULL, 102 pd_cfgs VARCHAR(2048) NOT NULL DEFAULT '', 103 status VARCHAR(32) NOT NULL, 104 state TINYINT(1) NOT NULL DEFAULT 0 COMMENT '0: normal, 1: exited before finish', 105 source_bytes BIGINT(20) UNSIGNED NOT NULL DEFAULT 0, 106 PRIMARY KEY (task_id) 107 );` 108 109 compactionLowerThreshold = 512 * units.MiB 110 compactionUpperThreshold = 32 * units.GiB 111 ) 112 113 // DeliverPauser is a shared pauser to pause progress to (*chunkRestore).encodeLoop 114 var DeliverPauser = common.NewPauser() 115 116 // nolint:gochecknoinits // TODO: refactor 117 func init() { 118 failpoint.Inject("SetMinDeliverBytes", func(v failpoint.Value) { 119 minDeliverBytes = uint64(v.(int)) 120 }) 121 } 122 123 type saveCp struct { 124 tableName string 125 merger checkpoints.TableCheckpointMerger 126 } 127 128 type errorSummary struct { 129 status checkpoints.CheckpointStatus 130 err error 131 } 132 133 type errorSummaries struct { 134 sync.Mutex 135 logger log.Logger 136 summary map[string]errorSummary 137 } 138 139 // makeErrorSummaries returns an initialized errorSummaries instance 140 func makeErrorSummaries(logger log.Logger) errorSummaries { 141 return errorSummaries{ 142 logger: logger, 143 summary: make(map[string]errorSummary), 144 } 145 } 146 147 func (es *errorSummaries) emitLog() { 148 es.Lock() 149 defer es.Unlock() 150 151 if errorCount := len(es.summary); errorCount > 0 { 152 logger := es.logger 153 logger.Error("tables failed to be imported", zap.Int("count", errorCount)) 154 for tableName, errorSummary := range es.summary { 155 logger.Error("-", 156 zap.String("table", tableName), 157 zap.String("status", errorSummary.status.MetricName()), 158 log.ShortError(errorSummary.err), 159 ) 160 } 161 } 162 } 163 164 func (es *errorSummaries) record(tableName string, err error, status checkpoints.CheckpointStatus) { 165 es.Lock() 166 defer es.Unlock() 167 es.summary[tableName] = errorSummary{status: status, err: err} 168 } 169 170 const ( 171 diskQuotaStateIdle int32 = iota 172 diskQuotaStateChecking 173 diskQuotaStateImporting 174 175 diskQuotaMaxReaders = 1 << 30 176 ) 177 178 // diskQuotaLock is essentially a read/write lock. The implement here is inspired by sync.RWMutex. 179 // diskQuotaLock removed the unnecessary blocking `RLock` method and add a non-blocking `TryRLock` method. 180 type diskQuotaLock struct { 181 w sync.Mutex // held if there are pending writers 182 writerSem chan struct{} // semaphore for writers to wait for completing readers 183 readerCount atomic.Int32 // number of pending readers 184 readerWait atomic.Int32 // number of departing readers 185 } 186 187 func newDiskQuotaLock() *diskQuotaLock { 188 return &diskQuotaLock{writerSem: make(chan struct{})} 189 } 190 191 func (d *diskQuotaLock) Lock() { 192 d.w.Lock() 193 // Announce to readers there is a pending writer. 194 r := d.readerCount.Sub(diskQuotaMaxReaders) + diskQuotaMaxReaders 195 if r != 0 && d.readerWait.Add(r) != 0 { 196 // Wait for active readers. 197 <-d.writerSem 198 } 199 } 200 201 func (d *diskQuotaLock) Unlock() { 202 d.readerCount.Add(diskQuotaMaxReaders) 203 d.w.Unlock() 204 } 205 206 func (d *diskQuotaLock) TryRLock() (locked bool) { 207 r := d.readerCount.Load() 208 for r >= 0 { 209 if d.readerCount.CAS(r, r+1) { 210 return true 211 } 212 r = d.readerCount.Load() 213 } 214 return false 215 } 216 217 func (d *diskQuotaLock) RUnlock() { 218 if d.readerCount.Dec() < 0 { 219 if d.readerWait.Dec() == 0 { 220 // The last reader unblocks the writer. 221 d.writerSem <- struct{}{} 222 } 223 } 224 } 225 226 type Controller struct { 227 cfg *config.Config 228 dbMetas []*mydump.MDDatabaseMeta 229 dbInfos map[string]*checkpoints.TidbDBInfo 230 tableWorkers *worker.Pool 231 indexWorkers *worker.Pool 232 regionWorkers *worker.Pool 233 ioWorkers *worker.Pool 234 checksumWorks *worker.Pool 235 pauser *common.Pauser 236 backend backend.Backend 237 tidbGlue glue.Glue 238 239 alterTableLock sync.Mutex 240 sysVars map[string]string 241 tls *common.TLS 242 checkTemplate Template 243 244 errorSummaries errorSummaries 245 246 checkpointsDB checkpoints.DB 247 saveCpCh chan saveCp 248 checkpointsWg sync.WaitGroup 249 250 closedEngineLimit *worker.Pool 251 store storage.ExternalStorage 252 metaMgrBuilder metaMgrBuilder 253 taskMgr taskMetaMgr 254 255 diskQuotaLock *diskQuotaLock 256 diskQuotaState atomic.Int32 257 compactState atomic.Int32 258 } 259 260 func NewRestoreController( 261 ctx context.Context, 262 dbMetas []*mydump.MDDatabaseMeta, 263 cfg *config.Config, 264 s storage.ExternalStorage, 265 g glue.Glue, 266 ) (*Controller, error) { 267 return NewRestoreControllerWithPauser(ctx, dbMetas, cfg, s, DeliverPauser, g) 268 } 269 270 func NewRestoreControllerWithPauser( 271 ctx context.Context, 272 dbMetas []*mydump.MDDatabaseMeta, 273 cfg *config.Config, 274 s storage.ExternalStorage, 275 pauser *common.Pauser, 276 g glue.Glue, 277 ) (*Controller, error) { 278 tls, err := cfg.ToTLS() 279 if err != nil { 280 return nil, err 281 } 282 283 cpdb, err := g.OpenCheckpointsDB(ctx, cfg) 284 if err != nil { 285 return nil, errors.Annotate(err, "open checkpoint db failed") 286 } 287 288 taskCp, err := cpdb.TaskCheckpoint(ctx) 289 if err != nil { 290 return nil, errors.Annotate(err, "get task checkpoint failed") 291 } 292 if err := verifyCheckpoint(cfg, taskCp); err != nil { 293 return nil, errors.Trace(err) 294 } 295 // reuse task id to reuse task meta correctly. 296 if taskCp != nil { 297 cfg.TaskID = taskCp.TaskID 298 } 299 300 var backend backend.Backend 301 switch cfg.TikvImporter.Backend { 302 case config.BackendImporter: 303 var err error 304 backend, err = importer.NewImporter(ctx, tls, cfg.TikvImporter.Addr, cfg.TiDB.PdAddr) 305 if err != nil { 306 return nil, errors.Annotate(err, "open importer backend failed") 307 } 308 case config.BackendTiDB: 309 db, err := DBFromConfig(cfg.TiDB) 310 if err != nil { 311 return nil, errors.Annotate(err, "open tidb backend failed") 312 } 313 backend = tidb.NewTiDBBackend(db, cfg.TikvImporter.OnDuplicate) 314 case config.BackendLocal: 315 var rLimit local.Rlim_t 316 rLimit, err = local.GetSystemRLimit() 317 if err != nil { 318 return nil, err 319 } 320 maxOpenFiles := int(rLimit / local.Rlim_t(cfg.App.TableConcurrency)) 321 // check overflow 322 if maxOpenFiles < 0 { 323 maxOpenFiles = math.MaxInt32 324 } 325 326 backend, err = local.NewLocalBackend(ctx, tls, cfg.TiDB.PdAddr, &cfg.TikvImporter, 327 cfg.Checkpoint.Enable, g, maxOpenFiles) 328 if err != nil { 329 return nil, errors.Annotate(err, "build local backend failed") 330 } 331 err = verifyLocalFile(ctx, cpdb, cfg.TikvImporter.SortedKVDir) 332 if err != nil { 333 return nil, err 334 } 335 default: 336 return nil, errors.New("unknown backend: " + cfg.TikvImporter.Backend) 337 } 338 339 var metaBuilder metaMgrBuilder 340 switch cfg.TikvImporter.Backend { 341 case config.BackendLocal, config.BackendImporter: 342 // TODO: support Lightning via SQL 343 db, err := g.GetDB() 344 if err != nil { 345 return nil, errors.Trace(err) 346 } 347 metaBuilder = &dbMetaMgrBuilder{ 348 db: db, 349 taskID: cfg.TaskID, 350 schema: cfg.App.MetaSchemaName, 351 needChecksum: cfg.PostRestore.Checksum != config.OpLevelOff, 352 } 353 default: 354 metaBuilder = noopMetaMgrBuilder{} 355 } 356 357 rc := &Controller{ 358 cfg: cfg, 359 dbMetas: dbMetas, 360 tableWorkers: nil, 361 indexWorkers: nil, 362 regionWorkers: worker.NewPool(ctx, cfg.App.RegionConcurrency, "region"), 363 ioWorkers: worker.NewPool(ctx, cfg.App.IOConcurrency, "io"), 364 checksumWorks: worker.NewPool(ctx, cfg.TiDB.ChecksumTableConcurrency, "checksum"), 365 pauser: pauser, 366 backend: backend, 367 tidbGlue: g, 368 sysVars: defaultImportantVariables, 369 tls: tls, 370 checkTemplate: NewSimpleTemplate(), 371 372 errorSummaries: makeErrorSummaries(log.L()), 373 checkpointsDB: cpdb, 374 saveCpCh: make(chan saveCp), 375 closedEngineLimit: worker.NewPool(ctx, cfg.App.TableConcurrency*2, "closed-engine"), 376 377 store: s, 378 metaMgrBuilder: metaBuilder, 379 diskQuotaLock: newDiskQuotaLock(), 380 taskMgr: nil, 381 } 382 383 return rc, nil 384 } 385 386 func (rc *Controller) Close() { 387 rc.backend.Close() 388 rc.tidbGlue.GetSQLExecutor().Close() 389 } 390 391 func (rc *Controller) Run(ctx context.Context) error { 392 opts := []func(context.Context) error{ 393 rc.setGlobalVariables, 394 rc.restoreSchema, 395 rc.preCheckRequirements, 396 rc.restoreTables, 397 rc.fullCompact, 398 rc.switchToNormalMode, 399 rc.cleanCheckpoints, 400 } 401 402 task := log.L().Begin(zap.InfoLevel, "the whole procedure") 403 404 var err error 405 finished := false 406 outside: 407 for i, process := range opts { 408 err = process(ctx) 409 if i == len(opts)-1 { 410 finished = true 411 } 412 logger := task.With(zap.Int("step", i), log.ShortError(err)) 413 414 switch { 415 case err == nil: 416 case log.IsContextCanceledError(err): 417 logger.Info("task canceled") 418 err = nil 419 break outside 420 default: 421 logger.Error("run failed") 422 fmt.Fprintf(os.Stderr, "Error: %s\n", err) 423 break outside // ps : not continue 424 } 425 } 426 427 // if process is cancelled, should make sure checkpoints are written to db. 428 if !finished { 429 rc.waitCheckpointFinish() 430 } 431 432 task.End(zap.ErrorLevel, err) 433 rc.errorSummaries.emitLog() 434 435 return errors.Trace(err) 436 } 437 438 type schemaStmtType int 439 440 func (stmtType schemaStmtType) String() string { 441 switch stmtType { 442 case schemaCreateDatabase: 443 return "restore database schema" 444 case schemaCreateTable: 445 return "restore table schema" 446 case schemaCreateView: 447 return "restore view schema" 448 } 449 return "unknown statement of schema" 450 } 451 452 const ( 453 schemaCreateDatabase schemaStmtType = iota 454 schemaCreateTable 455 schemaCreateView 456 ) 457 458 type schemaJob struct { 459 dbName string 460 tblName string // empty for create db jobs 461 stmtType schemaStmtType 462 stmts []*schemaStmt 463 } 464 465 type schemaStmt struct { 466 sql string 467 } 468 469 type restoreSchemaWorker struct { 470 ctx context.Context 471 quit context.CancelFunc 472 jobCh chan *schemaJob 473 errCh chan error 474 wg sync.WaitGroup 475 glue glue.Glue 476 store storage.ExternalStorage 477 } 478 479 func (worker *restoreSchemaWorker) makeJobs( 480 dbMetas []*mydump.MDDatabaseMeta, 481 getTables func(context.Context, string) ([]*model.TableInfo, error), 482 ) error { 483 defer func() { 484 close(worker.jobCh) 485 worker.quit() 486 }() 487 var err error 488 // 1. restore databases, execute statements concurrency 489 for _, dbMeta := range dbMetas { 490 restoreSchemaJob := &schemaJob{ 491 dbName: dbMeta.Name, 492 stmtType: schemaCreateDatabase, 493 stmts: make([]*schemaStmt, 0, 1), 494 } 495 restoreSchemaJob.stmts = append(restoreSchemaJob.stmts, &schemaStmt{ 496 sql: createDatabaseIfNotExistStmt(dbMeta.Name), 497 }) 498 err = worker.appendJob(restoreSchemaJob) 499 if err != nil { 500 return err 501 } 502 } 503 err = worker.wait() 504 if err != nil { 505 return err 506 } 507 // 2. restore tables, execute statements concurrency 508 for _, dbMeta := range dbMetas { 509 // we can ignore error here, and let check failed later if schema not match 510 tables, _ := getTables(worker.ctx, dbMeta.Name) 511 tableMap := make(map[string]struct{}) 512 for _, t := range tables { 513 tableMap[t.Name.L] = struct{}{} 514 } 515 for _, tblMeta := range dbMeta.Tables { 516 if _, ok := tableMap[strings.ToLower(tblMeta.Name)]; ok { 517 // we already has this table in TiDB. 518 // we should skip ddl job and let SchemaValid check. 519 continue 520 } else if tblMeta.SchemaFile.FileMeta.Path == "" { 521 return errors.Errorf("table `%s`.`%s` schema not found", dbMeta.Name, tblMeta.Name) 522 } 523 sql, err := tblMeta.GetSchema(worker.ctx, worker.store) 524 if sql != "" { 525 stmts, err := createTableIfNotExistsStmt(worker.glue.GetParser(), sql, dbMeta.Name, tblMeta.Name) 526 if err != nil { 527 return err 528 } 529 restoreSchemaJob := &schemaJob{ 530 dbName: dbMeta.Name, 531 tblName: tblMeta.Name, 532 stmtType: schemaCreateTable, 533 stmts: make([]*schemaStmt, 0, len(stmts)), 534 } 535 for _, sql := range stmts { 536 restoreSchemaJob.stmts = append(restoreSchemaJob.stmts, &schemaStmt{ 537 sql: sql, 538 }) 539 } 540 err = worker.appendJob(restoreSchemaJob) 541 if err != nil { 542 return err 543 } 544 } 545 if err != nil { 546 return err 547 } 548 } 549 } 550 err = worker.wait() 551 if err != nil { 552 return err 553 } 554 // 3. restore views. Since views can cross database we must restore views after all table schemas are restored. 555 for _, dbMeta := range dbMetas { 556 for _, viewMeta := range dbMeta.Views { 557 sql, err := viewMeta.GetSchema(worker.ctx, worker.store) 558 if sql != "" { 559 stmts, err := createTableIfNotExistsStmt(worker.glue.GetParser(), sql, dbMeta.Name, viewMeta.Name) 560 if err != nil { 561 return err 562 } 563 restoreSchemaJob := &schemaJob{ 564 dbName: dbMeta.Name, 565 tblName: viewMeta.Name, 566 stmtType: schemaCreateView, 567 stmts: make([]*schemaStmt, 0, len(stmts)), 568 } 569 for _, sql := range stmts { 570 restoreSchemaJob.stmts = append(restoreSchemaJob.stmts, &schemaStmt{ 571 sql: sql, 572 }) 573 } 574 err = worker.appendJob(restoreSchemaJob) 575 if err != nil { 576 return err 577 } 578 // we don't support restore views concurrency, cauz it maybe will raise a error 579 err = worker.wait() 580 if err != nil { 581 return err 582 } 583 } 584 if err != nil { 585 return err 586 } 587 } 588 } 589 return nil 590 } 591 592 func (worker *restoreSchemaWorker) doJob() { 593 var session *sql.Conn 594 defer func() { 595 if session != nil { 596 _ = session.Close() 597 } 598 }() 599 loop: 600 for { 601 select { 602 case <-worker.ctx.Done(): 603 // don't `return` or throw `worker.ctx.Err()`here, 604 // if we `return`, we can't mark cancelled jobs as done, 605 // if we `throw(worker.ctx.Err())`, it will be blocked to death 606 break loop 607 case job := <-worker.jobCh: 608 if job == nil { 609 // successful exit 610 return 611 } 612 var err error 613 if session == nil { 614 session, err = func() (*sql.Conn, error) { 615 // TODO: support lightning in SQL 616 db, err := worker.glue.GetDB() 617 if err != nil { 618 return nil, errors.Trace(err) 619 } 620 return db.Conn(worker.ctx) 621 }() 622 if err != nil { 623 worker.wg.Done() 624 worker.throw(err) 625 // don't return 626 break loop 627 } 628 } 629 logger := log.With(zap.String("db", job.dbName), zap.String("table", job.tblName)) 630 sqlWithRetry := common.SQLWithRetry{ 631 Logger: log.L(), 632 DB: session, 633 } 634 for _, stmt := range job.stmts { 635 task := logger.Begin(zap.DebugLevel, fmt.Sprintf("execute SQL: %s", stmt.sql)) 636 err = sqlWithRetry.Exec(worker.ctx, "run create schema job", stmt.sql) 637 task.End(zap.ErrorLevel, err) 638 if err != nil { 639 err = errors.Annotatef(err, "%s %s failed", job.stmtType.String(), common.UniqueTable(job.dbName, job.tblName)) 640 worker.wg.Done() 641 worker.throw(err) 642 // don't return 643 break loop 644 } 645 } 646 worker.wg.Done() 647 } 648 } 649 // mark the cancelled job as `Done`, a little tricky, 650 // cauz we need make sure `worker.wg.Wait()` wouldn't blocked forever 651 for range worker.jobCh { 652 worker.wg.Done() 653 } 654 } 655 656 func (worker *restoreSchemaWorker) wait() error { 657 // avoid to `worker.wg.Wait()` blocked forever when all `doJob`'s goroutine exited. 658 // don't worry about goroutine below, it never become a zombie, 659 // cauz we have mechanism to clean cancelled jobs in `worker.jobCh`. 660 // means whole jobs has been send to `worker.jobCh` would be done. 661 waitCh := make(chan struct{}) 662 go func() { 663 worker.wg.Wait() 664 close(waitCh) 665 }() 666 select { 667 case err := <-worker.errCh: 668 return err 669 case <-worker.ctx.Done(): 670 return worker.ctx.Err() 671 case <-waitCh: 672 return nil 673 } 674 } 675 676 func (worker *restoreSchemaWorker) throw(err error) { 677 select { 678 case <-worker.ctx.Done(): 679 // don't throw `worker.ctx.Err()` again, it will be blocked to death. 680 return 681 case worker.errCh <- err: 682 worker.quit() 683 } 684 } 685 686 func (worker *restoreSchemaWorker) appendJob(job *schemaJob) error { 687 worker.wg.Add(1) 688 select { 689 case err := <-worker.errCh: 690 // cancel the job 691 worker.wg.Done() 692 return err 693 case <-worker.ctx.Done(): 694 // cancel the job 695 worker.wg.Done() 696 return worker.ctx.Err() 697 case worker.jobCh <- job: 698 return nil 699 } 700 } 701 702 func (rc *Controller) restoreSchema(ctx context.Context) error { 703 // create table with schema file 704 // we can handle the duplicated created with createIfNotExist statement 705 // and we will check the schema in TiDB is valid with the datafile in DataCheck later. 706 logTask := log.L().Begin(zap.InfoLevel, "restore all schema") 707 concurrency := utils.MinInt(rc.cfg.App.RegionConcurrency, 8) 708 childCtx, cancel := context.WithCancel(ctx) 709 worker := restoreSchemaWorker{ 710 ctx: childCtx, 711 quit: cancel, 712 jobCh: make(chan *schemaJob, concurrency), 713 errCh: make(chan error), 714 glue: rc.tidbGlue, 715 store: rc.store, 716 } 717 for i := 0; i < concurrency; i++ { 718 go worker.doJob() 719 } 720 getTableFunc := rc.backend.FetchRemoteTableModels 721 if !rc.tidbGlue.OwnsSQLExecutor() { 722 getTableFunc = rc.tidbGlue.GetTables 723 } 724 err := worker.makeJobs(rc.dbMetas, getTableFunc) 725 logTask.End(zap.ErrorLevel, err) 726 if err != nil { 727 return err 728 } 729 730 dbInfos, err := LoadSchemaInfo(ctx, rc.dbMetas, getTableFunc) 731 if err != nil { 732 return errors.Trace(err) 733 } 734 rc.dbInfos = dbInfos 735 736 if rc.cfg.App.CheckRequirements && rc.tidbGlue.OwnsSQLExecutor() { 737 if err = rc.DataCheck(ctx); err != nil { 738 return errors.Trace(err) 739 } 740 // print check template only if check requirements is true. 741 fmt.Println(rc.checkTemplate.Output()) 742 if !rc.checkTemplate.Success() { 743 return errors.Errorf("tidb-lightning pre-check failed." + 744 " Please fix the failed check(s) or set --check-requirements=false to skip checks") 745 } 746 } 747 748 // Load new checkpoints 749 err = rc.checkpointsDB.Initialize(ctx, rc.cfg, dbInfos) 750 if err != nil { 751 return errors.Trace(err) 752 } 753 failpoint.Inject("InitializeCheckpointExit", func() { 754 log.L().Warn("exit triggered", zap.String("failpoint", "InitializeCheckpointExit")) 755 os.Exit(0) 756 }) 757 758 go rc.listenCheckpointUpdates() 759 760 rc.sysVars = ObtainImportantVariables(ctx, rc.tidbGlue.GetSQLExecutor()) 761 762 // Estimate the number of chunks for progress reporting 763 err = rc.estimateChunkCountIntoMetrics(ctx) 764 if err != nil { 765 return errors.Trace(err) 766 } 767 768 return nil 769 } 770 771 // verifyCheckpoint check whether previous task checkpoint is compatible with task config 772 func verifyCheckpoint(cfg *config.Config, taskCp *checkpoints.TaskCheckpoint) error { 773 if taskCp == nil { 774 return nil 775 } 776 // always check the backend value even with 'check-requirements = false' 777 retryUsage := "destroy all checkpoints" 778 if cfg.Checkpoint.Driver == config.CheckpointDriverFile { 779 retryUsage = fmt.Sprintf("delete the file '%s'", cfg.Checkpoint.DSN) 780 } 781 retryUsage += " and remove all restored tables and try again" 782 783 if cfg.TikvImporter.Backend != taskCp.Backend { 784 return errors.Errorf("config 'tikv-importer.backend' value '%s' different from checkpoint value '%s', please %s", cfg.TikvImporter.Backend, taskCp.Backend, retryUsage) 785 } 786 787 if cfg.App.CheckRequirements { 788 if build.ReleaseVersion != taskCp.LightningVer { 789 var displayVer string 790 if len(taskCp.LightningVer) != 0 { 791 displayVer = fmt.Sprintf("at '%s'", taskCp.LightningVer) 792 } else { 793 displayVer = "before v4.0.6/v3.0.19" 794 } 795 return errors.Errorf("lightning version is '%s', but checkpoint was created %s, please %s", build.ReleaseVersion, displayVer, retryUsage) 796 } 797 798 errorFmt := "config '%s' value '%s' different from checkpoint value '%s'. You may set 'check-requirements = false' to skip this check or " + retryUsage 799 if cfg.Mydumper.SourceDir != taskCp.SourceDir { 800 return errors.Errorf(errorFmt, "mydumper.data-source-dir", cfg.Mydumper.SourceDir, taskCp.SourceDir) 801 } 802 803 if cfg.TikvImporter.Backend == config.BackendLocal && cfg.TikvImporter.SortedKVDir != taskCp.SortedKVDir { 804 return errors.Errorf(errorFmt, "mydumper.sorted-kv-dir", cfg.TikvImporter.SortedKVDir, taskCp.SortedKVDir) 805 } 806 807 if cfg.TikvImporter.Backend == config.BackendImporter && cfg.TikvImporter.Addr != taskCp.ImporterAddr { 808 return errors.Errorf(errorFmt, "tikv-importer.addr", cfg.TikvImporter.Backend, taskCp.Backend) 809 } 810 811 if cfg.TiDB.Host != taskCp.TiDBHost { 812 return errors.Errorf(errorFmt, "tidb.host", cfg.TiDB.Host, taskCp.TiDBHost) 813 } 814 815 if cfg.TiDB.Port != taskCp.TiDBPort { 816 return errors.Errorf(errorFmt, "tidb.port", cfg.TiDB.Port, taskCp.TiDBPort) 817 } 818 819 if cfg.TiDB.PdAddr != taskCp.PdAddr { 820 return errors.Errorf(errorFmt, "tidb.pd-addr", cfg.TiDB.PdAddr, taskCp.PdAddr) 821 } 822 } 823 824 return nil 825 } 826 827 // for local backend, we should check if local SST exists in disk, otherwise we'll lost data 828 func verifyLocalFile(ctx context.Context, cpdb checkpoints.DB, dir string) error { 829 targetTables, err := cpdb.GetLocalStoringTables(ctx) 830 if err != nil { 831 return errors.Trace(err) 832 } 833 for tableName, engineIDs := range targetTables { 834 for _, engineID := range engineIDs { 835 _, eID := backend.MakeUUID(tableName, engineID) 836 file := local.File{UUID: eID} 837 err := file.Exist(dir) 838 if err != nil { 839 log.L().Error("can't find local file", 840 zap.String("table name", tableName), 841 zap.Int32("engine ID", engineID)) 842 return errors.Trace(err) 843 } 844 } 845 } 846 return nil 847 } 848 849 func (rc *Controller) estimateChunkCountIntoMetrics(ctx context.Context) error { 850 estimatedChunkCount := 0.0 851 estimatedEngineCnt := int64(0) 852 batchSize := rc.cfg.Mydumper.BatchSize 853 if batchSize <= 0 { 854 // if rows in source files are not sorted by primary key(if primary is number or cluster index enabled), 855 // the key range in each data engine may have overlap, thus a bigger engine size can somewhat alleviate it. 856 batchSize = config.DefaultBatchSize 857 } 858 for _, dbMeta := range rc.dbMetas { 859 for _, tableMeta := range dbMeta.Tables { 860 tableName := common.UniqueTable(dbMeta.Name, tableMeta.Name) 861 dbCp, err := rc.checkpointsDB.Get(ctx, tableName) 862 if err != nil { 863 return errors.Trace(err) 864 } 865 866 fileChunks := make(map[string]float64) 867 for engineID, eCp := range dbCp.Engines { 868 if eCp.Status < checkpoints.CheckpointStatusImported { 869 estimatedEngineCnt++ 870 } 871 if engineID == indexEngineID { 872 continue 873 } 874 for _, c := range eCp.Chunks { 875 if _, ok := fileChunks[c.Key.Path]; !ok { 876 fileChunks[c.Key.Path] = 0.0 877 } 878 remainChunkCnt := float64(c.Chunk.EndOffset-c.Chunk.Offset) / float64(c.Chunk.EndOffset-c.Key.Offset) 879 fileChunks[c.Key.Path] += remainChunkCnt 880 } 881 } 882 // estimate engines count if engine cp is empty 883 if len(dbCp.Engines) == 0 { 884 estimatedEngineCnt += ((tableMeta.TotalSize + int64(batchSize) - 1) / int64(batchSize)) + 1 885 } 886 for _, fileMeta := range tableMeta.DataFiles { 887 if cnt, ok := fileChunks[fileMeta.FileMeta.Path]; ok { 888 estimatedChunkCount += cnt 889 continue 890 } 891 if fileMeta.FileMeta.Type == mydump.SourceTypeCSV { 892 cfg := rc.cfg.Mydumper 893 if fileMeta.FileMeta.FileSize > int64(cfg.MaxRegionSize) && cfg.StrictFormat && !cfg.CSV.Header { 894 estimatedChunkCount += math.Round(float64(fileMeta.FileMeta.FileSize) / float64(cfg.MaxRegionSize)) 895 } else { 896 estimatedChunkCount++ 897 } 898 } else { 899 estimatedChunkCount++ 900 } 901 } 902 } 903 } 904 metric.ChunkCounter.WithLabelValues(metric.ChunkStateEstimated).Add(estimatedChunkCount) 905 metric.ProcessedEngineCounter.WithLabelValues(metric.ChunkStateEstimated, metric.TableResultSuccess). 906 Add(float64(estimatedEngineCnt)) 907 rc.tidbGlue.Record(glue.RecordEstimatedChunk, uint64(estimatedChunkCount)) 908 return nil 909 } 910 911 func (rc *Controller) saveStatusCheckpoint(tableName string, engineID int32, err error, statusIfSucceed checkpoints.CheckpointStatus) { 912 merger := &checkpoints.StatusCheckpointMerger{Status: statusIfSucceed, EngineID: engineID} 913 914 log.L().Debug("update checkpoint", zap.String("table", tableName), zap.Int32("engine_id", engineID), 915 zap.Uint8("new_status", uint8(statusIfSucceed)), zap.Error(err)) 916 917 switch { 918 case err == nil: 919 break 920 case !common.IsContextCanceledError(err): 921 merger.SetInvalid() 922 rc.errorSummaries.record(tableName, err, statusIfSucceed) 923 default: 924 return 925 } 926 927 if engineID == checkpoints.WholeTableEngineID { 928 metric.RecordTableCount(statusIfSucceed.MetricName(), err) 929 } else { 930 metric.RecordEngineCount(statusIfSucceed.MetricName(), err) 931 } 932 933 rc.saveCpCh <- saveCp{tableName: tableName, merger: merger} 934 } 935 936 // listenCheckpointUpdates will combine several checkpoints together to reduce database load. 937 func (rc *Controller) listenCheckpointUpdates() { 938 rc.checkpointsWg.Add(1) 939 940 var lock sync.Mutex 941 coalesed := make(map[string]*checkpoints.TableCheckpointDiff) 942 943 hasCheckpoint := make(chan struct{}, 1) 944 defer close(hasCheckpoint) 945 946 go func() { 947 for range hasCheckpoint { 948 lock.Lock() 949 cpd := coalesed 950 coalesed = make(map[string]*checkpoints.TableCheckpointDiff) 951 lock.Unlock() 952 953 if len(cpd) > 0 { 954 rc.checkpointsDB.Update(cpd) 955 web.BroadcastCheckpointDiff(cpd) 956 } 957 rc.checkpointsWg.Done() 958 } 959 }() 960 961 for scp := range rc.saveCpCh { 962 lock.Lock() 963 cpd, ok := coalesed[scp.tableName] 964 if !ok { 965 cpd = checkpoints.NewTableCheckpointDiff() 966 coalesed[scp.tableName] = cpd 967 } 968 scp.merger.MergeInto(cpd) 969 970 if len(hasCheckpoint) == 0 { 971 rc.checkpointsWg.Add(1) 972 hasCheckpoint <- struct{}{} 973 } 974 975 lock.Unlock() 976 977 //nolint:scopelint // This would be either INLINED or ERASED, at compile time. 978 failpoint.Inject("FailIfImportedChunk", func(val failpoint.Value) { 979 if merger, ok := scp.merger.(*checkpoints.ChunkCheckpointMerger); ok && merger.Checksum.SumKVS() >= uint64(val.(int)) { 980 rc.checkpointsWg.Done() 981 rc.checkpointsWg.Wait() 982 panic("forcing failure due to FailIfImportedChunk") 983 } 984 }) 985 986 //nolint:scopelint // This would be either INLINED or ERASED, at compile time. 987 failpoint.Inject("FailIfStatusBecomes", func(val failpoint.Value) { 988 if merger, ok := scp.merger.(*checkpoints.StatusCheckpointMerger); ok && merger.EngineID >= 0 && int(merger.Status) == val.(int) { 989 rc.checkpointsWg.Done() 990 rc.checkpointsWg.Wait() 991 panic("forcing failure due to FailIfStatusBecomes") 992 } 993 }) 994 995 //nolint:scopelint // This would be either INLINED or ERASED, at compile time. 996 failpoint.Inject("FailIfIndexEngineImported", func(val failpoint.Value) { 997 if merger, ok := scp.merger.(*checkpoints.StatusCheckpointMerger); ok && 998 merger.EngineID == checkpoints.WholeTableEngineID && 999 merger.Status == checkpoints.CheckpointStatusIndexImported && val.(int) > 0 { 1000 rc.checkpointsWg.Done() 1001 rc.checkpointsWg.Wait() 1002 panic("forcing failure due to FailIfIndexEngineImported") 1003 } 1004 }) 1005 1006 //nolint:scopelint // This would be either INLINED or ERASED, at compile time. 1007 failpoint.Inject("KillIfImportedChunk", func(val failpoint.Value) { 1008 if merger, ok := scp.merger.(*checkpoints.ChunkCheckpointMerger); ok && merger.Checksum.SumKVS() >= uint64(val.(int)) { 1009 if err := common.KillMySelf(); err != nil { 1010 log.L().Warn("KillMySelf() failed to kill itself", log.ShortError(err)) 1011 } 1012 } 1013 }) 1014 } 1015 rc.checkpointsWg.Done() 1016 } 1017 1018 // buildRunPeriodicActionAndCancelFunc build the runPeriodicAction func and a cancel func 1019 func (rc *Controller) buildRunPeriodicActionAndCancelFunc(ctx context.Context, stop <-chan struct{}) (func(), func(bool)) { 1020 cancelFuncs := make([]func(bool), 0) 1021 closeFuncs := make([]func(), 0) 1022 // a nil channel blocks forever. 1023 // if the cron duration is zero we use the nil channel to skip the action. 1024 var logProgressChan <-chan time.Time 1025 if rc.cfg.Cron.LogProgress.Duration > 0 { 1026 logProgressTicker := time.NewTicker(rc.cfg.Cron.LogProgress.Duration) 1027 closeFuncs = append(closeFuncs, func() { 1028 logProgressTicker.Stop() 1029 }) 1030 logProgressChan = logProgressTicker.C 1031 } 1032 1033 glueProgressTicker := time.NewTicker(3 * time.Second) 1034 closeFuncs = append(closeFuncs, func() { 1035 glueProgressTicker.Stop() 1036 }) 1037 1038 var switchModeChan <-chan time.Time 1039 // tidb backend don't need to switch tikv to import mode 1040 if rc.cfg.TikvImporter.Backend != config.BackendTiDB && rc.cfg.Cron.SwitchMode.Duration > 0 { 1041 switchModeTicker := time.NewTicker(rc.cfg.Cron.SwitchMode.Duration) 1042 cancelFuncs = append(cancelFuncs, func(bool) { switchModeTicker.Stop() }) 1043 cancelFuncs = append(cancelFuncs, func(do bool) { 1044 if do { 1045 log.L().Info("switch to normal mode") 1046 if err := rc.switchToNormalMode(ctx); err != nil { 1047 log.L().Warn("switch tikv to normal mode failed", zap.Error(err)) 1048 } 1049 } 1050 }) 1051 switchModeChan = switchModeTicker.C 1052 } 1053 1054 var checkQuotaChan <-chan time.Time 1055 // only local storage has disk quota concern. 1056 if rc.cfg.TikvImporter.Backend == config.BackendLocal && rc.cfg.Cron.CheckDiskQuota.Duration > 0 { 1057 checkQuotaTicker := time.NewTicker(rc.cfg.Cron.CheckDiskQuota.Duration) 1058 cancelFuncs = append(cancelFuncs, func(bool) { checkQuotaTicker.Stop() }) 1059 checkQuotaChan = checkQuotaTicker.C 1060 } 1061 1062 return func() { 1063 defer func() { 1064 for _, f := range closeFuncs { 1065 f() 1066 } 1067 }() 1068 // tidb backend don't need to switch tikv to import mode 1069 if rc.cfg.TikvImporter.Backend != config.BackendTiDB && rc.cfg.Cron.SwitchMode.Duration > 0 { 1070 rc.switchToImportMode(ctx) 1071 } 1072 start := time.Now() 1073 for { 1074 select { 1075 case <-ctx.Done(): 1076 log.L().Warn("stopping periodic actions", log.ShortError(ctx.Err())) 1077 return 1078 case <-stop: 1079 log.L().Info("everything imported, stopping periodic actions") 1080 return 1081 1082 case <-switchModeChan: 1083 // periodically switch to import mode, as requested by TiKV 3.0 1084 rc.switchToImportMode(ctx) 1085 1086 case <-logProgressChan: 1087 // log the current progress periodically, so OPS will know that we're still working 1088 nanoseconds := float64(time.Since(start).Nanoseconds()) 1089 // the estimated chunk is not accurate(likely under estimated), but the actual count is not accurate 1090 // before the last table start, so use the bigger of the two should be a workaround 1091 estimated := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStateEstimated)) 1092 pending := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) 1093 if estimated < pending { 1094 estimated = pending 1095 } 1096 finished := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStateFinished)) 1097 totalTables := metric.ReadCounter(metric.TableCounter.WithLabelValues(metric.TableStatePending, metric.TableResultSuccess)) 1098 completedTables := metric.ReadCounter(metric.TableCounter.WithLabelValues(metric.TableStateCompleted, metric.TableResultSuccess)) 1099 bytesRead := metric.ReadHistogramSum(metric.RowReadBytesHistogram) 1100 engineEstimated := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues(metric.ChunkStateEstimated, metric.TableResultSuccess)) 1101 enginePending := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues(metric.ChunkStatePending, metric.TableResultSuccess)) 1102 if engineEstimated < enginePending { 1103 engineEstimated = enginePending 1104 } 1105 engineFinished := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues(metric.TableStateImported, metric.TableResultSuccess)) 1106 bytesWritten := metric.ReadCounter(metric.BytesCounter.WithLabelValues(metric.TableStateWritten)) 1107 bytesImported := metric.ReadCounter(metric.BytesCounter.WithLabelValues(metric.TableStateImported)) 1108 1109 var state string 1110 var remaining zap.Field 1111 switch { 1112 case finished >= estimated: 1113 if engineFinished < engineEstimated { 1114 state = "importing" 1115 } else { 1116 state = "post-processing" 1117 } 1118 case finished > 0: 1119 state = "writing" 1120 default: 1121 state = "preparing" 1122 } 1123 1124 // since we can't accurately estimate the extra time cost by import after all writing are finished, 1125 // so here we use estimatedWritingProgress * 0.8 + estimatedImportingProgress * 0.2 as the total 1126 // progress. 1127 remaining = zap.Skip() 1128 totalPercent := 0.0 1129 if finished > 0 { 1130 writePercent := math.Min(finished/estimated, 1.0) 1131 importPercent := 1.0 1132 if bytesWritten > 0 { 1133 totalBytes := bytesWritten / writePercent 1134 importPercent = math.Min(bytesImported/totalBytes, 1.0) 1135 } 1136 totalPercent = writePercent*0.8 + importPercent*0.2 1137 if totalPercent < 1.0 { 1138 remainNanoseconds := (1.0 - totalPercent) / totalPercent * nanoseconds 1139 remaining = zap.Duration("remaining", time.Duration(remainNanoseconds).Round(time.Second)) 1140 } 1141 } 1142 1143 formatPercent := func(finish, estimate float64) string { 1144 speed := "" 1145 if estimated > 0 { 1146 speed = fmt.Sprintf(" (%.1f%%)", finish/estimate*100) 1147 } 1148 return speed 1149 } 1150 1151 // avoid output bytes speed if there are no unfinished chunks 1152 chunkSpeed := zap.Skip() 1153 if bytesRead > 0 { 1154 chunkSpeed = zap.Float64("speed(MiB/s)", bytesRead/(1048576e-9*nanoseconds)) 1155 } 1156 1157 // Note: a speed of 28 MiB/s roughly corresponds to 100 GiB/hour. 1158 log.L().Info("progress", 1159 zap.String("total", fmt.Sprintf("%.1f%%", totalPercent*100)), 1160 // zap.String("files", fmt.Sprintf("%.0f/%.0f (%.1f%%)", finished, estimated, finished/estimated*100)), 1161 zap.String("tables", fmt.Sprintf("%.0f/%.0f%s", completedTables, totalTables, formatPercent(completedTables, totalTables))), 1162 zap.String("chunks", fmt.Sprintf("%.0f/%.0f%s", finished, estimated, formatPercent(finished, estimated))), 1163 zap.String("engines", fmt.Sprintf("%.f/%.f%s", engineFinished, engineEstimated, formatPercent(engineFinished, engineEstimated))), 1164 chunkSpeed, 1165 zap.String("state", state), 1166 remaining, 1167 ) 1168 1169 case <-checkQuotaChan: 1170 // verify the total space occupied by sorted-kv-dir is below the quota, 1171 // otherwise we perform an emergency import. 1172 rc.enforceDiskQuota(ctx) 1173 1174 case <-glueProgressTicker.C: 1175 finished := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStateFinished)) 1176 rc.tidbGlue.Record(glue.RecordFinishedChunk, uint64(finished)) 1177 } 1178 } 1179 }, func(do bool) { 1180 log.L().Info("cancel periodic actions", zap.Bool("do", do)) 1181 for _, f := range cancelFuncs { 1182 f(do) 1183 } 1184 } 1185 } 1186 1187 var checksumManagerKey struct{} 1188 1189 func (rc *Controller) restoreTables(ctx context.Context) error { 1190 logTask := log.L().Begin(zap.InfoLevel, "restore all tables data") 1191 if rc.tableWorkers == nil { 1192 rc.tableWorkers = worker.NewPool(ctx, rc.cfg.App.TableConcurrency, "table") 1193 } 1194 if rc.indexWorkers == nil { 1195 rc.indexWorkers = worker.NewPool(ctx, rc.cfg.App.IndexConcurrency, "index") 1196 } 1197 1198 // for local backend, we should disable some pd scheduler and change some settings, to 1199 // make split region and ingest sst more stable 1200 // because importer backend is mostly use for v3.x cluster which doesn't support these api, 1201 // so we also don't do this for import backend 1202 finishSchedulers := func() {} 1203 // if one lightning failed abnormally, and can't determine whether it needs to switch back, 1204 // we do not do switch back automatically 1205 cleanupFunc := func() {} 1206 switchBack := false 1207 taskFinished := false 1208 if rc.cfg.TikvImporter.Backend == config.BackendLocal { 1209 1210 logTask.Info("removing PD leader®ion schedulers") 1211 1212 restoreFn, err := rc.taskMgr.CheckAndPausePdSchedulers(ctx) 1213 finishSchedulers = func() { 1214 if restoreFn != nil { 1215 // use context.Background to make sure this restore function can still be executed even if ctx is canceled 1216 restoreCtx := context.Background() 1217 needSwitchBack, needCleanup, err := rc.taskMgr.CheckAndFinishRestore(restoreCtx, taskFinished) 1218 if err != nil { 1219 logTask.Warn("check restore pd schedulers failed", zap.Error(err)) 1220 return 1221 } 1222 switchBack = needSwitchBack 1223 if needSwitchBack { 1224 if restoreE := restoreFn(restoreCtx); restoreE != nil { 1225 logTask.Warn("failed to restore removed schedulers, you may need to restore them manually", zap.Error(restoreE)) 1226 } 1227 1228 logTask.Info("add back PD leader®ion schedulers") 1229 // clean up task metas 1230 if needCleanup { 1231 logTask.Info("cleanup task metas") 1232 if cleanupErr := rc.taskMgr.Cleanup(restoreCtx); cleanupErr != nil { 1233 logTask.Warn("failed to clean task metas, you may need to restore them manually", zap.Error(cleanupErr)) 1234 } 1235 // cleanup table meta and schema db if needed. 1236 cleanupFunc = func() { 1237 if e := rc.taskMgr.CleanupAllMetas(restoreCtx); err != nil { 1238 logTask.Warn("failed to clean table task metas, you may need to restore them manually", zap.Error(e)) 1239 } 1240 } 1241 } 1242 } 1243 } 1244 1245 rc.taskMgr.Close() 1246 } 1247 1248 if err != nil { 1249 return errors.Trace(err) 1250 } 1251 } 1252 defer func() { 1253 if switchBack { 1254 cleanupFunc() 1255 } 1256 }() 1257 1258 type task struct { 1259 tr *TableRestore 1260 cp *checkpoints.TableCheckpoint 1261 } 1262 1263 totalTables := 0 1264 for _, dbMeta := range rc.dbMetas { 1265 totalTables += len(dbMeta.Tables) 1266 } 1267 postProcessTaskChan := make(chan task, totalTables) 1268 1269 var wg sync.WaitGroup 1270 var restoreErr common.OnceError 1271 1272 stopPeriodicActions := make(chan struct{}) 1273 1274 periodicActions, cancelFunc := rc.buildRunPeriodicActionAndCancelFunc(ctx, stopPeriodicActions) 1275 go periodicActions() 1276 finishFuncCalled := false 1277 defer func() { 1278 if !finishFuncCalled { 1279 finishSchedulers() 1280 cancelFunc(switchBack) 1281 finishFuncCalled = true 1282 } 1283 }() 1284 1285 defer close(stopPeriodicActions) 1286 1287 taskCh := make(chan task, rc.cfg.App.IndexConcurrency) 1288 defer close(taskCh) 1289 1290 manager, err := newChecksumManager(ctx, rc) 1291 if err != nil { 1292 return errors.Trace(err) 1293 } 1294 ctx2 := context.WithValue(ctx, &checksumManagerKey, manager) 1295 for i := 0; i < rc.cfg.App.IndexConcurrency; i++ { 1296 go func() { 1297 for task := range taskCh { 1298 tableLogTask := task.tr.logger.Begin(zap.InfoLevel, "restore table") 1299 web.BroadcastTableCheckpoint(task.tr.tableName, task.cp) 1300 needPostProcess, err := task.tr.restoreTable(ctx2, rc, task.cp) 1301 err = errors.Annotatef(err, "restore table %s failed", task.tr.tableName) 1302 tableLogTask.End(zap.ErrorLevel, err) 1303 web.BroadcastError(task.tr.tableName, err) 1304 metric.RecordTableCount("completed", err) 1305 restoreErr.Set(err) 1306 if needPostProcess { 1307 postProcessTaskChan <- task 1308 } 1309 wg.Done() 1310 } 1311 }() 1312 } 1313 1314 // first collect all tables where the checkpoint is invalid 1315 allInvalidCheckpoints := make(map[string]checkpoints.CheckpointStatus) 1316 // collect all tables whose checkpoint's tableID can't match current tableID 1317 allDirtyCheckpoints := make(map[string]struct{}) 1318 for _, dbMeta := range rc.dbMetas { 1319 dbInfo, ok := rc.dbInfos[dbMeta.Name] 1320 if !ok { 1321 return errors.Errorf("database %s not found in rc.dbInfos", dbMeta.Name) 1322 } 1323 for _, tableMeta := range dbMeta.Tables { 1324 tableInfo, ok := dbInfo.Tables[tableMeta.Name] 1325 if !ok { 1326 return errors.Errorf("table info %s.%s not found", dbMeta.Name, tableMeta.Name) 1327 } 1328 1329 tableName := common.UniqueTable(dbInfo.Name, tableInfo.Name) 1330 cp, err := rc.checkpointsDB.Get(ctx, tableName) 1331 if err != nil { 1332 return errors.Trace(err) 1333 } 1334 if cp.Status <= checkpoints.CheckpointStatusMaxInvalid { 1335 allInvalidCheckpoints[tableName] = cp.Status 1336 } else if cp.TableID > 0 && cp.TableID != tableInfo.ID { 1337 allDirtyCheckpoints[tableName] = struct{}{} 1338 } 1339 } 1340 } 1341 1342 if len(allInvalidCheckpoints) != 0 { 1343 logger := log.L() 1344 logger.Error( 1345 "TiDB Lightning has failed last time. To prevent data loss, this run will stop now. Please resolve errors first", 1346 zap.Int("count", len(allInvalidCheckpoints)), 1347 ) 1348 1349 for tableName, status := range allInvalidCheckpoints { 1350 failedStep := status * 10 1351 var action strings.Builder 1352 action.WriteString("./tidb-lightning-ctl --checkpoint-error-") 1353 switch failedStep { 1354 case checkpoints.CheckpointStatusAlteredAutoInc, checkpoints.CheckpointStatusAnalyzed: 1355 action.WriteString("ignore") 1356 default: 1357 action.WriteString("destroy") 1358 } 1359 action.WriteString("='") 1360 action.WriteString(tableName) 1361 action.WriteString("' --config=...") 1362 1363 logger.Info("-", 1364 zap.String("table", tableName), 1365 zap.Uint8("status", uint8(status)), 1366 zap.String("failedStep", failedStep.MetricName()), 1367 zap.Stringer("recommendedAction", &action), 1368 ) 1369 } 1370 1371 logger.Info("You may also run `./tidb-lightning-ctl --checkpoint-error-destroy=all --config=...` to start from scratch") 1372 logger.Info("For details of this failure, read the log file from the PREVIOUS run") 1373 1374 return errors.New("TiDB Lightning has failed last time; please resolve these errors first") 1375 } 1376 if len(allDirtyCheckpoints) > 0 { 1377 logger := log.L() 1378 logger.Error( 1379 "TiDB Lightning has detected tables with illegal checkpoints. To prevent data mismatch, this run will stop now. Please remove these checkpoints first", 1380 zap.Int("count", len(allDirtyCheckpoints)), 1381 ) 1382 1383 for tableName := range allDirtyCheckpoints { 1384 logger.Info("-", 1385 zap.String("table", tableName), 1386 zap.String("recommendedAction", "./tidb-lightning-ctl --checkpoint-remove='"+tableName+"' --config=..."), 1387 ) 1388 } 1389 1390 logger.Info("You may also run `./tidb-lightning-ctl --checkpoint-remove=all --config=...` to start from scratch") 1391 1392 return errors.New("TiDB Lightning has detected tables with illegal checkpoints; please remove these checkpoints first") 1393 } 1394 1395 for _, dbMeta := range rc.dbMetas { 1396 dbInfo := rc.dbInfos[dbMeta.Name] 1397 for _, tableMeta := range dbMeta.Tables { 1398 tableInfo := dbInfo.Tables[tableMeta.Name] 1399 tableName := common.UniqueTable(dbInfo.Name, tableInfo.Name) 1400 cp, err := rc.checkpointsDB.Get(ctx, tableName) 1401 if err != nil { 1402 return errors.Trace(err) 1403 } 1404 igCols, err := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(dbInfo.Name, tableInfo.Name, rc.cfg.Mydumper.CaseSensitive) 1405 if err != nil { 1406 return errors.Trace(err) 1407 } 1408 tr, err := NewTableRestore(tableName, tableMeta, dbInfo, tableInfo, cp, igCols.Columns) 1409 if err != nil { 1410 return errors.Trace(err) 1411 } 1412 1413 wg.Add(1) 1414 select { 1415 case taskCh <- task{tr: tr, cp: cp}: 1416 case <-ctx.Done(): 1417 return ctx.Err() 1418 } 1419 } 1420 } 1421 1422 wg.Wait() 1423 // if context is done, should return directly 1424 select { 1425 case <-ctx.Done(): 1426 err = restoreErr.Get() 1427 if err == nil { 1428 err = ctx.Err() 1429 } 1430 logTask.End(zap.ErrorLevel, err) 1431 return err 1432 default: 1433 } 1434 1435 // stop periodic tasks for restore table such as pd schedulers and switch-mode tasks. 1436 // this can help make cluster switching back to normal state more quickly. 1437 // finishSchedulers() 1438 // cancelFunc(switchBack) 1439 // finishFuncCalled = true 1440 taskFinished = true 1441 1442 close(postProcessTaskChan) 1443 // otherwise, we should run all tasks in the post-process task chan 1444 for i := 0; i < rc.cfg.App.TableConcurrency; i++ { 1445 wg.Add(1) 1446 go func() { 1447 defer wg.Done() 1448 for task := range postProcessTaskChan { 1449 metaMgr := rc.metaMgrBuilder.TableMetaMgr(task.tr) 1450 // force all the remain post-process tasks to be executed 1451 _, err = task.tr.postProcess(ctx2, rc, task.cp, true, metaMgr) 1452 restoreErr.Set(err) 1453 } 1454 }() 1455 } 1456 wg.Wait() 1457 1458 err = restoreErr.Get() 1459 logTask.End(zap.ErrorLevel, err) 1460 return err 1461 } 1462 1463 func (tr *TableRestore) restoreTable( 1464 ctx context.Context, 1465 rc *Controller, 1466 cp *checkpoints.TableCheckpoint, 1467 ) (bool, error) { 1468 // 1. Load the table info. 1469 1470 select { 1471 case <-ctx.Done(): 1472 return false, ctx.Err() 1473 default: 1474 } 1475 1476 metaMgr := rc.metaMgrBuilder.TableMetaMgr(tr) 1477 // no need to do anything if the chunks are already populated 1478 if len(cp.Engines) > 0 { 1479 tr.logger.Info("reusing engines and files info from checkpoint", 1480 zap.Int("enginesCnt", len(cp.Engines)), 1481 zap.Int("filesCnt", cp.CountChunks()), 1482 ) 1483 } else if cp.Status < checkpoints.CheckpointStatusAllWritten { 1484 versionStr, err := rc.tidbGlue.GetSQLExecutor().ObtainStringWithLog( 1485 ctx, "SELECT version()", "fetch tidb version", log.L()) 1486 if err != nil { 1487 return false, errors.Trace(err) 1488 } 1489 1490 tidbVersion, err := version.ExtractTiDBVersion(versionStr) 1491 if err != nil { 1492 return false, errors.Trace(err) 1493 } 1494 1495 if err := tr.populateChunks(ctx, rc, cp); err != nil { 1496 return false, errors.Trace(err) 1497 } 1498 1499 // fetch the max chunk row_id max value as the global max row_id 1500 rowIDMax := int64(0) 1501 for _, engine := range cp.Engines { 1502 if len(engine.Chunks) > 0 && engine.Chunks[len(engine.Chunks)-1].Chunk.RowIDMax > rowIDMax { 1503 rowIDMax = engine.Chunks[len(engine.Chunks)-1].Chunk.RowIDMax 1504 } 1505 } 1506 1507 // "show table next_row_id" is only available after v4.0.0 1508 if tidbVersion.Major >= 4 && (rc.cfg.TikvImporter.Backend == config.BackendLocal || rc.cfg.TikvImporter.Backend == config.BackendImporter) { 1509 // first, insert a new-line into meta table 1510 if err = metaMgr.InitTableMeta(ctx); err != nil { 1511 return false, err 1512 } 1513 1514 checksum, rowIDBase, err := metaMgr.AllocTableRowIDs(ctx, rowIDMax) 1515 if err != nil { 1516 return false, err 1517 } 1518 tr.RebaseChunkRowIDs(cp, rowIDBase) 1519 1520 if checksum != nil { 1521 if cp.Checksum != *checksum { 1522 cp.Checksum = *checksum 1523 rc.saveCpCh <- saveCp{ 1524 tableName: tr.tableName, 1525 merger: &checkpoints.TableChecksumMerger{ 1526 Checksum: cp.Checksum, 1527 }, 1528 } 1529 } 1530 tr.logger.Info("checksum before restore table", zap.Object("checksum", &cp.Checksum)) 1531 } 1532 } 1533 if err := rc.checkpointsDB.InsertEngineCheckpoints(ctx, tr.tableName, cp.Engines); err != nil { 1534 return false, errors.Trace(err) 1535 } 1536 web.BroadcastTableCheckpoint(tr.tableName, cp) 1537 1538 // rebase the allocator so it exceeds the number of rows. 1539 if tr.tableInfo.Core.PKIsHandle && tr.tableInfo.Core.ContainsAutoRandomBits() { 1540 cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoRandID) 1541 if err := tr.alloc.Get(autoid.AutoRandomType).Rebase(tr.tableInfo.ID, cp.AllocBase, false); err != nil { 1542 return false, err 1543 } 1544 } else { 1545 cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoIncID) 1546 if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(tr.tableInfo.ID, cp.AllocBase, false); err != nil { 1547 return false, err 1548 } 1549 } 1550 rc.saveCpCh <- saveCp{ 1551 tableName: tr.tableName, 1552 merger: &checkpoints.RebaseCheckpointMerger{ 1553 AllocBase: cp.AllocBase, 1554 }, 1555 } 1556 } 1557 1558 // 2. Restore engines (if still needed) 1559 err := tr.restoreEngines(ctx, rc, cp) 1560 if err != nil { 1561 return false, errors.Trace(err) 1562 } 1563 1564 err = metaMgr.UpdateTableStatus(ctx, metaStatusRestoreFinished) 1565 if err != nil { 1566 return false, errors.Trace(err) 1567 } 1568 1569 // 3. Post-process. With the last parameter set to false, we can allow delay analyze execute latter 1570 return tr.postProcess(ctx, rc, cp, false /* force-analyze */, metaMgr) 1571 } 1572 1573 // do full compaction for the whole data. 1574 func (rc *Controller) fullCompact(ctx context.Context) error { 1575 if !rc.cfg.PostRestore.Compact { 1576 log.L().Info("skip full compaction") 1577 return nil 1578 } 1579 1580 // wait until any existing level-1 compact to complete first. 1581 task := log.L().Begin(zap.InfoLevel, "wait for completion of existing level 1 compaction") 1582 for !rc.compactState.CAS(compactStateIdle, compactStateDoing) { 1583 time.Sleep(100 * time.Millisecond) 1584 } 1585 task.End(zap.ErrorLevel, nil) 1586 1587 return errors.Trace(rc.doCompact(ctx, FullLevelCompact)) 1588 } 1589 1590 func (rc *Controller) doCompact(ctx context.Context, level int32) error { 1591 tls := rc.tls.WithHost(rc.cfg.TiDB.PdAddr) 1592 return tikv.ForAllStores( 1593 ctx, 1594 tls, 1595 tikv.StoreStateDisconnected, 1596 func(c context.Context, store *tikv.Store) error { 1597 return tikv.Compact(c, tls, store.Address, level) 1598 }, 1599 ) 1600 } 1601 1602 func (rc *Controller) switchToImportMode(ctx context.Context) { 1603 rc.switchTiKVMode(ctx, sstpb.SwitchMode_Import) 1604 } 1605 1606 func (rc *Controller) switchToNormalMode(ctx context.Context) error { 1607 rc.switchTiKVMode(ctx, sstpb.SwitchMode_Normal) 1608 return nil 1609 } 1610 1611 func (rc *Controller) switchTiKVMode(ctx context.Context, mode sstpb.SwitchMode) { 1612 // It is fine if we miss some stores which did not switch to Import mode, 1613 // since we're running it periodically, so we exclude disconnected stores. 1614 // But it is essential all stores be switched back to Normal mode to allow 1615 // normal operation. 1616 var minState tikv.StoreState 1617 if mode == sstpb.SwitchMode_Import { 1618 minState = tikv.StoreStateOffline 1619 } else { 1620 minState = tikv.StoreStateDisconnected 1621 } 1622 tls := rc.tls.WithHost(rc.cfg.TiDB.PdAddr) 1623 // we ignore switch mode failure since it is not fatal. 1624 // no need log the error, it is done in kv.SwitchMode already. 1625 _ = tikv.ForAllStores( 1626 ctx, 1627 tls, 1628 minState, 1629 func(c context.Context, store *tikv.Store) error { 1630 return tikv.SwitchMode(c, tls, store.Address, mode) 1631 }, 1632 ) 1633 } 1634 1635 func (rc *Controller) enforceDiskQuota(ctx context.Context) { 1636 if !rc.diskQuotaState.CAS(diskQuotaStateIdle, diskQuotaStateChecking) { 1637 // do not run multiple the disk quota check / import simultaneously. 1638 // (we execute the lock check in background to avoid blocking the cron thread) 1639 return 1640 } 1641 1642 go func() { 1643 // locker is assigned when we detect the disk quota is exceeded. 1644 // before the disk quota is confirmed exceeded, we keep the diskQuotaLock 1645 // unlocked to avoid periodically interrupting the writer threads. 1646 var locker sync.Locker 1647 defer func() { 1648 rc.diskQuotaState.Store(diskQuotaStateIdle) 1649 if locker != nil { 1650 locker.Unlock() 1651 } 1652 }() 1653 1654 isRetrying := false 1655 1656 for { 1657 // sleep for a cycle if we are retrying because there is nothing new to import. 1658 if isRetrying { 1659 select { 1660 case <-ctx.Done(): 1661 return 1662 case <-time.After(rc.cfg.Cron.CheckDiskQuota.Duration): 1663 } 1664 } else { 1665 isRetrying = true 1666 } 1667 1668 quota := int64(rc.cfg.TikvImporter.DiskQuota) 1669 largeEngines, inProgressLargeEngines, totalDiskSize, totalMemSize := rc.backend.CheckDiskQuota(quota) 1670 metric.LocalStorageUsageBytesGauge.WithLabelValues("disk").Set(float64(totalDiskSize)) 1671 metric.LocalStorageUsageBytesGauge.WithLabelValues("mem").Set(float64(totalMemSize)) 1672 1673 logger := log.With( 1674 zap.Int64("diskSize", totalDiskSize), 1675 zap.Int64("memSize", totalMemSize), 1676 zap.Int64("quota", quota), 1677 zap.Int("largeEnginesCount", len(largeEngines)), 1678 zap.Int("inProgressLargeEnginesCount", inProgressLargeEngines)) 1679 1680 if len(largeEngines) == 0 && inProgressLargeEngines == 0 { 1681 logger.Debug("disk quota respected") 1682 return 1683 } 1684 1685 if locker == nil { 1686 // blocks all writers when we detected disk quota being exceeded. 1687 rc.diskQuotaLock.Lock() 1688 locker = rc.diskQuotaLock 1689 } 1690 1691 logger.Warn("disk quota exceeded") 1692 if len(largeEngines) == 0 { 1693 logger.Warn("all large engines are already importing, keep blocking all writes") 1694 continue 1695 } 1696 1697 // flush all engines so that checkpoints can be updated. 1698 if err := rc.backend.FlushAll(ctx); err != nil { 1699 logger.Error("flush engine for disk quota failed, check again later", log.ShortError(err)) 1700 return 1701 } 1702 1703 // at this point, all engines are synchronized on disk. 1704 // we then import every large engines one by one and complete. 1705 // if any engine failed to import, we just try again next time, since the data are still intact. 1706 rc.diskQuotaState.Store(diskQuotaStateImporting) 1707 task := logger.Begin(zap.WarnLevel, "importing large engines for disk quota") 1708 var importErr error 1709 for _, engine := range largeEngines { 1710 if err := rc.backend.UnsafeImportAndReset(ctx, engine); err != nil { 1711 importErr = multierr.Append(importErr, err) 1712 } 1713 } 1714 task.End(zap.ErrorLevel, importErr) 1715 return 1716 } 1717 }() 1718 } 1719 1720 func (rc *Controller) setGlobalVariables(ctx context.Context) error { 1721 // set new collation flag base on tidb config 1722 enabled := ObtainNewCollationEnabled(ctx, rc.tidbGlue.GetSQLExecutor()) 1723 // we should enable/disable new collation here since in server mode, tidb config 1724 // may be different in different tasks 1725 collate.SetNewCollationEnabledForTest(enabled) 1726 1727 return nil 1728 } 1729 1730 func (rc *Controller) waitCheckpointFinish() { 1731 // wait checkpoint process finish so that we can do cleanup safely 1732 close(rc.saveCpCh) 1733 rc.checkpointsWg.Wait() 1734 } 1735 1736 func (rc *Controller) cleanCheckpoints(ctx context.Context) error { 1737 rc.waitCheckpointFinish() 1738 1739 if !rc.cfg.Checkpoint.Enable { 1740 return nil 1741 } 1742 1743 logger := log.With( 1744 zap.Bool("keepAfterSuccess", rc.cfg.Checkpoint.KeepAfterSuccess), 1745 zap.Int64("taskID", rc.cfg.TaskID), 1746 ) 1747 1748 task := logger.Begin(zap.InfoLevel, "clean checkpoints") 1749 var err error 1750 if rc.cfg.Checkpoint.KeepAfterSuccess { 1751 err = rc.checkpointsDB.MoveCheckpoints(ctx, rc.cfg.TaskID) 1752 } else { 1753 err = rc.checkpointsDB.RemoveCheckpoint(ctx, "all") 1754 } 1755 task.End(zap.ErrorLevel, err) 1756 return errors.Annotate(err, "clean checkpoints") 1757 } 1758 1759 func (rc *Controller) isLocalBackend() bool { 1760 return rc.cfg.TikvImporter.Backend == config.BackendLocal 1761 } 1762 1763 // preCheckRequirements checks 1764 // 1. Cluster resource 1765 // 2. Local node resource 1766 // 3. Lightning configuration 1767 // before restore tables start. 1768 func (rc *Controller) preCheckRequirements(ctx context.Context) error { 1769 if err := rc.ClusterIsAvailable(ctx); err != nil { 1770 return errors.Trace(err) 1771 } 1772 1773 if err := rc.StoragePermission(ctx); err != nil { 1774 return errors.Trace(err) 1775 } 1776 if err := rc.metaMgrBuilder.Init(ctx); err != nil { 1777 return err 1778 } 1779 taskExist := false 1780 1781 if rc.isLocalBackend() { 1782 source, err := rc.EstimateSourceData(ctx) 1783 if err != nil { 1784 return errors.Trace(err) 1785 } 1786 1787 pdController, err := pdutil.NewPdController(ctx, rc.cfg.TiDB.PdAddr, 1788 rc.tls.TLSConfig(), rc.tls.ToPDSecurityOption()) 1789 if err != nil { 1790 return errors.Trace(err) 1791 } 1792 1793 rc.taskMgr = rc.metaMgrBuilder.TaskMetaMgr(pdController) 1794 taskExist, err = rc.taskMgr.CheckTaskExist(ctx) 1795 if err != nil { 1796 return errors.Trace(err) 1797 } 1798 if !taskExist { 1799 err = rc.LocalResource(ctx, source) 1800 if err != nil { 1801 rc.taskMgr.CleanupTask(ctx) 1802 return errors.Trace(err) 1803 } 1804 if err := rc.ClusterResource(ctx, source); err != nil { 1805 rc.taskMgr.CleanupTask(ctx) 1806 return errors.Trace(err) 1807 } 1808 } 1809 } 1810 if rc.cfg.App.CheckRequirements && rc.tidbGlue.OwnsSQLExecutor() { 1811 // print check template only if check requirements is true. 1812 fmt.Print(rc.checkTemplate.Output()) 1813 if !rc.checkTemplate.Success() { 1814 if !taskExist && rc.taskMgr != nil { 1815 rc.taskMgr.CleanupTask(ctx) 1816 } 1817 return errors.Errorf("tidb-lightning pre-check failed." + 1818 " Please fix the failed check(s) or set --check-requirements=false to skip checks") 1819 } 1820 } 1821 return nil 1822 } 1823 1824 // DataCheck checks the data schema which needs #rc.restoreSchema finished. 1825 func (rc *Controller) DataCheck(ctx context.Context) error { 1826 if !rc.cfg.App.CheckRequirements { 1827 log.L().Info("skip data check due to user requirement") 1828 return nil 1829 } 1830 var err error 1831 err = rc.HasLargeCSV(rc.dbMetas) 1832 if err != nil { 1833 return errors.Trace(err) 1834 } 1835 checkPointCriticalMsgs := make([]string, 0, len(rc.dbMetas)) 1836 schemaCriticalMsgs := make([]string, 0, len(rc.dbMetas)) 1837 var msgs []string 1838 for _, dbInfo := range rc.dbMetas { 1839 for _, tableInfo := range dbInfo.Tables { 1840 // if hasCheckpoint is true, the table will start import from the checkpoint 1841 // so we can skip TableHasDataInCluster and SchemaIsValid check. 1842 noCheckpoint := true 1843 if rc.cfg.Checkpoint.Enable { 1844 if msgs, noCheckpoint, err = rc.CheckpointIsValid(ctx, tableInfo); err != nil { 1845 return errors.Trace(err) 1846 } 1847 if len(msgs) != 0 { 1848 checkPointCriticalMsgs = append(checkPointCriticalMsgs, msgs...) 1849 } 1850 } 1851 if noCheckpoint && rc.cfg.TikvImporter.Backend != config.BackendTiDB { 1852 if msgs, err = rc.SchemaIsValid(ctx, tableInfo); err != nil { 1853 return errors.Trace(err) 1854 } 1855 if len(msgs) != 0 { 1856 schemaCriticalMsgs = append(schemaCriticalMsgs, msgs...) 1857 } 1858 } 1859 } 1860 } 1861 if len(checkPointCriticalMsgs) != 0 { 1862 rc.checkTemplate.Collect(Critical, false, strings.Join(checkPointCriticalMsgs, "\n")) 1863 } else { 1864 rc.checkTemplate.Collect(Critical, true, "checkpoints are valid") 1865 } 1866 if len(schemaCriticalMsgs) != 0 { 1867 rc.checkTemplate.Collect(Critical, false, strings.Join(schemaCriticalMsgs, "\n")) 1868 } else { 1869 rc.checkTemplate.Collect(Critical, true, "table schemas are valid") 1870 } 1871 return nil 1872 } 1873 1874 type chunkRestore struct { 1875 parser mydump.Parser 1876 index int 1877 chunk *checkpoints.ChunkCheckpoint 1878 } 1879 1880 func newChunkRestore( 1881 ctx context.Context, 1882 index int, 1883 cfg *config.Config, 1884 chunk *checkpoints.ChunkCheckpoint, 1885 ioWorkers *worker.Pool, 1886 store storage.ExternalStorage, 1887 tableInfo *checkpoints.TidbTableInfo, 1888 ) (*chunkRestore, error) { 1889 blockBufSize := int64(cfg.Mydumper.ReadBlockSize) 1890 1891 var reader storage.ReadSeekCloser 1892 var err error 1893 if chunk.FileMeta.Type == mydump.SourceTypeParquet { 1894 reader, err = mydump.OpenParquetReader(ctx, store, chunk.FileMeta.Path, chunk.FileMeta.FileSize) 1895 } else { 1896 reader, err = store.Open(ctx, chunk.FileMeta.Path) 1897 } 1898 if err != nil { 1899 return nil, errors.Trace(err) 1900 } 1901 1902 var parser mydump.Parser 1903 switch chunk.FileMeta.Type { 1904 case mydump.SourceTypeCSV: 1905 hasHeader := cfg.Mydumper.CSV.Header && chunk.Chunk.Offset == 0 1906 parser = mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, blockBufSize, ioWorkers, hasHeader) 1907 case mydump.SourceTypeSQL: 1908 parser = mydump.NewChunkParser(cfg.TiDB.SQLMode, reader, blockBufSize, ioWorkers) 1909 case mydump.SourceTypeParquet: 1910 parser, err = mydump.NewParquetParser(ctx, store, reader, chunk.FileMeta.Path) 1911 if err != nil { 1912 return nil, errors.Trace(err) 1913 } 1914 default: 1915 panic(fmt.Sprintf("file '%s' with unknown source type '%s'", chunk.Key.Path, chunk.FileMeta.Type.String())) 1916 } 1917 1918 if err = parser.SetPos(chunk.Chunk.Offset, chunk.Chunk.PrevRowIDMax); err != nil { 1919 return nil, errors.Trace(err) 1920 } 1921 if len(chunk.ColumnPermutation) > 0 { 1922 parser.SetColumns(getColumnNames(tableInfo.Core, chunk.ColumnPermutation)) 1923 } 1924 1925 return &chunkRestore{ 1926 parser: parser, 1927 index: index, 1928 chunk: chunk, 1929 }, nil 1930 } 1931 1932 func (cr *chunkRestore) close() { 1933 cr.parser.Close() 1934 } 1935 1936 func getColumnNames(tableInfo *model.TableInfo, permutation []int) []string { 1937 colIndexes := make([]int, 0, len(permutation)) 1938 for i := 0; i < len(permutation); i++ { 1939 colIndexes = append(colIndexes, -1) 1940 } 1941 colCnt := 0 1942 for i, p := range permutation { 1943 if p >= 0 { 1944 colIndexes[p] = i 1945 colCnt++ 1946 } 1947 } 1948 1949 names := make([]string, 0, colCnt) 1950 for _, idx := range colIndexes { 1951 // skip columns with index -1 1952 if idx >= 0 { 1953 // original fields contains _tidb_rowid field 1954 if idx == len(tableInfo.Columns) { 1955 names = append(names, model.ExtraHandleName.O) 1956 } else { 1957 names = append(names, tableInfo.Columns[idx].Name.O) 1958 } 1959 } 1960 } 1961 return names 1962 } 1963 1964 var ( 1965 maxKVQueueSize = 32 // Cache at most this number of rows before blocking the encode loop 1966 minDeliverBytes uint64 = 96 * units.KiB // 96 KB (data + index). batch at least this amount of bytes to reduce number of messages 1967 ) 1968 1969 type deliveredKVs struct { 1970 kvs kv.Row // if kvs is nil, this indicated we've got the last message. 1971 columns []string 1972 offset int64 1973 rowID int64 1974 } 1975 1976 type deliverResult struct { 1977 totalDur time.Duration 1978 err error 1979 } 1980 1981 //nolint:nakedret // TODO: refactor 1982 func (cr *chunkRestore) deliverLoop( 1983 ctx context.Context, 1984 kvsCh <-chan []deliveredKVs, 1985 t *TableRestore, 1986 engineID int32, 1987 dataEngine, indexEngine *backend.LocalEngineWriter, 1988 rc *Controller, 1989 ) (deliverTotalDur time.Duration, err error) { 1990 var channelClosed bool 1991 1992 deliverLogger := t.logger.With( 1993 zap.Int32("engineNumber", engineID), 1994 zap.Int("fileIndex", cr.index), 1995 zap.Stringer("path", &cr.chunk.Key), 1996 zap.String("task", "deliver"), 1997 ) 1998 // Fetch enough KV pairs from the source. 1999 dataKVs := rc.backend.MakeEmptyRows() 2000 indexKVs := rc.backend.MakeEmptyRows() 2001 2002 dataSynced := true 2003 for !channelClosed { 2004 var dataChecksum, indexChecksum verify.KVChecksum 2005 var columns []string 2006 var kvPacket []deliveredKVs 2007 // init these two field as checkpoint current value, so even if there are no kv pairs delivered, 2008 // chunk checkpoint should stay the same 2009 offset := cr.chunk.Chunk.Offset 2010 rowID := cr.chunk.Chunk.PrevRowIDMax 2011 2012 populate: 2013 for dataChecksum.SumSize()+indexChecksum.SumSize() < minDeliverBytes { 2014 select { 2015 case kvPacket = <-kvsCh: 2016 if len(kvPacket) == 0 { 2017 channelClosed = true 2018 break populate 2019 } 2020 for _, p := range kvPacket { 2021 p.kvs.ClassifyAndAppend(&dataKVs, &dataChecksum, &indexKVs, &indexChecksum) 2022 columns = p.columns 2023 offset = p.offset 2024 rowID = p.rowID 2025 } 2026 case <-ctx.Done(): 2027 err = ctx.Err() 2028 return 2029 } 2030 } 2031 2032 err = func() error { 2033 // We use `TryRLock` with sleep here to avoid blocking current goroutine during importing when disk-quota is 2034 // triggered, so that we can save chunkCheckpoint as soon as possible after `FlushEngine` is called. 2035 // This implementation may not be very elegant or even completely correct, but it is currently a relatively 2036 // simple and effective solution. 2037 for !rc.diskQuotaLock.TryRLock() { 2038 // try to update chunk checkpoint, this can help save checkpoint after importing when disk-quota is triggered 2039 if !dataSynced { 2040 dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine) 2041 } 2042 time.Sleep(time.Millisecond) 2043 } 2044 defer rc.diskQuotaLock.RUnlock() 2045 2046 // Write KVs into the engine 2047 start := time.Now() 2048 2049 if err = dataEngine.WriteRows(ctx, columns, dataKVs); err != nil { 2050 if !common.IsContextCanceledError(err) { 2051 deliverLogger.Error("write to data engine failed", log.ShortError(err)) 2052 } 2053 2054 return errors.Trace(err) 2055 } 2056 if err = indexEngine.WriteRows(ctx, columns, indexKVs); err != nil { 2057 if !common.IsContextCanceledError(err) { 2058 deliverLogger.Error("write to index engine failed", log.ShortError(err)) 2059 } 2060 return errors.Trace(err) 2061 } 2062 2063 deliverDur := time.Since(start) 2064 deliverTotalDur += deliverDur 2065 metric.BlockDeliverSecondsHistogram.Observe(deliverDur.Seconds()) 2066 metric.BlockDeliverBytesHistogram.WithLabelValues(metric.BlockDeliverKindData).Observe(float64(dataChecksum.SumSize())) 2067 metric.BlockDeliverBytesHistogram.WithLabelValues(metric.BlockDeliverKindIndex).Observe(float64(indexChecksum.SumSize())) 2068 metric.BlockDeliverKVPairsHistogram.WithLabelValues(metric.BlockDeliverKindData).Observe(float64(dataChecksum.SumKVS())) 2069 metric.BlockDeliverKVPairsHistogram.WithLabelValues(metric.BlockDeliverKindIndex).Observe(float64(indexChecksum.SumKVS())) 2070 return nil 2071 }() 2072 if err != nil { 2073 return 2074 } 2075 dataSynced = false 2076 2077 dataKVs = dataKVs.Clear() 2078 indexKVs = indexKVs.Clear() 2079 2080 // Update the table, and save a checkpoint. 2081 // (the write to the importer is effective immediately, thus update these here) 2082 // No need to apply a lock since this is the only thread updating `cr.chunk.**`. 2083 // In local mode, we should write these checkpoint after engine flushed. 2084 cr.chunk.Checksum.Add(&dataChecksum) 2085 cr.chunk.Checksum.Add(&indexChecksum) 2086 cr.chunk.Chunk.Offset = offset 2087 cr.chunk.Chunk.PrevRowIDMax = rowID 2088 2089 if dataChecksum.SumKVS() != 0 || indexChecksum.SumKVS() != 0 { 2090 // No need to save checkpoint if nothing was delivered. 2091 dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine) 2092 } 2093 failpoint.Inject("SlowDownWriteRows", func() { 2094 deliverLogger.Warn("Slowed down write rows") 2095 }) 2096 failpoint.Inject("FailAfterWriteRows", nil) 2097 // TODO: for local backend, we may save checkpoint more frequently, e.g. after written 2098 // 10GB kv pairs to data engine, we can do a flush for both data & index engine, then we 2099 // can safely update current checkpoint. 2100 2101 failpoint.Inject("LocalBackendSaveCheckpoint", func() { 2102 if !rc.isLocalBackend() && (dataChecksum.SumKVS() != 0 || indexChecksum.SumKVS() != 0) { 2103 // No need to save checkpoint if nothing was delivered. 2104 saveCheckpoint(rc, t, engineID, cr.chunk) 2105 } 2106 }) 2107 } 2108 2109 return 2110 } 2111 2112 func (cr *chunkRestore) maybeSaveCheckpoint( 2113 rc *Controller, 2114 t *TableRestore, 2115 engineID int32, 2116 chunk *checkpoints.ChunkCheckpoint, 2117 data, index *backend.LocalEngineWriter, 2118 ) bool { 2119 if data.IsSynced() && index.IsSynced() { 2120 saveCheckpoint(rc, t, engineID, chunk) 2121 return true 2122 } 2123 return false 2124 } 2125 2126 func saveCheckpoint(rc *Controller, t *TableRestore, engineID int32, chunk *checkpoints.ChunkCheckpoint) { 2127 // We need to update the AllocBase every time we've finished a file. 2128 // The AllocBase is determined by the maximum of the "handle" (_tidb_rowid 2129 // or integer primary key), which can only be obtained by reading all data. 2130 2131 var base int64 2132 if t.tableInfo.Core.PKIsHandle && t.tableInfo.Core.ContainsAutoRandomBits() { 2133 base = t.alloc.Get(autoid.AutoRandomType).Base() + 1 2134 } else { 2135 base = t.alloc.Get(autoid.RowIDAllocType).Base() + 1 2136 } 2137 rc.saveCpCh <- saveCp{ 2138 tableName: t.tableName, 2139 merger: &checkpoints.RebaseCheckpointMerger{ 2140 AllocBase: base, 2141 }, 2142 } 2143 rc.saveCpCh <- saveCp{ 2144 tableName: t.tableName, 2145 merger: &checkpoints.ChunkCheckpointMerger{ 2146 EngineID: engineID, 2147 Key: chunk.Key, 2148 Checksum: chunk.Checksum, 2149 Pos: chunk.Chunk.Offset, 2150 RowID: chunk.Chunk.PrevRowIDMax, 2151 ColumnPermutation: chunk.ColumnPermutation, 2152 }, 2153 } 2154 } 2155 2156 //nolint:nakedret // TODO: refactor 2157 func (cr *chunkRestore) encodeLoop( 2158 ctx context.Context, 2159 kvsCh chan<- []deliveredKVs, 2160 t *TableRestore, 2161 logger log.Logger, 2162 kvEncoder kv.Encoder, 2163 deliverCompleteCh <-chan deliverResult, 2164 rc *Controller, 2165 ) (readTotalDur time.Duration, encodeTotalDur time.Duration, err error) { 2166 send := func(kvs []deliveredKVs) error { 2167 select { 2168 case kvsCh <- kvs: 2169 return nil 2170 case <-ctx.Done(): 2171 return ctx.Err() 2172 case deliverResult, ok := <-deliverCompleteCh: 2173 if deliverResult.err == nil && !ok { 2174 deliverResult.err = ctx.Err() 2175 } 2176 if deliverResult.err == nil { 2177 deliverResult.err = errors.New("unexpected premature fulfillment") 2178 logger.DPanic("unexpected: deliverCompleteCh prematurely fulfilled with no error", zap.Bool("chIsOpen", ok)) 2179 } 2180 return errors.Trace(deliverResult.err) 2181 } 2182 } 2183 2184 pauser, maxKvPairsCnt := rc.pauser, rc.cfg.TikvImporter.MaxKVPairs 2185 initializedColumns, reachEOF := false, false 2186 for !reachEOF { 2187 if err = pauser.Wait(ctx); err != nil { 2188 return 2189 } 2190 offset, _ := cr.parser.Pos() 2191 if offset >= cr.chunk.Chunk.EndOffset { 2192 break 2193 } 2194 2195 var readDur, encodeDur time.Duration 2196 canDeliver := false 2197 kvPacket := make([]deliveredKVs, 0, maxKvPairsCnt) 2198 curOffset := offset 2199 var newOffset, rowID int64 2200 var kvSize uint64 2201 outLoop: 2202 for !canDeliver { 2203 readDurStart := time.Now() 2204 err = cr.parser.ReadRow() 2205 columnNames := cr.parser.Columns() 2206 newOffset, rowID = cr.parser.Pos() 2207 2208 switch errors.Cause(err) { 2209 case nil: 2210 if !initializedColumns { 2211 if len(cr.chunk.ColumnPermutation) == 0 { 2212 if err = t.initializeColumns(columnNames, cr.chunk); err != nil { 2213 return 2214 } 2215 } 2216 initializedColumns = true 2217 } 2218 case io.EOF: 2219 reachEOF = true 2220 break outLoop 2221 default: 2222 err = errors.Annotatef(err, "in file %s at offset %d", &cr.chunk.Key, newOffset) 2223 return 2224 } 2225 readDur += time.Since(readDurStart) 2226 encodeDurStart := time.Now() 2227 lastRow := cr.parser.LastRow() 2228 // sql -> kv 2229 kvs, encodeErr := kvEncoder.Encode(logger, lastRow.Row, lastRow.RowID, cr.chunk.ColumnPermutation, curOffset) 2230 encodeDur += time.Since(encodeDurStart) 2231 cr.parser.RecycleRow(lastRow) 2232 if encodeErr != nil { 2233 err = errors.Annotatef(encodeErr, "in file %s at offset %d", &cr.chunk.Key, newOffset) 2234 return 2235 } 2236 kvPacket = append(kvPacket, deliveredKVs{kvs: kvs, columns: columnNames, offset: newOffset, rowID: rowID}) 2237 kvSize += kvs.Size() 2238 failpoint.Inject("mock-kv-size", func(val failpoint.Value) { 2239 kvSize += uint64(val.(int)) 2240 }) 2241 // pebble cannot allow > 4.0G kv in one batch. 2242 // we will meet pebble panic when import sql file and each kv has the size larger than 4G / maxKvPairsCnt. 2243 // so add this check. 2244 if kvSize >= minDeliverBytes || len(kvPacket) >= maxKvPairsCnt || newOffset == cr.chunk.Chunk.EndOffset { 2245 canDeliver = true 2246 kvSize = 0 2247 } 2248 curOffset = newOffset 2249 } 2250 encodeTotalDur += encodeDur 2251 metric.RowEncodeSecondsHistogram.Observe(encodeDur.Seconds()) 2252 readTotalDur += readDur 2253 metric.RowReadSecondsHistogram.Observe(readDur.Seconds()) 2254 metric.RowReadBytesHistogram.Observe(float64(newOffset - offset)) 2255 2256 if len(kvPacket) != 0 { 2257 deliverKvStart := time.Now() 2258 if err = send(kvPacket); err != nil { 2259 return 2260 } 2261 metric.RowKVDeliverSecondsHistogram.Observe(time.Since(deliverKvStart).Seconds()) 2262 } 2263 } 2264 2265 err = send([]deliveredKVs{}) 2266 return 2267 } 2268 2269 func (cr *chunkRestore) restore( 2270 ctx context.Context, 2271 t *TableRestore, 2272 engineID int32, 2273 dataEngine, indexEngine *backend.LocalEngineWriter, 2274 rc *Controller, 2275 ) error { 2276 // Create the encoder. 2277 kvEncoder, err := rc.backend.NewEncoder(t.encTable, &kv.SessionOptions{ 2278 SQLMode: rc.cfg.TiDB.SQLMode, 2279 Timestamp: cr.chunk.Timestamp, 2280 SysVars: rc.sysVars, 2281 // use chunk.PrevRowIDMax as the auto random seed, so it can stay the same value after recover from checkpoint. 2282 AutoRandomSeed: cr.chunk.Chunk.PrevRowIDMax, 2283 }) 2284 if err != nil { 2285 return err 2286 } 2287 2288 kvsCh := make(chan []deliveredKVs, maxKVQueueSize) 2289 deliverCompleteCh := make(chan deliverResult) 2290 2291 defer func() { 2292 kvEncoder.Close() 2293 kvEncoder = nil 2294 close(kvsCh) 2295 }() 2296 2297 go func() { 2298 defer close(deliverCompleteCh) 2299 dur, err := cr.deliverLoop(ctx, kvsCh, t, engineID, dataEngine, indexEngine, rc) 2300 select { 2301 case <-ctx.Done(): 2302 case deliverCompleteCh <- deliverResult{dur, err}: 2303 } 2304 }() 2305 2306 logTask := t.logger.With( 2307 zap.Int32("engineNumber", engineID), 2308 zap.Int("fileIndex", cr.index), 2309 zap.Stringer("path", &cr.chunk.Key), 2310 ).Begin(zap.InfoLevel, "restore file") 2311 2312 readTotalDur, encodeTotalDur, err := cr.encodeLoop(ctx, kvsCh, t, logTask.Logger, kvEncoder, deliverCompleteCh, rc) 2313 if err != nil { 2314 return err 2315 } 2316 2317 select { 2318 case deliverResult, ok := <-deliverCompleteCh: 2319 if ok { 2320 logTask.End(zap.ErrorLevel, deliverResult.err, 2321 zap.Duration("readDur", readTotalDur), 2322 zap.Duration("encodeDur", encodeTotalDur), 2323 zap.Duration("deliverDur", deliverResult.totalDur), 2324 zap.Object("checksum", &cr.chunk.Checksum), 2325 ) 2326 return errors.Trace(deliverResult.err) 2327 } 2328 // else, this must cause by ctx cancel 2329 return ctx.Err() 2330 case <-ctx.Done(): 2331 return ctx.Err() 2332 } 2333 }