Skip to content

Commit

Permalink
*: Move tikv gc configuration to sysvars (#21988)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo committed Jan 7, 2021
1 parent 18d3135 commit ec73315
Show file tree
Hide file tree
Showing 8 changed files with 235 additions and 24 deletions.
5 changes: 2 additions & 3 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -650,9 +650,8 @@ func (s *testSerialSuite) TestRecoverTableByJobID(c *C) {
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))

// if GC enable is not exists in mysql.tidb
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")
tk.MustExec(fmt.Sprintf("recover table by job %d", jobID))
tk.MustExec("DROP TABLE t_recover")

err = gcutil.EnableGC(tk.Se)
c.Assert(err, IsNil)
Expand Down
7 changes: 3 additions & 4 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5376,10 +5376,9 @@ func (s *testRecoverTable) TestRecoverTable(c *C) {
// set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))

// if GC enable is not exists in mysql.tidb
_, err = tk.Exec("recover table t_recover")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")
// Should recover, and we can drop it straight away.
tk.MustExec("recover table t_recover")
tk.MustExec("drop table t_recover")

err = gcutil.EnableGC(tk.Se)
c.Assert(err, IsNil)
Expand Down
4 changes: 4 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -510,6 +510,10 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustQuery("select @@global.tidb_enable_extended_stats").Check(testkit.Rows("1"))
tk.MustExec("SET GLOBAL tidb_enable_extended_stats = off")
tk.MustQuery("select @@global.tidb_enable_extended_stats").Check(testkit.Rows("0"))

// Test issue #22145
tk.MustExec(`set global sync_relay_log = "'"`)

}

func (s *testSuite5) TestTruncateIncorrectIntSessionVar(c *C) {
Expand Down
139 changes: 137 additions & 2 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,27 @@ var (
sessionExecuteCompileDurationGeneral = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblGeneral)
sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal)
sessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral)

tiKVGCAutoConcurrency = "tikv_gc_auto_concurrency"
)

var gcVariableComments = map[string]string{
variable.TiDBGCRunInterval: "GC run interval, at least 10m, in Go format.",
variable.TiDBGCLifetime: "All versions within life time will not be collected by GC, at least 10m, in Go format.",
variable.TiDBGCConcurrency: "How many goroutines used to do GC parallel, [1, 128], default 2",
variable.TiDBGCEnable: "Current GC enable status",
tiKVGCAutoConcurrency: "Let TiDB pick the concurrency automatically. If set false, tikv_gc_concurrency will be used",
variable.TiDBGCScanLockMode: "Mode of scanning locks, \"physical\" or \"legacy\"",
}

var gcVariableMap = map[string]string{
variable.TiDBGCRunInterval: "tikv_gc_run_interval",
variable.TiDBGCLifetime: "tikv_gc_life_time",
variable.TiDBGCConcurrency: "tikv_gc_concurrency",
variable.TiDBGCEnable: "tikv_gc_enable",
variable.TiDBGCScanLockMode: "tikv_gc_scan_lock_mode",
}

// Session context, it is consistent with the lifecycle of a client connection.
type Session interface {
sessionctx.Context
Expand Down Expand Up @@ -988,6 +1007,14 @@ func (s *session) getExecRet(ctx sessionctx.Context, sql string) (string, error)
return value, nil
}

func (s *session) varFromTiDBTable(name string) bool {
switch name {
case variable.TiDBGCConcurrency, variable.TiDBGCEnable, variable.TiDBGCRunInterval, variable.TiDBGCLifetime, variable.TiDBGCScanLockMode:
return true
}
return false
}

// GetAllSysVars implements GlobalVarAccessor.GetAllSysVars interface.
func (s *session) GetAllSysVars() (map[string]string, error) {
if s.Value(sessionctx.Initing) != nil {
Expand All @@ -1002,7 +1029,13 @@ func (s *session) GetAllSysVars() (map[string]string, error) {
ret := make(map[string]string, len(rows))
for _, r := range rows {
k, v := r.GetString(0), r.GetString(1)
ret[k] = v
if s.varFromTiDBTable(k) {
if v, err = s.getTiDBTableValue(k, v); err != nil {
ret[k] = v
}
} else {
ret[k] = v
}
}
return ret, nil
}
Expand All @@ -1029,6 +1062,10 @@ func (s *session) GetGlobalSysVar(name string) (string, error) {
}
return "", err
}
// Fetch mysql.tidb values if required
if s.varFromTiDBTable(name) {
return s.getTiDBTableValue(name, sysVar)
}
return sysVar, nil
}

