Skip to content

Commit

Permalink
*: add global/instance variable to config top sql (#24934)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored May 31, 2021
1 parent 456e72c commit 702c86b
Show file tree
Hide file tree
Showing 8 changed files with 170 additions and 31 deletions.
21 changes: 0 additions & 21 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"`
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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"},
},
Expand Down Expand Up @@ -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
Expand Down
3 changes: 1 addition & 2 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down Expand Up @@ -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())
}
}
6 changes: 2 additions & 4 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
69 changes: 69 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
4 changes: 2 additions & 2 deletions executor/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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())
Expand Down
55 changes: 55 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
39 changes: 39 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -673,6 +685,10 @@ const (
DefTiDBTrackAggregateMemoryUsage = true
DefTiDBEnableExchangePartition = false
DefCTEMaxRecursionDepth = 1000
DefTiDBTopSQLEnable = false
DefTiDBTopSQLAgentAddress = ""
DefTiDBTopSQLPrecisionSeconds = 1
DefTiDBTopSQLMaxStatementCount = 200
)

// Process global variables.
Expand All @@ -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() != ""
}

0 comments on commit 702c86b

Please sign in to comment.