Skip to content

Commit

Permalink
*: add support for removed sysvars (#28931)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored Oct 22, 2021
1 parent 00054d8 commit 6766833
Show file tree
Hide file tree
Showing 15 changed files with 91 additions and 95 deletions.
3 changes: 0 additions & 3 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1998,9 +1998,6 @@ func setTemporaryType(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.Cr
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
if !ctx.GetSessionVars().EnableGlobalTemporaryTable {
return errors.New("SET tidb_enable_global_temporary_table=1 to enable temporary tables")
}
// "create global temporary table ... on commit preserve rows"
if !s.OnCommitDelete {
return errors.Trace(errUnsupportedOnCommitPreserve)
Expand Down
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1008,6 +1008,7 @@ const (
ErrDataInConsistentExtraIndex = 8133
ErrDataInConsistentMisMatchIndex = 8134
ErrAsOf = 8135
ErrVariableNoLongerSupported = 8136

// 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 @@ -1054,6 +1054,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil),
ErrMultiStatementDisabled: mysql.Message("client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk", nil),
ErrAsOf: mysql.Message("invalid as of timestamp: %s", nil),
ErrVariableNoLongerSupported: mysql.Message("option '%s' is no longer supported. Reason: %s", nil),
ErrInvalidAttributesSpec: mysql.Message("Invalid attributes '%s': %s", nil),
ErrPlacementPolicyExists: mysql.Message("Placement policy '%-.192s' already exists", nil),
ErrPlacementPolicyNotExists: mysql.Message("Unknown placement policy '%-.192s'", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2031,3 +2031,8 @@ error = '''
snapshot is older than GC safe point %s
'''

["variable:8136"]
error = '''
option '%s' is no longer supported. Reason: %s
'''

2 changes: 0 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8862,7 +8862,6 @@ func (s *testStaleTxnSuite) TestInvalidReadTemporaryTable(c *C) {
UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment)
tk.MustExec(updateSafePoint)

tk.MustExec("set @@tidb_enable_global_temporary_table=1")
tk.MustExec("use test")
tk.MustExec("drop table if exists tmp1")
tk.MustExec("create global temporary table tmp1 " +
Expand Down Expand Up @@ -8984,7 +8983,6 @@ func (s *testSuite) TestTableSampleTemporaryTable(c *C) {
UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment)
tk.MustExec(updateSafePoint)

tk.MustExec("set @@tidb_enable_global_temporary_table=1")
tk.MustExec("use test")
tk.MustExec("drop table if exists tmp1")
tk.MustExec("create global temporary table tmp1 " +
Expand Down
3 changes: 3 additions & 0 deletions executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,9 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres
sessionVars := e.ctx.GetSessionVars()
sysVar := variable.GetSysVar(name)
if sysVar == nil {
if variable.IsRemovedSysVar(name) {
return nil // removed vars permit parse-but-ignore
}
return variable.ErrUnknownSystemVar.GenWithStackByArgs(name)
}
if v.IsGlobal {
Expand Down
43 changes: 31 additions & 12 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
Expand Down Expand Up @@ -397,23 +398,24 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect cte_max_recursion_depth value: '-1'"))
tk.MustQuery("select @@cte_max_recursion_depth").Check(testkit.Rows("0"))

// test for tidb_slow_log_masking
tk.MustQuery(`select @@global.tidb_slow_log_masking;`).Check(testkit.Rows("0"))
tk.MustExec("set global tidb_slow_log_masking = 1")
tk.MustQuery(`select @@global.tidb_slow_log_masking;`).Check(testkit.Rows("1"))
tk.MustExec("set global tidb_slow_log_masking = 0")
tk.MustQuery(`select @@global.tidb_slow_log_masking;`).Check(testkit.Rows("0"))
tk.MustExec("set session tidb_slow_log_masking = 0")
tk.MustQuery(`select @@session.tidb_slow_log_masking;`).Check(testkit.Rows("0"))
tk.MustExec("set session tidb_slow_log_masking = 1")
tk.MustQuery(`select @@session.tidb_slow_log_masking;`).Check(testkit.Rows("1"))
// test for tidb_redact_log
tk.MustQuery(`select @@global.tidb_redact_log;`).Check(testkit.Rows("0"))
tk.MustExec("set global tidb_redact_log = 1")
tk.MustQuery(`select @@global.tidb_redact_log;`).Check(testkit.Rows("1"))
tk.MustExec("set global tidb_redact_log = 0")
tk.MustQuery(`select @@global.tidb_redact_log;`).Check(testkit.Rows("0"))
tk.MustExec("set session tidb_redact_log = 0")
tk.MustQuery(`select @@session.tidb_redact_log;`).Check(testkit.Rows("0"))
tk.MustExec("set session tidb_redact_log = 1")
tk.MustQuery(`select @@session.tidb_redact_log;`).Check(testkit.Rows("1"))

tk.MustQuery("select @@tidb_dml_batch_size;").Check(testkit.Rows("0"))
tk.MustExec("set @@session.tidb_dml_batch_size = 120")
tk.MustQuery("select @@tidb_dml_batch_size;").Check(testkit.Rows("120"))
tk.MustExec("set @@session.tidb_dml_batch_size = -120")
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_dml_batch_size value: '?'")) // redacted because of tidb_slow_log_masking = 1 above
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_dml_batch_size value: '?'")) // redacted because of tidb_redact_log = 1 above
tk.MustQuery("select @@session.tidb_dml_batch_size").Check(testkit.Rows("0"))
tk.MustExec("set session tidb_slow_log_masking = 0")
tk.MustExec("set session tidb_redact_log = 0")
tk.MustExec("set session tidb_dml_batch_size = -120")
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_dml_batch_size value: '-120'")) // without redaction

Expand Down Expand Up @@ -1328,6 +1330,23 @@ func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) {

}

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

// test for tidb_enable_noop_functions
// In SET context, it just noops:
tk.MustExec(`SET tidb_enable_global_temporary_table = 1`)
tk.MustExec(`SET tidb_slow_log_masking = 1`)
tk.MustExec(`SET GLOBAL tidb_enable_global_temporary_table = 1`)
tk.MustExec(`SET GLOBAL tidb_slow_log_masking = 1`)

// In SELECT context it returns a specifc error
// (to avoid presenting dummy data)
tk.MustGetErrCode("SELECT @@tidb_slow_log_masking", errno.ErrVariableNoLongerSupported)
tk.MustGetErrCode("SELECT @@tidb_enable_global_temporary_table", errno.ErrVariableNoLongerSupported)

}

func (s *testSuite5) TestSetClusterConfig(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
6 changes: 6 additions & 0 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1267,6 +1267,12 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) {
sysVar := variable.GetSysVar(name)
if sysVar == nil {
er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name)
if err := variable.CheckSysVarIsRemoved(name); err != nil {
// Removed vars still return an error, but we customize it from
// "unknown" to an explanation of why it is not supported.
// This is important so users at least know they had the name correct.
er.err = err
}
return
}
if sem.IsEnabled() && sem.IsInvisibleSysVar(sysVar.Name) {
Expand Down
1 change: 0 additions & 1 deletion session/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1676,7 +1676,6 @@ func BenchmarkInsertIntoSelect(b *testing.B) {
st.Close()
}()

mustExecute(se, `set @@tidb_enable_global_temporary_table = 1`)
mustExecute(se, `set @@tmp_table_size = 1000000000`)
mustExecute(se, `create global temporary table tmp (id int, dt varchar(512)) on commit delete rows`)
mustExecute(se, `create table src (id int, dt varchar(512))`)
Expand Down
57 changes: 0 additions & 57 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3925,35 +3925,6 @@ func (s *testSessionSuite3) TestSetVarHint(c *C) {
c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "[planner:3126]Hint SET_VAR(group_concat_max_len=2048) is ignored as conflicting/duplicated.")
}

// TestDeprecateSlowLogMasking should be in serial suite because it changes a global variable.
func (s *testSessionSerialSuite) TestDeprecateSlowLogMasking(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

tk.MustExec("set @@global.tidb_redact_log=0")
tk.MustQuery("select @@global.tidb_redact_log").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.tidb_slow_log_masking").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_redact_log=1")
tk.MustQuery("select @@global.tidb_redact_log").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_slow_log_masking").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_slow_log_masking=0")
tk.MustQuery("select @@global.tidb_redact_log").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.tidb_slow_log_masking").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_redact_log=0")
tk.MustQuery("select @@session.tidb_redact_log").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.tidb_slow_log_masking").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_redact_log=1")
tk.MustQuery("select @@session.tidb_redact_log").Check(testkit.Rows("1"))
tk.MustQuery("select @@session.tidb_slow_log_masking").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_slow_log_masking=0")
tk.MustQuery("select @@session.tidb_redact_log").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.tidb_slow_log_masking").Check(testkit.Rows("0"))
}

func (s *testSessionSerialSuite) TestDoDDLJobQuit(c *C) {
// test https://github.com/pingcap/tidb/issues/18714, imitate DM's use environment
// use isolated store, because in below failpoint we will cancel its context
Expand Down Expand Up @@ -4812,34 +4783,6 @@ func (s *testSessionSuite) TestTMPTableSize(c *C) {
tk.MustExec("rollback")
}

func (s *testSessionSuite) TestTiDBEnableGlobalTemporaryTable(c *C) {
// Test the @@tidb_enable_global_temporary_table system variable.
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

// variable 'tidb_enable_global_temporary_table' should be seen when show variables
tk.MustQuery("show variables like 'tidb_enable_global_temporary_table'").Check(testkit.Rows("tidb_enable_global_temporary_table ON"))
tk.MustQuery("show global variables like 'tidb_enable_global_temporary_table'").Check(testkit.Rows("tidb_enable_global_temporary_table ON"))

// variable 'tidb_enable_global_temporary_table' is turned on by default
tk.MustQuery("select @@global.tidb_enable_global_temporary_table").Check(testkit.Rows("1"))
tk.MustQuery("select @@tidb_enable_global_temporary_table").Check(testkit.Rows("1"))
c.Assert(tk.Se.GetSessionVars().EnableGlobalTemporaryTable, IsTrue)

// can create global temporary table when 'tidb_enable_global_temporary_table' is on
tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows")
tk.MustQuery("show tables like 'temp_test'").Check(testkit.Rows("temp_test"))

// Get error messages when 'tidb_enable_global_temporary_table' is off
tk.MustExec("set tidb_enable_global_temporary_table=off")
tk.MustQuery("select @@tidb_enable_global_temporary_table").Check(testkit.Rows("0"))
c.Assert(tk.Se.GetSessionVars().EnableGlobalTemporaryTable, IsFalse)
tk.MustGetErrMsg(
"create global temporary table temp_test(id int primary key auto_increment) on commit delete rows",
"SET tidb_enable_global_temporary_table=1 to enable temporary tables",
)
}

func (s *testStatisticsSuite) cleanEnv(c *C, store kv.Storage, do *domain.Domain) {
tk := testkit.NewTestKit(c, store)
tk.MustExec("use test")
Expand Down
3 changes: 2 additions & 1 deletion sessionctx/variable/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,5 +41,6 @@ var (
// ErrFunctionsNoopImpl is an error to say the behavior is protected by the tidb_enable_noop_functions sysvar.
// This is copied from expression.ErrFunctionsNoopImpl to prevent circular dependencies.
// It needs to be public for tests.
ErrFunctionsNoopImpl = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", nil))
ErrFunctionsNoopImpl = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", nil))
ErrVariableNoLongerSupported = dbterror.ClassVariable.NewStd(mysql.ErrVariableNoLongerSupported)
)
41 changes: 41 additions & 0 deletions sessionctx/variable/removed.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package variable

// Removed sysvars is a way of removing sysvars, while allowing limited
// Parse-but-ignore support in SET context, and a more specific error in
// SELECT @@varname context.
//
// This helps ensure some compatibility for applications while being
// careful not to return dummy data.

var removedSysVars = map[string]string{
TiDBEnableGlobalTemporaryTable: "temporary table support is now always enabled",
TiDBSlowLogMasking: "use tidb_redact_log instead",
}

// IsRemovedSysVar returns true if the sysvar has been removed
func IsRemovedSysVar(varName string) bool {
_, ok := removedSysVars[varName]
return ok
}

// CheckSysVarIsRemoved returns an error if the sysvar has been removed
func CheckSysVarIsRemoved(varName string) error {
if reason, ok := removedSysVars[varName]; ok {
return ErrVariableNoLongerSupported.GenWithStackByArgs(varName, reason)
}
return nil
}
4 changes: 0 additions & 4 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -928,9 +928,6 @@ type SessionVars struct {
// See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_tmp_table_size
TMPTableSize int64

// EnableGlobalTemporaryTable indicates whether to enable global temporary table
EnableGlobalTemporaryTable bool

// EnableStableResultMode if stabilize query results.
EnableStableResultMode bool

Expand Down Expand Up @@ -1190,7 +1187,6 @@ func NewSessionVars() *SessionVars {
AllowFallbackToTiKV: make(map[kv.StoreType]struct{}),
CTEMaxRecursionDepth: DefCTEMaxRecursionDepth,
TMPTableSize: DefTMPTableSize,
EnableGlobalTemporaryTable: DefTiDBEnableGlobalTemporaryTable,
MPPStoreLastFailTime: make(map[string]time.Time),
MPPStoreFailTTL: DefTiDBMPPStoreFailTTL,
}
Expand Down
15 changes: 1 addition & 14 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1620,15 +1620,7 @@ var defaultSysVars = []*SysVar{
s.PartitionPruneMode.Store(strings.ToLower(strings.TrimSpace(val)))
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, skipInit: true, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBSlowLogMasking, TiDBRedactLog)
return normalizedValue, nil
}, GetSession: func(s *SessionVars) (string, error) {
return s.systems[TiDBRedactLog], nil
}, GetGlobal: func(s *SessionVars) (string, error) {
return s.GlobalVarsAccessor.GetGlobalSysVar(TiDBRedactLog)
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBSlowLogMasking}, Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.EnableRedactLog = TiDBOptOn(val)
errors.RedactLogEnabled.Store(s.EnableRedactLog)
return nil
Expand Down Expand Up @@ -1789,11 +1781,6 @@ var defaultSysVars = []*SysVar{
TopSQLVariable.ReportIntervalSeconds.Store(val)
return nil
}},

{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableGlobalTemporaryTable, Value: BoolToOnOff(DefTiDBEnableGlobalTemporaryTable), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.EnableGlobalTemporaryTable = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal, Name: SkipNameResolve, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal, Name: DefaultAuthPlugin, Value: mysql.AuthNativePassword, Type: TypeEnum, PossibleValues: []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableOrderedResultMode, Value: BoolToOnOff(DefTiDBEnableOrderedResultMode), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
Expand Down
1 change: 0 additions & 1 deletion sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -745,7 +745,6 @@ const (
DefTiDBTopSQLMaxStatementCount = 200
DefTiDBTopSQLMaxCollect = 10000
DefTiDBTopSQLReportIntervalSeconds = 60
DefTiDBEnableGlobalTemporaryTable = true
DefTMPTableSize = 16777216
DefTiDBEnableLocalTxn = false
DefTiDBEnableOrderedResultMode = false
Expand Down

0 comments on commit 6766833

Please sign in to comment.