github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/backend/local/local.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 local
    15  
    16  import (
    17  	"bytes"
    18  	"container/heap"
    19  	"context"
    20  	"encoding/binary"
    21  	"encoding/json"
    22  	"fmt"
    23  	"io"
    24  	"math"
    25  	"os"
    26  	"path/filepath"
    27  	"sort"
    28  	"strings"
    29  	"sync"
    30  	"time"
    31  
    32  	"github.com/cockroachdb/pebble"
    33  	"github.com/cockroachdb/pebble/sstable"
    34  	"github.com/coreos/go-semver/semver"
    35  	"github.com/docker/go-units"
    36  	"github.com/google/btree"
    37  	"github.com/google/uuid"
    38  	"github.com/pingcap/errors"
    39  	"github.com/pingcap/failpoint"
    40  	"github.com/pingcap/kvproto/pkg/errorpb"
    41  	sst "github.com/pingcap/kvproto/pkg/import_sstpb"
    42  	"github.com/pingcap/kvproto/pkg/kvrpcpb"
    43  	"github.com/pingcap/kvproto/pkg/metapb"
    44  	"github.com/pingcap/parser/model"
    45  	"github.com/pingcap/parser/mysql"
    46  	"github.com/pingcap/tidb/distsql"
    47  	"github.com/pingcap/tidb/table"
    48  	"github.com/pingcap/tidb/tablecodec"
    49  	"github.com/pingcap/tidb/util/codec"
    50  	"github.com/pingcap/tidb/util/hack"
    51  	"github.com/pingcap/tidb/util/ranger"
    52  	"github.com/tikv/client-go/v2/oracle"
    53  	"go.uber.org/atomic"
    54  	"go.uber.org/multierr"
    55  	"go.uber.org/zap"
    56  	"golang.org/x/sync/errgroup"
    57  	"google.golang.org/grpc"
    58  	"google.golang.org/grpc/backoff"
    59  	"google.golang.org/grpc/codes"
    60  	"google.golang.org/grpc/credentials"
    61  	"google.golang.org/grpc/keepalive"
    62  	"google.golang.org/grpc/status"
    63  
    64  	"github.com/pingcap/br/pkg/conn"
    65  	"github.com/pingcap/br/pkg/lightning/backend"
    66  	"github.com/pingcap/br/pkg/lightning/backend/kv"
    67  	"github.com/pingcap/br/pkg/lightning/checkpoints"
    68  	"github.com/pingcap/br/pkg/lightning/common"
    69  	"github.com/pingcap/br/pkg/lightning/config"
    70  	"github.com/pingcap/br/pkg/lightning/glue"
    71  	"github.com/pingcap/br/pkg/lightning/log"
    72  	"github.com/pingcap/br/pkg/lightning/manual"
    73  	"github.com/pingcap/br/pkg/lightning/metric"
    74  	"github.com/pingcap/br/pkg/lightning/tikv"
    75  	"github.com/pingcap/br/pkg/lightning/worker"
    76  	"github.com/pingcap/br/pkg/logutil"
    77  	"github.com/pingcap/br/pkg/membuf"
    78  	"github.com/pingcap/br/pkg/pdutil"
    79  	split "github.com/pingcap/br/pkg/restore"
    80  	"github.com/pingcap/br/pkg/utils"
    81  	"github.com/pingcap/br/pkg/version"
    82  )
    83  
    84  const (
    85  	dialTimeout             = 5 * time.Minute
    86  	bigValueSize            = 1 << 16 // 64K
    87  	maxRetryTimes           = 5
    88  	defaultRetryBackoffTime = 3 * time.Second
    89  
    90  	gRPCKeepAliveTime    = 10 * time.Minute
    91  	gRPCKeepAliveTimeout = 5 * time.Minute
    92  	gRPCBackOffMaxDelay  = 10 * time.Minute
    93  
    94  	// See: https://github.com/tikv/tikv/blob/e030a0aae9622f3774df89c62f21b2171a72a69e/etc/config-template.toml#L360
    95  	regionMaxKeyCount      = 1_440_000
    96  	defaultRegionSplitSize = 96 * units.MiB
    97  
    98  	propRangeIndex = "tikv.range_index"
    99  
   100  	defaultPropSizeIndexDistance = 4 * units.MiB
   101  	defaultPropKeysIndexDistance = 40 * 1024
   102  
   103  	// the lower threshold of max open files for pebble db.
   104  	openFilesLowerThreshold = 128
   105  
   106  	duplicateDBName       = "duplicates"
   107  	remoteDuplicateDBName = "remote_duplicates"
   108  	scanRegionLimit       = 128
   109  )
   110  
   111  var (
   112  	// Local backend is compatible with TiDB [4.0.0, NextMajorVersion).
   113  	localMinTiDBVersion = *semver.New("4.0.0")
   114  	localMinTiKVVersion = *semver.New("4.0.0")
   115  	localMinPDVersion   = *semver.New("4.0.0")
   116  	localMaxTiDBVersion = version.NextMajorVersion()
   117  	localMaxTiKVVersion = version.NextMajorVersion()
   118  	localMaxPDVersion   = version.NextMajorVersion()
   119  	tiFlashMinVersion   = *semver.New("4.0.5")
   120  
   121  	errorEngineClosed = errors.New("engine is closed")
   122  )
   123  
   124  var (
   125  	engineMetaKey      = []byte{0, 'm', 'e', 't', 'a'}
   126  	normalIterStartKey = []byte{1}
   127  )
   128  
   129  // Range record start and end key for localStoreDir.DB
   130  // so we can write it to tikv in streaming
   131  type Range struct {
   132  	start []byte
   133  	end   []byte
   134  }
   135  
   136  // localFileMeta contains some field that is necessary to continue the engine restore/import process.
   137  // These field should be written to disk when we update chunk checkpoint
   138  type localFileMeta struct {
   139  	TS uint64 `json:"ts"`
   140  	// Length is the number of KV pairs stored by the engine.
   141  	Length atomic.Int64 `json:"length"`
   142  	// TotalSize is the total pre-compressed KV byte size stored by engine.
   143  	TotalSize atomic.Int64 `json:"total_size"`
   144  	// Duplicates is the number of duplicates kv pairs detected when importing. Note that the value is
   145  	// probably larger than real value, because we may import same range more than once. For accurate
   146  	// information, you should iterate the duplicate db after import is finished.
   147  	Duplicates atomic.Int64 `json:"duplicates"`
   148  }
   149  
   150  type importMutexState uint32
   151  
   152  const (
   153  	importMutexStateImport importMutexState = 1 << iota
   154  	importMutexStateClose
   155  	// importMutexStateReadLock is a special state because in this state we lock engine with read lock
   156  	// and add isImportingAtomic with this value. In other state, we directly store with the state value.
   157  	// so this must always the last value of this enum.
   158  	importMutexStateReadLock
   159  )
   160  
   161  // either a sstMeta or a flush message
   162  type metaOrFlush struct {
   163  	meta    *sstMeta
   164  	flushCh chan struct{}
   165  }
   166  
   167  type File struct {
   168  	localFileMeta
   169  	closed       atomic.Bool
   170  	db           *pebble.DB
   171  	UUID         uuid.UUID
   172  	localWriters sync.Map
   173  
   174  	// isImportingAtomic is an atomic variable indicating whether this engine is importing.
   175  	// This should not be used as a "spin lock" indicator.
   176  	isImportingAtomic atomic.Uint32
   177  	// flush and ingest sst hold the rlock, other operation hold the wlock.
   178  	mutex sync.RWMutex
   179  
   180  	ctx            context.Context
   181  	cancel         context.CancelFunc
   182  	sstDir         string
   183  	sstMetasChan   chan metaOrFlush
   184  	ingestErr      common.OnceError
   185  	wg             sync.WaitGroup
   186  	sstIngester    sstIngester
   187  	finishedRanges syncedRanges
   188  
   189  	// sst seq lock
   190  	seqLock sync.Mutex
   191  	// seq number for incoming sst meta
   192  	nextSeq int32
   193  	// max seq of sst metas ingested into pebble
   194  	finishedMetaSeq atomic.Int32
   195  
   196  	config    backend.LocalEngineConfig
   197  	tableInfo *checkpoints.TidbTableInfo
   198  
   199  	// total size of SST files waiting to be ingested
   200  	pendingFileSize atomic.Int64
   201  
   202  	// statistics for pebble kv iter.
   203  	importedKVSize  atomic.Int64
   204  	importedKVCount atomic.Int64
   205  
   206  	keyAdapter         KeyAdapter
   207  	duplicateDetection bool
   208  	duplicateDB        *pebble.DB
   209  }
   210  
   211  func (e *File) setError(err error) {
   212  	if err != nil {
   213  		e.ingestErr.Set(err)
   214  		e.cancel()
   215  	}
   216  }
   217  
   218  func (e *File) Close() error {
   219  	log.L().Debug("closing local engine", zap.Stringer("engine", e.UUID), zap.Stack("stack"))
   220  	if e.db == nil {
   221  		return nil
   222  	}
   223  	err := errors.Trace(e.db.Close())
   224  	e.db = nil
   225  	return err
   226  }
   227  
   228  // Cleanup remove meta and db files
   229  func (e *File) Cleanup(dataDir string) error {
   230  	if err := os.RemoveAll(e.sstDir); err != nil {
   231  		return errors.Trace(err)
   232  	}
   233  
   234  	dbPath := filepath.Join(dataDir, e.UUID.String())
   235  	return os.RemoveAll(dbPath)
   236  }
   237  
   238  // Exist checks if db folder existing (meta sometimes won't flush before lightning exit)
   239  func (e *File) Exist(dataDir string) error {
   240  	dbPath := filepath.Join(dataDir, e.UUID.String())
   241  	if _, err := os.Stat(dbPath); err != nil {
   242  		return err
   243  	}
   244  	return nil
   245  }
   246  
   247  func (e *File) getSizeProperties() (*sizeProperties, error) {
   248  	sstables, err := e.db.SSTables(pebble.WithProperties())
   249  	if err != nil {
   250  		log.L().Warn("get table properties failed", zap.Stringer("engine", e.UUID), log.ShortError(err))
   251  		return nil, errors.Trace(err)
   252  	}
   253  
   254  	sizeProps := newSizeProperties()
   255  	for _, level := range sstables {
   256  		for _, info := range level {
   257  			if prop, ok := info.Properties.UserProperties[propRangeIndex]; ok {
   258  				data := hack.Slice(prop)
   259  				rangeProps, err := decodeRangeProperties(data)
   260  				if err != nil {
   261  					log.L().Warn("decodeRangeProperties failed", zap.Stringer("engine", e.UUID),
   262  						zap.Stringer("fileNum", info.FileNum), log.ShortError(err))
   263  					return nil, errors.Trace(err)
   264  				}
   265  				if e.duplicateDetection {
   266  					newRangeProps := make(rangeProperties, 0, len(rangeProps))
   267  					for _, p := range rangeProps {
   268  						if !bytes.Equal(p.Key, engineMetaKey) {
   269  							p.Key, _, _, err = e.keyAdapter.Decode(nil, p.Key)
   270  							if err != nil {
   271  								log.L().Warn(
   272  									"decodeRangeProperties failed because the props key is invalid",
   273  									zap.Stringer("engine", e.UUID),
   274  									zap.Stringer("fileNum", info.FileNum),
   275  									zap.Binary("key", p.Key),
   276  								)
   277  								return nil, errors.Trace(err)
   278  							}
   279  							newRangeProps = append(newRangeProps, p)
   280  						}
   281  					}
   282  					rangeProps = newRangeProps
   283  				}
   284  				sizeProps.addAll(rangeProps)
   285  			}
   286  		}
   287  	}
   288  
   289  	return sizeProps, nil
   290  }
   291  
   292  func isStateLocked(state importMutexState) bool {
   293  	return state&(importMutexStateClose|importMutexStateImport) != 0
   294  }
   295  
   296  func (e *File) isLocked() bool {
   297  	// the engine is locked only in import or close state.
   298  	return isStateLocked(importMutexState(e.isImportingAtomic.Load()))
   299  }
   300  
   301  func (e *File) getEngineFileSize() backend.EngineFileSize {
   302  	metrics := e.db.Metrics()
   303  	total := metrics.Total()
   304  	var memSize int64
   305  	e.localWriters.Range(func(k, v interface{}) bool {
   306  		w := k.(*Writer)
   307  		if w.writer != nil {
   308  			memSize += int64(w.writer.writer.EstimatedSize())
   309  		} else {
   310  			// if kvs are still in memory, only calculate half of the total size
   311  			// in our tests, SST file size is about 50% of the raw kv size
   312  			memSize += w.batchSize / 2
   313  		}
   314  
   315  		return true
   316  	})
   317  
   318  	pendingSize := e.pendingFileSize.Load()
   319  	// TODO: should also add the in-processing compaction sst writer size into MemSize
   320  	return backend.EngineFileSize{
   321  		UUID:        e.UUID,
   322  		DiskSize:    total.Size + pendingSize,
   323  		MemSize:     memSize,
   324  		IsImporting: e.isLocked(),
   325  	}
   326  }
   327  
   328  // rLock locks the local file with shard read state. Only used for flush and ingest SST files.
   329  func (e *File) rLock() {
   330  	e.mutex.RLock()
   331  	e.isImportingAtomic.Add(uint32(importMutexStateReadLock))
   332  }
   333  
   334  func (e *File) rUnlock() {
   335  	if e == nil {
   336  		return
   337  	}
   338  
   339  	e.isImportingAtomic.Sub(uint32(importMutexStateReadLock))
   340  	e.mutex.RUnlock()
   341  }
   342  
   343  // lock locks the local file for importing.
   344  func (e *File) lock(state importMutexState) {
   345  	e.mutex.Lock()
   346  	e.isImportingAtomic.Store(uint32(state))
   347  }
   348  
   349  // lockUnless tries to lock the local file unless it is already locked into the state given by
   350  // ignoreStateMask. Returns whether the lock is successful.
   351  func (e *File) lockUnless(newState, ignoreStateMask importMutexState) bool {
   352  	curState := e.isImportingAtomic.Load()
   353  	if curState&uint32(ignoreStateMask) != 0 {
   354  		return false
   355  	}
   356  	e.lock(newState)
   357  	return true
   358  }
   359  
   360  // tryRLock tries to read-lock the local file unless it is already write locked.
   361  // Returns whether the lock is successful.
   362  func (e *File) tryRLock() bool {
   363  	curState := e.isImportingAtomic.Load()
   364  	// engine is in import/close state.
   365  	if isStateLocked(importMutexState(curState)) {
   366  		return false
   367  	}
   368  	e.rLock()
   369  	return true
   370  }
   371  
   372  func (e *File) unlock() {
   373  	if e == nil {
   374  		return
   375  	}
   376  	e.isImportingAtomic.Store(0)
   377  	e.mutex.Unlock()
   378  }
   379  
   380  type metaSeq struct {
   381  	// the sequence for this flush message, a flush call can return only if
   382  	// all the other flush will lower `flushSeq` are done
   383  	flushSeq int32
   384  	// the max sstMeta sequence number in this flush, after the flush is done (all SSTs are ingested),
   385  	// we can save chunks will a lower meta sequence number safely.
   386  	metaSeq int32
   387  }
   388  
   389  type metaSeqHeap struct {
   390  	arr []metaSeq
   391  }
   392  
   393  func (h *metaSeqHeap) Len() int {
   394  	return len(h.arr)
   395  }
   396  
   397  func (h *metaSeqHeap) Less(i, j int) bool {
   398  	return h.arr[i].flushSeq < h.arr[j].flushSeq
   399  }
   400  
   401  func (h *metaSeqHeap) Swap(i, j int) {
   402  	h.arr[i], h.arr[j] = h.arr[j], h.arr[i]
   403  }
   404  
   405  func (h *metaSeqHeap) Push(x interface{}) {
   406  	h.arr = append(h.arr, x.(metaSeq))
   407  }
   408  
   409  func (h *metaSeqHeap) Pop() interface{} {
   410  	item := h.arr[len(h.arr)-1]
   411  	h.arr = h.arr[:len(h.arr)-1]
   412  	return item
   413  }
   414  
   415  func (e *File) ingestSSTLoop() {
   416  	defer e.wg.Done()
   417  
   418  	type flushSeq struct {
   419  		seq int32
   420  		ch  chan struct{}
   421  	}
   422  
   423  	seq := atomic.NewInt32(0)
   424  	finishedSeq := atomic.NewInt32(0)
   425  	var seqLock sync.Mutex
   426  	// a flush is finished iff all the compaction&ingest tasks with a lower seq number are finished.
   427  	flushQueue := make([]flushSeq, 0)
   428  	// inSyncSeqs is a heap that stores all the finished compaction tasks whose seq is bigger than `finishedSeq + 1`
   429  	// this mean there are still at lease one compaction task with a lower seq unfinished.
   430  	inSyncSeqs := &metaSeqHeap{arr: make([]metaSeq, 0)}
   431  
   432  	type metaAndSeq struct {
   433  		metas []*sstMeta
   434  		seq   int32
   435  	}
   436  
   437  	concurrency := e.config.CompactConcurrency
   438  	// when compaction is disabled, ingest is an serial action, so 1 routine is enough
   439  	if !e.config.Compact {
   440  		concurrency = 1
   441  	}
   442  	metaChan := make(chan metaAndSeq, concurrency)
   443  	for i := 0; i < concurrency; i++ {
   444  		e.wg.Add(1)
   445  		go func() {
   446  			defer e.wg.Done()
   447  			defer func() {
   448  				if e.ingestErr.Get() != nil {
   449  					seqLock.Lock()
   450  					for _, f := range flushQueue {
   451  						f.ch <- struct{}{}
   452  					}
   453  					flushQueue = flushQueue[:0]
   454  					seqLock.Unlock()
   455  				}
   456  			}()
   457  			for {
   458  				select {
   459  				case <-e.ctx.Done():
   460  					return
   461  				case metas, ok := <-metaChan:
   462  					if !ok {
   463  						return
   464  					}
   465  					ingestMetas := metas.metas
   466  					if e.config.Compact {
   467  						newMeta, err := e.sstIngester.mergeSSTs(metas.metas, e.sstDir)
   468  						if err != nil {
   469  							e.setError(err)
   470  							return
   471  						}
   472  						ingestMetas = []*sstMeta{newMeta}
   473  					}
   474  					// batchIngestSSTs will change ingestMetas' order, so we record the max seq here
   475  					metasMaxSeq := ingestMetas[len(ingestMetas)-1].seq
   476  
   477  					if err := e.batchIngestSSTs(ingestMetas); err != nil {
   478  						e.setError(err)
   479  						return
   480  					}
   481  					seqLock.Lock()
   482  					finSeq := finishedSeq.Load()
   483  					if metas.seq == finSeq+1 {
   484  						finSeq = metas.seq
   485  						finMetaSeq := metasMaxSeq
   486  						for len(inSyncSeqs.arr) > 0 {
   487  							if inSyncSeqs.arr[0].flushSeq == finSeq+1 {
   488  								finSeq++
   489  								finMetaSeq = inSyncSeqs.arr[0].metaSeq
   490  								heap.Remove(inSyncSeqs, 0)
   491  							} else {
   492  								break
   493  							}
   494  						}
   495  
   496  						var flushChans []chan struct{}
   497  						for _, seq := range flushQueue {
   498  							if seq.seq <= finSeq {
   499  								flushChans = append(flushChans, seq.ch)
   500  							} else {
   501  								break
   502  							}
   503  						}
   504  						flushQueue = flushQueue[len(flushChans):]
   505  						finishedSeq.Store(finSeq)
   506  						e.finishedMetaSeq.Store(finMetaSeq)
   507  						seqLock.Unlock()
   508  						for _, c := range flushChans {
   509  							c <- struct{}{}
   510  						}
   511  					} else {
   512  						heap.Push(inSyncSeqs, metaSeq{flushSeq: metas.seq, metaSeq: metasMaxSeq})
   513  						seqLock.Unlock()
   514  					}
   515  				}
   516  			}
   517  		}()
   518  	}
   519  
   520  	compactAndIngestSSTs := func(metas []*sstMeta) {
   521  		if len(metas) > 0 {
   522  			seqLock.Lock()
   523  			metaSeq := seq.Add(1)
   524  			seqLock.Unlock()
   525  			select {
   526  			case <-e.ctx.Done():
   527  			case metaChan <- metaAndSeq{metas: metas, seq: metaSeq}:
   528  			}
   529  		}
   530  	}
   531  
   532  	pendingMetas := make([]*sstMeta, 0, 16)
   533  	totalSize := int64(0)
   534  	metasTmp := make([]*sstMeta, 0)
   535  	addMetas := func() {
   536  		if len(metasTmp) == 0 {
   537  			return
   538  		}
   539  		metas := metasTmp
   540  		metasTmp = make([]*sstMeta, 0, len(metas))
   541  		if !e.config.Compact {
   542  			compactAndIngestSSTs(metas)
   543  			return
   544  		}
   545  		for _, m := range metas {
   546  			if m.totalCount > 0 {
   547  				pendingMetas = append(pendingMetas, m)
   548  				totalSize += m.totalSize
   549  				if totalSize >= e.config.CompactThreshold {
   550  					compactMetas := pendingMetas
   551  					pendingMetas = make([]*sstMeta, 0, len(pendingMetas))
   552  					totalSize = 0
   553  					compactAndIngestSSTs(compactMetas)
   554  				}
   555  			}
   556  		}
   557  	}
   558  readMetaLoop:
   559  	for {
   560  		closed := false
   561  		select {
   562  		case <-e.ctx.Done():
   563  			close(metaChan)
   564  			return
   565  		case m, ok := <-e.sstMetasChan:
   566  			if !ok {
   567  				closed = true
   568  				break
   569  			}
   570  			if m.flushCh != nil {
   571  				// meet a flush event, we should trigger a ingest task if there are pending metas,
   572  				// and then waiting for all the running flush tasks to be done.
   573  				if len(metasTmp) > 0 {
   574  					addMetas()
   575  				}
   576  				if len(pendingMetas) > 0 {
   577  					seqLock.Lock()
   578  					metaSeq := seq.Add(1)
   579  					flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: metaSeq})
   580  					seqLock.Unlock()
   581  					select {
   582  					case metaChan <- metaAndSeq{metas: pendingMetas, seq: metaSeq}:
   583  					case <-e.ctx.Done():
   584  						close(metaChan)
   585  						return
   586  					}
   587  
   588  					pendingMetas = make([]*sstMeta, 0, len(pendingMetas))
   589  					totalSize = 0
   590  				} else {
   591  					// none remaining metas needed to be ingested
   592  					seqLock.Lock()
   593  					curSeq := seq.Load()
   594  					finSeq := finishedSeq.Load()
   595  					// if all pending SST files are written, directly do a db.Flush
   596  					if curSeq == finSeq {
   597  						seqLock.Unlock()
   598  						m.flushCh <- struct{}{}
   599  					} else {
   600  						// waiting for pending compaction tasks
   601  						flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: curSeq})
   602  						seqLock.Unlock()
   603  					}
   604  				}
   605  				continue readMetaLoop
   606  			}
   607  			metasTmp = append(metasTmp, m.meta)
   608  			// try to drain all the sst meta from the chan to make sure all the SSTs are processed before handle a flush msg.
   609  			if len(e.sstMetasChan) > 0 {
   610  				continue readMetaLoop
   611  			}
   612  
   613  			addMetas()
   614  		}
   615  		if closed {
   616  			compactAndIngestSSTs(pendingMetas)
   617  			close(metaChan)
   618  			return
   619  		}
   620  	}
   621  }
   622  
   623  func (e *File) addSST(ctx context.Context, m *sstMeta) (int32, error) {
   624  	// set pending size after SST file is generated
   625  	e.pendingFileSize.Add(m.fileSize)
   626  	// make sure sstMeta is sent into the chan in order
   627  	e.seqLock.Lock()
   628  	defer e.seqLock.Unlock()
   629  	e.nextSeq++
   630  	seq := e.nextSeq
   631  	m.seq = seq
   632  	select {
   633  	case e.sstMetasChan <- metaOrFlush{meta: m}:
   634  	case <-ctx.Done():
   635  		return 0, ctx.Err()
   636  	case <-e.ctx.Done():
   637  	}
   638  	return seq, e.ingestErr.Get()
   639  }
   640  
   641  func (e *File) batchIngestSSTs(metas []*sstMeta) error {
   642  	if len(metas) == 0 {
   643  		return nil
   644  	}
   645  	sort.Slice(metas, func(i, j int) bool {
   646  		return bytes.Compare(metas[i].minKey, metas[j].minKey) < 0
   647  	})
   648  
   649  	metaLevels := make([][]*sstMeta, 0)
   650  	for _, meta := range metas {
   651  		inserted := false
   652  		for i, l := range metaLevels {
   653  			if bytes.Compare(l[len(l)-1].maxKey, meta.minKey) >= 0 {
   654  				continue
   655  			}
   656  			metaLevels[i] = append(l, meta)
   657  			inserted = true
   658  			break
   659  		}
   660  		if !inserted {
   661  			metaLevels = append(metaLevels, []*sstMeta{meta})
   662  		}
   663  	}
   664  
   665  	for _, l := range metaLevels {
   666  		if err := e.ingestSSTs(l); err != nil {
   667  			return err
   668  		}
   669  	}
   670  	return nil
   671  }
   672  
   673  func (e *File) ingestSSTs(metas []*sstMeta) error {
   674  	// use raw RLock to avoid change the lock state during flushing.
   675  	e.mutex.RLock()
   676  	defer e.mutex.RUnlock()
   677  	if e.closed.Load() {
   678  		return errorEngineClosed
   679  	}
   680  	totalSize := int64(0)
   681  	totalCount := int64(0)
   682  	fileSize := int64(0)
   683  	for _, m := range metas {
   684  		totalSize += m.totalSize
   685  		totalCount += m.totalCount
   686  		fileSize += m.fileSize
   687  	}
   688  	log.L().Info("write data to local DB",
   689  		zap.Int64("size", totalSize),
   690  		zap.Int64("kvs", totalCount),
   691  		zap.Int("files", len(metas)),
   692  		zap.Int64("sstFileSize", fileSize),
   693  		zap.String("file", metas[0].path),
   694  		logutil.Key("firstKey", metas[0].minKey),
   695  		logutil.Key("lastKey", metas[len(metas)-1].maxKey))
   696  	if err := e.sstIngester.ingest(metas); err != nil {
   697  		return errors.Trace(err)
   698  	}
   699  	count := int64(0)
   700  	size := int64(0)
   701  	for _, m := range metas {
   702  		count += m.totalCount
   703  		size += m.totalSize
   704  	}
   705  	e.Length.Add(count)
   706  	e.TotalSize.Add(size)
   707  	return nil
   708  }
   709  
   710  func (e *File) flushLocalWriters(parentCtx context.Context) error {
   711  	eg, ctx := errgroup.WithContext(parentCtx)
   712  	e.localWriters.Range(func(k, v interface{}) bool {
   713  		eg.Go(func() error {
   714  			w := k.(*Writer)
   715  			return w.flush(ctx)
   716  		})
   717  		return true
   718  	})
   719  	return eg.Wait()
   720  }
   721  
   722  func (e *File) flushEngineWithoutLock(ctx context.Context) error {
   723  	if err := e.flushLocalWriters(ctx); err != nil {
   724  		return err
   725  	}
   726  	flushChan := make(chan struct{}, 1)
   727  	select {
   728  	case e.sstMetasChan <- metaOrFlush{flushCh: flushChan}:
   729  	case <-ctx.Done():
   730  		return ctx.Err()
   731  	case <-e.ctx.Done():
   732  		return e.ctx.Err()
   733  	}
   734  
   735  	select {
   736  	case <-flushChan:
   737  	case <-ctx.Done():
   738  		return ctx.Err()
   739  	case <-e.ctx.Done():
   740  		return e.ctx.Err()
   741  	}
   742  	if err := e.ingestErr.Get(); err != nil {
   743  		return errors.Trace(err)
   744  	}
   745  	if err := e.saveEngineMeta(); err != nil {
   746  		return err
   747  	}
   748  
   749  	flushFinishedCh, err := e.db.AsyncFlush()
   750  	if err != nil {
   751  		return errors.Trace(err)
   752  	}
   753  	select {
   754  	case <-flushFinishedCh:
   755  		return nil
   756  	case <-ctx.Done():
   757  		return ctx.Err()
   758  	case <-e.ctx.Done():
   759  		return e.ctx.Err()
   760  	}
   761  }
   762  
   763  func saveEngineMetaToDB(meta *localFileMeta, db *pebble.DB) error {
   764  	jsonBytes, err := json.Marshal(meta)
   765  	if err != nil {
   766  		return errors.Trace(err)
   767  	}
   768  	// note: we can't set Sync to true since we disabled WAL.
   769  	return db.Set(engineMetaKey, jsonBytes, &pebble.WriteOptions{Sync: false})
   770  }
   771  
   772  // saveEngineMeta saves the metadata about the DB into the DB itself.
   773  // This method should be followed by a Flush to ensure the data is actually synchronized
   774  func (e *File) saveEngineMeta() error {
   775  	log.L().Debug("save engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()),
   776  		zap.Int64("size", e.TotalSize.Load()))
   777  	return errors.Trace(saveEngineMetaToDB(&e.localFileMeta, e.db))
   778  }
   779  
   780  func (e *File) loadEngineMeta() error {
   781  	jsonBytes, closer, err := e.db.Get(engineMetaKey)
   782  	if err != nil {
   783  		if err == pebble.ErrNotFound {
   784  			log.L().Debug("local db missing engine meta", zap.Stringer("uuid", e.UUID), zap.Error(err))
   785  			return nil
   786  		}
   787  		return err
   788  	}
   789  	defer closer.Close()
   790  
   791  	if err = json.Unmarshal(jsonBytes, &e.localFileMeta); err != nil {
   792  		log.L().Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.UUID), zap.ByteString("content", jsonBytes), zap.Error(err))
   793  		return err
   794  	}
   795  	log.L().Debug("load engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()),
   796  		zap.Int64("size", e.TotalSize.Load()))
   797  	return nil
   798  }
   799  
   800  type local struct {
   801  	engines sync.Map // sync version of map[uuid.UUID]*File
   802  
   803  	pdCtl    *pdutil.PdController
   804  	conns    common.GRPCConns
   805  	splitCli split.SplitClient
   806  	tls      *common.TLS
   807  	pdAddr   string
   808  	g        glue.Glue
   809  
   810  	localStoreDir   string
   811  	regionSplitSize int64
   812  	regionSplitKeys int64
   813  
   814  	rangeConcurrency  *worker.Pool
   815  	ingestConcurrency *worker.Pool
   816  	batchWriteKVPairs int
   817  	checkpointEnabled bool
   818  
   819  	tcpConcurrency int
   820  	maxOpenFiles   int
   821  
   822  	engineMemCacheSize      int
   823  	localWriterMemCacheSize int64
   824  	supportMultiIngest      bool
   825  
   826  	duplicateDetection bool
   827  	duplicateDB        *pebble.DB
   828  }
   829  
   830  // connPool is a lazy pool of gRPC channels.
   831  // When `Get` called, it lazily allocates new connection if connection not full.
   832  // If it's full, then it will return allocated channels round-robin.
   833  type connPool struct {
   834  	mu sync.Mutex
   835  
   836  	conns   []*grpc.ClientConn
   837  	next    int
   838  	cap     int
   839  	newConn func(ctx context.Context) (*grpc.ClientConn, error)
   840  }
   841  
   842  func (p *connPool) takeConns() (conns []*grpc.ClientConn) {
   843  	p.mu.Lock()
   844  	defer p.mu.Unlock()
   845  	p.conns, conns = nil, p.conns
   846  	p.next = 0
   847  	return conns
   848  }
   849  
   850  // Close closes the conn pool.
   851  func (p *connPool) Close() {
   852  	for _, c := range p.takeConns() {
   853  		if err := c.Close(); err != nil {
   854  			log.L().Warn("failed to close clientConn", zap.String("target", c.Target()), log.ShortError(err))
   855  		}
   856  	}
   857  }
   858  
   859  // get tries to get an existing connection from the pool, or make a new one if the pool not full.
   860  func (p *connPool) get(ctx context.Context) (*grpc.ClientConn, error) {
   861  	p.mu.Lock()
   862  	defer p.mu.Unlock()
   863  	if len(p.conns) < p.cap {
   864  		c, err := p.newConn(ctx)
   865  		if err != nil {
   866  			return nil, errors.Trace(err)
   867  		}
   868  		p.conns = append(p.conns, c)
   869  		return c, nil
   870  	}
   871  
   872  	conn := p.conns[p.next]
   873  	p.next = (p.next + 1) % p.cap
   874  	return conn, nil
   875  }
   876  
   877  // newConnPool creates a new connPool by the specified conn factory function and capacity.
   878  func newConnPool(cap int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *connPool {
   879  	return &connPool{
   880  		cap:     cap,
   881  		conns:   make([]*grpc.ClientConn, 0, cap),
   882  		newConn: newConn,
   883  
   884  		mu: sync.Mutex{},
   885  	}
   886  }
   887  
   888  var bufferPool = membuf.NewPool(1024, manual.Allocator{})
   889  
   890  func openDuplicateDB(storeDir string) (*pebble.DB, error) {
   891  	dbPath := filepath.Join(storeDir, duplicateDBName)
   892  	// TODO: Optimize the opts for better write.
   893  	opts := &pebble.Options{}
   894  	return pebble.Open(dbPath, opts)
   895  }
   896  
   897  // NewLocalBackend creates new connections to tikv.
   898  func NewLocalBackend(
   899  	ctx context.Context,
   900  	tls *common.TLS,
   901  	pdAddr string,
   902  	cfg *config.TikvImporter,
   903  	enableCheckpoint bool,
   904  	g glue.Glue,
   905  	maxOpenFiles int,
   906  ) (backend.Backend, error) {
   907  	localFile := cfg.SortedKVDir
   908  	rangeConcurrency := cfg.RangeConcurrency
   909  
   910  	pdCtl, err := pdutil.NewPdController(ctx, pdAddr, tls.TLSConfig(), tls.ToPDSecurityOption())
   911  	if err != nil {
   912  		return backend.MakeBackend(nil), errors.Annotate(err, "construct pd client failed")
   913  	}
   914  	splitCli := split.NewSplitClient(pdCtl.GetPDClient(), tls.TLSConfig())
   915  
   916  	shouldCreate := true
   917  	if enableCheckpoint {
   918  		if info, err := os.Stat(localFile); err != nil {
   919  			if !os.IsNotExist(err) {
   920  				return backend.MakeBackend(nil), err
   921  			}
   922  		} else if info.IsDir() {
   923  			shouldCreate = false
   924  		}
   925  	}
   926  
   927  	if shouldCreate {
   928  		err = os.Mkdir(localFile, 0o700)
   929  		if err != nil {
   930  			return backend.MakeBackend(nil), errors.Annotate(err, "invalid sorted-kv-dir for local backend, please change the config or delete the path")
   931  		}
   932  	}
   933  
   934  	var duplicateDB *pebble.DB
   935  	if cfg.DuplicateDetection {
   936  		duplicateDB, err = openDuplicateDB(localFile)
   937  		if err != nil {
   938  			return backend.MakeBackend(nil), errors.Annotate(err, "open duplicate db failed")
   939  		}
   940  	}
   941  
   942  	regionSplitSize := int64(cfg.RegionSplitSize)
   943  	regionSplitKeys := int64(regionMaxKeyCount)
   944  	if regionSplitSize > defaultRegionSplitSize {
   945  		regionSplitKeys = int64(float64(regionSplitSize) / float64(defaultRegionSplitSize) * float64(regionMaxKeyCount))
   946  	}
   947  
   948  	local := &local{
   949  		engines:  sync.Map{},
   950  		pdCtl:    pdCtl,
   951  		splitCli: splitCli,
   952  		tls:      tls,
   953  		pdAddr:   pdAddr,
   954  		g:        g,
   955  
   956  		localStoreDir:   localFile,
   957  		regionSplitSize: regionSplitSize,
   958  		regionSplitKeys: regionSplitKeys,
   959  
   960  		rangeConcurrency:  worker.NewPool(ctx, rangeConcurrency, "range"),
   961  		ingestConcurrency: worker.NewPool(ctx, rangeConcurrency*2, "ingest"),
   962  		tcpConcurrency:    rangeConcurrency,
   963  		batchWriteKVPairs: cfg.SendKVPairs,
   964  		checkpointEnabled: enableCheckpoint,
   965  		maxOpenFiles:      utils.MaxInt(maxOpenFiles, openFilesLowerThreshold),
   966  
   967  		engineMemCacheSize:      int(cfg.EngineMemCacheSize),
   968  		localWriterMemCacheSize: int64(cfg.LocalWriterMemCacheSize),
   969  		duplicateDetection:      cfg.DuplicateDetection,
   970  		duplicateDB:             duplicateDB,
   971  	}
   972  	local.conns = common.NewGRPCConns()
   973  	if err = local.checkMultiIngestSupport(ctx, pdCtl); err != nil {
   974  		return backend.MakeBackend(nil), err
   975  	}
   976  
   977  	return backend.MakeBackend(local), nil
   978  }
   979  
   980  func (local *local) checkMultiIngestSupport(ctx context.Context, pdCtl *pdutil.PdController) error {
   981  	stores, err := conn.GetAllTiKVStores(ctx, pdCtl.GetPDClient(), conn.SkipTiFlash)
   982  	if err != nil {
   983  		return errors.Trace(err)
   984  	}
   985  	for _, s := range stores {
   986  		// only check up stores
   987  		if s.State != metapb.StoreState_Up {
   988  			continue
   989  		}
   990  		var err error
   991  		for i := 0; i < maxRetryTimes; i++ {
   992  			if i > 0 {
   993  				select {
   994  				case <-time.After(100 * time.Millisecond):
   995  				case <-ctx.Done():
   996  					return ctx.Err()
   997  				}
   998  			}
   999  			client, err1 := local.getImportClient(ctx, s.Id)
  1000  			if err1 != nil {
  1001  				err = err1
  1002  				log.L().Warn("get import client failed", zap.Error(err), zap.String("store", s.Address))
  1003  				continue
  1004  			}
  1005  			_, err = client.MultiIngest(ctx, &sst.MultiIngestRequest{})
  1006  			if err == nil {
  1007  				break
  1008  			}
  1009  			if st, ok := status.FromError(err); ok {
  1010  				if st.Code() == codes.Unimplemented {
  1011  					log.L().Info("multi ingest not support", zap.Any("unsupported store", s))
  1012  					local.supportMultiIngest = false
  1013  					return nil
  1014  				}
  1015  			}
  1016  			log.L().Warn("check multi ingest support failed", zap.Error(err), zap.String("store", s.Address),
  1017  				zap.Int("retry", i))
  1018  		}
  1019  		if err != nil {
  1020  			log.L().Warn("check multi failed all retry, fallback to false", log.ShortError(err))
  1021  			local.supportMultiIngest = false
  1022  			return nil
  1023  		}
  1024  	}
  1025  
  1026  	local.supportMultiIngest = true
  1027  	log.L().Info("multi ingest support")
  1028  	return nil
  1029  }
  1030  
  1031  // rlock read locks a local file and returns the File instance if it exists.
  1032  func (local *local) rLockEngine(engineId uuid.UUID) *File {
  1033  	if e, ok := local.engines.Load(engineId); ok {
  1034  		engine := e.(*File)
  1035  		engine.rLock()
  1036  		return engine
  1037  	}
  1038  	return nil
  1039  }
  1040  
  1041  // lock locks a local file and returns the File instance if it exists.
  1042  func (local *local) lockEngine(engineID uuid.UUID, state importMutexState) *File {
  1043  	if e, ok := local.engines.Load(engineID); ok {
  1044  		engine := e.(*File)
  1045  		engine.lock(state)
  1046  		return engine
  1047  	}
  1048  	return nil
  1049  }
  1050  
  1051  // tryRLockAllEngines tries to read lock all engines, return all `File`s that are successfully locked.
  1052  func (local *local) tryRLockAllEngines() []*File {
  1053  	var allEngines []*File
  1054  	local.engines.Range(func(k, v interface{}) bool {
  1055  		engine := v.(*File)
  1056  		// skip closed engine
  1057  		if engine.tryRLock() {
  1058  			if !engine.closed.Load() {
  1059  				allEngines = append(allEngines, engine)
  1060  			} else {
  1061  				engine.rUnlock()
  1062  			}
  1063  		}
  1064  		return true
  1065  	})
  1066  	return allEngines
  1067  }
  1068  
  1069  // lockAllEnginesUnless tries to lock all engines, unless those which are already locked in the
  1070  // state given by ignoreStateMask. Returns the list of locked engines.
  1071  func (local *local) lockAllEnginesUnless(newState, ignoreStateMask importMutexState) []*File {
  1072  	var allEngines []*File
  1073  	local.engines.Range(func(k, v interface{}) bool {
  1074  		engine := v.(*File)
  1075  		if engine.lockUnless(newState, ignoreStateMask) {
  1076  			allEngines = append(allEngines, engine)
  1077  		}
  1078  		return true
  1079  	})
  1080  	return allEngines
  1081  }
  1082  
  1083  func (local *local) makeConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) {
  1084  	store, err := local.splitCli.GetStore(ctx, storeID)
  1085  	if err != nil {
  1086  		return nil, errors.Trace(err)
  1087  	}
  1088  	opt := grpc.WithInsecure()
  1089  	if local.tls.TLSConfig() != nil {
  1090  		opt = grpc.WithTransportCredentials(credentials.NewTLS(local.tls.TLSConfig()))
  1091  	}
  1092  	ctx, cancel := context.WithTimeout(ctx, dialTimeout)
  1093  
  1094  	bfConf := backoff.DefaultConfig
  1095  	bfConf.MaxDelay = gRPCBackOffMaxDelay
  1096  	// we should use peer address for tiflash. for tikv, peer address is empty
  1097  	addr := store.GetPeerAddress()
  1098  	if addr == "" {
  1099  		addr = store.GetAddress()
  1100  	}
  1101  	conn, err := grpc.DialContext(
  1102  		ctx,
  1103  		addr,
  1104  		opt,
  1105  		grpc.WithConnectParams(grpc.ConnectParams{Backoff: bfConf}),
  1106  		grpc.WithKeepaliveParams(keepalive.ClientParameters{
  1107  			Time:                gRPCKeepAliveTime,
  1108  			Timeout:             gRPCKeepAliveTimeout,
  1109  			PermitWithoutStream: true,
  1110  		}),
  1111  	)
  1112  	cancel()
  1113  	if err != nil {
  1114  		return nil, errors.Trace(err)
  1115  	}
  1116  	return conn, nil
  1117  }
  1118  
  1119  func (local *local) getGrpcConn(ctx context.Context, storeID uint64) (*grpc.ClientConn, error) {
  1120  	return local.conns.GetGrpcConn(ctx, storeID, local.tcpConcurrency,
  1121  		func(ctx context.Context) (*grpc.ClientConn, error) {
  1122  			return local.makeConn(ctx, storeID)
  1123  		})
  1124  }
  1125  
  1126  // Close the local backend.
  1127  func (local *local) Close() {
  1128  	allEngines := local.lockAllEnginesUnless(importMutexStateClose, 0)
  1129  	local.engines = sync.Map{}
  1130  
  1131  	for _, engine := range allEngines {
  1132  		engine.Close()
  1133  		engine.unlock()
  1134  	}
  1135  	local.conns.Close()
  1136  
  1137  	if local.duplicateDB != nil {
  1138  		// Check whether there are duplicates.
  1139  		iter := local.duplicateDB.NewIter(&pebble.IterOptions{})
  1140  		hasDuplicates := iter.First()
  1141  		allIsWell := true
  1142  		if err := iter.Error(); err != nil {
  1143  			log.L().Warn("iterate duplicate db failed", zap.Error(err))
  1144  			allIsWell = false
  1145  		}
  1146  		if err := iter.Close(); err != nil {
  1147  			log.L().Warn("close duplicate db iter failed", zap.Error(err))
  1148  			allIsWell = false
  1149  		}
  1150  		if err := local.duplicateDB.Close(); err != nil {
  1151  			log.L().Warn("close duplicate db failed", zap.Error(err))
  1152  			allIsWell = false
  1153  		}
  1154  		// If checkpoint is disabled or we don't detect any duplicate, then this duplicate
  1155  		// db dir will be useless, so we clean up this dir.
  1156  		if allIsWell && (!local.checkpointEnabled || !hasDuplicates) {
  1157  			if err := os.RemoveAll(filepath.Join(local.localStoreDir, duplicateDBName)); err != nil {
  1158  				log.L().Warn("remove duplicate db file failed", zap.Error(err))
  1159  			}
  1160  		}
  1161  		local.duplicateDB = nil
  1162  	}
  1163  
  1164  	// if checkpoint is disable or we finish load all data successfully, then files in this
  1165  	// dir will be useless, so we clean up this dir and all files in it.
  1166  	if !local.checkpointEnabled || common.IsEmptyDir(local.localStoreDir) {
  1167  		err := os.RemoveAll(local.localStoreDir)
  1168  		if err != nil {
  1169  			log.L().Warn("remove local db file failed", zap.Error(err))
  1170  		}
  1171  	}
  1172  }
  1173  
  1174  // FlushEngine ensure the written data is saved successfully, to make sure no data lose after restart
  1175  func (local *local) FlushEngine(ctx context.Context, engineID uuid.UUID) error {
  1176  	engineFile := local.rLockEngine(engineID)
  1177  
  1178  	// the engine cannot be deleted after while we've acquired the lock identified by UUID.
  1179  	if engineFile == nil {
  1180  		return errors.Errorf("engine '%s' not found", engineID)
  1181  	}
  1182  	defer engineFile.rUnlock()
  1183  	if engineFile.closed.Load() {
  1184  		return nil
  1185  	}
  1186  	return engineFile.flushEngineWithoutLock(ctx)
  1187  }
  1188  
  1189  func (local *local) FlushAllEngines(parentCtx context.Context) (err error) {
  1190  	allEngines := local.tryRLockAllEngines()
  1191  	defer func() {
  1192  		for _, engine := range allEngines {
  1193  			engine.rUnlock()
  1194  		}
  1195  	}()
  1196  
  1197  	eg, ctx := errgroup.WithContext(parentCtx)
  1198  	for _, engineFile := range allEngines {
  1199  		ef := engineFile
  1200  		eg.Go(func() error {
  1201  			return ef.flushEngineWithoutLock(ctx)
  1202  		})
  1203  	}
  1204  	return eg.Wait()
  1205  }
  1206  
  1207  func (local *local) RetryImportDelay() time.Duration {
  1208  	return defaultRetryBackoffTime
  1209  }
  1210  
  1211  func (local *local) MaxChunkSize() int {
  1212  	// a batch size write to leveldb
  1213  	return int(local.regionSplitSize)
  1214  }
  1215  
  1216  func (local *local) ShouldPostProcess() bool {
  1217  	return true
  1218  }
  1219  
  1220  func (local *local) openEngineDB(engineUUID uuid.UUID, readOnly bool) (*pebble.DB, error) {
  1221  	opt := &pebble.Options{
  1222  		MemTableSize: local.engineMemCacheSize,
  1223  		// the default threshold value may cause write stall.
  1224  		MemTableStopWritesThreshold: 8,
  1225  		MaxConcurrentCompactions:    16,
  1226  		// set threshold to half of the max open files to avoid trigger compaction
  1227  		L0CompactionThreshold: math.MaxInt32,
  1228  		L0StopWritesThreshold: math.MaxInt32,
  1229  		LBaseMaxBytes:         16 * units.TiB,
  1230  		MaxOpenFiles:          local.maxOpenFiles,
  1231  		DisableWAL:            true,
  1232  		ReadOnly:              readOnly,
  1233  		TablePropertyCollectors: []func() pebble.TablePropertyCollector{
  1234  			newRangePropertiesCollector,
  1235  		},
  1236  	}
  1237  	// set level target file size to avoid pebble auto triggering compaction that split ingest SST files into small SST.
  1238  	opt.Levels = []pebble.LevelOptions{
  1239  		{
  1240  			TargetFileSize: 16 * units.GiB,
  1241  		},
  1242  	}
  1243  
  1244  	dbPath := filepath.Join(local.localStoreDir, engineUUID.String())
  1245  	db, err := pebble.Open(dbPath, opt)
  1246  	return db, errors.Trace(err)
  1247  }
  1248  
  1249  // This method must be called with holding mutex of File
  1250  func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error {
  1251  	engineCfg := backend.LocalEngineConfig{}
  1252  	if cfg.Local != nil {
  1253  		engineCfg = *cfg.Local
  1254  	}
  1255  	db, err := local.openEngineDB(engineUUID, false)
  1256  	if err != nil {
  1257  		return err
  1258  	}
  1259  
  1260  	sstDir := engineSSTDir(local.localStoreDir, engineUUID)
  1261  	if err := os.RemoveAll(sstDir); err != nil {
  1262  		return errors.Trace(err)
  1263  	}
  1264  	if !common.IsDirExists(sstDir) {
  1265  		if err := os.Mkdir(sstDir, 0o755); err != nil {
  1266  			return errors.Trace(err)
  1267  		}
  1268  	}
  1269  	engineCtx, cancel := context.WithCancel(ctx)
  1270  
  1271  	keyAdapter := KeyAdapter(noopKeyAdapter{})
  1272  	if local.duplicateDetection {
  1273  		keyAdapter = duplicateKeyAdapter{}
  1274  	}
  1275  	e, _ := local.engines.LoadOrStore(engineUUID, &File{
  1276  		UUID:               engineUUID,
  1277  		sstDir:             sstDir,
  1278  		sstMetasChan:       make(chan metaOrFlush, 64),
  1279  		ctx:                engineCtx,
  1280  		cancel:             cancel,
  1281  		config:             engineCfg,
  1282  		tableInfo:          cfg.TableInfo,
  1283  		duplicateDetection: local.duplicateDetection,
  1284  		duplicateDB:        local.duplicateDB,
  1285  		keyAdapter:         keyAdapter,
  1286  	})
  1287  	engine := e.(*File)
  1288  	engine.db = db
  1289  	engine.sstIngester = dbSSTIngester{e: engine}
  1290  	if err = engine.loadEngineMeta(); err != nil {
  1291  		return errors.Trace(err)
  1292  	}
  1293  	if err = local.allocateTSIfNotExists(ctx, engine); err != nil {
  1294  		return errors.Trace(err)
  1295  	}
  1296  	engine.wg.Add(1)
  1297  	go engine.ingestSSTLoop()
  1298  	return nil
  1299  }
  1300  
  1301  func (local *local) allocateTSIfNotExists(ctx context.Context, engine *File) error {
  1302  	if engine.TS > 0 {
  1303  		return nil
  1304  	}
  1305  	physical, logical, err := local.pdCtl.GetPDClient().GetTS(ctx)
  1306  	if err != nil {
  1307  		return err
  1308  	}
  1309  	ts := oracle.ComposeTS(physical, logical)
  1310  	engine.TS = ts
  1311  	return engine.saveEngineMeta()
  1312  }
  1313  
  1314  // CloseEngine closes backend engine by uuid
  1315  // NOTE: we will return nil if engine is not exist. This will happen if engine import&cleanup successfully
  1316  // but exit before update checkpoint. Thus after restart, we will try to import this engine again.
  1317  func (local *local) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error {
  1318  	// flush mem table to storage, to free memory,
  1319  	// ask others' advise, looks like unnecessary, but with this we can control memory precisely.
  1320  	engine, ok := local.engines.Load(engineUUID)
  1321  	if !ok {
  1322  		// recovery mode, we should reopen this engine file
  1323  		db, err := local.openEngineDB(engineUUID, true)
  1324  		if err != nil {
  1325  			// if engine db does not exist, just skip
  1326  			if os.IsNotExist(errors.Cause(err)) {
  1327  				return nil
  1328  			}
  1329  			return err
  1330  		}
  1331  		engineFile := &File{
  1332  			UUID:               engineUUID,
  1333  			db:                 db,
  1334  			sstMetasChan:       make(chan metaOrFlush),
  1335  			tableInfo:          cfg.TableInfo,
  1336  			duplicateDetection: local.duplicateDetection,
  1337  			duplicateDB:        local.duplicateDB,
  1338  		}
  1339  		engineFile.sstIngester = dbSSTIngester{e: engineFile}
  1340  		if err = engineFile.loadEngineMeta(); err != nil {
  1341  			return err
  1342  		}
  1343  		local.engines.Store(engineUUID, engineFile)
  1344  		return nil
  1345  	}
  1346  
  1347  	engineFile := engine.(*File)
  1348  	engineFile.rLock()
  1349  	if engineFile.closed.Load() {
  1350  		engineFile.rUnlock()
  1351  		return nil
  1352  	}
  1353  
  1354  	err := engineFile.flushEngineWithoutLock(ctx)
  1355  	engineFile.rUnlock()
  1356  
  1357  	// use mutex to make sure we won't close sstMetasChan while other routines
  1358  	// trying to do flush.
  1359  	engineFile.lock(importMutexStateClose)
  1360  	engineFile.closed.Store(true)
  1361  	close(engineFile.sstMetasChan)
  1362  	engineFile.unlock()
  1363  	if err != nil {
  1364  		return errors.Trace(err)
  1365  	}
  1366  	engineFile.wg.Wait()
  1367  	return engineFile.ingestErr.Get()
  1368  }
  1369  
  1370  func (local *local) getImportClient(ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) {
  1371  	conn, err := local.getGrpcConn(ctx, storeID)
  1372  	if err != nil {
  1373  		return nil, err
  1374  	}
  1375  	return sst.NewImportSSTClient(conn), nil
  1376  }
  1377  
  1378  type rangeStats struct {
  1379  	count      int64
  1380  	totalBytes int64
  1381  }
  1382  
  1383  // WriteToTiKV writer engine key-value pairs to tikv and return the sst meta generated by tikv.
  1384  // we don't need to do cleanup for the pairs written to tikv if encounters an error,
  1385  // tikv will takes the responsibility to do so.
  1386  func (local *local) WriteToTiKV(
  1387  	ctx context.Context,
  1388  	engineFile *File,
  1389  	region *split.RegionInfo,
  1390  	start, end []byte,
  1391  ) ([]*sst.SSTMeta, Range, rangeStats, error) {
  1392  	for _, peer := range region.Region.GetPeers() {
  1393  		var e error
  1394  		for i := 0; i < maxRetryTimes; i++ {
  1395  			store, err := local.pdCtl.GetStoreInfo(ctx, peer.StoreId)
  1396  			if err != nil {
  1397  				e = err
  1398  				continue
  1399  			}
  1400  			if store.Status.Capacity > 0 {
  1401  				// The available disk percent of TiKV
  1402  				ratio := store.Status.Available * 100 / store.Status.Capacity
  1403  				if ratio < 10 {
  1404  					return nil, Range{}, rangeStats{}, errors.Errorf("The available disk of TiKV (%s) only left %d, and capacity is %d",
  1405  						store.Store.Address, store.Status.Available, store.Status.Capacity)
  1406  				}
  1407  			}
  1408  			break
  1409  		}
  1410  		if e != nil {
  1411  			log.L().Error("failed to get StoreInfo from pd http api", zap.Error(e))
  1412  		}
  1413  	}
  1414  	begin := time.Now()
  1415  	regionRange := intersectRange(region.Region, Range{start: start, end: end})
  1416  	opt := &pebble.IterOptions{LowerBound: regionRange.start, UpperBound: regionRange.end}
  1417  	iter := newKeyIter(ctx, engineFile, opt)
  1418  	defer iter.Close()
  1419  
  1420  	stats := rangeStats{}
  1421  
  1422  	iter.First()
  1423  	if iter.Error() != nil {
  1424  		return nil, Range{}, stats, errors.Annotate(iter.Error(), "failed to read the first key")
  1425  	}
  1426  	if !iter.Valid() {
  1427  		log.L().Info("keys within region is empty, skip ingest", logutil.Key("start", start),
  1428  			logutil.Key("regionStart", region.Region.StartKey), logutil.Key("end", end),
  1429  			logutil.Key("regionEnd", region.Region.EndKey))
  1430  		return nil, regionRange, stats, nil
  1431  	}
  1432  	firstKey := codec.EncodeBytes([]byte{}, iter.Key())
  1433  	iter.Last()
  1434  	if iter.Error() != nil {
  1435  		return nil, Range{}, stats, errors.Annotate(iter.Error(), "failed to seek to the last key")
  1436  	}
  1437  	lastKey := codec.EncodeBytes([]byte{}, iter.Key())
  1438  
  1439  	u := uuid.New()
  1440  	meta := &sst.SSTMeta{
  1441  		Uuid:        u[:],
  1442  		RegionId:    region.Region.GetId(),
  1443  		RegionEpoch: region.Region.GetRegionEpoch(),
  1444  		Range: &sst.Range{
  1445  			Start: firstKey,
  1446  			End:   lastKey,
  1447  		},
  1448  	}
  1449  
  1450  	leaderID := region.Leader.GetId()
  1451  	clients := make([]sst.ImportSST_WriteClient, 0, len(region.Region.GetPeers()))
  1452  	requests := make([]*sst.WriteRequest, 0, len(region.Region.GetPeers()))
  1453  	for _, peer := range region.Region.GetPeers() {
  1454  		cli, err := local.getImportClient(ctx, peer.StoreId)
  1455  		if err != nil {
  1456  			return nil, Range{}, stats, err
  1457  		}
  1458  
  1459  		wstream, err := cli.Write(ctx)
  1460  		if err != nil {
  1461  			return nil, Range{}, stats, errors.Trace(err)
  1462  		}
  1463  
  1464  		// Bind uuid for this write request
  1465  		req := &sst.WriteRequest{
  1466  			Chunk: &sst.WriteRequest_Meta{
  1467  				Meta: meta,
  1468  			},
  1469  		}
  1470  		if err = wstream.Send(req); err != nil {
  1471  			return nil, Range{}, stats, errors.Trace(err)
  1472  		}
  1473  		req.Chunk = &sst.WriteRequest_Batch{
  1474  			Batch: &sst.WriteBatch{
  1475  				CommitTs: engineFile.TS,
  1476  			},
  1477  		}
  1478  		clients = append(clients, wstream)
  1479  		requests = append(requests, req)
  1480  	}
  1481  
  1482  	bytesBuf := bufferPool.NewBuffer()
  1483  	defer bytesBuf.Destroy()
  1484  	pairs := make([]*sst.Pair, 0, local.batchWriteKVPairs)
  1485  	count := 0
  1486  	size := int64(0)
  1487  	totalCount := int64(0)
  1488  	firstLoop := true
  1489  	regionMaxSize := local.regionSplitSize * 4 / 3
  1490  
  1491  	for iter.First(); iter.Valid(); iter.Next() {
  1492  		size += int64(len(iter.Key()) + len(iter.Value()))
  1493  		// here we reuse the `*sst.Pair`s to optimize object allocation
  1494  		if firstLoop {
  1495  			pair := &sst.Pair{
  1496  				Key:   bytesBuf.AddBytes(iter.Key()),
  1497  				Value: bytesBuf.AddBytes(iter.Value()),
  1498  			}
  1499  			pairs = append(pairs, pair)
  1500  		} else {
  1501  			pairs[count].Key = bytesBuf.AddBytes(iter.Key())
  1502  			pairs[count].Value = bytesBuf.AddBytes(iter.Value())
  1503  		}
  1504  		count++
  1505  		totalCount++
  1506  
  1507  		if count >= local.batchWriteKVPairs {
  1508  			for i := range clients {
  1509  				requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count]
  1510  				if err := clients[i].Send(requests[i]); err != nil {
  1511  					return nil, Range{}, stats, errors.Trace(err)
  1512  				}
  1513  			}
  1514  			count = 0
  1515  			bytesBuf.Reset()
  1516  			firstLoop = false
  1517  		}
  1518  		if size >= regionMaxSize || totalCount >= local.regionSplitKeys {
  1519  			break
  1520  		}
  1521  	}
  1522  
  1523  	if iter.Error() != nil {
  1524  		return nil, Range{}, stats, errors.Trace(iter.Error())
  1525  	}
  1526  
  1527  	if count > 0 {
  1528  		for i := range clients {
  1529  			requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count]
  1530  			if err := clients[i].Send(requests[i]); err != nil {
  1531  				return nil, Range{}, stats, errors.Trace(err)
  1532  			}
  1533  		}
  1534  	}
  1535  
  1536  	var leaderPeerMetas []*sst.SSTMeta
  1537  	for i, wStream := range clients {
  1538  		resp, closeErr := wStream.CloseAndRecv()
  1539  		if closeErr != nil {
  1540  			return nil, Range{}, stats, errors.Trace(closeErr)
  1541  		}
  1542  		if resp.Error != nil {
  1543  			return nil, Range{}, stats, errors.New(resp.Error.Message)
  1544  		}
  1545  		if leaderID == region.Region.Peers[i].GetId() {
  1546  			leaderPeerMetas = resp.Metas
  1547  			log.L().Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas))
  1548  		}
  1549  	}
  1550  
  1551  	// if there is not leader currently, we should directly return an error
  1552  	if len(leaderPeerMetas) == 0 {
  1553  		log.L().Warn("write to tikv no leader", logutil.Region(region.Region), logutil.Leader(region.Leader),
  1554  			zap.Uint64("leader_id", leaderID), logutil.SSTMeta(meta),
  1555  			zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size))
  1556  		return nil, Range{}, stats, errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d",
  1557  			region.Region.Id, leaderID)
  1558  	}
  1559  
  1560  	log.L().Debug("write to kv", zap.Reflect("region", region), zap.Uint64("leader", leaderID),
  1561  		zap.Reflect("meta", meta), zap.Reflect("return metas", leaderPeerMetas),
  1562  		zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size),
  1563  		zap.Int64("buf_size", bytesBuf.TotalSize()),
  1564  		zap.Stringer("takeTime", time.Since(begin)))
  1565  
  1566  	finishedRange := regionRange
  1567  	if iter.Valid() && iter.Next() {
  1568  		firstKey := append([]byte{}, iter.Key()...)
  1569  		finishedRange = Range{start: regionRange.start, end: firstKey}
  1570  		log.L().Info("write to tikv partial finish", zap.Int64("count", totalCount),
  1571  			zap.Int64("size", size), logutil.Key("startKey", regionRange.start), logutil.Key("endKey", regionRange.end),
  1572  			logutil.Key("remainStart", firstKey), logutil.Key("remainEnd", regionRange.end),
  1573  			logutil.Region(region.Region), logutil.Leader(region.Leader))
  1574  	}
  1575  	stats.count = totalCount
  1576  	stats.totalBytes = size
  1577  
  1578  	return leaderPeerMetas, finishedRange, stats, nil
  1579  }
  1580  
  1581  func (local *local) Ingest(ctx context.Context, metas []*sst.SSTMeta, region *split.RegionInfo) (*sst.IngestResponse, error) {
  1582  	leader := region.Leader
  1583  	if leader == nil {
  1584  		leader = region.Region.GetPeers()[0]
  1585  	}
  1586  
  1587  	cli, err := local.getImportClient(ctx, leader.StoreId)
  1588  	if err != nil {
  1589  		return nil, err
  1590  	}
  1591  	reqCtx := &kvrpcpb.Context{
  1592  		RegionId:    region.Region.GetId(),
  1593  		RegionEpoch: region.Region.GetRegionEpoch(),
  1594  		Peer:        leader,
  1595  	}
  1596  
  1597  	if !local.supportMultiIngest {
  1598  		if len(metas) != 1 {
  1599  			return nil, errors.New("batch ingest is not support")
  1600  		}
  1601  		req := &sst.IngestRequest{
  1602  			Context: reqCtx,
  1603  			Sst:     metas[0],
  1604  		}
  1605  		resp, err := cli.Ingest(ctx, req)
  1606  		return resp, errors.Trace(err)
  1607  	}
  1608  
  1609  	req := &sst.MultiIngestRequest{
  1610  		Context: reqCtx,
  1611  		Ssts:    metas,
  1612  	}
  1613  	resp, err := cli.MultiIngest(ctx, req)
  1614  	return resp, errors.Trace(err)
  1615  }
  1616  
  1617  func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit int64, keysLimit int64) []Range {
  1618  	ranges := make([]Range, 0, sizeProps.totalSize/uint64(sizeLimit))
  1619  	curSize := uint64(0)
  1620  	curKeys := uint64(0)
  1621  	curKey := fullRange.start
  1622  	sizeProps.iter(func(p *rangeProperty) bool {
  1623  		if bytes.Equal(p.Key, engineMetaKey) {
  1624  			return true
  1625  		}
  1626  		curSize += p.Size
  1627  		curKeys += p.Keys
  1628  		if int64(curSize) >= sizeLimit || int64(curKeys) >= keysLimit {
  1629  			// in case the sizeLimit or keysLimit is too small
  1630  			endKey := p.Key
  1631  			if bytes.Equal(curKey, endKey) {
  1632  				endKey = nextKey(endKey)
  1633  			}
  1634  			ranges = append(ranges, Range{start: curKey, end: endKey})
  1635  			curKey = endKey
  1636  			curSize = 0
  1637  			curKeys = 0
  1638  		}
  1639  		return true
  1640  	})
  1641  
  1642  	if curKeys > 0 {
  1643  		ranges = append(ranges, Range{start: curKey, end: fullRange.end})
  1644  	} else {
  1645  		ranges[len(ranges)-1].end = fullRange.end
  1646  	}
  1647  	return ranges
  1648  }
  1649  
  1650  func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File) ([]Range, error) {
  1651  	iter := newKeyIter(ctx, engineFile, &pebble.IterOptions{})
  1652  	defer iter.Close()
  1653  
  1654  	iterError := func(e string) error {
  1655  		err := iter.Error()
  1656  		if err != nil {
  1657  			return errors.Annotate(err, e)
  1658  		}
  1659  		return errors.New(e)
  1660  	}
  1661  
  1662  	var firstKey, lastKey []byte
  1663  	if iter.First() {
  1664  		firstKey = append([]byte{}, iter.Key()...)
  1665  	} else {
  1666  		return nil, iterError("could not find first pair")
  1667  	}
  1668  	if iter.Last() {
  1669  		lastKey = append([]byte{}, iter.Key()...)
  1670  	} else {
  1671  		return nil, iterError("could not find last pair")
  1672  	}
  1673  	endKey := nextKey(lastKey)
  1674  
  1675  	engineFileTotalSize := engineFile.TotalSize.Load()
  1676  	engineFileLength := engineFile.Length.Load()
  1677  
  1678  	// <= 96MB no need to split into range
  1679  	if engineFileTotalSize <= local.regionSplitSize && engineFileLength <= local.regionSplitKeys {
  1680  		ranges := []Range{{start: firstKey, end: endKey}}
  1681  		return ranges, nil
  1682  	}
  1683  
  1684  	sizeProps, err := engineFile.getSizeProperties()
  1685  	if err != nil {
  1686  		return nil, errors.Trace(err)
  1687  	}
  1688  
  1689  	ranges := splitRangeBySizeProps(Range{start: firstKey, end: endKey}, sizeProps,
  1690  		local.regionSplitSize, local.regionSplitKeys)
  1691  
  1692  	log.L().Info("split engine key ranges", zap.Stringer("engine", engineFile.UUID),
  1693  		zap.Int64("totalSize", engineFileTotalSize), zap.Int64("totalCount", engineFileLength),
  1694  		logutil.Key("firstKey", firstKey), logutil.Key("lastKey", lastKey),
  1695  		zap.Int("ranges", len(ranges)))
  1696  
  1697  	return ranges, nil
  1698  }
  1699  
  1700  func (local *local) writeAndIngestByRange(
  1701  	ctxt context.Context,
  1702  	engineFile *File,
  1703  	start, end []byte,
  1704  ) error {
  1705  	ito := &pebble.IterOptions{
  1706  		LowerBound: start,
  1707  		UpperBound: end,
  1708  	}
  1709  
  1710  	iter := newKeyIter(ctxt, engineFile, ito)
  1711  	defer iter.Close()
  1712  	// Needs seek to first because NewIter returns an iterator that is unpositioned
  1713  	hasKey := iter.First()
  1714  	if iter.Error() != nil {
  1715  		return errors.Annotate(iter.Error(), "failed to read the first key")
  1716  	}
  1717  	if !hasKey {
  1718  		log.L().Info("There is no pairs in iterator",
  1719  			logutil.Key("start", start),
  1720  			logutil.Key("end", end))
  1721  		engineFile.finishedRanges.add(Range{start: start, end: end})
  1722  		return nil
  1723  	}
  1724  	pairStart := append([]byte{}, iter.Key()...)
  1725  	iter.Last()
  1726  	if iter.Error() != nil {
  1727  		return errors.Annotate(iter.Error(), "failed to seek to the last key")
  1728  	}
  1729  	pairEnd := append([]byte{}, iter.Key()...)
  1730  
  1731  	var regions []*split.RegionInfo
  1732  	var err error
  1733  	ctx, cancel := context.WithCancel(ctxt)
  1734  	defer cancel()
  1735  
  1736  WriteAndIngest:
  1737  	for retry := 0; retry < maxRetryTimes; {
  1738  		if retry != 0 {
  1739  			select {
  1740  			case <-time.After(time.Second):
  1741  			case <-ctx.Done():
  1742  				return ctx.Err()
  1743  			}
  1744  		}
  1745  		startKey := codec.EncodeBytes([]byte{}, pairStart)
  1746  		endKey := codec.EncodeBytes([]byte{}, nextKey(pairEnd))
  1747  		regions, err = paginateScanRegion(ctx, local.splitCli, startKey, endKey, scanRegionLimit)
  1748  		if err != nil || len(regions) == 0 {
  1749  			log.L().Warn("scan region failed", log.ShortError(err), zap.Int("region_len", len(regions)),
  1750  				logutil.Key("startKey", startKey), logutil.Key("endKey", endKey), zap.Int("retry", retry))
  1751  			retry++
  1752  			continue WriteAndIngest
  1753  		}
  1754  
  1755  		for _, region := range regions {
  1756  			log.L().Debug("get region", zap.Int("retry", retry), zap.Binary("startKey", startKey),
  1757  				zap.Binary("endKey", endKey), zap.Uint64("id", region.Region.GetId()),
  1758  				zap.Stringer("epoch", region.Region.GetRegionEpoch()), zap.Binary("start", region.Region.GetStartKey()),
  1759  				zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers()))
  1760  
  1761  			w := local.ingestConcurrency.Apply()
  1762  			err = local.writeAndIngestPairs(ctx, engineFile, region, pairStart, end)
  1763  			local.ingestConcurrency.Recycle(w)
  1764  			if err != nil {
  1765  				if common.IsContextCanceledError(err) {
  1766  					return err
  1767  				}
  1768  				_, regionStart, _ := codec.DecodeBytes(region.Region.StartKey, []byte{})
  1769  				// if we have at least succeeded one region, retry without increasing the retry count
  1770  				if bytes.Compare(regionStart, pairStart) > 0 {
  1771  					pairStart = regionStart
  1772  				} else {
  1773  					retry++
  1774  				}
  1775  				log.L().Info("retry write and ingest kv pairs", logutil.Key("startKey", pairStart),
  1776  					logutil.Key("endKey", end), log.ShortError(err), zap.Int("retry", retry))
  1777  				continue WriteAndIngest
  1778  			}
  1779  		}
  1780  
  1781  		return err
  1782  	}
  1783  
  1784  	return err
  1785  }
  1786  
  1787  type retryType int
  1788  
  1789  const (
  1790  	retryNone retryType = iota
  1791  	retryWrite
  1792  	retryIngest
  1793  )
  1794  
  1795  func (local *local) writeAndIngestPairs(
  1796  	ctx context.Context,
  1797  	engineFile *File,
  1798  	region *split.RegionInfo,
  1799  	start, end []byte,
  1800  ) error {
  1801  	var err error
  1802  
  1803  loopWrite:
  1804  	for i := 0; i < maxRetryTimes; i++ {
  1805  		var metas []*sst.SSTMeta
  1806  		var finishedRange Range
  1807  		var rangeStats rangeStats
  1808  		metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engineFile, region, start, end)
  1809  		if err != nil {
  1810  			if common.IsContextCanceledError(err) {
  1811  				return err
  1812  			}
  1813  
  1814  			log.L().Warn("write to tikv failed", log.ShortError(err), zap.Int("retry", i))
  1815  			continue loopWrite
  1816  		}
  1817  
  1818  		if len(metas) == 0 {
  1819  			return nil
  1820  		}
  1821  
  1822  		batch := 1
  1823  		if local.supportMultiIngest {
  1824  			batch = len(metas)
  1825  		}
  1826  
  1827  		for i := 0; i < len(metas); i += batch {
  1828  			start := i * batch
  1829  			end := utils.MinInt((i+1)*batch, len(metas))
  1830  			ingestMetas := metas[start:end]
  1831  			errCnt := 0
  1832  			for errCnt < maxRetryTimes {
  1833  				log.L().Debug("ingest meta", zap.Reflect("meta", ingestMetas))
  1834  				var resp *sst.IngestResponse
  1835  				failpoint.Inject("FailIngestMeta", func(val failpoint.Value) {
  1836  					// only inject the error once
  1837  					switch val.(string) {
  1838  					case "notleader":
  1839  						resp = &sst.IngestResponse{
  1840  							Error: &errorpb.Error{
  1841  								NotLeader: &errorpb.NotLeader{
  1842  									RegionId: region.Region.Id,
  1843  									Leader:   region.Leader,
  1844  								},
  1845  							},
  1846  						}
  1847  					case "epochnotmatch":
  1848  						resp = &sst.IngestResponse{
  1849  							Error: &errorpb.Error{
  1850  								EpochNotMatch: &errorpb.EpochNotMatch{
  1851  									CurrentRegions: []*metapb.Region{region.Region},
  1852  								},
  1853  							},
  1854  						}
  1855  					}
  1856  					if resp != nil {
  1857  						err = nil
  1858  					}
  1859  				})
  1860  				if resp == nil {
  1861  					resp, err = local.Ingest(ctx, ingestMetas, region)
  1862  				}
  1863  				if err != nil {
  1864  					if common.IsContextCanceledError(err) {
  1865  						return err
  1866  					}
  1867  					log.L().Warn("ingest failed", log.ShortError(err), logutil.SSTMetas(ingestMetas),
  1868  						logutil.Region(region.Region), logutil.Leader(region.Leader))
  1869  					errCnt++
  1870  					continue
  1871  				}
  1872  
  1873  				var retryTy retryType
  1874  				var newRegion *split.RegionInfo
  1875  				retryTy, newRegion, err = local.isIngestRetryable(ctx, resp, region, ingestMetas)
  1876  				if common.IsContextCanceledError(err) {
  1877  					return err
  1878  				}
  1879  				if err == nil {
  1880  					// ingest next meta
  1881  					break
  1882  				}
  1883  				switch retryTy {
  1884  				case retryNone:
  1885  					log.L().Warn("ingest failed noretry", log.ShortError(err), logutil.SSTMetas(ingestMetas),
  1886  						logutil.Region(region.Region), logutil.Leader(region.Leader))
  1887  					// met non-retryable error retry whole Write procedure
  1888  					return err
  1889  				case retryWrite:
  1890  					region = newRegion
  1891  					continue loopWrite
  1892  				case retryIngest:
  1893  					region = newRegion
  1894  					continue
  1895  				}
  1896  			}
  1897  		}
  1898  
  1899  		if err != nil {
  1900  			log.L().Warn("write and ingest region, will retry import full range", log.ShortError(err),
  1901  				logutil.Region(region.Region), logutil.Key("start", start),
  1902  				logutil.Key("end", end))
  1903  		} else {
  1904  			engineFile.importedKVSize.Add(rangeStats.totalBytes)
  1905  			engineFile.importedKVCount.Add(rangeStats.count)
  1906  			engineFile.finishedRanges.add(finishedRange)
  1907  			metric.BytesCounter.WithLabelValues(metric.TableStateImported).Add(float64(rangeStats.totalBytes))
  1908  		}
  1909  		return errors.Trace(err)
  1910  	}
  1911  
  1912  	return errors.Trace(err)
  1913  }
  1914  
  1915  func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File, ranges []Range) error {
  1916  	if engineFile.Length.Load() == 0 {
  1917  		// engine is empty, this is likes because it's a index engine but the table contains no index
  1918  		log.L().Info("engine contains no data", zap.Stringer("uuid", engineFile.UUID))
  1919  		return nil
  1920  	}
  1921  	log.L().Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges)))
  1922  
  1923  	var allErrLock sync.Mutex
  1924  	var allErr error
  1925  	var wg sync.WaitGroup
  1926  	metErr := atomic.NewBool(false)
  1927  
  1928  	for _, r := range ranges {
  1929  		startKey := r.start
  1930  		endKey := r.end
  1931  		w := local.rangeConcurrency.Apply()
  1932  		// if meet error here, skip try more here to allow fail fast.
  1933  		if metErr.Load() {
  1934  			local.rangeConcurrency.Recycle(w)
  1935  			break
  1936  		}
  1937  		wg.Add(1)
  1938  		go func(w *worker.Worker) {
  1939  			defer func() {
  1940  				local.rangeConcurrency.Recycle(w)
  1941  				wg.Done()
  1942  			}()
  1943  			var err error
  1944  			// max retry backoff time: 2+4+8+16=30s
  1945  			backOffTime := time.Second
  1946  			for i := 0; i < maxRetryTimes; i++ {
  1947  				err = local.writeAndIngestByRange(ctx, engineFile, startKey, endKey)
  1948  				if err == nil || common.IsContextCanceledError(err) {
  1949  					return
  1950  				}
  1951  				log.L().Warn("write and ingest by range failed",
  1952  					zap.Int("retry time", i+1), log.ShortError(err))
  1953  				backOffTime *= 2
  1954  				select {
  1955  				case <-time.After(backOffTime):
  1956  				case <-ctx.Done():
  1957  					return
  1958  				}
  1959  			}
  1960  
  1961  			allErrLock.Lock()
  1962  			allErr = multierr.Append(allErr, err)
  1963  			allErrLock.Unlock()
  1964  			if err != nil {
  1965  				metErr.Store(true)
  1966  			}
  1967  		}(w)
  1968  	}
  1969  
  1970  	// wait for all sub tasks finish to avoid panic. if we return on the first error,
  1971  	// the outer tasks may close the pebble db but some sub tasks still read from the db
  1972  	wg.Wait()
  1973  	return allErr
  1974  }
  1975  
  1976  type syncedRanges struct {
  1977  	sync.Mutex
  1978  	ranges []Range
  1979  }
  1980  
  1981  func (r *syncedRanges) add(g Range) {
  1982  	r.Lock()
  1983  	r.ranges = append(r.ranges, g)
  1984  	r.Unlock()
  1985  }
  1986  
  1987  func (r *syncedRanges) reset() {
  1988  	r.Lock()
  1989  	r.ranges = r.ranges[:0]
  1990  	r.Unlock()
  1991  }
  1992  
  1993  func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error {
  1994  	lf := local.lockEngine(engineUUID, importMutexStateImport)
  1995  	if lf == nil {
  1996  		// skip if engine not exist. See the comment of `CloseEngine` for more detail.
  1997  		return nil
  1998  	}
  1999  	defer lf.unlock()
  2000  
  2001  	lfTotalSize := lf.TotalSize.Load()
  2002  	lfLength := lf.Length.Load()
  2003  	if lfTotalSize == 0 {
  2004  		log.L().Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID))
  2005  		return nil
  2006  	}
  2007  
  2008  	// split sorted file into range by 96MB size per file
  2009  	ranges, err := local.readAndSplitIntoRange(ctx, lf)
  2010  	if err != nil {
  2011  		return err
  2012  	}
  2013  
  2014  	log.L().Info("start import engine", zap.Stringer("uuid", engineUUID),
  2015  		zap.Int("ranges", len(ranges)), zap.Int64("count", lfLength), zap.Int64("size", lfTotalSize))
  2016  	for {
  2017  		unfinishedRanges := lf.unfinishedRanges(ranges)
  2018  		if len(unfinishedRanges) == 0 {
  2019  			break
  2020  		}
  2021  		log.L().Info("import engine unfinished ranges", zap.Int("count", len(unfinishedRanges)))
  2022  
  2023  		// if all the kv can fit in one region, skip split regions. TiDB will split one region for
  2024  		// the table when table is created.
  2025  		needSplit := len(unfinishedRanges) > 1 || lfTotalSize > local.regionSplitSize || lfLength > local.regionSplitKeys
  2026  		// split region by given ranges
  2027  		for i := 0; i < maxRetryTimes; i++ {
  2028  			err = local.SplitAndScatterRegionByRanges(ctx, unfinishedRanges, lf.tableInfo, needSplit)
  2029  			if err == nil || common.IsContextCanceledError(err) {
  2030  				break
  2031  			}
  2032  
  2033  			log.L().Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID),
  2034  				log.ShortError(err), zap.Int("retry", i))
  2035  		}
  2036  		if err != nil {
  2037  			log.L().Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err))
  2038  			return err
  2039  		}
  2040  
  2041  		// start to write to kv and ingest
  2042  		err = local.writeAndIngestByRanges(ctx, lf, unfinishedRanges)
  2043  		if err != nil {
  2044  			log.L().Error("write and ingest engine failed", log.ShortError(err))
  2045  			return err
  2046  		}
  2047  	}
  2048  
  2049  	if lf.Duplicates.Load() > 0 {
  2050  		if err := lf.saveEngineMeta(); err != nil {
  2051  			log.L().Error("failed to save engine meta", log.ShortError(err))
  2052  			return err
  2053  		}
  2054  		log.L().Warn("duplicate detected during import engine", zap.Stringer("uuid", engineUUID),
  2055  			zap.Int64("size", lfTotalSize), zap.Int64("kvs", lfLength), zap.Int64("duplicate-kvs", lf.Duplicates.Load()),
  2056  			zap.Int64("importedSize", lf.importedKVSize.Load()), zap.Int64("importedCount", lf.importedKVCount.Load()))
  2057  	}
  2058  
  2059  	log.L().Info("import engine success", zap.Stringer("uuid", engineUUID),
  2060  		zap.Int64("size", lfTotalSize), zap.Int64("kvs", lfLength),
  2061  		zap.Int64("importedSize", lf.importedKVSize.Load()), zap.Int64("importedCount", lf.importedKVCount.Load()))
  2062  	return nil
  2063  }
  2064  
  2065  func (local *local) CollectLocalDuplicateRows(ctx context.Context, tbl table.Table) error {
  2066  	if local.duplicateDB == nil {
  2067  		return nil
  2068  	}
  2069  	log.L().Info("Begin collect duplicate local keys", zap.String("table", tbl.Meta().Name.String()))
  2070  	physicalTS, logicalTS, err := local.pdCtl.GetPDClient().GetTS(ctx)
  2071  	if err != nil {
  2072  		return err
  2073  	}
  2074  	ts := oracle.ComposeTS(physicalTS, logicalTS)
  2075  	// TODO: Here we use this db to store the duplicate rows. We shall remove this parameter and store the result in
  2076  	//  a TiDB table.
  2077  	duplicateManager, err := NewDuplicateManager(local.duplicateDB, local.splitCli, ts, local.tls, local.tcpConcurrency)
  2078  	if err != nil {
  2079  		return errors.Annotate(err, "open duplicatemanager failed")
  2080  	}
  2081  	if err := duplicateManager.CollectDuplicateRowsFromLocalIndex(ctx, tbl, local.duplicateDB); err != nil {
  2082  		return errors.Annotate(err, "collect local duplicate rows failed")
  2083  	}
  2084  	return local.reportDuplicateRows(tbl, local.duplicateDB)
  2085  }
  2086  
  2087  func (local *local) CollectRemoteDuplicateRows(ctx context.Context, tbl table.Table) error {
  2088  	log.L().Info("Begin collect remote duplicate keys", zap.String("table", tbl.Meta().Name.String()))
  2089  	physicalTS, logicalTS, err := local.pdCtl.GetPDClient().GetTS(ctx)
  2090  	if err != nil {
  2091  		return err
  2092  	}
  2093  	ts := oracle.ComposeTS(physicalTS, logicalTS)
  2094  	dbPath := filepath.Join(local.localStoreDir, remoteDuplicateDBName)
  2095  	// TODO: Optimize the opts for better write.
  2096  	opts := &pebble.Options{}
  2097  	duplicateDB, err := pebble.Open(dbPath, opts)
  2098  	if err != nil {
  2099  		return errors.Annotate(err, "open duplicate db failed")
  2100  	}
  2101  
  2102  	// TODO: Here we use the temp created db to store the duplicate rows. We shall remove this parameter and store the
  2103  	//  result in a TiDB table.
  2104  	duplicateManager, err := NewDuplicateManager(duplicateDB, local.splitCli, ts, local.tls, local.tcpConcurrency)
  2105  	if err != nil {
  2106  		return errors.Annotate(err, "open duplicatemanager failed")
  2107  	}
  2108  	if err = duplicateManager.CollectDuplicateRowsFromTiKV(ctx, tbl); err != nil {
  2109  		return errors.Annotate(err, "collect remote duplicate rows failed")
  2110  	}
  2111  	err = local.reportDuplicateRows(tbl, duplicateDB)
  2112  	duplicateDB.Close()
  2113  	return err
  2114  }
  2115  
  2116  func (local *local) reportDuplicateRows(tbl table.Table, db *pebble.DB) error {
  2117  	log.L().Info("Begin report duplicate rows", zap.String("table", tbl.Meta().Name.String()))
  2118  	decoder, err := kv.NewTableKVDecoder(tbl, &kv.SessionOptions{
  2119  		SQLMode: mysql.ModeStrictAllTables,
  2120  	})
  2121  	if err != nil {
  2122  		return errors.Annotate(err, "create decoder failed")
  2123  	}
  2124  
  2125  	ranges := ranger.FullIntRange(false)
  2126  	keysRanges := distsql.TableRangesToKVRanges(tbl.Meta().ID, ranges, nil)
  2127  	keyAdapter := duplicateKeyAdapter{}
  2128  	var nextUserKey []byte = nil
  2129  	for _, r := range keysRanges {
  2130  		startKey := codec.EncodeBytes([]byte{}, r.StartKey)
  2131  		endKey := codec.EncodeBytes([]byte{}, r.EndKey)
  2132  		opts := &pebble.IterOptions{
  2133  			LowerBound: startKey,
  2134  			UpperBound: endKey,
  2135  		}
  2136  		iter := db.NewIter(opts)
  2137  		for iter.SeekGE(startKey); iter.Valid(); iter.Next() {
  2138  			nextUserKey, _, _, err = keyAdapter.Decode(nextUserKey[:0], iter.Key())
  2139  			if err != nil {
  2140  				log.L().Error("decode key error from index for duplicatedb",
  2141  					zap.Error(err), logutil.Key("key", iter.Key()))
  2142  				continue
  2143  			}
  2144  
  2145  			h, err := decoder.DecodeHandleFromTable(nextUserKey)
  2146  			if err != nil {
  2147  				log.L().Error("decode handle error from index for duplicatedb",
  2148  					zap.Error(err), logutil.Key("key", iter.Key()))
  2149  				continue
  2150  			}
  2151  			rows, _, err := decoder.DecodeRawRowData(h, iter.Value())
  2152  			if err != nil {
  2153  				log.L().Error("decode row error from index for duplicatedb",
  2154  					zap.Error(err), logutil.Key("key", iter.Key()))
  2155  				continue
  2156  			}
  2157  			// TODO: We need to output the duplicate rows into files or database.
  2158  			//  Here I just output them for debug.
  2159  			r := "row "
  2160  			for _, row := range rows {
  2161  				r += "," + row.String()
  2162  			}
  2163  			log.L().Info(r)
  2164  		}
  2165  		iter.Close()
  2166  	}
  2167  	return nil
  2168  }
  2169  
  2170  func (e *File) unfinishedRanges(ranges []Range) []Range {
  2171  	e.finishedRanges.Lock()
  2172  	defer e.finishedRanges.Unlock()
  2173  
  2174  	e.finishedRanges.ranges = sortAndMergeRanges(e.finishedRanges.ranges)
  2175  
  2176  	return filterOverlapRange(ranges, e.finishedRanges.ranges)
  2177  }
  2178  
  2179  // sortAndMergeRanges sort the ranges and merge range that overlaps with each other into a single range.
  2180  func sortAndMergeRanges(ranges []Range) []Range {
  2181  	if len(ranges) == 0 {
  2182  		return ranges
  2183  	}
  2184  
  2185  	sort.Slice(ranges, func(i, j int) bool {
  2186  		return bytes.Compare(ranges[i].start, ranges[j].start) < 0
  2187  	})
  2188  
  2189  	curEnd := ranges[0].end
  2190  	i := 0
  2191  	for j := 1; j < len(ranges); j++ {
  2192  		if bytes.Compare(curEnd, ranges[j].start) >= 0 {
  2193  			if bytes.Compare(curEnd, ranges[j].end) < 0 {
  2194  				curEnd = ranges[j].end
  2195  			}
  2196  		} else {
  2197  			ranges[i].end = curEnd
  2198  			i++
  2199  			ranges[i].start = ranges[j].start
  2200  			curEnd = ranges[j].end
  2201  		}
  2202  	}
  2203  	ranges[i].end = curEnd
  2204  	return ranges[:i+1]
  2205  }
  2206  
  2207  func filterOverlapRange(ranges []Range, finishedRanges []Range) []Range {
  2208  	if len(ranges) == 0 || len(finishedRanges) == 0 {
  2209  		return ranges
  2210  	}
  2211  
  2212  	result := make([]Range, 0)
  2213  	for _, r := range ranges {
  2214  		start := r.start
  2215  		end := r.end
  2216  		for len(finishedRanges) > 0 && bytes.Compare(finishedRanges[0].start, end) < 0 {
  2217  			fr := finishedRanges[0]
  2218  			if bytes.Compare(fr.start, start) > 0 {
  2219  				result = append(result, Range{start: start, end: fr.start})
  2220  			}
  2221  			if bytes.Compare(fr.end, start) > 0 {
  2222  				start = fr.end
  2223  			}
  2224  			if bytes.Compare(fr.end, end) > 0 {
  2225  				break
  2226  			}
  2227  			finishedRanges = finishedRanges[1:]
  2228  		}
  2229  		if bytes.Compare(start, end) < 0 {
  2230  			result = append(result, Range{start: start, end: end})
  2231  		}
  2232  	}
  2233  	return result
  2234  }
  2235  
  2236  func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error {
  2237  	// the only way to reset the engine + reclaim the space is to delete and reopen it 🤷
  2238  	localEngine := local.lockEngine(engineUUID, importMutexStateClose)
  2239  	if localEngine == nil {
  2240  		log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID))
  2241  		return nil
  2242  	}
  2243  	defer localEngine.unlock()
  2244  	if err := localEngine.Close(); err != nil {
  2245  		return err
  2246  	}
  2247  	if err := localEngine.Cleanup(local.localStoreDir); err != nil {
  2248  		return err
  2249  	}
  2250  	db, err := local.openEngineDB(engineUUID, false)
  2251  	if err == nil {
  2252  		// Reset localFileMeta except `Duplicates`.
  2253  		meta := localFileMeta{
  2254  			Duplicates: *atomic.NewInt64(localEngine.localFileMeta.Duplicates.Load()),
  2255  		}
  2256  		if err := saveEngineMetaToDB(&meta, db); err != nil {
  2257  			return errors.Trace(err)
  2258  		}
  2259  		localEngine.db = db
  2260  		localEngine.localFileMeta = meta
  2261  		if !common.IsDirExists(localEngine.sstDir) {
  2262  			if err := os.Mkdir(localEngine.sstDir, 0o755); err != nil {
  2263  				return errors.Trace(err)
  2264  			}
  2265  		}
  2266  		if err = local.allocateTSIfNotExists(ctx, localEngine); err != nil {
  2267  			return errors.Trace(err)
  2268  		}
  2269  	}
  2270  	localEngine.pendingFileSize.Store(0)
  2271  	localEngine.finishedRanges.reset()
  2272  
  2273  	return err
  2274  }
  2275  
  2276  func (local *local) CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error {
  2277  	localEngine := local.lockEngine(engineUUID, importMutexStateClose)
  2278  	// release this engine after import success
  2279  	if localEngine == nil {
  2280  		log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID))
  2281  		return nil
  2282  	}
  2283  	defer localEngine.unlock()
  2284  
  2285  	// since closing the engine causes all subsequent operations on it panic,
  2286  	// we make sure to delete it from the engine map before calling Close().
  2287  	// (note that Close() returning error does _not_ mean the pebble DB
  2288  	// remains open/usable.)
  2289  	local.engines.Delete(engineUUID)
  2290  	err := localEngine.Close()
  2291  	if err != nil {
  2292  		return err
  2293  	}
  2294  	err = localEngine.Cleanup(local.localStoreDir)
  2295  	if err != nil {
  2296  		return err
  2297  	}
  2298  	localEngine.TotalSize.Store(0)
  2299  	localEngine.Length.Store(0)
  2300  	return nil
  2301  }
  2302  
  2303  func (local *local) CheckRequirements(ctx context.Context, checkCtx *backend.CheckCtx) error {
  2304  	versionStr, err := local.g.GetSQLExecutor().ObtainStringWithLog(
  2305  		ctx,
  2306  		"SELECT version();",
  2307  		"check TiDB version",
  2308  		log.L())
  2309  	if err != nil {
  2310  		return errors.Trace(err)
  2311  	}
  2312  	if err := checkTiDBVersion(ctx, versionStr, localMinTiDBVersion, localMaxTiDBVersion); err != nil {
  2313  		return err
  2314  	}
  2315  	if err := tikv.CheckPDVersion(ctx, local.tls, local.pdAddr, localMinPDVersion, localMaxPDVersion); err != nil {
  2316  		return err
  2317  	}
  2318  	if err := tikv.CheckTiKVVersion(ctx, local.tls, local.pdAddr, localMinTiKVVersion, localMaxTiKVVersion); err != nil {
  2319  		return err
  2320  	}
  2321  
  2322  	tidbVersion, _ := version.ExtractTiDBVersion(versionStr)
  2323  	return checkTiFlashVersion(ctx, local.g, checkCtx, *tidbVersion)
  2324  }
  2325  
  2326  func checkTiDBVersion(_ context.Context, versionStr string, requiredMinVersion, requiredMaxVersion semver.Version) error {
  2327  	return version.CheckTiDBVersion(versionStr, requiredMinVersion, requiredMaxVersion)
  2328  }
  2329  
  2330  var tiFlashReplicaQuery = "SELECT TABLE_SCHEMA, TABLE_NAME FROM information_schema.TIFLASH_REPLICA WHERE REPLICA_COUNT > 0;"
  2331  
  2332  type tblName struct {
  2333  	schema string
  2334  	name   string
  2335  }
  2336  
  2337  type tblNames []tblName
  2338  
  2339  func (t tblNames) String() string {
  2340  	var b strings.Builder
  2341  	b.WriteByte('[')
  2342  	for i, n := range t {
  2343  		if i > 0 {
  2344  			b.WriteString(", ")
  2345  		}
  2346  		b.WriteString(common.UniqueTable(n.schema, n.name))
  2347  	}
  2348  	b.WriteByte(']')
  2349  	return b.String()
  2350  }
  2351  
  2352  // check TiFlash replicas.
  2353  // local backend doesn't support TiFlash before tidb v4.0.5
  2354  func checkTiFlashVersion(ctx context.Context, g glue.Glue, checkCtx *backend.CheckCtx, tidbVersion semver.Version) error {
  2355  	if tidbVersion.Compare(tiFlashMinVersion) >= 0 {
  2356  		return nil
  2357  	}
  2358  
  2359  	res, err := g.GetSQLExecutor().QueryStringsWithLog(ctx, tiFlashReplicaQuery, "fetch tiflash replica info", log.L())
  2360  	if err != nil {
  2361  		return errors.Annotate(err, "fetch tiflash replica info failed")
  2362  	}
  2363  
  2364  	tiFlashTablesMap := make(map[tblName]struct{}, len(res))
  2365  	for _, tblInfo := range res {
  2366  		name := tblName{schema: tblInfo[0], name: tblInfo[1]}
  2367  		tiFlashTablesMap[name] = struct{}{}
  2368  	}
  2369  
  2370  	tiFlashTables := make(tblNames, 0)
  2371  	for _, dbMeta := range checkCtx.DBMetas {
  2372  		for _, tblMeta := range dbMeta.Tables {
  2373  			if len(tblMeta.DataFiles) == 0 {
  2374  				continue
  2375  			}
  2376  			name := tblName{schema: tblMeta.DB, name: tblMeta.Name}
  2377  			if _, ok := tiFlashTablesMap[name]; ok {
  2378  				tiFlashTables = append(tiFlashTables, name)
  2379  			}
  2380  		}
  2381  	}
  2382  
  2383  	if len(tiFlashTables) > 0 {
  2384  		helpInfo := "Please either upgrade TiDB to version >= 4.0.5 or add TiFlash replica after load data."
  2385  		return errors.Errorf("lightning local backend doesn't support TiFlash in this TiDB version. conflict tables: %s. "+helpInfo, tiFlashTables)
  2386  	}
  2387  	return nil
  2388  }
  2389  
  2390  func (local *local) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) {
  2391  	return tikv.FetchRemoteTableModelsFromTLS(ctx, local.tls, schemaName)
  2392  }
  2393  
  2394  func (local *local) MakeEmptyRows() kv.Rows {
  2395  	return kv.MakeRowsFromKvPairs(nil)
  2396  }
  2397  
  2398  func (local *local) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) {
  2399  	return kv.NewTableKVEncoder(tbl, options)
  2400  }
  2401  
  2402  func engineSSTDir(storeDir string, engineUUID uuid.UUID) string {
  2403  	return filepath.Join(storeDir, engineUUID.String()+".sst")
  2404  }
  2405  
  2406  func (local *local) LocalWriter(ctx context.Context, cfg *backend.LocalWriterConfig, engineUUID uuid.UUID) (backend.EngineWriter, error) {
  2407  	e, ok := local.engines.Load(engineUUID)
  2408  	if !ok {
  2409  		return nil, errors.Errorf("could not find engine for %s", engineUUID.String())
  2410  	}
  2411  	engineFile := e.(*File)
  2412  	return openLocalWriter(ctx, cfg, engineFile, local.localWriterMemCacheSize)
  2413  }
  2414  
  2415  func openLocalWriter(ctx context.Context, cfg *backend.LocalWriterConfig, f *File, cacheSize int64) (*Writer, error) {
  2416  	w := &Writer{
  2417  		local:              f,
  2418  		memtableSizeLimit:  cacheSize,
  2419  		kvBuffer:           bufferPool.NewBuffer(),
  2420  		isKVSorted:         cfg.IsKVSorted,
  2421  		isWriteBatchSorted: true,
  2422  	}
  2423  	// pre-allocate a long enough buffer to avoid a lot of runtime.growslice
  2424  	// this can help save about 3% of CPU.
  2425  	if !w.isKVSorted {
  2426  		w.writeBatch = make([]common.KvPair, units.MiB)
  2427  	}
  2428  	f.localWriters.Store(w, nil)
  2429  	return w, nil
  2430  }
  2431  
  2432  func (local *local) isIngestRetryable(
  2433  	ctx context.Context,
  2434  	resp *sst.IngestResponse,
  2435  	region *split.RegionInfo,
  2436  	metas []*sst.SSTMeta,
  2437  ) (retryType, *split.RegionInfo, error) {
  2438  	if resp.GetError() == nil {
  2439  		return retryNone, nil, nil
  2440  	}
  2441  
  2442  	getRegion := func() (*split.RegionInfo, error) {
  2443  		for i := 0; ; i++ {
  2444  			newRegion, err := local.splitCli.GetRegion(ctx, region.Region.GetStartKey())
  2445  			if err != nil {
  2446  				return nil, errors.Trace(err)
  2447  			}
  2448  			if newRegion != nil {
  2449  				return newRegion, nil
  2450  			}
  2451  			log.L().Warn("get region by key return nil, will retry", logutil.Region(region.Region), logutil.Leader(region.Leader),
  2452  				zap.Int("retry", i))
  2453  			select {
  2454  			case <-ctx.Done():
  2455  				return nil, ctx.Err()
  2456  			case <-time.After(time.Second):
  2457  			}
  2458  		}
  2459  	}
  2460  
  2461  	var newRegion *split.RegionInfo
  2462  	var err error
  2463  	switch errPb := resp.GetError(); {
  2464  	case errPb.NotLeader != nil:
  2465  		if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil {
  2466  			newRegion = &split.RegionInfo{
  2467  				Leader: newLeader,
  2468  				Region: region.Region,
  2469  			}
  2470  		} else {
  2471  			newRegion, err = getRegion()
  2472  			if err != nil {
  2473  				return retryNone, nil, errors.Trace(err)
  2474  			}
  2475  		}
  2476  		// TODO: because in some case, TiKV may return retryable error while the ingest is succeeded.
  2477  		// Thus directly retry ingest may cause TiKV panic. So always return retryWrite here to avoid
  2478  		// this issue.
  2479  		// See: https://github.com/tikv/tikv/issues/9496
  2480  		return retryWrite, newRegion, errors.Errorf("not leader: %s", errPb.GetMessage())
  2481  	case errPb.EpochNotMatch != nil:
  2482  		if currentRegions := errPb.GetEpochNotMatch().GetCurrentRegions(); currentRegions != nil {
  2483  			var currentRegion *metapb.Region
  2484  			for _, r := range currentRegions {
  2485  				if insideRegion(r, metas) {
  2486  					currentRegion = r
  2487  					break
  2488  				}
  2489  			}
  2490  			if currentRegion != nil {
  2491  				var newLeader *metapb.Peer
  2492  				for _, p := range currentRegion.Peers {
  2493  					if p.GetStoreId() == region.Leader.GetStoreId() {
  2494  						newLeader = p
  2495  						break
  2496  					}
  2497  				}
  2498  				if newLeader != nil {
  2499  					newRegion = &split.RegionInfo{
  2500  						Leader: newLeader,
  2501  						Region: currentRegion,
  2502  					}
  2503  				}
  2504  			}
  2505  		}
  2506  		retryTy := retryNone
  2507  		if newRegion != nil {
  2508  			retryTy = retryWrite
  2509  		}
  2510  		return retryTy, newRegion, errors.Errorf("epoch not match: %s", errPb.GetMessage())
  2511  	case strings.Contains(errPb.Message, "raft: proposal dropped"):
  2512  		// TODO: we should change 'Raft raft: proposal dropped' to a error type like 'NotLeader'
  2513  		newRegion, err = getRegion()
  2514  		if err != nil {
  2515  			return retryNone, nil, errors.Trace(err)
  2516  		}
  2517  		return retryWrite, newRegion, errors.New(errPb.GetMessage())
  2518  	}
  2519  	return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage())
  2520  }
  2521  
  2522  // return the smallest []byte that is bigger than current bytes.
  2523  // special case when key is empty, empty bytes means infinity in our context, so directly return itself.
  2524  func nextKey(key []byte) []byte {
  2525  	if len(key) == 0 {
  2526  		return []byte{}
  2527  	}
  2528  
  2529  	// in tikv <= 4.x, tikv will truncate the row key, so we should fetch the next valid row key
  2530  	// See: https://github.com/tikv/tikv/blob/f7f22f70e1585d7ca38a59ea30e774949160c3e8/components/raftstore/src/coprocessor/split_observer.rs#L36-L41
  2531  	if tablecodec.IsRecordKey(key) {
  2532  		tableID, handle, _ := tablecodec.DecodeRecordKey(key)
  2533  		nextHandle := handle.Next()
  2534  		// int handle overflow, use the next table prefix as nextKey
  2535  		if nextHandle.Compare(handle) <= 0 {
  2536  			return tablecodec.EncodeTablePrefix(tableID + 1)
  2537  		}
  2538  		return tablecodec.EncodeRowKeyWithHandle(tableID, nextHandle)
  2539  	}
  2540  
  2541  	// if key is an index, directly append a 0x00 to the key.
  2542  	res := make([]byte, 0, len(key)+1)
  2543  	res = append(res, key...)
  2544  	res = append(res, 0)
  2545  	return res
  2546  }
  2547  
  2548  type rangeOffsets struct {
  2549  	Size uint64
  2550  	Keys uint64
  2551  }
  2552  
  2553  type rangeProperty struct {
  2554  	Key []byte
  2555  	rangeOffsets
  2556  }
  2557  
  2558  func (r *rangeProperty) Less(than btree.Item) bool {
  2559  	ta := than.(*rangeProperty)
  2560  	return bytes.Compare(r.Key, ta.Key) < 0
  2561  }
  2562  
  2563  var _ btree.Item = &rangeProperty{}
  2564  
  2565  type rangeProperties []rangeProperty
  2566  
  2567  func decodeRangeProperties(data []byte) (rangeProperties, error) {
  2568  	r := make(rangeProperties, 0, 16)
  2569  	for len(data) > 0 {
  2570  		if len(data) < 4 {
  2571  			return nil, io.ErrUnexpectedEOF
  2572  		}
  2573  		keyLen := int(binary.BigEndian.Uint32(data[:4]))
  2574  		data = data[4:]
  2575  		if len(data) < keyLen+8*2 {
  2576  			return nil, io.ErrUnexpectedEOF
  2577  		}
  2578  		key := data[:keyLen]
  2579  		data = data[keyLen:]
  2580  		size := binary.BigEndian.Uint64(data[:8])
  2581  		keys := binary.BigEndian.Uint64(data[8:])
  2582  		data = data[16:]
  2583  		r = append(r, rangeProperty{Key: key, rangeOffsets: rangeOffsets{Size: size, Keys: keys}})
  2584  	}
  2585  
  2586  	return r, nil
  2587  }
  2588  
  2589  func (r rangeProperties) Encode() []byte {
  2590  	b := make([]byte, 0, 1024)
  2591  	idx := 0
  2592  	for _, p := range r {
  2593  		b = append(b, 0, 0, 0, 0)
  2594  		binary.BigEndian.PutUint32(b[idx:], uint32(len(p.Key)))
  2595  		idx += 4
  2596  		b = append(b, p.Key...)
  2597  		idx += len(p.Key)
  2598  
  2599  		b = append(b, 0, 0, 0, 0, 0, 0, 0, 0)
  2600  		binary.BigEndian.PutUint64(b[idx:], p.Size)
  2601  		idx += 8
  2602  
  2603  		b = append(b, 0, 0, 0, 0, 0, 0, 0, 0)
  2604  		binary.BigEndian.PutUint64(b[idx:], p.Keys)
  2605  		idx += 8
  2606  	}
  2607  	return b
  2608  }
  2609  
  2610  func (r rangeProperties) get(key []byte) rangeOffsets {
  2611  	idx := sort.Search(len(r), func(i int) bool {
  2612  		return bytes.Compare(r[i].Key, key) >= 0
  2613  	})
  2614  	return r[idx].rangeOffsets
  2615  }
  2616  
  2617  type RangePropertiesCollector struct {
  2618  	props               rangeProperties
  2619  	lastOffsets         rangeOffsets
  2620  	lastKey             []byte
  2621  	currentOffsets      rangeOffsets
  2622  	propSizeIdxDistance uint64
  2623  	propKeysIdxDistance uint64
  2624  }
  2625  
  2626  func newRangePropertiesCollector() pebble.TablePropertyCollector {
  2627  	return &RangePropertiesCollector{
  2628  		props:               make([]rangeProperty, 0, 1024),
  2629  		propSizeIdxDistance: defaultPropSizeIndexDistance,
  2630  		propKeysIdxDistance: defaultPropKeysIndexDistance,
  2631  	}
  2632  }
  2633  
  2634  func (c *RangePropertiesCollector) sizeInLastRange() uint64 {
  2635  	return c.currentOffsets.Size - c.lastOffsets.Size
  2636  }
  2637  
  2638  func (c *RangePropertiesCollector) keysInLastRange() uint64 {
  2639  	return c.currentOffsets.Keys - c.lastOffsets.Keys
  2640  }
  2641  
  2642  func (c *RangePropertiesCollector) insertNewPoint(key []byte) {
  2643  	c.lastOffsets = c.currentOffsets
  2644  	c.props = append(c.props, rangeProperty{Key: append([]byte{}, key...), rangeOffsets: c.currentOffsets})
  2645  }
  2646  
  2647  // implement `pebble.TablePropertyCollector`
  2648  // implement `TablePropertyCollector.Add`
  2649  func (c *RangePropertiesCollector) Add(key pebble.InternalKey, value []byte) error {
  2650  	c.currentOffsets.Size += uint64(len(value)) + uint64(len(key.UserKey))
  2651  	c.currentOffsets.Keys++
  2652  	if len(c.lastKey) == 0 || c.sizeInLastRange() >= c.propSizeIdxDistance ||
  2653  		c.keysInLastRange() >= c.propKeysIdxDistance {
  2654  		c.insertNewPoint(key.UserKey)
  2655  	}
  2656  	c.lastKey = append(c.lastKey[:0], key.UserKey...)
  2657  	return nil
  2658  }
  2659  
  2660  func (c *RangePropertiesCollector) Finish(userProps map[string]string) error {
  2661  	if c.sizeInLastRange() > 0 || c.keysInLastRange() > 0 {
  2662  		c.insertNewPoint(c.lastKey)
  2663  	}
  2664  
  2665  	userProps[propRangeIndex] = string(c.props.Encode())
  2666  	return nil
  2667  }
  2668  
  2669  // The name of the property collector.
  2670  func (c *RangePropertiesCollector) Name() string {
  2671  	return propRangeIndex
  2672  }
  2673  
  2674  type sizeProperties struct {
  2675  	totalSize    uint64
  2676  	indexHandles *btree.BTree
  2677  }
  2678  
  2679  func newSizeProperties() *sizeProperties {
  2680  	return &sizeProperties{indexHandles: btree.New(32)}
  2681  }
  2682  
  2683  func (s *sizeProperties) add(item *rangeProperty) {
  2684  	if old := s.indexHandles.ReplaceOrInsert(item); old != nil {
  2685  		o := old.(*rangeProperty)
  2686  		item.Keys += o.Keys
  2687  		item.Size += o.Size
  2688  	}
  2689  }
  2690  
  2691  func (s *sizeProperties) addAll(props rangeProperties) {
  2692  	prevRange := rangeOffsets{}
  2693  	for _, r := range props {
  2694  		s.add(&rangeProperty{
  2695  			Key:          r.Key,
  2696  			rangeOffsets: rangeOffsets{Keys: r.Keys - prevRange.Keys, Size: r.Size - prevRange.Size},
  2697  		})
  2698  		prevRange = r.rangeOffsets
  2699  	}
  2700  	if len(props) > 0 {
  2701  		s.totalSize = props[len(props)-1].Size
  2702  	}
  2703  }
  2704  
  2705  // iter the tree until f return false
  2706  func (s *sizeProperties) iter(f func(p *rangeProperty) bool) {
  2707  	s.indexHandles.Ascend(func(i btree.Item) bool {
  2708  		prop := i.(*rangeProperty)
  2709  		return f(prop)
  2710  	})
  2711  }
  2712  
  2713  type sstMeta struct {
  2714  	path       string
  2715  	minKey     []byte
  2716  	maxKey     []byte
  2717  	totalSize  int64
  2718  	totalCount int64
  2719  	// used for calculate disk-quota
  2720  	fileSize int64
  2721  	seq      int32
  2722  }
  2723  
  2724  type Writer struct {
  2725  	sync.Mutex
  2726  	local             *File
  2727  	memtableSizeLimit int64
  2728  
  2729  	// if the KVs are append in order, we can directly write the into SST file,
  2730  	// else we must first store them in writeBatch and then batch flush into SST file.
  2731  	isKVSorted bool
  2732  	writer     *sstWriter
  2733  
  2734  	// bytes buffer for writeBatch
  2735  	kvBuffer   *membuf.Buffer
  2736  	writeBatch []common.KvPair
  2737  	// if the kvs in writeBatch are in order, we can avoid doing a `sort.Slice` which
  2738  	// is quite slow. in our bench, the sort operation eats about 5% of total CPU
  2739  	isWriteBatchSorted bool
  2740  
  2741  	batchCount int
  2742  	batchSize  int64
  2743  	totalSize  int64
  2744  	totalCount int64
  2745  
  2746  	lastMetaSeq int32
  2747  }
  2748  
  2749  func (w *Writer) appendRowsSorted(kvs []common.KvPair) error {
  2750  	if w.writer == nil {
  2751  		writer, err := w.createSSTWriter()
  2752  		if err != nil {
  2753  			return errors.Trace(err)
  2754  		}
  2755  		w.writer = writer
  2756  		w.writer.minKey = append([]byte{}, kvs[0].Key...)
  2757  	}
  2758  
  2759  	totalKeyLen := 0
  2760  	for i := 0; i < len(kvs); i++ {
  2761  		totalKeyLen += w.local.keyAdapter.EncodedLen(kvs[i].Key)
  2762  	}
  2763  	buf := make([]byte, totalKeyLen)
  2764  	encodedKvs := make([]common.KvPair, len(kvs))
  2765  	for i := 0; i < len(kvs); i++ {
  2766  		encodedKey := w.local.keyAdapter.Encode(buf, kvs[i].Key, kvs[i].RowID, kvs[i].Offset)
  2767  		buf = buf[len(encodedKey):]
  2768  		encodedKvs[i] = common.KvPair{Key: encodedKey, Val: kvs[i].Val}
  2769  		w.batchSize += int64(len(encodedKvs[i].Key) + len(encodedKvs[i].Val))
  2770  	}
  2771  
  2772  	w.batchCount += len(encodedKvs)
  2773  	w.totalCount += int64(len(encodedKvs))
  2774  	return w.writer.writeKVs(encodedKvs)
  2775  }
  2776  
  2777  func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) error {
  2778  	l := len(w.writeBatch)
  2779  	cnt := w.batchCount
  2780  	var lastKey []byte
  2781  	if len(w.writeBatch) > 0 {
  2782  		lastKey = w.writeBatch[len(w.writeBatch)-1].Key
  2783  	}
  2784  	for _, pair := range kvs {
  2785  		if w.isWriteBatchSorted && bytes.Compare(lastKey, pair.Key) > 0 {
  2786  			w.isWriteBatchSorted = false
  2787  		}
  2788  		lastKey = pair.Key
  2789  		w.batchSize += int64(len(pair.Key) + len(pair.Val))
  2790  		buf := w.kvBuffer.AllocBytes(w.local.keyAdapter.EncodedLen(pair.Key))
  2791  		key := w.local.keyAdapter.Encode(buf, pair.Key, pair.RowID, pair.Offset)
  2792  		val := w.kvBuffer.AddBytes(pair.Val)
  2793  		if cnt < l {
  2794  			w.writeBatch[cnt].Key = key
  2795  			w.writeBatch[cnt].Val = val
  2796  		} else {
  2797  			w.writeBatch = append(w.writeBatch, common.KvPair{Key: key, Val: val})
  2798  		}
  2799  		cnt++
  2800  	}
  2801  	w.batchCount = cnt
  2802  
  2803  	if w.batchSize > w.memtableSizeLimit {
  2804  		if err := w.flushKVs(ctx); err != nil {
  2805  			return err
  2806  		}
  2807  	}
  2808  	w.totalCount += int64(len(kvs))
  2809  	return nil
  2810  }
  2811  
  2812  func (local *local) EngineFileSizes() (res []backend.EngineFileSize) {
  2813  	local.engines.Range(func(k, v interface{}) bool {
  2814  		engine := v.(*File)
  2815  		res = append(res, engine.getEngineFileSize())
  2816  		return true
  2817  	})
  2818  	return
  2819  }
  2820  
  2821  func (w *Writer) AppendRows(ctx context.Context, tableName string, columnNames []string, rows kv.Rows) error {
  2822  	kvs := kv.KvPairsFromRows(rows)
  2823  	if len(kvs) == 0 {
  2824  		return nil
  2825  	}
  2826  
  2827  	if w.local.closed.Load() {
  2828  		return errorEngineClosed
  2829  	}
  2830  
  2831  	w.Lock()
  2832  	defer w.Unlock()
  2833  
  2834  	// if chunk has _tidb_rowid field, we can't ensure that the rows are sorted.
  2835  	if w.isKVSorted && w.writer == nil {
  2836  		for _, c := range columnNames {
  2837  			if c == model.ExtraHandleName.L {
  2838  				w.isKVSorted = false
  2839  			}
  2840  		}
  2841  	}
  2842  
  2843  	if w.isKVSorted {
  2844  		return w.appendRowsSorted(kvs)
  2845  	}
  2846  	return w.appendRowsUnsorted(ctx, kvs)
  2847  }
  2848  
  2849  func (w *Writer) flush(ctx context.Context) error {
  2850  	w.Lock()
  2851  	defer w.Unlock()
  2852  	if w.batchCount == 0 {
  2853  		return nil
  2854  	}
  2855  
  2856  	w.totalSize += w.batchSize
  2857  	if len(w.writeBatch) > 0 {
  2858  		if err := w.flushKVs(ctx); err != nil {
  2859  			return errors.Trace(err)
  2860  		}
  2861  	}
  2862  
  2863  	if w.writer != nil {
  2864  		meta, err := w.writer.close()
  2865  		if err != nil {
  2866  			return errors.Trace(err)
  2867  		}
  2868  		w.writer = nil
  2869  		w.batchCount = 0
  2870  		if meta != nil && meta.totalSize > 0 {
  2871  			return w.addSST(ctx, meta)
  2872  		}
  2873  	}
  2874  
  2875  	return nil
  2876  }
  2877  
  2878  type flushStatus struct {
  2879  	local *File
  2880  	seq   int32
  2881  }
  2882  
  2883  func (f flushStatus) Flushed() bool {
  2884  	return f.seq <= f.local.finishedMetaSeq.Load()
  2885  }
  2886  
  2887  func (w *Writer) Close(ctx context.Context) (backend.ChunkFlushStatus, error) {
  2888  	defer w.kvBuffer.Destroy()
  2889  	defer w.local.localWriters.Delete(w)
  2890  	err := w.flush(ctx)
  2891  	// FIXME: in theory this line is useless, but In our benchmark with go1.15
  2892  	// this can resolve the memory consistently increasing issue.
  2893  	// maybe this is a bug related to go GC mechanism.
  2894  	w.writeBatch = nil
  2895  	return flushStatus{local: w.local, seq: w.lastMetaSeq}, err
  2896  }
  2897  
  2898  func (w *Writer) IsSynced() bool {
  2899  	return w.batchCount == 0 && w.lastMetaSeq <= w.local.finishedMetaSeq.Load()
  2900  }
  2901  
  2902  func (w *Writer) flushKVs(ctx context.Context) error {
  2903  	writer, err := w.createSSTWriter()
  2904  	if err != nil {
  2905  		return errors.Trace(err)
  2906  	}
  2907  	if !w.isWriteBatchSorted {
  2908  		sort.Slice(w.writeBatch[:w.batchCount], func(i, j int) bool {
  2909  			return bytes.Compare(w.writeBatch[i].Key, w.writeBatch[j].Key) < 0
  2910  		})
  2911  		w.isWriteBatchSorted = true
  2912  	}
  2913  
  2914  	writer.minKey = append(writer.minKey[:0], w.writeBatch[0].Key...)
  2915  	err = writer.writeKVs(w.writeBatch[:w.batchCount])
  2916  	if err != nil {
  2917  		return errors.Trace(err)
  2918  	}
  2919  	meta, err := writer.close()
  2920  	if err != nil {
  2921  		return errors.Trace(err)
  2922  	}
  2923  	err = w.addSST(ctx, meta)
  2924  	if err != nil {
  2925  		return errors.Trace(err)
  2926  	}
  2927  
  2928  	w.totalSize += w.batchSize
  2929  	w.batchSize = 0
  2930  	w.batchCount = 0
  2931  	w.kvBuffer.Reset()
  2932  	return nil
  2933  }
  2934  
  2935  func (w *Writer) addSST(ctx context.Context, meta *sstMeta) error {
  2936  	seq, err := w.local.addSST(ctx, meta)
  2937  	if err != nil {
  2938  		return err
  2939  	}
  2940  	w.lastMetaSeq = seq
  2941  	return nil
  2942  }
  2943  
  2944  func (w *Writer) createSSTWriter() (*sstWriter, error) {
  2945  	path := filepath.Join(w.local.sstDir, uuid.New().String()+".sst")
  2946  	writer, err := newSSTWriter(path)
  2947  	if err != nil {
  2948  		return nil, err
  2949  	}
  2950  	sw := &sstWriter{sstMeta: &sstMeta{path: path}, writer: writer}
  2951  	return sw, nil
  2952  }
  2953  
  2954  var errorUnorderedSSTInsertion = errors.New("inserting KVs into SST without order")
  2955  
  2956  type sstWriter struct {
  2957  	*sstMeta
  2958  	writer *sstable.Writer
  2959  }
  2960  
  2961  func newSSTWriter(path string) (*sstable.Writer, error) {
  2962  	f, err := os.Create(path)
  2963  	if err != nil {
  2964  		return nil, errors.Trace(err)
  2965  	}
  2966  	writer := sstable.NewWriter(f, sstable.WriterOptions{
  2967  		TablePropertyCollectors: []func() pebble.TablePropertyCollector{
  2968  			newRangePropertiesCollector,
  2969  		},
  2970  		BlockSize: 16 * 1024,
  2971  	})
  2972  	return writer, nil
  2973  }
  2974  
  2975  func (sw *sstWriter) writeKVs(kvs []common.KvPair) error {
  2976  	if len(kvs) == 0 {
  2977  		return nil
  2978  	}
  2979  
  2980  	if bytes.Compare(kvs[0].Key, sw.maxKey) <= 0 {
  2981  		return errorUnorderedSSTInsertion
  2982  	}
  2983  
  2984  	internalKey := sstable.InternalKey{
  2985  		Trailer: uint64(sstable.InternalKeyKindSet),
  2986  	}
  2987  	var lastKey []byte
  2988  	for _, p := range kvs {
  2989  		if bytes.Equal(p.Key, lastKey) {
  2990  			log.L().Warn("duplicated key found, skip write", logutil.Key("key", p.Key))
  2991  			continue
  2992  		}
  2993  		internalKey.UserKey = p.Key
  2994  		if err := sw.writer.Add(internalKey, p.Val); err != nil {
  2995  			return errors.Trace(err)
  2996  		}
  2997  		sw.totalSize += int64(len(p.Key)) + int64(len(p.Val))
  2998  	}
  2999  	sw.totalCount += int64(len(kvs))
  3000  	sw.maxKey = append(sw.maxKey[:0], kvs[len(kvs)-1].Key...)
  3001  	return nil
  3002  }
  3003  
  3004  func (sw *sstWriter) close() (*sstMeta, error) {
  3005  	if err := sw.writer.Close(); err != nil {
  3006  		return nil, errors.Trace(err)
  3007  	}
  3008  	meta, err := sw.writer.Metadata()
  3009  	if err != nil {
  3010  		return nil, errors.Trace(err)
  3011  	}
  3012  	sw.fileSize = int64(meta.Size)
  3013  	return sw.sstMeta, nil
  3014  }
  3015  
  3016  type sstIter struct {
  3017  	name   string
  3018  	key    []byte
  3019  	val    []byte
  3020  	iter   sstable.Iterator
  3021  	reader *sstable.Reader
  3022  	valid  bool
  3023  }
  3024  
  3025  func (i *sstIter) Close() error {
  3026  	if err := i.iter.Close(); err != nil {
  3027  		return errors.Trace(err)
  3028  	}
  3029  	err := i.reader.Close()
  3030  	return errors.Trace(err)
  3031  }
  3032  
  3033  type sstIterHeap struct {
  3034  	iters []*sstIter
  3035  }
  3036  
  3037  func (h *sstIterHeap) Len() int {
  3038  	return len(h.iters)
  3039  }
  3040  
  3041  func (h *sstIterHeap) Less(i, j int) bool {
  3042  	return bytes.Compare(h.iters[i].key, h.iters[j].key) < 0
  3043  }
  3044  
  3045  func (h *sstIterHeap) Swap(i, j int) {
  3046  	h.iters[i], h.iters[j] = h.iters[j], h.iters[i]
  3047  }
  3048  
  3049  func (h *sstIterHeap) Push(x interface{}) {
  3050  	h.iters = append(h.iters, x.(*sstIter))
  3051  }
  3052  
  3053  func (h *sstIterHeap) Pop() interface{} {
  3054  	item := h.iters[len(h.iters)-1]
  3055  	h.iters = h.iters[:len(h.iters)-1]
  3056  	return item
  3057  }
  3058  
  3059  func (h *sstIterHeap) Next() ([]byte, []byte, error) {
  3060  	for {
  3061  		if len(h.iters) == 0 {
  3062  			return nil, nil, nil
  3063  		}
  3064  
  3065  		iter := h.iters[0]
  3066  		if iter.valid {
  3067  			iter.valid = false
  3068  			return iter.key, iter.val, iter.iter.Error()
  3069  		}
  3070  
  3071  		var k *pebble.InternalKey
  3072  		k, iter.val = iter.iter.Next()
  3073  		if k != nil {
  3074  			iter.key = k.UserKey
  3075  			iter.valid = true
  3076  			heap.Fix(h, 0)
  3077  		} else {
  3078  			err := iter.Close()
  3079  			heap.Remove(h, 0)
  3080  			if err != nil {
  3081  				return nil, nil, errors.Trace(err)
  3082  			}
  3083  		}
  3084  	}
  3085  }
  3086  
  3087  // sstIngester is a interface used to merge and ingest SST files.
  3088  // it's a interface mainly used for test convenience
  3089  type sstIngester interface {
  3090  	mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error)
  3091  	ingest([]*sstMeta) error
  3092  }
  3093  
  3094  type dbSSTIngester struct {
  3095  	e *File
  3096  }
  3097  
  3098  func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) {
  3099  	if len(metas) == 0 {
  3100  		return nil, errors.New("sst metas is empty")
  3101  	} else if len(metas) == 1 {
  3102  		return metas[0], nil
  3103  	}
  3104  
  3105  	start := time.Now()
  3106  	newMeta := &sstMeta{
  3107  		seq: metas[len(metas)-1].seq,
  3108  	}
  3109  	mergeIter := &sstIterHeap{
  3110  		iters: make([]*sstIter, 0, len(metas)),
  3111  	}
  3112  
  3113  	for _, p := range metas {
  3114  		f, err := os.Open(p.path)
  3115  		if err != nil {
  3116  			return nil, errors.Trace(err)
  3117  		}
  3118  		reader, err := sstable.NewReader(f, sstable.ReaderOptions{})
  3119  		if err != nil {
  3120  			return nil, errors.Trace(err)
  3121  		}
  3122  		iter, err := reader.NewIter(nil, nil)
  3123  		if err != nil {
  3124  			return nil, errors.Trace(err)
  3125  		}
  3126  		key, val := iter.Next()
  3127  		if key == nil {
  3128  			continue
  3129  		}
  3130  		if iter.Error() != nil {
  3131  			return nil, errors.Trace(iter.Error())
  3132  		}
  3133  		mergeIter.iters = append(mergeIter.iters, &sstIter{
  3134  			name:   p.path,
  3135  			iter:   iter,
  3136  			key:    key.UserKey,
  3137  			val:    val,
  3138  			reader: reader,
  3139  			valid:  true,
  3140  		})
  3141  		newMeta.totalSize += p.totalSize
  3142  		newMeta.totalCount += p.totalCount
  3143  	}
  3144  	heap.Init(mergeIter)
  3145  
  3146  	name := filepath.Join(dir, fmt.Sprintf("%s.sst", uuid.New()))
  3147  	writer, err := newSSTWriter(name)
  3148  	if err != nil {
  3149  		return nil, errors.Trace(err)
  3150  	}
  3151  	newMeta.path = name
  3152  
  3153  	internalKey := sstable.InternalKey{
  3154  		Trailer: uint64(sstable.InternalKeyKindSet),
  3155  	}
  3156  	key, val, err := mergeIter.Next()
  3157  	if err != nil {
  3158  		return nil, err
  3159  	}
  3160  	if key == nil {
  3161  		return nil, errors.New("all ssts are empty!")
  3162  	}
  3163  	newMeta.minKey = append(newMeta.minKey[:0], key...)
  3164  	lastKey := make([]byte, 0)
  3165  	for {
  3166  		if bytes.Equal(lastKey, key) {
  3167  			log.L().Warn("duplicated key found, skipped", zap.Binary("key", lastKey))
  3168  			continue
  3169  		}
  3170  		internalKey.UserKey = key
  3171  		err = writer.Add(internalKey, val)
  3172  		if err != nil {
  3173  			return nil, err
  3174  		}
  3175  		lastKey = append(lastKey[:0], key...)
  3176  		key, val, err = mergeIter.Next()
  3177  		if err != nil {
  3178  			return nil, err
  3179  		}
  3180  		if key == nil {
  3181  			break
  3182  		}
  3183  	}
  3184  	err = writer.Close()
  3185  	if err != nil {
  3186  		return nil, errors.Trace(err)
  3187  	}
  3188  	meta, err := writer.Metadata()
  3189  	if err != nil {
  3190  		return nil, errors.Trace(err)
  3191  	}
  3192  	newMeta.maxKey = lastKey
  3193  	newMeta.fileSize = int64(meta.Size)
  3194  
  3195  	dur := time.Since(start)
  3196  	log.L().Info("compact sst", zap.Int("fileCount", len(metas)), zap.Int64("size", newMeta.totalSize),
  3197  		zap.Int64("count", newMeta.totalCount), zap.Duration("cost", dur), zap.String("file", name))
  3198  
  3199  	// async clean raw SSTs.
  3200  	go func() {
  3201  		totalSize := int64(0)
  3202  		for _, m := range metas {
  3203  			totalSize += m.fileSize
  3204  			if err := os.Remove(m.path); err != nil {
  3205  				log.L().Warn("async cleanup sst file failed", zap.Error(err))
  3206  			}
  3207  		}
  3208  		// decrease the pending size after clean up
  3209  		i.e.pendingFileSize.Sub(totalSize)
  3210  	}()
  3211  
  3212  	return newMeta, err
  3213  }
  3214  
  3215  func (i dbSSTIngester) ingest(metas []*sstMeta) error {
  3216  	if len(metas) == 0 {
  3217  		return nil
  3218  	}
  3219  	paths := make([]string, 0, len(metas))
  3220  	for _, m := range metas {
  3221  		paths = append(paths, m.path)
  3222  	}
  3223  	return i.e.db.Ingest(paths)
  3224  }