Skip to content

Commit

Permalink
*: replace conf item pprof_sql_cpu with srv var `tidb_pprof_… (#14416)
Browse files Browse the repository at this point in the history
  • Loading branch information
lysu authored and zz-jason committed Jan 11, 2020
1 parent bf155a7 commit f0678b4
Show file tree
Hide file tree
Showing 9 changed files with 26 additions and 14 deletions.
1 change: 0 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,6 @@ type Performance struct {
BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"`
TxnEntryCountLimit uint64 `toml:"txn-entry-count-limit" json:"txn-entry-count-limit"`
TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"`
PProfSQLCPU bool `toml:"pprof-sql-cpu" json:"pprof-sql-cpu"`
}

// PlanCache is the PlanCache section of the config.
Expand Down
8 changes: 8 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,9 @@ func (s *testSuite2) TestSetVar(c *C) {
tk.MustExec("set @@tidb_general_log = 1")
tk.MustExec("set @@tidb_general_log = 0")

tk.MustExec("set @@tidb_pprof_sql_cpu = 1")
tk.MustExec("set @@tidb_pprof_sql_cpu = 0")

tk.MustExec(`set tidb_force_priority = "no_priority"`)
tk.MustQuery(`select @@tidb_force_priority;`).Check(testkit.Rows("NO_PRIORITY"))
tk.MustExec(`set tidb_force_priority = "low_priority"`)
Expand Down Expand Up @@ -448,6 +451,11 @@ func (s *testSuite2) TestValidateSetVar(c *C) {
tk.MustExec("set @@tidb_batch_delete=1;")
tk.MustExec("set @@tidb_batch_delete=0;")

tk.MustExec("set @@tidb_pprof_sql_cpu=1;")
tk.MustQuery("select @@tidb_pprof_sql_cpu;").Check(testkit.Rows("1"))
tk.MustExec("set @@tidb_pprof_sql_cpu=0;")
tk.MustQuery("select @@tidb_pprof_sql_cpu;").Check(testkit.Rows("0"))

_, err = tk.Exec("set @@tidb_batch_delete=3;")
c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err))

Expand Down
3 changes: 1 addition & 2 deletions server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,6 @@ import (
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/arena"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/hack"
Expand Down Expand Up @@ -870,7 +869,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error {
cc.lastPacket = data
cmd := data[0]
data = data[1:]
if util.EnablePProfSQLCPU.Load() {
if variable.EnablePProfSQLCPU.Load() {
defer pprof.SetGoroutineLabels(ctx)
lastSQL := getLastStmtInConn{cc}.String()
ctx = pprof.WithLabels(ctx, pprof.Labels("sql", parser.Normalize(lastSQL)))
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -840,6 +840,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.MemQuotaNestedLoopApply = tidbOptInt64(val, DefTiDBMemQuotaNestedLoopApply)
case TiDBGeneralLog:
atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt32(val, DefTiDBGeneralLog)))
case TiDBPProfSQLCPU:
EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0)
case TiDBSlowLogThreshold:
atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold)))
case TiDBRecordPlanInSlowLog:
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -694,6 +694,7 @@ var defaultSysVars = []*SysVar{
{ScopeGlobal | ScopeSession, TiDBSkipIsolationLevelCheck, BoolToIntStr(DefTiDBSkipIsolationLevelCheck)},
/* The following variable is defined as session scope but is actually server scope. */
{ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)},
{ScopeSession, TiDBPProfSQLCPU, strconv.Itoa(DefTiDBPProfSQLCPU)},
{ScopeSession, TiDBSlowLogThreshold, strconv.Itoa(logutil.DefaultSlowThreshold)},
{ScopeSession, TiDBRecordPlanInSlowLog, strconv.Itoa(logutil.DefaultRecordPlanInSlowLog)},
{ScopeSession, TiDBDDLSlowOprThreshold, strconv.Itoa(DefTiDBDDLSlowOprThreshold)},
Expand Down
6 changes: 6 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"os"

"github.com/pingcap/parser/mysql"
"github.com/uber-go/atomic"
)