Expand Down Expand Up @@ -1056,13 +1093,111 @@ func (s *session) SetGlobalSysVar(name, value string) error {
return err
}
name = strings.ToLower(name)
// update mysql.tidb if required.
if s.varFromTiDBTable(name) {
if err = s.setTiDBTableValue(name, sVal); err != nil {
return err
}
}
variable.CheckDeprecationSetSystemVar(s.sessionVars, name)
sql := fmt.Sprintf(`REPLACE %s.%s VALUES ('%s', '%s');`,
mysql.SystemDB, mysql.GlobalVariablesTable, name, sVal)
mysql.SystemDB, mysql.GlobalVariablesTable, name, escapeUserString(sVal))
_, _, err = s.ExecRestrictedSQL(sql)
return err
}

// escape user supplied string for internal SQL. Not safe for all cases, since it doesn't
// handle quote-type, sql-mode, character set breakout.
func escapeUserString(str string) string {
return strings.ReplaceAll(str, `'`, `\'`)
}

// setTiDBTableValue handles tikv_* sysvars which need to update mysql.tidb
// for backwards compatibility. Validation has already been performed.
func (s *session) setTiDBTableValue(name, val string) error {
if name == variable.TiDBGCConcurrency {
autoConcurrency := "false"
if val == "-1" {
autoConcurrency = "true"
}
sql := fmt.Sprintf(`INSERT INTO mysql.tidb (variable_name, variable_value, comment) VALUES ('%[1]s', '%[2]s', '%[3]s')
ON DUPLICATE KEY UPDATE variable_value = '%[2]s'`, tiKVGCAutoConcurrency, autoConcurrency, gcVariableComments[name])
_, _, err := s.ExecRestrictedSQL(sql)
if err != nil {
return err
}
}
val = onOffToTrueFalse(val)
sql := fmt.Sprintf(`INSERT INTO mysql.tidb (variable_name, variable_value, comment) VALUES ('%[1]s', '%[2]s', '%[3]s')
ON DUPLICATE KEY UPDATE variable_value = '%[2]s'`, gcVariableMap[name], escapeUserString(val), gcVariableComments[name])
_, _, err := s.ExecRestrictedSQL(sql)
return err
}

// In mysql.tidb the convention has been to store the string value "true"/"false",
// but sysvars use the convention ON/OFF.
func trueFalseToOnOff(str string) string {
if strings.EqualFold("true", str) {
return variable.BoolOn
} else if strings.EqualFold("false", str) {
return variable.BoolOff
}
return str
}

// In mysql.tidb the convention has been to store the string value "true"/"false",
// but sysvars use the convention ON/OFF.
func onOffToTrueFalse(str string) string {
if strings.EqualFold("ON", str) {
return "true"
} else if strings.EqualFold("OFF", str) {
return "false"
}
return str
}

