Skip to content

Commit

Permalink
*: misc cleanups to variable package (#24507)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored May 11, 2021
1 parent e26b847 commit c8bc701
Show file tree
Hide file tree
Showing 16 changed files with 127 additions and 140 deletions.
4 changes: 3 additions & 1 deletion ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,9 @@ func LoadGlobalVars(ctx sessionctx.Context, varNames []string) error {
for _, row := range rows {
varName := row.GetString(0)
varValue := row.GetString(1)
variable.SetLocalSystemVar(varName, varValue)
if err = ctx.GetSessionVars().SetSystemVar(varName, varValue); err != nil {
return err
}
}
}
return nil
Expand Down
2 changes: 1 addition & 1 deletion executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,7 +479,7 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe
panic(fmt.Sprintf("Error happened when buildGroupConcat: %s", err.Error()))
}
var s string
s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
if err != nil {
panic(fmt.Sprintf("Error happened when buildGroupConcat: no system variable named '%s'", variable.GroupConcatMaxLen))
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {

func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) {
sessionVars := ctx.GetSessionVars()
concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency)
concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency)
if err != nil {
return 0, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,7 @@ func (c *checksumContext) HandleResponse(update *tipb.ChecksumResponse) {

func getChecksumTableConcurrency(ctx sessionctx.Context) (int, error) {
sessionVars := ctx.GetSessionVars()
concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency)
concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency)
if err != nil {
return 0, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -1675,7 +1675,7 @@ func (e *memtableRetriever) setDataFromSessionVar(ctx sessionctx.Context) error
sessionVars := ctx.GetSessionVars()
for _, v := range variable.GetSysVars() {
var value string
value, err = variable.GetSessionSystemVar(sessionVars, v.Name)
value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -570,7 +570,7 @@ func (s *testSuite5) TestSetCharset(c *C) {

check := func(args ...string) {
for i, v := range characterSetVariables {
sVar, err := variable.GetSessionSystemVar(sessionVars, v)
sVar, err := variable.GetSessionOrGlobalSystemVar(sessionVars, v)
c.Assert(err, IsNil)
c.Assert(sVar, Equals, args[i], Commentf("%d: %s", i, characterSetVariables[i]))
}
Expand Down
2 changes: 1 addition & 1 deletion executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -693,7 +693,7 @@ func (e *ShowExec) fetchShowVariables() (err error) {
if v.Hidden {
continue
}
value, err = variable.GetSessionSystemVar(sessionVars, v.Name)
value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,7 @@ func (a *AggFuncDesc) GetAggFunc(ctx sessionctx.Context) Aggregation {
var s string
var err error
var maxLen uint64
s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
if err != nil {
panic(fmt.Sprintf("Error happened when GetAggFunc: no system variable named '%s'", variable.GroupConcatMaxLen))
}
Expand Down
2 changes: 1 addition & 1 deletion expression/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ func getStmtTimestamp(ctx sessionctx.Context) (time.Time, error) {
}

sessionVars := ctx.GetSessionVars()
timestampStr, err := variable.GetSessionSystemVar(sessionVars, "timestamp")
timestampStr, err := variable.GetSessionOrGlobalSystemVar(sessionVars, "timestamp")
if err != nil {
return now, err
}
Expand Down
36 changes: 22 additions & 14 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1215,34 +1215,42 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) {
er.ctxStackAppend(f, types.EmptyName)
return
}
var val string
var err error
if v.ExplicitScope {
err = variable.ValidateGetSystemVar(name, v.IsGlobal)
if err != nil {
er.err = err
return
}
}
sysVar := variable.GetSysVar(name)
if sysVar == nil {
er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name)
return
}
// Variable is @@gobal.variable_name or variable is only global scope variable.
if v.IsGlobal || sysVar.Scope == variable.ScopeGlobal {
if v.ExplicitScope && !sysVar.HasNoneScope() {
if v.IsGlobal && !sysVar.HasGlobalScope() {
er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL")
return
}
if !v.IsGlobal && !sysVar.HasSessionScope() {
er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION")
return
}
}
var val string
var err error
if sysVar.HasNoneScope() {
val = sysVar.Value
} else if v.IsGlobal || !sysVar.HasSessionScope() {
// The condition "|| !sysVar.HasSessionScope()" is a workaround
// for issue https://github.com/pingcap/tidb/issues/24368
// Where global values are cached incorrectly. When this issue closes,
// the if statement here can be simplified.
val, err = variable.GetGlobalSystemVar(sessionVars, name)
} else {
val, err = variable.GetSessionSystemVar(sessionVars, name)
val, err = variable.GetSessionOrGlobalSystemVar(sessionVars, name)
}
if err != nil {
er.err = err
return
}
nativeVal, nativeType, nativeFlag := sysVar.GetNativeValType(val)
e := expression.DatumToConstant(nativeVal, nativeType, nativeFlag)
e.GetType().Charset, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection)
e.GetType().Collate, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CollationConnection)
e.GetType().Charset, _ = sessionVars.GetSystemVar(variable.CharacterSetConnection)
e.GetType().Collate, _ = sessionVars.GetSystemVar(variable.CollationConnection)
er.ctxStackAppend(e, types.EmptyName)
}

Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/noop.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: SuperReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal, Name: serverReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal, Name: ReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true},
Expand Down
31 changes: 1 addition & 30 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1445,20 +1445,6 @@ func (s *SessionVars) LazyCheckKeyNotExists() bool {
return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning)
}

// SetLocalSystemVar sets values of the local variables which in "server" scope.
func SetLocalSystemVar(name string, val string) {
switch name {
case TiDBDDLReorgWorkerCount:
SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount)))
case TiDBDDLReorgBatchSize:
SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize)))
case TiDBDDLErrorCountLimit:
SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit))
case TiDBRowFormatVersion:
SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2))
}
}

