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