Skip to content

Commit

Permalink
log-backup: set gc disable when restore log (#39729) (#39763)
Browse files Browse the repository at this point in the history
close #39602
  • Loading branch information
ti-chi-bot authored Dec 8, 2022
1 parent 1bef7eb commit cc36829
Show file tree
Hide file tree
Showing 4 changed files with 129 additions and 3 deletions.
4 changes: 2 additions & 2 deletions br/pkg/restore/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -2064,9 +2064,9 @@ func (rc *Client) RestoreKVFiles(
}

if supportBatch {
err = ApplyKVFilesWithBatchMethod(ctx, iter, int(pitrBatchCount), uint64(pitrBatchSize), applyFunc)
err = ApplyKVFilesWithBatchMethod(ectx, iter, int(pitrBatchCount), uint64(pitrBatchSize), applyFunc)
} else {
err = ApplyKVFilesWithSingelMethod(ctx, iter, applyFunc)
err = ApplyKVFilesWithSingelMethod(ectx, iter, applyFunc)
}
if err != nil {
return errors.Trace(err)
Expand Down
39 changes: 39 additions & 0 deletions br/pkg/task/stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -458,6 +458,33 @@ func (s *streamMgr) checkStreamStartEnable(g glue.Glue) error {
return nil
}

type RestoreFunc func() error

// KeepGcDisabled keeps GC disabled and return a function that used to gc enabled.
// gc.ratio-threshold = "-1.0", which represents disable gc in TiKV.
func KeepGcDisabled(g glue.Glue, store kv.Storage) (RestoreFunc, error) {
se, err := g.CreateSession(store)
if err != nil {
return nil, errors.Trace(err)
}

execCtx := se.GetSessionCtx().(sqlexec.RestrictedSQLExecutor)
oldRatio, err := utils.GetGcRatio(execCtx)
if err != nil {
return nil, errors.Trace(err)
}

newRatio := "-1.0"
err = utils.SetGcRatio(execCtx, newRatio)
if err != nil {
return nil, errors.Trace(err)
}

return func() error {
return utils.SetGcRatio(execCtx, oldRatio)
}, nil
}

// RunStreamCommand run all kinds of `stream task`
func RunStreamCommand(
ctx context.Context,
Expand Down Expand Up @@ -1143,6 +1170,18 @@ func restoreStream(
// mode or emptied schedulers
defer restorePostWork(ctx, client, restoreSchedulers)

// It need disable GC in TiKV when PiTR.
// because the process of PITR is concurrent and kv events isn't sorted by tso.
restoreGc, err := KeepGcDisabled(g, mgr.GetStorage())
if err != nil {
return errors.Trace(err)
}
defer func() {
if err = restoreGc(); err != nil {
log.Error("failed to set gc enabled", zap.Error(err))
}
}()

err = client.InstallLogFileManager(ctx, cfg.StartTS, cfg.RestoreTS)
if err != nil {
return err
Expand Down
34 changes: 34 additions & 0 deletions br/pkg/utils/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"strings"
"sync"

"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -98,6 +99,39 @@ func IsLogBackupEnabled(ctx sqlexec.RestrictedSQLExecutor) (bool, error) {
return true, nil
}

func GetGcRatio(ctx sqlexec.RestrictedSQLExecutor) (string, error) {
valStr := "show config where name = 'gc.ratio-threshold' and type = 'tikv'"
rows, fields, errSQL := ctx.ExecRestrictedSQL(
kv.WithInternalSourceType(context.Background(), kv.InternalTxnBR),
nil,
valStr,
)
if errSQL != nil {
return "", errSQL
}
if len(rows) == 0 {
// no rows mean not support log backup.
return "", nil
}

d := rows[0].GetDatum(3, &fields[3].Column.FieldType)
return d.ToString()
}

func SetGcRatio(ctx sqlexec.RestrictedSQLExecutor, ratio string) error {
_, _, err := ctx.ExecRestrictedSQL(
kv.WithInternalSourceType(context.Background(), kv.InternalTxnBR),
nil,
"set config tikv `gc.ratio-threshold`=%?",
ratio,
)
if err != nil {
return errors.Annotatef(err, "failed to set config `gc.ratio-threshold`=%s", ratio)
}
log.Warn("set config tikv gc.ratio-threshold", zap.String("ratio", ratio))
return nil
}

// LogBackupTaskCountInc increases the count of log backup task.
func LogBackupTaskCountInc() {
LogBackupTaskMutex.Lock()
Expand Down
55 changes: 54 additions & 1 deletion br/pkg/utils/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ package utils_test

import (
"context"
"strings"
"testing"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -35,7 +36,19 @@ func (m *mockRestrictedSQLExecutor) ExecRestrictedSQL(ctx context.Context, opts
if m.errHappen {
return nil, nil, errors.New("injected error")
}
return m.rows, m.fields, nil

if strings.Contains(sql, "show config") {
return m.rows, m.fields, nil
} else if strings.Contains(sql, "set config") && strings.Contains(sql, "gc.ratio-threshold") {
value := args[0].(string)

for _, r := range m.rows {
d := types.Datum{}
d.SetString(value, "")
chunk.MutRow(r).SetDatum(3, d)
}
}
return nil, nil, nil
}

func TestIsLogBackupEnabled(t *testing.T) {
Expand Down Expand Up @@ -115,3 +128,43 @@ func TestCheckLogBackupTaskExist(t *testing.T) {
utils.LogBackupTaskCountDec()
require.False(t, utils.CheckLogBackupTaskExist())
}

func TestGc(t *testing.T) {
// config format:
// MySQL [(none)]> show config where name = 'gc.ratio-threshold';
// +------+-------------------+--------------------+-------+
// | Type | Instance | Name | Value |
// +------+-------------------+--------------------+-------+
// | tikv | 172.16.6.46:3460 | gc.ratio-threshold | 1.1 |
// | tikv | 172.16.6.47:3460 | gc.ratio-threshold | 1.1 |
// +------+-------------------+--------------------+-------+
fields := make([]*ast.ResultField, 4)
tps := []*types.FieldType{
types.NewFieldType(mysql.TypeString),
types.NewFieldType(mysql.TypeString),
types.NewFieldType(mysql.TypeString),
types.NewFieldType(mysql.TypeString),
}
for i := 0; i < len(tps); i++ {
rf := new(ast.ResultField)
rf.Column = new(model.ColumnInfo)
rf.Column.FieldType = *tps[i]
fields[i] = rf
}
rows := make([]chunk.Row, 0, 2)
row := chunk.MutRowFromValues("tikv", " 127.0.0.1:20161", "log-backup.enable", "1.1").ToRow()
rows = append(rows, row)
row = chunk.MutRowFromValues("tikv", " 127.0.0.1:20162", "log-backup.enable", "1.1").ToRow()
rows = append(rows, row)

s := &mockRestrictedSQLExecutor{rows: rows, fields: fields}
ratio, err := utils.GetGcRatio(s)
require.Nil(t, err)
require.Equal(t, ratio, "1.1")

err = utils.SetGcRatio(s, "-1.0")
require.Nil(t, err)
ratio, err = utils.GetGcRatio(s)
require.Nil(t, err)
require.Equal(t, ratio, "-1.0")
}

0 comments on commit cc36829

Please sign in to comment.