// special session variables.
const (
SQLModeVar = "sql_mode"
Expand All @@ -1469,22 +1455,7 @@ const (
TransactionIsolation = "transaction_isolation"
TxnIsolationOneShot = "tx_isolation_one_shot"
MaxExecutionTime = "max_execution_time"
)

// these variables are useless for TiDB, but still need to validate their values for some compatible issues.
// TODO: some more variables need to be added here.
const (
serverReadOnly = "read_only"
)

var (
// TxIsolationNames are the valid values of the variable "tx_isolation" or "transaction_isolation".
TxIsolationNames = map[string]struct{}{
"READ-UNCOMMITTED": {},
"READ-COMMITTED": {},
"REPEATABLE-READ": {},
"SERIALIZABLE": {},
}
ReadOnly = "read_only"
)

// TableDelta stands for the changed count for one table or partition.
Expand Down
23 changes: 19 additions & 4 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,11 @@ func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool
return nil
}

// HasNoneScope returns true if the scope for the sysVar is None.
func (sv *SysVar) HasNoneScope() bool {
return sv.Scope == ScopeNone
}

// HasSessionScope returns true if the scope for the sysVar includes session.
func (sv *SysVar) HasSessionScope() bool {
return sv.Scope&ScopeSession != 0
Expand Down Expand Up @@ -1060,6 +1065,7 @@ var defaultSysVars = []*SysVar{
} else if formatVersion == DefTiDBRowFormatV2 {
s.RowEncoder.Enable = true
}
SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2))
return nil
}},
{Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
Expand Down Expand Up @@ -1138,9 +1144,18 @@ var defaultSysVars = []*SysVar{
return nil
}},
{Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true},
{Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount)},
{Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true},
{Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true},
{Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error {
SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount)))
return nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error {
SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize)))
return nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error {
SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit))
return nil
}},
{Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", SetSession: func(s *SessionVars, val string) error {
s.setDDLReorgPriority(val)
return nil
Expand Down Expand Up @@ -1210,7 +1225,7 @@ var defaultSysVars = []*SysVar{
// To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope.

if normalizedValue == Off {
for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly} {
for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, ReadOnly} {
val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope
if scope == ScopeGlobal { // global scope
var err error
Expand Down
54 changes: 31 additions & 23 deletions sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,14 +223,22 @@ func (*testSysVarSuite) TestScope(c *C) {
sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
c.Assert(sv.HasSessionScope(), IsTrue)
c.Assert(sv.HasGlobalScope(), IsTrue)
c.Assert(sv.HasNoneScope(), IsFalse)

sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
c.Assert(sv.HasSessionScope(), IsFalse)
c.Assert(sv.HasGlobalScope(), IsTrue)
c.Assert(sv.HasNoneScope(), IsFalse)

sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
c.Assert(sv.HasSessionScope(), IsTrue)
c.Assert(sv.HasGlobalScope(), IsFalse)
c.Assert(sv.HasNoneScope(), IsFalse)

sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}}
c.Assert(sv.HasSessionScope(), IsFalse)
c.Assert(sv.HasGlobalScope(), IsFalse)
c.Assert(sv.HasNoneScope(), IsTrue)
}

func (*testSysVarSuite) TestBuiltInCase(c *C) {
Expand Down Expand Up @@ -444,106 +452,106 @@ func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) {
}

func (*testSysVarSuite) TestInstanceScopedVars(c *C) {
// This tests instance scoped variables through GetSessionSystemVar().
// This tests instance scoped variables through GetSessionOrGlobalSystemVar().
// Eventually these should be changed to use getters so that the switch
// statement in GetSessionOnlySysVars can be removed.

vars := NewSessionVars()

val, err := GetSessionSystemVar(vars, TiDBCurrentTS)
val, err := GetSessionOrGlobalSystemVar(vars, TiDBCurrentTS)
c.Assert(err, IsNil)
c.Assert(val, Equals, fmt.Sprintf("%d", vars.TxnCtx.StartTS))

val, err = GetSessionSystemVar(vars, TiDBLastTxnInfo)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastTxnInfo)
c.Assert(err, IsNil)
c.Assert(val, Equals, vars.LastTxnInfo)

val, err = GetSessionSystemVar(vars, TiDBLastQueryInfo)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastQueryInfo)
c.Assert(err, IsNil)
info, err := json.Marshal(vars.LastQueryInfo)
c.Assert(err, IsNil)
c.Assert(val, Equals, string(info))

val, err = GetSessionSystemVar(vars, TiDBGeneralLog)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBGeneralLog)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(ProcessGeneralLog.Load()))

