diff --git a/config/config.go b/config/config.go index cfadfe3fd94e7..14467afaa0b4f 100644 --- a/config/config.go +++ b/config/config.go @@ -137,7 +137,6 @@ type Config struct { DelayCleanTableLock uint64 `toml:"delay-clean-table-lock" json:"delay-clean-table-lock"` SplitRegionMaxNum uint64 `toml:"split-region-max-num" json:"split-region-max-num"` StmtSummary StmtSummary `toml:"stmt-summary" json:"stmt-summary"` - TopSQL TopSQL `toml:"top-sql" json:"top-sql"` // RepairMode indicates that the TiDB is in the repair mode for table meta. RepairMode bool `toml:"repair-mode" json:"repair-mode"` RepairTableList []string `toml:"repair-table-list" json:"repair-table-list"` @@ -527,16 +526,6 @@ type StmtSummary struct { HistorySize int `toml:"history-size" json:"history-size"` } -// TopSQL is the config for top sql. -type TopSQL struct { - // Enable statement summary or not. - Enable bool `toml:"enable" json:"enable"` - // The refresh interval of statement summary. - RefreshInterval int `toml:"refresh-interval" json:"refresh-interval"` - // The maximum number of statements kept in memory. - MaxStmtCount uint `toml:"max-stmt-count" json:"max-stmt-count"` -} - // IsolationRead is the config for isolation read. type IsolationRead struct { // Engines filters tidb-server access paths by engine type. @@ -666,11 +655,6 @@ var defaultConf = Config{ RefreshInterval: 1800, HistorySize: 24, }, - TopSQL: TopSQL{ - Enable: true, - RefreshInterval: 1, - MaxStmtCount: 5000, - }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, }, @@ -958,11 +942,6 @@ func TableLockEnabled() bool { return GetGlobalConfig().EnableTableLock } -// TopSQLEnabled uses to check whether enabled the top SQL feature. -func TopSQLEnabled() bool { - return GetGlobalConfig().TopSQL.Enable -} - // TableLockDelayClean uses to get the time of delay clean table lock. var TableLockDelayClean = func() uint64 { return GetGlobalConfig().DelayCleanTableLock diff --git a/distsql/request_builder.go b/distsql/request_builder.go index d34ecc9ab0c47..0fd44b044ae3b 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -278,7 +277,7 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde // SetResourceGroupTag sets the request resource group tag. func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext) *RequestBuilder { - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { builder.Request.ResourceGroupTag = sc.GetResourceGroupTag() } return builder diff --git a/executor/executor.go b/executor/executor.go index 3474d7e33cd8f..f9cffcdfdba54 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -977,7 +977,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { var planDigest *parser.Digest _, sqlDigest := seVars.StmtCtx.SQLDigest() - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { _, planDigest = seVars.StmtCtx.GetPlanDigest() } return &tikvstore.LockCtx{ @@ -1822,7 +1822,7 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd } func setResourceGroupTagForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { - if snapshot != nil && config.TopSQLEnabled() { + if snapshot != nil && variable.TopSQLEnabled() { snapshot.SetOption(kv.ResourceGroupTag, sc.GetResourceGroupTag()) } } diff --git a/executor/executor_test.go b/executor/executor_test.go index d04c202e1e221..c41b6f2a7773e 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8364,10 +8364,8 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tbInfo := testGetTableByName(c, tk.Se, "test", "t") // Enable Top SQL - cfg := config.GetGlobalConfig() - newCfg := *cfg - newCfg.TopSQL.Enable = true - config.StoreGlobalConfig(&newCfg) + variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.AgentAddress.Store("mock-agent") c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") diff --git a/executor/set_test.go b/executor/set_test.go index eda1696497cea..4134cb128d990 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1376,3 +1376,72 @@ func (s *testSuite5) TestSetClusterConfigJSONData(c *C) { } } } + +func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_enable_top_sql='On';") + tk.MustQuery("select @@tidb_enable_top_sql;").Check(testkit.Rows("1")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsTrue) + tk.MustExec("set @@tidb_enable_top_sql='off';") + tk.MustQuery("select @@tidb_enable_top_sql;").Check(testkit.Rows("0")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsFalse) + tk.MustExec("set @@global.tidb_enable_top_sql='On';") + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("1")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsTrue) + tk.MustExec("set @@global.tidb_enable_top_sql='off';") + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("0")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsFalse) + + tk.MustExec("set @@tidb_top_sql_agent_address='127.0.0.1:4001';") + tk.MustQuery("select @@tidb_top_sql_agent_address;").Check(testkit.Rows("127.0.0.1:4001")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "127.0.0.1:4001") + tk.MustExec("set @@tidb_top_sql_agent_address='';") + tk.MustQuery("select @@tidb_top_sql_agent_address;").Check(testkit.Rows("")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "") + tk.MustExec("set @@global.tidb_top_sql_agent_address='127.0.0.1:4001';") + tk.MustQuery("select @@global.tidb_top_sql_agent_address;").Check(testkit.Rows("127.0.0.1:4001")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "127.0.0.1:4001") + tk.MustExec("set @@global.tidb_top_sql_agent_address='';") + tk.MustQuery("select @@global.tidb_top_sql_agent_address;").Check(testkit.Rows("")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "") + + tk.MustExec("set @@tidb_top_sql_precision_seconds=60;") + tk.MustQuery("select @@tidb_top_sql_precision_seconds;").Check(testkit.Rows("60")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(60)) + _, err := tk.Exec("set @@tidb_top_sql_precision_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_precision_seconds'") + _, err = tk.Exec("set @@tidb_top_sql_precision_seconds='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_precision_seconds' can't be set to the value of '-1'") + tk.MustQuery("select @@tidb_top_sql_precision_seconds;").Check(testkit.Rows("60")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(60)) + tk.MustExec("set @@global.tidb_top_sql_precision_seconds=2;") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + _, err = tk.Exec("set @@global.tidb_top_sql_precision_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_precision_seconds'") + _, err = tk.Exec("set @@global.tidb_top_sql_precision_seconds='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_precision_seconds' can't be set to the value of '-1'") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + + tk.MustExec("set @@tidb_top_sql_max_statement_count=5000;") + tk.MustQuery("select @@tidb_top_sql_max_statement_count;").Check(testkit.Rows("5000")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(5000)) + _, err = tk.Exec("set @@tidb_top_sql_max_statement_count='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_statement_count'") + _, err = tk.Exec("set @@tidb_top_sql_max_statement_count='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '-1'") + tk.MustQuery("select @@tidb_top_sql_max_statement_count;").Check(testkit.Rows("5000")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(5000)) + tk.MustExec("set @@global.tidb_top_sql_max_statement_count=2;") + tk.MustQuery("select @@global.tidb_top_sql_max_statement_count;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(2)) + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_statement_count'") + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '-1'") + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='5001';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '5001'") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(2)) +} diff --git a/executor/update.go b/executor/update.go index 194f8e415a17a..cbc61f6121b88 100644 --- a/executor/update.go +++ b/executor/update.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -264,7 +264,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { txn, err := e.ctx.Txn(true) if err == nil { txn.SetOption(kv.ResourceGroupTag, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTag()) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 23e17b0cec704..ff53d355e4443 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1627,6 +1627,61 @@ var defaultSysVars = []*SysVar{ {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"}}, + + // variable for top SQL feature. + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(TopSQLVariable.Enable.Load()), nil + }, SetSession: func(vars *SessionVars, s string) error { + TopSQLVariable.Enable.Store(TiDBOptOn(s)) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + TopSQLVariable.Enable.Store(TiDBOptOn(s)) + return nil + }}, + // TODO(crazycs520): Add validation + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLAgentAddress, Value: DefTiDBTopSQLAgentAddress, Type: TypeStr, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return TopSQLVariable.AgentAddress.Load(), nil + }, SetSession: func(vars *SessionVars, s string) error { + TopSQLVariable.AgentAddress.Store(s) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + TopSQLVariable.AgentAddress.Store(s) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil + }, SetSession: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.PrecisionSeconds.Store(val) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.PrecisionSeconds.Store(val) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLMaxStatementCount, Value: strconv.Itoa(DefTiDBTopSQLMaxStatementCount), Type: TypeInt, MinValue: 0, MaxValue: 5000, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatInt(TopSQLVariable.MaxStatementCount.Load(), 10), nil + }, SetSession: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.MaxStatementCount.Store(val) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.MaxStatementCount.Store(val) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 5d7897ff9273a..63f03e4c14695 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -531,6 +531,18 @@ const ( // TiDBEnableDynamicPrivileges enables MySQL 8.0 compatible dynamic privileges (experimental). TiDBEnableDynamicPrivileges = "tidb_enable_dynamic_privileges" + + // TiDBEnableTopSQL indicates whether the top SQL is enabled. + TiDBEnableTopSQL = "tidb_enable_top_sql" + + // TiDBTopSQLAgentAddress indicates the top SQL agent address. + TiDBTopSQLAgentAddress = "tidb_top_sql_agent_address" + + // TiDBTopSQLPrecisionSeconds indicates the top SQL precision seconds. + TiDBTopSQLPrecisionSeconds = "tidb_top_sql_precision_seconds" + + // TiDBTopSQLMaxStatementCount indicates the max number of statements been collected. + TiDBTopSQLMaxStatementCount = "tidb_top_sql_max_statement_count" ) // TiDB vars that have only global scope @@ -673,6 +685,10 @@ const ( DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false DefCTEMaxRecursionDepth = 1000 + DefTiDBTopSQLEnable = false + DefTiDBTopSQLAgentAddress = "" + DefTiDBTopSQLPrecisionSeconds = 1 + DefTiDBTopSQLMaxStatementCount = 200 ) // Process global variables. @@ -697,4 +713,27 @@ var ( CapturePlanBaseline = serverGlobalVariable{globalVal: Off} DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) + TopSQLVariable = TopSQL{ + Enable: atomic.NewBool(DefTiDBTopSQLEnable), + AgentAddress: atomic.NewString(DefTiDBTopSQLAgentAddress), + PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), + MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), + } ) + +// TopSQL is the variable for control top sql feature. +type TopSQL struct { + // Enable statement summary or not. + Enable *atomic.Bool + // AgentAddress indicate the collect agent address. + AgentAddress *atomic.String + // The refresh interval of statement summary. + PrecisionSeconds *atomic.Int64 + // The maximum number of statements kept in memory. + MaxStatementCount *atomic.Int64 +} + +// TopSQLEnabled uses to check whether enabled the top SQL feature. +func TopSQLEnabled() bool { + return TopSQLVariable.Enable.Load() && TopSQLVariable.AgentAddress.Load() != "" +}