From d9f72f2950e319666e2e4833e8f9eff8a97b8000 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 28 Apr 2021 13:33:18 +0800 Subject: [PATCH 01/54] init Signed-off-by: crazycs --- sessionctx/variable/tidb_vars.go | 4 +- tidb-server/main.go | 7 ++ util/traceresource/manager.go | 110 +++++++++++++++++++++++++++++++ 3 files changed, 119 insertions(+), 2 deletions(-) create mode 100644 util/traceresource/manager.go diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 7fdccecb5a97e..b92f07887b73f 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -604,7 +604,7 @@ const ( DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB. DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 0 + DefTiDBPProfSQLCPU = 1 DefTiDBRetryLimit = 10 DefTiDBDisableTxnAutoRetry = true DefTiDBConstraintCheckInPlace = false @@ -680,7 +680,7 @@ const ( // Process global variables. var ( ProcessGeneralLog = atomic.NewBool(false) - EnablePProfSQLCPU = atomic.NewBool(false) + EnablePProfSQLCPU = atomic.NewBool(true) ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount maxDDLReorgWorkerCount int32 = 128 ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize diff --git a/tidb-server/main.go b/tidb-server/main.go index 05f4ecc59c25a..873aa63fcbb13 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -64,6 +64,7 @@ import ( "github.com/pingcap/tidb/util/sys/linux" storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" + "github.com/pingcap/tidb/util/traceresource" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" pd "github.com/tikv/pd/client" @@ -178,6 +179,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() + setProfiler() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) @@ -685,3 +687,8 @@ func stringToList(repairString string) []string { return r == ',' || r == ' ' || r == '"' }) } + +func setProfiler() { + sp := traceresource.NewStmtProfiler() + sp.Run() +} diff --git a/util/traceresource/manager.go b/util/traceresource/manager.go new file mode 100644 index 0000000000000..11ee1d6508662 --- /dev/null +++ b/util/traceresource/manager.go @@ -0,0 +1,110 @@ +package traceresource + +import ( + "bytes" + "fmt" + "github.com/google/pprof/profile" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" + "runtime/pprof" + "time" +) + +type StmtProfiler struct { + taskCh chan *bytes.Buffer + cacheBufCh chan *bytes.Buffer +} + +func NewStmtProfiler() *StmtProfiler { + return &StmtProfiler{ + taskCh: make(chan *bytes.Buffer, 128), + cacheBufCh: make(chan *bytes.Buffer, 128), + } +} + +func (sp *StmtProfiler) Run() { + logutil.BgLogger().Info("profiler started") + go sp.startCPUProfileWorker() + go sp.startAnalyzeProfileWorker() +} + +func (sp *StmtProfiler) startCPUProfileWorker() { + for { + buf := sp.getBuffer() + if err := pprof.StartCPUProfile(buf); err != nil { + return + } + sp.sleep(time.Second * 10) + pprof.StopCPUProfile() + sp.taskCh <- buf + } +} + +func (sp *StmtProfiler) startAnalyzeProfileWorker() { + var buf *bytes.Buffer + for { + buf = <-sp.taskCh + reader := bytes.NewReader(buf.Bytes()) + p, err := profile.Parse(reader) + if err != nil { + logutil.BgLogger().Error("parse profile error", zap.Error(err)) + continue + } + tagMap := sp.parseCPUProfileTags(p) + if len(tagMap) == 0 { + continue + } + fmt.Printf("-------- [ BEGIN ] ----------\n") + for k, tags := range tagMap { + fmt.Println(k) + for t, v := range tags { + fmt.Printf("%s : %s, %s\n", time.Duration(v), k, t) + } + } + fmt.Printf("-------- [ END ] ----------\n\n") + sp.putBuffer(buf) + } +} + +func (sp *StmtProfiler) getBuffer() *bytes.Buffer { + select { + case buf := <-sp.cacheBufCh: + buf.Reset() + return buf + default: + return bytes.NewBuffer(make([]byte, 0, 100*1024)) + } +} + +func (sp *StmtProfiler) putBuffer(buf *bytes.Buffer) { + select { + case sp.cacheBufCh <- buf: + default: + } +} + +func (sp *StmtProfiler) sleep(d time.Duration) { + timer := time.NewTimer(d) + select { + case <-timer.C: + } + timer.Stop() +} + +func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) map[string]map[string]int64 { + tagMap := make(map[string]map[string]int64) + idx := len(p.SampleType) - 1 + for _, s := range p.Sample { + for key, vals := range s.Label { + for _, val := range vals { + valueMap, ok := tagMap[key] + if !ok { + valueMap = make(map[string]int64) + tagMap[key] = valueMap + } + valueMap[val] += s.Value[idx] + } + } + } + return tagMap +} From a3e1207e564a47037441f4e8be595dafd2a22797 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 28 Apr 2021 14:10:29 +0800 Subject: [PATCH 02/54] refine output Signed-off-by: crazycs --- util/traceresource/manager.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/util/traceresource/manager.go b/util/traceresource/manager.go index 11ee1d6508662..9f0f084b25cc5 100644 --- a/util/traceresource/manager.go +++ b/util/traceresource/manager.go @@ -54,14 +54,16 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { if len(tagMap) == 0 { continue } - fmt.Printf("-------- [ BEGIN ] ----------\n") + logutil.BgLogger().Info("-------- [ BEGIN ] ----------") for k, tags := range tagMap { - fmt.Println(k) + if k != "sql" { + continue + } for t, v := range tags { - fmt.Printf("%s : %s, %s\n", time.Duration(v), k, t) + fmt.Printf("%s : %s\n", time.Duration(v), t) } } - fmt.Printf("-------- [ END ] ----------\n\n") + fmt.Printf("\n\n") sp.putBuffer(buf) } } From 37af9ee548e0a5826963dc3ef54c1e14958f6630 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 19 May 2021 12:35:48 +0800 Subject: [PATCH 03/54] add config and rename pkg name Signed-off-by: crazycs --- config/config.go | 15 ++++++++ server/conn.go | 2 +- sessionctx/variable/tidb_vars.go | 4 +-- tidb-server/main.go | 4 +-- util/{traceresource => tracecpu}/manager.go | 38 ++++++++++++--------- 5 files changed, 41 insertions(+), 22 deletions(-) rename util/{traceresource => tracecpu}/manager.go (81%) diff --git a/config/config.go b/config/config.go index 664cb5bceaa7d..10eafeb738458 100644 --- a/config/config.go +++ b/config/config.go @@ -137,6 +137,7 @@ 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"` + TopStmt TopStmt `toml:"top-stmt" json:"top-stmt"` // 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,6 +528,15 @@ type StmtSummary struct { HistorySize int `toml:"history-size" json:"history-size"` } +type TopStmt 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. @@ -656,6 +666,11 @@ var defaultConf = Config{ RefreshInterval: 1800, HistorySize: 24, }, + TopStmt: TopStmt{ + Enable: true, + RefreshInterval: 1, + MaxStmtCount: 5000, + }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, }, diff --git a/server/conn.go b/server/conn.go index 78cdd1a46c12d..c5ed0df16205e 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1026,7 +1026,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] - if variable.EnablePProfSQLCPU.Load() { + if variable.EnablePProfSQLCPU.Load() || config.GetGlobalConfig().TopStmt.Enable { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { defer pprof.SetGoroutineLabels(ctx) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index b92f07887b73f..7fdccecb5a97e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -604,7 +604,7 @@ const ( DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB. DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 1 + DefTiDBPProfSQLCPU = 0 DefTiDBRetryLimit = 10 DefTiDBDisableTxnAutoRetry = true DefTiDBConstraintCheckInPlace = false @@ -680,7 +680,7 @@ const ( // Process global variables. var ( ProcessGeneralLog = atomic.NewBool(false) - EnablePProfSQLCPU = atomic.NewBool(true) + EnablePProfSQLCPU = atomic.NewBool(false) ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount maxDDLReorgWorkerCount int32 = 128 ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize diff --git a/tidb-server/main.go b/tidb-server/main.go index 873aa63fcbb13..0b272ad9b8b9e 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -64,7 +64,7 @@ import ( "github.com/pingcap/tidb/util/sys/linux" storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" - "github.com/pingcap/tidb/util/traceresource" + "github.com/pingcap/tidb/util/tracecpu" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" pd "github.com/tikv/pd/client" @@ -689,6 +689,6 @@ func stringToList(repairString string) []string { } func setProfiler() { - sp := traceresource.NewStmtProfiler() + sp := tracecpu.NewStmtProfiler() sp.Run() } diff --git a/util/traceresource/manager.go b/util/tracecpu/manager.go similarity index 81% rename from util/traceresource/manager.go rename to util/tracecpu/manager.go index 9f0f084b25cc5..9e82419668f5c 100644 --- a/util/traceresource/manager.go +++ b/util/tracecpu/manager.go @@ -1,13 +1,15 @@ -package traceresource +package tracecpu import ( "bytes" "fmt" + "runtime/pprof" + "time" + "github.com/google/pprof/profile" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" - "runtime/pprof" - "time" ) type StmtProfiler struct { @@ -30,16 +32,26 @@ func (sp *StmtProfiler) Run() { func (sp *StmtProfiler) startCPUProfileWorker() { for { - buf := sp.getBuffer() - if err := pprof.StartCPUProfile(buf); err != nil { - return + cfg := config.GetGlobalConfig() + interval := time.Duration(cfg.TopStmt.RefreshInterval) * time.Second + if cfg.TopStmt.Enable { + sp.doCPUProfile(interval) + } else { + time.Sleep(interval) } - sp.sleep(time.Second * 10) - pprof.StopCPUProfile() - sp.taskCh <- buf } } +func (sp *StmtProfiler) doCPUProfile(interval time.Duration) { + buf := sp.getBuffer() + if err := pprof.StartCPUProfile(buf); err != nil { + return + } + time.Sleep(interval) + pprof.StopCPUProfile() + sp.taskCh <- buf +} + func (sp *StmtProfiler) startAnalyzeProfileWorker() { var buf *bytes.Buffer for { @@ -85,14 +97,6 @@ func (sp *StmtProfiler) putBuffer(buf *bytes.Buffer) { } } -func (sp *StmtProfiler) sleep(d time.Duration) { - timer := time.NewTimer(d) - select { - case <-timer.C: - } - timer.Stop() -} - func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) map[string]map[string]int64 { tagMap := make(map[string]map[string]int64) idx := len(p.SampleType) - 1 From ebbbf9e2c4ba1cc45c5256eacc072469b9fd3d21 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 19 May 2021 15:13:27 +0800 Subject: [PATCH 04/54] refine sleep interval Signed-off-by: crazycs --- util/tracecpu/manager.go | 58 ++++++++++++++++++++++++---------------- 1 file changed, 35 insertions(+), 23 deletions(-) diff --git a/util/tracecpu/manager.go b/util/tracecpu/manager.go index 9e82419668f5c..6091b38dae52f 100644 --- a/util/tracecpu/manager.go +++ b/util/tracecpu/manager.go @@ -13,14 +13,19 @@ import ( ) type StmtProfiler struct { - taskCh chan *bytes.Buffer - cacheBufCh chan *bytes.Buffer + taskCh chan *profileTask + cacheBufCh chan *profileTask +} + +type profileTask struct { + buf *bytes.Buffer + end int64 } func NewStmtProfiler() *StmtProfiler { return &StmtProfiler{ - taskCh: make(chan *bytes.Buffer, 128), - cacheBufCh: make(chan *bytes.Buffer, 128), + taskCh: make(chan *profileTask, 128), + cacheBufCh: make(chan *profileTask, 128), } } @@ -33,30 +38,34 @@ func (sp *StmtProfiler) Run() { func (sp *StmtProfiler) startCPUProfileWorker() { for { cfg := config.GetGlobalConfig() - interval := time.Duration(cfg.TopStmt.RefreshInterval) * time.Second if cfg.TopStmt.Enable { - sp.doCPUProfile(interval) + sp.doCPUProfile(cfg.TopStmt.RefreshInterval) } else { - time.Sleep(interval) + time.Sleep(time.Second) } } } -func (sp *StmtProfiler) doCPUProfile(interval time.Duration) { - buf := sp.getBuffer() - if err := pprof.StartCPUProfile(buf); err != nil { +func (sp *StmtProfiler) doCPUProfile(interval int) { + task := sp.newProfileTask() + if err := pprof.StartCPUProfile(task.buf); err != nil { return } - time.Sleep(interval) + ns := int(time.Second)*interval - time.Now().Nanosecond() + time.Sleep(time.Nanosecond * time.Duration(ns)) pprof.StopCPUProfile() - sp.taskCh <- buf + sp.sendProfileTask(task) +} + +func (sp *StmtProfiler) sendProfileTask(task *profileTask) { + task.end = time.Now().Unix() + sp.taskCh <- task } func (sp *StmtProfiler) startAnalyzeProfileWorker() { - var buf *bytes.Buffer for { - buf = <-sp.taskCh - reader := bytes.NewReader(buf.Bytes()) + task := <-sp.taskCh + reader := bytes.NewReader(task.buf.Bytes()) p, err := profile.Parse(reader) if err != nil { logutil.BgLogger().Error("parse profile error", zap.Error(err)) @@ -76,23 +85,26 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { } } fmt.Printf("\n\n") - sp.putBuffer(buf) + sp.putTaskToBuffer(task) } } -func (sp *StmtProfiler) getBuffer() *bytes.Buffer { +func (sp *StmtProfiler) newProfileTask() *profileTask { + var task *profileTask select { - case buf := <-sp.cacheBufCh: - buf.Reset() - return buf + case task = <-sp.cacheBufCh: + task.buf.Reset() default: - return bytes.NewBuffer(make([]byte, 0, 100*1024)) + task = &profileTask{ + buf: bytes.NewBuffer(make([]byte, 0, 100*1024)), + } } + return task } -func (sp *StmtProfiler) putBuffer(buf *bytes.Buffer) { +func (sp *StmtProfiler) putTaskToBuffer(task *profileTask) { select { - case sp.cacheBufCh <- buf: + case sp.cacheBufCh <- task: default: } } From 4d6d2ff5ea6575b9f30f33387b3737f237daf331 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 19 May 2021 19:11:15 +0800 Subject: [PATCH 05/54] add plan digest Signed-off-by: crazycs --- executor/adapter.go | 30 +++++---- planner/core/common_plans.go | 3 - server/conn.go | 37 +++++++---- session/session.go | 12 +++- sessionctx/stmtctx/stmtctx.go | 10 --- util/tracecpu/{manager.go => profile.go} | 85 +++++++++++++++++++----- 6 files changed, 117 insertions(+), 60 deletions(-) rename util/tracecpu/{manager.go => profile.go} (59%) diff --git a/executor/adapter.go b/executor/adapter.go index 784696996cb94..9dad882df46f8 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -20,6 +20,7 @@ import ( "math" "runtime/trace" "strings" + "sync" "sync/atomic" "time" @@ -198,6 +199,11 @@ type ExecStmt struct { // OutputNames will be set if using cached plan OutputNames []*types.FieldName PsStmt *plannercore.CachedPrepareStmt + // cache for plan digest and normalized + planDigestMemo struct { + sync.Once + digest string + } } // PointGet short path for point exec directly from plan, keep only necessary steps @@ -868,6 +874,13 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { } } +func (a *ExecStmt) PlanDigest() string { + a.planDigestMemo.Do(func() { + _, a.planDigestMemo.digest = plannercore.NormalizePlan(a.Plan) + }) + return a.planDigestMemo.digest +} + // LogSlowQuery is used to print the slow query in the log files. func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { sessVars := a.Ctx.GetSessionVars() @@ -919,7 +932,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diskMax := sessVars.StmtCtx.DiskTracker.MaxConsumed() - _, planDigest := getPlanDigest(a.Ctx, a.Plan) + planDigest := a.PlanDigest() slowItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql.String(), @@ -1010,17 +1023,6 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { return variable.SlowLogPlanPrefix + planTree + variable.SlowLogPlanSuffix } -// getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. -func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (normalized, planDigest string) { - normalized, planDigest = sctx.GetSessionVars().StmtCtx.GetPlanDigest() - if len(normalized) > 0 { - return - } - normalized, planDigest = plannercore.NormalizePlan(p) - sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) - return -} - // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. func getEncodedPlan(sctx sessionctx.Context, p plannercore.Plan, genHint bool, n ast.StmtNode) (encodedPlan, hintStr string) { var hintSet bool @@ -1092,11 +1094,11 @@ func (a *ExecStmt) SummaryStmt(succ bool) { var planDigestGen func() string if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { - _, planDigest := getPlanDigest(a.Ctx, a.Plan) + planDigest := a.PlanDigest() return planDigest } } else { - _, planDigest = getPlanDigest(a.Ctx, a.Plan) + planDigest = a.PlanDigest() } execDetail := stmtCtx.GetExecDetails() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 3818486955646..2d93a70707ef0 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -376,7 +376,6 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } e.names = cachedVal.OutPutNames e.Plan = cachedVal.Plan - stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) return nil } break @@ -406,7 +405,6 @@ REBUILD: } cached := NewPSTMTPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) - stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { hitVal := false for i, cacheVal := range cacheVals.([]*PSTMTPlanCacheValue) { @@ -466,7 +464,6 @@ func (e *Execute) tryCachePointPlan(ctx context.Context, sctx sessionctx.Context prepared.CachedPlan = p prepared.CachedNames = names preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) - sctx.GetSessionVars().StmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) } return err } diff --git a/server/conn.go b/server/conn.go index c5ed0df16205e..b4d697e0676a5 100644 --- a/server/conn.go +++ b/server/conn.go @@ -85,6 +85,7 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/tracecpu" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -1026,17 +1027,24 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] - if variable.EnablePProfSQLCPU.Load() || config.GetGlobalConfig().TopStmt.Enable { - label := getLastStmtInConn{cc}.PProfLabel() - if len(label) > 0 { + if config.GetGlobalConfig().TopStmt.Enable { + normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabel() + if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) - ctx = pprof.WithLabels(ctx, pprof.Labels("sql", label)) + ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) + pprof.SetGoroutineLabels(ctx) + } + } else if variable.EnablePProfSQLCPU.Load() { + normalizedSQL, _ := getLastStmtInConn{cc}.PProfLabel() + if len(normalizedSQL) > 0 { + defer pprof.SetGoroutineLabels(ctx) + ctx = pprof.WithLabels(ctx, pprof.Labels("sql", normalizedSQL)) pprof.SetGoroutineLabels(ctx) } } if trace.IsEnabled() { lc := getLastStmtInConn{cc} - sqlType := lc.PProfLabel() + sqlType, _ := lc.PProfLabel() if len(sqlType) > 0 { var task *trace.Task ctx, task = trace.NewTask(ctx, sqlType) @@ -2142,26 +2150,27 @@ func (cc getLastStmtInConn) String() string { } // PProfLabel return sql label used to tag pprof. -func (cc getLastStmtInConn) PProfLabel() string { +func (cc getLastStmtInConn) PProfLabel() (string, string) { if len(cc.lastPacket) == 0 { - return "" + return "", "" } cmd, data := cc.lastPacket[0], cc.lastPacket[1:] switch cmd { case mysql.ComInitDB: - return "UseDB" + return "UseDB", "" case mysql.ComFieldList: - return "ListFields" + return "ListFields", "" case mysql.ComStmtClose: - return "CloseStmt" + return "CloseStmt", "" case mysql.ComStmtReset: - return "ResetStmt" + return "ResetStmt", "" case mysql.ComQuery, mysql.ComStmtPrepare: - return parser.Normalize(queryStrForLog(string(hack.String(data)))) + return parser.NormalizeDigest(queryStrForLog(string(hack.String(data)))) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return queryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) + str := cc.preparedStmt2StringNoArgs(stmtID) + return queryStrForLog(str), parser.DigestNormalized(str) default: - return "" + return "", "" } } diff --git a/session/session.go b/session/session.go index efd6706c4ffb3..e2d71a98f87a0 100644 --- a/session/session.go +++ b/session/session.go @@ -23,6 +23,7 @@ import ( "encoding/json" "fmt" "net" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -41,6 +42,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" @@ -1601,6 +1603,15 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } + + if es, ok := s.(*executor.ExecStmt); ok && es.Plan != nil { + planDigest := es.PlanDigest() + if len(planDigest) > 0 { + ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelPlanDigest, planDigest)) + pprof.SetGoroutineLabels(ctx) + } + } + se.SetValue(sessionctx.QueryString, s.OriginText()) if _, ok := s.(*executor.ExecStmt).StmtNode.(ast.DDLNode); ok { se.SetValue(sessionctx.LastExecuteDDL, true) @@ -1774,7 +1785,6 @@ func (s *session) cachedPlanExec(ctx context.Context, stmt.Text = prepared.Stmt.Text() stmtCtx.OriginalSQL = stmt.Text stmtCtx.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) - stmtCtx.SetPlanDigest(prepareStmt.NormalizedPlan, prepareStmt.PlanDigest) logQuery(stmt.GetTextToLog(), s.sessionVars) if !s.isInternal() && config.GetGlobalConfig().EnableTelemetry { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d8a75aec48610..456b2f20ac08b 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -243,16 +243,6 @@ func (sc *StatementContext) InitSQLDigest(normalized, digest string) { }) } -// GetPlanDigest gets the normalized plan and plan digest. -func (sc *StatementContext) GetPlanDigest() (normalized, planDigest string) { - return sc.planNormalized, sc.planDigest -} - -// SetPlanDigest sets the normalized plan and plan digest. -func (sc *StatementContext) SetPlanDigest(normalized, planDigest string) { - sc.planNormalized, sc.planDigest = normalized, planDigest -} - // GetEncodedPlan gets the encoded plan, it is used to avoid repeated encode. func (sc *StatementContext) GetEncodedPlan() string { return sc.encodedPlan diff --git a/util/tracecpu/manager.go b/util/tracecpu/profile.go similarity index 59% rename from util/tracecpu/manager.go rename to util/tracecpu/profile.go index 6091b38dae52f..f8ce3d1fc9280 100644 --- a/util/tracecpu/manager.go +++ b/util/tracecpu/profile.go @@ -12,6 +12,12 @@ import ( "go.uber.org/zap" ) +const ( + LabelSQL = "sql" + LabelSQLDigest = "sql_digest" + LabelPlanDigest = "plan_digest" +) + type StmtProfiler struct { taskCh chan *profileTask cacheBufCh chan *profileTask @@ -71,17 +77,15 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { logutil.BgLogger().Error("parse profile error", zap.Error(err)) continue } - tagMap := sp.parseCPUProfileTags(p) - if len(tagMap) == 0 { + stmtMap := sp.parseCPUProfileTags(p) + if len(stmtMap) == 0 { continue } logutil.BgLogger().Info("-------- [ BEGIN ] ----------") - for k, tags := range tagMap { - if k != "sql" { - continue - } - for t, v := range tags { - fmt.Printf("%s : %s\n", time.Duration(v), t) + for _, stmt := range stmtMap { + fmt.Printf("%s\n", stmt.normalizedSQL) + for p, v := range stmt.plans { + fmt.Printf("\t %s : %s\n", p, time.Duration(v)) } } fmt.Printf("\n\n") @@ -109,20 +113,65 @@ func (sp *StmtProfiler) putTaskToBuffer(task *profileTask) { } } -func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) map[string]map[string]int64 { - tagMap := make(map[string]map[string]int64) +func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) (stmtMap map[string]*stmtStats) { + stmtMap = make(map[string]*stmtStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { - for key, vals := range s.Label { - for _, val := range vals { - valueMap, ok := tagMap[key] - if !ok { - valueMap = make(map[string]int64) - tagMap[key] = valueMap + digests, ok := s.Label[LabelSQLDigest] + if !ok || len(digests) == 0 { + continue + } + sqls, ok := s.Label[LabelSQL] + if !ok || len(sqls) != len(digests) { + continue + } + for i, digest := range digests { + stmt, ok := stmtMap[digest] + if !ok { + stmt = &stmtStats{ + plans: make(map[string]int64), + total: 0, + isInternal: false, + normalizedSQL: sqls[i], } - valueMap[val] += s.Value[idx] + stmtMap[digest] = stmt + } + stmt.total += s.Value[idx] + + plans := s.Label[LabelPlanDigest] + for _, plan := range plans { + stmt.plans[plan] += s.Value[idx] } } } - return tagMap + for _, stmt := range stmtMap { + stmt.tune() + } + return stmtMap +} + +type stmtStats struct { + plans map[string]int64 + total int64 + isInternal bool + normalizedSQL string +} + +// tune use to adjust stats +func (s *stmtStats) tune() { + if len(s.plans) == 0 { + s.plans[""] = s.total + return + } + planTotal := int64(0) + for _, v := range s.plans { + planTotal += v + } + remain := s.total - planTotal + if remain <= 0 { + return + } + for k, v := range s.plans { + s.plans[k] = v + (v/planTotal)*remain + } } From 8754bebdc21b0ad23cddc9830529ac15b99c4662 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 19 May 2021 20:54:35 +0800 Subject: [PATCH 06/54] fix bug for execute prepare statements Signed-off-by: crazycs --- ddl/db_change_test.go | 2 +- executor/executor.go | 23 +++++++++++++++++------ executor/prepared.go | 4 ++-- planner/core/cbo_test.go | 2 +- planner/core/physical_plan_test.go | 2 +- planner/core/prepare_test.go | 2 +- server/conn.go | 4 ++-- session/session.go | 4 ++-- session/session_test.go | 2 +- util/tracecpu/profile.go | 8 ++++---- 10 files changed, 32 insertions(+), 21 deletions(-) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 041f35c7734a8..a15d09d191e9a 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -415,7 +415,7 @@ func (t *testExecInfo) compileSQL(idx int) (err error) { ctx := context.TODO() se.PrepareTxnCtx(ctx) sctx := se.(sessionctx.Context) - if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil { + if err = executor.ResetContextOfStmt(context.Background(),sctx, c.rawStmt); err != nil { return errors.Trace(err) } c.stmt, err = compiler.Compile(ctx, c.rawStmt) diff --git a/executor/executor.go b/executor/executor.go index 1666f6955bba9..e9b6c0d9e6055 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,6 +18,7 @@ import ( "fmt" "math" "runtime" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -63,6 +64,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" + "github.com/pingcap/tidb/util/tracecpu" "go.uber.org/zap" ) @@ -1591,8 +1593,8 @@ func (e *UnionExec) Close() error { // ResetContextOfStmt resets the StmtContext and session variables. // Before every execution, we must clear statement context. -func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { - vars := ctx.GetSessionVars() +func ResetContextOfStmt(ctx context.Context, sctx sessionctx.Context, s ast.StmtNode) (err error) { + vars := sctx.GetSessionVars() sc := &stmtctx.StatementContext{ TimeZone: vars.Location(), MemTracker: memory.NewTracker(memory.LabelForSQLText, vars.MemQuotaQuery), @@ -1606,14 +1608,14 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { } switch globalConfig.OOMAction { case config.OOMActionCancel: - action := &memory.PanicOnExceed{ConnID: ctx.GetSessionVars().ConnectionID} - action.SetLogHook(domain.GetDomain(ctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) + action := &memory.PanicOnExceed{ConnID: sctx.GetSessionVars().ConnectionID} + action.SetLogHook(domain.GetDomain(sctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) sc.MemTracker.SetActionOnExceed(action) case config.OOMActionLog: fallthrough default: - action := &memory.LogOnExceed{ConnID: ctx.GetSessionVars().ConnectionID} - action.SetLogHook(domain.GetDomain(ctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) + action := &memory.LogOnExceed{ConnID: sctx.GetSessionVars().ConnectionID} + action.SetLogHook(domain.GetDomain(sctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) sc.MemTracker.SetActionOnExceed(action) } if execStmt, ok := s.(*ast.ExecuteStmt); ok { @@ -1621,6 +1623,15 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { if err != nil { return } + if config.GetGlobalConfig().TopStmt.Enable { + sc.OriginalSQL = s.Text() + sc.SQLDigest() + normalizedSQL, digest := sc.SQLDigest() + if len(normalizedSQL) > 0 { + ctx = pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) + pprof.SetGoroutineLabels(ctx) + } + } } // execute missed stmtID uses empty sql sc.OriginalSQL = s.Text() diff --git a/executor/prepared.go b/executor/prepared.go index 448ee3b7fdc66..55138ff25d9c2 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -135,7 +135,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } stmt := stmts[0] - err = ResetContextOfStmt(e.ctx, stmt) + err = ResetContextOfStmt(ctx,e.ctx, stmt) if err != nil { return err } @@ -316,7 +316,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, sctx.GetSessionVars().DurationCompile = time.Since(startTime) }() execStmt := &ast.ExecuteStmt{ExecID: ID} - if err := ResetContextOfStmt(sctx, execStmt); err != nil { + if err := ResetContextOfStmt(ctx,sctx, execStmt); err != nil { return nil, false, false, err } execStmt.BinaryArgs = args diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 822e52a0a0254..398f5fe398bc4 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -400,7 +400,7 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - err = executor.ResetContextOfStmt(ctx, stmt) + err = executor.ResetContextOfStmt(context.Background(),ctx, stmt) c.Assert(err, IsNil) is := domain.GetDomain(ctx).InfoSchema() err = core.Preprocess(ctx, stmt, is) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 6a8993286e36a..fa9574820d011 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1292,7 +1292,7 @@ func (s *testPlanSuite) TestIndexMergeHint(c *C) { stmt, err := s.ParseOneStmt(test, "", "") c.Assert(err, IsNil, comment) sctx := se.(sessionctx.Context) - err = executor.ResetContextOfStmt(sctx, stmt) + err = executor.ResetContextOfStmt(context.Background(),sctx, stmt) c.Assert(err, IsNil) p, _, err := planner.Optimize(ctx, se, stmt, s.is) c.Assert(err, IsNil) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index d6bfe69f82b39..5c8b7abdb11fa 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -207,7 +207,7 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { c.Check(err, IsNil) execPlan, ok := p.(*core.Execute) c.Check(ok, IsTrue) - err = executor.ResetContextOfStmt(tk.Se, stmt) + err = executor.ResetContextOfStmt(context.Background(),tk.Se, stmt) c.Assert(err, IsNil) err = execPlan.OptimizePreparedPlan(ctx, tk.Se, is) c.Check(err, IsNil) diff --git a/server/conn.go b/server/conn.go index b4d697e0676a5..d9a8f4d148523 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2165,11 +2165,11 @@ func (cc getLastStmtInConn) PProfLabel() (string, string) { case mysql.ComStmtReset: return "ResetStmt", "" case mysql.ComQuery, mysql.ComStmtPrepare: - return parser.NormalizeDigest(queryStrForLog(string(hack.String(data)))) + return parser.NormalizeDigest(string(hack.String(data))) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) str := cc.preparedStmt2StringNoArgs(stmtID) - return queryStrForLog(str), parser.DigestNormalized(str) + return parser.NormalizeDigest(str) default: return "", "" } diff --git a/session/session.go b/session/session.go index e2d71a98f87a0..6af6643ca38e9 100644 --- a/session/session.go +++ b/session/session.go @@ -1492,7 +1492,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex s.sessionVars.StartTime = time.Now() // Some executions are done in compile stage, so we reset them before compile. - if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { + if err := executor.ResetContextOfStmt(ctx, s, stmtNode); err != nil { return nil, err } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1759,7 +1759,7 @@ func (s *session) cachedPlanExec(ctx context.Context, is = s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) } execAst := &ast.ExecuteStmt{ExecID: stmtID} - if err := executor.ResetContextOfStmt(s, execAst); err != nil { + if err := executor.ResetContextOfStmt(ctx, s, execAst); err != nil { return nil, err } execAst.BinaryArgs = args diff --git a/session/session_test.go b/session/session_test.go index df2a167921e56..38cfd31e6f036 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -735,7 +735,7 @@ func (s *testSessionSuite) TestRetryCleanTxn(c *C) { c.Assert(err, IsNil) compiler := executor.Compiler{Ctx: tk.Se} stmt, _ := compiler.Compile(context.TODO(), stmtNode) - executor.ResetContextOfStmt(tk.Se, stmtNode) + executor.ResetContextOfStmt(context.Background(), tk.Se, stmtNode) history.Add(stmt, tk.Se.GetSessionVars().StmtCtx) _, err = tk.Exec("commit") c.Assert(err, NotNil) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index f8ce3d1fc9280..d3f6f06e9fa8b 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -82,13 +82,13 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { continue } logutil.BgLogger().Info("-------- [ BEGIN ] ----------") - for _, stmt := range stmtMap { - fmt.Printf("%s\n", stmt.normalizedSQL) + for digest, stmt := range stmtMap { + logutil.BgLogger().Info(fmt.Sprintf("%s , %v", stmt.normalizedSQL, digest)) for p, v := range stmt.plans { - fmt.Printf("\t %s : %s\n", p, time.Duration(v)) + logutil.BgLogger().Info(fmt.Sprintf(" %s : %s", p, time.Duration(v))) } } - fmt.Printf("\n\n") + logutil.BgLogger().Info("------") sp.putTaskToBuffer(task) } } From 469d67cc41502e87add307528a914d03c15a9f78 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 20 May 2021 11:50:45 +0800 Subject: [PATCH 07/54] fix exec stmt plan digest bug Signed-off-by: crazycs --- ddl/db_change_test.go | 2 +- executor/adapter.go | 31 ++++++++++++++++++++---------- executor/executor.go | 16 +++++++-------- executor/prepared.go | 4 ++-- planner/core/cbo_test.go | 2 +- planner/core/physical_plan_test.go | 2 +- planner/core/prepare_test.go | 2 +- session/session.go | 14 ++------------ session/session_test.go | 2 +- util/tracecpu/profile.go | 3 +-- 10 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index a15d09d191e9a..041f35c7734a8 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -415,7 +415,7 @@ func (t *testExecInfo) compileSQL(idx int) (err error) { ctx := context.TODO() se.PrepareTxnCtx(ctx) sctx := se.(sessionctx.Context) - if err = executor.ResetContextOfStmt(context.Background(),sctx, c.rawStmt); err != nil { + if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil { return errors.Trace(err) } c.stmt, err = compiler.Compile(ctx, c.rawStmt) diff --git a/executor/adapter.go b/executor/adapter.go index 9dad882df46f8..3044fb9b1bf51 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -18,9 +18,9 @@ import ( "context" "fmt" "math" + "runtime/pprof" "runtime/trace" "strings" - "sync" "sync/atomic" "time" @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tracecpu" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -199,11 +200,8 @@ type ExecStmt struct { // OutputNames will be set if using cached plan OutputNames []*types.FieldName PsStmt *plannercore.CachedPrepareStmt - // cache for plan digest and normalized - planDigestMemo struct { - sync.Once - digest string - } + // cache for plan digest + planDigest string } // PointGet short path for point exec directly from plan, keep only necessary steps @@ -339,6 +337,19 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } + if config.GetGlobalConfig().TopStmt.Enable && a.Plan != nil { + // ExecuteExec will rewrite `a.Plan`, so goroutine set label should be executed after `a.buildExecutor`. + normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() + planDigest := a.PlanDigest() + if len(planDigest) > 0 { + ctx = pprof.WithLabels(ctx, pprof.Labels( + tracecpu.LabelSQL, normalizedSQL, + tracecpu.LabelSQLDigest, sqlDigest, + tracecpu.LabelPlanDigest, planDigest)) + pprof.SetGoroutineLabels(ctx) + } + } + if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -875,10 +886,10 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { } func (a *ExecStmt) PlanDigest() string { - a.planDigestMemo.Do(func() { - _, a.planDigestMemo.digest = plannercore.NormalizePlan(a.Plan) - }) - return a.planDigestMemo.digest + if len(a.planDigest) == 0 { + _, a.planDigest = plannercore.NormalizePlan(a.Plan) + } + return a.planDigest } // LogSlowQuery is used to print the slow query in the log files. diff --git a/executor/executor.go b/executor/executor.go index e9b6c0d9e6055..2b1d48b44cfd7 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1593,8 +1593,8 @@ func (e *UnionExec) Close() error { // ResetContextOfStmt resets the StmtContext and session variables. // Before every execution, we must clear statement context. -func ResetContextOfStmt(ctx context.Context, sctx sessionctx.Context, s ast.StmtNode) (err error) { - vars := sctx.GetSessionVars() +func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { + vars := ctx.GetSessionVars() sc := &stmtctx.StatementContext{ TimeZone: vars.Location(), MemTracker: memory.NewTracker(memory.LabelForSQLText, vars.MemQuotaQuery), @@ -1608,14 +1608,14 @@ func ResetContextOfStmt(ctx context.Context, sctx sessionctx.Context, s ast.Stmt } switch globalConfig.OOMAction { case config.OOMActionCancel: - action := &memory.PanicOnExceed{ConnID: sctx.GetSessionVars().ConnectionID} - action.SetLogHook(domain.GetDomain(sctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) + action := &memory.PanicOnExceed{ConnID: ctx.GetSessionVars().ConnectionID} + action.SetLogHook(domain.GetDomain(ctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) sc.MemTracker.SetActionOnExceed(action) case config.OOMActionLog: fallthrough default: - action := &memory.LogOnExceed{ConnID: sctx.GetSessionVars().ConnectionID} - action.SetLogHook(domain.GetDomain(sctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) + action := &memory.LogOnExceed{ConnID: ctx.GetSessionVars().ConnectionID} + action.SetLogHook(domain.GetDomain(ctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota) sc.MemTracker.SetActionOnExceed(action) } if execStmt, ok := s.(*ast.ExecuteStmt); ok { @@ -1628,8 +1628,8 @@ func ResetContextOfStmt(ctx context.Context, sctx sessionctx.Context, s ast.Stmt sc.SQLDigest() normalizedSQL, digest := sc.SQLDigest() if len(normalizedSQL) > 0 { - ctx = pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) - pprof.SetGoroutineLabels(ctx) + goctx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) + pprof.SetGoroutineLabels(goctx) } } } diff --git a/executor/prepared.go b/executor/prepared.go index 55138ff25d9c2..448ee3b7fdc66 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -135,7 +135,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } stmt := stmts[0] - err = ResetContextOfStmt(ctx,e.ctx, stmt) + err = ResetContextOfStmt(e.ctx, stmt) if err != nil { return err } @@ -316,7 +316,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, sctx.GetSessionVars().DurationCompile = time.Since(startTime) }() execStmt := &ast.ExecuteStmt{ExecID: ID} - if err := ResetContextOfStmt(ctx,sctx, execStmt); err != nil { + if err := ResetContextOfStmt(sctx, execStmt); err != nil { return nil, false, false, err } execStmt.BinaryArgs = args diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 398f5fe398bc4..822e52a0a0254 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -400,7 +400,7 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - err = executor.ResetContextOfStmt(context.Background(),ctx, stmt) + err = executor.ResetContextOfStmt(ctx, stmt) c.Assert(err, IsNil) is := domain.GetDomain(ctx).InfoSchema() err = core.Preprocess(ctx, stmt, is) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index fa9574820d011..6a8993286e36a 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1292,7 +1292,7 @@ func (s *testPlanSuite) TestIndexMergeHint(c *C) { stmt, err := s.ParseOneStmt(test, "", "") c.Assert(err, IsNil, comment) sctx := se.(sessionctx.Context) - err = executor.ResetContextOfStmt(context.Background(),sctx, stmt) + err = executor.ResetContextOfStmt(sctx, stmt) c.Assert(err, IsNil) p, _, err := planner.Optimize(ctx, se, stmt, s.is) c.Assert(err, IsNil) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 5c8b7abdb11fa..d6bfe69f82b39 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -207,7 +207,7 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { c.Check(err, IsNil) execPlan, ok := p.(*core.Execute) c.Check(ok, IsTrue) - err = executor.ResetContextOfStmt(context.Background(),tk.Se, stmt) + err = executor.ResetContextOfStmt(tk.Se, stmt) c.Assert(err, IsNil) err = execPlan.OptimizePreparedPlan(ctx, tk.Se, is) c.Check(err, IsNil) diff --git a/session/session.go b/session/session.go index 6af6643ca38e9..a1a723dca3cb3 100644 --- a/session/session.go +++ b/session/session.go @@ -23,7 +23,6 @@ import ( "encoding/json" "fmt" "net" - "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -42,7 +41,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" @@ -1492,7 +1490,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex s.sessionVars.StartTime = time.Now() // Some executions are done in compile stage, so we reset them before compile. - if err := executor.ResetContextOfStmt(ctx, s, stmtNode); err != nil { + if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { return nil, err } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1604,14 +1602,6 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. ctx = opentracing.ContextWithSpan(ctx, span1) } - if es, ok := s.(*executor.ExecStmt); ok && es.Plan != nil { - planDigest := es.PlanDigest() - if len(planDigest) > 0 { - ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelPlanDigest, planDigest)) - pprof.SetGoroutineLabels(ctx) - } - } - se.SetValue(sessionctx.QueryString, s.OriginText()) if _, ok := s.(*executor.ExecStmt).StmtNode.(ast.DDLNode); ok { se.SetValue(sessionctx.LastExecuteDDL, true) @@ -1759,7 +1749,7 @@ func (s *session) cachedPlanExec(ctx context.Context, is = s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) } execAst := &ast.ExecuteStmt{ExecID: stmtID} - if err := executor.ResetContextOfStmt(ctx, s, execAst); err != nil { + if err := executor.ResetContextOfStmt(s, execAst); err != nil { return nil, err } execAst.BinaryArgs = args diff --git a/session/session_test.go b/session/session_test.go index 38cfd31e6f036..df2a167921e56 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -735,7 +735,7 @@ func (s *testSessionSuite) TestRetryCleanTxn(c *C) { c.Assert(err, IsNil) compiler := executor.Compiler{Ctx: tk.Se} stmt, _ := compiler.Compile(context.TODO(), stmtNode) - executor.ResetContextOfStmt(context.Background(), tk.Se, stmtNode) + executor.ResetContextOfStmt(tk.Se, stmtNode) history.Add(stmt, tk.Se.GetSessionVars().StmtCtx) _, err = tk.Exec("commit") c.Assert(err, NotNil) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index d3f6f06e9fa8b..d48b4e267ee5d 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -85,10 +85,9 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { for digest, stmt := range stmtMap { logutil.BgLogger().Info(fmt.Sprintf("%s , %v", stmt.normalizedSQL, digest)) for p, v := range stmt.plans { - logutil.BgLogger().Info(fmt.Sprintf(" %s : %s", p, time.Duration(v))) + logutil.BgLogger().Info(fmt.Sprintf(" %s : %s", time.Duration(v), p)) } } - logutil.BgLogger().Info("------") sp.putTaskToBuffer(task) } } From 037f85447454bd0b4cc626667f7fc6d58e9d7cf5 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 20 May 2021 12:08:54 +0800 Subject: [PATCH 08/54] add debug config to output profile Signed-off-by: crazycs --- config/config.go | 3 +++ executor/adapter.go | 10 +++++----- session/session.go | 1 - sessionctx/stmtctx/stmtctx.go | 3 --- util/tracecpu/profile.go | 7 +++++++ 5 files changed, 15 insertions(+), 9 deletions(-) diff --git a/config/config.go b/config/config.go index 10eafeb738458..f83115f20303b 100644 --- a/config/config.go +++ b/config/config.go @@ -535,6 +535,8 @@ type TopStmt struct { 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"` + + Debug bool } // IsolationRead is the config for isolation read. @@ -670,6 +672,7 @@ var defaultConf = Config{ Enable: true, RefreshInterval: 1, MaxStmtCount: 5000, + Debug: true, }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, diff --git a/executor/adapter.go b/executor/adapter.go index 3044fb9b1bf51..863c45a7a7fe7 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -340,7 +340,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if config.GetGlobalConfig().TopStmt.Enable && a.Plan != nil { // ExecuteExec will rewrite `a.Plan`, so goroutine set label should be executed after `a.buildExecutor`. normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() - planDigest := a.PlanDigest() + planDigest := a.getPlanDigest() if len(planDigest) > 0 { ctx = pprof.WithLabels(ctx, pprof.Labels( tracecpu.LabelSQL, normalizedSQL, @@ -885,7 +885,7 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { } } -func (a *ExecStmt) PlanDigest() string { +func (a *ExecStmt) getPlanDigest() string { if len(a.planDigest) == 0 { _, a.planDigest = plannercore.NormalizePlan(a.Plan) } @@ -943,7 +943,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diskMax := sessVars.StmtCtx.DiskTracker.MaxConsumed() - planDigest := a.PlanDigest() + planDigest := a.getPlanDigest() slowItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql.String(), @@ -1105,11 +1105,11 @@ func (a *ExecStmt) SummaryStmt(succ bool) { var planDigestGen func() string if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { - planDigest := a.PlanDigest() + planDigest := a.getPlanDigest() return planDigest } } else { - planDigest = a.PlanDigest() + planDigest = a.getPlanDigest() } execDetail := stmtCtx.GetExecDetails() diff --git a/session/session.go b/session/session.go index a1a723dca3cb3..b0626a20b5ca1 100644 --- a/session/session.go +++ b/session/session.go @@ -1601,7 +1601,6 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - se.SetValue(sessionctx.QueryString, s.OriginText()) if _, ok := s.(*executor.ExecStmt).StmtNode.(ast.DDLNode); ok { se.SetValue(sessionctx.LastExecuteDDL, true) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 456b2f20ac08b..49619d7a48092 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -147,9 +147,6 @@ type StatementContext struct { normalized string digest string } - // planNormalized use for cache the normalized plan, avoid duplicate builds. - planNormalized string - planDigest string encodedPlan string planHint string planHintSet bool diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index d48b4e267ee5d..7a77d7937badc 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -3,7 +3,9 @@ package tracecpu import ( "bytes" "fmt" + "io/ioutil" "runtime/pprof" + "strconv" "time" "github.com/google/pprof/profile" @@ -81,13 +83,18 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { if len(stmtMap) == 0 { continue } + total := int64(0) logutil.BgLogger().Info("-------- [ BEGIN ] ----------") for digest, stmt := range stmtMap { logutil.BgLogger().Info(fmt.Sprintf("%s , %v", stmt.normalizedSQL, digest)) for p, v := range stmt.plans { logutil.BgLogger().Info(fmt.Sprintf(" %s : %s", time.Duration(v), p)) + total += v } } + if config.GetGlobalConfig().TopStmt.Debug && total > (500*int64(time.Millisecond)) { + ioutil.WriteFile("cpu.profile."+strconv.Itoa(int(task.end)), task.buf.Bytes(), 0644) + } sp.putTaskToBuffer(task) } } From 4628d01cd7b57a57bab123106ba3f494991ca8d1 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 20 May 2021 16:14:12 +0800 Subject: [PATCH 09/54] fix pprof profile http api Signed-off-by: crazycs --- config/config.go | 3 - server/http_status.go | 11 ++- tidb-server/main.go | 3 +- util/tracecpu/profile.go | 164 ++++++++++++++++++++++++++++++++++++--- 4 files changed, 164 insertions(+), 17 deletions(-) diff --git a/config/config.go b/config/config.go index f83115f20303b..10eafeb738458 100644 --- a/config/config.go +++ b/config/config.go @@ -535,8 +535,6 @@ type TopStmt struct { 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"` - - Debug bool } // IsolationRead is the config for isolation read. @@ -672,7 +670,6 @@ var defaultConf = Config{ Enable: true, RefreshInterval: 1, MaxStmtCount: 5000, - Debug: true, }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, diff --git a/server/http_status.go b/server/http_status.go index b385ea0c45890..4ab190994ea86 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tidb/util/versioninfo" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" @@ -184,7 +185,7 @@ func (s *Server) startHTTPServer() { serverMux.HandleFunc("/debug/pprof/", pprof.Index) serverMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - serverMux.HandleFunc("/debug/pprof/profile", pprof.Profile) + serverMux.HandleFunc("/debug/pprof/profile", tracecpu.ProfileHTTPHandler) serverMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) serverMux.HandleFunc("/debug/pprof/trace", pprof.Trace) serverMux.HandleFunc("/debug/gogc", func(w http.ResponseWriter, r *http.Request) { @@ -251,7 +252,7 @@ func (s *Server) startHTTPServer() { serveError(w, http.StatusInternalServerError, fmt.Sprintf("Create zipped %s fail: %v", "profile", err)) return } - if err := rpprof.StartCPUProfile(fw); err != nil { + if err := tracecpu.StartCPUProfile(fw); err != nil { serveError(w, http.StatusInternalServerError, fmt.Sprintf("Could not enable CPU profiling: %s", err)) return @@ -261,7 +262,11 @@ func (s *Server) startHTTPServer() { sec = 10 } sleepWithCtx(r.Context(), time.Duration(sec)*time.Second) - rpprof.StopCPUProfile() + err = tracecpu.StopCPUProfile() + if err != nil { + serveError(w, http.StatusInternalServerError, fmt.Sprintf("Create zipped %s fail: %v", "config", err)) + return + } // dump config fw, err = zw.Create("config") diff --git a/tidb-server/main.go b/tidb-server/main.go index 0b272ad9b8b9e..65758f8b544f5 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -689,6 +689,5 @@ func stringToList(repairString string) []string { } func setProfiler() { - sp := tracecpu.NewStmtProfiler() - sp.Run() + tracecpu.GlobalStmtProfiler.Run() } diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 7a77d7937badc..c96aa5f935f57 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -2,14 +2,18 @@ package tracecpu import ( "bytes" + "errors" "fmt" - "io/ioutil" + "io" + "net/http" "runtime/pprof" "strconv" + "sync" "time" "github.com/google/pprof/profile" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -20,9 +24,16 @@ const ( LabelPlanDigest = "plan_digest" ) +var GlobalStmtProfiler = NewStmtProfiler() + type StmtProfiler struct { taskCh chan *profileTask cacheBufCh chan *profileTask + + mu struct { + sync.Mutex + ept *exportProfileTask + } } type profileTask struct { @@ -38,23 +49,25 @@ func NewStmtProfiler() *StmtProfiler { } func (sp *StmtProfiler) Run() { - logutil.BgLogger().Info("profiler started") + sp.mu.Lock() + defer sp.mu.Unlock() + logutil.BgLogger().Info("cpu profiler started") go sp.startCPUProfileWorker() go sp.startAnalyzeProfileWorker() } func (sp *StmtProfiler) startCPUProfileWorker() { for { - cfg := config.GetGlobalConfig() - if cfg.TopStmt.Enable { - sp.doCPUProfile(cfg.TopStmt.RefreshInterval) + if sp.isEnabled() { + sp.doCPUProfile() } else { time.Sleep(time.Second) } } } -func (sp *StmtProfiler) doCPUProfile(interval int) { +func (sp *StmtProfiler) doCPUProfile() { + interval := config.GetGlobalConfig().TopStmt.RefreshInterval task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { return @@ -79,6 +92,7 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { logutil.BgLogger().Error("parse profile error", zap.Error(err)) continue } + sp.handleExportProfileTask(p) stmtMap := sp.parseCPUProfileTags(p) if len(stmtMap) == 0 { continue @@ -92,9 +106,6 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { total += v } } - if config.GetGlobalConfig().TopStmt.Debug && total > (500*int64(time.Millisecond)) { - ioutil.WriteFile("cpu.profile."+strconv.Itoa(int(task.end)), task.buf.Bytes(), 0644) - } sp.putTaskToBuffer(task) } } @@ -181,3 +192,138 @@ func (s *stmtStats) tune() { s.plans[k] = v + (v/planTotal)*remain } } + +func (sp *StmtProfiler) handleExportProfileTask(p *profile.Profile) { + sp.mu.Lock() + defer sp.mu.Unlock() + if sp.mu.ept == nil { + return + } + sp.mu.ept.mergeProfile(p) +} + +func (sp *StmtProfiler) hasExportProfileTask() bool { + sp.mu.Lock() + has := sp.mu.ept != nil + sp.mu.Unlock() + return has +} + +func (sp *StmtProfiler) isEnabled() bool { + return config.GetGlobalConfig().TopStmt.Enable || sp.hasExportProfileTask() +} + +func StartCPUProfile(w io.Writer) error { + if GlobalStmtProfiler.isEnabled() { + return GlobalStmtProfiler.startExportCPUProfile(w) + } + return pprof.StartCPUProfile(w) +} + +func StopCPUProfile() error { + if GlobalStmtProfiler.isEnabled() { + return GlobalStmtProfiler.stopExportCPUProfile() + } + pprof.StopCPUProfile() + return nil +} + +func (sp *StmtProfiler) startExportCPUProfile(w io.Writer) error { + sp.mu.Lock() + defer sp.mu.Unlock() + if sp.mu.ept != nil { + return errors.New("cpu profiling already in use") + } + sp.mu.ept = &exportProfileTask{w: w} + return nil +} + +func (sp *StmtProfiler) stopExportCPUProfile() error { + sp.mu.Lock() + ept := sp.mu.ept + sp.mu.ept = nil + sp.mu.Unlock() + if ept.err != nil { + return ept.err + } + if w := ept.w; w != nil { + sp.removeLabel(ept.cpuProfile) + return ept.cpuProfile.Write(w) + } + return nil +} + +func (sp *StmtProfiler) removeLabel(p *profile.Profile) { + if p == nil { + return + } + keepLabelSQL := variable.EnablePProfSQLCPU.Load() + for _, s := range p.Sample { + for k := range s.Label { + if keepLabelSQL && k == LabelSQL { + continue + } + delete(s.Label, k) + } + } +} + +type exportProfileTask struct { + cpuProfile *profile.Profile + err error + w io.Writer +} + +func (t *exportProfileTask) mergeProfile(p *profile.Profile) { + if t.err != nil { + return + } + if t.cpuProfile == nil { + t.cpuProfile = p + } else { + t.cpuProfile, t.err = profile.Merge([]*profile.Profile{t.cpuProfile, p}) + } +} + +func ProfileHTTPHandler(w http.ResponseWriter, r *http.Request) { + w.Header().Set("X-Content-Type-Options", "nosniff") + sec, err := strconv.ParseInt(r.FormValue("seconds"), 10, 64) + if sec <= 0 || err != nil { + sec = 30 + } + + if durationExceedsWriteTimeout(r, float64(sec)) { + serveError(w, http.StatusBadRequest, "profile duration exceeds server's WriteTimeout") + return + } + + // Set Content Type assuming StartCPUProfile will work, + // because if it does it starts writing. + w.Header().Set("Content-Type", "application/octet-stream") + w.Header().Set("Content-Disposition", `attachment; filename="profile"`) + + err = StartCPUProfile(w) + if err != nil { + serveError(w, http.StatusInternalServerError, "Could not enable CPU profiling: "+err.Error()) + return + } + time.Sleep(time.Second * time.Duration(sec)) + err = StopCPUProfile() + if err != nil { + serveError(w, http.StatusInternalServerError, "Could not enable CPU profiling: "+err.Error()) + return + } +} + +func durationExceedsWriteTimeout(r *http.Request, seconds float64) bool { + srv, ok := r.Context().Value(http.ServerContextKey).(*http.Server) + return ok && srv.WriteTimeout != 0 && seconds >= srv.WriteTimeout.Seconds() +} + +func serveError(w http.ResponseWriter, status int, txt string) { + w.Header().Set("Content-Type", "text/plain; charset=utf-8") + w.Header().Set("X-Go-Pprof", "1") + w.Header().Del("Content-Disposition") + w.WriteHeader(status) + fmt.Fprintln(w, txt) +} From 13fd80fe81e847311edcf33833e50819887bb29a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 24 May 2021 15:49:06 +0800 Subject: [PATCH 10/54] dev Signed-off-by: crazycs --- util/tracecpu/collector.go | 4 ++++ util/tracecpu/profile.go | 10 +++++----- 2 files changed, 9 insertions(+), 5 deletions(-) create mode 100644 util/tracecpu/collector.go diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go new file mode 100644 index 0000000000000..dce43eedb2a70 --- /dev/null +++ b/util/tracecpu/collector.go @@ -0,0 +1,4 @@ +package tracecpu + +type SQLStatsCollector interface { +} diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index c96aa5f935f57..ad1697954b6b4 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -36,11 +36,6 @@ type StmtProfiler struct { } } -type profileTask struct { - buf *bytes.Buffer - end int64 -} - func NewStmtProfiler() *StmtProfiler { return &StmtProfiler{ taskCh: make(chan *profileTask, 128), @@ -110,6 +105,11 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { } } +type profileTask struct { + buf *bytes.Buffer + end int64 +} + func (sp *StmtProfiler) newProfileTask() *profileTask { var task *profileTask select { From 64d39b42c4635ff463726d8014308dcc9b59aa29 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 24 May 2021 20:55:22 +0800 Subject: [PATCH 11/54] add collector interface Signed-off-by: crazycs --- executor/adapter.go | 26 +++++----- executor/executor.go | 3 +- server/conn.go | 3 +- session/session.go | 10 ++++ sessionctx/stmtctx/stmtctx.go | 13 +++++ tidb-server/main.go | 9 ++++ util/tracecpu/collector.go | 91 +++++++++++++++++++++++++++++++++++ util/tracecpu/profile.go | 74 +++++++++++++++++----------- 8 files changed, 186 insertions(+), 43 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 863c45a7a7fe7..ca72f88931cb2 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -200,8 +200,6 @@ type ExecStmt struct { // OutputNames will be set if using cached plan OutputNames []*types.FieldName PsStmt *plannercore.CachedPrepareStmt - // cache for plan digest - planDigest string } // PointGet short path for point exec directly from plan, keep only necessary steps @@ -339,14 +337,14 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if config.GetGlobalConfig().TopStmt.Enable && a.Plan != nil { // ExecuteExec will rewrite `a.Plan`, so goroutine set label should be executed after `a.buildExecutor`. - normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() - planDigest := a.getPlanDigest() + _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() + normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(planDigest) > 0 { ctx = pprof.WithLabels(ctx, pprof.Labels( - tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, sqlDigest, tracecpu.LabelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) + tracecpu.GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) } } @@ -885,11 +883,15 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { } } -func (a *ExecStmt) getPlanDigest() string { - if len(a.planDigest) == 0 { - _, a.planDigest = plannercore.NormalizePlan(a.Plan) +// getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. +func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (normalized, planDigest string) { + normalized, planDigest = sctx.GetSessionVars().StmtCtx.GetPlanDigest() + if len(normalized) > 0 { + return } - return a.planDigest + normalized, planDigest = plannercore.NormalizePlan(p) + sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) + return } // LogSlowQuery is used to print the slow query in the log files. @@ -943,7 +945,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diskMax := sessVars.StmtCtx.DiskTracker.MaxConsumed() - planDigest := a.getPlanDigest() + _, planDigest := getPlanDigest(a.Ctx, a.Plan) slowItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql.String(), @@ -1105,11 +1107,11 @@ func (a *ExecStmt) SummaryStmt(succ bool) { var planDigestGen func() string if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { - planDigest := a.getPlanDigest() + _, planDigest := getPlanDigest(a.Ctx, a.Plan) return planDigest } } else { - planDigest = a.getPlanDigest() + _, planDigest = getPlanDigest(a.Ctx, a.Plan) } execDetail := stmtCtx.GetExecDetails() diff --git a/executor/executor.go b/executor/executor.go index 2b1d48b44cfd7..00d0495020e25 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1628,8 +1628,9 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.SQLDigest() normalizedSQL, digest := sc.SQLDigest() if len(normalizedSQL) > 0 { - goctx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) + goctx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQLDigest, digest)) pprof.SetGoroutineLabels(goctx) + tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalizedSQL) } } } diff --git a/server/conn.go b/server/conn.go index d9a8f4d148523..43190022485aa 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1031,8 +1031,9 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabel() if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) - ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQL, normalizedSQL, tracecpu.LabelSQLDigest, digest)) + ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQLDigest, digest)) pprof.SetGoroutineLabels(ctx) + tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalizedSQL) } } else if variable.EnablePProfSQLCPU.Load() { normalizedSQL, _ := getLastStmtInConn{cc}.PProfLabel() diff --git a/session/session.go b/session/session.go index b0626a20b5ca1..3a2e573060915 100644 --- a/session/session.go +++ b/session/session.go @@ -23,6 +23,7 @@ import ( "encoding/json" "fmt" "net" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -41,6 +42,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" @@ -1383,6 +1385,14 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) } + if config.GetGlobalConfig().TopStmt.Enable { + normalized, digest := parser.NormalizeDigest(sql) + if len(normalized) > 0 { + ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQLDigest, digest)) + pprof.SetGoroutineLabels(ctx) + tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalized) + } + } return stmts[0], nil } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 49619d7a48092..3a0072b309303 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -147,6 +147,9 @@ type StatementContext struct { normalized string digest string } + // planNormalized use for cache the normalized plan, avoid duplicate builds. + planNormalized string + planDigest string encodedPlan string planHint string planHintSet bool @@ -245,6 +248,16 @@ func (sc *StatementContext) GetEncodedPlan() string { return sc.encodedPlan } +// GetPlanDigest gets the normalized plan and plan digest. +func (sc *StatementContext) GetPlanDigest() (normalized, planDigest string) { + return sc.planNormalized, sc.planDigest +} + +// SetPlanDigest sets the normalized plan and plan digest. +func (sc *StatementContext) SetPlanDigest(normalized, planDigest string) { + sc.planNormalized, sc.planDigest = normalized, planDigest +} + // SetEncodedPlan sets the encoded plan, it is used to avoid repeated encode. func (sc *StatementContext) SetEncodedPlan(encodedPlan string) { sc.encodedPlan = encodedPlan diff --git a/tidb-server/main.go b/tidb-server/main.go index 65758f8b544f5..5a3924eda893e 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -57,6 +57,7 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/profile" "github.com/pingcap/tidb/util/sem" @@ -690,4 +691,12 @@ func stringToList(repairString string) []string { func setProfiler() { tracecpu.GlobalStmtProfiler.Run() + fn := func(origin string) string { + result, err := plancodec.DecodeNormalizedPlan(origin) + if err != nil { + return "" + } + return result + } + tracecpu.GlobalStmtProfiler.SetCollector(tracecpu.NewMockStatsCollector(fn)) } diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index dce43eedb2a70..082033f15b1b4 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -1,4 +1,95 @@ package tracecpu +import ( + "fmt" + "sync" + "time" + + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + type SQLStatsCollector interface { + Collect(ts int64, stats []SQLStats) + RegisterSQL(sqlDigest, normalizedSQL string) + RegisterPlan(planDigest string, normalizedPlan string) +} + +type SQLStats struct { + sqlDigest string + planDigest string + cpuTimeMs uint32 +} + +type mockStatsCollector struct { + // sql_digest -> normalized SQL + sqlmu sync.Mutex + sqlMap map[string]string + // plan_digest -> normalized plan + planMu sync.Mutex + planMap map[string]string + + decodePlanFn func(string) string +} + +func NewMockStatsCollector(decodePlanFn func(string) string) SQLStatsCollector { + return &mockStatsCollector{ + sqlMap: make(map[string]string), + planMap: make(map[string]string), + decodePlanFn: decodePlanFn, + } +} + +func (c *mockStatsCollector) Collect(ts int64, stats []SQLStats) { + if len(stats) == 0 { + return + } + total := int64(0) + logutil.BgLogger().Info("-------- [ BEGIN ] ----------", zap.Int64("ts", ts)) + for _, stmt := range stats { + logutil.BgLogger().Info(fmt.Sprintf("%s : %v, %v, %v, %v", time.Duration(stmt.cpuTimeMs)*time.Millisecond, shortString(stmt.sqlDigest, 5), shortString(stmt.planDigest, 5), c.getSQL(stmt.sqlDigest), shortString(c.getPlan(stmt.planDigest), 30))) + total += int64(stmt.cpuTimeMs) + } + logutil.BgLogger().Info("-------- [ END ] ", zap.Duration("total", time.Duration(total)*time.Millisecond)) +} + +func shortString(digest string, n int) string { + if len(digest) <= n { + return digest + } + return digest[:n] +} + +func (c *mockStatsCollector) getSQL(sqlDigest string) string { + c.sqlmu.Lock() + sql := c.sqlMap[sqlDigest] + c.sqlmu.Unlock() + return sql +} + +func (c *mockStatsCollector) getPlan(planDigest string) string { + c.planMu.Lock() + plan := c.planMap[planDigest] + c.planMu.Unlock() + return plan +} + +func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { + c.sqlmu.Lock() + _, ok := c.sqlMap[sqlDigest] + if !ok { + c.sqlMap[sqlDigest] = normalizedSQL + } + c.sqlmu.Unlock() + +} + +func (c *mockStatsCollector) RegisterPlan(planDigest string, normalizedPlan string) { + c.planMu.Lock() + _, ok := c.planMap[planDigest] + if !ok { + p := c.decodePlanFn(normalizedPlan) + c.planMap[planDigest] = p + } + c.planMu.Unlock() } diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index ad1697954b6b4..bc48bfc5d16b0 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -34,6 +34,7 @@ type StmtProfiler struct { sync.Mutex ept *exportProfileTask } + collector SQLStatsCollector } func NewStmtProfiler() *StmtProfiler { @@ -43,6 +44,24 @@ func NewStmtProfiler() *StmtProfiler { } } +func (sp *StmtProfiler) SetCollector(c SQLStatsCollector) { + sp.collector = c +} + +func (sp *StmtProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { + if sp.collector == nil { + return + } + sp.collector.RegisterSQL(sqlDigest, normalizedSQL) +} + +func (sp *StmtProfiler) RegisterPlan(planDigest string, normalizedPlan string) { + if sp.collector == nil { + return + } + sp.collector.RegisterPlan(planDigest, normalizedPlan) +} + func (sp *StmtProfiler) Run() { sp.mu.Lock() defer sp.mu.Unlock() @@ -87,19 +106,10 @@ func (sp *StmtProfiler) startAnalyzeProfileWorker() { logutil.BgLogger().Error("parse profile error", zap.Error(err)) continue } + stats := sp.parseCPUProfileTags(p) sp.handleExportProfileTask(p) - stmtMap := sp.parseCPUProfileTags(p) - if len(stmtMap) == 0 { - continue - } - total := int64(0) - logutil.BgLogger().Info("-------- [ BEGIN ] ----------") - for digest, stmt := range stmtMap { - logutil.BgLogger().Info(fmt.Sprintf("%s , %v", stmt.normalizedSQL, digest)) - for p, v := range stmt.plans { - logutil.BgLogger().Info(fmt.Sprintf(" %s : %s", time.Duration(v), p)) - total += v - } + if sp.collector != nil { + sp.collector.Collect(task.end, stats) } sp.putTaskToBuffer(task) } @@ -130,26 +140,21 @@ func (sp *StmtProfiler) putTaskToBuffer(task *profileTask) { } } -func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) (stmtMap map[string]*stmtStats) { - stmtMap = make(map[string]*stmtStats) +func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { + stmtMap := make(map[string]*stmtStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { digests, ok := s.Label[LabelSQLDigest] if !ok || len(digests) == 0 { continue } - sqls, ok := s.Label[LabelSQL] - if !ok || len(sqls) != len(digests) { - continue - } - for i, digest := range digests { + for _, digest := range digests { stmt, ok := stmtMap[digest] if !ok { stmt = &stmtStats{ - plans: make(map[string]int64), - total: 0, - isInternal: false, - normalizedSQL: sqls[i], + plans: make(map[string]int64), + total: 0, + isInternal: false, } stmtMap[digest] = stmt } @@ -161,17 +166,28 @@ func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) (stmtMap map[str } } } - for _, stmt := range stmtMap { + return sp.createSQLStats(stmtMap) +} + +func (sp *StmtProfiler) createSQLStats(stmtMap map[string]*stmtStats) []SQLStats { + stats := make([]SQLStats, 0, len(stmtMap)) + for sqlDigest, stmt := range stmtMap { stmt.tune() + for planDigest, val := range stmt.plans { + stats = append(stats, SQLStats{ + sqlDigest: sqlDigest, + planDigest: planDigest, + cpuTimeMs: uint32(time.Duration(val).Milliseconds()), + }) + } } - return stmtMap + return stats } type stmtStats struct { - plans map[string]int64 - total int64 - isInternal bool - normalizedSQL string + plans map[string]int64 + total int64 + isInternal bool } // tune use to adjust stats From 251e725a87e33eec55547a03f30117022f602a93 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 15:10:44 +0800 Subject: [PATCH 12/54] fix bug of execute prepare stmt doesn't collected plan Signed-off-by: crazycs --- executor/adapter.go | 31 +++++++++++++++++++------------ server/conn.go | 33 +++++++++++++++++++++++++++++---- 2 files changed, 48 insertions(+), 16 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index ca72f88931cb2..dcda8ffc059b6 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -210,6 +210,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } + ctx = a.setPProfLabel(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -284,6 +285,23 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return is.SchemaMetaVersion(), nil } +func (a *ExecStmt) setPProfLabel(ctx context.Context) context.Context { + if !config.GetGlobalConfig().TopStmt.Enable || a.Plan == nil { + return ctx + } + // ExecuteExec will rewrite `a.Plan`, so set goroutine label should be executed after `a.buildExecutor`. + _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() + normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) + if len(planDigest) > 0 { + ctx = pprof.WithLabels(ctx, pprof.Labels( + tracecpu.LabelSQLDigest, sqlDigest, + tracecpu.LabelPlanDigest, planDigest)) + pprof.SetGoroutineLabels(ctx) + tracecpu.GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) + } + return ctx +} + // Exec builds an Executor from a plan. If the Executor doesn't return result, // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns // result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. @@ -335,18 +353,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } - if config.GetGlobalConfig().TopStmt.Enable && a.Plan != nil { - // ExecuteExec will rewrite `a.Plan`, so goroutine set label should be executed after `a.buildExecutor`. - _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() - normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) - if len(planDigest) > 0 { - ctx = pprof.WithLabels(ctx, pprof.Labels( - tracecpu.LabelSQLDigest, sqlDigest, - tracecpu.LabelPlanDigest, planDigest)) - pprof.SetGoroutineLabels(ctx) - tracecpu.GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) - } - } + ctx = a.setPProfLabel(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) diff --git a/server/conn.go b/server/conn.go index 43190022485aa..ff9c1b8d22579 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1028,7 +1028,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cmd := data[0] data = data[1:] if config.GetGlobalConfig().TopStmt.Enable { - normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabel() + normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabelNormalizedAndDigest() if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQLDigest, digest)) @@ -1036,7 +1036,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalizedSQL) } } else if variable.EnablePProfSQLCPU.Load() { - normalizedSQL, _ := getLastStmtInConn{cc}.PProfLabel() + normalizedSQL := getLastStmtInConn{cc}.PProfLabel() if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) ctx = pprof.WithLabels(ctx, pprof.Labels("sql", normalizedSQL)) @@ -1045,7 +1045,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { } if trace.IsEnabled() { lc := getLastStmtInConn{cc} - sqlType, _ := lc.PProfLabel() + sqlType := lc.PProfLabel() if len(sqlType) > 0 { var task *trace.Task ctx, task = trace.NewTask(ctx, sqlType) @@ -2151,7 +2151,32 @@ func (cc getLastStmtInConn) String() string { } // PProfLabel return sql label used to tag pprof. -func (cc getLastStmtInConn) PProfLabel() (string, string) { +func (cc getLastStmtInConn) PProfLabel() string { + if len(cc.lastPacket) == 0 { + return "" + } + cmd, data := cc.lastPacket[0], cc.lastPacket[1:] + switch cmd { + case mysql.ComInitDB: + return "UseDB" + case mysql.ComFieldList: + return "ListFields" + case mysql.ComStmtClose: + return "CloseStmt" + case mysql.ComStmtReset: + return "ResetStmt" + case mysql.ComQuery, mysql.ComStmtPrepare: + return parser.Normalize(queryStrForLog(string(hack.String(data)))) + case mysql.ComStmtExecute, mysql.ComStmtFetch: + stmtID := binary.LittleEndian.Uint32(data[0:4]) + return queryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) + default: + return "" + } +} + +// PProfLabelNormalizedAndDigest return sql and sql_digest label used to tag pprof. +func (cc getLastStmtInConn) PProfLabelNormalizedAndDigest() (string, string) { if len(cc.lastPacket) == 0 { return "", "" } From 7814dcb4715fa722377a92f6b8635571e647f2d2 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 16:25:12 +0800 Subject: [PATCH 13/54] refine code Signed-off-by: crazycs --- executor/adapter.go | 28 ++++++++++++++-------------- executor/executor.go | 7 +++---- planner/core/common_plans.go | 3 +++ server/conn.go | 6 +++--- sessionctx/stmtctx/stmtctx.go | 10 +++++----- 5 files changed, 28 insertions(+), 26 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 7d24b6ac49124..c8087967f0fdb 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -890,20 +890,6 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { } } -// getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. -func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, string) { - normalized, planDigest := sctx.GetSessionVars().StmtCtx.GetPlanDigest() - if len(normalized) > 0 && planDigest != nil { - return normalized, planDigest.String() - } - normalized, planDigest = plannercore.NormalizePlan(p) - if len(normalized) == 0 || planDigest == nil { - return "", "" - } - sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) - return normalized, planDigest.String() -} - // LogSlowQuery is used to print the slow query in the log files. func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { sessVars := a.Ctx.GetSessionVars() @@ -1046,6 +1032,20 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { return variable.SlowLogPlanPrefix + planTree + variable.SlowLogPlanSuffix } +// getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. +func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, string) { + normalized, planDigest := sctx.GetSessionVars().StmtCtx.GetPlanDigest() + if len(normalized) > 0 && planDigest != nil { + return normalized, planDigest.String() + } + normalized, planDigest = plannercore.NormalizePlan(p) + if len(normalized) == 0 || planDigest == nil { + return "", "" + } + sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) + return normalized, planDigest.String() +} + // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. func getEncodedPlan(sctx sessionctx.Context, p plannercore.Plan, genHint bool, n ast.StmtNode) (encodedPlan, hintStr string) { var hintSet bool diff --git a/executor/executor.go b/executor/executor.go index b0f6f74af5cf2..6fb9f7bda42fc 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1639,11 +1639,10 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { } if config.TopSQLEnabled() { sc.OriginalSQL = s.Text() - sc.SQLDigest() normalizedSQL, digest := sc.SQLDigest() - if len(normalizedSQL) > 0 { - goctx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQLDigest, digest.String())) - pprof.SetGoroutineLabels(goctx) + if len(normalizedSQL) > 0 && digest != nil { + goCtx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQLDigest, digest.String())) + pprof.SetGoroutineLabels(goCtx) tracecpu.GlobalStmtProfiler.RegisterSQL(digest.String(), normalizedSQL) } } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 87ec3ae1f1a87..7170c991be051 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -376,6 +376,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } e.names = cachedVal.OutPutNames e.Plan = cachedVal.Plan + stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) return nil } break @@ -405,6 +406,7 @@ REBUILD: } cached := NewPSTMTPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) + stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { hitVal := false for i, cacheVal := range cacheVals.([]*PSTMTPlanCacheValue) { @@ -464,6 +466,7 @@ func (e *Execute) tryCachePointPlan(ctx context.Context, sctx sessionctx.Context prepared.CachedPlan = p prepared.CachedNames = names preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) + sctx.GetSessionVars().StmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) } return err } diff --git a/server/conn.go b/server/conn.go index d6cde05c4047b..157ea5ea8eb35 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1036,10 +1036,10 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalizedSQL) } } else if variable.EnablePProfSQLCPU.Load() { - normalizedSQL := getLastStmtInConn{cc}.PProfLabel() - if len(normalizedSQL) > 0 { + label := getLastStmtInConn{cc}.PProfLabel() + if len(label) > 0 { defer pprof.SetGoroutineLabels(ctx) - ctx = pprof.WithLabels(ctx, pprof.Labels("sql", normalizedSQL)) + ctx = pprof.WithLabels(ctx, pprof.Labels("sql", label)) pprof.SetGoroutineLabels(ctx) } } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index fea338cf1dd7d..ea8bd70b8c0f2 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -246,11 +246,6 @@ func (sc *StatementContext) InitSQLDigest(normalized string, digest *parser.Dige }) } -// GetEncodedPlan gets the encoded plan, it is used to avoid repeated encode. -func (sc *StatementContext) GetEncodedPlan() string { - return sc.encodedPlan -} - // GetPlanDigest gets the normalized plan and plan digest. func (sc *StatementContext) GetPlanDigest() (normalized string, planDigest *parser.Digest) { return sc.planNormalized, sc.planDigest @@ -278,6 +273,11 @@ func (sc *StatementContext) SetPlanDigest(normalized string, planDigest *parser. } } +// GetEncodedPlan gets the encoded plan, it is used to avoid repeated encode. +func (sc *StatementContext) GetEncodedPlan() string { + return sc.encodedPlan +} + // SetEncodedPlan sets the encoded plan, it is used to avoid repeated encode. func (sc *StatementContext) SetEncodedPlan(encodedPlan string) { sc.encodedPlan = encodedPlan From f6b2c2085481826cee4ef678ede2ffdba7772171 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 16:55:46 +0800 Subject: [PATCH 14/54] refine code Signed-off-by: crazycs --- executor/adapter.go | 15 ++++----------- executor/executor.go | 5 +---- server/conn.go | 19 ++++++++++++------- session/session.go | 5 +---- util/tracecpu/profile.go | 27 +++++++++++++++++++++------ 5 files changed, 39 insertions(+), 32 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index c8087967f0fdb..2781c38d50e52 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "math" - "runtime/pprof" "runtime/trace" "strings" "sync/atomic" @@ -293,11 +292,7 @@ func (a *ExecStmt) setPProfLabel(ctx context.Context) context.Context { _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(planDigest) > 0 { - ctx = pprof.WithLabels(ctx, pprof.Labels( - tracecpu.LabelSQLDigest, sqlDigest.String(), - tracecpu.LabelPlanDigest, planDigest)) - pprof.SetGoroutineLabels(ctx) - tracecpu.GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) + ctx = tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest.String(), planDigest, normalizedPlan) } return ctx } @@ -1034,15 +1029,13 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { // getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, string) { - normalized, planDigest := sctx.GetSessionVars().StmtCtx.GetPlanDigest() + sc := sctx.GetSessionVars().StmtCtx + normalized, planDigest := sc.GetPlanDigest() if len(normalized) > 0 && planDigest != nil { return normalized, planDigest.String() } normalized, planDigest = plannercore.NormalizePlan(p) - if len(normalized) == 0 || planDigest == nil { - return "", "" - } - sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) + sc.SetPlanDigest(normalized, planDigest) return normalized, planDigest.String() } diff --git a/executor/executor.go b/executor/executor.go index 6fb9f7bda42fc..c34d01c1524d3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,7 +18,6 @@ import ( "fmt" "math" "runtime" - "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -1641,9 +1640,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.OriginalSQL = s.Text() normalizedSQL, digest := sc.SQLDigest() if len(normalizedSQL) > 0 && digest != nil { - goCtx := pprof.WithLabels(context.Background(), pprof.Labels(tracecpu.LabelSQLDigest, digest.String())) - pprof.SetGoroutineLabels(goCtx) - tracecpu.GlobalStmtProfiler.RegisterSQL(digest.String(), normalizedSQL) + tracecpu.SetGoroutineLabelsWithSQL(context.Background(), normalizedSQL, digest.String()) } } } diff --git a/server/conn.go b/server/conn.go index 157ea5ea8eb35..e495189baf622 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1031,9 +1031,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabelNormalizedAndDigest() if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) - ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQLDigest, digest)) - pprof.SetGoroutineLabels(ctx) - tracecpu.GlobalStmtProfiler.RegisterSQL(digest, normalizedSQL) + ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalizedSQL, digest) } } else if variable.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() @@ -2175,6 +2173,13 @@ func (cc getLastStmtInConn) PProfLabel() string { } } +var ( + pprofLabelOfUseDB = parser.DigestNormalized("UseDB") + pprofLabelOfListFields = parser.DigestNormalized("ListFields") + pprofLabelOfCloseStmt = parser.DigestNormalized("CloseStmt") + pprofLabelOfResetStmt = parser.DigestNormalized("ResetStmt") +) + // PProfLabelNormalizedAndDigest return sql and sql_digest label used to tag pprof. func (cc getLastStmtInConn) PProfLabelNormalizedAndDigest() (string, string) { if len(cc.lastPacket) == 0 { @@ -2183,13 +2188,13 @@ func (cc getLastStmtInConn) PProfLabelNormalizedAndDigest() (string, string) { cmd, data := cc.lastPacket[0], cc.lastPacket[1:] switch cmd { case mysql.ComInitDB: - return "UseDB", "" + return "UseDB", pprofLabelOfUseDB.String() case mysql.ComFieldList: - return "ListFields", "" + return "ListFields", pprofLabelOfListFields.String() case mysql.ComStmtClose: - return "CloseStmt", "" + return "CloseStmt", pprofLabelOfCloseStmt.String() case mysql.ComStmtReset: - return "ResetStmt", "" + return "ResetStmt", pprofLabelOfResetStmt.String() case mysql.ComQuery, mysql.ComStmtPrepare: normalized, digest := parser.NormalizeDigest(string(hack.String(data))) return normalized, digest.String() diff --git a/session/session.go b/session/session.go index 00983041014d0..1d84b194ae120 100644 --- a/session/session.go +++ b/session/session.go @@ -23,7 +23,6 @@ import ( "encoding/json" "fmt" "net" - "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -1388,9 +1387,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter if config.TopSQLEnabled() { normalized, digest := parser.NormalizeDigest(sql) if len(normalized) > 0 && digest != nil { - ctx = pprof.WithLabels(ctx, pprof.Labels(tracecpu.LabelSQLDigest, digest.String())) - pprof.SetGoroutineLabels(ctx) - tracecpu.GlobalStmtProfiler.RegisterSQL(digest.String(), normalized) + ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalized, digest.String()) } } return stmts[0], nil diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 79d4496786178..a6d17f44a0260 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -2,6 +2,7 @@ package tracecpu import ( "bytes" + "context" "errors" "fmt" "io" @@ -19,9 +20,9 @@ import ( ) const ( - LabelSQL = "sql" - LabelSQLDigest = "sql_digest" - LabelPlanDigest = "plan_digest" + labelSQL = "sql" + labelSQLDigest = "sql_digest" + labelPlanDigest = "plan_digest" ) var GlobalStmtProfiler = NewStmtProfiler() @@ -144,7 +145,7 @@ func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { stmtMap := make(map[string]*stmtStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { - digests, ok := s.Label[LabelSQLDigest] + digests, ok := s.Label[labelSQLDigest] if !ok || len(digests) == 0 { continue } @@ -160,7 +161,7 @@ func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { } stmt.total += s.Value[idx] - plans := s.Label[LabelPlanDigest] + plans := s.Label[labelPlanDigest] for _, plan := range plans { stmt.plans[plan] += s.Value[idx] } @@ -244,6 +245,20 @@ func StopCPUProfile() error { return nil } +func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { + ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest)) + pprof.SetGoroutineLabels(ctx) + GlobalStmtProfiler.RegisterSQL(sqlDigest, normalizedSQL) + return ctx +} + +func SetGoroutineLabelsWithSQLAndPlan(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { + ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) + pprof.SetGoroutineLabels(ctx) + GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) + return ctx +} + func (sp *StmtProfiler) startExportCPUProfile(w io.Writer) error { sp.mu.Lock() defer sp.mu.Unlock() @@ -276,7 +291,7 @@ func (sp *StmtProfiler) removeLabel(p *profile.Profile) { keepLabelSQL := variable.EnablePProfSQLCPU.Load() for _, s := range p.Sample { for k := range s.Label { - if keepLabelSQL && k == LabelSQL { + if keepLabelSQL && k == labelSQL { continue } delete(s.Label, k) From ae0997cbec0ec10f4cfa9755491d382a9c6e9f51 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 17:13:16 +0800 Subject: [PATCH 15/54] refine code Signed-off-by: crazycs --- server/conn.go | 17 +++++------------ util/misc.go | 9 +++++++++ util/tracecpu/profile.go | 7 ++++++- 3 files changed, 20 insertions(+), 13 deletions(-) diff --git a/server/conn.go b/server/conn.go index e495189baf622..66f16e8297f2c 100644 --- a/server/conn.go +++ b/server/conn.go @@ -79,6 +79,7 @@ import ( storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" + util2 "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/arena" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" @@ -908,14 +909,6 @@ func (cc *clientConn) ShutdownOrNotify() bool { return false } -func queryStrForLog(query string) string { - const size = 4096 - if len(query) > size { - return query[:size] + fmt.Sprintf("(len: %d)", len(query)) - } - return query -} - func errStrForLog(err error, enableRedactLog bool) string { if enableRedactLog { // currently, only ErrParse is considered when enableRedactLog because it may contain sensitive information like @@ -2133,10 +2126,10 @@ func (cc getLastStmtInConn) String() string { if cc.ctx.GetSessionVars().EnableRedactLog { sql = parser.Normalize(sql) } - return queryStrForLog(sql) + return util2.QueryStrForLog(sql) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return queryStrForLog(cc.preparedStmt2String(stmtID)) + return util2.QueryStrForLog(cc.preparedStmt2String(stmtID)) case mysql.ComStmtClose, mysql.ComStmtReset: stmtID := binary.LittleEndian.Uint32(data[0:4]) return mysql.Command2Str[cmd] + " " + strconv.Itoa(int(stmtID)) @@ -2164,10 +2157,10 @@ func (cc getLastStmtInConn) PProfLabel() string { case mysql.ComStmtReset: return "ResetStmt" case mysql.ComQuery, mysql.ComStmtPrepare: - return parser.Normalize(queryStrForLog(string(hack.String(data)))) + return parser.Normalize(util2.QueryStrForLog(string(hack.String(data)))) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return queryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) + return util2.QueryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) default: return "" } diff --git a/util/misc.go b/util/misc.go index 817c5715af818..ccba554bb07d0 100644 --- a/util/misc.go +++ b/util/misc.go @@ -535,3 +535,12 @@ func GetLocalIP() string { } return "" } + +// QueryStrForLog trim the query if the query length more than 4096 +func QueryStrForLog(query string) string { + const size = 4096 + if len(query) > size { + return query[:size] + fmt.Sprintf("(len: %d)", len(query)) + } + return query +} diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index a6d17f44a0260..58778d53df541 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -15,6 +15,7 @@ import ( "github.com/google/pprof/profile" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -246,7 +247,11 @@ func StopCPUProfile() error { } func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { - ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest)) + if variable.EnablePProfSQLCPU.Load() { + ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) + } else { + ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest)) + } pprof.SetGoroutineLabels(ctx) GlobalStmtProfiler.RegisterSQL(sqlDigest, normalizedSQL) return ctx From 12038b31dc56375ff9f087c3b0d5e5b7e749f318 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 19:26:18 +0800 Subject: [PATCH 16/54] add test and refine code Signed-off-by: crazycs --- executor/adapter.go | 9 +- executor/executor.go | 2 +- server/conn.go | 4 +- session/session.go | 4 +- tidb-server/main.go | 15 +-- util/tracecpu/collector.go | 91 ++--------------- util/tracecpu/profile.go | 101 ++++++++++-------- util/tracecpu/profile_test.go | 186 ++++++++++++++++++++++++++++++++++ 8 files changed, 264 insertions(+), 148 deletions(-) create mode 100644 util/tracecpu/profile_test.go diff --git a/executor/adapter.go b/executor/adapter.go index 2781c38d50e52..d22cd75f65256 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -209,7 +209,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - ctx = a.setPProfLabel(ctx) + ctx = a.setPProfLabelWithPlan(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -284,11 +284,10 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return is.SchemaMetaVersion(), nil } -func (a *ExecStmt) setPProfLabel(ctx context.Context) context.Context { +func (a *ExecStmt) setPProfLabelWithPlan(ctx context.Context) context.Context { if a.Plan == nil || !config.TopSQLEnabled() { return ctx } - // ExecuteExec will rewrite `a.Plan`, so set goroutine label should be executed after `a.buildExecutor`. _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(planDigest) > 0 { @@ -347,8 +346,8 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if err != nil { return nil, err } - - ctx = a.setPProfLabel(ctx) + // ExecuteExec will rewrite `a.Plan`, so set goroutine label should be executed after `a.buildExecutor`. + ctx = a.setPProfLabelWithPlan(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) diff --git a/executor/executor.go b/executor/executor.go index c34d01c1524d3..ee3433a49c0d0 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1639,7 +1639,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { if config.TopSQLEnabled() { sc.OriginalSQL = s.Text() normalizedSQL, digest := sc.SQLDigest() - if len(normalizedSQL) > 0 && digest != nil { + if digest != nil { tracecpu.SetGoroutineLabelsWithSQL(context.Background(), normalizedSQL, digest.String()) } } diff --git a/server/conn.go b/server/conn.go index 66f16e8297f2c..aa14ef7bb7fa9 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1021,7 +1021,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cmd := data[0] data = data[1:] if config.TopSQLEnabled() { - normalizedSQL, digest := getLastStmtInConn{cc}.PProfLabelNormalizedAndDigest() + normalizedSQL, digest := getLastStmtInConn{cc}.pprofLabelNormalizedAndDigest() if len(normalizedSQL) > 0 { defer pprof.SetGoroutineLabels(ctx) ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalizedSQL, digest) @@ -2174,7 +2174,7 @@ var ( ) // PProfLabelNormalizedAndDigest return sql and sql_digest label used to tag pprof. -func (cc getLastStmtInConn) PProfLabelNormalizedAndDigest() (string, string) { +func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { if len(cc.lastPacket) == 0 { return "", "" } diff --git a/session/session.go b/session/session.go index 1d84b194ae120..85896ebf6ce0e 100644 --- a/session/session.go +++ b/session/session.go @@ -1386,8 +1386,8 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter } if config.TopSQLEnabled() { normalized, digest := parser.NormalizeDigest(sql) - if len(normalized) > 0 && digest != nil { - ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalized, digest.String()) + if digest != nil { + tracecpu.SetGoroutineLabelsWithSQL(ctx, normalized, digest.String()) } } return stmts[0], nil diff --git a/tidb-server/main.go b/tidb-server/main.go index 881fb82613c9c..4875f94fd85e8 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -57,7 +57,6 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/profile" "github.com/pingcap/tidb/util/sem" @@ -180,7 +179,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() - setProfiler() + setupSQLStatsProfiler() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) @@ -689,14 +688,6 @@ func stringToList(repairString string) []string { }) } -func setProfiler() { - tracecpu.GlobalStmtProfiler.Run() - fn := func(origin string) string { - result, err := plancodec.DecodeNormalizedPlan(origin) - if err != nil { - return "" - } - return result - } - tracecpu.GlobalStmtProfiler.SetCollector(tracecpu.NewMockStatsCollector(fn)) +func setupSQLStatsProfiler() { + tracecpu.GlobalSQLStatsProfiler.Run() } diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index 082033f15b1b4..f2176958bb518 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -1,95 +1,16 @@ package tracecpu -import ( - "fmt" - "sync" - "time" - - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" -) - +// SQLStatsCollector uses to collect SQL stats. +// TODO: add a collector. type SQLStatsCollector interface { Collect(ts int64, stats []SQLStats) RegisterSQL(sqlDigest, normalizedSQL string) RegisterPlan(planDigest string, normalizedPlan string) } +// SQLStats indicate the SQL stats. type SQLStats struct { - sqlDigest string - planDigest string - cpuTimeMs uint32 -} - -type mockStatsCollector struct { - // sql_digest -> normalized SQL - sqlmu sync.Mutex - sqlMap map[string]string - // plan_digest -> normalized plan - planMu sync.Mutex - planMap map[string]string - - decodePlanFn func(string) string -} - -func NewMockStatsCollector(decodePlanFn func(string) string) SQLStatsCollector { - return &mockStatsCollector{ - sqlMap: make(map[string]string), - planMap: make(map[string]string), - decodePlanFn: decodePlanFn, - } -} - -func (c *mockStatsCollector) Collect(ts int64, stats []SQLStats) { - if len(stats) == 0 { - return - } - total := int64(0) - logutil.BgLogger().Info("-------- [ BEGIN ] ----------", zap.Int64("ts", ts)) - for _, stmt := range stats { - logutil.BgLogger().Info(fmt.Sprintf("%s : %v, %v, %v, %v", time.Duration(stmt.cpuTimeMs)*time.Millisecond, shortString(stmt.sqlDigest, 5), shortString(stmt.planDigest, 5), c.getSQL(stmt.sqlDigest), shortString(c.getPlan(stmt.planDigest), 30))) - total += int64(stmt.cpuTimeMs) - } - logutil.BgLogger().Info("-------- [ END ] ", zap.Duration("total", time.Duration(total)*time.Millisecond)) -} - -func shortString(digest string, n int) string { - if len(digest) <= n { - return digest - } - return digest[:n] -} - -func (c *mockStatsCollector) getSQL(sqlDigest string) string { - c.sqlmu.Lock() - sql := c.sqlMap[sqlDigest] - c.sqlmu.Unlock() - return sql -} - -func (c *mockStatsCollector) getPlan(planDigest string) string { - c.planMu.Lock() - plan := c.planMap[planDigest] - c.planMu.Unlock() - return plan -} - -func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { - c.sqlmu.Lock() - _, ok := c.sqlMap[sqlDigest] - if !ok { - c.sqlMap[sqlDigest] = normalizedSQL - } - c.sqlmu.Unlock() - -} - -func (c *mockStatsCollector) RegisterPlan(planDigest string, normalizedPlan string) { - c.planMu.Lock() - _, ok := c.planMap[planDigest] - if !ok { - p := c.decodePlanFn(normalizedPlan) - c.planMap[planDigest] = p - } - c.planMu.Unlock() + SQLDigest string + PlanDigest string + CPUTimeMs uint32 } diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 58778d53df541..6bdce99dbd6f0 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -26,9 +26,10 @@ const ( labelPlanDigest = "plan_digest" ) -var GlobalStmtProfiler = NewStmtProfiler() +// GlobalSQLStatsProfiler is the global SQL stats profiler. +var GlobalSQLStatsProfiler = NewSQLStatsProfiler() -type StmtProfiler struct { +type sqlStatsProfiler struct { taskCh chan *profileTask cacheBufCh chan *profileTask @@ -39,32 +40,33 @@ type StmtProfiler struct { collector SQLStatsCollector } -func NewStmtProfiler() *StmtProfiler { - return &StmtProfiler{ +// NewSQLStatsProfiler create a sqlStatsProfiler. +func NewSQLStatsProfiler() *sqlStatsProfiler { + return &sqlStatsProfiler{ taskCh: make(chan *profileTask, 128), cacheBufCh: make(chan *profileTask, 128), } } -func (sp *StmtProfiler) SetCollector(c SQLStatsCollector) { +func (sp *sqlStatsProfiler) SetCollector(c SQLStatsCollector) { sp.collector = c } -func (sp *StmtProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { +func (sp *sqlStatsProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { if sp.collector == nil { return } sp.collector.RegisterSQL(sqlDigest, normalizedSQL) } -func (sp *StmtProfiler) RegisterPlan(planDigest string, normalizedPlan string) { +func (sp *sqlStatsProfiler) RegisterPlan(planDigest string, normalizedPlan string) { if sp.collector == nil { return } sp.collector.RegisterPlan(planDigest, normalizedPlan) } -func (sp *StmtProfiler) Run() { +func (sp *sqlStatsProfiler) Run() { sp.mu.Lock() defer sp.mu.Unlock() logutil.BgLogger().Info("cpu profiler started") @@ -72,7 +74,8 @@ func (sp *StmtProfiler) Run() { go sp.startAnalyzeProfileWorker() } -func (sp *StmtProfiler) startCPUProfileWorker() { +func (sp *sqlStatsProfiler) startCPUProfileWorker() { + defer util.Recover("top-sql", "profileWorker", nil, false) for { if sp.isEnabled() { sp.doCPUProfile() @@ -82,7 +85,7 @@ func (sp *StmtProfiler) startCPUProfileWorker() { } } -func (sp *StmtProfiler) doCPUProfile() { +func (sp *sqlStatsProfiler) doCPUProfile() { interval := config.GetGlobalConfig().TopSQL.RefreshInterval task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { @@ -94,12 +97,13 @@ func (sp *StmtProfiler) doCPUProfile() { sp.sendProfileTask(task) } -func (sp *StmtProfiler) sendProfileTask(task *profileTask) { +func (sp *sqlStatsProfiler) sendProfileTask(task *profileTask) { task.end = time.Now().Unix() sp.taskCh <- task } -func (sp *StmtProfiler) startAnalyzeProfileWorker() { +func (sp *sqlStatsProfiler) startAnalyzeProfileWorker() { + defer util.Recover("top-sql", "analyzeProfileWorker", nil, false) for { task := <-sp.taskCh reader := bytes.NewReader(task.buf.Bytes()) @@ -122,7 +126,7 @@ type profileTask struct { end int64 } -func (sp *StmtProfiler) newProfileTask() *profileTask { +func (sp *sqlStatsProfiler) newProfileTask() *profileTask { var task *profileTask select { case task = <-sp.cacheBufCh: @@ -135,15 +139,15 @@ func (sp *StmtProfiler) newProfileTask() *profileTask { return task } -func (sp *StmtProfiler) putTaskToBuffer(task *profileTask) { +func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { select { case sp.cacheBufCh <- task: default: } } -func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { - stmtMap := make(map[string]*stmtStats) +func (sp *sqlStatsProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { + sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { digests, ok := s.Label[labelSQLDigest] @@ -151,14 +155,14 @@ func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { continue } for _, digest := range digests { - stmt, ok := stmtMap[digest] + stmt, ok := sqlMap[digest] if !ok { - stmt = &stmtStats{ + stmt = &sqlStats{ plans: make(map[string]int64), total: 0, isInternal: false, } - stmtMap[digest] = stmt + sqlMap[digest] = stmt } stmt.total += s.Value[idx] @@ -168,32 +172,32 @@ func (sp *StmtProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { } } } - return sp.createSQLStats(stmtMap) + return sp.createSQLStats(sqlMap) } -func (sp *StmtProfiler) createSQLStats(stmtMap map[string]*stmtStats) []SQLStats { - stats := make([]SQLStats, 0, len(stmtMap)) - for sqlDigest, stmt := range stmtMap { +func (sp *sqlStatsProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLStats { + stats := make([]SQLStats, 0, len(sqlMap)) + for sqlDigest, stmt := range sqlMap { stmt.tune() for planDigest, val := range stmt.plans { stats = append(stats, SQLStats{ - sqlDigest: sqlDigest, - planDigest: planDigest, - cpuTimeMs: uint32(time.Duration(val).Milliseconds()), + SQLDigest: sqlDigest, + PlanDigest: planDigest, + CPUTimeMs: uint32(time.Duration(val).Milliseconds()), }) } } return stats } -type stmtStats struct { +type sqlStats struct { plans map[string]int64 total int64 isInternal bool } // tune use to adjust stats -func (s *stmtStats) tune() { +func (s *sqlStats) tune() { if len(s.plans) == 0 { s.plans[""] = s.total return @@ -211,7 +215,7 @@ func (s *stmtStats) tune() { } } -func (sp *StmtProfiler) handleExportProfileTask(p *profile.Profile) { +func (sp *sqlStatsProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept == nil { @@ -220,51 +224,61 @@ func (sp *StmtProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.ept.mergeProfile(p) } -func (sp *StmtProfiler) hasExportProfileTask() bool { +func (sp *sqlStatsProfiler) hasExportProfileTask() bool { sp.mu.Lock() has := sp.mu.ept != nil sp.mu.Unlock() return has } -func (sp *StmtProfiler) isEnabled() bool { +func (sp *sqlStatsProfiler) isEnabled() bool { return config.GetGlobalConfig().TopSQL.Enable || sp.hasExportProfileTask() } +// StartCPUProfile same like pprof.StartCPUProfile. +// Because the GlobalSQLStatsProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, +// other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalStmtProfiler.isEnabled() { - return GlobalStmtProfiler.startExportCPUProfile(w) + if GlobalSQLStatsProfiler.isEnabled() { + return GlobalSQLStatsProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) } +// StopCPUProfile same like pprof.StopCPUProfile. +// other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. func StopCPUProfile() error { - if GlobalStmtProfiler.isEnabled() { - return GlobalStmtProfiler.stopExportCPUProfile() + if GlobalSQLStatsProfiler.isEnabled() { + return GlobalSQLStatsProfiler.stopExportCPUProfile() } pprof.StopCPUProfile() return nil } +// SetGoroutineLabelsWithSQL sets the SQL digest label into the goroutine. func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { + if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { + return ctx + } if variable.EnablePProfSQLCPU.Load() { ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) } else { ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest)) } pprof.SetGoroutineLabels(ctx) - GlobalStmtProfiler.RegisterSQL(sqlDigest, normalizedSQL) + GlobalSQLStatsProfiler.RegisterSQL(sqlDigest, normalizedSQL) return ctx } +// SetGoroutineLabelsWithSQLAndPlan sets the SQL and plan digest label into the goroutine. func SetGoroutineLabelsWithSQLAndPlan(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) - GlobalStmtProfiler.RegisterPlan(planDigest, normalizedPlan) + GlobalSQLStatsProfiler.RegisterPlan(planDigest, normalizedPlan) return ctx } -func (sp *StmtProfiler) startExportCPUProfile(w io.Writer) error { +func (sp *sqlStatsProfiler) startExportCPUProfile(w io.Writer) error { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept != nil { @@ -274,7 +288,7 @@ func (sp *StmtProfiler) startExportCPUProfile(w io.Writer) error { return nil } -func (sp *StmtProfiler) stopExportCPUProfile() error { +func (sp *sqlStatsProfiler) stopExportCPUProfile() error { sp.mu.Lock() ept := sp.mu.ept sp.mu.ept = nil @@ -289,7 +303,7 @@ func (sp *StmtProfiler) stopExportCPUProfile() error { return nil } -func (sp *StmtProfiler) removeLabel(p *profile.Profile) { +func (sp *sqlStatsProfiler) removeLabel(p *profile.Profile) { if p == nil { return } @@ -321,6 +335,8 @@ func (t *exportProfileTask) mergeProfile(p *profile.Profile) { } } +// ProfileHTTPHandler is same as pprof.Profile. +// The difference is ProfileHTTPHandler uses tracecpu.StartCPUProfile/StopCPUProfile to fetch profile data. func ProfileHTTPHandler(w http.ResponseWriter, r *http.Request) { w.Header().Set("X-Content-Type-Options", "nosniff") sec, err := strconv.ParseInt(r.FormValue("seconds"), 10, 64) @@ -361,5 +377,8 @@ func serveError(w http.ResponseWriter, status int, txt string) { w.Header().Set("X-Go-Pprof", "1") w.Header().Del("Content-Disposition") w.WriteHeader(status) - fmt.Fprintln(w, txt) + _, err := fmt.Fprintln(w, txt) + if err != nil { + logutil.BgLogger().Info("write http response error", zap.Error(err)) + } } diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go new file mode 100644 index 0000000000000..50bc4d46c17a6 --- /dev/null +++ b/util/tracecpu/profile_test.go @@ -0,0 +1,186 @@ +package tracecpu_test + +import ( + "bytes" + "context" + "crypto/sha256" + "encoding/hex" + "sync" + "testing" + "time" + + "github.com/google/pprof/profile" + . "github.com/pingcap/check" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/tracecpu" +) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +var _ = SerialSuites(&testSuite{}) + +type testSuite struct{} + +func (s *testSuite) SetUpSuite(c *C) { + cfg := config.GetGlobalConfig() + newCfg := *cfg + newCfg.TopSQL.Enable = true + newCfg.TopSQL.RefreshInterval = 1 + config.StoreGlobalConfig(&newCfg) + tracecpu.GlobalSQLStatsProfiler.Run() +} + +func (s *testSuite) TestSQLStatsProfile(c *C) { + collector := newMockStatsCollector() + tracecpu.GlobalSQLStatsProfiler.SetCollector(collector) + reqs := []struct { + sql string + plan string + }{ + {"select * from t where a=?", "point-get"}, + {"select * from t where a>?", "table-scan"}, + {"insert into t values (?)", ""}, + } + var wg sync.WaitGroup + for _, req := range reqs { + wg.Add(1) + go func(sql, plan string) { + defer wg.Done() + s.mockExecuteSQL(sql, plan) + }(req.sql, req.plan) + } + wg.Wait() + + // test for StartCPUProfile. + buf := bytes.NewBuffer(nil) + err := tracecpu.StartCPUProfile(buf) + c.Assert(err, IsNil) + time.Sleep(time.Second) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + _, err = profile.Parse(buf) + c.Assert(err, IsNil) + + // test for collect SQL stats. + for _, req := range reqs { + stats := collector.getSQLStats(req.sql, req.plan) + c.Assert(stats, NotNil) + sql := collector.getSQL(stats.SQLDigest) + plan := collector.getPlan(stats.PlanDigest) + c.Assert(sql, Equals, req.sql) + c.Assert(plan, Equals, req.plan) + } +} + +func (s *testSuite) mockExecuteSQL(sql, plan string) { + ctx := context.Background() + sqlDigest := genDigest(sql) + ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, sql, sqlDigest) + s.mockExecute(time.Millisecond * 20) + planDigest := genDigest(plan) + tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest, planDigest, plan) + s.mockExecute(time.Millisecond * 50) +} + +func genDigest(str string) string { + if str == "" { + return "" + } + hasher := sha256.New() + hasher.Write(hack.Slice(str)) + return hex.EncodeToString(hasher.Sum(nil)) +} + +func (s *testSuite) mockExecute(d time.Duration) { + start := time.Now() + for { + for i := 0; i < 10e5; i++ { + } + if time.Since(start) > d { + return + } + } +} + +type mockStatsCollector struct { + // sql_digest -> normalized SQL + sqlmu sync.Mutex + sqlMap map[string]string + // plan_digest -> normalized plan + planMu sync.Mutex + planMap map[string]string + // sql -> sql stats + sqlStatsMap map[string]*tracecpu.SQLStats +} + +func newMockStatsCollector() *mockStatsCollector { + return &mockStatsCollector{ + sqlMap: make(map[string]string), + planMap: make(map[string]string), + sqlStatsMap: make(map[string]*tracecpu.SQLStats), + } +} + +func (c *mockStatsCollector) hash(stat tracecpu.SQLStats) string { + return stat.SQLDigest + stat.PlanDigest +} + +func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { + if len(stats) == 0 { + return + } + + for _, stmt := range stats { + hash := c.hash(stmt) + stats, ok := c.sqlStatsMap[hash] + if !ok { + tmp := stmt + stats = &tmp + c.sqlStatsMap[hash] = stats + } + stats.CPUTimeMs += stmt.CPUTimeMs + } +} + +func (c *mockStatsCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { + sqlDigest, planDigest := genDigest(sql), genDigest(plan) + hash := c.hash(tracecpu.SQLStats{SQLDigest: sqlDigest, PlanDigest: planDigest}) + return c.sqlStatsMap[hash] +} + +func (c *mockStatsCollector) getSQL(sqlDigest string) string { + c.sqlmu.Lock() + sql := c.sqlMap[sqlDigest] + c.sqlmu.Unlock() + return sql +} + +func (c *mockStatsCollector) getPlan(planDigest string) string { + c.planMu.Lock() + plan := c.planMap[planDigest] + c.planMu.Unlock() + return plan +} + +func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { + c.sqlmu.Lock() + _, ok := c.sqlMap[sqlDigest] + if !ok { + c.sqlMap[sqlDigest] = normalizedSQL + } + c.sqlmu.Unlock() + +} + +func (c *mockStatsCollector) RegisterPlan(planDigest string, normalizedPlan string) { + c.planMu.Lock() + _, ok := c.planMap[planDigest] + if !ok { + c.planMap[planDigest] = normalizedPlan + } + c.planMu.Unlock() +} From 49d099bf26a30235e2889fc218a11894b87f5e9a Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 20:22:29 +0800 Subject: [PATCH 17/54] refine test and add license Signed-off-by: crazycs --- server/sql_info_fetcher.go | 8 ++--- util/tracecpu/collector.go | 13 ++++++++ util/tracecpu/profile.go | 24 +++++++++++--- util/tracecpu/profile_test.go | 59 ++++++++++++++++++++++++++++------- 4 files changed, 84 insertions(+), 20 deletions(-) diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go index 6fc80daf506d6..736bee649ec57 100644 --- a/server/sql_info_fetcher.go +++ b/server/sql_info_fetcher.go @@ -20,7 +20,6 @@ import ( "encoding/json" "fmt" "net/http" - "runtime/pprof" "strconv" "strings" "time" @@ -35,6 +34,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/tracecpu" ) type sqlInfoFetcher struct { @@ -275,13 +275,13 @@ func (sh *sqlInfoFetcher) getExplainAnalyze(ctx context.Context, sql string, res } func (sh *sqlInfoFetcher) catchCPUProfile(ctx context.Context, sec int, buf *bytes.Buffer, errChan chan<- error) { - if err := pprof.StartCPUProfile(buf); err != nil { + if err := tracecpu.StartCPUProfile(buf); err != nil { errChan <- err return } sleepWithCtx(ctx, time.Duration(sec)*time.Second) - pprof.StopCPUProfile() - errChan <- nil + err := tracecpu.StopCPUProfile() + errChan <- err } func (sh *sqlInfoFetcher) getStatsForTable(pair tableNamePair) (*handle.JSONTable, error) { diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index f2176958bb518..f1e1c527d4763 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -1,3 +1,16 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package tracecpu // SQLStatsCollector uses to collect SQL stats. diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 6bdce99dbd6f0..1607d372d7b6c 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -1,3 +1,16 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package tracecpu import ( @@ -77,7 +90,7 @@ func (sp *sqlStatsProfiler) Run() { func (sp *sqlStatsProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { - if sp.isEnabled() { + if sp.IsEnabled() { sp.doCPUProfile() } else { time.Sleep(time.Second) @@ -231,7 +244,8 @@ func (sp *sqlStatsProfiler) hasExportProfileTask() bool { return has } -func (sp *sqlStatsProfiler) isEnabled() bool { +// IsEnabled return true if it is(should be) enabled. It exports for tests. +func (sp *sqlStatsProfiler) IsEnabled() bool { return config.GetGlobalConfig().TopSQL.Enable || sp.hasExportProfileTask() } @@ -239,7 +253,7 @@ func (sp *sqlStatsProfiler) isEnabled() bool { // Because the GlobalSQLStatsProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalSQLStatsProfiler.isEnabled() { + if GlobalSQLStatsProfiler.IsEnabled() { return GlobalSQLStatsProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) @@ -248,7 +262,7 @@ func StartCPUProfile(w io.Writer) error { // StopCPUProfile same like pprof.StopCPUProfile. // other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. func StopCPUProfile() error { - if GlobalSQLStatsProfiler.isEnabled() { + if GlobalSQLStatsProfiler.IsEnabled() { return GlobalSQLStatsProfiler.stopExportCPUProfile() } pprof.StopCPUProfile() @@ -296,7 +310,7 @@ func (sp *sqlStatsProfiler) stopExportCPUProfile() error { if ept.err != nil { return ept.err } - if w := ept.w; w != nil { + if w := ept.w; w != nil && ept.cpuProfile != nil { sp.removeLabel(ept.cpuProfile) return ept.cpuProfile.Write(w) } diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 50bc4d46c17a6..0d332e4b19f51 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -1,3 +1,16 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package tracecpu_test import ( @@ -53,19 +66,12 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { s.mockExecuteSQL(sql, plan) }(req.sql, req.plan) } - wg.Wait() // test for StartCPUProfile. - buf := bytes.NewBuffer(nil) - err := tracecpu.StartCPUProfile(buf) - c.Assert(err, IsNil) - time.Sleep(time.Second) - err = tracecpu.StopCPUProfile() - c.Assert(err, IsNil) - _, err = profile.Parse(buf) - c.Assert(err, IsNil) + s.testFetchProfile(c, time.Second+time.Millisecond*200) // test for collect SQL stats. + wg.Wait() for _, req := range reqs { stats := collector.getSQLStats(req.sql, req.plan) c.Assert(stats, NotNil) @@ -76,14 +82,45 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { } } +func (s *testSuite) TestIsEnabled(c *C) { + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + config.GetGlobalConfig().TopSQL.Enable = false + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) + + config.GetGlobalConfig().TopSQL.Enable = true + err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) + c.Assert(err, IsNil) + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + config.GetGlobalConfig().TopSQL.Enable = false + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + + config.GetGlobalConfig().TopSQL.Enable = false + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) + config.GetGlobalConfig().TopSQL.Enable = true + c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) +} + +func (s *testSuite) testFetchProfile(c *C, d time.Duration) { + buf := bytes.NewBuffer(nil) + err := tracecpu.StartCPUProfile(buf) + c.Assert(err, IsNil) + time.Sleep(d) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + _, err = profile.Parse(buf) + c.Assert(err, IsNil) +} + func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := genDigest(sql) ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, sql, sqlDigest) - s.mockExecute(time.Millisecond * 20) + s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest, planDigest, plan) - s.mockExecute(time.Millisecond * 50) + s.mockExecute(time.Millisecond * 200) } func genDigest(str string) string { From e74b1f3aa927c6f469bae2079933cf4a33240e1b Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 20:46:42 +0800 Subject: [PATCH 18/54] refine code and comment Signed-off-by: crazycs --- executor/executor.go | 1 + util/tracecpu/collector.go | 2 +- util/tracecpu/profile.go | 27 ++++++++++++++------------- 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index ee3433a49c0d0..e3b377519866c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1636,6 +1636,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { if err != nil { return } + // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. if config.TopSQLEnabled() { sc.OriginalSQL = s.Text() normalizedSQL, digest := sc.SQLDigest() diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index f1e1c527d4763..969dd47b7d830 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -14,7 +14,7 @@ package tracecpu // SQLStatsCollector uses to collect SQL stats. -// TODO: add a collector. +// TODO: add a collector to collect and store the SQL stats. type SQLStatsCollector interface { Collect(ts int64, stats []SQLStats) RegisterSQL(sqlDigest, normalizedSQL string) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 1607d372d7b6c..a935002d388ec 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -61,6 +61,14 @@ func NewSQLStatsProfiler() *sqlStatsProfiler { } } +func (sp *sqlStatsProfiler) Run() { + sp.mu.Lock() + defer sp.mu.Unlock() + logutil.BgLogger().Info("cpu profiler started") + go sp.startCPUProfileWorker() + go sp.startAnalyzeProfileWorker() +} + func (sp *sqlStatsProfiler) SetCollector(c SQLStatsCollector) { sp.collector = c } @@ -79,14 +87,6 @@ func (sp *sqlStatsProfiler) RegisterPlan(planDigest string, normalizedPlan strin sp.collector.RegisterPlan(planDigest, normalizedPlan) } -func (sp *sqlStatsProfiler) Run() { - sp.mu.Lock() - defer sp.mu.Unlock() - logutil.BgLogger().Info("cpu profiler started") - go sp.startCPUProfileWorker() - go sp.startAnalyzeProfileWorker() -} - func (sp *sqlStatsProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { @@ -339,14 +339,15 @@ type exportProfileTask struct { } func (t *exportProfileTask) mergeProfile(p *profile.Profile) { - if t.err != nil { + if t.err != nil || p == nil { return } - if t.cpuProfile == nil { - t.cpuProfile = p - } else { - t.cpuProfile, t.err = profile.Merge([]*profile.Profile{t.cpuProfile, p}) + ps := make([]*profile.Profile, 0, 2) + if t.cpuProfile != nil { + ps = append(ps, t.cpuProfile) } + ps = append(ps, p) + t.cpuProfile, t.err = profile.Merge(ps) } // ProfileHTTPHandler is same as pprof.Profile. From 469f6ebabdca7b89f44405cef491f3d7121ffe09 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 10:32:32 +0800 Subject: [PATCH 19/54] fix race test Signed-off-by: crazycs --- util/tracecpu/profile_test.go | 86 +++++++++++++++++++++++------------ 1 file changed, 58 insertions(+), 28 deletions(-) diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 0d332e4b19f51..3e0a501af0182 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/tracecpu" + "github.com/uber-go/atomic" ) func TestT(t *testing.T) { @@ -67,8 +68,16 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { }(req.sql, req.plan) } + cnt := collector.getCollectCnt() // test for StartCPUProfile. - s.testFetchProfile(c, time.Second+time.Millisecond*200) + buf := bytes.NewBuffer(nil) + err := tracecpu.StartCPUProfile(buf) + c.Assert(err, IsNil) + s.waitCollectCnt(collector, cnt+1) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + _, err = profile.Parse(buf) + c.Assert(err, IsNil) // test for collect SQL stats. wg.Wait() @@ -84,33 +93,45 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { func (s *testSuite) TestIsEnabled(c *C) { c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) - config.GetGlobalConfig().TopSQL.Enable = false + s.setTopSQLEnable(false) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) - config.GetGlobalConfig().TopSQL.Enable = true + s.setTopSQLEnable(true) err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) c.Assert(err, IsNil) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) - config.GetGlobalConfig().TopSQL.Enable = false + s.setTopSQLEnable(false) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) err = tracecpu.StopCPUProfile() c.Assert(err, IsNil) - config.GetGlobalConfig().TopSQL.Enable = false + s.setTopSQLEnable(false) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) - config.GetGlobalConfig().TopSQL.Enable = true + s.setTopSQLEnable(true) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) } -func (s *testSuite) testFetchProfile(c *C, d time.Duration) { - buf := bytes.NewBuffer(nil) - err := tracecpu.StartCPUProfile(buf) - c.Assert(err, IsNil) - time.Sleep(d) - err = tracecpu.StopCPUProfile() - c.Assert(err, IsNil) - _, err = profile.Parse(buf) - c.Assert(err, IsNil) +func (s *testSuite) waitCollectCnt(collector *mockStatsCollector, cnt int64) { + timeout := time.After(time.Second * 5) + for { + // Wait for collector collect sql stats count >= expected count + if collector.getCollectCnt() >= cnt { + break + } + select { + case <-timeout: + break + default: + time.Sleep(time.Millisecond * 10) + } + } +} + +func (s *testSuite) setTopSQLEnable(enabled bool) { + cfg := config.GetGlobalConfig() + newCfg := *cfg + newCfg.TopSQL.Enable = enabled + config.StoreGlobalConfig(&newCfg) } func (s *testSuite) mockExecuteSQL(sql, plan string) { @@ -144,14 +165,14 @@ func (s *testSuite) mockExecute(d time.Duration) { } type mockStatsCollector struct { + sync.Mutex // sql_digest -> normalized SQL - sqlmu sync.Mutex sqlMap map[string]string // plan_digest -> normalized plan - planMu sync.Mutex planMap map[string]string - // sql -> sql stats + // (sql + plan_digest) -> sql stats sqlStatsMap map[string]*tracecpu.SQLStats + collectCnt atomic.Int64 } func newMockStatsCollector() *mockStatsCollector { @@ -167,10 +188,12 @@ func (c *mockStatsCollector) hash(stat tracecpu.SQLStats) string { } func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { + c.collectCnt.Inc() if len(stats) == 0 { return } - + c.Lock() + defer c.Unlock() for _, stmt := range stats { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] @@ -183,41 +206,48 @@ func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { } } +func (c *mockStatsCollector) getCollectCnt() int64 { + return c.collectCnt.Load() +} + func (c *mockStatsCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { + c.Lock() sqlDigest, planDigest := genDigest(sql), genDigest(plan) hash := c.hash(tracecpu.SQLStats{SQLDigest: sqlDigest, PlanDigest: planDigest}) - return c.sqlStatsMap[hash] + tmp := c.sqlStatsMap[hash] + c.Unlock() + return tmp } func (c *mockStatsCollector) getSQL(sqlDigest string) string { - c.sqlmu.Lock() + c.Lock() sql := c.sqlMap[sqlDigest] - c.sqlmu.Unlock() + c.Unlock() return sql } func (c *mockStatsCollector) getPlan(planDigest string) string { - c.planMu.Lock() + c.Lock() plan := c.planMap[planDigest] - c.planMu.Unlock() + c.Unlock() return plan } func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { - c.sqlmu.Lock() + c.Lock() _, ok := c.sqlMap[sqlDigest] if !ok { c.sqlMap[sqlDigest] = normalizedSQL } - c.sqlmu.Unlock() + c.Unlock() } func (c *mockStatsCollector) RegisterPlan(planDigest string, normalizedPlan string) { - c.planMu.Lock() + c.Lock() _, ok := c.planMap[planDigest] if !ok { c.planMap[planDigest] = normalizedPlan } - c.planMu.Unlock() + c.Unlock() } From db33a404f9b30b3dda970b83d4526f6bbd70bb13 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 10:50:59 +0800 Subject: [PATCH 20/54] make test stable Signed-off-by: crazycs --- util/tracecpu/profile_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 3e0a501af0182..3a877faacea68 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -73,7 +73,7 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { buf := bytes.NewBuffer(nil) err := tracecpu.StartCPUProfile(buf) c.Assert(err, IsNil) - s.waitCollectCnt(collector, cnt+1) + s.waitCollectCnt(collector, cnt+2) err = tracecpu.StopCPUProfile() c.Assert(err, IsNil) _, err = profile.Parse(buf) @@ -141,7 +141,7 @@ func (s *testSuite) mockExecuteSQL(sql, plan string) { s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest, planDigest, plan) - s.mockExecute(time.Millisecond * 200) + s.mockExecute(time.Millisecond * 300) } func genDigest(str string) string { @@ -188,7 +188,7 @@ func (c *mockStatsCollector) hash(stat tracecpu.SQLStats) string { } func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { - c.collectCnt.Inc() + defer c.collectCnt.Inc() if len(stats) == 0 { return } From 94448170264b01da63b7dea4bf7cc7b4fdf24419 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 10:54:50 +0800 Subject: [PATCH 21/54] rename variable Signed-off-by: crazycs --- util/tracecpu/collector.go | 4 ++-- util/tracecpu/profile.go | 4 ++-- util/tracecpu/profile_test.go | 24 ++++++++++++------------ 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index 969dd47b7d830..6c62d10c5022b 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -13,9 +13,9 @@ package tracecpu -// SQLStatsCollector uses to collect SQL stats. +// TopSQLCollector uses to collect SQL stats. // TODO: add a collector to collect and store the SQL stats. -type SQLStatsCollector interface { +type TopSQLCollector interface { Collect(ts int64, stats []SQLStats) RegisterSQL(sqlDigest, normalizedSQL string) RegisterPlan(planDigest string, normalizedPlan string) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index a935002d388ec..36eda2be33624 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -50,7 +50,7 @@ type sqlStatsProfiler struct { sync.Mutex ept *exportProfileTask } - collector SQLStatsCollector + collector TopSQLCollector } // NewSQLStatsProfiler create a sqlStatsProfiler. @@ -69,7 +69,7 @@ func (sp *sqlStatsProfiler) Run() { go sp.startAnalyzeProfileWorker() } -func (sp *sqlStatsProfiler) SetCollector(c SQLStatsCollector) { +func (sp *sqlStatsProfiler) SetCollector(c TopSQLCollector) { sp.collector = c } diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 3a877faacea68..78c817b8bacf8 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -111,7 +111,7 @@ func (s *testSuite) TestIsEnabled(c *C) { c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) } -func (s *testSuite) waitCollectCnt(collector *mockStatsCollector, cnt int64) { +func (s *testSuite) waitCollectCnt(collector *mockCollector, cnt int64) { timeout := time.After(time.Second * 5) for { // Wait for collector collect sql stats count >= expected count @@ -164,7 +164,7 @@ func (s *testSuite) mockExecute(d time.Duration) { } } -type mockStatsCollector struct { +type mockCollector struct { sync.Mutex // sql_digest -> normalized SQL sqlMap map[string]string @@ -175,19 +175,19 @@ type mockStatsCollector struct { collectCnt atomic.Int64 } -func newMockStatsCollector() *mockStatsCollector { - return &mockStatsCollector{ +func newMockStatsCollector() *mockCollector { + return &mockCollector{ sqlMap: make(map[string]string), planMap: make(map[string]string), sqlStatsMap: make(map[string]*tracecpu.SQLStats), } } -func (c *mockStatsCollector) hash(stat tracecpu.SQLStats) string { +func (c *mockCollector) hash(stat tracecpu.SQLStats) string { return stat.SQLDigest + stat.PlanDigest } -func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { +func (c *mockCollector) Collect(ts int64, stats []tracecpu.SQLStats) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -206,11 +206,11 @@ func (c *mockStatsCollector) Collect(ts int64, stats []tracecpu.SQLStats) { } } -func (c *mockStatsCollector) getCollectCnt() int64 { +func (c *mockCollector) getCollectCnt() int64 { return c.collectCnt.Load() } -func (c *mockStatsCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { +func (c *mockCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { c.Lock() sqlDigest, planDigest := genDigest(sql), genDigest(plan) hash := c.hash(tracecpu.SQLStats{SQLDigest: sqlDigest, PlanDigest: planDigest}) @@ -219,21 +219,21 @@ func (c *mockStatsCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { return tmp } -func (c *mockStatsCollector) getSQL(sqlDigest string) string { +func (c *mockCollector) getSQL(sqlDigest string) string { c.Lock() sql := c.sqlMap[sqlDigest] c.Unlock() return sql } -func (c *mockStatsCollector) getPlan(planDigest string) string { +func (c *mockCollector) getPlan(planDigest string) string { c.Lock() plan := c.planMap[planDigest] c.Unlock() return plan } -func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { +func (c *mockCollector) RegisterSQL(sqlDigest, normalizedSQL string) { c.Lock() _, ok := c.sqlMap[sqlDigest] if !ok { @@ -243,7 +243,7 @@ func (c *mockStatsCollector) RegisterSQL(sqlDigest, normalizedSQL string) { } -func (c *mockStatsCollector) RegisterPlan(planDigest string, normalizedPlan string) { +func (c *mockCollector) RegisterPlan(planDigest string, normalizedPlan string) { c.Lock() _, ok := c.planMap[planDigest] if !ok { From 9a54407a6372495379f356eb141407b1c4301a3d Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 13:21:19 +0800 Subject: [PATCH 22/54] add sleep if profile failed Signed-off-by: crazycs --- util/tracecpu/profile.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 36eda2be33624..d53a4d78ccd50 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -102,6 +102,8 @@ func (sp *sqlStatsProfiler) doCPUProfile() { interval := config.GetGlobalConfig().TopSQL.RefreshInterval task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { + // Sleep a while before retry. + time.Sleep(time.Millisecond) return } ns := int(time.Second)*interval - time.Now().Nanosecond() From 2026c29fa36652e98fc10d6157441afe423bbc24 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 14:17:50 +0800 Subject: [PATCH 23/54] reduce normalize Signed-off-by: crazycs --- executor/executor.go | 14 ++++++-------- planner/optimize.go | 8 ++++---- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 0b60a2f1b4e78..5271b04a55e41 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1653,17 +1653,15 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.MemTracker.SetActionOnExceed(action) } if execStmt, ok := s.(*ast.ExecuteStmt); ok { - s, err = planner.GetPreparedStmt(execStmt, vars) + prepareStmt, err := planner.GetPreparedStmt(execStmt, vars) if err != nil { - return + return err } + s = prepareStmt.PreparedAst.Stmt + sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. - if config.TopSQLEnabled() { - sc.OriginalSQL = s.Text() - normalizedSQL, digest := sc.SQLDigest() - if digest != nil { - tracecpu.SetGoroutineLabelsWithSQL(context.Background(), normalizedSQL, digest.String()) - } + if config.TopSQLEnabled() && prepareStmt.SQLDigest != nil { + tracecpu.SetGoroutineLabelsWithSQL(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) } } // execute missed stmtID uses empty sql diff --git a/planner/optimize.go b/planner/optimize.go index c667d6b124ef9..6d87d6ffac5ed 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -44,7 +44,7 @@ import ( ) // GetPreparedStmt extract the prepared statement from the execute statement. -func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.StmtNode, error) { +func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*plannercore.CachedPrepareStmt, error) { var ok bool execID := stmt.ExecID if stmt.Name != "" { @@ -57,7 +57,7 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.Stm if !ok { return nil, errors.Errorf("invalid CachedPrepareStmt type") } - return preparedObj.PreparedAst.Stmt, nil + return preparedObj, nil } return nil, plannercore.ErrStmtNotFound } @@ -65,12 +65,12 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.Stm // IsReadOnly check whether the ast.Node is a read only statement. func IsReadOnly(node ast.Node, vars *variable.SessionVars) bool { if execStmt, isExecStmt := node.(*ast.ExecuteStmt); isExecStmt { - s, err := GetPreparedStmt(execStmt, vars) + prepareStmt, err := GetPreparedStmt(execStmt, vars) if err != nil { logutil.BgLogger().Warn("GetPreparedStmt failed", zap.Error(err)) return false } - return ast.IsReadOnly(s) + return ast.IsReadOnly(prepareStmt.PreparedAst.Stmt) } return ast.IsReadOnly(node) } From fa98a85cde17170f3937d58e992cb98e422b9134 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 26 May 2021 21:07:15 +0800 Subject: [PATCH 24/54] add more test Signed-off-by: crazycs --- server/tidb_test.go | 233 ++++++++++++++++++++++++++++++++++ util/tracecpu/mock/mock.go | 167 ++++++++++++++++++++++++ util/tracecpu/profile_test.go | 150 +++------------------- 3 files changed, 418 insertions(+), 132 deletions(-) create mode 100644 util/tracecpu/mock/mock.go diff --git a/server/tidb_test.go b/server/tidb_test.go index 426e45cd0c57b..14f4027ef9260 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -15,23 +15,29 @@ package server import ( + "bytes" "context" "crypto/rand" "crypto/rsa" "crypto/tls" "crypto/x509" "crypto/x509/pkix" + "database/sql" "encoding/pem" + "fmt" "math/big" "net/http" "os" "path/filepath" + "regexp" + "strings" "sync/atomic" "time" "github.com/go-sql-driver/mysql" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/parser" tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" @@ -43,7 +49,10 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/tracecpu/mock" ) type tidbTestSuite struct { @@ -54,6 +63,10 @@ type tidbTestSerialSuite struct { *tidbTestSuiteBase } +type tidbTestTopSQLSuite struct { + *tidbTestSuiteBase +} + type tidbTestSuiteBase struct { *testServerClient tidbdrv *TiDBDriver @@ -70,12 +83,22 @@ func newTiDBTestSuiteBase() *tidbTestSuiteBase { var _ = Suite(&tidbTestSuite{newTiDBTestSuiteBase()}) var _ = SerialSuites(&tidbTestSerialSuite{newTiDBTestSuiteBase()}) +var _ = SerialSuites(&tidbTestTopSQLSuite{newTiDBTestSuiteBase()}) func (ts *tidbTestSuite) SetUpSuite(c *C) { metrics.RegisterMetrics() ts.tidbTestSuiteBase.SetUpSuite(c) } +func (ts *tidbTestTopSQLSuite) SetUpSuite(c *C) { + ts.tidbTestSuiteBase.SetUpSuite(c) + config.UpdateGlobal(func(conf *config.Config) { + conf.TopSQL.Enable = true + conf.TopSQL.RefreshInterval = 1 + }) + tracecpu.GlobalSQLStatsProfiler.Run() +} + func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { var err error ts.store, err = mockstore.NewMockStore() @@ -1153,3 +1176,213 @@ func (ts *tidbTestSerialSuite) TestPrepareCount(c *C) { c.Assert(err, IsNil) c.Assert(atomic.LoadInt64(&variable.PreparedStmtCount), Equals, prepareCnt) } + +func (ts *tidbTestTopSQLSuite) TestTopSQLStatsProfile(c *C) { + db, err := sql.Open("mysql", ts.getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer func() { + err := db.Close() + c.Assert(err, IsNil) + }() + collector := mock.NewTopSQLCollector() + tracecpu.GlobalSQLStatsProfiler.SetCollector(collector) + + dbt := &DBTest{c, db} + dbt.mustExec("drop database if exists topsql") + dbt.mustExec("create database topsql") + dbt.mustExec("use topsql;") + dbt.mustExec("create table t (a int auto_increment, b int, unique index idx(a));") + dbt.mustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") + dbt.mustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") + + // Test case 1: DML query: insert/update/replace/delete/select + cases1 := []struct { + sql string + planRegexp string + cancel func() + }{ + {sql: "insert into t () values (),(),(),(),(),(),();", planRegexp: ""}, + {sql: "insert into t (b) values (1),(1),(1),(1),(1),(1),(1),(1);", planRegexp: ""}, + {sql: "replace into t (b) values (1),(1),(1),(1),(1),(1),(1),(1);", planRegexp: ""}, + {sql: "update t set b=a where b is null limit 1;", planRegexp: ".*Limit.*TableReader.*"}, + {sql: "delete from t where b is null limit 2;", planRegexp: ".*Limit.*TableReader.*"}, + {sql: "select * from t use index(idx) where a>0;", planRegexp: ".*IndexLookUp.*"}, + {sql: "select * from t ignore index(idx) where a>0;", planRegexp: ".*TableReader.*"}, + {sql: "select /*+ HASH_JOIN(t1, t2) */ * from t t1 join t t2 on t1.a=t2.a where t1.b is not null;", planRegexp: ".*HashJoin.*"}, + {sql: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t t1 join t t2 on t1.a=t2.a where t1.b is not null;", planRegexp: ".*IndexHashJoin.*"}, + {sql: "select * from t where a=1;", planRegexp: ".*Point_Get.*"}, + {sql: "select * from t where a in (1,2,3,4)", planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases1 { + ctx, cancel := context.WithCancel(context.Background()) + cases1[i].cancel = cancel + sqlStr := ca.sql + go ts.loopExec(ctx, c, func(db *sql.DB) { + dbt := &DBTest{c, db} + if strings.HasPrefix(sqlStr, "select") { + rows := dbt.mustQuery(sqlStr) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + db.Exec(sqlStr) + } + }) + } + + // Test case 2: prepare/execute sql + cases2 := []struct { + prepare string + args []interface{} + planRegexp string + cancel func() + }{ + {prepare: "insert into t1 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "replace into t1 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "update t1 set b=a where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "delete from t1 where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "select * from t1 use index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*IndexLookUp.*"}, + {prepare: "select * from t1 ignore index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*TableReader.*"}, + {prepare: "select /*+ HASH_JOIN(t1, t2) */ * from t1 t1 join t1 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*HashJoin.*"}, + {prepare: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t1 t1 join t1 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*IndexHashJoin.*"}, + {prepare: "select * from t1 where a=?;", args: []interface{}{1}, planRegexp: ".*Point_Get.*"}, + {prepare: "select * from t1 where a in (?,?,?,?)", args: []interface{}{1, 2, 3, 4}, planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases2 { + ctx, cancel := context.WithCancel(context.Background()) + cases2[i].cancel = cancel + prepare, args := ca.prepare, ca.args + go ts.loopExec(ctx, c, func(db *sql.DB) { + stmt, err := db.Prepare(prepare) + c.Assert(err, IsNil) + if strings.HasPrefix(prepare, "select") { + rows, err := stmt.Query(args...) + c.Assert(err, IsNil) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + stmt.Exec(args...) + } + }) + } + + // Test case 3: prepare, execute stmt using @val... + cases3 := []struct { + prepare string + args []interface{} + planRegexp string + cancel func() + }{ + {prepare: "insert into t2 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "replace into t2 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "update t2 set b=a where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "delete from t2 where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "select * from t2 use index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*IndexLookUp.*"}, + {prepare: "select * from t2 ignore index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*TableReader.*"}, + {prepare: "select /*+ HASH_JOIN(t1, t2) */ * from t2 t1 join t2 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*HashJoin.*"}, + {prepare: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t2 t1 join t2 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*IndexHashJoin.*"}, + {prepare: "select * from t2 where a=?;", args: []interface{}{1}, planRegexp: ".*Point_Get.*"}, + {prepare: "select * from t2 where a in (?,?,?,?)", args: []interface{}{1, 2, 3, 4}, planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases3 { + ctx, cancel := context.WithCancel(context.Background()) + cases3[i].cancel = cancel + prepare, args := ca.prepare, ca.args + go ts.loopExec(ctx, c, func(db *sql.DB) { + _, err := db.Exec(fmt.Sprintf("prepare stmt from '%v'", prepare)) + c.Assert(err, IsNil) + sqlBuf := bytes.NewBuffer(nil) + sqlBuf.WriteString("execute stmt ") + for i := range args { + _, err = db.Exec(fmt.Sprintf("set @%c=%v", 'a'+i, args[i])) + c.Assert(err, IsNil) + if i == 0 { + sqlBuf.WriteString("using ") + } else { + sqlBuf.WriteByte(',') + } + sqlBuf.WriteByte('@') + sqlBuf.WriteByte('a' + byte(i)) + } + if strings.HasPrefix(prepare, "select") { + rows, err := db.Query(sqlBuf.String()) + c.Assert(err, IsNil, Commentf("%v", sqlBuf.String())) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + db.Exec(sqlBuf.String()) + } + }) + } + + // Wait the top sql collector to collect profile data. + collector.WaitCollectCnt(1) + + checkFn := func(sql, planRegexp string) { + commentf := Commentf("sql: %v", sql) + stats := collector.GetSQLStatsBySQLWithRetry(sql, len(planRegexp) > 0) + // since 1 sql may has many plan, check `len(stats) > 0` instead of `len(stats) == 1`. + c.Assert(len(stats) > 0, IsTrue, commentf) + + match := false + for _, s := range stats { + sqlStr := collector.GetSQL(s.SQLDigest) + encodedPlan := collector.GetPlan(s.PlanDigest) + // Normalize the user SQL before check. + normalizedSQL := parser.Normalize(sql) + c.Assert(sqlStr, Equals, normalizedSQL, commentf) + // decode plan before check. + normalizedPlan, err := plancodec.DecodeNormalizedPlan(encodedPlan) + c.Assert(err, IsNil) + // remove '\n' '\t' before do regexp match. + normalizedPlan = strings.Replace(normalizedPlan, "\n", " ", -1) + normalizedPlan = strings.Replace(normalizedPlan, "\t", " ", -1) + ok, err := regexp.MatchString(planRegexp, normalizedPlan) + c.Assert(err, IsNil, commentf) + if ok { + match = true + break + } + } + c.Assert(match, IsTrue, commentf) + } + + // Check result of test case 1. + for _, ca := range cases1 { + checkFn(ca.sql, ca.planRegexp) + ca.cancel() + } + + // Check result of test case 2. + for _, ca := range cases2 { + checkFn(ca.prepare, ca.planRegexp) + ca.cancel() + } + + // Check result of test case 3. + for _, ca := range cases3 { + checkFn(ca.prepare, ca.planRegexp) + ca.cancel() + } +} + +func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, c *C, fn func(db *sql.DB)) { + db, err := sql.Open("mysql", ts.getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer func() { + err := db.Close() + c.Assert(err, IsNil) + }() + dbt := &DBTest{c, db} + dbt.mustExec("use topsql;") + for { + select { + case <-ctx.Done(): + return + default: + } + fn(db) + } +} diff --git a/util/tracecpu/mock/mock.go b/util/tracecpu/mock/mock.go new file mode 100644 index 0000000000000..fa9de4474836b --- /dev/null +++ b/util/tracecpu/mock/mock.go @@ -0,0 +1,167 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package mock + +import ( + "sync" + "time" + + "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/tracecpu" + "github.com/uber-go/atomic" +) + +// TopSQLCollector uses for testing. +type TopSQLCollector struct { + sync.Mutex + // sql_digest -> normalized SQL + sqlMap map[string]string + // plan_digest -> normalized plan + planMap map[string]string + // (sql + plan_digest) -> sql stats + sqlStatsMap map[string]*tracecpu.SQLStats + collectCnt atomic.Int64 +} + +// NewTopSQLCollector uses for testing. +func NewTopSQLCollector() *TopSQLCollector { + return &TopSQLCollector{ + sqlMap: make(map[string]string), + planMap: make(map[string]string), + sqlStatsMap: make(map[string]*tracecpu.SQLStats), + } +} + +// Collect uses for testing. +func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLStats) { + defer c.collectCnt.Inc() + if len(stats) == 0 { + return + } + c.Lock() + defer c.Unlock() + for _, stmt := range stats { + hash := c.hash(stmt) + stats, ok := c.sqlStatsMap[hash] + if !ok { + tmp := stmt + stats = &tmp + c.sqlStatsMap[hash] = stats + } + stats.CPUTimeMs += stmt.CPUTimeMs + } +} + +// GetSQLStatsBySQLWithRetry uses for testing. +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLStats { + after := time.After(time.Second * 10) + for { + select { + case <-after: + return nil + default: + } + stats := c.GetSQLStatsBySQL(sql, planIsNotNull) + if len(stats) > 0 { + return stats + } + c.WaitCollectCnt(1) + } +} + +// GetSQLStatsBySQL uses for testing. +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLStats { + stats := make([]*tracecpu.SQLStats, 0, 2) + sqlDigest := GenSQLDigest(sql) + c.Lock() + for _, stmt := range c.sqlStatsMap { + if stmt.SQLDigest == sqlDigest { + if planIsNotNull { + plan := c.planMap[stmt.PlanDigest] + if len(plan) > 0 { + stats = append(stats, stmt) + } + } else { + stats = append(stats, stmt) + } + } + } + c.Unlock() + return stats +} + +// GetSQL uses for testing. +func (c *TopSQLCollector) GetSQL(sqlDigest string) string { + c.Lock() + sql := c.sqlMap[sqlDigest] + c.Unlock() + return sql +} + +// GetPlan uses for testing. +func (c *TopSQLCollector) GetPlan(planDigest string) string { + c.Lock() + plan := c.planMap[planDigest] + c.Unlock() + return plan +} + +// RegisterSQL uses for testing. +func (c *TopSQLCollector) RegisterSQL(sqlDigest, normalizedSQL string) { + c.Lock() + _, ok := c.sqlMap[sqlDigest] + if !ok { + c.sqlMap[sqlDigest] = normalizedSQL + } + c.Unlock() + +} + +// RegisterPlan uses for testing. +func (c *TopSQLCollector) RegisterPlan(planDigest string, normalizedPlan string) { + c.Lock() + _, ok := c.planMap[planDigest] + if !ok { + c.planMap[planDigest] = normalizedPlan + } + c.Unlock() +} + +// WaitCollectCnt uses for testing. +func (c *TopSQLCollector) WaitCollectCnt(count int64) { + timeout := time.After(time.Second * 10) + end := c.collectCnt.Load() + count + for { + // Wait for collector collect sql stats count >= expected count + if c.collectCnt.Load() >= end { + break + } + select { + case <-timeout: + break + default: + time.Sleep(time.Millisecond * 10) + } + } +} + +func (c *TopSQLCollector) hash(stat tracecpu.SQLStats) string { + return stat.SQLDigest + stat.PlanDigest +} + +// GenSQLDigest uses for testing. +func GenSQLDigest(sql string) string { + _, digest := parser.NormalizeDigest(sql) + return digest.String() +} diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 78c817b8bacf8..c53127d285097 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -16,18 +16,15 @@ package tracecpu_test import ( "bytes" "context" - "crypto/sha256" - "encoding/hex" - "sync" "testing" "time" "github.com/google/pprof/profile" . "github.com/pingcap/check" + "github.com/pingcap/parser" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/tracecpu" - "github.com/uber-go/atomic" + "github.com/pingcap/tidb/util/tracecpu/mock" ) func TestT(t *testing.T) { @@ -48,8 +45,8 @@ func (s *testSuite) SetUpSuite(c *C) { tracecpu.GlobalSQLStatsProfiler.Run() } -func (s *testSuite) TestSQLStatsProfile(c *C) { - collector := newMockStatsCollector() +func (s *testSuite) TestTopSQLStatsProfile(c *C) { + collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLStatsProfiler.SetCollector(collector) reqs := []struct { sql string @@ -59,40 +56,36 @@ func (s *testSuite) TestSQLStatsProfile(c *C) { {"select * from t where a>?", "table-scan"}, {"insert into t values (?)", ""}, } - var wg sync.WaitGroup + for _, req := range reqs { - wg.Add(1) go func(sql, plan string) { - defer wg.Done() - s.mockExecuteSQL(sql, plan) + for { + s.mockExecuteSQL(sql, plan) + } }(req.sql, req.plan) } - cnt := collector.getCollectCnt() // test for StartCPUProfile. buf := bytes.NewBuffer(nil) err := tracecpu.StartCPUProfile(buf) c.Assert(err, IsNil) - s.waitCollectCnt(collector, cnt+2) + collector.WaitCollectCnt(2) err = tracecpu.StopCPUProfile() c.Assert(err, IsNil) _, err = profile.Parse(buf) c.Assert(err, IsNil) - // test for collect SQL stats. - wg.Wait() for _, req := range reqs { - stats := collector.getSQLStats(req.sql, req.plan) - c.Assert(stats, NotNil) - sql := collector.getSQL(stats.SQLDigest) - plan := collector.getPlan(stats.PlanDigest) + stats := collector.GetSQLStatsBySQLWithRetry(req.sql, len(req.plan) > 0) + c.Assert(len(stats), Equals, 1) + sql := collector.GetSQL(stats[0].SQLDigest) + plan := collector.GetPlan(stats[0].PlanDigest) c.Assert(sql, Equals, req.sql) c.Assert(plan, Equals, req.plan) } } func (s *testSuite) TestIsEnabled(c *C) { - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) s.setTopSQLEnable(false) c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) @@ -111,32 +104,15 @@ func (s *testSuite) TestIsEnabled(c *C) { c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) } -func (s *testSuite) waitCollectCnt(collector *mockCollector, cnt int64) { - timeout := time.After(time.Second * 5) - for { - // Wait for collector collect sql stats count >= expected count - if collector.getCollectCnt() >= cnt { - break - } - select { - case <-timeout: - break - default: - time.Sleep(time.Millisecond * 10) - } - } -} - func (s *testSuite) setTopSQLEnable(enabled bool) { - cfg := config.GetGlobalConfig() - newCfg := *cfg - newCfg.TopSQL.Enable = enabled - config.StoreGlobalConfig(&newCfg) + config.UpdateGlobal(func(conf *config.Config) { + conf.TopSQL.Enable = enabled + }) } func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() - sqlDigest := genDigest(sql) + sqlDigest := mock.GenSQLDigest(sql) ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, sql, sqlDigest) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) @@ -148,9 +124,7 @@ func genDigest(str string) string { if str == "" { return "" } - hasher := sha256.New() - hasher.Write(hack.Slice(str)) - return hex.EncodeToString(hasher.Sum(nil)) + return parser.DigestNormalized(str).String() } func (s *testSuite) mockExecute(d time.Duration) { @@ -163,91 +137,3 @@ func (s *testSuite) mockExecute(d time.Duration) { } } } - -type mockCollector struct { - sync.Mutex - // sql_digest -> normalized SQL - sqlMap map[string]string - // plan_digest -> normalized plan - planMap map[string]string - // (sql + plan_digest) -> sql stats - sqlStatsMap map[string]*tracecpu.SQLStats - collectCnt atomic.Int64 -} - -func newMockStatsCollector() *mockCollector { - return &mockCollector{ - sqlMap: make(map[string]string), - planMap: make(map[string]string), - sqlStatsMap: make(map[string]*tracecpu.SQLStats), - } -} - -func (c *mockCollector) hash(stat tracecpu.SQLStats) string { - return stat.SQLDigest + stat.PlanDigest -} - -func (c *mockCollector) Collect(ts int64, stats []tracecpu.SQLStats) { - defer c.collectCnt.Inc() - if len(stats) == 0 { - return - } - c.Lock() - defer c.Unlock() - for _, stmt := range stats { - hash := c.hash(stmt) - stats, ok := c.sqlStatsMap[hash] - if !ok { - tmp := stmt - stats = &tmp - c.sqlStatsMap[hash] = stats - } - stats.CPUTimeMs += stmt.CPUTimeMs - } -} - -func (c *mockCollector) getCollectCnt() int64 { - return c.collectCnt.Load() -} - -func (c *mockCollector) getSQLStats(sql, plan string) *tracecpu.SQLStats { - c.Lock() - sqlDigest, planDigest := genDigest(sql), genDigest(plan) - hash := c.hash(tracecpu.SQLStats{SQLDigest: sqlDigest, PlanDigest: planDigest}) - tmp := c.sqlStatsMap[hash] - c.Unlock() - return tmp -} - -func (c *mockCollector) getSQL(sqlDigest string) string { - c.Lock() - sql := c.sqlMap[sqlDigest] - c.Unlock() - return sql -} - -func (c *mockCollector) getPlan(planDigest string) string { - c.Lock() - plan := c.planMap[planDigest] - c.Unlock() - return plan -} - -func (c *mockCollector) RegisterSQL(sqlDigest, normalizedSQL string) { - c.Lock() - _, ok := c.sqlMap[sqlDigest] - if !ok { - c.sqlMap[sqlDigest] = normalizedSQL - } - c.Unlock() - -} - -func (c *mockCollector) RegisterPlan(planDigest string, normalizedPlan string) { - c.Lock() - _, ok := c.planMap[planDigest] - if !ok { - c.planMap[planDigest] = normalizedPlan - } - c.Unlock() -} From 04d78383abd0e60c4645fe0e47b235ce6221177e Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 11:53:27 +0800 Subject: [PATCH 25/54] add more comment Signed-off-by: crazycs --- util/tracecpu/profile.go | 59 ++++++++++++++++++++++++++++++---------- 1 file changed, 45 insertions(+), 14 deletions(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index d53a4d78ccd50..ec6f765082a74 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -127,7 +127,7 @@ func (sp *sqlStatsProfiler) startAnalyzeProfileWorker() { logutil.BgLogger().Error("parse profile error", zap.Error(err)) continue } - stats := sp.parseCPUProfileTags(p) + stats := sp.parseCPUProfileBySQLLabels(p) sp.handleExportProfileTask(p) if sp.collector != nil { sp.collector.Collect(task.end, stats) @@ -161,7 +161,13 @@ func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { } } -func (sp *sqlStatsProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { +// parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, +// output the SQLStats slice. +// The sql_digest label is been set by `SetGoroutineLabelsWithSQL` function after parse the SQL. +// The plan_digest label is been set by `SetGoroutineLabelsWithSQLAndPlan` function after build the SQL plan. +// Since `sqlStatsProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data +// without those label will be ignore. +func (sp *sqlStatsProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLStats { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -173,9 +179,8 @@ func (sp *sqlStatsProfiler) parseCPUProfileTags(p *profile.Profile) []SQLStats { stmt, ok := sqlMap[digest] if !ok { stmt = &sqlStats{ - plans: make(map[string]int64), - total: 0, - isInternal: false, + plans: make(map[string]int64), + total: 0, } sqlMap[digest] = stmt } @@ -206,12 +211,25 @@ func (sp *sqlStatsProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLSta } type sqlStats struct { - plans map[string]int64 - total int64 - isInternal bool -} - -// tune use to adjust stats + plans map[string]int64 + total int64 +} + +// tune use to adjust sql stats. Consider following situation: +// The `sqlStats` maybe: +// plans: { +// "table_scan": 200ms, // The cpu time of the sql that plan with `table_scan` is 200ms. +// "index_scan": 300ms, // The cpu time of the sql that plan with `table_scan` is 300ms. +// }, +// total: 600ms, // The total cpu time of the sql is 600ms. +// total_time - table_scan_time - index_scan_time = 100ms, and this 100ms means those sample data only contain the +// sql_digest label, doesn't contain the plan_digest label. This is cause by the `pprof profile` is base on sample. +// After this tune function, the `sqlStats` become to: +// plans: { +// "table_scan": 240ms, // 200 + (200/(200+300))*100 +// "index_scan": 360ms, // 300 + (300/(200+300))*100 +// }, +// total: 600ms, func (s *sqlStats) tune() { if len(s.plans) == 0 { s.plans[""] = s.total @@ -319,6 +337,9 @@ func (sp *sqlStatsProfiler) stopExportCPUProfile() error { return nil } +// removeLabel uses to remove labels for export cpu profile data. +// Since the sql_digest and plan_digest label is strange for other users. +// If `variable.EnablePProfSQLCPU` is true means wanto keep the `sql` label, otherwise, remove the `sql` label too. func (sp *sqlStatsProfiler) removeLabel(p *profile.Profile) { if p == nil { return @@ -326,10 +347,14 @@ func (sp *sqlStatsProfiler) removeLabel(p *profile.Profile) { keepLabelSQL := variable.EnablePProfSQLCPU.Load() for _, s := range p.Sample { for k := range s.Label { - if keepLabelSQL && k == labelSQL { - continue + switch k { + case labelSQL: + if !keepLabelSQL { + delete(s.Label, k) + } + case labelSQLDigest, labelPlanDigest: + delete(s.Label, k) } - delete(s.Label, k) } } } @@ -376,6 +401,12 @@ func ProfileHTTPHandler(w http.ResponseWriter, r *http.Request) { serveError(w, http.StatusInternalServerError, "Could not enable CPU profiling: "+err.Error()) return } + // TODO: fix me. + // |<-- 1s -->| + // -|----------|----------|----------|----------|----------|-----------|-----> Background profile task timeline. + // |________________________________| + // (start cpu profile) v v (stop cpu profile) // expected profile timeline + // |________________________________| // actual profile timeline time.Sleep(time.Second * time.Duration(sec)) err = StopCPUProfile() if err != nil { From 15986e2a45d7de2fcb083ade72457f6693bcf2e1 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 11:56:08 +0800 Subject: [PATCH 26/54] add link Signed-off-by: crazycs --- util/tracecpu/profile.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index ec6f765082a74..ff5c374acb701 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -162,7 +162,7 @@ func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { } // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, -// output the SQLStats slice. +// output the SQLStats slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetGoroutineLabelsWithSQL` function after parse the SQL. // The plan_digest label is been set by `SetGoroutineLabelsWithSQLAndPlan` function after build the SQL plan. // Since `sqlStatsProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data From 080dba19344e402170771ee08a70f440223803e7 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 13:40:56 +0800 Subject: [PATCH 27/54] use prepared stmt normalized sql to avoid re-normalize the prepare sql Signed-off-by: crazycs --- server/conn.go | 4 ++++ server/conn_stmt.go | 24 ++++++++++++++++++++---- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/server/conn.go b/server/conn.go index aa14ef7bb7fa9..562e96efcb123 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2193,6 +2193,10 @@ func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { return normalized, digest.String() case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) + prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) + if prepareObj != nil && prepareObj.SQLDigest != nil { + return prepareObj.NormalizedSQL, prepareObj.SQLDigest.String() + } str := cc.preparedStmt2StringNoArgs(stmtID) normalized, digest := parser.NormalizeDigest(str) return normalized, digest.String() diff --git a/server/conn_stmt.go b/server/conn_stmt.go index e9f56306d9800..de417d33d4607 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -680,14 +680,30 @@ func (cc *clientConn) preparedStmt2StringNoArgs(stmtID uint32) string { if sv == nil { return "" } + preparedObj, invalid := cc.preparedStmtID2CachePreparedStmt(stmtID) + if invalid { + return "invalidate CachedPrepareStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) + } + if preparedObj == nil { + return "prepared statement not found, ID: " + strconv.FormatUint(uint64(stmtID), 10) + } + return preparedObj.PreparedAst.Stmt.Text() +} + +func (cc *clientConn) preparedStmtID2CachePreparedStmt(stmtID uint32) (_ *plannercore.CachedPrepareStmt, invalid bool) { + sv := cc.ctx.GetSessionVars() + if sv == nil { + return nil, false + } preparedPointer, ok := sv.PreparedStmts[stmtID] if !ok { - return "prepared statement not found, ID: " + strconv.FormatUint(uint64(stmtID), 10) + // not found + return nil, false } preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) if !ok { - return "invalidate CachedPrepareStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) + // invalid cache. should never happen. + return nil, true } - preparedAst := preparedObj.PreparedAst - return preparedAst.Stmt.Text() + return preparedObj, false } From 3e43ffaac2e54bee1b7ed12dbd9488a45a8c77fc Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 15:00:44 +0800 Subject: [PATCH 28/54] address comment Signed-off-by: crazycs --- util/tracecpu/profile.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index ff5c374acb701..11a35609dbe0a 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -62,8 +62,6 @@ func NewSQLStatsProfiler() *sqlStatsProfiler { } func (sp *sqlStatsProfiler) Run() { - sp.mu.Lock() - defer sp.mu.Unlock() logutil.BgLogger().Info("cpu profiler started") go sp.startCPUProfileWorker() go sp.startAnalyzeProfileWorker() @@ -104,6 +102,7 @@ func (sp *sqlStatsProfiler) doCPUProfile() { if err := pprof.StartCPUProfile(task.buf); err != nil { // Sleep a while before retry. time.Sleep(time.Millisecond) + sp.putTaskToBuffer(task) return } ns := int(time.Second)*interval - time.Now().Nanosecond() @@ -125,6 +124,7 @@ func (sp *sqlStatsProfiler) startAnalyzeProfileWorker() { p, err := profile.Parse(reader) if err != nil { logutil.BgLogger().Error("parse profile error", zap.Error(err)) + sp.putTaskToBuffer(task) continue } stats := sp.parseCPUProfileBySQLLabels(p) @@ -295,9 +295,9 @@ func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest str return ctx } if variable.EnablePProfSQLCPU.Load() { - ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) + ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) } else { - ctx = pprof.WithLabels(context.Background(), pprof.Labels(labelSQLDigest, sqlDigest)) + ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) } pprof.SetGoroutineLabels(ctx) GlobalSQLStatsProfiler.RegisterSQL(sqlDigest, normalizedSQL) From 684aa3a9386668e2860f65df7b728042aeabeac5 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 15:21:54 +0800 Subject: [PATCH 29/54] address comment and add fix me comment Signed-off-by: crazycs --- server/conn.go | 2 +- session/session.go | 1 + statistics/handle/handle.go | 5 +++++ util/tracecpu/profile.go | 5 +++++ 4 files changed, 12 insertions(+), 1 deletion(-) diff --git a/server/conn.go b/server/conn.go index 562e96efcb123..ec0328071e231 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1023,7 +1023,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { if config.TopSQLEnabled() { normalizedSQL, digest := getLastStmtInConn{cc}.pprofLabelNormalizedAndDigest() if len(normalizedSQL) > 0 { - defer pprof.SetGoroutineLabels(ctx) + defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalizedSQL, digest) } } else if variable.EnablePProfSQLCPU.Load() { diff --git a/session/session.go b/session/session.go index 3cb210d188f3f..df0ae04df0841 100644 --- a/session/session.go +++ b/session/session.go @@ -1389,6 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter if config.TopSQLEnabled() { normalized, digest := parser.NormalizeDigest(sql) if digest != nil { + // Fixme: reset/clean the label when sql execute finish. tracecpu.SetGoroutineLabelsWithSQL(ctx, normalized, digest.String()) } } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 0026c97bb0c7c..13aac419e6575 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" @@ -43,6 +44,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/tracecpu" atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -121,6 +123,9 @@ func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context. func (h *Handle) execRestrictedSQL(ctx context.Context, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { + if config.TopSQLEnabled() { + defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) + } stmt, err := exec.ParseWithParams(ctx, sql, params...) if err != nil { return nil, nil, errors.Trace(err) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 11a35609dbe0a..d7654cd15563c 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -289,6 +289,11 @@ func StopCPUProfile() error { return nil } +// ResetGoroutineLabelsWithOriginalCtx resets the goroutine label with the original ctx. +func ResetGoroutineLabelsWithOriginalCtx(ctx context.Context) { + pprof.SetGoroutineLabels(ctx) +} + // SetGoroutineLabelsWithSQL sets the SQL digest label into the goroutine. func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { From 65fd410320b295d9f66edf64641c0835b816015d Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 27 May 2021 15:50:58 +0800 Subject: [PATCH 30/54] change top sql enable config default value to false Signed-off-by: crazycs --- config/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index cfadfe3fd94e7..4eb89fd5ef042 100644 --- a/config/config.go +++ b/config/config.go @@ -667,7 +667,7 @@ var defaultConf = Config{ HistorySize: 24, }, TopSQL: TopSQL{ - Enable: true, + Enable: false, RefreshInterval: 1, MaxStmtCount: 5000, }, From 272360906554647e5a97cb161658f61bb013f4b6 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 May 2021 16:39:30 +0800 Subject: [PATCH 31/54] fix tiny test bug and address comment Signed-off-by: crazycs --- util/tracecpu/mock/mock.go | 6 ++++-- util/tracecpu/profile_test.go | 9 ++++++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/util/tracecpu/mock/mock.go b/util/tracecpu/mock/mock.go index fa9de4474836b..4c081b03a4061 100644 --- a/util/tracecpu/mock/mock.go +++ b/util/tracecpu/mock/mock.go @@ -55,8 +55,10 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLStats) { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] if !ok { - tmp := stmt - stats = &tmp + stats = &tracecpu.SQLStats{ + SQLDigest: stmt.SQLDigest, + PlanDigest: stmt.PlanDigest, + } c.sqlStatsMap[hash] = stats } stats.CPUTimeMs += stmt.CPUTimeMs diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index c53127d285097..9ab9993debe2d 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -57,10 +57,17 @@ func (s *testSuite) TestTopSQLStatsProfile(c *C) { {"insert into t values (?)", ""}, } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() for _, req := range reqs { go func(sql, plan string) { for { - s.mockExecuteSQL(sql, plan) + select { + case <-ctx.Done(): + return + default: + s.mockExecuteSQL(sql, plan) + } } }(req.sql, req.plan) } From 7c81f9808c0fb1985fa9641923e17120bc4dec6e Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 May 2021 17:57:04 +0800 Subject: [PATCH 32/54] rename function name to make code clear Signed-off-by: crazycs --- executor/adapter.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 702857f8c242c..f949dec628f61 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -214,7 +214,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - ctx = a.setPProfLabelWithPlan(ctx) + ctx = a.setPlanLabelForTopSQL(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -290,7 +290,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return a.InfoSchema.SchemaMetaVersion(), nil } -func (a *ExecStmt) setPProfLabelWithPlan(ctx context.Context) context.Context { +func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { if a.Plan == nil || !config.TopSQLEnabled() { return ctx } @@ -371,8 +371,8 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if err != nil { return nil, err } - // ExecuteExec will rewrite `a.Plan`, so set goroutine label should be executed after `a.buildExecutor`. - ctx = a.setPProfLabelWithPlan(ctx) + // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. + ctx = a.setPlanLabelForTopSQL(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) From eead580000d360887464bb4aa0778db321a7f192 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 May 2021 21:19:38 +0800 Subject: [PATCH 33/54] address comment Signed-off-by: crazycs --- executor/adapter.go | 8 ++++---- executor/executor.go | 2 +- server/conn.go | 2 +- session/session.go | 2 +- util/tracecpu/profile.go | 12 ++++++------ util/tracecpu/profile_test.go | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index f949dec628f61..dfcbe50014b0b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -214,7 +214,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - ctx = a.setPlanLabelForTopSQL(ctx) + ctx = a.wrapAndSetPlanLabelForTopSQL(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -290,14 +290,14 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return a.InfoSchema.SchemaMetaVersion(), nil } -func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { +func (a *ExecStmt) wrapAndSetPlanLabelForTopSQL(ctx context.Context) context.Context { if a.Plan == nil || !config.TopSQLEnabled() { return ctx } _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(planDigest) > 0 { - ctx = tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest.String(), planDigest, normalizedPlan) + ctx = tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) } return ctx } @@ -372,7 +372,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. - ctx = a.setPlanLabelForTopSQL(ctx) + ctx = a.wrapAndSetPlanLabelForTopSQL(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) diff --git a/executor/executor.go b/executor/executor.go index 5271b04a55e41..c03ffe5ed7bf5 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1661,7 +1661,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. if config.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - tracecpu.SetGoroutineLabelsWithSQL(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) + tracecpu.SetSQLLabels(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) } } // execute missed stmtID uses empty sql diff --git a/server/conn.go b/server/conn.go index ae83240cd1582..167e11e6818fe 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1023,7 +1023,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { normalizedSQL, digest := getLastStmtInConn{cc}.pprofLabelNormalizedAndDigest() if len(normalizedSQL) > 0 { defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) - ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, normalizedSQL, digest) + ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest) } } else if variable.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() diff --git a/session/session.go b/session/session.go index feb04aa172a6a..f68d1bbb9afbe 100644 --- a/session/session.go +++ b/session/session.go @@ -1389,7 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when sql execute finish. - tracecpu.SetGoroutineLabelsWithSQL(ctx, normalized, digest.String()) + tracecpu.SetSQLLabels(ctx, normalized, digest.String()) } } return stmts[0], nil diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index d7654cd15563c..4466c1457f8e2 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -163,8 +163,8 @@ func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, // output the SQLStats slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ -// The sql_digest label is been set by `SetGoroutineLabelsWithSQL` function after parse the SQL. -// The plan_digest label is been set by `SetGoroutineLabelsWithSQLAndPlan` function after build the SQL plan. +// The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. +// The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. // Since `sqlStatsProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. func (sp *sqlStatsProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLStats { @@ -294,8 +294,8 @@ func ResetGoroutineLabelsWithOriginalCtx(ctx context.Context) { pprof.SetGoroutineLabels(ctx) } -// SetGoroutineLabelsWithSQL sets the SQL digest label into the goroutine. -func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { +// SetSQLLabels sets the SQL digest label into the goroutine. +func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { return ctx } @@ -309,8 +309,8 @@ func SetGoroutineLabelsWithSQL(ctx context.Context, normalizedSQL, sqlDigest str return ctx } -// SetGoroutineLabelsWithSQLAndPlan sets the SQL and plan digest label into the goroutine. -func SetGoroutineLabelsWithSQLAndPlan(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { +// SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. +func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) GlobalSQLStatsProfiler.RegisterPlan(planDigest, normalizedPlan) diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 9ab9993debe2d..439c78c8f486d 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -120,10 +120,10 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - ctx = tracecpu.SetGoroutineLabelsWithSQL(ctx, sql, sqlDigest) + ctx = tracecpu.SetSQLLabels(ctx, sql, sqlDigest) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) - tracecpu.SetGoroutineLabelsWithSQLAndPlan(ctx, sqlDigest, planDigest, plan) + tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest, plan) s.mockExecute(time.Millisecond * 300) } From 8d33e882dc208761e39c62eae49f12fd276baeba Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 May 2021 21:35:51 +0800 Subject: [PATCH 34/54] address comment Signed-off-by: crazycs --- server/conn.go | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/server/conn.go b/server/conn.go index 167e11e6818fe..971db8f26c56a 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1021,7 +1021,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { data = data[1:] if config.TopSQLEnabled() { normalizedSQL, digest := getLastStmtInConn{cc}.pprofLabelNormalizedAndDigest() - if len(normalizedSQL) > 0 { + if len(digest) > 0 { defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest) } @@ -2165,13 +2165,6 @@ func (cc getLastStmtInConn) PProfLabel() string { } } -var ( - pprofLabelOfUseDB = parser.DigestNormalized("UseDB") - pprofLabelOfListFields = parser.DigestNormalized("ListFields") - pprofLabelOfCloseStmt = parser.DigestNormalized("CloseStmt") - pprofLabelOfResetStmt = parser.DigestNormalized("ResetStmt") -) - // PProfLabelNormalizedAndDigest return sql and sql_digest label used to tag pprof. func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { if len(cc.lastPacket) == 0 { @@ -2180,13 +2173,13 @@ func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { cmd, data := cc.lastPacket[0], cc.lastPacket[1:] switch cmd { case mysql.ComInitDB: - return "UseDB", pprofLabelOfUseDB.String() + return "UseDB", "" case mysql.ComFieldList: - return "ListFields", pprofLabelOfListFields.String() + return "ListFields", "" case mysql.ComStmtClose: - return "CloseStmt", pprofLabelOfCloseStmt.String() + return "CloseStmt", "" case mysql.ComStmtReset: - return "ResetStmt", pprofLabelOfResetStmt.String() + return "ResetStmt", "" case mysql.ComQuery, mysql.ComStmtPrepare: normalized, digest := parser.NormalizeDigest(string(hack.String(data))) return normalized, digest.String() From d8074790f9366503b9e10e3da05a4afa5f37788a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 09:53:10 +0800 Subject: [PATCH 35/54] rename and output optimize time cost Signed-off-by: crazycs --- server/tidb_test.go | 6 +-- tidb-server/main.go | 6 +-- util/tracecpu/profile.go | 82 ++++++++++++++++++----------------- util/tracecpu/profile_test.go | 16 +++---- 4 files changed, 56 insertions(+), 54 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index 14f4027ef9260..888b38e873e1d 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -96,7 +96,7 @@ func (ts *tidbTestTopSQLSuite) SetUpSuite(c *C) { conf.TopSQL.Enable = true conf.TopSQL.RefreshInterval = 1 }) - tracecpu.GlobalSQLStatsProfiler.Run() + tracecpu.GlobalTopSQLCPUProfiler.Run() } func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { @@ -1177,7 +1177,7 @@ func (ts *tidbTestSerialSuite) TestPrepareCount(c *C) { c.Assert(atomic.LoadInt64(&variable.PreparedStmtCount), Equals, prepareCnt) } -func (ts *tidbTestTopSQLSuite) TestTopSQLStatsProfile(c *C) { +func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { db, err := sql.Open("mysql", ts.getDSN()) c.Assert(err, IsNil, Commentf("Error connecting")) defer func() { @@ -1185,7 +1185,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLStatsProfile(c *C) { c.Assert(err, IsNil) }() collector := mock.NewTopSQLCollector() - tracecpu.GlobalSQLStatsProfiler.SetCollector(collector) + tracecpu.GlobalTopSQLCPUProfiler.SetCollector(collector) dbt := &DBTest{c, db} dbt.mustExec("drop database if exists topsql") diff --git a/tidb-server/main.go b/tidb-server/main.go index 4875f94fd85e8..69ce4bc86d92a 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -179,7 +179,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() - setupSQLStatsProfiler() + setupTopSQLProfiler() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) @@ -688,6 +688,6 @@ func stringToList(repairString string) []string { }) } -func setupSQLStatsProfiler() { - tracecpu.GlobalSQLStatsProfiler.Run() +func setupTopSQLProfiler() { + tracecpu.GlobalTopSQLCPUProfiler.Run() } diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 4466c1457f8e2..a307aeeebfed7 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -37,12 +37,13 @@ const ( labelSQL = "sql" labelSQLDigest = "sql_digest" labelPlanDigest = "plan_digest" + labelOptimize = "optimize" ) -// GlobalSQLStatsProfiler is the global SQL stats profiler. -var GlobalSQLStatsProfiler = NewSQLStatsProfiler() +// GlobalTopSQLCPUProfiler is the global SQL stats profiler. +var GlobalTopSQLCPUProfiler = NewTopSQLCPUProfiler() -type sqlStatsProfiler struct { +type topSQLCPUProfiler struct { taskCh chan *profileTask cacheBufCh chan *profileTask @@ -53,39 +54,39 @@ type sqlStatsProfiler struct { collector TopSQLCollector } -// NewSQLStatsProfiler create a sqlStatsProfiler. -func NewSQLStatsProfiler() *sqlStatsProfiler { - return &sqlStatsProfiler{ +// NewTopSQLCPUProfiler create a topSQLCPUProfiler. +func NewTopSQLCPUProfiler() *topSQLCPUProfiler { + return &topSQLCPUProfiler{ taskCh: make(chan *profileTask, 128), cacheBufCh: make(chan *profileTask, 128), } } -func (sp *sqlStatsProfiler) Run() { +func (sp *topSQLCPUProfiler) Run() { logutil.BgLogger().Info("cpu profiler started") go sp.startCPUProfileWorker() go sp.startAnalyzeProfileWorker() } -func (sp *sqlStatsProfiler) SetCollector(c TopSQLCollector) { +func (sp *topSQLCPUProfiler) SetCollector(c TopSQLCollector) { sp.collector = c } -func (sp *sqlStatsProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { +func (sp *topSQLCPUProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { if sp.collector == nil { return } sp.collector.RegisterSQL(sqlDigest, normalizedSQL) } -func (sp *sqlStatsProfiler) RegisterPlan(planDigest string, normalizedPlan string) { +func (sp *topSQLCPUProfiler) RegisterPlan(planDigest string, normalizedPlan string) { if sp.collector == nil { return } sp.collector.RegisterPlan(planDigest, normalizedPlan) } -func (sp *sqlStatsProfiler) startCPUProfileWorker() { +func (sp *topSQLCPUProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { if sp.IsEnabled() { @@ -96,7 +97,7 @@ func (sp *sqlStatsProfiler) startCPUProfileWorker() { } } -func (sp *sqlStatsProfiler) doCPUProfile() { +func (sp *topSQLCPUProfiler) doCPUProfile() { interval := config.GetGlobalConfig().TopSQL.RefreshInterval task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { @@ -111,12 +112,12 @@ func (sp *sqlStatsProfiler) doCPUProfile() { sp.sendProfileTask(task) } -func (sp *sqlStatsProfiler) sendProfileTask(task *profileTask) { +func (sp *topSQLCPUProfiler) sendProfileTask(task *profileTask) { task.end = time.Now().Unix() sp.taskCh <- task } -func (sp *sqlStatsProfiler) startAnalyzeProfileWorker() { +func (sp *topSQLCPUProfiler) startAnalyzeProfileWorker() { defer util.Recover("top-sql", "analyzeProfileWorker", nil, false) for { task := <-sp.taskCh @@ -141,7 +142,7 @@ type profileTask struct { end int64 } -func (sp *sqlStatsProfiler) newProfileTask() *profileTask { +func (sp *topSQLCPUProfiler) newProfileTask() *profileTask { var task *profileTask select { case task = <-sp.cacheBufCh: @@ -154,7 +155,7 @@ func (sp *sqlStatsProfiler) newProfileTask() *profileTask { return task } -func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { +func (sp *topSQLCPUProfiler) putTaskToBuffer(task *profileTask) { select { case sp.cacheBufCh <- task: default: @@ -165,9 +166,9 @@ func (sp *sqlStatsProfiler) putTaskToBuffer(task *profileTask) { // output the SQLStats slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. // The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. -// Since `sqlStatsProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data +// Since `topSQLCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. -func (sp *sqlStatsProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLStats { +func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLStats { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -195,7 +196,7 @@ func (sp *sqlStatsProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQL return sp.createSQLStats(sqlMap) } -func (sp *sqlStatsProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLStats { +func (sp *topSQLCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLStats { stats := make([]SQLStats, 0, len(sqlMap)) for sqlDigest, stmt := range sqlMap { stmt.tune() @@ -223,11 +224,14 @@ type sqlStats struct { // }, // total: 600ms, // The total cpu time of the sql is 600ms. // total_time - table_scan_time - index_scan_time = 100ms, and this 100ms means those sample data only contain the -// sql_digest label, doesn't contain the plan_digest label. This is cause by the `pprof profile` is base on sample. +// sql_digest label, doesn't contain the plan_digest label. This is cause by the `pprof profile` is base on sample, +// and the plan digest can only be set after optimizer generated execution plan. So the remain 100ms means the plan +// optimizer takes time to generated plan. // After this tune function, the `sqlStats` become to: // plans: { -// "table_scan": 240ms, // 200 + (200/(200+300))*100 -// "index_scan": 360ms, // 300 + (300/(200+300))*100 +// "optimize" : 100ms, // 600 - 200 - 300 +// "table_scan": 200ms, +// "index_scan": 300ms, // }, // total: 600ms, func (s *sqlStats) tune() { @@ -239,16 +243,14 @@ func (s *sqlStats) tune() { for _, v := range s.plans { planTotal += v } - remain := s.total - planTotal - if remain <= 0 { + optimize := s.total - planTotal + if optimize <= 0 { return } - for k, v := range s.plans { - s.plans[k] = v + (v/planTotal)*remain - } + s.plans[labelOptimize] = optimize } -func (sp *sqlStatsProfiler) handleExportProfileTask(p *profile.Profile) { +func (sp *topSQLCPUProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept == nil { @@ -257,7 +259,7 @@ func (sp *sqlStatsProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.ept.mergeProfile(p) } -func (sp *sqlStatsProfiler) hasExportProfileTask() bool { +func (sp *topSQLCPUProfiler) hasExportProfileTask() bool { sp.mu.Lock() has := sp.mu.ept != nil sp.mu.Unlock() @@ -265,16 +267,16 @@ func (sp *sqlStatsProfiler) hasExportProfileTask() bool { } // IsEnabled return true if it is(should be) enabled. It exports for tests. -func (sp *sqlStatsProfiler) IsEnabled() bool { +func (sp *topSQLCPUProfiler) IsEnabled() bool { return config.GetGlobalConfig().TopSQL.Enable || sp.hasExportProfileTask() } // StartCPUProfile same like pprof.StartCPUProfile. -// Because the GlobalSQLStatsProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, +// Because the GlobalTopSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalSQLStatsProfiler.IsEnabled() { - return GlobalSQLStatsProfiler.startExportCPUProfile(w) + if GlobalTopSQLCPUProfiler.IsEnabled() { + return GlobalTopSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) } @@ -282,8 +284,8 @@ func StartCPUProfile(w io.Writer) error { // StopCPUProfile same like pprof.StopCPUProfile. // other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. func StopCPUProfile() error { - if GlobalSQLStatsProfiler.IsEnabled() { - return GlobalSQLStatsProfiler.stopExportCPUProfile() + if GlobalTopSQLCPUProfiler.IsEnabled() { + return GlobalTopSQLCPUProfiler.stopExportCPUProfile() } pprof.StopCPUProfile() return nil @@ -305,7 +307,7 @@ func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context. ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) } pprof.SetGoroutineLabels(ctx) - GlobalSQLStatsProfiler.RegisterSQL(sqlDigest, normalizedSQL) + GlobalTopSQLCPUProfiler.RegisterSQL(sqlDigest, normalizedSQL) return ctx } @@ -313,11 +315,11 @@ func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context. func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) - GlobalSQLStatsProfiler.RegisterPlan(planDigest, normalizedPlan) + GlobalTopSQLCPUProfiler.RegisterPlan(planDigest, normalizedPlan) return ctx } -func (sp *sqlStatsProfiler) startExportCPUProfile(w io.Writer) error { +func (sp *topSQLCPUProfiler) startExportCPUProfile(w io.Writer) error { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept != nil { @@ -327,7 +329,7 @@ func (sp *sqlStatsProfiler) startExportCPUProfile(w io.Writer) error { return nil } -func (sp *sqlStatsProfiler) stopExportCPUProfile() error { +func (sp *topSQLCPUProfiler) stopExportCPUProfile() error { sp.mu.Lock() ept := sp.mu.ept sp.mu.ept = nil @@ -345,7 +347,7 @@ func (sp *sqlStatsProfiler) stopExportCPUProfile() error { // removeLabel uses to remove labels for export cpu profile data. // Since the sql_digest and plan_digest label is strange for other users. // If `variable.EnablePProfSQLCPU` is true means wanto keep the `sql` label, otherwise, remove the `sql` label too. -func (sp *sqlStatsProfiler) removeLabel(p *profile.Profile) { +func (sp *topSQLCPUProfiler) removeLabel(p *profile.Profile) { if p == nil { return } diff --git a/util/tracecpu/profile_test.go b/util/tracecpu/profile_test.go index 439c78c8f486d..22b55106fc403 100644 --- a/util/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -42,12 +42,12 @@ func (s *testSuite) SetUpSuite(c *C) { newCfg.TopSQL.Enable = true newCfg.TopSQL.RefreshInterval = 1 config.StoreGlobalConfig(&newCfg) - tracecpu.GlobalSQLStatsProfiler.Run() + tracecpu.GlobalTopSQLCPUProfiler.Run() } -func (s *testSuite) TestTopSQLStatsProfile(c *C) { +func (s *testSuite) TestTopSQLCPUProfile(c *C) { collector := mock.NewTopSQLCollector() - tracecpu.GlobalSQLStatsProfiler.SetCollector(collector) + tracecpu.GlobalTopSQLCPUProfiler.SetCollector(collector) reqs := []struct { sql string plan string @@ -94,21 +94,21 @@ func (s *testSuite) TestTopSQLStatsProfile(c *C) { func (s *testSuite) TestIsEnabled(c *C) { s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) + c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsFalse) s.setTopSQLEnable(true) err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) c.Assert(err, IsNil) - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) err = tracecpu.StopCPUProfile() c.Assert(err, IsNil) s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsFalse) + c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsFalse) s.setTopSQLEnable(true) - c.Assert(tracecpu.GlobalSQLStatsProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) } func (s *testSuite) setTopSQLEnable(enabled bool) { From a4e15ff287a56b7b1957b9314b26a1c019b5290a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 09:58:45 +0800 Subject: [PATCH 36/54] use atomic to avoid race Signed-off-by: crazycs --- util/tracecpu/profile.go | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index a307aeeebfed7..a5d1f4fc6fbeb 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -23,6 +23,7 @@ import ( "runtime/pprof" "strconv" "sync" + "sync/atomic" "time" "github.com/google/pprof/profile" @@ -51,7 +52,7 @@ type topSQLCPUProfiler struct { sync.Mutex ept *exportProfileTask } - collector TopSQLCollector + collector atomic.Value } // NewTopSQLCPUProfiler create a topSQLCPUProfiler. @@ -69,21 +70,31 @@ func (sp *topSQLCPUProfiler) Run() { } func (sp *topSQLCPUProfiler) SetCollector(c TopSQLCollector) { - sp.collector = c + sp.collector.Store(c) +} + +func (sp *topSQLCPUProfiler) getCollector() TopSQLCollector { + c, ok := sp.collector.Load().(TopSQLCollector) + if !ok || c == nil { + return nil + } + return c } func (sp *topSQLCPUProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { - if sp.collector == nil { + c := sp.getCollector() + if c == nil { return } - sp.collector.RegisterSQL(sqlDigest, normalizedSQL) + c.RegisterSQL(sqlDigest, normalizedSQL) } func (sp *topSQLCPUProfiler) RegisterPlan(planDigest string, normalizedPlan string) { - if sp.collector == nil { + c := sp.getCollector() + if c == nil { return } - sp.collector.RegisterPlan(planDigest, normalizedPlan) + c.RegisterPlan(planDigest, normalizedPlan) } func (sp *topSQLCPUProfiler) startCPUProfileWorker() { @@ -130,8 +141,8 @@ func (sp *topSQLCPUProfiler) startAnalyzeProfileWorker() { } stats := sp.parseCPUProfileBySQLLabels(p) sp.handleExportProfileTask(p) - if sp.collector != nil { - sp.collector.Collect(task.end, stats) + if c := sp.getCollector(); c != nil { + c.Collect(task.end, stats) } sp.putTaskToBuffer(task) } From 3193c6461f3b807f2b0af5d959da872cdec06dc2 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 10:00:21 +0800 Subject: [PATCH 37/54] address comment Signed-off-by: crazycs --- util/tracecpu/profile.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index a5d1f4fc6fbeb..5046ebcbce413 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -42,7 +42,7 @@ const ( ) // GlobalTopSQLCPUProfiler is the global SQL stats profiler. -var GlobalTopSQLCPUProfiler = NewTopSQLCPUProfiler() +var GlobalTopSQLCPUProfiler = newTopSQLCPUProfiler() type topSQLCPUProfiler struct { taskCh chan *profileTask @@ -55,8 +55,8 @@ type topSQLCPUProfiler struct { collector atomic.Value } -// NewTopSQLCPUProfiler create a topSQLCPUProfiler. -func NewTopSQLCPUProfiler() *topSQLCPUProfiler { +// newTopSQLCPUProfiler create a topSQLCPUProfiler. +func newTopSQLCPUProfiler() *topSQLCPUProfiler { return &topSQLCPUProfiler{ taskCh: make(chan *profileTask, 128), cacheBufCh: make(chan *profileTask, 128), From 51f4127661e1d9ba59aebbf44daebf06462d8972 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 11:16:08 +0800 Subject: [PATCH 38/54] address comment Signed-off-by: crazycs --- server/conn.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/server/conn.go b/server/conn.go index 971db8f26c56a..00ab5f1db95f5 100644 --- a/server/conn.go +++ b/server/conn.go @@ -78,7 +78,7 @@ import ( storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" - util2 "github.com/pingcap/tidb/util" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/arena" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" @@ -2125,10 +2125,10 @@ func (cc getLastStmtInConn) String() string { if cc.ctx.GetSessionVars().EnableRedactLog { sql = parser.Normalize(sql) } - return util2.QueryStrForLog(sql) + return tidbutil.QueryStrForLog(sql) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return util2.QueryStrForLog(cc.preparedStmt2String(stmtID)) + return tidbutil.QueryStrForLog(cc.preparedStmt2String(stmtID)) case mysql.ComStmtClose, mysql.ComStmtReset: stmtID := binary.LittleEndian.Uint32(data[0:4]) return mysql.Command2Str[cmd] + " " + strconv.Itoa(int(stmtID)) @@ -2156,10 +2156,10 @@ func (cc getLastStmtInConn) PProfLabel() string { case mysql.ComStmtReset: return "ResetStmt" case mysql.ComQuery, mysql.ComStmtPrepare: - return parser.Normalize(util2.QueryStrForLog(string(hack.String(data)))) + return parser.Normalize(tidbutil.QueryStrForLog(string(hack.String(data)))) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return util2.QueryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) + return tidbutil.QueryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) default: return "" } From eb5990914880ac8c32231c74225942317a9119c9 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 11:19:57 +0800 Subject: [PATCH 39/54] address comment Signed-off-by: crazycs --- util/tracecpu/collector.go | 6 +++--- util/tracecpu/mock/mock.go | 16 ++++++++-------- util/tracecpu/profile.go | 10 +++++----- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index 6c62d10c5022b..f0282188844eb 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -16,13 +16,13 @@ package tracecpu // TopSQLCollector uses to collect SQL stats. // TODO: add a collector to collect and store the SQL stats. type TopSQLCollector interface { - Collect(ts int64, stats []SQLStats) + Collect(ts int64, stats []TopSQLRecord) RegisterSQL(sqlDigest, normalizedSQL string) RegisterPlan(planDigest string, normalizedPlan string) } -// SQLStats indicate the SQL stats. -type SQLStats struct { +// TopSQLRecord contains the SQL meta and execution information. +type TopSQLRecord struct { SQLDigest string PlanDigest string CPUTimeMs uint32 diff --git a/util/tracecpu/mock/mock.go b/util/tracecpu/mock/mock.go index 4c081b03a4061..69fcef89e9b87 100644 --- a/util/tracecpu/mock/mock.go +++ b/util/tracecpu/mock/mock.go @@ -30,7 +30,7 @@ type TopSQLCollector struct { // plan_digest -> normalized plan planMap map[string]string // (sql + plan_digest) -> sql stats - sqlStatsMap map[string]*tracecpu.SQLStats + sqlStatsMap map[string]*tracecpu.TopSQLRecord collectCnt atomic.Int64 } @@ -39,12 +39,12 @@ func NewTopSQLCollector() *TopSQLCollector { return &TopSQLCollector{ sqlMap: make(map[string]string), planMap: make(map[string]string), - sqlStatsMap: make(map[string]*tracecpu.SQLStats), + sqlStatsMap: make(map[string]*tracecpu.TopSQLRecord), } } // Collect uses for testing. -func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLStats) { +func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.TopSQLRecord) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -55,7 +55,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLStats) { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] if !ok { - stats = &tracecpu.SQLStats{ + stats = &tracecpu.TopSQLRecord{ SQLDigest: stmt.SQLDigest, PlanDigest: stmt.PlanDigest, } @@ -66,7 +66,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLStats) { } // GetSQLStatsBySQLWithRetry uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLStats { +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.TopSQLRecord { after := time.After(time.Second * 10) for { select { @@ -83,8 +83,8 @@ func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bo } // GetSQLStatsBySQL uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLStats { - stats := make([]*tracecpu.SQLStats, 0, 2) +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.TopSQLRecord { + stats := make([]*tracecpu.TopSQLRecord, 0, 2) sqlDigest := GenSQLDigest(sql) c.Lock() for _, stmt := range c.sqlStatsMap { @@ -158,7 +158,7 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } -func (c *TopSQLCollector) hash(stat tracecpu.SQLStats) string { +func (c *TopSQLCollector) hash(stat tracecpu.TopSQLRecord) string { return stat.SQLDigest + stat.PlanDigest } diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 5046ebcbce413..3c09a7629ded5 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -174,12 +174,12 @@ func (sp *topSQLCPUProfiler) putTaskToBuffer(task *profileTask) { } // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, -// output the SQLStats slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ +// output the TopSQLRecord slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. // The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. // Since `topSQLCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. -func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLStats { +func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []TopSQLRecord { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -207,12 +207,12 @@ func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQ return sp.createSQLStats(sqlMap) } -func (sp *topSQLCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLStats { - stats := make([]SQLStats, 0, len(sqlMap)) +func (sp *topSQLCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []TopSQLRecord { + stats := make([]TopSQLRecord, 0, len(sqlMap)) for sqlDigest, stmt := range sqlMap { stmt.tune() for planDigest, val := range stmt.plans { - stats = append(stats, SQLStats{ + stats = append(stats, TopSQLRecord{ SQLDigest: sqlDigest, PlanDigest: planDigest, CPUTimeMs: uint32(time.Duration(val).Milliseconds()), From e606073ed23fb09fd85410dd58ef42167c67a8d3 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 12:50:26 +0800 Subject: [PATCH 40/54] address comment Signed-off-by: crazycs --- server/conn.go | 8 -------- util/tracecpu/collector.go | 2 ++ util/tracecpu/profile.go | 3 +-- 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/server/conn.go b/server/conn.go index 00ab5f1db95f5..fbc9dcf65a563 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2172,14 +2172,6 @@ func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { } cmd, data := cc.lastPacket[0], cc.lastPacket[1:] switch cmd { - case mysql.ComInitDB: - return "UseDB", "" - case mysql.ComFieldList: - return "ListFields", "" - case mysql.ComStmtClose: - return "CloseStmt", "" - case mysql.ComStmtReset: - return "ResetStmt", "" case mysql.ComQuery, mysql.ComStmtPrepare: normalized, digest := parser.NormalizeDigest(string(hack.String(data))) return normalized, digest.String() diff --git a/util/tracecpu/collector.go b/util/tracecpu/collector.go index f0282188844eb..444e499634acf 100644 --- a/util/tracecpu/collector.go +++ b/util/tracecpu/collector.go @@ -16,6 +16,8 @@ package tracecpu // TopSQLCollector uses to collect SQL stats. // TODO: add a collector to collect and store the SQL stats. type TopSQLCollector interface { + // Collect uses to collect the SQL execution information. + // ts is a Unix time, unit is second. Collect(ts int64, stats []TopSQLRecord) RegisterSQL(sqlDigest, normalizedSQL string) RegisterPlan(planDigest string, normalizedPlan string) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 3c09a7629ded5..12d00dc1128e4 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -38,7 +38,6 @@ const ( labelSQL = "sql" labelSQLDigest = "sql_digest" labelPlanDigest = "plan_digest" - labelOptimize = "optimize" ) // GlobalTopSQLCPUProfiler is the global SQL stats profiler. @@ -258,7 +257,7 @@ func (s *sqlStats) tune() { if optimize <= 0 { return } - s.plans[labelOptimize] = optimize + s.plans[""] += optimize } func (sp *topSQLCPUProfiler) handleExportProfileTask(p *profile.Profile) { From 26786daf8fd2e9e1ab3fb6537d554a1d3f4cd8e7 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 13:38:53 +0800 Subject: [PATCH 41/54] avoid double digest cost for top sql Signed-off-by: crazycs --- executor/prepared.go | 9 +++++++-- server/conn.go | 33 +-------------------------------- server/conn_stmt.go | 8 ++++++++ session/session.go | 9 ++++++++- 4 files changed, 24 insertions(+), 35 deletions(-) diff --git a/executor/prepared.go b/executor/prepared.go index f494ebac3dc9d..4445149ca6ba5 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner" @@ -35,6 +36,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/tracecpu" "go.uber.org/zap" ) @@ -178,6 +180,10 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { Params: sorter.markers, SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), } + normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) + if config.TopSQLEnabled() { + ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest.String()) + } if !plannercore.PreparedPlanCacheEnabled() { prepared.UseCache = false @@ -213,11 +219,10 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { vars.PreparedStmtNameToID[e.name] = e.ID } - normalized, digest := parser.NormalizeDigest(prepared.Stmt.Text()) preparedObj := &plannercore.CachedPrepareStmt{ PreparedAst: prepared, VisitInfos: destBuilder.GetVisitInfo(), - NormalizedSQL: normalized, + NormalizedSQL: normalizedSQL, SQLDigest: digest, ForUpdateRead: destBuilder.GetIsForUpdateRead(), } diff --git a/server/conn.go b/server/conn.go index fbc9dcf65a563..315428c440790 100644 --- a/server/conn.go +++ b/server/conn.go @@ -85,7 +85,6 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tidb/util/tracecpu" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -1019,13 +1018,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] - if config.TopSQLEnabled() { - normalizedSQL, digest := getLastStmtInConn{cc}.pprofLabelNormalizedAndDigest() - if len(digest) > 0 { - defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) - ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest) - } - } else if variable.EnablePProfSQLCPU.Load() { + if variable.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { defer pprof.SetGoroutineLabels(ctx) @@ -2164,27 +2157,3 @@ func (cc getLastStmtInConn) PProfLabel() string { return "" } } - -// PProfLabelNormalizedAndDigest return sql and sql_digest label used to tag pprof. -func (cc getLastStmtInConn) pprofLabelNormalizedAndDigest() (string, string) { - if len(cc.lastPacket) == 0 { - return "", "" - } - cmd, data := cc.lastPacket[0], cc.lastPacket[1:] - switch cmd { - case mysql.ComQuery, mysql.ComStmtPrepare: - normalized, digest := parser.NormalizeDigest(string(hack.String(data))) - return normalized, digest.String() - case mysql.ComStmtExecute, mysql.ComStmtFetch: - stmtID := binary.LittleEndian.Uint32(data[0:4]) - prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) - if prepareObj != nil && prepareObj.SQLDigest != nil { - return prepareObj.NormalizedSQL, prepareObj.SQLDigest.String() - } - str := cc.preparedStmt2StringNoArgs(stmtID) - normalized, digest := parser.NormalizeDigest(str) - return normalized, digest.String() - default: - return "", "" - } -} diff --git a/server/conn_stmt.go b/server/conn_stmt.go index de417d33d4607..e225ffbdeb276 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" @@ -55,6 +56,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/tracecpu" ) func (cc *clientConn) handleStmtPrepare(ctx context.Context, sql string) error { @@ -265,6 +267,12 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err return errors.Annotate(mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch"), cc.preparedStmt2String(stmtID)) } + if config.TopSQLEnabled() { + prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) + if prepareObj != nil && prepareObj.SQLDigest != nil { + tracecpu.SetSQLLabels(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String()) + } + } sql := "" if prepared, ok := cc.ctx.GetStatement(int(stmtID)).(*TiDBStatement); ok { sql = prepared.sql diff --git a/session/session.go b/session/session.go index f68d1bbb9afbe..7011682774cbd 100644 --- a/session/session.go +++ b/session/session.go @@ -1502,6 +1502,11 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { return nil, err } + normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() + if config.TopSQLEnabled() { + ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest.String()) + } + if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { return nil, err } @@ -1509,7 +1514,6 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex // Uncorrelated subqueries will execute once when building plan, so we reset process info before building plan. cmd32 := atomic.LoadUint32(&s.GetSessionVars().CommandValue) s.SetProcessInfo(stmtNode.Text(), time.Now(), byte(cmd32), 0) - _, digest := s.sessionVars.StmtCtx.SQLDigest() s.txn.onStmtStart(digest.String()) defer s.txn.onStmtEnd() @@ -1878,6 +1882,9 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if !ok { return nil, errors.Errorf("invalid CachedPrepareStmt type") } + if config.TopSQLEnabled() && preparedStmt.SQLDigest != nil { + ctx = tracecpu.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) + } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) if err != nil { From d473ef38c6837c315b1e7a9141ddcf19e3370d2b Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 14:09:18 +0800 Subject: [PATCH 42/54] address comment Signed-off-by: crazycs --- util/tracecpu/profile.go | 46 +++++++++++++++++----------------------- 1 file changed, 20 insertions(+), 26 deletions(-) diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 12d00dc1128e4..cd59bd3362587 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -44,8 +44,7 @@ const ( var GlobalTopSQLCPUProfiler = newTopSQLCPUProfiler() type topSQLCPUProfiler struct { - taskCh chan *profileTask - cacheBufCh chan *profileTask + taskCh chan *profileTask mu struct { sync.Mutex @@ -54,11 +53,19 @@ type topSQLCPUProfiler struct { collector atomic.Value } +var ( + defaultProfileBufSize = 100 * 1024 + profileBufPool = sync.Pool{ + New: func() interface{} { + return bytes.NewBuffer(make([]byte, 0, defaultProfileBufSize)) + }, + } +) + // newTopSQLCPUProfiler create a topSQLCPUProfiler. func newTopSQLCPUProfiler() *topSQLCPUProfiler { return &topSQLCPUProfiler{ - taskCh: make(chan *profileTask, 128), - cacheBufCh: make(chan *profileTask, 128), + taskCh: make(chan *profileTask, 128), } } @@ -108,21 +115,17 @@ func (sp *topSQLCPUProfiler) startCPUProfileWorker() { } func (sp *topSQLCPUProfiler) doCPUProfile() { - interval := config.GetGlobalConfig().TopSQL.RefreshInterval + intervalSecond := config.GetGlobalConfig().TopSQL.RefreshInterval task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { // Sleep a while before retry. - time.Sleep(time.Millisecond) + time.Sleep(time.Second) sp.putTaskToBuffer(task) return } - ns := int(time.Second)*interval - time.Now().Nanosecond() + ns := int(time.Second)*intervalSecond - time.Now().Nanosecond() time.Sleep(time.Nanosecond * time.Duration(ns)) pprof.StopCPUProfile() - sp.sendProfileTask(task) -} - -func (sp *topSQLCPUProfiler) sendProfileTask(task *profileTask) { task.end = time.Now().Unix() sp.taskCh <- task } @@ -131,8 +134,7 @@ func (sp *topSQLCPUProfiler) startAnalyzeProfileWorker() { defer util.Recover("top-sql", "analyzeProfileWorker", nil, false) for { task := <-sp.taskCh - reader := bytes.NewReader(task.buf.Bytes()) - p, err := profile.Parse(reader) + p, err := profile.ParseData(task.buf.Bytes()) if err != nil { logutil.BgLogger().Error("parse profile error", zap.Error(err)) sp.putTaskToBuffer(task) @@ -153,23 +155,15 @@ type profileTask struct { } func (sp *topSQLCPUProfiler) newProfileTask() *profileTask { - var task *profileTask - select { - case task = <-sp.cacheBufCh: - task.buf.Reset() - default: - task = &profileTask{ - buf: bytes.NewBuffer(make([]byte, 0, 100*1024)), - } + buf := profileBufPool.Get().(*bytes.Buffer) + return &profileTask{ + buf: buf, } - return task } func (sp *topSQLCPUProfiler) putTaskToBuffer(task *profileTask) { - select { - case sp.cacheBufCh <- task: - default: - } + task.buf.Reset() + profileBufPool.Put(task.buf) } // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, From cfcf574b441781f03e768e1d69bfc0d484670508 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 15:56:00 +0800 Subject: [PATCH 43/54] add topsql pkg Signed-off-by: crazycs --- executor/adapter.go | 5 +- executor/executor.go | 4 +- executor/prepared.go | 4 +- server/conn_stmt.go | 4 +- server/http_status.go | 2 +- server/sql_info_fetcher.go | 2 +- server/tidb_test.go | 4 +- session/session.go | 8 +-- statistics/handle/handle.go | 2 +- tidb-server/main.go | 8 +-- util/topsql/topsql.go | 57 +++++++++++++++++++ util/{ => topsql}/tracecpu/collector.go | 0 util/{ => topsql}/tracecpu/mock/mock.go | 2 +- util/{ => topsql}/tracecpu/profile.go | 27 +-------- .../tracecpu_test.go} | 11 ++-- 15 files changed, 86 insertions(+), 54 deletions(-) create mode 100644 util/topsql/topsql.go rename util/{ => topsql}/tracecpu/collector.go (100%) rename util/{ => topsql}/tracecpu/mock/mock.go (98%) rename util/{ => topsql}/tracecpu/profile.go (94%) rename util/{tracecpu/profile_test.go => topsql/tracecpu_test.go} (92%) diff --git a/executor/adapter.go b/executor/adapter.go index b84c9c087dadc..1a6e3a2206c0f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -56,8 +56,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" - "github.com/pingcap/tidb/util/tracecpu" - + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -297,7 +296,7 @@ func (a *ExecStmt) wrapAndSetPlanLabelForTopSQL(ctx context.Context) context.Con _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(planDigest) > 0 { - ctx = tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) + ctx = topsql.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) } return ctx } diff --git a/executor/executor.go b/executor/executor.go index 6cf4c5289a777..3dcf2935027b4 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -66,7 +66,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" ) @@ -1661,7 +1661,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - tracecpu.SetSQLLabels(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) + topsql.SetSQLLabels(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index 194175ebb9b03..9ebd371b22b96 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" ) @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest.String()) + ctx = topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 3bf824c749d03..a1d28fd27fd39 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -56,7 +56,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hack" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql" ) func (cc *clientConn) handleStmtPrepare(ctx context.Context, sql string) error { @@ -270,7 +270,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if variable.TopSQLEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { - tracecpu.SetSQLLabels(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String()) + topsql.SetSQLLabels(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String()) } } sql := "" diff --git a/server/http_status.go b/server/http_status.go index 73a3f313198f9..67eace56562fe 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -43,7 +43,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/versioninfo" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go index 736bee649ec57..57f51f544b90b 100644 --- a/server/sql_info_fetcher.go +++ b/server/sql_info_fetcher.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu" ) type sqlInfoFetcher struct { diff --git a/server/tidb_test.go b/server/tidb_test.go index 63c5fc51e2f1a..0aef3ef05f855 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -51,8 +51,8 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/tracecpu" - "github.com/pingcap/tidb/util/tracecpu/mock" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" ) type tidbTestSuite struct { diff --git a/session/session.go b/session/session.go index 3a8ec31ab2159..7b799130b0260 100644 --- a/session/session.go +++ b/session/session.go @@ -41,7 +41,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" @@ -1389,7 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when sql execute finish. - tracecpu.SetSQLLabels(ctx, normalized, digest.String()) + topsql.SetSQLLabels(ctx, normalized, digest.String()) } } return stmts[0], nil @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, digest.String()) + ctx = topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - ctx = tracecpu.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) + ctx = topsql.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 66bf5bac6d936..30365f9e2cb6f 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -43,7 +43,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu" atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) diff --git a/tidb-server/main.go b/tidb-server/main.go index 69ce4bc86d92a..5047c6b63f1b8 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -64,7 +64,7 @@ import ( "github.com/pingcap/tidb/util/sys/linux" storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" pd "github.com/tikv/pd/client" @@ -179,7 +179,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() - setupTopSQLProfiler() + topsql.SetupTopSQL() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) @@ -687,7 +687,3 @@ func stringToList(repairString string) []string { return r == ',' || r == ' ' || r == '"' }) } - -func setupTopSQLProfiler() { - tracecpu.GlobalTopSQLCPUProfiler.Run() -} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go new file mode 100644 index 0000000000000..547b84ecd9e2e --- /dev/null +++ b/util/topsql/topsql.go @@ -0,0 +1,57 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package topsql + +import ( + "context" + + "github.com/pingcap/tidb/util/topsql/tracecpu" +) + +func SetupTopSQL() { + tracecpu.GlobalTopSQLCPUProfiler.Run() +} + +// SetSQLLabels sets the SQL digest label into the goroutine. +func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { + if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { + return ctx + } + ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, sqlDigest) + registerSQL(sqlDigest, normalizedSQL) + return ctx +} + +// SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. +func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { + ctx = tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest) + registerPlan(planDigest, normalizedPlan) + return ctx +} + +func registerSQL(sqlDigest, normalizedSQL string) { + c := tracecpu.GlobalTopSQLCPUProfiler.GetCollector() + if c == nil { + return + } + c.RegisterSQL(sqlDigest, normalizedSQL) +} + +func registerPlan(planDigest string, normalizedPlan string) { + c := tracecpu.GlobalTopSQLCPUProfiler.GetCollector() + if c == nil { + return + } + c.RegisterPlan(planDigest, normalizedPlan) +} diff --git a/util/tracecpu/collector.go b/util/topsql/tracecpu/collector.go similarity index 100% rename from util/tracecpu/collector.go rename to util/topsql/tracecpu/collector.go diff --git a/util/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go similarity index 98% rename from util/tracecpu/mock/mock.go rename to util/topsql/tracecpu/mock/mock.go index 69fcef89e9b87..293c09b8a6c23 100644 --- a/util/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/parser" - "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/uber-go/atomic" ) diff --git a/util/tracecpu/profile.go b/util/topsql/tracecpu/profile.go similarity index 94% rename from util/tracecpu/profile.go rename to util/topsql/tracecpu/profile.go index 38a6f63251314..deedd7ca41463 100644 --- a/util/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -78,7 +78,7 @@ func (sp *topSQLCPUProfiler) SetCollector(c TopSQLCollector) { sp.collector.Store(c) } -func (sp *topSQLCPUProfiler) getCollector() TopSQLCollector { +func (sp *topSQLCPUProfiler) GetCollector() TopSQLCollector { c, ok := sp.collector.Load().(TopSQLCollector) if !ok || c == nil { return nil @@ -86,22 +86,6 @@ func (sp *topSQLCPUProfiler) getCollector() TopSQLCollector { return c } -func (sp *topSQLCPUProfiler) RegisterSQL(sqlDigest, normalizedSQL string) { - c := sp.getCollector() - if c == nil { - return - } - c.RegisterSQL(sqlDigest, normalizedSQL) -} - -func (sp *topSQLCPUProfiler) RegisterPlan(planDigest string, normalizedPlan string) { - c := sp.getCollector() - if c == nil { - return - } - c.RegisterPlan(planDigest, normalizedPlan) -} - func (sp *topSQLCPUProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { @@ -141,7 +125,7 @@ func (sp *topSQLCPUProfiler) startAnalyzeProfileWorker() { } stats := sp.parseCPUProfileBySQLLabels(p) sp.handleExportProfileTask(p) - if c := sp.getCollector(); c != nil { + if c := sp.GetCollector(); c != nil { c.Collect(task.end, stats) } sp.putTaskToBuffer(task) @@ -301,24 +285,19 @@ func ResetGoroutineLabelsWithOriginalCtx(ctx context.Context) { // SetSQLLabels sets the SQL digest label into the goroutine. func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { - if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { - return ctx - } if variable.EnablePProfSQLCPU.Load() { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) } else { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) } pprof.SetGoroutineLabels(ctx) - GlobalTopSQLCPUProfiler.RegisterSQL(sqlDigest, normalizedSQL) return ctx } // SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. -func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { +func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest string) context.Context { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) - GlobalTopSQLCPUProfiler.RegisterPlan(planDigest, normalizedPlan) return ctx } diff --git a/util/tracecpu/profile_test.go b/util/topsql/tracecpu_test.go similarity index 92% rename from util/tracecpu/profile_test.go rename to util/topsql/tracecpu_test.go index bdf7b7d448649..3cf194cedf587 100644 --- a/util/tracecpu/profile_test.go +++ b/util/topsql/tracecpu_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tracecpu_test +package topsql_test import ( "bytes" @@ -23,8 +23,9 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/tracecpu" - "github.com/pingcap/tidb/util/tracecpu/mock" + "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" ) func TestT(t *testing.T) { @@ -116,10 +117,10 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - ctx = tracecpu.SetSQLLabels(ctx, sql, sqlDigest) + ctx = topsql.SetSQLLabels(ctx, sql, sqlDigest) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) - tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest, plan) + topsql.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest, plan) s.mockExecute(time.Millisecond * 300) } From fdad7e9050288843025ff6a67851613f3711104a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 16:02:53 +0800 Subject: [PATCH 44/54] refine comment Signed-off-by: crazycs --- util/topsql/topsql.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 547b84ecd9e2e..76273feae0426 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -19,11 +19,12 @@ import ( "github.com/pingcap/tidb/util/topsql/tracecpu" ) +// SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { tracecpu.GlobalTopSQLCPUProfiler.Run() } -// SetSQLLabels sets the SQL digest label into the goroutine. +// SetSQLLabels sets the SQL digest label. func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { return ctx @@ -33,7 +34,7 @@ func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context. return ctx } -// SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. +// SetSQLAndPlanLabels sets the SQL and plan digest label. func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { ctx = tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest) registerPlan(planDigest, normalizedPlan) From 2e648bcab6477334bedd0207991da8265e936663 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 21:33:37 +0800 Subject: [PATCH 45/54] refine code and address comment Signed-off-by: crazycs --- executor/adapter.go | 13 ++- executor/executor.go | 8 +- executor/prepared.go | 2 +- server/conn.go | 3 + server/tidb_test.go | 4 +- session/session.go | 4 +- statistics/handle/handle.go | 4 +- .../{tracecpu => collector}/collector.go | 2 +- util/topsql/topsql.go | 21 ++--- util/topsql/tracecpu/mock/mock.go | 18 ++--- util/topsql/tracecpu/profile.go | 80 ++++++++----------- util/topsql/tracecpu_test.go | 16 ++-- 12 files changed, 87 insertions(+), 88 deletions(-) rename util/topsql/{tracecpu => collector}/collector.go (98%) diff --git a/executor/adapter.go b/executor/adapter.go index 1a6e3a2206c0f..989b33ba77b2b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -213,7 +213,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - ctx = a.wrapAndSetPlanLabelForTopSQL(ctx) + a.setPlanLabelForTopSQL(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -289,16 +289,15 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return a.InfoSchema.SchemaMetaVersion(), nil } -func (a *ExecStmt) wrapAndSetPlanLabelForTopSQL(ctx context.Context) context.Context { +func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) { if a.Plan == nil || !variable.TopSQLEnabled() { - return ctx + return } _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) - if len(planDigest) > 0 { - ctx = topsql.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) + if sqlDigest != nil { + topsql.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) } - return ctx } // Exec builds an Executor from a plan. If the Executor doesn't return result, @@ -371,7 +370,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. - ctx = a.wrapAndSetPlanLabelForTopSQL(ctx) + a.setPlanLabelForTopSQL(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) diff --git a/executor/executor.go b/executor/executor.go index 3dcf2935027b4..d8a79bcc325a0 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,6 +18,7 @@ import ( "fmt" "math" "runtime" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -1660,8 +1661,13 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { s = prepareStmt.PreparedAst.Stmt sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. + goCtx := context.Background() + if variable.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { + goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", util.QueryStrForLog(prepareStmt.NormalizedSQL))) + pprof.SetGoroutineLabels(goCtx) + } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - topsql.SetSQLLabels(context.Background(), prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) + topsql.SetSQLLabels(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index 9ebd371b22b96..478f9ac51df83 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - ctx = topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) + topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn.go b/server/conn.go index 315428c440790..bba2a41a6a8c1 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1018,6 +1018,9 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] + if variable.TopSQLEnabled() { + defer pprof.SetGoroutineLabels(ctx) + } if variable.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { diff --git a/server/tidb_test.go b/server/tidb_test.go index 0aef3ef05f855..09025e8bec95b 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -92,7 +92,7 @@ func (ts *tidbTestSuite) SetUpSuite(c *C) { func (ts *tidbTestTopSQLSuite) SetUpSuite(c *C) { ts.tidbTestSuiteBase.SetUpSuite(c) - tracecpu.GlobalTopSQLCPUProfiler.Run() + tracecpu.GlobalSQLCPUProfiler.Run() } func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { @@ -1181,7 +1181,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { c.Assert(err, IsNil) }() collector := mock.NewTopSQLCollector() - tracecpu.GlobalTopSQLCPUProfiler.SetCollector(collector) + tracecpu.GlobalSQLCPUProfiler.SetCollector(collector) dbt := &DBTest{c, db} dbt.mustExec("drop database if exists topsql") diff --git a/session/session.go b/session/session.go index 7b799130b0260..009ffd42c26fc 100644 --- a/session/session.go +++ b/session/session.go @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - ctx = topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) + topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - ctx = topsql.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) + topsql.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 30365f9e2cb6f..9bf7ec8e11818 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -17,6 +17,7 @@ import ( "context" "encoding/json" "fmt" + "runtime/pprof" "sort" "strconv" "sync" @@ -43,7 +44,6 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/topsql/tracecpu" atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -123,7 +123,7 @@ func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context. func (h *Handle) execRestrictedSQL(ctx context.Context, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { if variable.TopSQLEnabled() { - defer tracecpu.ResetGoroutineLabelsWithOriginalCtx(ctx) + defer pprof.SetGoroutineLabels(ctx) } stmt, err := exec.ParseWithParams(ctx, sql, params...) if err != nil { diff --git a/util/topsql/tracecpu/collector.go b/util/topsql/collector/collector.go similarity index 98% rename from util/topsql/tracecpu/collector.go rename to util/topsql/collector/collector.go index 444e499634acf..f46032de4fc07 100644 --- a/util/topsql/tracecpu/collector.go +++ b/util/topsql/collector/collector.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tracecpu +package collector // TopSQLCollector uses to collect SQL stats. // TODO: add a collector to collect and store the SQL stats. diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 76273feae0426..0ce855c16fdf8 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -21,28 +21,29 @@ import ( // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { - tracecpu.GlobalTopSQLCPUProfiler.Run() + tracecpu.GlobalSQLCPUProfiler.Run() } // SetSQLLabels sets the SQL digest label. -func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { +func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { - return ctx + return } - ctx = tracecpu.SetSQLLabels(ctx, normalizedSQL, sqlDigest) + tracecpu.SetSQLLabels(ctx, sqlDigest) registerSQL(sqlDigest, normalizedSQL) - return ctx } // SetSQLAndPlanLabels sets the SQL and plan digest label. -func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) context.Context { - ctx = tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest) +func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) { + if len(sqlDigest) == 0 || len(planDigest) == 0 { + return + } + tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest) registerPlan(planDigest, normalizedPlan) - return ctx } func registerSQL(sqlDigest, normalizedSQL string) { - c := tracecpu.GlobalTopSQLCPUProfiler.GetCollector() + c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return } @@ -50,7 +51,7 @@ func registerSQL(sqlDigest, normalizedSQL string) { } func registerPlan(planDigest string, normalizedPlan string) { - c := tracecpu.GlobalTopSQLCPUProfiler.GetCollector() + c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return } diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 293c09b8a6c23..ce7ee0ad20bff 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/parser" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/collector" "github.com/uber-go/atomic" ) @@ -30,7 +30,7 @@ type TopSQLCollector struct { // plan_digest -> normalized plan planMap map[string]string // (sql + plan_digest) -> sql stats - sqlStatsMap map[string]*tracecpu.TopSQLRecord + sqlStatsMap map[string]*collector.TopSQLRecord collectCnt atomic.Int64 } @@ -39,12 +39,12 @@ func NewTopSQLCollector() *TopSQLCollector { return &TopSQLCollector{ sqlMap: make(map[string]string), planMap: make(map[string]string), - sqlStatsMap: make(map[string]*tracecpu.TopSQLRecord), + sqlStatsMap: make(map[string]*collector.TopSQLRecord), } } // Collect uses for testing. -func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.TopSQLRecord) { +func (c *TopSQLCollector) Collect(ts int64, stats []collector.TopSQLRecord) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -55,7 +55,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.TopSQLRecord) { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] if !ok { - stats = &tracecpu.TopSQLRecord{ + stats = &collector.TopSQLRecord{ SQLDigest: stmt.SQLDigest, PlanDigest: stmt.PlanDigest, } @@ -66,7 +66,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.TopSQLRecord) { } // GetSQLStatsBySQLWithRetry uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.TopSQLRecord { +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*collector.TopSQLRecord { after := time.After(time.Second * 10) for { select { @@ -83,8 +83,8 @@ func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bo } // GetSQLStatsBySQL uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.TopSQLRecord { - stats := make([]*tracecpu.TopSQLRecord, 0, 2) +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*collector.TopSQLRecord { + stats := make([]*collector.TopSQLRecord, 0, 2) sqlDigest := GenSQLDigest(sql) c.Lock() for _, stmt := range c.sqlStatsMap { @@ -158,7 +158,7 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } -func (c *TopSQLCollector) hash(stat tracecpu.TopSQLRecord) string { +func (c *TopSQLCollector) hash(stat collector.TopSQLRecord) string { return stat.SQLDigest + stat.PlanDigest } diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index deedd7ca41463..a5d8ffde80971 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/collector" "go.uber.org/zap" ) @@ -39,10 +40,10 @@ const ( labelPlanDigest = "plan_digest" ) -// GlobalTopSQLCPUProfiler is the global SQL stats profiler. -var GlobalTopSQLCPUProfiler = newTopSQLCPUProfiler() +// GlobalSQLCPUProfiler is the global SQL stats profiler. +var GlobalSQLCPUProfiler = newSQLCPUProfiler() -type topSQLCPUProfiler struct { +type sqlCPUProfiler struct { taskCh chan *profileTask mu struct { @@ -61,32 +62,32 @@ var ( } ) -// newTopSQLCPUProfiler create a topSQLCPUProfiler. -func newTopSQLCPUProfiler() *topSQLCPUProfiler { - return &topSQLCPUProfiler{ +// newSQLCPUProfiler create a sqlCPUProfiler. +func newSQLCPUProfiler() *sqlCPUProfiler { + return &sqlCPUProfiler{ taskCh: make(chan *profileTask, 128), } } -func (sp *topSQLCPUProfiler) Run() { +func (sp *sqlCPUProfiler) Run() { logutil.BgLogger().Info("cpu profiler started") go sp.startCPUProfileWorker() go sp.startAnalyzeProfileWorker() } -func (sp *topSQLCPUProfiler) SetCollector(c TopSQLCollector) { +func (sp *sqlCPUProfiler) SetCollector(c collector.TopSQLCollector) { sp.collector.Store(c) } -func (sp *topSQLCPUProfiler) GetCollector() TopSQLCollector { - c, ok := sp.collector.Load().(TopSQLCollector) +func (sp *sqlCPUProfiler) GetCollector() collector.TopSQLCollector { + c, ok := sp.collector.Load().(collector.TopSQLCollector) if !ok || c == nil { return nil } return c } -func (sp *topSQLCPUProfiler) startCPUProfileWorker() { +func (sp *sqlCPUProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { if sp.IsEnabled() { @@ -97,7 +98,7 @@ func (sp *topSQLCPUProfiler) startCPUProfileWorker() { } } -func (sp *topSQLCPUProfiler) doCPUProfile() { +func (sp *sqlCPUProfiler) doCPUProfile() { intervalSecond := variable.TopSQLVariable.PrecisionSeconds.Load() task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { @@ -113,7 +114,7 @@ func (sp *topSQLCPUProfiler) doCPUProfile() { sp.taskCh <- task } -func (sp *topSQLCPUProfiler) startAnalyzeProfileWorker() { +func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { defer util.Recover("top-sql", "analyzeProfileWorker", nil, false) for { task := <-sp.taskCh @@ -137,14 +138,14 @@ type profileTask struct { end int64 } -func (sp *topSQLCPUProfiler) newProfileTask() *profileTask { +func (sp *sqlCPUProfiler) newProfileTask() *profileTask { buf := profileBufPool.Get().(*bytes.Buffer) return &profileTask{ buf: buf, } } -func (sp *topSQLCPUProfiler) putTaskToBuffer(task *profileTask) { +func (sp *sqlCPUProfiler) putTaskToBuffer(task *profileTask) { task.buf.Reset() profileBufPool.Put(task.buf) } @@ -153,9 +154,9 @@ func (sp *topSQLCPUProfiler) putTaskToBuffer(task *profileTask) { // output the TopSQLRecord slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. // The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. -// Since `topSQLCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data +// Since `sqlCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. -func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []TopSQLRecord { +func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []collector.TopSQLRecord { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -183,12 +184,12 @@ func (sp *topSQLCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []To return sp.createSQLStats(sqlMap) } -func (sp *topSQLCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []TopSQLRecord { - stats := make([]TopSQLRecord, 0, len(sqlMap)) +func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []collector.TopSQLRecord { + stats := make([]collector.TopSQLRecord, 0, len(sqlMap)) for sqlDigest, stmt := range sqlMap { stmt.tune() for planDigest, val := range stmt.plans { - stats = append(stats, TopSQLRecord{ + stats = append(stats, collector.TopSQLRecord{ SQLDigest: sqlDigest, PlanDigest: planDigest, CPUTimeMs: uint32(time.Duration(val).Milliseconds()), @@ -237,7 +238,7 @@ func (s *sqlStats) tune() { s.plans[""] += optimize } -func (sp *topSQLCPUProfiler) handleExportProfileTask(p *profile.Profile) { +func (sp *sqlCPUProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept == nil { @@ -246,7 +247,7 @@ func (sp *topSQLCPUProfiler) handleExportProfileTask(p *profile.Profile) { sp.mu.ept.mergeProfile(p) } -func (sp *topSQLCPUProfiler) hasExportProfileTask() bool { +func (sp *sqlCPUProfiler) hasExportProfileTask() bool { sp.mu.Lock() has := sp.mu.ept != nil sp.mu.Unlock() @@ -254,16 +255,16 @@ func (sp *topSQLCPUProfiler) hasExportProfileTask() bool { } // IsEnabled return true if it is(should be) enabled. It exports for tests. -func (sp *topSQLCPUProfiler) IsEnabled() bool { +func (sp *sqlCPUProfiler) IsEnabled() bool { return variable.TopSQLEnabled() || sp.hasExportProfileTask() } // StartCPUProfile same like pprof.StartCPUProfile. -// Because the GlobalTopSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, +// Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalTopSQLCPUProfiler.IsEnabled() { - return GlobalTopSQLCPUProfiler.startExportCPUProfile(w) + if GlobalSQLCPUProfiler.IsEnabled() { + return GlobalSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) } @@ -271,37 +272,26 @@ func StartCPUProfile(w io.Writer) error { // StopCPUProfile same like pprof.StopCPUProfile. // other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. func StopCPUProfile() error { - if GlobalTopSQLCPUProfiler.IsEnabled() { - return GlobalTopSQLCPUProfiler.stopExportCPUProfile() + if GlobalSQLCPUProfiler.IsEnabled() { + return GlobalSQLCPUProfiler.stopExportCPUProfile() } pprof.StopCPUProfile() return nil } -// ResetGoroutineLabelsWithOriginalCtx resets the goroutine label with the original ctx. -func ResetGoroutineLabelsWithOriginalCtx(ctx context.Context) { - pprof.SetGoroutineLabels(ctx) -} - // SetSQLLabels sets the SQL digest label into the goroutine. -func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) context.Context { - if variable.EnablePProfSQLCPU.Load() { - ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelSQL, util.QueryStrForLog(normalizedSQL))) - } else { - ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) - } +func SetSQLLabels(ctx context.Context, sqlDigest string) { + ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) pprof.SetGoroutineLabels(ctx) - return ctx } // SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. -func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest string) context.Context { +func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest string) { ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) pprof.SetGoroutineLabels(ctx) - return ctx } -func (sp *topSQLCPUProfiler) startExportCPUProfile(w io.Writer) error { +func (sp *sqlCPUProfiler) startExportCPUProfile(w io.Writer) error { sp.mu.Lock() defer sp.mu.Unlock() if sp.mu.ept != nil { @@ -311,7 +301,7 @@ func (sp *topSQLCPUProfiler) startExportCPUProfile(w io.Writer) error { return nil } -func (sp *topSQLCPUProfiler) stopExportCPUProfile() error { +func (sp *sqlCPUProfiler) stopExportCPUProfile() error { sp.mu.Lock() ept := sp.mu.ept sp.mu.ept = nil @@ -329,7 +319,7 @@ func (sp *topSQLCPUProfiler) stopExportCPUProfile() error { // removeLabel uses to remove labels for export cpu profile data. // Since the sql_digest and plan_digest label is strange for other users. // If `variable.EnablePProfSQLCPU` is true means wanto keep the `sql` label, otherwise, remove the `sql` label too. -func (sp *topSQLCPUProfiler) removeLabel(p *profile.Profile) { +func (sp *sqlCPUProfiler) removeLabel(p *profile.Profile) { if p == nil { return } diff --git a/util/topsql/tracecpu_test.go b/util/topsql/tracecpu_test.go index 3cf194cedf587..4b6753a1c814c 100644 --- a/util/topsql/tracecpu_test.go +++ b/util/topsql/tracecpu_test.go @@ -41,12 +41,12 @@ func (s *testSuite) SetUpSuite(c *C) { variable.TopSQLVariable.Enable.Store(true) variable.TopSQLVariable.AgentAddress.Store("mock") variable.TopSQLVariable.PrecisionSeconds.Store(1) - tracecpu.GlobalTopSQLCPUProfiler.Run() + tracecpu.GlobalSQLCPUProfiler.Run() } func (s *testSuite) TestTopSQLCPUProfile(c *C) { collector := mock.NewTopSQLCollector() - tracecpu.GlobalTopSQLCPUProfiler.SetCollector(collector) + tracecpu.GlobalSQLCPUProfiler.SetCollector(collector) reqs := []struct { sql string plan string @@ -93,21 +93,21 @@ func (s *testSuite) TestTopSQLCPUProfile(c *C) { func (s *testSuite) TestIsEnabled(c *C) { s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsFalse) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsFalse) s.setTopSQLEnable(true) err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) c.Assert(err, IsNil) - c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) err = tracecpu.StopCPUProfile() c.Assert(err, IsNil) s.setTopSQLEnable(false) - c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsFalse) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsFalse) s.setTopSQLEnable(true) - c.Assert(tracecpu.GlobalTopSQLCPUProfiler.IsEnabled(), IsTrue) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) } func (s *testSuite) setTopSQLEnable(enabled bool) { @@ -117,7 +117,7 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - ctx = topsql.SetSQLLabels(ctx, sql, sqlDigest) + topsql.SetSQLLabels(ctx, sql, sqlDigest) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) topsql.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest, plan) From d731285f41c488afec31ca81c84caa9da2197b40 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 22:05:39 +0800 Subject: [PATCH 46/54] refactor the top sql api Signed-off-by: crazycs --- executor/adapter.go | 6 +++--- executor/executor.go | 2 +- executor/prepared.go | 2 +- server/conn_stmt.go | 2 +- session/session.go | 6 +++--- util/topsql/topsql.go | 27 +++++++++++++-------------- util/topsql/tracecpu/profile.go | 16 ++++++---------- util/topsql/tracecpu_test.go | 4 ++-- 8 files changed, 30 insertions(+), 35 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 989b33ba77b2b..84659f6366b00 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -293,10 +293,10 @@ func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) { if a.Plan == nil || !variable.TopSQLEnabled() { return } - _, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() + normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) - if sqlDigest != nil { - topsql.SetSQLAndPlanLabels(ctx, sqlDigest.String(), planDigest, normalizedPlan) + if len(normalizedPlan) > 0 { + topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest.String(), normalizedPlan, planDigest) } } diff --git a/executor/executor.go b/executor/executor.go index d8a79bcc325a0..b70cc78debe08 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1667,7 +1667,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { pprof.SetGoroutineLabels(goCtx) } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - topsql.SetSQLLabels(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String()) + topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String(), "", "") } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index 478f9ac51df83..53cf22877a199 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) + topsql.AttachSQLInfo(ctx, normalizedSQL, digest.String(), "", "") } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index a1d28fd27fd39..590064e244b86 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -270,7 +270,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if variable.TopSQLEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { - topsql.SetSQLLabels(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String()) + topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String(), "", "") } } sql := "" diff --git a/session/session.go b/session/session.go index 009ffd42c26fc..753bb1e42cd40 100644 --- a/session/session.go +++ b/session/session.go @@ -1389,7 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when sql execute finish. - topsql.SetSQLLabels(ctx, normalized, digest.String()) + topsql.AttachSQLInfo(ctx, normalized, digest.String(), "", "") } } return stmts[0], nil @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - topsql.SetSQLLabels(ctx, normalizedSQL, digest.String()) + topsql.AttachSQLInfo(ctx, normalizedSQL, digest.String(), "", "") } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - topsql.SetSQLLabels(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String()) + topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String(), "", "") } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 0ce855c16fdf8..4e4cd50b5e58a 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -15,6 +15,7 @@ package topsql import ( "context" + "runtime/pprof" "github.com/pingcap/tidb/util/topsql/tracecpu" ) @@ -24,25 +25,23 @@ func SetupTopSQL() { tracecpu.GlobalSQLCPUProfiler.Run() } -// SetSQLLabels sets the SQL digest label. -func SetSQLLabels(ctx context.Context, normalizedSQL, sqlDigest string) { +// AttachSQLInfo attach the sql information info top sql. +func AttachSQLInfo(ctx context.Context, normalizedSQL, sqlDigest, normalizedPlan, planDigest string) { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { return } - tracecpu.SetSQLLabels(ctx, sqlDigest) - registerSQL(sqlDigest, normalizedSQL) -} - -// SetSQLAndPlanLabels sets the SQL and plan digest label. -func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest, normalizedPlan string) { - if len(sqlDigest) == 0 || len(planDigest) == 0 { - return + ctx = tracecpu.CtxWithDigest(ctx, sqlDigest, planDigest) + pprof.SetGoroutineLabels(ctx) + + if len(planDigest) == 0 { + // If plan digest is '', indicate it is the first time to attach the SQL info, since it only know the sql digest. + linkSQLTextWithDigest(sqlDigest, normalizedSQL) + } else { + linkPlanTextWithDigest(planDigest, normalizedPlan) } - tracecpu.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest) - registerPlan(planDigest, normalizedPlan) } -func registerSQL(sqlDigest, normalizedSQL string) { +func linkSQLTextWithDigest(sqlDigest, normalizedSQL string) { c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return @@ -50,7 +49,7 @@ func registerSQL(sqlDigest, normalizedSQL string) { c.RegisterSQL(sqlDigest, normalizedSQL) } -func registerPlan(planDigest string, normalizedPlan string) { +func linkPlanTextWithDigest(planDigest string, normalizedPlan string) { c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index a5d8ffde80971..818301cf10114 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -279,16 +279,12 @@ func StopCPUProfile() error { return nil } -// SetSQLLabels sets the SQL digest label into the goroutine. -func SetSQLLabels(ctx context.Context, sqlDigest string) { - ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) - pprof.SetGoroutineLabels(ctx) -} - -// SetSQLAndPlanLabels sets the SQL and plan digest label into the goroutine. -func SetSQLAndPlanLabels(ctx context.Context, sqlDigest, planDigest string) { - ctx = pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) - pprof.SetGoroutineLabels(ctx) +// CtxWithDigest wrap the ctx with sql digest, if plan digest is not null, wrap with plan digest too. +func CtxWithDigest(ctx context.Context, sqlDigest, planDigest string) context.Context { + if len(planDigest) == 0 { + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) + } + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) } func (sp *sqlCPUProfiler) startExportCPUProfile(w io.Writer) error { diff --git a/util/topsql/tracecpu_test.go b/util/topsql/tracecpu_test.go index 4b6753a1c814c..4b2067f1228c1 100644 --- a/util/topsql/tracecpu_test.go +++ b/util/topsql/tracecpu_test.go @@ -117,10 +117,10 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - topsql.SetSQLLabels(ctx, sql, sqlDigest) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, "", "") s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) - topsql.SetSQLAndPlanLabels(ctx, sqlDigest, planDigest, plan) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, plan, planDigest) s.mockExecute(time.Millisecond * 300) } From fca381db6be127c3f4ce8ecd92cbd3b7870f60e4 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 22:34:35 +0800 Subject: [PATCH 47/54] tiny refactor collector and address comment Signed-off-by: crazycs --- util/topsql/collector/collector.go | 13 ++------- util/topsql/topsql.go | 11 +++++-- util/topsql/tracecpu/mock/mock.go | 18 ++++++------ util/topsql/tracecpu/profile.go | 47 ++++++++++++++++++++---------- 4 files changed, 52 insertions(+), 37 deletions(-) diff --git a/util/topsql/collector/collector.go b/util/topsql/collector/collector.go index f46032de4fc07..5cfcfa078d8bf 100644 --- a/util/topsql/collector/collector.go +++ b/util/topsql/collector/collector.go @@ -13,19 +13,12 @@ package collector +import "github.com/pingcap/tidb/util/topsql/tracecpu" + // TopSQLCollector uses to collect SQL stats. // TODO: add a collector to collect and store the SQL stats. type TopSQLCollector interface { - // Collect uses to collect the SQL execution information. - // ts is a Unix time, unit is second. - Collect(ts int64, stats []TopSQLRecord) + tracecpu.Collector RegisterSQL(sqlDigest, normalizedSQL string) RegisterPlan(planDigest string, normalizedPlan string) } - -// TopSQLRecord contains the SQL meta and execution information. -type TopSQLRecord struct { - SQLDigest string - PlanDigest string - CPUTimeMs uint32 -} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 4e4cd50b5e58a..aaf9814f9af2d 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -17,6 +17,7 @@ import ( "context" "runtime/pprof" + "github.com/pingcap/tidb/util/topsql/collector" "github.com/pingcap/tidb/util/topsql/tracecpu" ) @@ -46,7 +47,10 @@ func linkSQLTextWithDigest(sqlDigest, normalizedSQL string) { if c == nil { return } - c.RegisterSQL(sqlDigest, normalizedSQL) + topc, ok := c.(collector.TopSQLCollector) + if ok { + topc.RegisterSQL(sqlDigest, normalizedSQL) + } } func linkPlanTextWithDigest(planDigest string, normalizedPlan string) { @@ -54,5 +58,8 @@ func linkPlanTextWithDigest(planDigest string, normalizedPlan string) { if c == nil { return } - c.RegisterPlan(planDigest, normalizedPlan) + topc, ok := c.(collector.TopSQLCollector) + if ok { + topc.RegisterPlan(planDigest, normalizedPlan) + } } diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index ce7ee0ad20bff..58117bf7e0d18 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/parser" - "github.com/pingcap/tidb/util/topsql/collector" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/uber-go/atomic" ) @@ -30,7 +30,7 @@ type TopSQLCollector struct { // plan_digest -> normalized plan planMap map[string]string // (sql + plan_digest) -> sql stats - sqlStatsMap map[string]*collector.TopSQLRecord + sqlStatsMap map[string]*tracecpu.SQLCPUResult collectCnt atomic.Int64 } @@ -39,12 +39,12 @@ func NewTopSQLCollector() *TopSQLCollector { return &TopSQLCollector{ sqlMap: make(map[string]string), planMap: make(map[string]string), - sqlStatsMap: make(map[string]*collector.TopSQLRecord), + sqlStatsMap: make(map[string]*tracecpu.SQLCPUResult), } } // Collect uses for testing. -func (c *TopSQLCollector) Collect(ts int64, stats []collector.TopSQLRecord) { +func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLCPUResult) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -55,7 +55,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []collector.TopSQLRecord) { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] if !ok { - stats = &collector.TopSQLRecord{ + stats = &tracecpu.SQLCPUResult{ SQLDigest: stmt.SQLDigest, PlanDigest: stmt.PlanDigest, } @@ -66,7 +66,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []collector.TopSQLRecord) { } // GetSQLStatsBySQLWithRetry uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*collector.TopSQLRecord { +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { after := time.After(time.Second * 10) for { select { @@ -83,8 +83,8 @@ func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bo } // GetSQLStatsBySQL uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*collector.TopSQLRecord { - stats := make([]*collector.TopSQLRecord, 0, 2) +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { + stats := make([]*tracecpu.SQLCPUResult, 0, 2) sqlDigest := GenSQLDigest(sql) c.Lock() for _, stmt := range c.sqlStatsMap { @@ -158,7 +158,7 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } -func (c *TopSQLCollector) hash(stat collector.TopSQLRecord) string { +func (c *TopSQLCollector) hash(stat tracecpu.SQLCPUResult) string { return stat.SQLDigest + stat.PlanDigest } diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 818301cf10114..58b4da34f5301 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/topsql/collector" "go.uber.org/zap" ) @@ -43,8 +42,22 @@ const ( // GlobalSQLCPUProfiler is the global SQL stats profiler. var GlobalSQLCPUProfiler = newSQLCPUProfiler() +// Collector uses to collect SQL execution cpu time. +type Collector interface { + // Collect uses to collect the SQL execution cpu time. + // ts is a Unix time, unit is second. + Collect(ts int64, stats []SQLCPUResult) +} + +// SQLCPUResult contains the SQL meta and cpu time. +type SQLCPUResult struct { + SQLDigest string + PlanDigest string + CPUTimeMs uint32 +} + type sqlCPUProfiler struct { - taskCh chan *profileTask + taskCh chan *profileData mu struct { sync.Mutex @@ -65,7 +78,7 @@ var ( // newSQLCPUProfiler create a sqlCPUProfiler. func newSQLCPUProfiler() *sqlCPUProfiler { return &sqlCPUProfiler{ - taskCh: make(chan *profileTask, 128), + taskCh: make(chan *profileData, 128), } } @@ -75,12 +88,12 @@ func (sp *sqlCPUProfiler) Run() { go sp.startAnalyzeProfileWorker() } -func (sp *sqlCPUProfiler) SetCollector(c collector.TopSQLCollector) { +func (sp *sqlCPUProfiler) SetCollector(c Collector) { sp.collector.Store(c) } -func (sp *sqlCPUProfiler) GetCollector() collector.TopSQLCollector { - c, ok := sp.collector.Load().(collector.TopSQLCollector) +func (sp *sqlCPUProfiler) GetCollector() Collector { + c, ok := sp.collector.Load().(Collector) if !ok || c == nil { return nil } @@ -133,30 +146,30 @@ func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { } } -type profileTask struct { +type profileData struct { buf *bytes.Buffer end int64 } -func (sp *sqlCPUProfiler) newProfileTask() *profileTask { +func (sp *sqlCPUProfiler) newProfileTask() *profileData { buf := profileBufPool.Get().(*bytes.Buffer) - return &profileTask{ + return &profileData{ buf: buf, } } -func (sp *sqlCPUProfiler) putTaskToBuffer(task *profileTask) { +func (sp *sqlCPUProfiler) putTaskToBuffer(task *profileData) { task.buf.Reset() profileBufPool.Put(task.buf) } // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, -// output the TopSQLRecord slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ +// output the SQLCPUResult slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. // The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. // Since `sqlCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. -func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []collector.TopSQLRecord { +func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLCPUResult { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -184,12 +197,12 @@ func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []colle return sp.createSQLStats(sqlMap) } -func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []collector.TopSQLRecord { - stats := make([]collector.TopSQLRecord, 0, len(sqlMap)) +func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPUResult { + stats := make([]SQLCPUResult, 0, len(sqlMap)) for sqlDigest, stmt := range sqlMap { stmt.tune() for planDigest, val := range stmt.plans { - stats = append(stats, collector.TopSQLRecord{ + stats = append(stats, SQLCPUResult{ SQLDigest: sqlDigest, PlanDigest: planDigest, CPUTimeMs: uint32(time.Duration(val).Milliseconds()), @@ -217,7 +230,7 @@ type sqlStats struct { // optimizer takes time to generated plan. // After this tune function, the `sqlStats` become to: // plans: { -// "optimize" : 100ms, // 600 - 200 - 300 +// "" : 100ms, // 600 - 200 - 300 = 100ms, indicate the optimizer generated plan time cost. // "table_scan": 200ms, // "index_scan": 300ms, // }, @@ -377,6 +390,8 @@ func ProfileHTTPHandler(w http.ResponseWriter, r *http.Request) { return } // TODO: fix me. + // This can be fixed by always starts a 1 second profiling one by one, + // but to aggregate (merge) multiple profiles into one according to the precision. // |<-- 1s -->| // -|----------|----------|----------|----------|----------|-----------|-----> Background profile task timeline. // |________________________________| From d737aa02a1e730a3483dfb5c70816aef94ce168d Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 11:47:15 +0800 Subject: [PATCH 48/54] use []byte instead of string as digest for top sql Signed-off-by: crazycs --- executor/adapter.go | 16 ++++++++------ executor/executor.go | 2 +- executor/prepared.go | 2 +- server/conn_stmt.go | 2 +- session/session.go | 6 +++--- util/topsql/collector/collector.go | 4 ++-- util/topsql/topsql.go | 8 +++---- util/topsql/tracecpu/mock/mock.go | 34 +++++++++++++++++------------- util/topsql/tracecpu/profile.go | 16 ++++++++------ util/topsql/tracecpu_test.go | 10 ++++----- 10 files changed, 54 insertions(+), 46 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 84659f6366b00..9746b0d35043e 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -296,7 +297,7 @@ func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) { normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(normalizedPlan) > 0 { - topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest.String(), normalizedPlan, planDigest) + topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest.Bytes(), normalizedPlan, planDigest.Bytes()) } } @@ -981,7 +982,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { DiskMax: diskMax, Succ: succ, Plan: getPlanTree(a.Ctx, a.Plan), - PlanDigest: planDigest, + PlanDigest: planDigest.String(), Prepared: a.isPreparedStmt, HasMoreResults: hasMoreResults, PlanFromCache: sessVars.FoundInPlanCache, @@ -1055,15 +1056,15 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { } // getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. -func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, string) { +func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, *parser.Digest) { sc := sctx.GetSessionVars().StmtCtx normalized, planDigest := sc.GetPlanDigest() if len(normalized) > 0 && planDigest != nil { - return normalized, planDigest.String() + return normalized, planDigest } normalized, planDigest = plannercore.NormalizePlan(p) sc.SetPlanDigest(normalized, planDigest) - return normalized, planDigest.String() + return normalized, planDigest } // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. @@ -1138,10 +1139,11 @@ func (a *ExecStmt) SummaryStmt(succ bool) { if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { _, planDigest := getPlanDigest(a.Ctx, a.Plan) - return planDigest + return planDigest.String() } } else { - _, planDigest = getPlanDigest(a.Ctx, a.Plan) + _, tmp := getPlanDigest(a.Ctx, a.Plan) + planDigest = tmp.String() } execDetail := stmtCtx.GetExecDetails() diff --git a/executor/executor.go b/executor/executor.go index b70cc78debe08..aa966807a7de9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1667,7 +1667,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { pprof.SetGoroutineLabels(goCtx) } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.String(), "", "") + topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.Bytes(), "", nil) } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index 53cf22877a199..bdc743cd548d8 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest.String(), "", "") + topsql.AttachSQLInfo(ctx, normalizedSQL, digest.Bytes(), "", nil) } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 590064e244b86..15b760123b2a8 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -270,7 +270,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if variable.TopSQLEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.String(), "", "") + topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.Bytes(), "", nil) } } sql := "" diff --git a/session/session.go b/session/session.go index 753bb1e42cd40..373b4b0c19fac 100644 --- a/session/session.go +++ b/session/session.go @@ -1389,7 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when sql execute finish. - topsql.AttachSQLInfo(ctx, normalized, digest.String(), "", "") + topsql.AttachSQLInfo(ctx, normalized, digest.Bytes(), "", nil) } } return stmts[0], nil @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest.String(), "", "") + topsql.AttachSQLInfo(ctx, normalizedSQL, digest.Bytes(), "", nil) } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.String(), "", "") + topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.Bytes(), "", nil) } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) diff --git a/util/topsql/collector/collector.go b/util/topsql/collector/collector.go index 5cfcfa078d8bf..9cc2a429ea990 100644 --- a/util/topsql/collector/collector.go +++ b/util/topsql/collector/collector.go @@ -19,6 +19,6 @@ import "github.com/pingcap/tidb/util/topsql/tracecpu" // TODO: add a collector to collect and store the SQL stats. type TopSQLCollector interface { tracecpu.Collector - RegisterSQL(sqlDigest, normalizedSQL string) - RegisterPlan(planDigest string, normalizedPlan string) + RegisterSQL(sqlDigest []byte, normalizedSQL string) + RegisterPlan(planDigest []byte, normalizedPlan string) } diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index aaf9814f9af2d..da1bb631f051b 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -27,14 +27,14 @@ func SetupTopSQL() { } // AttachSQLInfo attach the sql information info top sql. -func AttachSQLInfo(ctx context.Context, normalizedSQL, sqlDigest, normalizedPlan, planDigest string) { +func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest []byte, normalizedPlan string, planDigest []byte) { if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { return } ctx = tracecpu.CtxWithDigest(ctx, sqlDigest, planDigest) pprof.SetGoroutineLabels(ctx) - if len(planDigest) == 0 { + if len(normalizedSQL) == 0 || len(planDigest) == 0 { // If plan digest is '', indicate it is the first time to attach the SQL info, since it only know the sql digest. linkSQLTextWithDigest(sqlDigest, normalizedSQL) } else { @@ -42,7 +42,7 @@ func AttachSQLInfo(ctx context.Context, normalizedSQL, sqlDigest, normalizedPlan } } -func linkSQLTextWithDigest(sqlDigest, normalizedSQL string) { +func linkSQLTextWithDigest(sqlDigest []byte, normalizedSQL string) { c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return @@ -53,7 +53,7 @@ func linkSQLTextWithDigest(sqlDigest, normalizedSQL string) { } } -func linkPlanTextWithDigest(planDigest string, normalizedPlan string) { +func linkPlanTextWithDigest(planDigest []byte, normalizedPlan string) { c := tracecpu.GlobalSQLCPUProfiler.GetCollector() if c == nil { return diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 58117bf7e0d18..2737306f1a096 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -14,10 +14,12 @@ package mock import ( + "bytes" "sync" "time" "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/uber-go/atomic" ) @@ -88,9 +90,9 @@ func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tr sqlDigest := GenSQLDigest(sql) c.Lock() for _, stmt := range c.sqlStatsMap { - if stmt.SQLDigest == sqlDigest { + if bytes.Equal(stmt.SQLDigest, sqlDigest.Bytes()) { if planIsNotNull { - plan := c.planMap[stmt.PlanDigest] + plan := c.planMap[string(stmt.PlanDigest)] if len(plan) > 0 { stats = append(stats, stmt) } @@ -104,38 +106,40 @@ func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tr } // GetSQL uses for testing. -func (c *TopSQLCollector) GetSQL(sqlDigest string) string { +func (c *TopSQLCollector) GetSQL(sqlDigest []byte) string { c.Lock() - sql := c.sqlMap[sqlDigest] + sql := c.sqlMap[string(sqlDigest)] c.Unlock() return sql } // GetPlan uses for testing. -func (c *TopSQLCollector) GetPlan(planDigest string) string { +func (c *TopSQLCollector) GetPlan(planDigest []byte) string { c.Lock() - plan := c.planMap[planDigest] + plan := c.planMap[string(planDigest)] c.Unlock() return plan } // RegisterSQL uses for testing. -func (c *TopSQLCollector) RegisterSQL(sqlDigest, normalizedSQL string) { +func (c *TopSQLCollector) RegisterSQL(sqlDigest []byte, normalizedSQL string) { + digestStr := string(hack.String(sqlDigest)) c.Lock() - _, ok := c.sqlMap[sqlDigest] + _, ok := c.sqlMap[digestStr] if !ok { - c.sqlMap[sqlDigest] = normalizedSQL + c.sqlMap[digestStr] = normalizedSQL } c.Unlock() } // RegisterPlan uses for testing. -func (c *TopSQLCollector) RegisterPlan(planDigest string, normalizedPlan string) { +func (c *TopSQLCollector) RegisterPlan(planDigest []byte, normalizedPlan string) { + digestStr := string(hack.String(planDigest)) c.Lock() - _, ok := c.planMap[planDigest] + _, ok := c.planMap[digestStr] if !ok { - c.planMap[planDigest] = normalizedPlan + c.planMap[digestStr] = normalizedPlan } c.Unlock() } @@ -159,11 +163,11 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } func (c *TopSQLCollector) hash(stat tracecpu.SQLCPUResult) string { - return stat.SQLDigest + stat.PlanDigest + return string(stat.SQLDigest) + string(stat.PlanDigest) } // GenSQLDigest uses for testing. -func GenSQLDigest(sql string) string { +func GenSQLDigest(sql string) *parser.Digest { _, digest := parser.NormalizeDigest(sql) - return digest.String() + return digest } diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 58b4da34f5301..c0fdbe5fee52f 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -29,6 +29,7 @@ import ( "github.com/google/pprof/profile" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -51,8 +52,8 @@ type Collector interface { // SQLCPUResult contains the SQL meta and cpu time. type SQLCPUResult struct { - SQLDigest string - PlanDigest string + SQLDigest []byte + PlanDigest []byte CPUTimeMs uint32 } @@ -203,8 +204,8 @@ func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPURe stmt.tune() for planDigest, val := range stmt.plans { stats = append(stats, SQLCPUResult{ - SQLDigest: sqlDigest, - PlanDigest: planDigest, + SQLDigest: []byte(sqlDigest), + PlanDigest: []byte(planDigest), CPUTimeMs: uint32(time.Duration(val).Milliseconds()), }) } @@ -293,11 +294,12 @@ func StopCPUProfile() error { } // CtxWithDigest wrap the ctx with sql digest, if plan digest is not null, wrap with plan digest too. -func CtxWithDigest(ctx context.Context, sqlDigest, planDigest string) context.Context { +func CtxWithDigest(ctx context.Context, sqlDigest, planDigest []byte) context.Context { if len(planDigest) == 0 { - return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest)) + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest)))) } - return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest, labelPlanDigest, planDigest)) + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest)), + labelPlanDigest, string(hack.String(planDigest)))) } func (sp *sqlCPUProfiler) startExportCPUProfile(w io.Writer) error { diff --git a/util/topsql/tracecpu_test.go b/util/topsql/tracecpu_test.go index 4b2067f1228c1..8c392a3d6f7a5 100644 --- a/util/topsql/tracecpu_test.go +++ b/util/topsql/tracecpu_test.go @@ -117,18 +117,18 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - topsql.AttachSQLInfo(ctx, sql, sqlDigest, "", "") + topsql.AttachSQLInfo(ctx, sql, sqlDigest.Bytes(), "", nil) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) - topsql.AttachSQLInfo(ctx, sql, sqlDigest, plan, planDigest) + topsql.AttachSQLInfo(ctx, sql, sqlDigest.Bytes(), plan, planDigest.Bytes()) s.mockExecute(time.Millisecond * 300) } -func genDigest(str string) string { +func genDigest(str string) *parser.Digest { if str == "" { - return "" + return parser.NewDigest(nil) } - return parser.DigestNormalized(str).String() + return parser.DigestNormalized(str) } func (s *testSuite) mockExecute(d time.Duration) { From 657be6596bcb0775065181a67966cbac9d26ea4a Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 13:04:20 +0800 Subject: [PATCH 49/54] Update util/topsql/topsql.go Co-authored-by: Wenxuan --- util/topsql/topsql.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index da1bb631f051b..995131a10b54d 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -34,7 +34,7 @@ func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest []byte, ctx = tracecpu.CtxWithDigest(ctx, sqlDigest, planDigest) pprof.SetGoroutineLabels(ctx) - if len(normalizedSQL) == 0 || len(planDigest) == 0 { + if len(normalizedPlan) == 0 || len(planDigest) == 0 { // If plan digest is '', indicate it is the first time to attach the SQL info, since it only know the sql digest. linkSQLTextWithDigest(sqlDigest, normalizedSQL) } else { From 18ca0223ceb14c1c8bf0a0a606a756e3d58903f2 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 13:21:26 +0800 Subject: [PATCH 50/54] address comment Signed-off-by: crazycs --- executor/adapter.go | 2 +- executor/executor.go | 2 +- executor/prepared.go | 2 +- server/conn_stmt.go | 2 +- session/session.go | 8 ++++---- statistics/handle/handle.go | 1 + util/topsql/topsql.go | 21 ++++++++++++++------- util/topsql/tracecpu_test.go | 4 ++-- 8 files changed, 25 insertions(+), 17 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 9746b0d35043e..406a5f62e97dd 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -297,7 +297,7 @@ func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) { normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(normalizedPlan) > 0 { - topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest.Bytes(), normalizedPlan, planDigest.Bytes()) + topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) } } diff --git a/executor/executor.go b/executor/executor.go index aa966807a7de9..d9cbc58179519 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1667,7 +1667,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { pprof.SetGoroutineLabels(goCtx) } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest.Bytes(), "", nil) + topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index bdc743cd548d8..053351a6f011a 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest.Bytes(), "", nil) + topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 15b760123b2a8..297d65971efa5 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -270,7 +270,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if variable.TopSQLEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest.Bytes(), "", nil) + topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil) } } sql := "" diff --git a/session/session.go b/session/session.go index 373b4b0c19fac..455dda96978ba 100644 --- a/session/session.go +++ b/session/session.go @@ -1388,8 +1388,8 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter if variable.TopSQLEnabled() { normalized, digest := parser.NormalizeDigest(sql) if digest != nil { - // Fixme: reset/clean the label when sql execute finish. - topsql.AttachSQLInfo(ctx, normalized, digest.Bytes(), "", nil) + // Fixme: reset/clean the label when internal sql execute finish. + topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) } } return stmts[0], nil @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest.Bytes(), "", nil) + topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest.Bytes(), "", nil) + topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 9bf7ec8e11818..8d4ca1bc678bb 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -123,6 +123,7 @@ func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context. func (h *Handle) execRestrictedSQL(ctx context.Context, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { if variable.TopSQLEnabled() { + // Restore the goroutine label by using the original ctx after execution is finished. defer pprof.SetGoroutineLabels(ctx) } stmt, err := exec.ParseWithParams(ctx, sql, params...) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 995131a10b54d..255ece7256c09 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -17,6 +17,7 @@ import ( "context" "runtime/pprof" + "github.com/pingcap/parser" "github.com/pingcap/tidb/util/topsql/collector" "github.com/pingcap/tidb/util/topsql/tracecpu" ) @@ -27,19 +28,25 @@ func SetupTopSQL() { } // AttachSQLInfo attach the sql information info top sql. -func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest []byte, normalizedPlan string, planDigest []byte) { - if len(normalizedSQL) == 0 || len(sqlDigest) == 0 { - return +func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest *parser.Digest, normalizedPlan string, planDigest *parser.Digest) context.Context { + if len(normalizedSQL) == 0 || sqlDigest == nil || len(sqlDigest.Bytes()) == 0 { + return ctx + } + var sqlDigestBytes, planDigestBytes []byte + sqlDigestBytes = sqlDigest.Bytes() + if planDigest != nil { + planDigestBytes = planDigest.Bytes() } - ctx = tracecpu.CtxWithDigest(ctx, sqlDigest, planDigest) + ctx = tracecpu.CtxWithDigest(ctx, sqlDigestBytes, planDigestBytes) pprof.SetGoroutineLabels(ctx) - if len(normalizedPlan) == 0 || len(planDigest) == 0 { + if len(normalizedPlan) == 0 || len(planDigestBytes) == 0 { // If plan digest is '', indicate it is the first time to attach the SQL info, since it only know the sql digest. - linkSQLTextWithDigest(sqlDigest, normalizedSQL) + linkSQLTextWithDigest(sqlDigestBytes, normalizedSQL) } else { - linkPlanTextWithDigest(planDigest, normalizedPlan) + linkPlanTextWithDigest(planDigestBytes, normalizedPlan) } + return ctx } func linkSQLTextWithDigest(sqlDigest []byte, normalizedSQL string) { diff --git a/util/topsql/tracecpu_test.go b/util/topsql/tracecpu_test.go index 8c392a3d6f7a5..6091457a9db07 100644 --- a/util/topsql/tracecpu_test.go +++ b/util/topsql/tracecpu_test.go @@ -117,10 +117,10 @@ func (s *testSuite) setTopSQLEnable(enabled bool) { func (s *testSuite) mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) - topsql.AttachSQLInfo(ctx, sql, sqlDigest.Bytes(), "", nil) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, "", nil) s.mockExecute(time.Millisecond * 100) planDigest := genDigest(plan) - topsql.AttachSQLInfo(ctx, sql, sqlDigest.Bytes(), plan, planDigest.Bytes()) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, plan, planDigest) s.mockExecute(time.Millisecond * 300) } From 612839c23acc9084a01179aa78515e82a4f8389d Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 13:36:56 +0800 Subject: [PATCH 51/54] return ctx Signed-off-by: crazycs --- executor/adapter.go | 11 ++++++----- executor/prepared.go | 2 +- server/conn_stmt.go | 2 +- session/session.go | 6 +++--- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 406a5f62e97dd..c8b0a10392c79 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -214,7 +214,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } - a.setPlanLabelForTopSQL(ctx) + ctx = a.setPlanLabelForTopSQL(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -290,15 +290,16 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return a.InfoSchema.SchemaMetaVersion(), nil } -func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) { +func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { if a.Plan == nil || !variable.TopSQLEnabled() { - return + return ctx } normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) if len(normalizedPlan) > 0 { - topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) + ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) } + return ctx } // Exec builds an Executor from a plan. If the Executor doesn't return result, @@ -371,7 +372,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. - a.setPlanLabelForTopSQL(ctx) + ctx = a.setPlanLabelForTopSQL(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) diff --git a/executor/prepared.go b/executor/prepared.go index 053351a6f011a..2a11977ef607e 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -182,7 +182,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) + ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) } if !plannercore.PreparedPlanCacheEnabled() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 297d65971efa5..88caf209db58f 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -270,7 +270,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if variable.TopSQLEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil) + ctx = topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil) } } sql := "" diff --git a/session/session.go b/session/session.go index 455dda96978ba..a889400646acf 100644 --- a/session/session.go +++ b/session/session.go @@ -1389,7 +1389,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when internal sql execute finish. - topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) + ctx = topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) } } return stmts[0], nil @@ -1504,7 +1504,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() if variable.TopSQLEnabled() { - topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) + ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) } if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { @@ -1883,7 +1883,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) + ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) From 42c8081f926391679e5bc52efa06248533ecbe74 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 13:50:04 +0800 Subject: [PATCH 52/54] address comment Signed-off-by: crazycs --- executor/adapter.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index c8b0a10392c79..f1d6a8b8e6040 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -296,10 +296,7 @@ func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { } normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) - if len(normalizedPlan) > 0 { - ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) - } - return ctx + return topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) } // Exec builds an Executor from a plan. If the Executor doesn't return result, From a9b00b992a17526ef26f4554dba3e2eba4227da7 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 13:52:08 +0800 Subject: [PATCH 53/54] Update executor/executor.go Co-authored-by: Wenxuan --- executor/executor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/executor.go b/executor/executor.go index d9cbc58179519..6d4ddb4acd035 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1667,7 +1667,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { pprof.SetGoroutineLabels(goCtx) } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) + goCtx = topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) } } // execute missed stmtID uses empty sql From 890d287f38e79c6c22e5cdd6db3e2fc4ca4ef959 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 1 Jun 2021 19:58:44 +0800 Subject: [PATCH 54/54] set label as early as posible Signed-off-by: crazycs --- server/conn_stmt.go | 7 +++++++ session/session.go | 3 --- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 88caf209db58f..df85f7ce45f52 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -130,6 +130,13 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e stmtID := binary.LittleEndian.Uint32(data[0:4]) pos += 4 + if variable.TopSQLEnabled() { + preparedStmt, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) + if preparedStmt != nil && preparedStmt.SQLDigest != nil { + ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) + } + } + stmt := cc.ctx.GetStatement(int(stmtID)) if stmt == nil { return mysql.NewErr(mysql.ErrUnknownStmtHandler, diff --git a/session/session.go b/session/session.go index a889400646acf..63ca963ca05ba 100644 --- a/session/session.go +++ b/session/session.go @@ -1882,9 +1882,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if !ok { return nil, errors.Errorf("invalid CachedPrepareStmt type") } - if variable.TopSQLEnabled() && preparedStmt.SQLDigest != nil { - ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) - } executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) ok, err = s.IsCachedExecOk(ctx, preparedStmt) if err != nil {