diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 93f7a9af2a51b..89873ff086650 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -316,12 +315,6 @@ func (e *Execute) checkPreparedPriv(ctx context.Context, sctx sessionctx.Context return err } -func (e *Execute) setFoundInPlanCache(sctx sessionctx.Context, opt bool) error { - vars := sctx.GetSessionVars() - err := vars.SetSystemVar(variable.TiDBFoundInPlanCache, variable.BoolToOnOff(opt)) - return err -} - // GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key. func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareStmt) string { useBinding := sctx.GetSessionVars().UsePlanBaselines @@ -418,10 +411,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } else { planCacheCounter.Inc() } - err = e.setFoundInPlanCache(sctx, true) - if err != nil { - return err - } + sessVars.FoundInPlanCache = true e.names = names e.Plan = plan stmtCtx.PointExec = true @@ -460,17 +450,11 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, logutil.BgLogger().Debug("rebuild range failed", zap.Error(err)) goto REBUILD } - err = e.setFoundInPlanCache(sctx, true) - if err != nil { - return err - } + sessVars.FoundInPlanCache = true if len(bindSQL) > 0 { // When the `len(bindSQL) > 0`, it means we use the binding. // So we need to record this. - err = sessVars.SetSystemVar(variable.TiDBFoundInBinding, variable.BoolToOnOff(true)) - if err != nil { - return err - } + sessVars.FoundInBinding = true } if metrics.ResettablePlanCacheCounterFortTest { metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() @@ -534,7 +518,7 @@ REBUILD: sctx.PreparedPlanCache().Put(cacheKey, []*PlanCacheValue{cached}) } } - err = e.setFoundInPlanCache(sctx, false) + sessVars.FoundInPlanCache = false return err } diff --git a/planner/optimize.go b/planner/optimize.go index b6e2d84781265..aa3cfe4a6f1c4 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -186,9 +186,8 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in for _, warn := range warns { sessVars.StmtCtx.AppendWarning(warn) } - if err := setFoundInBinding(sctx, true, chosenBinding.BindSQL); err != nil { - logutil.BgLogger().Warn("set tidb_found_in_binding failed", zap.Error(err)) - } + sessVars.StmtCtx.BindSQL = chosenBinding.BindSQL + sessVars.FoundInBinding = true if sessVars.StmtCtx.InVerboseExplain { sessVars.StmtCtx.AppendNote(errors.Errorf("Using the bindSQL: %v", chosenBinding.BindSQL)) } @@ -664,13 +663,6 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin return } -func setFoundInBinding(sctx sessionctx.Context, opt bool, bindSQL string) error { - vars := sctx.GetSessionVars() - vars.StmtCtx.BindSQL = bindSQL - err := vars.SetSystemVar(variable.TiDBFoundInBinding, variable.BoolToOnOff(opt)) - return err -} - func init() { plannercore.OptimizeAstNode = Optimize plannercore.IsReadOnly = IsReadOnly diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 6eb70beabfc99..30e72ad4bfb5c 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -155,7 +155,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024", IsHintUpdatable: true}, {Scope: ScopeNone, Name: CharacterSetSystem, Value: "utf8"}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetFilesystem) }}, {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4af4de4e12837..98c217cf3cf5a 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -71,7 +71,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: TiDBAllowFunctionForExpressionIndex, ReadOnly: true, Value: collectAllowFuncName4ExpressionIndex()}, /* The system variables below have SESSION scope */ - {Scope: ScopeSession, Name: Timestamp, Value: DefTimestamp, skipInit: true, MinValue: 0, MaxValue: 2147483647, Type: TypeFloat, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: Timestamp, Value: DefTimestamp, MinValue: 0, MaxValue: 2147483647, Type: TypeFloat, GetSession: func(s *SessionVars) (string, error) { if timestamp, ok := s.systems[Timestamp]; ok && timestamp != DefTimestamp { return timestamp, nil } @@ -81,18 +81,18 @@ var defaultSysVars = []*SysVar{ timestamp, ok := s.systems[Timestamp] return timestamp, ok && timestamp != DefTimestamp, nil }}, - {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.Itoa(s.SysWarningCount), nil }}, - {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.Itoa(int(s.SysErrorCount)), nil }}, - {Scope: ScopeSession, Name: LastInsertID, Value: "", skipInit: true, Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: LastInsertID, Value: "", Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.StmtCtx.PrevLastInsertID, 10), nil }, GetStateValue: func(s *SessionVars) (string, bool, error) { return "", false, nil }}, - {Scope: ScopeSession, Name: Identity, Value: "", skipInit: true, Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: Identity, Value: "", Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.StmtCtx.PrevLastInsertID, 10), nil }, GetStateValue: func(s *SessionVars) (string, bool, error) { return "", false, nil @@ -163,7 +163,7 @@ var defaultSysVars = []*SysVar{ s.AllowWriteRowID = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, skipInit: true, Value: strconv.Itoa(DefChecksumTableConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, + {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, Value: strconv.Itoa(DefChecksumTableConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.BatchInsert = TiDBOptOn(val) return nil @@ -176,13 +176,13 @@ var defaultSysVars = []*SysVar{ s.BatchCommit = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.TxnCtx.StartTS, 10), nil }}, - {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return s.LastTxnInfo, nil }}, - {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { info, err := json.Marshal(s.LastQueryInfo) if err != nil { return "", err @@ -279,16 +279,10 @@ var defaultSysVars = []*SysVar{ s.MetricSchemaRangeDuration = TidbOptInt64(val, DefTiDBMetricSchemaRangeDuration) return nil }}, - {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { - s.FoundInPlanCache = TiDBOptOn(val) - return nil - }, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(s.PrevFoundInPlanCache), nil }}, - {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { - s.FoundInBinding = TiDBOptOn(val) - return nil - }, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(s.PrevFoundInBinding), nil }}, {Scope: ScopeSession, Name: RandSeed1, Type: TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { @@ -316,7 +310,7 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeSession, Name: TiDBLastDDLInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBLastDDLInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { info, err := json.Marshal(s.LastDDLInfo) if err != nil { return "", err @@ -867,7 +861,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool, skipInit: true}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TimeZone, Value: "SYSTEM", IsHintUpdatable: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if strings.EqualFold(normalizedValue, "SYSTEM") { return "SYSTEM", nil @@ -882,7 +876,7 @@ var defaultSysVars = []*SysVar{ s.TimeZone = tz return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: Off, Type: TypeBool, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { // TiDB does not yet support foreign keys. // Return the original value in the warning, so that users are not confused. @@ -911,10 +905,10 @@ var defaultSysVars = []*SysVar{ s.AutoIncrementOffset = tidbOptPositiveInt32(val, DefAutoIncrementOffset) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetClient) }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == "" { return normalizedValue, nil } @@ -960,7 +954,7 @@ var defaultSysVars = []*SysVar{ s.LockWaitTimeout = lockWaitSec * 1000 return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", IsHintUpdatable: true, skipInit: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", IsHintUpdatable: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { // https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len // Minimum Value 4 // Maximum Value (64-bit platforms) 18446744073709551615 @@ -1047,7 +1041,7 @@ var defaultSysVars = []*SysVar{ s.BroadcastJoinThresholdSize = TidbOptInt64(val, DefBroadcastJoinThresholdSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, skipInit: true, Value: strconv.Itoa(DefBuildStatsConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { s.AllowCartesianBCJ = TidbOptInt(val, DefOptCartesianBCJ) return nil diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 6d94cb81e8ac0..1146732e6030c 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -703,33 +703,27 @@ func TestSkipInitIsUsed(t *testing.T) { // skipInit only ever applied to session scope, so if anyone is setting it on // a variable without session, that doesn't make sense. require.True(t, sv.HasSessionScope(), fmt.Sprintf("skipInit has no effect on a variable without session scope: %s", sv.Name)) + // Since SetSession is the "init function" there is no init function to skip. + require.NotNil(t, sv.SetSession, fmt.Sprintf("skipInit has no effect on variables without an init (setsession) func: %s", sv.Name)) + // Skipinit has no use on noop funcs, since noop funcs always skipinit. + require.False(t, sv.IsNoop, fmt.Sprintf("skipInit has no effect on noop variables: %s", sv.Name)) + // Many of these variables might allow skipInit to be removed, // they need to be checked first. The purpose of this test is to make // sure we don't introduce any new variables with skipInit, which seems // to be a problem. switch sv.Name { - case Timestamp, - WarningCount, - ErrorCount, - LastInsertID, - Identity, - TiDBTxnScope, + case TiDBTxnScope, TiDBSnapshot, TiDBOptDistinctAggPushDown, TiDBOptWriteRowID, - TiDBChecksumTableConcurrency, TiDBBatchInsert, TiDBBatchDelete, TiDBBatchCommit, - TiDBCurrentTS, - TiDBLastTxnInfo, - TiDBLastQueryInfo, TiDBEnableChunkRPC, TxnIsolationOneShot, TiDBOptimizerSelectivityLevel, TiDBOptimizerEnableOuterJoinReorder, - TiDBLogFileMaxDays, - TiDBConfig, TiDBDDLReorgPriority, TiDBSlowQueryFile, TiDBWaitSplitRegionFinish, @@ -738,27 +732,17 @@ func TestSkipInitIsUsed(t *testing.T) { TiDBAllowRemoveAutoInc, TiDBMetricSchemaStep, TiDBMetricSchemaRangeDuration, - TiDBFoundInPlanCache, - TiDBFoundInBinding, RandSeed1, RandSeed2, - TiDBLastDDLInfo, - SQLLogBin, - ForeignKeyChecks, CollationDatabase, - CharacterSetClient, - CharacterSetResults, CollationConnection, CharsetDatabase, - GroupConcatMaxLen, CharacterSetConnection, CharacterSetServer, - TiDBBuildStatsConcurrency, TiDBOptTiFlashConcurrencyFactor, TiDBOptSeekFactor, TiDBOptJoinReorderThreshold, - TiDBStatsLoadSyncWait, - CharacterSetFilesystem: + TiDBStatsLoadSyncWait: continue } require.Equal(t, false, sv.skipInit, fmt.Sprintf("skipInit should not be set on new system variables. variable %s is in violation", sv.Name))