github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/checkpoints/glue_checkpoint.go (about)

     1  // Copyright 2020 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  	"encoding/json"
    19  	"fmt"
    20  	"io"
    21  	"strings"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/parser/ast"
    25  	"github.com/pingcap/tidb-lightning/lightning/common"
    26  	"github.com/pingcap/tidb-lightning/lightning/config"
    27  	"github.com/pingcap/tidb-lightning/lightning/log"
    28  	"github.com/pingcap/tidb-lightning/lightning/mydump"
    29  	verify "github.com/pingcap/tidb-lightning/lightning/verification"
    30  	"github.com/pingcap/tidb/types"
    31  	"github.com/pingcap/tidb/util/chunk"
    32  	"github.com/pingcap/tidb/util/sqlexec"
    33  	"go.uber.org/zap"
    34  )
    35  
    36  type Session interface {
    37  	Close()
    38  	Execute(context.Context, string) ([]sqlexec.RecordSet, error)
    39  	CommitTxn(context.Context) error
    40  	RollbackTxn(context.Context)
    41  	PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error)
    42  	ExecutePreparedStmt(ctx context.Context, stmtID uint32, param []types.Datum) (sqlexec.RecordSet, error)
    43  	DropPreparedStmt(stmtID uint32) error
    44  }
    45  
    46  // GlueCheckpointsDB is almost same with MySQLCheckpointsDB, but it uses TiDB's internal data structure which requires a
    47  // lot to keep same with database/sql.
    48  // TODO: Encapsulate Begin/Commit/Rollback txn, form SQL with args and query/iter/scan TiDB's RecordSet into a interface
    49  // to reuse MySQLCheckpointsDB.
    50  type GlueCheckpointsDB struct {
    51  	// getSessionFunc will get a new session from TiDB
    52  	getSessionFunc func() (Session, error)
    53  	schema         string
    54  }
    55  
    56  var _ CheckpointsDB = (*GlueCheckpointsDB)(nil)
    57  
    58  func NewGlueCheckpointsDB(ctx context.Context, se Session, f func() (Session, error), schemaName string) (*GlueCheckpointsDB, error) {
    59  	var escapedSchemaName strings.Builder
    60  	common.WriteMySQLIdentifier(&escapedSchemaName, schemaName)
    61  	schema := escapedSchemaName.String()
    62  	logger := log.With(zap.String("schema", schemaName))
    63  
    64  	sql := fmt.Sprintf(CreateDBTemplate, schema)
    65  	err := common.Retry("create checkpoints database", logger, func() error {
    66  		_, err := se.Execute(ctx, sql)
    67  		return err
    68  	})
    69  	if err != nil {
    70  		return nil, errors.Trace(err)
    71  	}
    72  
    73  	sql = fmt.Sprintf(CreateTaskTableTemplate, schema, CheckpointTableNameTask)
    74  	err = common.Retry("create task checkpoints table", logger, func() error {
    75  		_, err := se.Execute(ctx, sql)
    76  		return err
    77  	})
    78  	if err != nil {
    79  		return nil, errors.Trace(err)
    80  	}
    81  
    82  	sql = fmt.Sprintf(CreateTableTableTemplate, schema, CheckpointTableNameTable)
    83  	err = common.Retry("create table checkpoints table", logger, func() error {
    84  		_, err := se.Execute(ctx, sql)
    85  		return err
    86  	})
    87  	if err != nil {
    88  		return nil, errors.Trace(err)
    89  	}
    90  
    91  	sql = fmt.Sprintf(CreateEngineTableTemplate, schema, CheckpointTableNameEngine)
    92  	err = common.Retry("create engine checkpoints table", logger, func() error {
    93  		_, err := se.Execute(ctx, sql)
    94  		return err
    95  	})
    96  	if err != nil {
    97  		return nil, errors.Trace(err)
    98  	}
    99  
   100  	sql = fmt.Sprintf(CreateChunkTableTemplate, schema, CheckpointTableNameChunk)
   101  	err = common.Retry("create chunks checkpoints table", logger, func() error {
   102  		_, err := se.Execute(ctx, sql)
   103  		return err
   104  	})
   105  	if err != nil {
   106  		return nil, errors.Trace(err)
   107  	}
   108  
   109  	return &GlueCheckpointsDB{
   110  		getSessionFunc: f,
   111  		schema:         schema,
   112  	}, nil
   113  }
   114  
   115  func (g GlueCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error {
   116  	logger := log.L()
   117  	se, err := g.getSessionFunc()
   118  	if err != nil {
   119  		return errors.Trace(err)
   120  	}
   121  	defer se.Close()
   122  
   123  	err = Transact(ctx, "insert checkpoints", se, logger, func(c context.Context, s Session) error {
   124  		stmtID, _, _, err := s.PrepareStmt(fmt.Sprintf(InitTaskTemplate, g.schema, CheckpointTableNameTask))
   125  		if err != nil {
   126  			return errors.Trace(err)
   127  		}
   128  		defer s.DropPreparedStmt(stmtID)
   129  		_, err = s.ExecutePreparedStmt(c, stmtID, []types.Datum{
   130  			types.NewIntDatum(cfg.TaskID),
   131  			types.NewStringDatum(cfg.Mydumper.SourceDir),
   132  			types.NewStringDatum(cfg.TikvImporter.Backend),
   133  			types.NewStringDatum(cfg.TikvImporter.Addr),
   134  			types.NewStringDatum(cfg.TiDB.Host),
   135  			types.NewIntDatum(int64(cfg.TiDB.Port)),
   136  			types.NewStringDatum(cfg.TiDB.PdAddr),
   137  			types.NewStringDatum(cfg.TikvImporter.SortedKVDir),
   138  			types.NewStringDatum(common.ReleaseVersion),
   139  		})
   140  		if err != nil {
   141  			return errors.Trace(err)
   142  		}
   143  
   144  		stmtID2, _, _, err := s.PrepareStmt(fmt.Sprintf(InitTableTemplate, g.schema, CheckpointTableNameTable))
   145  		if err != nil {
   146  			return errors.Trace(err)
   147  		}
   148  		defer s.DropPreparedStmt(stmtID2)
   149  
   150  		for _, db := range dbInfo {
   151  			for _, table := range db.Tables {
   152  				tableName := common.UniqueTable(db.Name, table.Name)
   153  				_, err = s.ExecutePreparedStmt(c, stmtID2, []types.Datum{
   154  					types.NewIntDatum(cfg.TaskID),
   155  					types.NewStringDatum(tableName),
   156  					types.NewIntDatum(0),
   157  					types.NewIntDatum(table.ID),
   158  				})
   159  				if err != nil {
   160  					return errors.Trace(err)
   161  				}
   162  			}
   163  		}
   164  		return nil
   165  	})
   166  	return errors.Trace(err)
   167  }
   168  
   169  func (g GlueCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) {
   170  	logger := log.L()
   171  	sql := fmt.Sprintf(ReadTaskTemplate, g.schema, CheckpointTableNameTask)
   172  	se, err := g.getSessionFunc()
   173  	if err != nil {
   174  		return nil, errors.Trace(err)
   175  	}
   176  	defer se.Close()
   177  
   178  	var taskCp *TaskCheckpoint
   179  	err = common.Retry("fetch task checkpoint", logger, func() error {
   180  		rs, err := se.Execute(ctx, sql)
   181  		if err != nil {
   182  			return errors.Trace(err)
   183  		}
   184  		r := rs[0]
   185  		defer r.Close()
   186  		req := r.NewChunk()
   187  		err = r.Next(ctx, req)
   188  		if err != nil {
   189  			return err
   190  		}
   191  		if req.NumRows() == 0 {
   192  			return nil
   193  		}
   194  
   195  		row := req.GetRow(0)
   196  		taskCp = &TaskCheckpoint{}
   197  		taskCp.TaskId = row.GetInt64(0)
   198  		taskCp.SourceDir = row.GetString(1)
   199  		taskCp.Backend = row.GetString(2)
   200  		taskCp.ImporterAddr = row.GetString(3)
   201  		taskCp.TiDBHost = row.GetString(4)
   202  		taskCp.TiDBPort = int(row.GetInt64(5))
   203  		taskCp.PdAddr = row.GetString(6)
   204  		taskCp.SortedKVDir = row.GetString(7)
   205  		taskCp.LightningVer = row.GetString(8)
   206  		return nil
   207  	})
   208  	if err != nil {
   209  		return nil, errors.Trace(err)
   210  	}
   211  	return taskCp, nil
   212  }
   213  
   214  func (g GlueCheckpointsDB) Get(ctx context.Context, tableName string) (*TableCheckpoint, error) {
   215  	cp := &TableCheckpoint{
   216  		Engines: map[int32]*EngineCheckpoint{},
   217  	}
   218  	logger := log.With(zap.String("table", tableName))
   219  	se, err := g.getSessionFunc()
   220  	if err != nil {
   221  		return nil, errors.Trace(err)
   222  	}
   223  	defer se.Close()
   224  
   225  	tableName = common.InterpolateMySQLString(tableName)
   226  	err = Transact(ctx, "read checkpoint", se, logger, func(c context.Context, s Session) error {
   227  		// 1. Populate the engines.
   228  		sql := fmt.Sprintf(ReadEngineTemplate, g.schema, CheckpointTableNameEngine)
   229  		sql = strings.ReplaceAll(sql, "?", tableName)
   230  		rs, err := s.Execute(ctx, sql)
   231  		if err != nil {
   232  			return errors.Trace(err)
   233  		}
   234  		r := rs[0]
   235  		req := r.NewChunk()
   236  		it := chunk.NewIterator4Chunk(req)
   237  		for {
   238  			err = r.Next(ctx, req)
   239  			if err != nil {
   240  				r.Close()
   241  				return err
   242  			}
   243  			if req.NumRows() == 0 {
   244  				break
   245  			}
   246  
   247  			for row := it.Begin(); row != it.End(); row = it.Next() {
   248  				engineID := int32(row.GetInt64(0))
   249  				status := uint8(row.GetUint64(1))
   250  				cp.Engines[engineID] = &EngineCheckpoint{
   251  					Status: CheckpointStatus(status),
   252  				}
   253  			}
   254  		}
   255  		r.Close()
   256  
   257  		// 2. Populate the chunks.
   258  		sql = fmt.Sprintf(ReadChunkTemplate, g.schema, CheckpointTableNameChunk)
   259  		sql = strings.ReplaceAll(sql, "?", tableName)
   260  		rs, err = s.Execute(ctx, sql)
   261  		if err != nil {
   262  			return errors.Trace(err)
   263  		}
   264  		r = rs[0]
   265  		req = r.NewChunk()
   266  		it = chunk.NewIterator4Chunk(req)
   267  		for {
   268  			err = r.Next(ctx, req)
   269  			if err != nil {
   270  				r.Close()
   271  				return err
   272  			}
   273  			if req.NumRows() == 0 {
   274  				break
   275  			}
   276  
   277  			for row := it.Begin(); row != it.End(); row = it.Next() {
   278  				value := &ChunkCheckpoint{}
   279  				engineID := int32(row.GetInt64(0))
   280  				value.Key.Path = row.GetString(1)
   281  				value.Key.Offset = row.GetInt64(2)
   282  				value.FileMeta.Type = mydump.SourceType(row.GetInt64(3))
   283  				value.FileMeta.Compression = mydump.Compression(row.GetInt64(4))
   284  				value.FileMeta.SortKey = row.GetString(5)
   285  				value.FileMeta.FileSize = row.GetInt64(6)
   286  				colPerm := row.GetBytes(7)
   287  				value.Chunk.Offset = row.GetInt64(8)
   288  				value.Chunk.EndOffset = row.GetInt64(9)
   289  				value.Chunk.PrevRowIDMax = row.GetInt64(10)
   290  				value.Chunk.RowIDMax = row.GetInt64(11)
   291  				kvcBytes := row.GetUint64(12)
   292  				kvcKVs := row.GetUint64(13)
   293  				kvcChecksum := row.GetUint64(14)
   294  				value.Timestamp = row.GetInt64(15)
   295  
   296  				value.FileMeta.Path = value.Key.Path
   297  				value.Checksum = verify.MakeKVChecksum(kvcBytes, kvcKVs, kvcChecksum)
   298  				if err := json.Unmarshal(colPerm, &value.ColumnPermutation); err != nil {
   299  					r.Close()
   300  					return errors.Trace(err)
   301  				}
   302  				cp.Engines[engineID].Chunks = append(cp.Engines[engineID].Chunks, value)
   303  			}
   304  		}
   305  		r.Close()
   306  
   307  		// 3. Fill in the remaining table info
   308  		sql = fmt.Sprintf(ReadTableRemainTemplate, g.schema, CheckpointTableNameTable)
   309  		sql = strings.ReplaceAll(sql, "?", tableName)
   310  		rs, err = s.Execute(ctx, sql)
   311  		if err != nil {
   312  			return errors.Trace(err)
   313  		}
   314  		r = rs[0]
   315  		defer r.Close()
   316  		req = r.NewChunk()
   317  		err = r.Next(ctx, req)
   318  		if err != nil {
   319  			return err
   320  		}
   321  		if req.NumRows() == 0 {
   322  			return nil
   323  		}
   324  
   325  		row := req.GetRow(0)
   326  		cp.Status = CheckpointStatus(row.GetUint64(0))
   327  		cp.AllocBase = row.GetInt64(1)
   328  		cp.TableID = row.GetInt64(2)
   329  		return nil
   330  	})
   331  
   332  	if err != nil {
   333  		return nil, errors.Trace(err)
   334  	}
   335  
   336  	return cp, nil
   337  }
   338  
   339  func (g GlueCheckpointsDB) Close() error {
   340  	return nil
   341  }
   342  
   343  func (g GlueCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableName string, checkpointMap map[int32]*EngineCheckpoint) error {
   344  	logger := log.With(zap.String("table", tableName))
   345  	se, err := g.getSessionFunc()
   346  	if err != nil {
   347  		return errors.Trace(err)
   348  	}
   349  	defer se.Close()
   350  
   351  	err = Transact(ctx, "update engine checkpoints", se, logger, func(c context.Context, s Session) error {
   352  		engineStmt, _, _, err := s.PrepareStmt(fmt.Sprintf(ReplaceEngineTemplate, g.schema, CheckpointTableNameEngine))
   353  		if err != nil {
   354  			return errors.Trace(err)
   355  		}
   356  		defer s.DropPreparedStmt(engineStmt)
   357  
   358  		chunkStmt, _, _, err := s.PrepareStmt(fmt.Sprintf(ReplaceChunkTemplate, g.schema, CheckpointTableNameChunk))
   359  		if err != nil {
   360  			return errors.Trace(err)
   361  		}
   362  		defer s.DropPreparedStmt(chunkStmt)
   363  
   364  		for engineID, engine := range checkpointMap {
   365  			_, err := s.ExecutePreparedStmt(c, engineStmt, []types.Datum{
   366  				types.NewStringDatum(tableName),
   367  				types.NewIntDatum(int64(engineID)),
   368  				types.NewUintDatum(uint64(engine.Status)),
   369  			})
   370  			if err != nil {
   371  				return errors.Trace(err)
   372  			}
   373  			for _, value := range engine.Chunks {
   374  				columnPerm, err := json.Marshal(value.ColumnPermutation)
   375  				if err != nil {
   376  					return errors.Trace(err)
   377  				}
   378  				_, err = s.ExecutePreparedStmt(c, chunkStmt, []types.Datum{
   379  					types.NewStringDatum(tableName),
   380  					types.NewIntDatum(int64(engineID)),
   381  					types.NewStringDatum(value.Key.Path),
   382  					types.NewIntDatum(value.Key.Offset),
   383  					types.NewIntDatum(int64(value.FileMeta.Type)),
   384  					types.NewIntDatum(int64(value.FileMeta.Compression)),
   385  					types.NewStringDatum(value.FileMeta.SortKey),
   386  					types.NewIntDatum(value.FileMeta.FileSize),
   387  					types.NewBytesDatum(columnPerm),
   388  					types.NewIntDatum(value.Chunk.Offset),
   389  					types.NewIntDatum(value.Chunk.EndOffset),
   390  					types.NewIntDatum(value.Chunk.PrevRowIDMax),
   391  					types.NewIntDatum(value.Chunk.RowIDMax),
   392  					types.NewIntDatum(value.Timestamp),
   393  				})
   394  				if err != nil {
   395  					return errors.Trace(err)
   396  				}
   397  			}
   398  		}
   399  		return nil
   400  	})
   401  	return errors.Trace(err)
   402  }
   403  
   404  func (g GlueCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) {
   405  	logger := log.L()
   406  	se, err := g.getSessionFunc()
   407  	if err != nil {
   408  		log.L().Error("can't get a session to update GlueCheckpointsDB", zap.Error(errors.Trace(err)))
   409  		return
   410  	}
   411  	defer se.Close()
   412  
   413  	chunkQuery := fmt.Sprintf(UpdateChunkTemplate, g.schema, CheckpointTableNameChunk)
   414  	rebaseQuery := fmt.Sprintf(UpdateTableRebaseTemplate, g.schema, CheckpointTableNameTable)
   415  	tableStatusQuery := fmt.Sprintf(UpdateTableStatusTemplate, g.schema, CheckpointTableNameTable)
   416  	engineStatusQuery := fmt.Sprintf(UpdateEngineTemplate, g.schema, CheckpointTableNameEngine)
   417  	err = Transact(context.Background(), "update checkpoints", se, logger, func(c context.Context, s Session) error {
   418  		chunkStmt, _, _, err := s.PrepareStmt(chunkQuery)
   419  		if err != nil {
   420  			return errors.Trace(err)
   421  		}
   422  		defer s.DropPreparedStmt(chunkStmt)
   423  		rebaseStmt, _, _, err := s.PrepareStmt(rebaseQuery)
   424  		if err != nil {
   425  			return errors.Trace(err)
   426  		}
   427  		defer s.DropPreparedStmt(rebaseStmt)
   428  		tableStatusStmt, _, _, err := s.PrepareStmt(tableStatusQuery)
   429  		if err != nil {
   430  			return errors.Trace(err)
   431  		}
   432  		defer s.DropPreparedStmt(tableStatusStmt)
   433  		engineStatusStmt, _, _, err := s.PrepareStmt(engineStatusQuery)
   434  		if err != nil {
   435  			return errors.Trace(err)
   436  		}
   437  		defer s.DropPreparedStmt(engineStatusStmt)
   438  
   439  		for tableName, cpd := range checkpointDiffs {
   440  			if cpd.hasStatus {
   441  				_, err := s.ExecutePreparedStmt(c, tableStatusStmt, []types.Datum{
   442  					types.NewUintDatum(uint64(cpd.status)),
   443  					types.NewStringDatum(tableName),
   444  				})
   445  				if err != nil {
   446  					return errors.Trace(err)
   447  				}
   448  			}
   449  			if cpd.hasRebase {
   450  				_, err := s.ExecutePreparedStmt(c, rebaseStmt, []types.Datum{
   451  					types.NewIntDatum(cpd.allocBase),
   452  					types.NewStringDatum(tableName),
   453  				})
   454  				if err != nil {
   455  					return errors.Trace(err)
   456  				}
   457  			}
   458  			for engineID, engineDiff := range cpd.engines {
   459  				if engineDiff.hasStatus {
   460  					_, err := s.ExecutePreparedStmt(c, engineStatusStmt, []types.Datum{
   461  						types.NewUintDatum(uint64(engineDiff.status)),
   462  						types.NewStringDatum(tableName),
   463  						types.NewIntDatum(int64(engineID)),
   464  					})
   465  					if err != nil {
   466  						return errors.Trace(err)
   467  					}
   468  				}
   469  				for key, diff := range engineDiff.chunks {
   470  					columnPerm, err := json.Marshal(diff.columnPermutation)
   471  					if err != nil {
   472  						return errors.Trace(err)
   473  					}
   474  					_, err = s.ExecutePreparedStmt(c, chunkStmt, []types.Datum{
   475  						types.NewIntDatum(diff.pos),
   476  						types.NewIntDatum(diff.rowID),
   477  						types.NewUintDatum(diff.checksum.SumSize()),
   478  						types.NewUintDatum(diff.checksum.SumKVS()),
   479  						types.NewUintDatum(diff.checksum.Sum()),
   480  						types.NewBytesDatum(columnPerm),
   481  						types.NewStringDatum(tableName),
   482  						types.NewIntDatum(int64(engineID)),
   483  						types.NewStringDatum(key.Path),
   484  						types.NewIntDatum(key.Offset),
   485  					})
   486  					if err != nil {
   487  						return errors.Trace(err)
   488  					}
   489  				}
   490  			}
   491  		}
   492  		return nil
   493  	})
   494  	if err != nil {
   495  		log.L().Error("save checkpoint failed", zap.Error(err))
   496  	}
   497  }
   498  
   499  func (g GlueCheckpointsDB) RemoveCheckpoint(ctx context.Context, tableName string) error {
   500  	logger := log.With(zap.String("table", tableName))
   501  	se, err := g.getSessionFunc()
   502  	if err != nil {
   503  		return errors.Trace(err)
   504  	}
   505  	defer se.Close()
   506  
   507  	if tableName == "all" {
   508  		return common.Retry("remove all checkpoints", logger, func() error {
   509  			_, err := se.Execute(ctx, "DROP SCHEMA "+g.schema)
   510  			return err
   511  		})
   512  	}
   513  	tableName = common.InterpolateMySQLString(tableName)
   514  	deleteChunkQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, g.schema, CheckpointTableNameChunk)
   515  	deleteChunkQuery = strings.ReplaceAll(deleteChunkQuery, "?", tableName)
   516  	deleteEngineQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, g.schema, CheckpointTableNameEngine)
   517  	deleteEngineQuery = strings.ReplaceAll(deleteEngineQuery, "?", tableName)
   518  	deleteTableQuery := fmt.Sprintf(DeleteCheckpointRecordTemplate, g.schema, CheckpointTableNameTable)
   519  	deleteTableQuery = strings.ReplaceAll(deleteTableQuery, "?", tableName)
   520  
   521  	return errors.Trace(Transact(ctx, "remove checkpoints", se, logger, func(c context.Context, s Session) error {
   522  		if _, e := s.Execute(c, deleteChunkQuery); e != nil {
   523  			return e
   524  		}
   525  		if _, e := s.Execute(c, deleteEngineQuery); e != nil {
   526  			return e
   527  		}
   528  		if _, e := s.Execute(c, deleteTableQuery); e != nil {
   529  			return e
   530  		}
   531  		return nil
   532  	}))
   533  }
   534  
   535  func (g GlueCheckpointsDB) MoveCheckpoints(ctx context.Context, taskID int64) error {
   536  	newSchema := fmt.Sprintf("`%s.%d.bak`", g.schema[1:len(g.schema)-1], taskID)
   537  	logger := log.With(zap.Int64("taskID", taskID))
   538  	se, err := g.getSessionFunc()
   539  	if err != nil {
   540  		return errors.Trace(err)
   541  	}
   542  	defer se.Close()
   543  
   544  	err = common.Retry("create backup checkpoints schema", logger, func() error {
   545  		_, err := se.Execute(ctx, "CREATE SCHEMA IF NOT EXISTS "+newSchema)
   546  		return err
   547  	})
   548  	if err != nil {
   549  		return errors.Trace(err)
   550  	}
   551  	for _, tbl := range []string{CheckpointTableNameChunk, CheckpointTableNameEngine,
   552  		CheckpointTableNameTable, CheckpointTableNameTask} {
   553  		query := fmt.Sprintf("RENAME TABLE %[1]s.%[3]s TO %[2]s.%[3]s", g.schema, newSchema, tbl)
   554  		err := common.Retry(fmt.Sprintf("move %s checkpoints table", tbl), logger, func() error {
   555  			_, err := se.Execute(ctx, query)
   556  			return err
   557  		})
   558  		if err != nil {
   559  			return errors.Trace(err)
   560  		}
   561  	}
   562  	return nil
   563  }
   564  
   565  func (g GlueCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[string][]int32, error) {
   566  	se, err := g.getSessionFunc()
   567  	if err != nil {
   568  		return nil, errors.Trace(err)
   569  	}
   570  	defer se.Close()
   571  
   572  	var targetTables map[string][]int32
   573  
   574  	// lightning didn't check CheckpointStatusMaxInvalid before this function is called, so we skip invalid ones
   575  	// engines should exist if
   576  	// 1. table status is earlier than CheckpointStatusIndexImported, and
   577  	// 2. engine status is earlier than CheckpointStatusImported, and
   578  	// 3. chunk has been read
   579  	query := fmt.Sprintf(`
   580  		SELECT DISTINCT t.table_name, c.engine_id 
   581  		FROM %s.%s t, %s.%s c, %s.%s e 
   582  		WHERE t.table_name = c.table_name AND t.table_name = e.table_name AND c.engine_id = e.engine_id 
   583  			AND %d < t.status AND t.status < %d 
   584  			AND %d < e.status AND e.status < %d
   585  			AND c.pos > c.offset;`,
   586  		g.schema, CheckpointTableNameTable, g.schema, CheckpointTableNameChunk, g.schema, CheckpointTableNameEngine,
   587  		CheckpointStatusMaxInvalid, CheckpointStatusIndexImported,
   588  		CheckpointStatusMaxInvalid, CheckpointStatusImported)
   589  
   590  	err = common.Retry("get local storing tables", log.L(), func() error {
   591  		targetTables = make(map[string][]int32)
   592  		rs, err := se.Execute(ctx, query)
   593  		if err != nil {
   594  			return errors.Trace(err)
   595  		}
   596  		rows, err := drainFirstRecordSet(ctx, rs)
   597  		if err != nil {
   598  			return errors.Trace(err)
   599  		}
   600  
   601  		for _, row := range rows {
   602  			tableName := row.GetString(0)
   603  			engineID := int32(row.GetInt64(1))
   604  			targetTables[tableName] = append(targetTables[tableName], engineID)
   605  		}
   606  		return nil
   607  	})
   608  	if err != nil {
   609  		return nil, errors.Trace(err)
   610  	}
   611  
   612  	return targetTables, err
   613  }
   614  
   615  func (g GlueCheckpointsDB) IgnoreErrorCheckpoint(ctx context.Context, tableName string) error {
   616  	logger := log.With(zap.String("table", tableName))
   617  	se, err := g.getSessionFunc()
   618  	if err != nil {
   619  		return errors.Trace(err)
   620  	}
   621  	defer se.Close()
   622  
   623  	var colName string
   624  	if tableName == "all" {
   625  		// This will expand to `WHERE 'all' = 'all'` and effectively allowing
   626  		// all tables to be included.
   627  		colName = "'all'"
   628  	} else {
   629  		colName = "table_name"
   630  	}
   631  
   632  	tableName = common.InterpolateMySQLString(tableName)
   633  
   634  	engineQuery := fmt.Sprintf(`
   635  		UPDATE %s.%s SET status = %d WHERE %s = %s AND status <= %d;
   636  	`, g.schema, CheckpointTableNameEngine, CheckpointStatusLoaded, colName, tableName, CheckpointStatusMaxInvalid)
   637  	tableQuery := fmt.Sprintf(`
   638  		UPDATE %s.%s SET status = %d WHERE %s = %s AND status <= %d;
   639  	`, g.schema, CheckpointTableNameTable, CheckpointStatusLoaded, colName, tableName, CheckpointStatusMaxInvalid)
   640  	return errors.Trace(Transact(ctx, "ignore error checkpoints", se, logger, func(c context.Context, s Session) error {
   641  		if _, e := s.Execute(c, engineQuery); e != nil {
   642  			return e
   643  		}
   644  		if _, e := s.Execute(c, tableQuery); e != nil {
   645  			return e
   646  		}
   647  		return nil
   648  	}))
   649  }
   650  
   651  func (g GlueCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tableName string) ([]DestroyedTableCheckpoint, error) {
   652  	logger := log.With(zap.String("table", tableName))
   653  	se, err := g.getSessionFunc()
   654  	if err != nil {
   655  		return nil, errors.Trace(err)
   656  	}
   657  	defer se.Close()
   658  
   659  	var colName, aliasedColName string
   660  
   661  	if tableName == "all" {
   662  		// These will expand to `WHERE 'all' = 'all'` and effectively allowing
   663  		// all tables to be included.
   664  		colName = "'all'"
   665  		aliasedColName = "'all'"
   666  	} else {
   667  		colName = "table_name"
   668  		aliasedColName = "t.table_name"
   669  	}
   670  
   671  	tableName = common.InterpolateMySQLString(tableName)
   672  
   673  	selectQuery := fmt.Sprintf(`
   674  		SELECT
   675  			t.table_name,
   676  			COALESCE(MIN(e.engine_id), 0),
   677  			COALESCE(MAX(e.engine_id), -1)
   678  		FROM %[1]s.%[4]s t
   679  		LEFT JOIN %[1]s.%[5]s e ON t.table_name = e.table_name
   680  		WHERE %[2]s = %[6]s AND t.status <= %[3]d
   681  		GROUP BY t.table_name;
   682  	`, g.schema, aliasedColName, CheckpointStatusMaxInvalid, CheckpointTableNameTable, CheckpointTableNameEngine, tableName)
   683  	deleteChunkQuery := fmt.Sprintf(`
   684  		DELETE FROM %[1]s.%[4]s WHERE table_name IN (SELECT table_name FROM %[1]s.%[5]s WHERE %[2]s = %[6]s AND status <= %[3]d)
   685  	`, g.schema, colName, CheckpointStatusMaxInvalid, CheckpointTableNameChunk, CheckpointTableNameTable, tableName)
   686  	deleteEngineQuery := fmt.Sprintf(`
   687  		DELETE FROM %[1]s.%[4]s WHERE table_name IN (SELECT table_name FROM %[1]s.%[5]s WHERE %[2]s = %[6]s AND status <= %[3]d)
   688  	`, g.schema, colName, CheckpointStatusMaxInvalid, CheckpointTableNameEngine, CheckpointTableNameTable, tableName)
   689  	deleteTableQuery := fmt.Sprintf(`
   690  		DELETE FROM %s.%s WHERE %s = %s AND status <= %d
   691  	`, g.schema, CheckpointTableNameTable, colName, tableName, CheckpointStatusMaxInvalid)
   692  
   693  	var targetTables []DestroyedTableCheckpoint
   694  	err = Transact(ctx, "destroy error checkpoints", se, logger, func(c context.Context, s Session) error {
   695  		// clean because it's in a retry
   696  		targetTables = nil
   697  		rs, err := s.Execute(c, selectQuery)
   698  		if err != nil {
   699  			return errors.Trace(err)
   700  		}
   701  		r := rs[0]
   702  		req := r.NewChunk()
   703  		it := chunk.NewIterator4Chunk(req)
   704  		for {
   705  			err = r.Next(ctx, req)
   706  			if err != nil {
   707  				r.Close()
   708  				return err
   709  			}
   710  			if req.NumRows() == 0 {
   711  				break
   712  			}
   713  
   714  			for row := it.Begin(); row != it.End(); row = it.Next() {
   715  				var dtc DestroyedTableCheckpoint
   716  				dtc.TableName = row.GetString(0)
   717  				dtc.MinEngineID = int32(row.GetInt64(1))
   718  				dtc.MaxEngineID = int32(row.GetInt64(2))
   719  				targetTables = append(targetTables, dtc)
   720  			}
   721  		}
   722  		r.Close()
   723  
   724  		if _, e := s.Execute(c, deleteChunkQuery); e != nil {
   725  			return errors.Trace(e)
   726  		}
   727  		if _, e := s.Execute(c, deleteEngineQuery); e != nil {
   728  			return errors.Trace(e)
   729  		}
   730  		if _, e := s.Execute(c, deleteTableQuery); e != nil {
   731  			return errors.Trace(e)
   732  		}
   733  		return nil
   734  	})
   735  
   736  	if err != nil {
   737  		return nil, errors.Trace(err)
   738  	}
   739  
   740  	return targetTables, nil
   741  }
   742  
   743  func (g GlueCheckpointsDB) DumpTables(ctx context.Context, csv io.Writer) error {
   744  	return errors.Errorf("dumping glue checkpoint into CSV not unsupported")
   745  }
   746  
   747  func (g GlueCheckpointsDB) DumpEngines(ctx context.Context, csv io.Writer) error {
   748  	return errors.Errorf("dumping glue checkpoint into CSV not unsupported")
   749  }
   750  
   751  func (g GlueCheckpointsDB) DumpChunks(ctx context.Context, csv io.Writer) error {
   752  	return errors.Errorf("dumping glue checkpoint into CSV not unsupported")
   753  }
   754  
   755  func Transact(ctx context.Context, purpose string, s Session, logger log.Logger, action func(context.Context, Session) error) error {
   756  	return common.Retry(purpose, logger, func() error {
   757  		_, err := s.Execute(ctx, "BEGIN")
   758  		if err != nil {
   759  			return errors.Annotate(err, "begin transaction failed")
   760  		}
   761  		err = action(ctx, s)
   762  		if err != nil {
   763  			s.RollbackTxn(ctx)
   764  			return err
   765  		}
   766  		err = s.CommitTxn(ctx)
   767  		if err != nil {
   768  			return errors.Annotate(err, "commit transaction failed")
   769  		}
   770  		return nil
   771  	})
   772  }
   773  
   774  // TODO: will use drainFirstRecordSet to reduce repeat in GlueCheckpointsDB later
   775  func drainFirstRecordSet(ctx context.Context, rss []sqlexec.RecordSet) ([]chunk.Row, error) {
   776  	if len(rss) != 1 {
   777  		return nil, errors.New("given result set doesn't have length 1")
   778  	}
   779  	rs := rss[0]
   780  	var rows []chunk.Row
   781  	req := rs.NewChunk()
   782  	for {
   783  		err := rs.Next(ctx, req)
   784  		if err != nil || req.NumRows() == 0 {
   785  			rs.Close()
   786  			return rows, err
   787  		}
   788  		iter := chunk.NewIterator4Chunk(req)
   789  		for r := iter.Begin(); r != iter.End(); r = iter.Next() {
   790  			rows = append(rows, r)
   791  		}
   792  		req = chunk.Renew(req, 1024)
   793  	}
   794  }