github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/restore/table_restore.go (about)

     1  // Copyright 2021 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package restore
    15  
    16  import (
    17  	"context"
    18  	"sort"
    19  	"sync"
    20  	"time"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/failpoint"
    24  	"github.com/pingcap/parser/model"
    25  	"github.com/pingcap/tidb/meta/autoid"
    26  	"github.com/pingcap/tidb/table"
    27  	"github.com/pingcap/tidb/table/tables"
    28  	"go.uber.org/multierr"
    29  	"go.uber.org/zap"
    30  
    31  	"github.com/pingcap/br/pkg/lightning/backend"
    32  	"github.com/pingcap/br/pkg/lightning/backend/kv"
    33  	"github.com/pingcap/br/pkg/lightning/checkpoints"
    34  	"github.com/pingcap/br/pkg/lightning/common"
    35  	"github.com/pingcap/br/pkg/lightning/config"
    36  	"github.com/pingcap/br/pkg/lightning/glue"
    37  	"github.com/pingcap/br/pkg/lightning/log"
    38  	"github.com/pingcap/br/pkg/lightning/metric"
    39  	"github.com/pingcap/br/pkg/lightning/mydump"
    40  	verify "github.com/pingcap/br/pkg/lightning/verification"
    41  	"github.com/pingcap/br/pkg/lightning/worker"
    42  	"github.com/pingcap/br/pkg/utils"
    43  )
    44  
    45  type TableRestore struct {
    46  	// The unique table name in the form "`db`.`tbl`".
    47  	tableName string
    48  	dbInfo    *checkpoints.TidbDBInfo
    49  	tableInfo *checkpoints.TidbTableInfo
    50  	tableMeta *mydump.MDTableMeta
    51  	encTable  table.Table
    52  	alloc     autoid.Allocators
    53  	logger    log.Logger
    54  
    55  	ignoreColumns []string
    56  }
    57  
    58  func NewTableRestore(
    59  	tableName string,
    60  	tableMeta *mydump.MDTableMeta,
    61  	dbInfo *checkpoints.TidbDBInfo,
    62  	tableInfo *checkpoints.TidbTableInfo,
    63  	cp *checkpoints.TableCheckpoint,
    64  	ignoreColumns []string,
    65  ) (*TableRestore, error) {
    66  	idAlloc := kv.NewPanickingAllocators(cp.AllocBase)
    67  	tbl, err := tables.TableFromMeta(idAlloc, tableInfo.Core)
    68  	if err != nil {
    69  		return nil, errors.Annotatef(err, "failed to tables.TableFromMeta %s", tableName)
    70  	}
    71  
    72  	return &TableRestore{
    73  		tableName:     tableName,
    74  		dbInfo:        dbInfo,
    75  		tableInfo:     tableInfo,
    76  		tableMeta:     tableMeta,
    77  		encTable:      tbl,
    78  		alloc:         idAlloc,
    79  		logger:        log.With(zap.String("table", tableName)),
    80  		ignoreColumns: ignoreColumns,
    81  	}, nil
    82  }
    83  
    84  func (tr *TableRestore) Close() {
    85  	tr.encTable = nil
    86  	tr.logger.Info("restore done")
    87  }
    88  
    89  func (tr *TableRestore) populateChunks(ctx context.Context, rc *Controller, cp *checkpoints.TableCheckpoint) error {
    90  	task := tr.logger.Begin(zap.InfoLevel, "load engines and files")
    91  	chunks, err := mydump.MakeTableRegions(ctx, tr.tableMeta, len(tr.tableInfo.Core.Columns), rc.cfg, rc.ioWorkers, rc.store)
    92  	if err == nil {
    93  		timestamp := time.Now().Unix()
    94  		failpoint.Inject("PopulateChunkTimestamp", func(v failpoint.Value) {
    95  			timestamp = int64(v.(int))
    96  		})
    97  		for _, chunk := range chunks {
    98  			engine, found := cp.Engines[chunk.EngineID]
    99  			if !found {
   100  				engine = &checkpoints.EngineCheckpoint{
   101  					Status: checkpoints.CheckpointStatusLoaded,
   102  				}
   103  				cp.Engines[chunk.EngineID] = engine
   104  			}
   105  			ccp := &checkpoints.ChunkCheckpoint{
   106  				Key: checkpoints.ChunkCheckpointKey{
   107  					Path:   chunk.FileMeta.Path,
   108  					Offset: chunk.Chunk.Offset,
   109  				},
   110  				FileMeta:          chunk.FileMeta,
   111  				ColumnPermutation: nil,
   112  				Chunk:             chunk.Chunk,
   113  				Timestamp:         timestamp,
   114  			}
   115  			if len(chunk.Chunk.Columns) > 0 {
   116  				perms, err := parseColumnPermutations(tr.tableInfo.Core, chunk.Chunk.Columns, tr.ignoreColumns)
   117  				if err != nil {
   118  					return errors.Trace(err)
   119  				}
   120  				ccp.ColumnPermutation = perms
   121  			}
   122  			engine.Chunks = append(engine.Chunks, ccp)
   123  		}
   124  
   125  		// Add index engine checkpoint
   126  		cp.Engines[indexEngineID] = &checkpoints.EngineCheckpoint{Status: checkpoints.CheckpointStatusLoaded}
   127  	}
   128  	task.End(zap.ErrorLevel, err,
   129  		zap.Int("enginesCnt", len(cp.Engines)),
   130  		zap.Int("filesCnt", len(chunks)),
   131  	)
   132  	return err
   133  }
   134  
   135  func (t *TableRestore) RebaseChunkRowIDs(cp *checkpoints.TableCheckpoint, rowIDBase int64) {
   136  	if rowIDBase == 0 {
   137  		return
   138  	}
   139  	for _, engine := range cp.Engines {
   140  		for _, chunk := range engine.Chunks {
   141  			chunk.Chunk.PrevRowIDMax += rowIDBase
   142  			chunk.Chunk.RowIDMax += rowIDBase
   143  		}
   144  	}
   145  }
   146  
   147  // initializeColumns computes the "column permutation" for an INSERT INTO
   148  // statement. Suppose a table has columns (a, b, c, d) in canonical order, and
   149  // we execute `INSERT INTO (d, b, a) VALUES ...`, we will need to remap the
   150  // columns as:
   151  //
   152  // - column `a` is at position 2
   153  // - column `b` is at position 1
   154  // - column `c` is missing
   155  // - column `d` is at position 0
   156  //
   157  // The column permutation of (d, b, a) is set to be [2, 1, -1, 0].
   158  //
   159  // The argument `columns` _must_ be in lower case.
   160  func (tr *TableRestore) initializeColumns(columns []string, ccp *checkpoints.ChunkCheckpoint) error {
   161  	colPerm, err := createColumnPermutation(columns, tr.ignoreColumns, tr.tableInfo.Core)
   162  	if err != nil {
   163  		return err
   164  	}
   165  	ccp.ColumnPermutation = colPerm
   166  	return nil
   167  }
   168  
   169  func createColumnPermutation(columns []string, ignoreColumns []string, tableInfo *model.TableInfo) ([]int, error) {
   170  	var colPerm []int
   171  	if len(columns) == 0 {
   172  		colPerm = make([]int, 0, len(tableInfo.Columns)+1)
   173  		shouldIncludeRowID := common.TableHasAutoRowID(tableInfo)
   174  
   175  		// no provided columns, so use identity permutation.
   176  		for i := range tableInfo.Columns {
   177  			colPerm = append(colPerm, i)
   178  		}
   179  		if shouldIncludeRowID {
   180  			colPerm = append(colPerm, -1)
   181  		}
   182  	} else {
   183  		var err error
   184  		colPerm, err = parseColumnPermutations(tableInfo, columns, ignoreColumns)
   185  		if err != nil {
   186  			return nil, errors.Trace(err)
   187  		}
   188  	}
   189  	return colPerm, nil
   190  }
   191  
   192  func (tr *TableRestore) restoreEngines(pCtx context.Context, rc *Controller, cp *checkpoints.TableCheckpoint) error {
   193  	indexEngineCp := cp.Engines[indexEngineID]
   194  	if indexEngineCp == nil {
   195  		return errors.Errorf("table %v index engine checkpoint not found", tr.tableName)
   196  	}
   197  
   198  	ctx, cancel := context.WithCancel(pCtx)
   199  	defer cancel()
   200  
   201  	// The table checkpoint status set to `CheckpointStatusIndexImported` only if
   202  	// both all data engines and the index engine had been imported to TiKV.
   203  	// But persist index engine checkpoint status and table checkpoint status are
   204  	// not an atomic operation, so `cp.Status < CheckpointStatusIndexImported`
   205  	// but `indexEngineCp.Status == CheckpointStatusImported` could happen
   206  	// when kill lightning after saving index engine checkpoint status before saving
   207  	// table checkpoint status.
   208  	var closedIndexEngine *backend.ClosedEngine
   209  	var restoreErr error
   210  	// if index-engine checkpoint is lower than `CheckpointStatusClosed`, there must be
   211  	// data-engines that need to be restore or import. Otherwise, all data-engines should
   212  	// be finished already.
   213  
   214  	idxEngineCfg := &backend.EngineConfig{
   215  		TableInfo: tr.tableInfo,
   216  	}
   217  	if indexEngineCp.Status < checkpoints.CheckpointStatusClosed {
   218  		indexWorker := rc.indexWorkers.Apply()
   219  		defer rc.indexWorkers.Recycle(indexWorker)
   220  
   221  		if rc.cfg.TikvImporter.Backend == config.BackendLocal {
   222  			// for index engine, the estimate factor is non-clustered index count
   223  			idxCnt := len(tr.tableInfo.Core.Indices)
   224  			if !common.TableHasAutoRowID(tr.tableInfo.Core) {
   225  				idxCnt--
   226  			}
   227  			threshold := estimateCompactionThreshold(cp, int64(idxCnt))
   228  			idxEngineCfg.Local = &backend.LocalEngineConfig{
   229  				Compact:            threshold > 0,
   230  				CompactConcurrency: 4,
   231  				CompactThreshold:   threshold,
   232  			}
   233  		}
   234  		// import backend can't reopen engine if engine is closed, so
   235  		// only open index engine if any data engines don't finish writing.
   236  		var indexEngine *backend.OpenedEngine
   237  		var err error
   238  		for engineID, engine := range cp.Engines {
   239  			if engineID == indexEngineID {
   240  				continue
   241  			}
   242  			if engine.Status < checkpoints.CheckpointStatusAllWritten {
   243  				indexEngine, err = rc.backend.OpenEngine(ctx, idxEngineCfg, tr.tableName, indexEngineID)
   244  				if err != nil {
   245  					return errors.Trace(err)
   246  				}
   247  				break
   248  			}
   249  		}
   250  
   251  		logTask := tr.logger.Begin(zap.InfoLevel, "import whole table")
   252  		var wg sync.WaitGroup
   253  		var engineErr common.OnceError
   254  		setError := func(err error) {
   255  			engineErr.Set(err)
   256  			// cancel this context to fail fast
   257  			cancel()
   258  		}
   259  
   260  		type engineCheckpoint struct {
   261  			engineID   int32
   262  			checkpoint *checkpoints.EngineCheckpoint
   263  		}
   264  		allEngines := make([]engineCheckpoint, 0, len(cp.Engines))
   265  		for engineID, engine := range cp.Engines {
   266  			allEngines = append(allEngines, engineCheckpoint{engineID: engineID, checkpoint: engine})
   267  		}
   268  		sort.Slice(allEngines, func(i, j int) bool { return allEngines[i].engineID < allEngines[j].engineID })
   269  
   270  		for _, ecp := range allEngines {
   271  			engineID := ecp.engineID
   272  			engine := ecp.checkpoint
   273  			select {
   274  			case <-ctx.Done():
   275  				// Set engineErr and break this for loop to wait all the sub-routines done before return.
   276  				// Directly return may cause panic because caller will close the pebble db but some sub routines
   277  				// are still reading from or writing to the pebble db.
   278  				engineErr.Set(ctx.Err())
   279  			default:
   280  			}
   281  			if engineErr.Get() != nil {
   282  				break
   283  			}
   284  
   285  			// Should skip index engine
   286  			if engineID < 0 {
   287  				continue
   288  			}
   289  
   290  			if engine.Status < checkpoints.CheckpointStatusImported {
   291  				wg.Add(1)
   292  
   293  				// If the number of chunks is small, it means that this engine may be finished in a few times.
   294  				// We do not limit it in TableConcurrency
   295  				restoreWorker := rc.tableWorkers.Apply()
   296  				go func(w *worker.Worker, eid int32, ecp *checkpoints.EngineCheckpoint) {
   297  					defer wg.Done()
   298  					engineLogTask := tr.logger.With(zap.Int32("engineNumber", eid)).Begin(zap.InfoLevel, "restore engine")
   299  					dataClosedEngine, err := tr.restoreEngine(ctx, rc, indexEngine, eid, ecp)
   300  					engineLogTask.End(zap.ErrorLevel, err)
   301  					rc.tableWorkers.Recycle(w)
   302  					if err == nil {
   303  						dataWorker := rc.closedEngineLimit.Apply()
   304  						defer rc.closedEngineLimit.Recycle(dataWorker)
   305  						err = tr.importEngine(ctx, dataClosedEngine, rc, eid, ecp)
   306  					}
   307  					if err != nil {
   308  						setError(err)
   309  					}
   310  				}(restoreWorker, engineID, engine)
   311  			}
   312  		}
   313  
   314  		wg.Wait()
   315  
   316  		restoreErr = engineErr.Get()
   317  		logTask.End(zap.ErrorLevel, restoreErr)
   318  		if restoreErr != nil {
   319  			return errors.Trace(restoreErr)
   320  		}
   321  
   322  		if indexEngine != nil {
   323  			closedIndexEngine, restoreErr = indexEngine.Close(ctx, idxEngineCfg)
   324  		} else {
   325  			closedIndexEngine, restoreErr = rc.backend.UnsafeCloseEngine(ctx, idxEngineCfg, tr.tableName, indexEngineID)
   326  		}
   327  
   328  		rc.saveStatusCheckpoint(tr.tableName, indexEngineID, restoreErr, checkpoints.CheckpointStatusClosed)
   329  	} else if indexEngineCp.Status == checkpoints.CheckpointStatusClosed {
   330  		// If index engine file has been closed but not imported only if context cancel occurred
   331  		// when `importKV()` execution, so `UnsafeCloseEngine` and continue import it.
   332  		closedIndexEngine, restoreErr = rc.backend.UnsafeCloseEngine(ctx, idxEngineCfg, tr.tableName, indexEngineID)
   333  	}
   334  	if restoreErr != nil {
   335  		return errors.Trace(restoreErr)
   336  	}
   337  
   338  	if cp.Status < checkpoints.CheckpointStatusIndexImported {
   339  		var err error
   340  		if indexEngineCp.Status < checkpoints.CheckpointStatusImported {
   341  			err = tr.importKV(ctx, closedIndexEngine, rc, indexEngineID)
   342  		}
   343  
   344  		failpoint.Inject("FailBeforeIndexEngineImported", func() {
   345  			panic("forcing failure due to FailBeforeIndexEngineImported")
   346  		})
   347  
   348  		rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusIndexImported)
   349  		if err != nil {
   350  			return errors.Trace(err)
   351  		}
   352  	}
   353  	return nil
   354  }
   355  
   356  func (tr *TableRestore) restoreEngine(
   357  	pCtx context.Context,
   358  	rc *Controller,
   359  	indexEngine *backend.OpenedEngine,
   360  	engineID int32,
   361  	cp *checkpoints.EngineCheckpoint,
   362  ) (*backend.ClosedEngine, error) {
   363  	ctx, cancel := context.WithCancel(pCtx)
   364  	defer cancel()
   365  	// all data has finished written, we can close the engine directly.
   366  	if cp.Status >= checkpoints.CheckpointStatusAllWritten {
   367  		engineCfg := &backend.EngineConfig{
   368  			TableInfo: tr.tableInfo,
   369  		}
   370  		closedEngine, err := rc.backend.UnsafeCloseEngine(ctx, engineCfg, tr.tableName, engineID)
   371  		// If any error occurred, recycle worker immediately
   372  		if err != nil {
   373  			return closedEngine, errors.Trace(err)
   374  		}
   375  		return closedEngine, nil
   376  	}
   377  
   378  	// if the key are ordered, LocalWrite can optimize the writing.
   379  	// table has auto-incremented _tidb_rowid must satisfy following restrictions:
   380  	// - clustered index disable and primary key is not number
   381  	// - no auto random bits (auto random or shard row id)
   382  	// - no partition table
   383  	// - no explicit _tidb_rowid field (At this time we can't determine if the source file contains _tidb_rowid field,
   384  	//   so we will do this check in LocalWriter when the first row is received.)
   385  	hasAutoIncrementAutoID := common.TableHasAutoRowID(tr.tableInfo.Core) &&
   386  		tr.tableInfo.Core.AutoRandomBits == 0 && tr.tableInfo.Core.ShardRowIDBits == 0 &&
   387  		tr.tableInfo.Core.Partition == nil
   388  	dataWriterCfg := &backend.LocalWriterConfig{
   389  		IsKVSorted: hasAutoIncrementAutoID,
   390  	}
   391  
   392  	logTask := tr.logger.With(zap.Int32("engineNumber", engineID)).Begin(zap.InfoLevel, "encode kv data and write")
   393  	dataEngineCfg := &backend.EngineConfig{
   394  		TableInfo: tr.tableInfo,
   395  		Local:     &backend.LocalEngineConfig{},
   396  	}
   397  	if !tr.tableMeta.IsRowOrdered {
   398  		dataEngineCfg.Local.Compact = true
   399  		dataEngineCfg.Local.CompactConcurrency = 4
   400  		dataEngineCfg.Local.CompactThreshold = compactionUpperThreshold
   401  	}
   402  	dataEngine, err := rc.backend.OpenEngine(ctx, dataEngineCfg, tr.tableName, engineID)
   403  	if err != nil {
   404  		return nil, errors.Trace(err)
   405  	}
   406  
   407  	var wg sync.WaitGroup
   408  	var chunkErr common.OnceError
   409  
   410  	type chunkFlushStatus struct {
   411  		dataStatus  backend.ChunkFlushStatus
   412  		indexStatus backend.ChunkFlushStatus
   413  		chunkCp     *checkpoints.ChunkCheckpoint
   414  	}
   415  
   416  	// chunks that are finished writing, but checkpoints are not finished due to flush not finished.
   417  	var checkFlushLock sync.Mutex
   418  	flushPendingChunks := make([]chunkFlushStatus, 0, 16)
   419  
   420  	chunkCpChan := make(chan *checkpoints.ChunkCheckpoint, 16)
   421  	go func() {
   422  		for {
   423  			select {
   424  			case cp, ok := <-chunkCpChan:
   425  				if !ok {
   426  					return
   427  				}
   428  				saveCheckpoint(rc, tr, engineID, cp)
   429  			case <-ctx.Done():
   430  				return
   431  			}
   432  		}
   433  	}()
   434  
   435  	// Restore table data
   436  	for chunkIndex, chunk := range cp.Chunks {
   437  		if chunk.Chunk.Offset >= chunk.Chunk.EndOffset {
   438  			continue
   439  		}
   440  
   441  		checkFlushLock.Lock()
   442  		finished := 0
   443  		for _, c := range flushPendingChunks {
   444  			if c.indexStatus.Flushed() && c.dataStatus.Flushed() {
   445  				chunkCpChan <- c.chunkCp
   446  				finished++
   447  			} else {
   448  				break
   449  			}
   450  		}
   451  		if finished > 0 {
   452  			flushPendingChunks = flushPendingChunks[finished:]
   453  		}
   454  		checkFlushLock.Unlock()
   455  
   456  		select {
   457  		case <-pCtx.Done():
   458  			return nil, pCtx.Err()
   459  		default:
   460  		}
   461  
   462  		if chunkErr.Get() != nil {
   463  			break
   464  		}
   465  
   466  		// Flows :
   467  		// 	1. read mydump file
   468  		// 	2. sql -> kvs
   469  		// 	3. load kvs data (into kv deliver server)
   470  		// 	4. flush kvs data (into tikv node)
   471  		cr, err := newChunkRestore(ctx, chunkIndex, rc.cfg, chunk, rc.ioWorkers, rc.store, tr.tableInfo)
   472  		if err != nil {
   473  			return nil, errors.Trace(err)
   474  		}
   475  		var remainChunkCnt float64
   476  		if chunk.Chunk.Offset < chunk.Chunk.EndOffset {
   477  			remainChunkCnt = float64(chunk.Chunk.EndOffset-chunk.Chunk.Offset) / float64(chunk.Chunk.EndOffset-chunk.Key.Offset)
   478  			metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending).Add(remainChunkCnt)
   479  		}
   480  
   481  		restoreWorker := rc.regionWorkers.Apply()
   482  		wg.Add(1)
   483  
   484  		dataWriter, err := dataEngine.LocalWriter(ctx, dataWriterCfg)
   485  		if err != nil {
   486  			return nil, errors.Trace(err)
   487  		}
   488  
   489  		indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{})
   490  		if err != nil {
   491  			return nil, errors.Trace(err)
   492  		}
   493  
   494  		go func(w *worker.Worker, cr *chunkRestore) {
   495  			// Restore a chunk.
   496  			defer func() {
   497  				cr.close()
   498  				wg.Done()
   499  				rc.regionWorkers.Recycle(w)
   500  			}()
   501  			metric.ChunkCounter.WithLabelValues(metric.ChunkStateRunning).Add(remainChunkCnt)
   502  			err := cr.restore(ctx, tr, engineID, dataWriter, indexWriter, rc)
   503  			var dataFlushStatus, indexFlushStaus backend.ChunkFlushStatus
   504  			if err == nil {
   505  				dataFlushStatus, err = dataWriter.Close(ctx)
   506  			}
   507  			if err == nil {
   508  				indexFlushStaus, err = indexWriter.Close(ctx)
   509  			}
   510  			if err == nil {
   511  				metric.ChunkCounter.WithLabelValues(metric.ChunkStateFinished).Add(remainChunkCnt)
   512  				metric.BytesCounter.WithLabelValues(metric.TableStateWritten).Add(float64(cr.chunk.Checksum.SumSize()))
   513  				if dataFlushStatus != nil && indexFlushStaus != nil {
   514  					if dataFlushStatus.Flushed() && indexFlushStaus.Flushed() {
   515  						saveCheckpoint(rc, tr, engineID, cr.chunk)
   516  					} else {
   517  						checkFlushLock.Lock()
   518  						flushPendingChunks = append(flushPendingChunks, chunkFlushStatus{
   519  							dataStatus:  dataFlushStatus,
   520  							indexStatus: indexFlushStaus,
   521  							chunkCp:     cr.chunk,
   522  						})
   523  						checkFlushLock.Unlock()
   524  					}
   525  				}
   526  			} else {
   527  				metric.ChunkCounter.WithLabelValues(metric.ChunkStateFailed).Add(remainChunkCnt)
   528  				chunkErr.Set(err)
   529  				cancel()
   530  			}
   531  		}(restoreWorker, cr)
   532  	}
   533  
   534  	wg.Wait()
   535  
   536  	// Report some statistics into the log for debugging.
   537  	totalKVSize := uint64(0)
   538  	totalSQLSize := int64(0)
   539  	for _, chunk := range cp.Chunks {
   540  		totalKVSize += chunk.Checksum.SumSize()
   541  		totalSQLSize += chunk.Chunk.EndOffset - chunk.Chunk.Offset
   542  	}
   543  
   544  	err = chunkErr.Get()
   545  	logTask.End(zap.ErrorLevel, err,
   546  		zap.Int64("read", totalSQLSize),
   547  		zap.Uint64("written", totalKVSize),
   548  	)
   549  
   550  	trySavePendingChunks := func(flushCtx context.Context) error {
   551  		checkFlushLock.Lock()
   552  		cnt := 0
   553  		for _, chunk := range flushPendingChunks {
   554  			if chunk.dataStatus.Flushed() && chunk.indexStatus.Flushed() {
   555  				saveCheckpoint(rc, tr, engineID, chunk.chunkCp)
   556  				cnt++
   557  			} else {
   558  				break
   559  			}
   560  		}
   561  		flushPendingChunks = flushPendingChunks[cnt:]
   562  		checkFlushLock.Unlock()
   563  		return nil
   564  	}
   565  
   566  	// in local mode, this check-point make no sense, because we don't do flush now,
   567  	// so there may be data lose if exit at here. So we don't write this checkpoint
   568  	// here like other mode.
   569  	if !rc.isLocalBackend() {
   570  		rc.saveStatusCheckpoint(tr.tableName, engineID, err, checkpoints.CheckpointStatusAllWritten)
   571  	}
   572  	if err != nil {
   573  		// if process is canceled, we should flush all chunk checkpoints for local backend
   574  		if rc.isLocalBackend() && common.IsContextCanceledError(err) {
   575  			// ctx is canceled, so to avoid Close engine failed, we use `context.Background()` here
   576  			if _, err2 := dataEngine.Close(context.Background(), dataEngineCfg); err2 != nil {
   577  				log.L().Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2))
   578  				return nil, errors.Trace(err)
   579  			}
   580  			if err2 := trySavePendingChunks(context.Background()); err2 != nil {
   581  				log.L().Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2))
   582  			}
   583  		}
   584  		return nil, errors.Trace(err)
   585  	}
   586  
   587  	closedDataEngine, err := dataEngine.Close(ctx, dataEngineCfg)
   588  	// For local backend, if checkpoint is enabled, we must flush index engine to avoid data loss.
   589  	// this flush action impact up to 10% of the performance, so we only do it if necessary.
   590  	if err == nil && rc.cfg.Checkpoint.Enable && rc.isLocalBackend() {
   591  		if err = indexEngine.Flush(ctx); err != nil {
   592  			return nil, errors.Trace(err)
   593  		}
   594  		if err = trySavePendingChunks(ctx); err != nil {
   595  			return nil, errors.Trace(err)
   596  		}
   597  	}
   598  	rc.saveStatusCheckpoint(tr.tableName, engineID, err, checkpoints.CheckpointStatusClosed)
   599  	if err != nil {
   600  		// If any error occurred, recycle worker immediately
   601  		return nil, errors.Trace(err)
   602  	}
   603  	return closedDataEngine, nil
   604  }
   605  
   606  func (tr *TableRestore) importEngine(
   607  	ctx context.Context,
   608  	closedEngine *backend.ClosedEngine,
   609  	rc *Controller,
   610  	engineID int32,
   611  	cp *checkpoints.EngineCheckpoint,
   612  ) error {
   613  	if cp.Status >= checkpoints.CheckpointStatusImported {
   614  		return nil
   615  	}
   616  
   617  	// 1. calling import
   618  	if err := tr.importKV(ctx, closedEngine, rc, engineID); err != nil {
   619  		return errors.Trace(err)
   620  	}
   621  
   622  	// 2. perform a level-1 compact if idling.
   623  	if rc.cfg.PostRestore.Level1Compact && rc.compactState.CAS(compactStateIdle, compactStateDoing) {
   624  		go func() {
   625  			// we ignore level-1 compact failure since it is not fatal.
   626  			// no need log the error, it is done in (*Importer).Compact already.
   627  			_ = rc.doCompact(ctx, Level1Compact)
   628  			rc.compactState.Store(compactStateIdle)
   629  		}()
   630  	}
   631  
   632  	return nil
   633  }
   634  
   635  // postProcess execute rebase-auto-id/checksum/analyze according to the task config.
   636  //
   637  // if the parameter forcePostProcess to true, postProcess force run checksum and analyze even if the
   638  // post-process-at-last config is true. And if this two phases are skipped, the first return value will be true.
   639  func (tr *TableRestore) postProcess(
   640  	ctx context.Context,
   641  	rc *Controller,
   642  	cp *checkpoints.TableCheckpoint,
   643  	forcePostProcess bool,
   644  	metaMgr tableMetaMgr,
   645  ) (bool, error) {
   646  	// there are no data in this table, no need to do post process
   647  	// this is important for tables that are just the dump table of views
   648  	// because at this stage, the table was already deleted and replaced by the related view
   649  	if !rc.backend.ShouldPostProcess() || len(cp.Engines) == 1 {
   650  		return false, nil
   651  	}
   652  
   653  	// 3. alter table set auto_increment
   654  	if cp.Status < checkpoints.CheckpointStatusAlteredAutoInc {
   655  		rc.alterTableLock.Lock()
   656  		tblInfo := tr.tableInfo.Core
   657  		var err error
   658  		if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() {
   659  			err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoRandomType).Base()+1)
   660  		} else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil {
   661  			// only alter auto increment id iff table contains auto-increment column or generated handle
   662  			err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.RowIDAllocType).Base()+1)
   663  		}
   664  		rc.alterTableLock.Unlock()
   665  		rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusAlteredAutoInc)
   666  		if err != nil {
   667  			return false, err
   668  		}
   669  		cp.Status = checkpoints.CheckpointStatusAlteredAutoInc
   670  	}
   671  
   672  	// tidb backend don't need checksum & analyze
   673  	if !rc.backend.ShouldPostProcess() {
   674  		tr.logger.Debug("skip checksum & analyze, not supported by this backend")
   675  		rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, nil, checkpoints.CheckpointStatusAnalyzeSkipped)
   676  		return false, nil
   677  	}
   678  
   679  	w := rc.checksumWorks.Apply()
   680  	defer rc.checksumWorks.Recycle(w)
   681  
   682  	finished := true
   683  	if cp.Status < checkpoints.CheckpointStatusChecksummed {
   684  		// 4. do table checksum
   685  		var localChecksum verify.KVChecksum
   686  		for _, engine := range cp.Engines {
   687  			for _, chunk := range engine.Chunks {
   688  				localChecksum.Add(&chunk.Checksum)
   689  			}
   690  		}
   691  
   692  		if rc.cfg.PostRestore.Checksum == config.OpLevelOff {
   693  			tr.logger.Info("skip checksum")
   694  			rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, nil, checkpoints.CheckpointStatusChecksumSkipped)
   695  		} else {
   696  			if forcePostProcess || !rc.cfg.PostRestore.PostProcessAtLast {
   697  				tr.logger.Info("local checksum", zap.Object("checksum", &localChecksum))
   698  				if rc.cfg.TikvImporter.DuplicateDetection {
   699  					if err := rc.backend.CollectLocalDuplicateRows(ctx, tr.encTable); err != nil {
   700  						tr.logger.Error("collect local duplicate keys failed", log.ShortError(err))
   701  					}
   702  				}
   703  				needChecksum, baseTotalChecksum, err := metaMgr.CheckAndUpdateLocalChecksum(ctx, &localChecksum)
   704  				if err != nil {
   705  					return false, err
   706  				}
   707  				if !needChecksum {
   708  					return false, nil
   709  				}
   710  				if rc.cfg.TikvImporter.DuplicateDetection {
   711  					if err := rc.backend.CollectRemoteDuplicateRows(ctx, tr.encTable); err != nil {
   712  						tr.logger.Error("collect remote duplicate keys failed", log.ShortError(err))
   713  						err = nil
   714  					}
   715  				}
   716  				if cp.Checksum.SumKVS() > 0 || baseTotalChecksum.SumKVS() > 0 {
   717  					localChecksum.Add(&cp.Checksum)
   718  					localChecksum.Add(baseTotalChecksum)
   719  					tr.logger.Info("merged local checksum", zap.Object("checksum", &localChecksum))
   720  				}
   721  
   722  				remoteChecksum, err := DoChecksum(ctx, tr.tableInfo)
   723  				// TODO: If there are duplicate keys, do not set the `ChecksumMismatch` error
   724  				err = tr.compareChecksum(remoteChecksum, localChecksum)
   725  				// with post restore level 'optional', we will skip checksum error
   726  				if rc.cfg.PostRestore.Checksum == config.OpLevelOptional {
   727  					if err != nil {
   728  						tr.logger.Warn("compare checksum failed, will skip this error and go on", log.ShortError(err))
   729  						err = nil
   730  					}
   731  				}
   732  				if err == nil {
   733  					err = metaMgr.FinishTable(ctx)
   734  				}
   735  
   736  				rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusChecksummed)
   737  				if err != nil {
   738  					return false, errors.Trace(err)
   739  				}
   740  
   741  				cp.Status = checkpoints.CheckpointStatusChecksummed
   742  			} else {
   743  				finished = false
   744  			}
   745  		}
   746  	}
   747  	if !finished {
   748  		return !finished, nil
   749  	}
   750  
   751  	// 5. do table analyze
   752  	if cp.Status < checkpoints.CheckpointStatusAnalyzed {
   753  		switch {
   754  		case rc.cfg.PostRestore.Analyze == config.OpLevelOff:
   755  			tr.logger.Info("skip analyze")
   756  			rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, nil, checkpoints.CheckpointStatusAnalyzeSkipped)
   757  			cp.Status = checkpoints.CheckpointStatusAnalyzed
   758  		case forcePostProcess || !rc.cfg.PostRestore.PostProcessAtLast:
   759  			err := tr.analyzeTable(ctx, rc.tidbGlue.GetSQLExecutor())
   760  			// witch post restore level 'optional', we will skip analyze error
   761  			if rc.cfg.PostRestore.Analyze == config.OpLevelOptional {
   762  				if err != nil {
   763  					tr.logger.Warn("analyze table failed, will skip this error and go on", log.ShortError(err))
   764  					err = nil
   765  				}
   766  			}
   767  			rc.saveStatusCheckpoint(tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusAnalyzed)
   768  			if err != nil {
   769  				return false, errors.Trace(err)
   770  			}
   771  			cp.Status = checkpoints.CheckpointStatusAnalyzed
   772  		default:
   773  			finished = false
   774  		}
   775  	}
   776  
   777  	return !finished, nil
   778  }
   779  
   780  func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignoreColumns []string) ([]int, error) {
   781  	colPerm := make([]int, 0, len(tableInfo.Columns)+1)
   782  
   783  	columnMap := make(map[string]int)
   784  	for i, column := range columns {
   785  		columnMap[column] = i
   786  	}
   787  
   788  	ignoreMap := make(map[string]int)
   789  	for _, column := range ignoreColumns {
   790  		if i, ok := columnMap[column]; ok {
   791  			ignoreMap[column] = i
   792  		}
   793  	}
   794  
   795  	tableColumnMap := make(map[string]int)
   796  	for i, col := range tableInfo.Columns {
   797  		tableColumnMap[col.Name.L] = i
   798  	}
   799  
   800  	// check if there are some unknown columns
   801  	var unknownCols []string
   802  	for _, c := range columns {
   803  		if _, ok := tableColumnMap[c]; !ok && c != model.ExtraHandleName.L {
   804  			if _, ignore := ignoreMap[c]; !ignore {
   805  				unknownCols = append(unknownCols, c)
   806  			}
   807  		}
   808  	}
   809  
   810  	if len(unknownCols) > 0 {
   811  		return colPerm, errors.Errorf("unknown columns in header %s", unknownCols)
   812  	}
   813  
   814  	for _, colInfo := range tableInfo.Columns {
   815  		if i, ok := columnMap[colInfo.Name.L]; ok {
   816  			if _, ignore := ignoreMap[colInfo.Name.L]; !ignore {
   817  				colPerm = append(colPerm, i)
   818  			} else {
   819  				log.L().Debug("column ignored by user requirements",
   820  					zap.Stringer("table", tableInfo.Name),
   821  					zap.String("colName", colInfo.Name.O),
   822  					zap.Stringer("colType", &colInfo.FieldType),
   823  				)
   824  				colPerm = append(colPerm, -1)
   825  			}
   826  		} else {
   827  			if len(colInfo.GeneratedExprString) == 0 {
   828  				log.L().Warn("column missing from data file, going to fill with default value",
   829  					zap.Stringer("table", tableInfo.Name),
   830  					zap.String("colName", colInfo.Name.O),
   831  					zap.Stringer("colType", &colInfo.FieldType),
   832  				)
   833  			}
   834  			colPerm = append(colPerm, -1)
   835  		}
   836  	}
   837  	if i, ok := columnMap[model.ExtraHandleName.L]; ok {
   838  		colPerm = append(colPerm, i)
   839  	} else if common.TableHasAutoRowID(tableInfo) {
   840  		colPerm = append(colPerm, -1)
   841  	}
   842  
   843  	return colPerm, nil
   844  }
   845  
   846  func (tr *TableRestore) importKV(
   847  	ctx context.Context,
   848  	closedEngine *backend.ClosedEngine,
   849  	rc *Controller,
   850  	engineID int32,
   851  ) error {
   852  	task := closedEngine.Logger().Begin(zap.InfoLevel, "import and cleanup engine")
   853  
   854  	err := closedEngine.Import(ctx)
   855  	rc.saveStatusCheckpoint(tr.tableName, engineID, err, checkpoints.CheckpointStatusImported)
   856  	// Also cleanup engine when encountered ErrDuplicateDetected, since all duplicates kv pairs are recorded.
   857  	if err == nil {
   858  		err = multierr.Append(err, closedEngine.Cleanup(ctx))
   859  	}
   860  
   861  	dur := task.End(zap.ErrorLevel, err)
   862  
   863  	if err != nil {
   864  		return errors.Trace(err)
   865  	}
   866  
   867  	metric.ImportSecondsHistogram.Observe(dur.Seconds())
   868  
   869  	failpoint.Inject("SlowDownImport", func() {})
   870  
   871  	return nil
   872  }
   873  
   874  // do checksum for each table.
   875  func (tr *TableRestore) compareChecksum(remoteChecksum *RemoteChecksum, localChecksum verify.KVChecksum) error {
   876  	if remoteChecksum.Checksum != localChecksum.Sum() ||
   877  		remoteChecksum.TotalKVs != localChecksum.SumKVS() ||
   878  		remoteChecksum.TotalBytes != localChecksum.SumSize() {
   879  		return errors.Errorf("checksum mismatched remote vs local => (checksum: %d vs %d) (total_kvs: %d vs %d) (total_bytes:%d vs %d)",
   880  			remoteChecksum.Checksum, localChecksum.Sum(),
   881  			remoteChecksum.TotalKVs, localChecksum.SumKVS(),
   882  			remoteChecksum.TotalBytes, localChecksum.SumSize(),
   883  		)
   884  	}
   885  
   886  	tr.logger.Info("checksum pass", zap.Object("local", &localChecksum))
   887  	return nil
   888  }
   889  
   890  func (tr *TableRestore) analyzeTable(ctx context.Context, g glue.SQLExecutor) error {
   891  	task := tr.logger.Begin(zap.InfoLevel, "analyze")
   892  	err := g.ExecuteWithLog(ctx, "ANALYZE TABLE "+tr.tableName, "analyze table", tr.logger)
   893  	task.End(zap.ErrorLevel, err)
   894  	return err
   895  }
   896  
   897  // estimate SST files compression threshold by total row file size
   898  // with a higher compression threshold, the compression time increases, but the iteration time decreases.
   899  // Try to limit the total SST files number under 500. But size compress 32GB SST files cost about 20min,
   900  // we set the upper bound to 32GB to avoid too long compression time.
   901  // factor is the non-clustered(1 for data engine and number of non-clustered index count for index engine).
   902  func estimateCompactionThreshold(cp *checkpoints.TableCheckpoint, factor int64) int64 {
   903  	totalRawFileSize := int64(0)
   904  	var lastFile string
   905  	for _, engineCp := range cp.Engines {
   906  		for _, chunk := range engineCp.Chunks {
   907  			if chunk.FileMeta.Path == lastFile {
   908  				continue
   909  			}
   910  			size := chunk.FileMeta.FileSize
   911  			if chunk.FileMeta.Type == mydump.SourceTypeParquet {
   912  				// parquet file is compressed, thus estimates with a factor of 2
   913  				size *= 2
   914  			}
   915  			totalRawFileSize += size
   916  			lastFile = chunk.FileMeta.Path
   917  		}
   918  	}
   919  	totalRawFileSize *= factor
   920  
   921  	// try restrict the total file number within 512
   922  	threshold := totalRawFileSize / 512
   923  	threshold = utils.NextPowerOfTwo(threshold)
   924  	if threshold < compactionLowerThreshold {
   925  		// disable compaction if threshold is smaller than lower bound
   926  		threshold = 0
   927  	} else if threshold > compactionUpperThreshold {
   928  		threshold = compactionUpperThreshold
   929  	}
   930  
   931  	return threshold
   932  }