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