Skip to content

Commit

Permalink
topsql, pprof: use hex string but not binary sql/plan digest in gorou…
Browse files Browse the repository at this point in the history
…tine label (#52216) (#52323)

close #52215
  • Loading branch information
ti-chi-bot committed Apr 3, 2024
1 parent 21ea4ee commit 9898d4c
Show file tree
Hide file tree
Showing 4 changed files with 33 additions and 19 deletions.
4 changes: 3 additions & 1 deletion util/cpuprofile/testutil/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package testutil

import (
"context"
"encoding/hex"
"runtime/pprof"
)

Expand All @@ -24,7 +25,8 @@ func MockCPULoad(ctx context.Context, labels ...string) {
lvs := []string{}
for _, label := range labels {
lvs = append(lvs, label)
lvs = append(lvs, label+" value")
val := hex.EncodeToString([]byte(label + " value"))
lvs = append(lvs, val)
// start goroutine with only 1 label.
go mockCPULoadByGoroutineWithLabel(ctx, label, label+" value")
}
Expand Down
1 change: 0 additions & 1 deletion util/topsql/collector/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ go_library(
deps = [
"//util",
"//util/cpuprofile",
"//util/hack",
"//util/logutil",
"//util/topsql/state",
"@com_github_google_pprof//profile",
Expand Down
33 changes: 23 additions & 10 deletions util/topsql/collector/cpu.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,14 @@ package collector

import (
"context"
"encoding/hex"
"runtime/pprof"
"sync"
"time"

"github.com/google/pprof/profile"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/cpuprofile"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
topsqlstate "github.com/pingcap/tidb/util/topsql/state"
"go.uber.org/zap"
Expand Down Expand Up @@ -193,12 +193,25 @@ func (sp *SQLCPUCollector) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLC

func (*SQLCPUCollector) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPUTimeRecord {
stats := make([]SQLCPUTimeRecord, 0, len(sqlMap))
for sqlDigest, stmt := range sqlMap {
for hexSQLDigest, stmt := range sqlMap {
stmt.tune()
for planDigest, val := range stmt.plans {

sqlDigest, err := hex.DecodeString(hexSQLDigest)
if err != nil {
logutil.BgLogger().Error("decode sql digest failed", zap.String("sqlDigest", hexSQLDigest), zap.Error(err))
continue
}

for hexPlanDigest, val := range stmt.plans {
planDigest, err := hex.DecodeString(hexPlanDigest)
if err != nil {
logutil.BgLogger().Error("decode plan digest failed", zap.String("planDigest", hexPlanDigest), zap.Error(err))
continue
}

stats = append(stats, SQLCPUTimeRecord{
SQLDigest: []byte(sqlDigest),
PlanDigest: []byte(planDigest),
SQLDigest: sqlDigest,
PlanDigest: planDigest,
CPUTimeMs: uint32(time.Duration(val).Milliseconds()),
})
}
Expand Down Expand Up @@ -255,12 +268,12 @@ func (s *sqlStats) tune() {
}

// CtxWithSQLDigest wrap the ctx with sql digest.
func CtxWithSQLDigest(ctx context.Context, sqlDigest []byte) context.Context {
return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest))))
func CtxWithSQLDigest(ctx context.Context, sqlDigest string) context.Context {
return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest))
}

// CtxWithSQLAndPlanDigest wrap the ctx with sql digest and plan digest.
func CtxWithSQLAndPlanDigest(ctx context.Context, sqlDigest, planDigest []byte) context.Context {
return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest)),
labelPlanDigest, string(hack.String(planDigest))))
func CtxWithSQLAndPlanDigest(ctx context.Context, sqlDigest, planDigest string) context.Context {
return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, sqlDigest,
labelPlanDigest, planDigest))
}
14 changes: 7 additions & 7 deletions util/topsql/topsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,11 +97,11 @@ func RegisterPlan(normalizedPlan string, planDigest *parser.Digest) {

// AttachAndRegisterSQLInfo attach the sql information into Top SQL and register the SQL meta information.
func AttachAndRegisterSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest *parser.Digest, isInternal bool) context.Context {
if sqlDigest == nil || len(sqlDigest.Bytes()) == 0 {
if sqlDigest == nil || len(sqlDigest.String()) == 0 {
return ctx
}
sqlDigestBytes := sqlDigest.Bytes()
ctx = collector.CtxWithSQLDigest(ctx, sqlDigestBytes)
ctx = collector.CtxWithSQLDigest(ctx, sqlDigest.String())
pprof.SetGoroutineLabels(ctx)

linkSQLTextWithDigest(sqlDigestBytes, normalizedSQL, isInternal)
Expand All @@ -124,15 +124,15 @@ func AttachAndRegisterSQLInfo(ctx context.Context, normalizedSQL string, sqlDige

// AttachSQLAndPlanInfo attach the sql and plan information into Top SQL
func AttachSQLAndPlanInfo(ctx context.Context, sqlDigest *parser.Digest, planDigest *parser.Digest) context.Context {
if sqlDigest == nil || len(sqlDigest.Bytes()) == 0 {
if sqlDigest == nil || len(sqlDigest.String()) == 0 {
return ctx
}
var planDigestBytes []byte
sqlDigestBytes := sqlDigest.Bytes()
var planDigestStr string
sqlDigestStr := sqlDigest.String()
if planDigest != nil {
planDigestBytes = planDigest.Bytes()
planDigestStr = planDigest.String()
}
ctx = collector.CtxWithSQLAndPlanDigest(ctx, sqlDigestBytes, planDigestBytes)
ctx = collector.CtxWithSQLAndPlanDigest(ctx, sqlDigestStr, planDigestStr)
pprof.SetGoroutineLabels(ctx)

failpoint.Inject("mockHighLoadForEachPlan", func(val failpoint.Value) {
Expand Down

0 comments on commit 9898d4c

Please sign in to comment.