// getTiDBTableValue handles tikv_* sysvars which need
// to read from mysql.tidb for backwards compatibility.
func (s *session) getTiDBTableValue(name, val string) (string, error) {
if name == variable.TiDBGCConcurrency {
// Check if autoconcurrency is set
sql := fmt.Sprintf(`SELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME='%s';`, tiKVGCAutoConcurrency)
autoConcurrencyVal, err := s.getExecRet(s, sql)
if err == nil && strings.EqualFold(autoConcurrencyVal, "true") {
return "-1", nil // convention for "AUTO"
}
}
sql := fmt.Sprintf(`SELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME='%s';`, gcVariableMap[name])
tblValue, err := s.getExecRet(s, sql)
if err != nil {
return val, nil // mysql.tidb value does not exist.
}
// Run validation on the tblValue. This will return an error if it can't be validated,
// but will also make it more consistent: disTribuTeD -> DISTRIBUTED etc
tblValue = trueFalseToOnOff(tblValue)
validatedVal, err := variable.ValidateSetSystemVar(s.sessionVars, name, tblValue, variable.ScopeGlobal)
if err != nil {
logutil.Logger(context.Background()).Warn("restoring sysvar value since validating mysql.tidb value failed",
zap.Error(err),
zap.String("name", name),
zap.String("tblName", gcVariableMap[name]),
zap.String("tblValue", tblValue),
zap.String("restoredValue", val))
sql := fmt.Sprintf(`REPLACE INTO mysql.tidb (variable_name, variable_value, comment)
VALUES ('%s', '%s', '%s')`, gcVariableMap[name], escapeUserString(val), gcVariableComments[name])
_, _, err = s.ExecRestrictedSQL(sql)
return val, err
}
if validatedVal != val {
// The sysvar value is out of sync.
sql := fmt.Sprintf(`REPLACE %s.%s VALUES ('%s', '%s');`,
mysql.SystemDB, mysql.GlobalVariablesTable, gcVariableMap[name], escapeUserString(validatedVal))
_, _, err = s.ExecRestrictedSQL(sql)
return validatedVal, err
}
return validatedVal, nil
}

func (s *session) ensureFullGlobalStats() error {
rows, _, err := s.ExecRestrictedSQL(`select count(1) from information_schema.tables t where t.create_options = 'partitioned'
and not exists (select 1 from mysql.stats_meta m where m.table_id = t.tidb_table_id)`)
Expand Down
43 changes: 43 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3866,6 +3866,49 @@ func (s *testSessionSerialSuite) TestIssue21943(c *C) {
c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_plan_from_cache' is a read only variable")
}

func (s *testSessionSerialSuite) TestTiKVSystemVars(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_enable'") // default is on from the sysvar
result.Check(testkit.Rows("tidb_gc_enable ON"))
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'")
result.Check(testkit.Rows()) // but no value in the table (yet) because the value has not been set and the GC has never been run

// update will set a value in the table
tk.MustExec("SET GLOBAL tidb_gc_enable = 1")
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'")
result.Check(testkit.Rows("true"))

tk.MustExec("UPDATE mysql.tidb SET variable_value = 'false' WHERE variable_name='tikv_gc_enable'")
result = tk.MustQuery("SELECT @@tidb_gc_enable;")
result.Check(testkit.Rows("0")) // reads from mysql.tidb value and changes to false

tk.MustExec("SET GLOBAL tidb_gc_concurrency = -1") // sets auto concurrency and concurrency
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'")
result.Check(testkit.Rows("true"))
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'")
result.Check(testkit.Rows("-1"))

tk.MustExec("SET GLOBAL tidb_gc_concurrency = 5") // sets auto concurrency and concurrency
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'")
result.Check(testkit.Rows("false"))
result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'")
result.Check(testkit.Rows("5"))

tk.MustExec("UPDATE mysql.tidb SET variable_value = 'true' WHERE variable_name='tikv_gc_auto_concurrency'")
result = tk.MustQuery("SELECT @@tidb_gc_concurrency;")
result.Check(testkit.Rows("-1")) // because auto_concurrency is turned on it takes precedence

_, err := tk.Exec("SET GLOBAL tidb_gc_run_interval = '9m'") // too small
c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_gc_run_interval'")

tk.MustExec("SET GLOBAL tidb_gc_run_interval = '700000000000ns'") // specified in ns, also valid

_, err = tk.Exec("SET GLOBAL tidb_gc_run_interval = '11mins'")
c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_gc_run_interval'") // wrong format

}

func (s *testSessionSerialSuite) TestProcessInfoIssue22068(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
27 changes: 27 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,8 @@ const (
TypeUnsigned TypeFlag = 5
// TypeTime for time of day (a TiDB extension)
TypeTime TypeFlag = 6
// TypeDuration for a golang duration (a TiDB extension)
TypeDuration TypeFlag = 7

// BoolOff is the canonical string representation of a boolean false.
BoolOff = "OFF"
Expand Down Expand Up @@ -136,6 +138,8 @@ func (sv *SysVar) ValidateFromType(vars *SessionVars, value string, scope ScopeF
return sv.checkEnumSystemVar(value, vars)
case TypeTime:
return sv.checkTimeSystemVar(value, vars)
case TypeDuration:
return sv.checkDurationSystemVar(value, vars)
}
return value, nil // typeString
}
Expand All @@ -160,6 +164,22 @@ func (sv *SysVar) checkTimeSystemVar(value string, vars *SessionVars) (string, e
return t.Format(FullDayTimeFormat), nil
}

func (sv *SysVar) checkDurationSystemVar(value string, vars *SessionVars) (string, error) {
d, err := time.ParseDuration(value)
if err != nil {
return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name)
}
// Check for min/max violations
if int64(d) < sv.MinValue {
return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name)
}
if uint64(d) > sv.MaxValue {
return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name)
}
// return a string representation of the duration
return d.String(), nil
}

