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

     1  // Copyright 2020 PingCAP, Inc. Licensed under Apache-2.0.
     2  
     3  package restore
     4  
     5  import (
     6  	"bytes"
     7  	"context"
     8  	"encoding/json"
     9  	"fmt"
    10  	"path/filepath"
    11  	"sort"
    12  	"strconv"
    13  	"strings"
    14  	"sync"
    15  	"time"
    16  
    17  	"github.com/pingcap/errors"
    18  	"github.com/pingcap/log"
    19  	"github.com/pingcap/parser/model"
    20  	filter "github.com/pingcap/tidb-tools/pkg/table-filter"
    21  	"github.com/pingcap/tidb/domain"
    22  	"github.com/pingcap/tidb/meta/autoid"
    23  	titable "github.com/pingcap/tidb/table"
    24  	"github.com/tikv/client-go/v2/oracle"
    25  	"go.uber.org/zap"
    26  	"golang.org/x/sync/errgroup"
    27  
    28  	"github.com/pingcap/br/pkg/cdclog"
    29  	berrors "github.com/pingcap/br/pkg/errors"
    30  	"github.com/pingcap/br/pkg/kv"
    31  	"github.com/pingcap/br/pkg/storage"
    32  	"github.com/pingcap/br/pkg/utils"
    33  )
    34  
    35  const (
    36  	tableLogPrefix = "t_"
    37  	logPrefix      = "cdclog"
    38  
    39  	metaFile      = "log.meta"
    40  	ddlEventsDir  = "ddls"
    41  	ddlFilePrefix = "ddl"
    42  
    43  	maxUint64 = ^uint64(0)
    44  
    45  	maxRetryTimes = 3
    46  )
    47  
    48  // concurrencyCfg set by user, which can adjust the restore performance.
    49  type concurrencyCfg struct {
    50  	BatchWriteKVPairs int
    51  	BatchFlushKVPairs int
    52  	BatchFlushKVSize  int64
    53  	Concurrency       uint
    54  	TCPConcurrency    int
    55  	IngestConcurrency uint
    56  }
    57  
    58  // LogMeta represents the log.meta generated by cdc log backup.
    59  type LogMeta struct {
    60  	Names            map[int64]string `json:"names"`
    61  	GlobalResolvedTS uint64           `json:"global_resolved_ts"`
    62  }
    63  
    64  // LogClient sends requests to restore files.
    65  type LogClient struct {
    66  	// lock DDL execution
    67  	// TODO remove lock by using db session pool if necessary
    68  	ddlLock sync.Mutex
    69  
    70  	restoreClient  *Client
    71  	splitClient    SplitClient
    72  	importerClient ImporterClient
    73  
    74  	// ingester is used to write and ingest kvs to tikv.
    75  	// lightning has the simlar logic and can reuse it.
    76  	ingester *Ingester
    77  
    78  	// range of log backup
    79  	startTS uint64
    80  	endTS   uint64
    81  
    82  	concurrencyCfg concurrencyCfg
    83  	// meta info parsed from log backup
    84  	meta         *LogMeta
    85  	eventPullers map[int64]*cdclog.EventPuller
    86  	tableBuffers map[int64]*cdclog.TableBuffer
    87  
    88  	tableFilter filter.Filter
    89  
    90  	// a map to store all drop schema ts, use it as a filter
    91  	dropTSMap sync.Map
    92  }
    93  
    94  // NewLogRestoreClient returns a new LogRestoreClient.
    95  func NewLogRestoreClient(
    96  	ctx context.Context,
    97  	restoreClient *Client,
    98  	startTS uint64,
    99  	endTS uint64,
   100  	tableFilter filter.Filter,
   101  	concurrency uint,
   102  	batchFlushPairs int,
   103  	batchFlushSize int64,
   104  	batchWriteKVPairs int,
   105  ) (*LogClient, error) {
   106  	var err error
   107  	if endTS == 0 {
   108  		// means restore all log data,
   109  		// so we get current ts from restore cluster
   110  		endTS, err = restoreClient.GetTS(ctx)
   111  		if err != nil {
   112  			return nil, errors.Trace(err)
   113  		}
   114  	}
   115  
   116  	tlsConf := restoreClient.GetTLSConfig()
   117  	splitClient := NewSplitClient(restoreClient.GetPDClient(), tlsConf)
   118  	importClient := NewImportClient(splitClient, tlsConf, restoreClient.keepaliveConf)
   119  
   120  	cfg := concurrencyCfg{
   121  		Concurrency:       concurrency,
   122  		BatchFlushKVPairs: batchFlushPairs,
   123  		BatchFlushKVSize:  batchFlushSize,
   124  		BatchWriteKVPairs: batchWriteKVPairs,
   125  		IngestConcurrency: concurrency * 16,
   126  		TCPConcurrency:    int(concurrency) * 16,
   127  	}
   128  
   129  	// commitTS append into encode key. we use a unified ts for once log restore.
   130  	commitTS := oracle.ComposeTS(time.Now().Unix()*1000, 0)
   131  	lc := &LogClient{
   132  		restoreClient:  restoreClient,
   133  		splitClient:    splitClient,
   134  		importerClient: importClient,
   135  		startTS:        startTS,
   136  		endTS:          endTS,
   137  		concurrencyCfg: cfg,
   138  		meta:           new(LogMeta),
   139  		eventPullers:   make(map[int64]*cdclog.EventPuller),
   140  		tableBuffers:   make(map[int64]*cdclog.TableBuffer),
   141  		tableFilter:    tableFilter,
   142  		ingester:       NewIngester(splitClient, cfg, commitTS, tlsConf),
   143  	}
   144  	return lc, nil
   145  }
   146  
   147  // ResetTSRange used for test.
   148  func (l *LogClient) ResetTSRange(startTS uint64, endTS uint64) {
   149  	l.startTS = startTS
   150  	l.endTS = endTS
   151  }
   152  
   153  func (l *LogClient) maybeTSInRange(ts uint64) bool {
   154  	// We choose the last event's ts as file name in cdclog when rotate.
   155  	// so even this file name's ts is larger than l.endTS,
   156  	// we still need to collect it, because it may have some events in this ts range.
   157  	// TODO: find another effective filter to collect files
   158  	return ts >= l.startTS
   159  }
   160  
   161  func (l *LogClient) tsInRange(ts uint64) bool {
   162  	return l.startTS <= ts && ts <= l.endTS
   163  }
   164  
   165  func (l *LogClient) shouldFilter(item *cdclog.SortItem) bool {
   166  	if val, ok := l.dropTSMap.Load(item.Schema); ok {
   167  		if val.(uint64) > item.TS {
   168  			return true
   169  		}
   170  	}
   171  	return false
   172  }
   173  
   174  // NeedRestoreDDL determines whether to collect ddl file by ts range.
   175  func (l *LogClient) NeedRestoreDDL(fileName string) (bool, error) {
   176  	names := strings.Split(fileName, ".")
   177  	if len(names) != 2 {
   178  		log.Warn("found wrong format of ddl file", zap.String("file", fileName))
   179  		return false, nil
   180  	}
   181  	if names[0] != ddlFilePrefix {
   182  		log.Warn("file doesn't start with ddl", zap.String("file", fileName))
   183  		return false, nil
   184  	}
   185  	ts, err := strconv.ParseUint(names[1], 10, 64)
   186  	if err != nil {
   187  		return false, errors.Trace(err)
   188  	}
   189  
   190  	// According to https://docs.aws.amazon.com/AmazonS3/latest/dev/ListingKeysUsingAPIs.html
   191  	// list API return in UTF-8 binary order, so the cdc log create DDL file used
   192  	// maxUint64 - the first DDL event's commit ts as the file name to return the latest ddl file.
   193  	// see details at https://github.com/pingcap/ticdc/pull/826/files#diff-d2e98b3ed211b7b9bb7b6da63dd48758R81
   194  	ts = maxUint64 - ts
   195  
   196  	// In cdc, we choose the first event as the file name of DDL file.
   197  	// so if the file ts is large than endTS, we can skip to execute it.
   198  	// FIXME find a unified logic to filter row changes files and ddl files.
   199  	if ts <= l.endTS {
   200  		return true, nil
   201  	}
   202  	log.Info("filter ddl file by ts", zap.String("name", fileName), zap.Uint64("ts", ts))
   203  	return false, nil
   204  }
   205  
   206  func (l *LogClient) collectDDLFiles(ctx context.Context) ([]string, error) {
   207  	ddlFiles := make([]string, 0)
   208  	opt := &storage.WalkOption{
   209  		SubDir:    ddlEventsDir,
   210  		ListCount: -1,
   211  	}
   212  	err := l.restoreClient.storage.WalkDir(ctx, opt, func(path string, size int64) error {
   213  		fileName := filepath.Base(path)
   214  		shouldRestore, err := l.NeedRestoreDDL(fileName)
   215  		if err != nil {
   216  			return errors.Trace(err)
   217  		}
   218  		if shouldRestore {
   219  			ddlFiles = append(ddlFiles, path)
   220  		}
   221  		return nil
   222  	})
   223  	if err != nil {
   224  		return nil, errors.Trace(err)
   225  	}
   226  
   227  	sort.Sort(sort.Reverse(sort.StringSlice(ddlFiles)))
   228  	return ddlFiles, nil
   229  }
   230  
   231  func (l *LogClient) isDBRelatedDDL(ddl *cdclog.MessageDDL) bool {
   232  	switch ddl.Type {
   233  	case model.ActionDropSchema, model.ActionCreateSchema, model.ActionModifySchemaCharsetAndCollate:
   234  		return true
   235  	}
   236  	return false
   237  }
   238  
   239  func (l *LogClient) isDropTable(ddl *cdclog.MessageDDL) bool {
   240  	return ddl.Type == model.ActionDropTable
   241  }
   242  
   243  func (l *LogClient) doDBDDLJob(ctx context.Context, ddls []string) error {
   244  	if len(ddls) == 0 {
   245  		log.Info("no ddls to restore")
   246  		return nil
   247  	}
   248  
   249  	for _, path := range ddls {
   250  		data, err := l.restoreClient.storage.ReadFile(ctx, path)
   251  		if err != nil {
   252  			return errors.Trace(err)
   253  		}
   254  		eventDecoder, err := cdclog.NewJSONEventBatchDecoder(data)
   255  		if err != nil {
   256  			return errors.Trace(err)
   257  		}
   258  		for eventDecoder.HasNext() {
   259  			item, err := eventDecoder.NextEvent(cdclog.DDL)
   260  			if err != nil {
   261  				return errors.Trace(err)
   262  			}
   263  			ddl := item.Data.(*cdclog.MessageDDL)
   264  			log.Debug("[doDBDDLJob] parse ddl", zap.String("query", ddl.Query))
   265  			if l.isDBRelatedDDL(ddl) && l.tsInRange(item.TS) {
   266  				err = l.restoreClient.db.se.Execute(ctx, ddl.Query)
   267  				if err != nil {
   268  					log.Error("[doDBDDLJob] exec ddl failed",
   269  						zap.String("query", ddl.Query), zap.Error(err))
   270  					return errors.Trace(err)
   271  				}
   272  				if ddl.Type == model.ActionDropSchema {
   273  					// store the drop schema ts, and then we need filter evetns which ts is small than this.
   274  					l.dropTSMap.Store(item.Schema, item.TS)
   275  				}
   276  			}
   277  		}
   278  	}
   279  	return nil
   280  }
   281  
   282  // NeedRestoreRowChange determine whether to collect this file by ts range.
   283  func (l *LogClient) NeedRestoreRowChange(fileName string) (bool, error) {
   284  	if fileName == logPrefix {
   285  		// this file name appeared when file sink enabled
   286  		return true, nil
   287  	}
   288  	names := strings.Split(fileName, ".")
   289  	if len(names) != 2 {
   290  		log.Warn("found wrong format of row changes file", zap.String("file", fileName))
   291  		return false, nil
   292  	}
   293  	if names[0] != logPrefix {
   294  		log.Warn("file doesn't start with row changes file", zap.String("file", fileName))
   295  		return false, nil
   296  	}
   297  	ts, err := strconv.ParseUint(names[1], 10, 64)
   298  	if err != nil {
   299  		return false, errors.Trace(err)
   300  	}
   301  	if l.maybeTSInRange(ts) {
   302  		return true, nil
   303  	}
   304  	log.Info("filter file by ts", zap.String("name", fileName), zap.Uint64("ts", ts))
   305  	return false, nil
   306  }
   307  
   308  func (l *LogClient) collectRowChangeFiles(ctx context.Context) (map[int64][]string, error) {
   309  	// we should collect all related tables row change files
   310  	// by log meta info and by given table filter
   311  	rowChangeFiles := make(map[int64][]string)
   312  
   313  	// need collect restore tableIDs
   314  	tableIDs := make([]int64, 0, len(l.meta.Names))
   315  
   316  	// we need remove duplicate table name in collection.
   317  	// when a table create and drop and create again.
   318  	// then we will have two different table id with same tables.
   319  	// we should keep the latest table id(larger table id), and filter the old one.
   320  	nameIDMap := make(map[string]int64)
   321  	for tableID, name := range l.meta.Names {
   322  		if tid, ok := nameIDMap[name]; ok {
   323  			if tid < tableID {
   324  				nameIDMap[name] = tableID
   325  			}
   326  		} else {
   327  			nameIDMap[name] = tableID
   328  		}
   329  	}
   330  	for name, tableID := range nameIDMap {
   331  		schema, table := ParseQuoteName(name)
   332  		if !l.tableFilter.MatchTable(schema, table) {
   333  			log.Info("filter tables", zap.String("schema", schema),
   334  				zap.String("table", table), zap.Int64("tableID", tableID))
   335  			continue
   336  		}
   337  		tableIDs = append(tableIDs, tableID)
   338  	}
   339  
   340  	for _, tID := range tableIDs {
   341  		tableID := tID
   342  		// FIXME update log meta logic here
   343  		dir := fmt.Sprintf("%s%d", tableLogPrefix, tableID)
   344  		opt := &storage.WalkOption{
   345  			SubDir:    dir,
   346  			ListCount: -1,
   347  		}
   348  		err := l.restoreClient.storage.WalkDir(ctx, opt, func(path string, size int64) error {
   349  			fileName := filepath.Base(path)
   350  			shouldRestore, err := l.NeedRestoreRowChange(fileName)
   351  			if err != nil {
   352  				return errors.Trace(err)
   353  			}
   354  			if shouldRestore {
   355  				rowChangeFiles[tableID] = append(rowChangeFiles[tableID], path)
   356  			}
   357  			return nil
   358  		})
   359  		if err != nil {
   360  			return nil, errors.Trace(err)
   361  		}
   362  	}
   363  
   364  	// sort file in order
   365  	for tID, files := range rowChangeFiles {
   366  		sortFiles := files
   367  		sort.Slice(sortFiles, func(i, j int) bool {
   368  			if filepath.Base(sortFiles[j]) == logPrefix {
   369  				return true
   370  			}
   371  			return sortFiles[i] < sortFiles[j]
   372  		})
   373  		rowChangeFiles[tID] = sortFiles
   374  	}
   375  
   376  	return rowChangeFiles, nil
   377  }
   378  
   379  func (l *LogClient) writeRows(ctx context.Context, kvs kv.Pairs) error {
   380  	log.Info("writeRows", zap.Int("kv count", len(kvs)))
   381  	if len(kvs) == 0 {
   382  		// shouldn't happen
   383  		log.Warn("not rows to write")
   384  		return nil
   385  	}
   386  
   387  	// stable sort kvs in memory
   388  	sort.SliceStable(kvs, func(i, j int) bool {
   389  		return bytes.Compare(kvs[i].Key, kvs[j].Key) < 0
   390  	})
   391  
   392  	// remove duplicate keys, and keep the last one
   393  	newKvs := make([]kv.Pair, 0, len(kvs))
   394  	for i := 0; i < len(kvs); i++ {
   395  		if i == len(kvs)-1 {
   396  			newKvs = append(newKvs, kvs[i])
   397  			break
   398  		}
   399  		if bytes.Equal(kvs[i].Key, kvs[i+1].Key) {
   400  			// skip this one
   401  			continue
   402  		}
   403  		newKvs = append(newKvs, kvs[i])
   404  	}
   405  
   406  	remainRange := newSyncdRanges()
   407  	remainRange.add(Range{
   408  		Start: newKvs[0].Key,
   409  		End:   kv.NextKey(newKvs[len(newKvs)-1].Key),
   410  	})
   411  	iterProducer := kv.NewSimpleKVIterProducer(newKvs)
   412  	for {
   413  		remain := remainRange.take()
   414  		if len(remain) == 0 {
   415  			log.Info("writeRows finish")
   416  			break
   417  		}
   418  		eg, ectx := errgroup.WithContext(ctx)
   419  		for _, r := range remain {
   420  			rangeReplica := r
   421  			l.ingester.WorkerPool.ApplyOnErrorGroup(eg, func() error {
   422  				err := l.ingester.writeAndIngestByRange(ectx, iterProducer, rangeReplica.Start, rangeReplica.End, remainRange)
   423  				if err != nil {
   424  					log.Warn("writeRows failed with range", zap.Any("range", rangeReplica), zap.Error(err))
   425  					return errors.Trace(err)
   426  				}
   427  				return nil
   428  			})
   429  		}
   430  		if err := eg.Wait(); err != nil {
   431  			return errors.Trace(err)
   432  		}
   433  		log.Info("writeRows ranges unfinished, retry it", zap.Int("remain ranges", len(remain)))
   434  	}
   435  	return nil
   436  }
   437  
   438  func (l *LogClient) reloadTableMeta(dom *domain.Domain, tableID int64, item *cdclog.SortItem) error {
   439  	err := dom.Reload()
   440  	if err != nil {
   441  		return errors.Trace(err)
   442  	}
   443  	// find tableID for this table on cluster
   444  	newTableID := l.tableBuffers[tableID].TableID()
   445  	var (
   446  		newTableInfo titable.Table
   447  		ok           bool
   448  	)
   449  	if newTableID != 0 {
   450  		newTableInfo, ok = dom.InfoSchema().TableByID(newTableID)
   451  		if !ok {
   452  			log.Error("[restoreFromPuller] can't get table info from dom by tableID",
   453  				zap.Int64("backup table id", tableID),
   454  				zap.Int64("restore table id", newTableID),
   455  			)
   456  			return errors.Trace(err)
   457  		}
   458  	} else {
   459  		// fall back to use schema table get info
   460  		newTableInfo, err = dom.InfoSchema().TableByName(
   461  			model.NewCIStr(item.Schema), model.NewCIStr(item.Table))
   462  		if err != nil {
   463  			log.Error("[restoreFromPuller] can't get table info from dom by table name",
   464  				zap.Int64("backup table id", tableID),
   465  				zap.Int64("restore table id", newTableID),
   466  				zap.String("restore table name", item.Table),
   467  				zap.String("restore schema name", item.Schema),
   468  			)
   469  			return errors.Trace(err)
   470  		}
   471  	}
   472  
   473  	dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(item.Schema))
   474  	if !ok {
   475  		return errors.Annotatef(berrors.ErrRestoreSchemaNotExists, "schema %s", item.Schema)
   476  	}
   477  	allocs := autoid.NewAllocatorsFromTblInfo(dom.Store(), dbInfo.ID, newTableInfo.Meta())
   478  
   479  	// reload
   480  	l.tableBuffers[tableID].ReloadMeta(newTableInfo, allocs)
   481  	log.Debug("reload table meta for table",
   482  		zap.Int64("backup table id", tableID),
   483  		zap.Int64("restore table id", newTableID),
   484  		zap.String("restore table name", item.Table),
   485  		zap.String("restore schema name", item.Schema),
   486  		zap.Any("allocator", len(allocs)),
   487  		zap.Any("auto", newTableInfo.Meta().GetAutoIncrementColInfo()),
   488  	)
   489  	return nil
   490  }
   491  
   492  func (l *LogClient) applyKVChanges(ctx context.Context, tableID int64) error {
   493  	log.Info("apply kv changes to tikv",
   494  		zap.Any("table", tableID),
   495  	)
   496  	dataKVs := kv.Pairs{}
   497  	indexKVs := kv.Pairs{}
   498  
   499  	tableBuffer := l.tableBuffers[tableID]
   500  	if tableBuffer.IsEmpty() {
   501  		log.Warn("no kv changes to apply")
   502  		return nil
   503  	}
   504  
   505  	var dataChecksum, indexChecksum kv.Checksum
   506  	for _, p := range tableBuffer.KvPairs {
   507  		p.ClassifyAndAppend(&dataKVs, &dataChecksum, &indexKVs, &indexChecksum)
   508  	}
   509  
   510  	err := l.writeRows(ctx, dataKVs)
   511  	if err != nil {
   512  		return errors.Trace(err)
   513  	}
   514  	dataKVs = dataKVs.Clear()
   515  
   516  	err = l.writeRows(ctx, indexKVs)
   517  	if err != nil {
   518  		return errors.Trace(err)
   519  	}
   520  	indexKVs = indexKVs.Clear()
   521  
   522  	tableBuffer.Clear()
   523  
   524  	return nil
   525  }
   526  
   527  func (l *LogClient) restoreTableFromPuller(
   528  	ctx context.Context,
   529  	tableID int64,
   530  	puller *cdclog.EventPuller,
   531  	dom *domain.Domain) error {
   532  	for {
   533  		item, err := puller.PullOneEvent(ctx)
   534  		if err != nil {
   535  			return errors.Trace(err)
   536  		}
   537  		if item == nil {
   538  			log.Info("[restoreFromPuller] nothing in this puller, we should stop and flush",
   539  				zap.Int64("table id", tableID))
   540  			err = l.applyKVChanges(ctx, tableID)
   541  			if err != nil {
   542  				return errors.Trace(err)
   543  			}
   544  			return nil
   545  		}
   546  		log.Debug("[restoreFromPuller] next event", zap.Any("item", item), zap.Int64("table id", tableID))
   547  		if l.startTS > item.TS {
   548  			log.Debug("[restoreFromPuller] item ts is smaller than start ts, skip this item",
   549  				zap.Uint64("start ts", l.startTS),
   550  				zap.Uint64("end ts", l.endTS),
   551  				zap.Uint64("item ts", item.TS),
   552  				zap.Int64("table id", tableID))
   553  			continue
   554  		}
   555  		if l.endTS < item.TS {
   556  			log.Warn("[restoreFromPuller] ts is larger than end ts, we should stop and flush",
   557  				zap.Uint64("start ts", l.startTS),
   558  				zap.Uint64("end ts", l.endTS),
   559  				zap.Uint64("item ts", item.TS),
   560  				zap.Int64("table id", tableID))
   561  			err = l.applyKVChanges(ctx, tableID)
   562  			if err != nil {
   563  				return errors.Trace(err)
   564  			}
   565  			return nil
   566  		}
   567  
   568  		if l.shouldFilter(item) {
   569  			log.Debug("[restoreFromPuller] filter item because later drop schema will affect on this item",
   570  				zap.Any("item", item),
   571  				zap.Int64("table id", tableID))
   572  			err = l.applyKVChanges(ctx, tableID)
   573  			if err != nil {
   574  				return errors.Trace(err)
   575  			}
   576  			continue
   577  		}
   578  
   579  		switch item.ItemType {
   580  		case cdclog.DDL:
   581  			name := l.meta.Names[tableID]
   582  			schema, table := ParseQuoteName(name)
   583  			ddl := item.Data.(*cdclog.MessageDDL)
   584  			// ddl not influence on this schema/table
   585  			if !(schema == item.Schema && (table == item.Table || l.isDBRelatedDDL(ddl))) {
   586  				log.Info("[restoreFromPuller] meet unrelated ddl, and continue pulling",
   587  					zap.String("item table", item.Table),
   588  					zap.String("table", table),
   589  					zap.String("item schema", item.Schema),
   590  					zap.String("schema", schema),
   591  					zap.Int64("backup table id", tableID),
   592  					zap.String("query", ddl.Query),
   593  					zap.Int64("table id", tableID))
   594  				continue
   595  			}
   596  
   597  			// database level ddl job has been executed at the beginning
   598  			if l.isDBRelatedDDL(ddl) {
   599  				log.Debug("[restoreFromPuller] meet database level ddl, continue pulling",
   600  					zap.String("ddl", ddl.Query),
   601  					zap.Int64("table id", tableID))
   602  				continue
   603  			}
   604  
   605  			// wait all previous kvs ingest finished
   606  			err = l.applyKVChanges(ctx, tableID)
   607  			if err != nil {
   608  				return errors.Trace(err)
   609  			}
   610  
   611  			log.Debug("[restoreFromPuller] execute ddl", zap.String("ddl", ddl.Query))
   612  
   613  			l.ddlLock.Lock()
   614  			err = l.restoreClient.db.se.Execute(ctx, fmt.Sprintf("use %s", item.Schema))
   615  			if err != nil {
   616  				return errors.Trace(err)
   617  			}
   618  
   619  			err = l.restoreClient.db.se.Execute(ctx, ddl.Query)
   620  			if err != nil {
   621  				return errors.Trace(err)
   622  			}
   623  			l.ddlLock.Unlock()
   624  
   625  			// if table dropped, we will pull next event to see if this table will create again.
   626  			// with next create table ddl, we can do reloadTableMeta.
   627  			if l.isDropTable(ddl) {
   628  				log.Info("[restoreFromPuller] skip reload because this is a drop table ddl",
   629  					zap.String("ddl", ddl.Query))
   630  				l.tableBuffers[tableID].ResetTableInfo()
   631  				continue
   632  			}
   633  			err = l.reloadTableMeta(dom, tableID, item)
   634  			if err != nil {
   635  				return errors.Trace(err)
   636  			}
   637  		case cdclog.RowChanged:
   638  			if l.tableBuffers[tableID].TableInfo() == nil {
   639  				err = l.reloadTableMeta(dom, tableID, item)
   640  				if err != nil {
   641  					// shouldn't happen
   642  					return errors.Trace(err)
   643  				}
   644  			}
   645  			err = l.tableBuffers[tableID].Append(item)
   646  			if err != nil {
   647  				return errors.Trace(err)
   648  			}
   649  			if l.tableBuffers[tableID].ShouldApply() {
   650  				err = l.applyKVChanges(ctx, tableID)
   651  				if err != nil {
   652  					return errors.Trace(err)
   653  				}
   654  			}
   655  		}
   656  	}
   657  }
   658  
   659  func (l *LogClient) restoreTables(ctx context.Context, dom *domain.Domain) error {
   660  	// 1. decode cdclog with in ts range
   661  	// 2. dispatch cdclog events to table level concurrently
   662  	// 		a. encode row changed files to kvpairs and ingest into tikv
   663  	// 		b. exec ddl
   664  	log.Debug("start restore tables")
   665  	workerPool := utils.NewWorkerPool(l.concurrencyCfg.Concurrency, "table log restore")
   666  	eg, ectx := errgroup.WithContext(ctx)
   667  	for tableID, puller := range l.eventPullers {
   668  		pullerReplica := puller
   669  		tableIDReplica := tableID
   670  		workerPool.ApplyOnErrorGroup(eg, func() error {
   671  			return l.restoreTableFromPuller(ectx, tableIDReplica, pullerReplica, dom)
   672  		})
   673  	}
   674  	return eg.Wait()
   675  }
   676  
   677  // RestoreLogData restore specify log data from storage.
   678  func (l *LogClient) RestoreLogData(ctx context.Context, dom *domain.Domain) error {
   679  	// 1. Retrieve log data from storage
   680  	// 2. Find proper data by TS range
   681  	// 3. Encode and ingest data to tikv
   682  
   683  	// parse meta file
   684  	data, err := l.restoreClient.storage.ReadFile(ctx, metaFile)
   685  	if err != nil {
   686  		return errors.Trace(err)
   687  	}
   688  	err = json.Unmarshal(data, l.meta)
   689  	if err != nil {
   690  		return errors.Trace(err)
   691  	}
   692  	log.Info("get meta from storage", zap.Binary("data", data))
   693  
   694  	if l.startTS > l.meta.GlobalResolvedTS {
   695  		return errors.Annotatef(berrors.ErrRestoreRTsConstrain,
   696  			"start ts:%d is greater than resolved ts:%d", l.startTS, l.meta.GlobalResolvedTS)
   697  	}
   698  	if l.endTS > l.meta.GlobalResolvedTS {
   699  		log.Info("end ts is greater than resolved ts,"+
   700  			" to keep consistency we only recover data until resolved ts",
   701  			zap.Uint64("end ts", l.endTS),
   702  			zap.Uint64("resolved ts", l.meta.GlobalResolvedTS))
   703  		l.endTS = l.meta.GlobalResolvedTS
   704  	}
   705  
   706  	// collect ddl files
   707  	ddlFiles, err := l.collectDDLFiles(ctx)
   708  	if err != nil {
   709  		return errors.Trace(err)
   710  	}
   711  
   712  	log.Info("collect ddl files", zap.Any("files", ddlFiles))
   713  
   714  	err = l.doDBDDLJob(ctx, ddlFiles)
   715  	if err != nil {
   716  		return errors.Trace(err)
   717  	}
   718  	log.Debug("db level ddl executed")
   719  
   720  	// collect row change files
   721  	rowChangesFiles, err := l.collectRowChangeFiles(ctx)
   722  	if err != nil {
   723  		return errors.Trace(err)
   724  	}
   725  
   726  	log.Info("collect row changed files", zap.Any("files", rowChangesFiles))
   727  
   728  	// create event puller to apply changes concurrently
   729  	for tableID, files := range rowChangesFiles {
   730  		name := l.meta.Names[tableID]
   731  		schema, table := ParseQuoteName(name)
   732  		log.Info("create puller for table",
   733  			zap.Int64("table id", tableID),
   734  			zap.String("schema", schema),
   735  			zap.String("table", table),
   736  		)
   737  		l.eventPullers[tableID], err = cdclog.NewEventPuller(ctx, schema, table, ddlFiles, files, l.restoreClient.storage)
   738  		if err != nil {
   739  			return errors.Trace(err)
   740  		}
   741  		// use table name to get table info
   742  		var tableInfo titable.Table
   743  		var allocs autoid.Allocators
   744  		infoSchema := dom.InfoSchema()
   745  		if infoSchema.TableExists(model.NewCIStr(schema), model.NewCIStr(table)) {
   746  			tableInfo, err = infoSchema.TableByName(model.NewCIStr(schema), model.NewCIStr(table))
   747  			if err != nil {
   748  				return errors.Trace(err)
   749  			}
   750  			dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(schema))
   751  			if !ok {
   752  				return errors.Annotatef(berrors.ErrRestoreSchemaNotExists, "schema %s", schema)
   753  			}
   754  			allocs = autoid.NewAllocatorsFromTblInfo(dom.Store(), dbInfo.ID, tableInfo.Meta())
   755  		}
   756  
   757  		l.tableBuffers[tableID] = cdclog.NewTableBuffer(tableInfo, allocs,
   758  			l.concurrencyCfg.BatchFlushKVPairs, l.concurrencyCfg.BatchFlushKVSize)
   759  	}
   760  	// restore files
   761  	return l.restoreTables(ctx, dom)
   762  }