github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/checkpoints/checkpoints.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 checkpoints 15 16 import ( 17 "context" 18 "database/sql" 19 "encoding/json" 20 "fmt" 21 "io" 22 "io/ioutil" 23 "math" 24 "os" 25 "sort" 26 "strings" 27 "sync" 28 29 "github.com/joho/sqltocsv" 30 "github.com/pingcap/errors" 31 "go.uber.org/zap" 32 "modernc.org/mathutil" 33 34 "github.com/pingcap/tidb-lightning/lightning/common" 35 "github.com/pingcap/tidb-lightning/lightning/config" 36 "github.com/pingcap/tidb-lightning/lightning/log" 37 "github.com/pingcap/tidb-lightning/lightning/mydump" 38 verify "github.com/pingcap/tidb-lightning/lightning/verification" 39 ) 40 41 type CheckpointStatus uint8 42 43 const ( 44 CheckpointStatusMissing CheckpointStatus = 0 45 CheckpointStatusMaxInvalid CheckpointStatus = 25 46 CheckpointStatusLoaded CheckpointStatus = 30 47 CheckpointStatusAllWritten CheckpointStatus = 60 48 CheckpointStatusClosed CheckpointStatus = 90 49 CheckpointStatusImported CheckpointStatus = 120 50 CheckpointStatusIndexImported CheckpointStatus = 140 51 CheckpointStatusAlteredAutoInc CheckpointStatus = 150 52 CheckpointStatusChecksumSkipped CheckpointStatus = 170 53 CheckpointStatusChecksummed CheckpointStatus = 180 54 CheckpointStatusAnalyzeSkipped CheckpointStatus = 200 55 CheckpointStatusAnalyzed CheckpointStatus = 210 56 ) 57 58 const WholeTableEngineID = math.MaxInt32 59 60 const ( 61 // the table names to store each kind of checkpoint in the checkpoint database 62 // remember to increase the version number in case of incompatible change. 63 CheckpointTableNameTask = "task_v2" 64 CheckpointTableNameTable = "table_v6" 65 CheckpointTableNameEngine = "engine_v5" 66 CheckpointTableNameChunk = "chunk_v5" 67 ) 68 69 const ( 70 // shared by MySQLCheckpointsDB and GlueCheckpointsDB 71 CreateDBTemplate = "CREATE DATABASE IF NOT EXISTS %s;" 72 CreateTaskTableTemplate = ` 73 CREATE TABLE IF NOT EXISTS %s.%s ( 74 id tinyint(1) PRIMARY KEY, 75 task_id bigint NOT NULL, 76 source_dir varchar(256) NOT NULL, 77 backend varchar(16) NOT NULL, 78 importer_addr varchar(256), 79 tidb_host varchar(128) NOT NULL, 80 tidb_port int NOT NULL, 81 pd_addr varchar(128) NOT NULL, 82 sorted_kv_dir varchar(256) NOT NULL, 83 lightning_ver varchar(48) NOT NULL 84 );` 85 CreateTableTableTemplate = ` 86 CREATE TABLE IF NOT EXISTS %s.%s ( 87 task_id bigint NOT NULL, 88 table_name varchar(261) NOT NULL PRIMARY KEY, 89 hash binary(32) NOT NULL, 90 status tinyint unsigned DEFAULT 30, 91 alloc_base bigint NOT NULL DEFAULT 0, 92 table_id bigint NOT NULL DEFAULT 0, 93 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 94 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 95 INDEX(task_id) 96 );` 97 CreateEngineTableTemplate = ` 98 CREATE TABLE IF NOT EXISTS %s.%s ( 99 table_name varchar(261) NOT NULL, 100 engine_id int NOT NULL, 101 status tinyint unsigned DEFAULT 30, 102 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 103 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 104 PRIMARY KEY(table_name, engine_id DESC) 105 );` 106 CreateChunkTableTemplate = ` 107 CREATE TABLE IF NOT EXISTS %s.%s ( 108 table_name varchar(261) NOT NULL, 109 engine_id int unsigned NOT NULL, 110 path varchar(2048) NOT NULL, 111 offset bigint NOT NULL, 112 type int NOT NULL, 113 compression int NOT NULL, 114 sort_key varchar(256) NOT NULL, 115 file_size bigint NOT NULL, 116 columns text NULL, 117 should_include_row_id BOOL NOT NULL, 118 end_offset bigint NOT NULL, 119 pos bigint NOT NULL, 120 prev_rowid_max bigint NOT NULL, 121 rowid_max bigint NOT NULL, 122 kvc_bytes bigint unsigned NOT NULL DEFAULT 0, 123 kvc_kvs bigint unsigned NOT NULL DEFAULT 0, 124 kvc_checksum bigint unsigned NOT NULL DEFAULT 0, 125 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 126 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 127 PRIMARY KEY(table_name, engine_id, path(500), offset) 128 );` 129 InitTaskTemplate = ` 130 REPLACE INTO %s.%s (id, task_id, source_dir, backend, importer_addr, tidb_host, tidb_port, pd_addr, sorted_kv_dir, lightning_ver) 131 VALUES (1, ?, ?, ?, ?, ?, ?, ?, ?, ?);` 132 InitTableTemplate = ` 133 INSERT INTO %s.%s (task_id, table_name, hash, table_id) VALUES (?, ?, ?, ?) 134 ON DUPLICATE KEY UPDATE task_id = CASE 135 WHEN hash = VALUES(hash) 136 THEN VALUES(task_id) 137 END;` 138 ReadTaskTemplate = ` 139 SELECT task_id, source_dir, backend, importer_addr, tidb_host, tidb_port, pd_addr, sorted_kv_dir, lightning_ver FROM %s.%s WHERE id = 1;` 140 ReadEngineTemplate = ` 141 SELECT engine_id, status FROM %s.%s WHERE table_name = ? ORDER BY engine_id DESC;` 142 ReadChunkTemplate = ` 143 SELECT 144 engine_id, path, offset, type, compression, sort_key, file_size, columns, 145 pos, end_offset, prev_rowid_max, rowid_max, 146 kvc_bytes, kvc_kvs, kvc_checksum, unix_timestamp(create_time) 147 FROM %s.%s WHERE table_name = ? 148 ORDER BY engine_id, path, offset;` 149 ReadTableRemainTemplate = ` 150 SELECT status, alloc_base, table_id FROM %s.%s WHERE table_name = ?;` 151 ReplaceEngineTemplate = ` 152 REPLACE INTO %s.%s (table_name, engine_id, status) VALUES (?, ?, ?);` 153 ReplaceChunkTemplate = ` 154 REPLACE INTO %s.%s ( 155 table_name, engine_id, 156 path, offset, type, compression, sort_key, file_size, columns, should_include_row_id, 157 pos, end_offset, prev_rowid_max, rowid_max, 158 kvc_bytes, kvc_kvs, kvc_checksum, create_time 159 ) VALUES ( 160 ?, ?, 161 ?, ?, ?, ?, ?, ?, ?, FALSE, 162 ?, ?, ?, ?, 163 0, 0, 0, from_unixtime(?) 164 );` 165 UpdateChunkTemplate = ` 166 UPDATE %s.%s SET pos = ?, prev_rowid_max = ?, kvc_bytes = ?, kvc_kvs = ?, kvc_checksum = ?, columns = ? 167 WHERE (table_name, engine_id, path, offset) = (?, ?, ?, ?);` 168 UpdateTableRebaseTemplate = ` 169 UPDATE %s.%s SET alloc_base = GREATEST(?, alloc_base) WHERE table_name = ?;` 170 UpdateTableStatusTemplate = ` 171 UPDATE %s.%s SET status = ? WHERE table_name = ?;` 172 UpdateEngineTemplate = ` 173 UPDATE %s.%s SET status = ? WHERE (table_name, engine_id) = (?, ?);` 174 DeleteCheckpointRecordTemplate = "DELETE FROM %s.%s WHERE table_name = ?;" 175 ) 176 177 func IsCheckpointTable(name string) bool { 178 switch name { 179 case CheckpointTableNameTask, CheckpointTableNameTable, CheckpointTableNameEngine, CheckpointTableNameChunk: 180 return true 181 default: 182 return false 183 } 184 } 185 186 func (status CheckpointStatus) MetricName() string { 187 switch status { 188 case CheckpointStatusLoaded: 189 return "pending" 190 case CheckpointStatusAllWritten: 191 return "written" 192 case CheckpointStatusClosed: 193 return "closed" 194 case CheckpointStatusImported: 195 return "imported" 196 case CheckpointStatusIndexImported: 197 return "index_imported" 198 case CheckpointStatusAlteredAutoInc: 199 return "altered_auto_inc" 200 case CheckpointStatusChecksummed, CheckpointStatusChecksumSkipped: 201 return "checksum" 202 case CheckpointStatusAnalyzed, CheckpointStatusAnalyzeSkipped: 203 return "analyzed" 204 case CheckpointStatusMissing: 205 return "missing" 206 default: 207 return "invalid" 208 } 209 } 210 211 type ChunkCheckpointKey struct { 212 Path string 213 Offset int64 214 } 215 216 func (key *ChunkCheckpointKey) String() string { 217 return fmt.Sprintf("%s:%d", key.Path, key.Offset) 218 } 219 220 func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) bool { 221 switch { 222 case key.Path < other.Path: 223 return true 224 case key.Path > other.Path: 225 return false 226 default: 227 return key.Offset < other.Offset 228 } 229 } 230 231 type ChunkCheckpoint struct { 232 Key ChunkCheckpointKey 233 FileMeta mydump.SourceFileMeta 234 ColumnPermutation []int 235 Chunk mydump.Chunk 236 Checksum verify.KVChecksum 237 Timestamp int64 238 } 239 240 func (ccp *ChunkCheckpoint) DeepCopy() *ChunkCheckpoint { 241 colPerm := make([]int, 0, len(ccp.ColumnPermutation)) 242 colPerm = append(colPerm, ccp.ColumnPermutation...) 243 return &ChunkCheckpoint{ 244 Key: ccp.Key, 245 FileMeta: ccp.FileMeta, 246 ColumnPermutation: colPerm, 247 Chunk: ccp.Chunk, 248 Checksum: ccp.Checksum, 249 Timestamp: ccp.Timestamp, 250 } 251 } 252 253 type EngineCheckpoint struct { 254 Status CheckpointStatus 255 Chunks []*ChunkCheckpoint // a sorted array 256 } 257 258 func (engine *EngineCheckpoint) DeepCopy() *EngineCheckpoint { 259 chunks := make([]*ChunkCheckpoint, 0, len(engine.Chunks)) 260 for _, chunk := range engine.Chunks { 261 chunks = append(chunks, chunk.DeepCopy()) 262 } 263 return &EngineCheckpoint{ 264 Status: engine.Status, 265 Chunks: chunks, 266 } 267 } 268 269 type TableCheckpoint struct { 270 Status CheckpointStatus 271 AllocBase int64 272 Engines map[int32]*EngineCheckpoint 273 TableID int64 274 } 275 276 func (cp *TableCheckpoint) DeepCopy() *TableCheckpoint { 277 engines := make(map[int32]*EngineCheckpoint, len(cp.Engines)) 278 for engineID, engine := range cp.Engines { 279 engines[engineID] = engine.DeepCopy() 280 } 281 return &TableCheckpoint{ 282 Status: cp.Status, 283 AllocBase: cp.AllocBase, 284 Engines: engines, 285 TableID: cp.TableID, 286 } 287 } 288 func (cp *TableCheckpoint) CountChunks() int { 289 result := 0 290 for _, engine := range cp.Engines { 291 result += len(engine.Chunks) 292 } 293 return result 294 } 295 296 type chunkCheckpointDiff struct { 297 pos int64 298 rowID int64 299 checksum verify.KVChecksum 300 columnPermutation []int 301 } 302 303 type engineCheckpointDiff struct { 304 hasStatus bool 305 status CheckpointStatus 306 chunks map[ChunkCheckpointKey]chunkCheckpointDiff 307 } 308 309 type TableCheckpointDiff struct { 310 hasStatus bool 311 hasRebase bool 312 status CheckpointStatus 313 allocBase int64 314 engines map[int32]engineCheckpointDiff 315 } 316 317 func NewTableCheckpointDiff() *TableCheckpointDiff { 318 return &TableCheckpointDiff{ 319 engines: make(map[int32]engineCheckpointDiff), 320 } 321 } 322 323 func (cpd *TableCheckpointDiff) insertEngineCheckpointDiff(engineID int32, newDiff engineCheckpointDiff) { 324 if oldDiff, ok := cpd.engines[engineID]; ok { 325 if newDiff.hasStatus { 326 oldDiff.hasStatus = true 327 oldDiff.status = newDiff.status 328 } 329 for key, chunkDiff := range newDiff.chunks { 330 oldDiff.chunks[key] = chunkDiff 331 } 332 newDiff = oldDiff 333 } 334 cpd.engines[engineID] = newDiff 335 } 336 337 func (cpd *TableCheckpointDiff) String() string { 338 return fmt.Sprintf( 339 "{hasStatus:%v, hasRebase:%v, status:%d, allocBase:%d, engines:[%d]}", 340 cpd.hasStatus, cpd.hasRebase, cpd.status, cpd.allocBase, len(cpd.engines), 341 ) 342 } 343 344 // Apply the diff to the existing chunk and engine checkpoints in `cp`. 345 func (cp *TableCheckpoint) Apply(cpd *TableCheckpointDiff) { 346 if cpd.hasStatus { 347 cp.Status = cpd.status 348 } 349 if cpd.hasRebase { 350 cp.AllocBase = cpd.allocBase 351 } 352 for engineID, engineDiff := range cpd.engines { 353 engine := cp.Engines[engineID] 354 if engine == nil { 355 continue 356 } 357 if engineDiff.hasStatus { 358 engine.Status = engineDiff.status 359 } 360 for key, diff := range engineDiff.chunks { 361 index := sort.Search(len(engine.Chunks), func(i int) bool { 362 return !engine.Chunks[i].Key.less(&key) 363 }) 364 if index >= len(engine.Chunks) { 365 continue 366 } 367 chunk := engine.Chunks[index] 368 if chunk.Key != key { 369 continue 370 } 371 chunk.Chunk.Offset = diff.pos 372 chunk.Chunk.PrevRowIDMax = diff.rowID 373 chunk.Checksum = diff.checksum 374 } 375 } 376 } 377 378 type TableCheckpointMerger interface { 379 // MergeInto the table checkpoint diff from a status update or chunk update. 380 // If there are multiple updates to the same table, only the last one will 381 // take effect. Therefore, the caller must ensure events for the same table 382 // are properly ordered by the global time (an old event must be merged 383 // before the new one). 384 MergeInto(cpd *TableCheckpointDiff) 385 } 386 387 type StatusCheckpointMerger struct { 388 EngineID int32 // WholeTableEngineID == apply to whole table. 389 Status CheckpointStatus 390 } 391 392 func (merger *StatusCheckpointMerger) SetInvalid() { 393 merger.Status /= 10 394 } 395 396 func (merger *StatusCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) { 397 if merger.EngineID == WholeTableEngineID || merger.Status <= CheckpointStatusMaxInvalid { 398 cpd.status = merger.Status 399 cpd.hasStatus = true 400 } 401 if merger.EngineID != WholeTableEngineID { 402 cpd.insertEngineCheckpointDiff(merger.EngineID, engineCheckpointDiff{ 403 hasStatus: true, 404 status: merger.Status, 405 chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), 406 }) 407 } 408 } 409 410 type ChunkCheckpointMerger struct { 411 EngineID int32 412 Key ChunkCheckpointKey 413 Checksum verify.KVChecksum 414 Pos int64 415 RowID int64 416 ColumnPermutation []int 417 } 418 419 func (merger *ChunkCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) { 420 cpd.insertEngineCheckpointDiff(merger.EngineID, engineCheckpointDiff{ 421 chunks: map[ChunkCheckpointKey]chunkCheckpointDiff{ 422 merger.Key: { 423 pos: merger.Pos, 424 rowID: merger.RowID, 425 checksum: merger.Checksum, 426 columnPermutation: merger.ColumnPermutation, 427 }, 428 }, 429 }) 430 } 431 432 type RebaseCheckpointMerger struct { 433 AllocBase int64 434 } 435 436 func (merger *RebaseCheckpointMerger) MergeInto(cpd *TableCheckpointDiff) { 437 cpd.hasRebase = true 438 cpd.allocBase = mathutil.MaxInt64(cpd.allocBase, merger.AllocBase) 439 } 440 441 type DestroyedTableCheckpoint struct { 442 TableName string 443 MinEngineID int32 444 MaxEngineID int32 445 } 446 447 type TaskCheckpoint struct { 448 TaskId int64 449 SourceDir string 450 Backend string 451 ImporterAddr string 452 TiDBHost string 453 TiDBPort int 454 PdAddr string 455 SortedKVDir string 456 LightningVer string 457 } 458 459 type CheckpointsDB interface { 460 Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error 461 TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) 462 Get(ctx context.Context, tableName string) (*TableCheckpoint, error) 463 Close() error 464 // InsertEngineCheckpoints initializes the checkpoints related to a table. 465 // It assumes the entire table has not been imported before and will fill in 466 // default values for the column permutations and checksums. 467 InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error 468 Update(checkpointDiffs map[string]*TableCheckpointDiff) 469 470 RemoveCheckpoint(ctx context.Context, tableName string) error 471 // MoveCheckpoints renames the checkpoint schema to include a suffix 472 // including the taskID (e.g. `tidb_lightning_checkpoints.1234567890.bak`). 473 MoveCheckpoints(ctx context.Context, taskID int64) error 474 // GetLocalStoringTables returns a map containing tables have engine files stored on local disk. 475 // currently only meaningful for local backend 476 GetLocalStoringTables(ctx context.Context) (map[string][]int32, error) 477 IgnoreErrorCheckpoint(ctx context.Context, tableName string) error 478 DestroyErrorCheckpoint(ctx context.Context, tableName string) ([]DestroyedTableCheckpoint, error) 479 DumpTables(ctx context.Context, csv io.Writer) error 480 DumpEngines(ctx context.Context, csv io.Writer) error 481 DumpChunks(ctx context.Context, csv io.Writer) error 482 } 483 484 func OpenCheckpointsDB(ctx context.Context, cfg *config.Config) (CheckpointsDB, error) { 485 if !cfg.Checkpoint.Enable { 486 return NewNullCheckpointsDB(), nil 487 } 488 489 switch cfg.Checkpoint.Driver { 490 case config.CheckpointDriverMySQL: 491 db, err := sql.Open("mysql", cfg.Checkpoint.DSN) 492 if err != nil { 493 return nil, errors.Trace(err) 494 } 495 cpdb, err := NewMySQLCheckpointsDB(ctx, db, cfg.Checkpoint.Schema) 496 if err != nil { 497 db.Close() 498 return nil, errors.Trace(err) 499 } 500 return cpdb, nil 501 502 case config.CheckpointDriverFile: 503 return NewFileCheckpointsDB(cfg.Checkpoint.DSN), nil 504 505 default: 506 return nil, errors.Errorf("Unknown checkpoint driver %s", cfg.Checkpoint.Driver) 507 } 508 } 509 510 func IsCheckpointsDBExists(ctx context.Context, cfg *config.Config) (bool, error) { 511 if !cfg.Checkpoint.Enable { 512 return false, nil 513 } 514 switch cfg.Checkpoint.Driver { 515 case config.CheckpointDriverMySQL: 516 db, err := sql.Open("mysql", cfg.Checkpoint.DSN) 517 if err != nil { 518 return false, errors.Trace(err) 519 } 520 defer db.Close() 521 checkSQL := "SHOW DATABASES WHERE `DATABASE` = ?" 522 rows, err := db.QueryContext(ctx, checkSQL, cfg.Checkpoint.Schema) 523 if err != nil { 524 return false, errors.Trace(err) 525 } 526 defer rows.Close() 527 return rows.Next(), nil 528 529 case config.CheckpointDriverFile: 530 _, err := os.Stat(cfg.Checkpoint.DSN) 531 if err == nil { 532 return true, err 533 } else if os.IsNotExist(err) { 534 return false, nil 535 } 536 return false, errors.Trace(err) 537 538 default: 539 return false, errors.Errorf("Unknown checkpoint driver %s", cfg.Checkpoint.Driver) 540 } 541 } 542 543 // NullCheckpointsDB is a checkpoints database with no checkpoints. 544 type NullCheckpointsDB struct{} 545 546 func NewNullCheckpointsDB() *NullCheckpointsDB { 547 return &NullCheckpointsDB{} 548 } 549 550 func (*NullCheckpointsDB) Initialize(context.Context, *config.Config, map[string]*TidbDBInfo) error { 551 return nil 552 } 553 554 func (*NullCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) { 555 return nil, nil 556 } 557 558 func (*NullCheckpointsDB) Close() error { 559 return nil 560 } 561 562 func (*NullCheckpointsDB) Get(_ context.Context, _ string) (*TableCheckpoint, error) { 563 return &TableCheckpoint{ 564 Status: CheckpointStatusLoaded, 565 Engines: map[int32]*EngineCheckpoint{}, 566 }, nil 567 } 568 569 func (*NullCheckpointsDB) InsertEngineCheckpoints(_ context.Context, _ string, _ map[int32]*EngineCheckpoint) error { 570 return nil 571 } 572 573 func (*NullCheckpointsDB) Update(map[string]*TableCheckpointDiff) {} 574 575 type MySQLCheckpointsDB struct { 576 db *sql.DB 577 schema string 578 } 579 580 func NewMySQLCheckpointsDB(ctx context.Context, db *sql.DB, schemaName string) (*MySQLCheckpointsDB, error) { 581 var escapedSchemaName strings.Builder 582 common.WriteMySQLIdentifier(&escapedSchemaName, schemaName) 583 schema := escapedSchemaName.String() 584 585 sql := common.SQLWithRetry{ 586 DB: db, 587 Logger: log.With(zap.String("schema", schemaName)), 588 HideQueryLog: true, 589 } 590 err := sql.Exec(ctx, "create checkpoints database", fmt.Sprintf(CreateDBTemplate, schema)) 591 if err != nil { 592 return nil, errors.Trace(err) 593 } 594 595 err = sql.Exec(ctx, "create task checkpoints table", fmt.Sprintf(CreateTaskTableTemplate, schema, CheckpointTableNameTask)) 596 if err != nil { 597 return nil, errors.Trace(err) 598 } 599 600 err = sql.Exec(ctx, "create table checkpoints table", fmt.Sprintf(CreateTableTableTemplate, schema, CheckpointTableNameTable)) 601 if err != nil { 602 return nil, errors.Trace(err) 603 } 604 605 err = sql.Exec(ctx, "create engine checkpoints table", fmt.Sprintf(CreateEngineTableTemplate, schema, CheckpointTableNameEngine)) 606 if err != nil { 607 return nil, errors.Trace(err) 608 } 609 610 err = sql.Exec(ctx, "create chunks checkpoints table", fmt.Sprintf(CreateChunkTableTemplate, schema, CheckpointTableNameChunk)) 611 if err != nil { 612 return nil, errors.Trace(err) 613 } 614 615 return &MySQLCheckpointsDB{ 616 db: db, 617 schema: schema, 618 }, nil 619 } 620 621 func (cpdb *MySQLCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error { 622 // We can have at most 65535 placeholders https://stackoverflow.com/q/4922345/ 623 // Since this step is not performance critical, we just insert the rows one-by-one. 624 s := common.SQLWithRetry{DB: cpdb.db, Logger: log.L()} 625 err := s.Transact(ctx, "insert checkpoints", func(c context.Context, tx *sql.Tx) error { 626 taskStmt, err := tx.PrepareContext(c, fmt.Sprintf(InitTaskTemplate, cpdb.schema, CheckpointTableNameTask)) 627 if err != nil { 628 return errors.Trace(err) 629 } 630 defer taskStmt.Close() 631 _, err = taskStmt.ExecContext(ctx, cfg.TaskID, cfg.Mydumper.SourceDir, cfg.TikvImporter.Backend, 632 cfg.TikvImporter.Addr, cfg.TiDB.Host, cfg.TiDB.Port, cfg.TiDB.PdAddr, cfg.TikvImporter.SortedKVDir, 633 common.ReleaseVersion) 634 if err != nil { 635 return errors.Trace(err) 636 } 637 638 // If `hash` is not the same but the `table_name` duplicates, 639 // the CASE expression will return NULL, which can be used to violate 640 // the NOT NULL requirement of `task_id` column, and caused this INSERT 641 // statement to fail with an irrecoverable error. 642 // We do need to capture the error is display a user friendly message 643 // (multiple nodes cannot import the same table) though. 644 stmt, err := tx.PrepareContext(c, fmt.Sprintf(InitTableTemplate, cpdb.schema, CheckpointTableNameTable)) 645 if err != nil { 646 return errors.Trace(err) 647 } 648 defer stmt.Close() 649 650 for _, db := range dbInfo { 651 for _, table := range db.Tables { 652 tableName := common.UniqueTable(db.Name, table.Name) 653 _, err = stmt.ExecContext(c, cfg.TaskID, tableName, 0, table.ID) 654 if err != nil { 655 return errors.Trace(err) 656 } 657 } 658 } 659 660 return nil 661 }) 662 if err != nil { 663 return errors.Trace(err) 664 } 665 666 return nil 667 } 668 669 func (cpdb *MySQLCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) { 670 s := common.SQLWithRetry{ 671 DB: cpdb.db, 672 Logger: log.L(), 673 } 674 675 taskQuery := fmt.Sprintf(ReadTaskTemplate, cpdb.schema, CheckpointTableNameTask) 676 taskCp := &TaskCheckpoint{} 677 err := s.QueryRow(ctx, "fetch task checkpoint", taskQuery, &taskCp.TaskId, &taskCp.SourceDir, &taskCp.Backend, 678 &taskCp.ImporterAddr, &taskCp.TiDBHost, &taskCp.TiDBPort, &taskCp.PdAddr, &taskCp.SortedKVDir, &taskCp.LightningVer) 679 680 if err != nil { 681 // if task checkpoint is empty, return nil 682 if errors.Cause(err) == sql.ErrNoRows { 683 return nil, nil 684 } 685 return nil, errors.Trace(err) 686 } 687 688 return taskCp, nil 689 } 690 691 func (cpdb *MySQLCheckpointsDB) Close() error { 692 return errors.Trace(cpdb.db.Close()) 693 } 694 695 func (cpdb *MySQLCheckpointsDB) Get(ctx context.Context, tableName string) (*TableCheckpoint, error) { 696 cp := &TableCheckpoint{ 697 Engines: map[int32]*EngineCheckpoint{}, 698 } 699 700 s := common.SQLWithRetry{ 701 DB: cpdb.db, 702 Logger: log.With(zap.String("table", tableName)), 703 } 704 err := s.Transact(ctx, "read checkpoint", func(c context.Context, tx *sql.Tx) error { 705 // 1. Populate the engines. 706 707 engineQuery := fmt.Sprintf(ReadEngineTemplate, cpdb.schema, CheckpointTableNameEngine) 708 engineRows, err := tx.QueryContext(c, engineQuery, tableName) 709 if err != nil { 710 return errors.Trace(err) 711 } 712 defer engineRows.Close() 713 for engineRows.Next() { 714 var ( 715 engineID int32 716 status uint8 717 ) 718 if err := engineRows.Scan(&engineID, &status); err != nil { 719 return errors.Trace(err) 720 } 721 cp.Engines[engineID] = &EngineCheckpoint{ 722 Status: CheckpointStatus(status), 723 } 724 } 725 if err := engineRows.Err(); err != nil { 726 return errors.Trace(err) 727 } 728 729 // 2. Populate the chunks. 730 731 chunkQuery := fmt.Sprintf(ReadChunkTemplate, cpdb.schema, CheckpointTableNameChunk) 732 chunkRows, err := tx.QueryContext(c, chunkQuery, tableName) 733 if err != nil { 734 return errors.Trace(err) 735 } 736 defer chunkRows.Close() 737 for chunkRows.Next() { 738 var ( 739 value = &ChunkCheckpoint{} 740 colPerm []byte 741 engineID int32 742 kvcBytes uint64 743 kvcKVs uint64 744 kvcChecksum uint64 745 ) 746 if err := chunkRows.Scan( 747 &engineID, &value.Key.Path, &value.Key.Offset, &value.FileMeta.Type, &value.FileMeta.Compression, 748 &value.FileMeta.SortKey, &value.FileMeta.FileSize, &colPerm, &value.Chunk.Offset, &value.Chunk.EndOffset, 749 &value.Chunk.PrevRowIDMax, &value.Chunk.RowIDMax, &kvcBytes, &kvcKVs, &kvcChecksum, 750 &value.Timestamp, 751 ); err != nil { 752 return errors.Trace(err) 753 } 754 value.FileMeta.Path = value.Key.Path 755 value.Checksum = verify.MakeKVChecksum(kvcBytes, kvcKVs, kvcChecksum) 756 if err := json.Unmarshal(colPerm, &value.ColumnPermutation); err != nil { 757 return errors.Trace(err) 758 } 759 cp.Engines[engineID].Chunks = append(cp.Engines[engineID].Chunks, value) 760 } 761 if err := chunkRows.Err(); err != nil { 762 return errors.Trace(err) 763 } 764 765 // 3. Fill in the remaining table info 766 767 tableQuery := fmt.Sprintf(ReadTableRemainTemplate, cpdb.schema, CheckpointTableNameTable) 768 tableRow := tx.QueryRowContext(c, tableQuery, tableName) 769 770 var status uint8 771 if err := tableRow.Scan(&status, &cp.AllocBase, &cp.TableID); err != nil { 772 return errors.Trace(err) 773 } 774 cp.Status = CheckpointStatus(status) 775 return nil 776 }) 777 if err != nil { 778 return nil, errors.Trace(err) 779 } 780 781 return cp, nil 782 } 783 784 func (cpdb *MySQLCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error { 785 s := common.SQLWithRetry{ 786 DB: cpdb.db, 787 Logger: log.With(zap.String("table", tableName)), 788 } 789 err := s.Transact(ctx, "update engine checkpoints", func(c context.Context, tx *sql.Tx) error { 790 engineStmt, err := tx.PrepareContext(c, fmt.Sprintf(ReplaceEngineTemplate, cpdb.schema, CheckpointTableNameEngine)) 791 if err != nil { 792 return errors.Trace(err) 793 } 794 defer engineStmt.Close() 795 796 chunkStmt, err := tx.PrepareContext(c, fmt.Sprintf(ReplaceChunkTemplate, cpdb.schema, CheckpointTableNameChunk)) 797 if err != nil { 798 return errors.Trace(err) 799 } 800 defer chunkStmt.Close() 801 802 for engineID, engine := range checkpoints { 803 _, err = engineStmt.ExecContext(c, tableName, engineID, engine.Status) 804 if err != nil { 805 return errors.Trace(err) 806 } 807 for _, value := range engine.Chunks { 808 columnPerm, err := json.Marshal(value.ColumnPermutation) 809 if err != nil { 810 return errors.Trace(err) 811 } 812 _, err = chunkStmt.ExecContext( 813 c, tableName, engineID, 814 value.Key.Path, value.Key.Offset, value.FileMeta.Type, value.FileMeta.Compression, 815 value.FileMeta.SortKey, value.FileMeta.FileSize, columnPerm, value.Chunk.Offset, value.Chunk.EndOffset, 816 value.Chunk.PrevRowIDMax, value.Chunk.RowIDMax, value.Timestamp, 817 ) 818 if err != nil { 819 return errors.Trace(err) 820 } 821 } 822 } 823 824 return nil 825 }) 826 if err != nil { 827 return errors.Trace(err) 828 } 829 830 return nil 831 } 832 833 func (cpdb *MySQLCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) { 834 chunkQuery := fmt.Sprintf(UpdateChunkTemplate, cpdb.schema, CheckpointTableNameChunk) 835 rebaseQuery := fmt.Sprintf(UpdateTableRebaseTemplate, cpdb.schema, CheckpointTableNameTable) 836 tableStatusQuery := fmt.Sprintf(UpdateTableStatusTemplate, cpdb.schema, CheckpointTableNameTable) 837 engineStatusQuery := fmt.Sprintf(UpdateEngineTemplate, cpdb.schema, CheckpointTableNameEngine) 838 839 s := common.SQLWithRetry{DB: cpdb.db, Logger: log.L()} 840 err := s.Transact(context.Background(), "update checkpoints", func(c context.Context, tx *sql.Tx) error { 841 chunkStmt, e := tx.PrepareContext(c, chunkQuery) 842 if e != nil { 843 return errors.Trace(e) 844 } 845 defer chunkStmt.Close() 846 rebaseStmt, e := tx.PrepareContext(c, rebaseQuery) 847 if e != nil { 848 return errors.Trace(e) 849 } 850 defer rebaseStmt.Close() 851 tableStatusStmt, e := tx.PrepareContext(c, tableStatusQuery) 852 if e != nil { 853 return errors.Trace(e) 854 } 855 defer tableStatusStmt.Close() 856 engineStatusStmt, e := tx.PrepareContext(c, engineStatusQuery) 857 if e != nil { 858 return errors.Trace(e) 859 } 860 defer engineStatusStmt.Close() 861 862 for tableName, cpd := range checkpointDiffs { 863 if cpd.hasStatus { 864 if _, e := tableStatusStmt.ExecContext(c, cpd.status, tableName); e != nil { 865 return errors.Trace(e) 866 } 867 } 868 if cpd.hasRebase { 869 if _, e := rebaseStmt.ExecContext(c, cpd.allocBase, tableName); e != nil { 870 return errors.Trace(e) 871 } 872 } 873 for engineID, engineDiff := range cpd.engines { 874 if engineDiff.hasStatus { 875 if _, e := engineStatusStmt.ExecContext(c, engineDiff.status, tableName, engineID); e != nil { 876 return errors.Trace(e) 877 } 878 } 879 for key, diff := range engineDiff.chunks { 880 columnPerm, err := json.Marshal(diff.columnPermutation) 881 if err != nil { 882 return errors.Trace(err) 883 } 884 if _, e := chunkStmt.ExecContext( 885 c, 886 diff.pos, diff.rowID, diff.checksum.SumSize(), diff.checksum.SumKVS(), diff.checksum.Sum(), 887 columnPerm, tableName, engineID, key.Path, key.Offset, 888 ); e != nil { 889 return errors.Trace(e) 890 } 891 } 892 } 893 } 894 895 return nil 896 }) 897 if err != nil { 898 log.L().Error("save checkpoint failed", zap.Error(err)) 899 } 900 } 901 902 type FileCheckpointsDB struct { 903 lock sync.Mutex // we need to ensure only a thread can access to `checkpoints` at a time 904 checkpoints CheckpointsModel 905 path string 906 } 907 908 func NewFileCheckpointsDB(path string) *FileCheckpointsDB { 909 cpdb := &FileCheckpointsDB{ 910 path: path, 911 checkpoints: CheckpointsModel{ 912 TaskCheckpoint: &TaskCheckpointModel{}, 913 Checkpoints: map[string]*TableCheckpointModel{}, 914 }, 915 } 916 // ignore all errors -- file maybe not created yet (and it is fine). 917 content, err := ioutil.ReadFile(path) 918 if err == nil { 919 err2 := cpdb.checkpoints.Unmarshal(content) 920 if err2 != nil { 921 log.L().Error("checkpoint file is broken", zap.String("path", path), zap.Error(err2)) 922 } 923 // FIXME: patch for empty map may need initialize manually, because currently 924 // FIXME: a map of zero size -> marshall -> unmarshall -> become nil, see checkpoint_test.go 925 if cpdb.checkpoints.Checkpoints == nil { 926 cpdb.checkpoints.Checkpoints = map[string]*TableCheckpointModel{} 927 } 928 for _, table := range cpdb.checkpoints.Checkpoints { 929 if table.Engines == nil { 930 table.Engines = map[int32]*EngineCheckpointModel{} 931 } 932 for _, engine := range table.Engines { 933 if engine.Chunks == nil { 934 engine.Chunks = map[string]*ChunkCheckpointModel{} 935 } 936 } 937 } 938 } else { 939 log.L().Info("open checkpoint file failed, going to create a new one", 940 zap.String("path", path), 941 log.ShortError(err), 942 ) 943 } 944 return cpdb 945 } 946 947 func (cpdb *FileCheckpointsDB) save() error { 948 serialized, err := cpdb.checkpoints.Marshal() 949 if err != nil { 950 return errors.Trace(err) 951 } 952 if err := ioutil.WriteFile(cpdb.path, serialized, 0644); err != nil { 953 return errors.Trace(err) 954 } 955 return nil 956 } 957 958 func (cpdb *FileCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error { 959 cpdb.lock.Lock() 960 defer cpdb.lock.Unlock() 961 962 cpdb.checkpoints.TaskCheckpoint = &TaskCheckpointModel{ 963 TaskId: cfg.TaskID, 964 SourceDir: cfg.Mydumper.SourceDir, 965 Backend: cfg.TikvImporter.Backend, 966 ImporterAddr: cfg.TikvImporter.Addr, 967 TidbHost: cfg.TiDB.Host, 968 TidbPort: int32(cfg.TiDB.Port), 969 PdAddr: cfg.TiDB.PdAddr, 970 SortedKvDir: cfg.TikvImporter.SortedKVDir, 971 LightningVer: common.ReleaseVersion, 972 } 973 974 if cpdb.checkpoints.Checkpoints == nil { 975 cpdb.checkpoints.Checkpoints = make(map[string]*TableCheckpointModel) 976 } 977 978 for _, db := range dbInfo { 979 for _, table := range db.Tables { 980 tableName := common.UniqueTable(db.Name, table.Name) 981 if _, ok := cpdb.checkpoints.Checkpoints[tableName]; !ok { 982 cpdb.checkpoints.Checkpoints[tableName] = &TableCheckpointModel{ 983 Status: uint32(CheckpointStatusLoaded), 984 Engines: map[int32]*EngineCheckpointModel{}, 985 TableID: table.ID, 986 } 987 } 988 // TODO check if hash matches 989 } 990 } 991 992 return errors.Trace(cpdb.save()) 993 } 994 995 func (cpdb *FileCheckpointsDB) TaskCheckpoint(_ context.Context) (*TaskCheckpoint, error) { 996 // this method is always called in lock 997 cp := cpdb.checkpoints.TaskCheckpoint 998 if cp == nil || cp.TaskId == 0 { 999 return nil, nil 1000 } 1001 1002 return &TaskCheckpoint{ 1003 TaskId: cp.TaskId, 1004 SourceDir: cp.SourceDir, 1005 Backend: cp.Backend, 1006 ImporterAddr: cp.ImporterAddr, 1007 TiDBHost: cp.TidbHost, 1008 TiDBPort: int(cp.TidbPort), 1009 PdAddr: cp.PdAddr, 1010 SortedKVDir: cp.SortedKvDir, 1011 LightningVer: cp.LightningVer, 1012 }, nil 1013 } 1014 1015 func (cpdb *FileCheckpointsDB) Close() error { 1016 cpdb.lock.Lock() 1017 defer cpdb.lock.Unlock() 1018 1019 return errors.Trace(cpdb.save()) 1020 } 1021 1022 func (cpdb *FileCheckpointsDB) Get(_ context.Context, tableName string) (*TableCheckpoint, error) { 1023 cpdb.lock.Lock() 1024 defer cpdb.lock.Unlock() 1025 1026 tableModel, ok := cpdb.checkpoints.Checkpoints[tableName] 1027 if !ok { 1028 tableModel = &TableCheckpointModel{} 1029 } 1030 1031 cp := &TableCheckpoint{ 1032 Status: CheckpointStatus(tableModel.Status), 1033 AllocBase: tableModel.AllocBase, 1034 Engines: make(map[int32]*EngineCheckpoint, len(tableModel.Engines)), 1035 TableID: tableModel.TableID, 1036 } 1037 1038 for engineID, engineModel := range tableModel.Engines { 1039 engine := &EngineCheckpoint{ 1040 Status: CheckpointStatus(engineModel.Status), 1041 Chunks: make([]*ChunkCheckpoint, 0, len(engineModel.Chunks)), 1042 } 1043 1044 for _, chunkModel := range engineModel.Chunks { 1045 colPerm := make([]int, 0, len(chunkModel.ColumnPermutation)) 1046 for _, c := range chunkModel.ColumnPermutation { 1047 colPerm = append(colPerm, int(c)) 1048 } 1049 engine.Chunks = append(engine.Chunks, &ChunkCheckpoint{ 1050 Key: ChunkCheckpointKey{ 1051 Path: chunkModel.Path, 1052 Offset: chunkModel.Offset, 1053 }, 1054 FileMeta: mydump.SourceFileMeta{ 1055 Path: chunkModel.Path, 1056 Type: mydump.SourceType(chunkModel.Type), 1057 Compression: mydump.Compression(chunkModel.Compression), 1058 SortKey: chunkModel.SortKey, 1059 FileSize: chunkModel.FileSize, 1060 }, 1061 ColumnPermutation: colPerm, 1062 Chunk: mydump.Chunk{ 1063 Offset: chunkModel.Pos, 1064 EndOffset: chunkModel.EndOffset, 1065 PrevRowIDMax: chunkModel.PrevRowidMax, 1066 RowIDMax: chunkModel.RowidMax, 1067 }, 1068 Checksum: verify.MakeKVChecksum(chunkModel.KvcBytes, chunkModel.KvcKvs, chunkModel.KvcChecksum), 1069 Timestamp: chunkModel.Timestamp, 1070 }) 1071 } 1072 1073 sort.Slice(engine.Chunks, func(i, j int) bool { 1074 return engine.Chunks[i].Key.less(&engine.Chunks[j].Key) 1075 }) 1076 1077 cp.Engines[engineID] = engine 1078 } 1079 1080 return cp, nil 1081 } 1082 1083 func (cpdb *FileCheckpointsDB) InsertEngineCheckpoints(_ context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error { 1084 cpdb.lock.Lock() 1085 defer cpdb.lock.Unlock() 1086 1087 tableModel := cpdb.checkpoints.Checkpoints[tableName] 1088 for engineID, engine := range checkpoints { 1089 engineModel := &EngineCheckpointModel{ 1090 Status: uint32(CheckpointStatusLoaded), 1091 Chunks: make(map[string]*ChunkCheckpointModel), 1092 } 1093 for _, value := range engine.Chunks { 1094 key := value.Key.String() 1095 chunk, ok := engineModel.Chunks[key] 1096 if !ok { 1097 chunk = &ChunkCheckpointModel{ 1098 Path: value.Key.Path, 1099 Offset: value.Key.Offset, 1100 } 1101 engineModel.Chunks[key] = chunk 1102 } 1103 chunk.Type = int32(value.FileMeta.Type) 1104 chunk.Compression = int32(value.FileMeta.Compression) 1105 chunk.SortKey = value.FileMeta.SortKey 1106 chunk.FileSize = value.FileMeta.FileSize 1107 chunk.Pos = value.Chunk.Offset 1108 chunk.EndOffset = value.Chunk.EndOffset 1109 chunk.PrevRowidMax = value.Chunk.PrevRowIDMax 1110 chunk.RowidMax = value.Chunk.RowIDMax 1111 chunk.Timestamp = value.Timestamp 1112 if len(value.ColumnPermutation) > 0 { 1113 chunk.ColumnPermutation = intSlice2Int32Slice(value.ColumnPermutation) 1114 } 1115 1116 } 1117 tableModel.Engines[engineID] = engineModel 1118 } 1119 1120 return errors.Trace(cpdb.save()) 1121 } 1122 1123 func (cpdb *FileCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) { 1124 cpdb.lock.Lock() 1125 defer cpdb.lock.Unlock() 1126 1127 for tableName, cpd := range checkpointDiffs { 1128 tableModel := cpdb.checkpoints.Checkpoints[tableName] 1129 if cpd.hasStatus { 1130 tableModel.Status = uint32(cpd.status) 1131 } 1132 if cpd.hasRebase { 1133 tableModel.AllocBase = cpd.allocBase 1134 } 1135 for engineID, engineDiff := range cpd.engines { 1136 engineModel := tableModel.Engines[engineID] 1137 if engineDiff.hasStatus { 1138 engineModel.Status = uint32(engineDiff.status) 1139 } 1140 1141 for key, diff := range engineDiff.chunks { 1142 chunkModel := engineModel.Chunks[key.String()] 1143 chunkModel.Pos = diff.pos 1144 chunkModel.PrevRowidMax = diff.rowID 1145 chunkModel.KvcBytes = diff.checksum.SumSize() 1146 chunkModel.KvcKvs = diff.checksum.SumKVS() 1147 chunkModel.KvcChecksum = diff.checksum.Sum() 1148 chunkModel.ColumnPermutation = intSlice2Int32Slice(diff.columnPermutation) 1149 } 1150 } 1151 } 1152 1153 if err := cpdb.save(); err != nil { 1154 log.L().Error("save checkpoint failed", zap.Error(err)) 1155 } 1156 } 1157 1158 // Management functions ---------------------------------------------------------------------------- 1159 1160 var cannotManageNullDB = errors.New("cannot perform this function while checkpoints is disabled") 1161 1162 func (*NullCheckpointsDB) RemoveCheckpoint(context.Context, string) error { 1163 return errors.Trace(cannotManageNullDB) 1164 } 1165 func (*NullCheckpointsDB) MoveCheckpoints(context.Context, int64) error { 1166 return errors.Trace(cannotManageNullDB) 1167 } 1168 func (*NullCheckpointsDB) GetLocalStoringTables(context.Context) (map[string][]int32, error) { 1169 return nil, nil 1170 } 1171 func (*NullCheckpointsDB) IgnoreErrorCheckpoint(context.Context, string) error { 1172 return errors.Trace(cannotManageNullDB) 1173 } 1174 func (*NullCheckpointsDB) DestroyErrorCheckpoint(context.Context, string) ([]DestroyedTableCheckpoint, error) { 1175 return nil, errors.Trace(cannotManageNullDB) 1176 } 1177 func (*NullCheckpointsDB) DumpTables(context.Context, io.Writer) error { 1178 return errors.Trace(cannotManageNullDB) 1179 } 1180 func (*NullCheckpointsDB) DumpEngines(context.Context, io.Writer) error { 1181 return errors.Trace(cannotManageNullDB) 1182 } 1183 func (*NullCheckpointsDB) DumpChunks(context.Context, io.Writer) error { 1184 return errors.Trace(cannotManageNullDB) 1185 } 1186 1187 func (cpdb *MySQLCheckpointsDB) RemoveCheckpoint(ctx context.Context, tableName string) error { 1188 s := common.SQLWithRetry{ 1189 DB: cpdb.db, 1190 Logger: log.With(zap.String("table", tableName)), 1191 } 1192 1193 if tableName == "all" { 1194 return s.Exec(ctx, "remove all checkpoints", "DROP SCHEMA "+cpdb.schema) 1195 } 1196 1197 deleteChunkQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, cpdb.schema, CheckpointTableNameChunk) 1198 deleteEngineQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, cpdb.schema, CheckpointTableNameEngine) 1199 deleteTableQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, cpdb.schema, CheckpointTableNameTable) 1200 1201 return s.Transact(ctx, "remove checkpoints", func(c context.Context, tx *sql.Tx) error { 1202 if _, e := tx.ExecContext(c, deleteChunkQuery, tableName); e != nil { 1203 return errors.Trace(e) 1204 } 1205 if _, e := tx.ExecContext(c, deleteEngineQuery, tableName); e != nil { 1206 return errors.Trace(e) 1207 } 1208 if _, e := tx.ExecContext(c, deleteTableQuery, tableName); e != nil { 1209 return errors.Trace(e) 1210 } 1211 return nil 1212 }) 1213 } 1214 1215 func (cpdb *MySQLCheckpointsDB) MoveCheckpoints(ctx context.Context, taskID int64) error { 1216 // The "cpdb.schema" is an escaped schema name of the form "`foo`". 1217 // We use "x[1:len(x)-1]" instead of unescaping it to keep the 1218 // double-backquotes (if any) intact. 1219 newSchema := fmt.Sprintf("`%s.%d.bak`", cpdb.schema[1:len(cpdb.schema)-1], taskID) 1220 s := common.SQLWithRetry{ 1221 DB: cpdb.db, 1222 Logger: log.With(zap.Int64("taskID", taskID)), 1223 } 1224 1225 createSchemaQuery := "CREATE SCHEMA IF NOT EXISTS " + newSchema 1226 if e := s.Exec(ctx, "create backup checkpoints schema", createSchemaQuery); e != nil { 1227 return e 1228 } 1229 for _, tbl := range []string{CheckpointTableNameChunk, CheckpointTableNameEngine, 1230 CheckpointTableNameTable, CheckpointTableNameTask} { 1231 query := fmt.Sprintf("RENAME TABLE %[1]s.%[3]s TO %[2]s.%[3]s", cpdb.schema, newSchema, tbl) 1232 if e := s.Exec(ctx, fmt.Sprintf("move %s checkpoints table", tbl), query); e != nil { 1233 return e 1234 } 1235 } 1236 1237 return nil 1238 } 1239 1240 func (cpdb *MySQLCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[string][]int32, error) { 1241 var targetTables map[string][]int32 1242 1243 // lightning didn't check CheckpointStatusMaxInvalid before this function is called, so we skip invalid ones 1244 // engines should exist if 1245 // 1. table status is earlier than CheckpointStatusIndexImported, and 1246 // 2. engine status is earlier than CheckpointStatusImported, and 1247 // 3. chunk has been read 1248 query := fmt.Sprintf(` 1249 SELECT DISTINCT t.table_name, c.engine_id 1250 FROM %s.%s t, %s.%s c, %s.%s e 1251 WHERE t.table_name = c.table_name AND t.table_name = e.table_name AND c.engine_id = e.engine_id 1252 AND %d < t.status AND t.status < %d 1253 AND %d < e.status AND e.status < %d 1254 AND c.pos > c.offset;`, 1255 cpdb.schema, CheckpointTableNameTable, cpdb.schema, CheckpointTableNameChunk, cpdb.schema, CheckpointTableNameEngine, 1256 CheckpointStatusMaxInvalid, CheckpointStatusIndexImported, 1257 CheckpointStatusMaxInvalid, CheckpointStatusImported) 1258 1259 err := common.Retry("get local storing tables", log.L(), func() error { 1260 targetTables = make(map[string][]int32) 1261 rows, err := cpdb.db.QueryContext(ctx, query) 1262 if err != nil { 1263 return errors.Trace(err) 1264 } 1265 defer rows.Close() 1266 for rows.Next() { 1267 var ( 1268 tableName string 1269 engineID int32 1270 ) 1271 if err := rows.Scan(&tableName, &engineID); err != nil { 1272 return errors.Trace(err) 1273 } 1274 targetTables[tableName] = append(targetTables[tableName], engineID) 1275 } 1276 if err := rows.Err(); err != nil { 1277 return errors.Trace(err) 1278 } 1279 return nil 1280 }) 1281 if err != nil { 1282 return nil, errors.Trace(err) 1283 } 1284 1285 return targetTables, err 1286 } 1287 1288 func (cpdb *MySQLCheckpointsDB) IgnoreErrorCheckpoint(ctx context.Context, tableName string) error { 1289 var colName string 1290 if tableName == "all" { 1291 // This will expand to `WHERE 'all' = 'all'` and effectively allowing 1292 // all tables to be included. 1293 colName = "'all'" 1294 } else { 1295 colName = "table_name" 1296 } 1297 1298 engineQuery := fmt.Sprintf(` 1299 UPDATE %s.%s SET status = %d WHERE %s = ? AND status <= %d; 1300 `, cpdb.schema, CheckpointTableNameEngine, CheckpointStatusLoaded, colName, CheckpointStatusMaxInvalid) 1301 tableQuery := fmt.Sprintf(` 1302 UPDATE %s.%s SET status = %d WHERE %s = ? AND status <= %d; 1303 `, cpdb.schema, CheckpointTableNameTable, CheckpointStatusLoaded, colName, CheckpointStatusMaxInvalid) 1304 1305 s := common.SQLWithRetry{ 1306 DB: cpdb.db, 1307 Logger: log.With(zap.String("table", tableName)), 1308 } 1309 err := s.Transact(ctx, "ignore error checkpoints", func(c context.Context, tx *sql.Tx) error { 1310 if _, e := tx.ExecContext(c, engineQuery, tableName); e != nil { 1311 return errors.Trace(e) 1312 } 1313 if _, e := tx.ExecContext(c, tableQuery, tableName); e != nil { 1314 return errors.Trace(e) 1315 } 1316 return nil 1317 }) 1318 return errors.Trace(err) 1319 } 1320 1321 func (cpdb *MySQLCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tableName string) ([]DestroyedTableCheckpoint, error) { 1322 var colName, aliasedColName string 1323 1324 if tableName == "all" { 1325 // These will expand to `WHERE 'all' = 'all'` and effectively allowing 1326 // all tables to be included. 1327 colName = "'all'" 1328 aliasedColName = "'all'" 1329 } else { 1330 colName = "table_name" 1331 aliasedColName = "t.table_name" 1332 } 1333 1334 selectQuery := fmt.Sprintf(` 1335 SELECT 1336 t.table_name, 1337 COALESCE(MIN(e.engine_id), 0), 1338 COALESCE(MAX(e.engine_id), -1) 1339 FROM %[1]s.%[4]s t 1340 LEFT JOIN %[1]s.%[5]s e ON t.table_name = e.table_name 1341 WHERE %[2]s = ? AND t.status <= %[3]d 1342 GROUP BY t.table_name; 1343 `, cpdb.schema, aliasedColName, CheckpointStatusMaxInvalid, CheckpointTableNameTable, CheckpointTableNameEngine) 1344 deleteChunkQuery := fmt.Sprintf(` 1345 DELETE FROM %[1]s.%[4]s WHERE table_name IN (SELECT table_name FROM %[1]s.%[5]s WHERE %[2]s = ? AND status <= %[3]d) 1346 `, cpdb.schema, colName, CheckpointStatusMaxInvalid, CheckpointTableNameChunk, CheckpointTableNameTable) 1347 deleteEngineQuery := fmt.Sprintf(` 1348 DELETE FROM %[1]s.%[4]s WHERE table_name IN (SELECT table_name FROM %[1]s.%[5]s WHERE %[2]s = ? AND status <= %[3]d) 1349 `, cpdb.schema, colName, CheckpointStatusMaxInvalid, CheckpointTableNameEngine, CheckpointTableNameTable) 1350 deleteTableQuery := fmt.Sprintf(` 1351 DELETE FROM %s.%s WHERE %s = ? AND status <= %d 1352 `, cpdb.schema, CheckpointTableNameTable, colName, CheckpointStatusMaxInvalid) 1353 1354 var targetTables []DestroyedTableCheckpoint 1355 1356 s := common.SQLWithRetry{ 1357 DB: cpdb.db, 1358 Logger: log.With(zap.String("table", tableName)), 1359 } 1360 err := s.Transact(ctx, "destroy error checkpoints", func(c context.Context, tx *sql.Tx) error { 1361 // Obtain the list of tables 1362 targetTables = nil 1363 rows, e := tx.QueryContext(c, selectQuery, tableName) 1364 if e != nil { 1365 return errors.Trace(e) 1366 } 1367 defer rows.Close() 1368 for rows.Next() { 1369 var dtc DestroyedTableCheckpoint 1370 if e := rows.Scan(&dtc.TableName, &dtc.MinEngineID, &dtc.MaxEngineID); e != nil { 1371 return errors.Trace(e) 1372 } 1373 targetTables = append(targetTables, dtc) 1374 } 1375 if e := rows.Err(); e != nil { 1376 return errors.Trace(e) 1377 } 1378 1379 // Delete the checkpoints 1380 if _, e := tx.ExecContext(c, deleteChunkQuery, tableName); e != nil { 1381 return errors.Trace(e) 1382 } 1383 if _, e := tx.ExecContext(c, deleteEngineQuery, tableName); e != nil { 1384 return errors.Trace(e) 1385 } 1386 if _, e := tx.ExecContext(c, deleteTableQuery, tableName); e != nil { 1387 return errors.Trace(e) 1388 } 1389 return nil 1390 }) 1391 if err != nil { 1392 return nil, errors.Trace(err) 1393 } 1394 1395 return targetTables, nil 1396 } 1397 1398 func (cpdb *MySQLCheckpointsDB) DumpTables(ctx context.Context, writer io.Writer) error { 1399 rows, err := cpdb.db.QueryContext(ctx, fmt.Sprintf(` 1400 SELECT 1401 task_id, 1402 table_name, 1403 hex(hash) AS hash, 1404 status, 1405 alloc_base, 1406 create_time, 1407 update_time 1408 FROM %s.%s; 1409 `, cpdb.schema, CheckpointTableNameTable)) 1410 if err != nil { 1411 return errors.Trace(err) 1412 } 1413 defer rows.Close() 1414 1415 return errors.Trace(sqltocsv.Write(writer, rows)) 1416 } 1417 1418 func (cpdb *MySQLCheckpointsDB) DumpEngines(ctx context.Context, writer io.Writer) error { 1419 rows, err := cpdb.db.QueryContext(ctx, fmt.Sprintf(` 1420 SELECT 1421 table_name, 1422 engine_id, 1423 status, 1424 create_time, 1425 update_time 1426 FROM %s.%s; 1427 `, cpdb.schema, CheckpointTableNameEngine)) 1428 if err != nil { 1429 return errors.Trace(err) 1430 } 1431 defer rows.Close() 1432 1433 return errors.Trace(sqltocsv.Write(writer, rows)) 1434 } 1435 1436 func (cpdb *MySQLCheckpointsDB) DumpChunks(ctx context.Context, writer io.Writer) error { 1437 rows, err := cpdb.db.QueryContext(ctx, fmt.Sprintf(` 1438 SELECT 1439 table_name, 1440 path, 1441 offset, 1442 type, 1443 compression, 1444 sort_key, 1445 file_size, 1446 columns, 1447 pos, 1448 end_offset, 1449 prev_rowid_max, 1450 rowid_max, 1451 kvc_bytes, 1452 kvc_kvs, 1453 kvc_checksum, 1454 create_time, 1455 update_time 1456 FROM %s.%s; 1457 `, cpdb.schema, CheckpointTableNameChunk)) 1458 if err != nil { 1459 return errors.Trace(err) 1460 } 1461 defer rows.Close() 1462 1463 return errors.Trace(sqltocsv.Write(writer, rows)) 1464 } 1465 1466 func (cpdb *FileCheckpointsDB) RemoveCheckpoint(_ context.Context, tableName string) error { 1467 cpdb.lock.Lock() 1468 defer cpdb.lock.Unlock() 1469 1470 if tableName == "all" { 1471 cpdb.checkpoints.Reset() 1472 return errors.Trace(os.Remove(cpdb.path)) 1473 } 1474 1475 delete(cpdb.checkpoints.Checkpoints, tableName) 1476 return errors.Trace(cpdb.save()) 1477 } 1478 1479 func (cpdb *FileCheckpointsDB) MoveCheckpoints(ctx context.Context, taskID int64) error { 1480 cpdb.lock.Lock() 1481 defer cpdb.lock.Unlock() 1482 1483 newPath := fmt.Sprintf("%s.%d.bak", cpdb.path, taskID) 1484 return errors.Trace(os.Rename(cpdb.path, newPath)) 1485 } 1486 1487 func (cpdb *FileCheckpointsDB) GetLocalStoringTables(_ context.Context) (map[string][]int32, error) { 1488 cpdb.lock.Lock() 1489 defer cpdb.lock.Unlock() 1490 1491 targetTables := make(map[string][]int32) 1492 1493 for tableName, tableModel := range cpdb.checkpoints.Checkpoints { 1494 if tableModel.Status <= uint32(CheckpointStatusMaxInvalid) || 1495 tableModel.Status >= uint32(CheckpointStatusIndexImported) { 1496 continue 1497 } 1498 for engineID, engineModel := range tableModel.Engines { 1499 if engineModel.Status <= uint32(CheckpointStatusMaxInvalid) || 1500 engineModel.Status >= uint32(CheckpointStatusImported) { 1501 continue 1502 } 1503 1504 for _, chunkModel := range engineModel.Chunks { 1505 if chunkModel.Pos > chunkModel.Offset { 1506 targetTables[tableName] = append(targetTables[tableName], engineID) 1507 break 1508 } 1509 } 1510 1511 } 1512 } 1513 1514 return targetTables, nil 1515 } 1516 1517 func (cpdb *FileCheckpointsDB) IgnoreErrorCheckpoint(_ context.Context, targetTableName string) error { 1518 cpdb.lock.Lock() 1519 defer cpdb.lock.Unlock() 1520 1521 for tableName, tableModel := range cpdb.checkpoints.Checkpoints { 1522 if !(targetTableName == "all" || targetTableName == tableName) { 1523 continue 1524 } 1525 if tableModel.Status <= uint32(CheckpointStatusMaxInvalid) { 1526 tableModel.Status = uint32(CheckpointStatusLoaded) 1527 } 1528 for _, engineModel := range tableModel.Engines { 1529 if engineModel.Status <= uint32(CheckpointStatusMaxInvalid) { 1530 engineModel.Status = uint32(CheckpointStatusLoaded) 1531 } 1532 } 1533 } 1534 return errors.Trace(cpdb.save()) 1535 } 1536 1537 func (cpdb *FileCheckpointsDB) DestroyErrorCheckpoint(_ context.Context, targetTableName string) ([]DestroyedTableCheckpoint, error) { 1538 cpdb.lock.Lock() 1539 defer cpdb.lock.Unlock() 1540 1541 var targetTables []DestroyedTableCheckpoint 1542 1543 for tableName, tableModel := range cpdb.checkpoints.Checkpoints { 1544 // Obtain the list of tables 1545 if !(targetTableName == "all" || targetTableName == tableName) { 1546 continue 1547 } 1548 if tableModel.Status <= uint32(CheckpointStatusMaxInvalid) { 1549 var minEngineID, maxEngineID int32 = math.MaxInt32, math.MinInt32 1550 for engineID := range tableModel.Engines { 1551 if engineID < minEngineID { 1552 minEngineID = engineID 1553 } 1554 if engineID > maxEngineID { 1555 maxEngineID = engineID 1556 } 1557 } 1558 1559 targetTables = append(targetTables, DestroyedTableCheckpoint{ 1560 TableName: tableName, 1561 MinEngineID: minEngineID, 1562 MaxEngineID: maxEngineID, 1563 }) 1564 } 1565 } 1566 1567 // Delete the checkpoints 1568 for _, dtcp := range targetTables { 1569 delete(cpdb.checkpoints.Checkpoints, dtcp.TableName) 1570 } 1571 if err := cpdb.save(); err != nil { 1572 return nil, errors.Trace(err) 1573 } 1574 1575 return targetTables, nil 1576 } 1577 1578 func (cpdb *FileCheckpointsDB) DumpTables(context.Context, io.Writer) error { 1579 return errors.Errorf("dumping file checkpoint into CSV not unsupported, you may copy %s instead", cpdb.path) 1580 } 1581 1582 func (cpdb *FileCheckpointsDB) DumpEngines(context.Context, io.Writer) error { 1583 return errors.Errorf("dumping file checkpoint into CSV not unsupported, you may copy %s instead", cpdb.path) 1584 } 1585 1586 func (cpdb *FileCheckpointsDB) DumpChunks(context.Context, io.Writer) error { 1587 return errors.Errorf("dumping file checkpoint into CSV not unsupported, you may copy %s instead", cpdb.path) 1588 } 1589 1590 func intSlice2Int32Slice(s []int) []int32 { 1591 res := make([]int32, 0, len(s)) 1592 for _, i := range s { 1593 res = append(res, int32(i)) 1594 } 1595 return res 1596 }