Skip to content

Commit 6940958

Browse files
committed
fix sql, some rename
Signed-off-by: Wenqi Mou <[email protected]>
1 parent b6c7dea commit 6940958

File tree

3 files changed

+16
-19
lines changed

3 files changed

+16
-19
lines changed

br/pkg/restore/log_client/client.go

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -827,6 +827,7 @@ func (rc *LogClient) RestoreKVFiles(
827827
}
828828
}()
829829

830+
log.Info("starting to restore kv files")
830831
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
831832
span1 := span.Tracer().StartSpan("Client.RestoreKVFiles", opentracing.ChildOf(span.Context()))
832833
defer span1.Finish()
@@ -843,10 +844,6 @@ func (rc *LogClient) RestoreKVFiles(
843844
// because the tableID of files is the same.
844845
rule, ok := rules[files[0].TableId]
845846
if !ok {
846-
// TODO handle new created table
847-
// For this version we do not handle new created table after full backup.
848-
// in next version we will perform rewrite and restore meta key to restore new created tables.
849-
// so we can simply skip the file that doesn't have the rule here.
850847
onProgress(kvCount)
851848
summary.CollectInt("FileSkip", len(files))
852849
log.Debug("skip file due to table id not matched", zap.Int64("table-id", files[0].TableId))
@@ -1502,7 +1499,7 @@ func (rc *LogClient) WrapCompactedFilesIterWithSplitHelper(
15021499
return wrapper.WithSplit(ctx, compactedIter, strategy), nil
15031500
}
15041501

1505-
// WrapLogFilesIteratorWithSplit applies a splitting strategy to the log files iterator.
1502+
// WrapLogFilesIterWithSplitHelper applies a splitting strategy to the log files iterator.
15061503
// It uses a region splitter to handle the splitting logic based on the provided rules.
15071504
func (rc *LogClient) WrapLogFilesIterWithSplitHelper(
15081505
ctx context.Context,

br/pkg/task/restore.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -538,7 +538,6 @@ func (cfg *RestoreConfig) adjustRestoreConfigForStreamRestore() {
538538
// another goroutine is used to iterate the backup file
539539
cfg.PitrConcurrency += 1
540540
log.Info("set restore kv files concurrency", zap.Int("concurrency", int(cfg.PitrConcurrency)))
541-
cfg.Config.Concurrency = cfg.PitrConcurrency
542541
if cfg.ConcurrencyPerStore.Value > 0 {
543542
log.Info("set restore compacted sst files concurrency per store",
544543
zap.Int("concurrency", int(cfg.ConcurrencyPerStore.Value)))

br/pkg/task/stream.go

Lines changed: 14 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -483,11 +483,11 @@ func (s *streamMgr) checkStreamStartEnable(ctx context.Context) error {
483483
return nil
484484
}
485485

486-
type RestoreGcFunc func(string) error
486+
type RestoreGCFunc func(string) error
487487

488488
// DisableGC disables and returns a function that can enable gc back.
489489
// gc.ratio-threshold = "-1.0", which represents disable gc in TiKV.
490-
func DisableGC(g glue.Glue, store kv.Storage) (RestoreGcFunc, string, error) {
490+
func DisableGC(g glue.Glue, store kv.Storage) (RestoreGCFunc, string, error) {
491491
se, err := g.CreateSession(store)
492492
if err != nil {
493493
return nil, "", errors.Trace(err)
@@ -1328,6 +1328,8 @@ func RunStreamRestore(
13281328
failpoint.Return(errors.New("failpoint: failed before full restore"))
13291329
})
13301330

1331+
// restore log.
1332+
cfg.adjustRestoreConfigForStreamRestore()
13311333
cfg.tiflashRecorder = tiflashrec.New()
13321334
logClient, err := createLogClient(ctx, g, cfg, mgr)
13331335
if err != nil {
@@ -1385,8 +1387,6 @@ func RunStreamRestore(
13851387
cfg.tiflashRecorder.Load(taskInfo.CheckpointInfo.Metadata.TiFlashItems)
13861388
}
13871389
}
1388-
// restore log.
1389-
cfg.adjustRestoreConfigForStreamRestore()
13901390
logRestoreConfig := &LogRestoreConfig{
13911391
RestoreConfig: cfg,
13921392
checkpointTaskInfo: taskInfo.CheckpointInfo,
@@ -1495,7 +1495,7 @@ func restoreStream(
14951495

14961496
// It need disable GC in TiKV when PiTR.
14971497
// because the process of PITR is concurrent and kv events isn't sorted by tso.
1498-
restoreGcFunc, oldGcRatio, err := DisableGC(g, mgr.GetStorage())
1498+
restoreGCFunc, oldGCRatio, err := DisableGC(g, mgr.GetStorage())
14991499
if err != nil {
15001500
return errors.Trace(err)
15011501
}
@@ -1509,24 +1509,24 @@ func restoreStream(
15091509

15101510
// If the oldGcRatio is negative, which is not normal status.
15111511
// It should set default value "1.1" after PiTR finished.
1512-
if strings.HasPrefix(oldGcRatio, "-") {
1513-
log.Warn("the original gc-ratio is negative, reset by default value 1.1", zap.String("old-gc-ratio", oldGcRatio))
1514-
oldGcRatio = utils.DefaultGcRatioVal
1512+
if strings.HasPrefix(oldGCRatio, "-") {
1513+
log.Warn("the original gc-ratio is negative, reset by default value 1.1", zap.String("old-gc-ratio", oldGCRatio))
1514+
oldGCRatio = utils.DefaultGcRatioVal
15151515
}
1516-
log.Info("start to restore gc", zap.String("ratio", oldGcRatio))
1517-
if err := restoreGcFunc(oldGcRatio); err != nil {
1516+
log.Info("start to restore gc", zap.String("ratio", oldGCRatio))
1517+
if err := restoreGCFunc(oldGCRatio); err != nil {
15181518
log.Error("failed to restore gc", zap.Error(err))
15191519
}
15201520
log.Info("finish restoring gc")
15211521
}()
15221522

15231523
var sstCheckpointSets map[string]struct{}
15241524
if cfg.UseCheckpoint {
1525-
gcRatioFromCheckpoint, err := client.LoadOrCreateCheckpointMetadataForLogRestore(ctx, cfg.StartTS, cfg.RestoreTS, oldGcRatio, cfg.tiflashRecorder)
1525+
gcRatioFromCheckpoint, err := client.LoadOrCreateCheckpointMetadataForLogRestore(ctx, cfg.StartTS, cfg.RestoreTS, oldGCRatio, cfg.tiflashRecorder)
15261526
if err != nil {
15271527
return errors.Trace(err)
15281528
}
1529-
oldGcRatio = gcRatioFromCheckpoint
1529+
oldGCRatio = gcRatioFromCheckpoint
15301530
sstCheckpointSets, err = client.InitCheckpointMetadataForCompactedSstRestore(ctx)
15311531
if err != nil {
15321532
return errors.Trace(err)
@@ -1734,7 +1734,8 @@ func createLogClient(ctx context.Context, g glue.Glue, cfg *RestoreConfig, mgr *
17341734
}
17351735
return nil, nil
17361736
}
1737-
err = client.InitClients(ctx, u, createCheckpointSessionFn, uint(cfg.Concurrency), cfg.ConcurrencyPerStore.Value)
1737+
1738+
err = client.InitClients(ctx, u, createCheckpointSessionFn, uint(cfg.PitrConcurrency), cfg.ConcurrencyPerStore.Value)
17381739
if err != nil {
17391740
return nil, errors.Trace(err)
17401741
}

0 commit comments

Comments
 (0)