val, err = GetSessionSystemVar(vars, TiDBPProfSQLCPU)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBPProfSQLCPU)
c.Assert(err, IsNil)
expected := "0"
if EnablePProfSQLCPU.Load() {
expected = "1"
}
c.Assert(val, Equals, expected)

val, err = GetSessionSystemVar(vars, TiDBExpensiveQueryTimeThreshold)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBExpensiveQueryTimeThreshold)
c.Assert(err, IsNil)
c.Assert(val, Equals, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)))

val, err = GetSessionSystemVar(vars, TiDBMemoryUsageAlarmRatio)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBMemoryUsageAlarmRatio)
c.Assert(err, IsNil)
c.Assert(val, Equals, fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()))

val, err = GetSessionSystemVar(vars, TiDBConfig)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBConfig)
c.Assert(err, IsNil)
conf := config.GetGlobalConfig()
j, err := json.MarshalIndent(conf, "", "\t")
c.Assert(err, IsNil)
c.Assert(val, Equals, config.HideConfig(string(j)))

val, err = GetSessionSystemVar(vars, TiDBForcePriority)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBForcePriority)
c.Assert(err, IsNil)
c.Assert(val, Equals, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))])

val, err = GetSessionSystemVar(vars, TiDBDDLSlowOprThreshold)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBDDLSlowOprThreshold)
c.Assert(err, IsNil)
c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10))

val, err = GetSessionSystemVar(vars, PluginDir)
val, err = GetSessionOrGlobalSystemVar(vars, PluginDir)
c.Assert(err, IsNil)
c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Dir)

val, err = GetSessionSystemVar(vars, PluginLoad)
val, err = GetSessionOrGlobalSystemVar(vars, PluginLoad)
c.Assert(err, IsNil)
c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Load)

val, err = GetSessionSystemVar(vars, TiDBSlowLogThreshold)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBSlowLogThreshold)
c.Assert(err, IsNil)
c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10))

val, err = GetSessionSystemVar(vars, TiDBRecordPlanInSlowLog)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBRecordPlanInSlowLog)
c.Assert(err, IsNil)
c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10))

val, err = GetSessionSystemVar(vars, TiDBEnableSlowLog)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableSlowLog)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog))

val, err = GetSessionSystemVar(vars, TiDBQueryLogMaxLen)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBQueryLogMaxLen)
c.Assert(err, IsNil)
c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10))

val, err = GetSessionSystemVar(vars, TiDBCheckMb4ValueInUTF8)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBCheckMb4ValueInUTF8)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8))

val, err = GetSessionSystemVar(vars, TiDBCapturePlanBaseline)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBCapturePlanBaseline)
c.Assert(err, IsNil)
c.Assert(val, Equals, CapturePlanBaseline.GetVal())

val, err = GetSessionSystemVar(vars, TiDBFoundInPlanCache)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInPlanCache)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInPlanCache))

val, err = GetSessionSystemVar(vars, TiDBFoundInBinding)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInBinding)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInBinding))

val, err = GetSessionSystemVar(vars, TiDBEnableCollectExecutionInfo)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableCollectExecutionInfo)
c.Assert(err, IsNil)
c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo))

val, err = GetSessionSystemVar(vars, TiDBTxnScope)
val, err = GetSessionOrGlobalSystemVar(vars, TiDBTxnScope)
c.Assert(err, IsNil)
c.Assert(val, Equals, vars.TxnScope.GetVarValue())
}
Loading

0 comments on commit c8bc701

Please sign in to comment.