Skip to content

Commit

Permalink
sysvar: add INSTANCE scope for system variable (pingcap#32888)
Browse files Browse the repository at this point in the history
  • Loading branch information
CbcWestwolf authored Mar 16, 2022
1 parent c8637a8 commit d981c0e
Show file tree
Hide file tree
Showing 14 changed files with 183 additions and 11 deletions.
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1016,6 +1016,7 @@ const (
ErrInconsistentHandle = 8139
ErrInconsistentIndexedValue = 8140
ErrAssertionFailed = 8141
ErrInstanceScope = 8142

// Error codes used by TiDB ddl package
ErrUnsupportedDDLOperation = 8200
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1014,6 +1014,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrInconsistentHandle: mysql.Message("writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v", []int{2, 3, 4, 5}),
ErrInconsistentIndexedValue: mysql.Message("writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s}", []int{3, 4}),
ErrAssertionFailed: mysql.Message("assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v", []int{0}),
ErrInstanceScope: mysql.Message("modifying %s will require SET GLOBAL in a future version of TiDB", nil),

ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil),
ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1371,6 +1371,11 @@ error = '''
data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v
'''

["executor:8142"]
error = '''
modifying %s will require SET GLOBAL in a future version of TiDB
'''

["executor:8212"]
error = '''
Failed to split region ranges: %s
Expand Down
1 change: 1 addition & 0 deletions executor/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ var (
ErrIllegalPrivilegeLevel = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalPrivilegeLevel)
ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges)
ErrViewInvalid = dbterror.ClassExecutor.NewStd(mysql.ErrViewInvalid)
ErrInstanceScope = dbterror.ClassExecutor.NewStd(mysql.ErrInstanceScope)

ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed)
ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed)
Expand Down
8 changes: 8 additions & 0 deletions executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,14 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres
}
return variable.ErrUnknownSystemVar.GenWithStackByArgs(name)
}

if sysVar.HasInstanceScope() && !v.IsGlobal && sessionVars.EnableLegacyInstanceScope {
// For backward compatibility we will change the v.IsGlobal to true,
// and append a warning saying this will not be supported in future.
v.IsGlobal = true
sessionVars.StmtCtx.AppendWarning(ErrInstanceScope.GenWithStackByArgs(sysVar.Name))
}

if v.IsGlobal {
valStr, err := e.getVarValue(v, sysVar)
if err != nil {
Expand Down
27 changes: 22 additions & 5 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -416,19 +416,19 @@ func TestSetVar(t *testing.T) {
tk.MustQuery(`select @@global.tidb_enable_parallel_apply`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@tidb_enable_parallel_apply`).Check(testkit.Rows("1"))

tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF"))
tk.MustExec("set tidb_general_log = 1")
tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("1"))
tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("1"))
tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log ON"))
tk.MustExec("set tidb_general_log = 0")
tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF"))
tk.MustExec("set tidb_general_log = on")
tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("1"))
tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("1"))
tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log ON"))
tk.MustExec("set tidb_general_log = off")
tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0"))
tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF"))
require.Error(t, tk.ExecToErr("set tidb_general_log = abc"))
require.Error(t, tk.ExecToErr("set tidb_general_log = 123"))
Expand Down Expand Up @@ -803,6 +803,7 @@ func TestValidateSetVar(t *testing.T) {
tk.MustQuery("select @@tidb_constraint_check_in_place;").Check(testkit.Rows("1"))

tk.MustExec("set @@tidb_general_log=0;")
tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope)))
tk.MustQuery("select @@tidb_general_log;").Check(testkit.Rows("0"))

tk.MustExec("set @@tidb_pprof_sql_cpu=1;")
Expand Down Expand Up @@ -1574,3 +1575,19 @@ func TestSetTopSQLVariables(t *testing.T) {
tk.MustQuery("show variables like '%top_sql%'").Check(testkit.Rows("tidb_enable_top_sql OFF", "tidb_top_sql_max_meta_count 5000", "tidb_top_sql_max_time_series_count 20"))
tk.MustQuery("show global variables like '%top_sql%'").Check(testkit.Rows("tidb_enable_top_sql OFF", "tidb_top_sql_max_meta_count 5000", "tidb_top_sql_max_time_series_count 20"))
}

func TestInstanceScopeSwitching(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

// enable 'switching' to SESSION variables
tk.MustExec("set tidb_enable_legacy_instance_scope = 1")
tk.MustExec("set tidb_general_log = 1")
tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope)))

// disable 'switching' to SESSION variables
tk.MustExec("set tidb_enable_legacy_instance_scope = 0")
tk.MustGetErrCode("set tidb_general_log = 1", errno.ErrGlobalVariable)
}
2 changes: 1 addition & 1 deletion planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1291,7 +1291,7 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) {
er.b.visitInfo = appendDynamicVisitInfo(er.b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err)
}
if v.ExplicitScope && !sysVar.HasNoneScope() {
if v.IsGlobal && !sysVar.HasGlobalScope() {
if v.IsGlobal && !(sysVar.HasGlobalScope() || sysVar.HasInstanceScope()) {
er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION")
return
}
Expand Down
7 changes: 7 additions & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1331,6 +1331,7 @@ func (s *session) GetGlobalSysVar(name string) (string, error) {
}

// SetGlobalSysVar implements GlobalVarAccessor.SetGlobalSysVar interface.
// it is called (but skipped) when setting instance scope
func (s *session) SetGlobalSysVar(name, value string) (err error) {
sv := variable.GetSysVar(name)
if sv == nil {
Expand All @@ -1342,6 +1343,9 @@ func (s *session) SetGlobalSysVar(name, value string) (err error) {
if err = sv.SetGlobalFromHook(s.sessionVars, value, false); err != nil {
return err
}
if sv.HasInstanceScope() { // skip for INSTANCE scope
return nil
}
if sv.GlobalConfigName != "" {
domain.GetDomain(s).NotifyGlobalConfigChange(sv.GlobalConfigName, variable.OnOffToTrueFalse(value))
}
Expand All @@ -1358,6 +1362,9 @@ func (s *session) SetGlobalSysVarOnly(name, value string) (err error) {
if err = sv.SetGlobalFromHook(s.sessionVars, value, true); err != nil {
return err
}
if sv.HasInstanceScope() { // skip for INSTANCE scope
return nil
}
return s.replaceGlobalVariablesTableValue(context.TODO(), sv.Name, value)
}

Expand Down
51 changes: 51 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -764,6 +764,42 @@ func (s *testSessionSuite) TestUpgradeSysvars(c *C) {
c.Assert(v, Equals, "OFF") // the default value is restored.
}

func (s *testSessionSuite) TestSetInstanceSysvarBySetGlobalSysVar(c *C) {
varName := "tidb_general_log"
defaultValue := "OFF" // This is the default value for tidb_general_log

tk := testkit.NewTestKitWithInit(c, s.store)
se := tk.Se.(variable.GlobalVarAccessor)

// Get globalSysVar twice and get the same default value
v, err := se.GetGlobalSysVar(varName)
c.Assert(err, IsNil)
c.Assert(v, Equals, defaultValue)
v, err = se.GetGlobalSysVar(varName)
c.Assert(err, IsNil)
c.Assert(v, Equals, defaultValue)

// session.GetGlobalSysVar would not get the value which session.SetGlobalSysVar writes,
// because SetGlobalSysVar calls SetGlobalFromHook, which uses TiDBGeneralLog's SetGlobal,
// but GetGlobalSysVar could not access TiDBGeneralLog's GetGlobal.

// set to "1"
err = se.SetGlobalSysVar(varName, "ON")
c.Assert(err, IsNil)
v, err = se.GetGlobalSysVar(varName)
tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("1"))
c.Assert(err, IsNil)
c.Assert(v, Equals, defaultValue)

// set back to "0"
err = se.SetGlobalSysVar(varName, defaultValue)
c.Assert(err, IsNil)
v, err = se.GetGlobalSysVar(varName)
tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("0"))
c.Assert(err, IsNil)
c.Assert(v, Equals, defaultValue)
}

func (s *testSessionSuite) TestMatchIdentity(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("CREATE USER `useridentity`@`%`")
Expand Down Expand Up @@ -6091,3 +6127,18 @@ func (s *testSessionSuite) TestSysdateIsNow(c *C) {
tk.MustQuery("show variables like '%tidb_sysdate_is_now%'").Check(testkit.Rows("tidb_sysdate_is_now ON"))
c.Assert(tk.Se.GetSessionVars().SysdateIsNow, IsTrue)
}

func (s *testSessionSuite) TestEnableLegacyInstanceScope(c *C) {
tk := testkit.NewTestKit(c, s.store)

// enable 'switching' to SESSION variables
tk.MustExec("set tidb_enable_legacy_instance_scope = 1")
tk.MustExec("set tidb_general_log = 1")
tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope)))
c.Assert(tk.Se.GetSessionVars().EnableLegacyInstanceScope, IsTrue)

// disable 'switching' to SESSION variables
tk.MustExec("set tidb_enable_legacy_instance_scope = 0")
tk.MustGetErrCode("set tidb_general_log = 1", errno.ErrGlobalVariable)
c.Assert(tk.Se.GetSessionVars().EnableLegacyInstanceScope, IsFalse)
}
5 changes: 5 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -992,6 +992,10 @@ type SessionVars struct {
// EnablePaging indicates whether enable paging in coprocessor requests.
EnablePaging bool

// EnableLegacyInstanceScope says if SET SESSION can be used to set an instance
// scope variable. The default is TRUE.
EnableLegacyInstanceScope bool

// ReadConsistency indicates the read consistency requirement.
ReadConsistency ReadConsistencyLevel

Expand Down Expand Up @@ -1242,6 +1246,7 @@ func NewSessionVars() *SessionVars {
MPPStoreFailTTL: DefTiDBMPPStoreFailTTL,
Rng: utilMath.NewWithTime(),
StatsLoadSyncWait: StatsLoadSyncWait.Load(),
EnableLegacyInstanceScope: DefEnableLegacyInstanceScope,
}
vars.KVVars = tikvstore.NewVariables(&vars.Killed)
vars.Concurrency = Concurrency{
Expand Down
9 changes: 6 additions & 3 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,11 +707,10 @@ var defaultSysVars = []*SysVar{
}
return nil
}},
/* The following variable is defined as session scope but is actually server scope. */
{Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeInstance, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetGlobal: func(s *SessionVars, val string) error {
ProcessGeneralLog.Store(TiDBOptOn(val))
return nil
}, GetSession: func(s *SessionVars) (string, error) {
}, GetGlobal: func(s *SessionVars) (string, error) {
return BoolToOnOff(ProcessGeneralLog.Load()), nil
}},
{Scope: ScopeSession, Name: TiDBLogFileMaxDays, Value: strconv.Itoa(config.GetGlobalConfig().Log.File.MaxDays), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, skipInit: true, SetSession: func(s *SessionVars, val string) error {
Expand Down Expand Up @@ -1266,6 +1265,10 @@ var defaultSysVars = []*SysVar{
s.EnablePaging = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableLegacyInstanceScope, Value: BoolToOnOff(DefEnableLegacyInstanceScope), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.EnableLegacyInstanceScope = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal, Name: TiDBPersistAnalyzeOptions, Value: BoolToOnOff(DefTiDBPersistAnalyzeOptions), skipInit: true, Type: TypeBool,
GetGlobal: func(s *SessionVars) (string, error) {
return BoolToOnOff(PersistAnalyzeOptions.Load()), nil
Expand Down
64 changes: 63 additions & 1 deletion sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,22 +246,32 @@ func TestScope(t *testing.T) {
sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
require.True(t, sv.HasSessionScope())
require.True(t, sv.HasGlobalScope())
require.False(t, sv.HasInstanceScope())
require.False(t, sv.HasNoneScope())

sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
require.False(t, sv.HasSessionScope())
require.True(t, sv.HasGlobalScope())
require.False(t, sv.HasInstanceScope())
require.False(t, sv.HasNoneScope())

sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
require.True(t, sv.HasSessionScope())
require.False(t, sv.HasGlobalScope())
require.False(t, sv.HasInstanceScope())
require.False(t, sv.HasNoneScope())

sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
require.False(t, sv.HasSessionScope())
require.False(t, sv.HasGlobalScope())
require.False(t, sv.HasInstanceScope())
require.True(t, sv.HasNoneScope())

sv = SysVar{Scope: ScopeInstance, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
require.False(t, sv.HasSessionScope())
require.False(t, sv.HasGlobalScope())
require.True(t, sv.HasInstanceScope())
require.False(t, sv.HasNoneScope())
}

func TestBuiltInCase(t *testing.T) {
Expand Down Expand Up @@ -696,7 +706,7 @@ func TestSettersandGetters(t *testing.T) {
require.Nil(t, sv.SetSession)
require.Nil(t, sv.GetSession)
}
if !sv.HasGlobalScope() {
if !sv.HasGlobalScope() && !sv.HasInstanceScope() {
require.Nil(t, sv.SetGlobal)
if sv.Name == Timestamp {
// The Timestamp sysvar will have GetGlobal func even though it does not have global scope.
Expand Down Expand Up @@ -845,6 +855,58 @@ func TestDefaultCharsetAndCollation(t *testing.T) {
require.Equal(t, val, mysql.DefaultCollationName)
}

func TestInstanceScope(t *testing.T) {
// Instance scope used to be settable via "SET SESSION", which is weird to any MySQL user.
// It is now settable via SET GLOBAL, but to work correctly a sysvar can only ever
// be INSTANCE scoped or GLOBAL scoped, never *both* at the same time (at least for now).
// Otherwise the semantics are confusing to users for how precedence applies.

for _, sv := range GetSysVars() {
require.False(t, sv.HasGlobalScope() && sv.HasInstanceScope(), "sysvar %s has both instance and global scope", sv.Name)
if sv.HasInstanceScope() {
require.NotNil(t, sv.GetGlobal)
require.NotNil(t, sv.SetGlobal)
}
}

count := len(GetSysVars())
sv := SysVar{Scope: ScopeInstance, Name: "newinstancesysvar", Value: On, Type: TypeBool,
SetGlobal: func(s *SessionVars, val string) error {
return fmt.Errorf("set should fail")
},
GetGlobal: func(s *SessionVars) (string, error) {
return "", fmt.Errorf("get should fail")
},
}

RegisterSysVar(&sv)
require.Len(t, GetSysVars(), count+1)

sysVar := GetSysVar("newinstancesysvar")
require.NotNil(t, sysVar)

vars := NewSessionVars()

// It is a boolean, try to set it to a bogus value
_, err := sysVar.Validate(vars, "ABCD", ScopeInstance)
require.Error(t, err)

// Boolean oN or 1 converts to canonical ON or OFF
normalizedVal, err := sysVar.Validate(vars, "oN", ScopeInstance)
require.Equal(t, "ON", normalizedVal)
require.NoError(t, err)
normalizedVal, err = sysVar.Validate(vars, "0", ScopeInstance)
require.Equal(t, "OFF", normalizedVal)
require.NoError(t, err)

err = sysVar.SetGlobalFromHook(vars, "OFF", true) // default is on
require.Equal(t, "set should fail", err.Error())

// Test unregistration restores previous count
UnregisterSysVar("newinstancesysvar")
require.Equal(t, len(GetSysVars()), count)
}

func TestIndexMergeSwitcher(t *testing.T) {
vars := NewSessionVars()
vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests()
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -593,6 +593,9 @@ const (
// TiDBTmpTableMaxSize indicates the max memory size of temporary tables.
TiDBTmpTableMaxSize = "tidb_tmp_table_max_size"

// TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION.
TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope"

// TiDBTableCacheLease indicates the read lock lease of a cached table.
TiDBTableCacheLease = "tidb_table_cache_lease"

Expand Down Expand Up @@ -795,6 +798,7 @@ const (
DefTiDBStmtSummaryMaxSQLLength = 4096
DefTiDBCapturePlanBaseline = Off
DefTiDBEnableIndexMerge = true
DefEnableLegacyInstanceScope = true
DefTiDBTableCacheLease = 3 // 3s
DefTiDBPersistAnalyzeOptions = true
DefTiDBEnableColumnTracking = false
Expand Down
9 changes: 8 additions & 1 deletion sessionctx/variable/variable.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ const (
ScopeGlobal ScopeFlag = 1 << 0
// ScopeSession means the system variable can only be changed in current session.
ScopeSession ScopeFlag = 1 << 1
// ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers.
ScopeInstance ScopeFlag = 1 << 2

// TypeStr is the default
TypeStr TypeFlag = 0
Expand Down Expand Up @@ -248,6 +250,11 @@ func (sv *SysVar) HasGlobalScope() bool {
return sv.Scope&ScopeGlobal != 0
}

// HasInstanceScope returns true if the scope for the sysVar includes instance
func (sv *SysVar) HasInstanceScope() bool {
return sv.Scope&ScopeInstance != 0
}

// Validate checks if system variable satisfies specific restriction.
func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) {
// Check that the scope is correct first.
Expand Down Expand Up @@ -299,7 +306,7 @@ func (sv *SysVar) validateScope(scope ScopeFlag) error {
if sv.ReadOnly || sv.Scope == ScopeNone {
return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only")
}
if scope == ScopeGlobal && !sv.HasGlobalScope() {
if scope == ScopeGlobal && !(sv.HasGlobalScope() || sv.HasInstanceScope()) {
return errLocalVariable.FastGenByArgs(sv.Name)
}
if scope == ScopeSession && !sv.HasSessionScope() {
Expand Down

0 comments on commit d981c0e

Please sign in to comment.