func (sv *SysVar) checkUInt64SystemVar(value string, vars *SessionVars) (string, error) {
if sv.AllowAutoValue && value == "-1" {
return value, nil
Expand Down Expand Up @@ -712,6 +732,13 @@ var defaultSysVars = []*SysVar{
{Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMergeJoin, Value: BoolToOnOff(DefTiDBEnableIndexMergeJoin), Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBTrackAggregateMemoryUsage, Value: BoolToOnOff(DefTiDBTrackAggregateMemoryUsage), Type: TypeBool},

/* tikv gc metrics */
{Scope: ScopeGlobal, Name: TiDBGCEnable, Value: BoolOn, Type: TypeBool},
{Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64},
{Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64},
{Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true},
{Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "PHYSICAL", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}},
}

// SynonymsSysVariables is synonyms of system variables.
Expand Down
15 changes: 15 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -518,6 +518,21 @@ const (
TiDBTrackAggregateMemoryUsage = "tidb_track_aggregate_memory_usage"
)

// TiDB vars that have only global scope

const (
// TiDBGCEnable turns garbage collection on or OFF
TiDBGCEnable = "tidb_gc_enable"
// TiDBGCRunInterval sets the interval that GC runs
TiDBGCRunInterval = "tidb_gc_run_interval"
// TiDBGCLifetime sets the retention window of older versions
TiDBGCLifetime = "tidb_gc_life_time"
// TiDBGCConcurrency sets the concurrency of garbage collection. -1 = AUTO value
TiDBGCConcurrency = "tidb_gc_concurrency"
// TiDBGCScanLockMode enables the green GC feature (default)
TiDBGCScanLockMode = "tidb_gc_scan_lock_mode"
)

// Default TiDB system variable values.
const (
DefHostname = "localhost"
Expand Down
19 changes: 4 additions & 15 deletions util/gcutil/gcutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,36 +26,25 @@ import (

const (
selectVariableValueSQL = `SELECT HIGH_PRIORITY variable_value FROM mysql.tidb WHERE variable_name='%s'`
insertVariableValueSQL = `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s')
ON DUPLICATE KEY
UPDATE variable_value = '%[2]s', comment = '%[3]s'`
)

// CheckGCEnable is use to check whether GC is enable.
func CheckGCEnable(ctx sessionctx.Context) (enable bool, err error) {
sql := fmt.Sprintf(selectVariableValueSQL, "tikv_gc_enable")
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql)
val, err := ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBGCEnable)
if err != nil {
return false, errors.Trace(err)
}
if len(rows) != 1 {
return false, errors.New("can not get 'tikv_gc_enable'")
}
return rows[0].GetString(0) == "true", nil
return variable.TiDBOptOn(val), nil
}

// DisableGC will disable GC enable variable.
func DisableGC(ctx sessionctx.Context) error {
sql := fmt.Sprintf(insertVariableValueSQL, "tikv_gc_enable", "false", "Current GC enable status")
_, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql)
return errors.Trace(err)
return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.BoolOff)
}

// EnableGC will enable GC enable variable.
func EnableGC(ctx sessionctx.Context) error {
sql := fmt.Sprintf(insertVariableValueSQL, "tikv_gc_enable", "true", "Current GC enable status")
_, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql)
return errors.Trace(err)
return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.BoolOn)
}

// ValidateSnapshot checks that the newly set snapshot time is after GC safe point time.
Expand Down

0 comments on commit ec73315

Please sign in to comment.