/*
Expand Down Expand Up @@ -105,6 +106,9 @@ const (
// tidb_general_log is used to log every query in the server in info level.
TiDBGeneralLog = "tidb_general_log"

// tidb_pprof_sql_cpu is used to add label sql label to pprof result.
TiDBPProfSQLCPU = "tidb_pprof_sql_cpu"

// tidb_slow_log_threshold is used to set the slow log threshold in the server.
TiDBSlowLogThreshold = "tidb_slow_log_threshold"

Expand Down Expand Up @@ -345,6 +349,7 @@ const (
DefTiDBMemQuotaNestedLoopApply = 32 << 30 // 32GB.
DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB.
DefTiDBGeneralLog = 0
DefTiDBPProfSQLCPU = 0
DefTiDBRetryLimit = 10
DefTiDBDisableTxnAutoRetry = true
DefTiDBConstraintCheckInPlace = false
Expand Down Expand Up @@ -377,6 +382,7 @@ const (
// Process global variables.
var (
ProcessGeneralLog uint32
EnablePProfSQLCPU = atomic.NewBool(false)
ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount
maxDDLReorgWorkerCount int32 = 128
ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize
Expand Down
8 changes: 7 additions & 1 deletion sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,12 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) {
return fmt.Sprintf("%d", s.TxnCtx.StartTS), true, nil
case TiDBGeneralLog:
return fmt.Sprintf("%d", atomic.LoadUint32(&ProcessGeneralLog)), true, nil
case TiDBPProfSQLCPU:
val := "0"
if EnablePProfSQLCPU.Load() {
val = "1"
}
return val, true, nil
case TiDBExpensiveQueryTimeThreshold:
return fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), true, nil
case TiDBConfig:
Expand Down Expand Up @@ -391,7 +397,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string,
return "1", nil
}
return value, ErrWrongValueForVar.GenWithStackByArgs(name, value)
case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin,
case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, TiDBPProfSQLCPU,
CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates,
SkipNameResolve, SQLSafeUpdates, TiDBConstraintCheckInPlace, serverReadOnly, SlaveAllowBatching,
Flush, PerformanceSchema, LocalInFile, ShowOldTemporals, KeepFilesOnCreate, AutoCommit,
Expand Down
7 changes: 1 addition & 6 deletions tidb-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ import (
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/gcworker"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/printer"
Expand Down Expand Up @@ -358,7 +357,7 @@ func loadConfig() string {
// hotReloadConfigItems lists all config items which support hot-reload.
var hotReloadConfigItems = []string{"Performance.MaxProcs", "Performance.MaxMemory", "Performance.CrossJoin",
"Performance.FeedbackProbability", "Performance.QueryFeedbackLimit", "Performance.PseudoEstimateRatio",
"OOMAction", "MemQuotaQuery", "StmtSummary.MaxStmtCount", "StmtSummary.MaxSQLLength", "TiKVClient.StoreLimit", "Performance.PProfSQLCPU"}
"OOMAction", "MemQuotaQuery", "StmtSummary.MaxStmtCount", "StmtSummary.MaxSQLLength", "TiKVClient.StoreLimit"}

func reloadConfig(nc, c *config.Config) {
// Just a part of config items need to be reload explicitly.
Expand All @@ -384,9 +383,6 @@ func reloadConfig(nc, c *config.Config) {
if nc.Performance.PseudoEstimateRatio != c.Performance.PseudoEstimateRatio {
statistics.RatioOfPseudoEstimate.Store(nc.Performance.PseudoEstimateRatio)
}
if nc.Performance.PProfSQLCPU != c.Performance.PProfSQLCPU {
util.EnablePProfSQLCPU.Store(nc.Performance.PProfSQLCPU)
}
if nc.TiKVClient.StoreLimit != c.TiKVClient.StoreLimit {
storeutil.StoreLimit.Store(nc.TiKVClient.StoreLimit)
}
Expand Down Expand Up @@ -495,7 +491,6 @@ func setGlobalVars() {
statistics.FeedbackProbability.Store(cfg.Performance.FeedbackProbability)
handle.MaxQueryFeedbackCount.Store(int64(cfg.Performance.QueryFeedbackLimit))
statistics.RatioOfPseudoEstimate.Store(cfg.Performance.PseudoEstimateRatio)
util.EnablePProfSQLCPU.Store(cfg.Performance.PProfSQLCPU)
ddl.RunWorker = cfg.RunDDL
if cfg.SplitTable {
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
Expand Down
4 changes: 0 additions & 4 deletions util/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/util/logutil"
"github.com/uber-go/atomic"
"go.uber.org/zap"
)

Expand All @@ -42,9 +41,6 @@ const (
GCTimeFormat = "20060102-15:04:05 -0700"
)

// EnablePProfSQLCPU control whether collect pprof cpu in SQL level.
var EnablePProfSQLCPU = atomic.NewBool(false)

// RunWithRetry will run the f with backoff and retry.
// retryCnt: Max retry count
// backoff: When run f failed, it will sleep backoff * triedCount time.Millisecond.
Expand Down

0 comments on commit f0678b4

Please sign in to comment.