planner: track more details about time spent on query optimization (#65096)

close pingcap/tidb#64375
This commit is contained in:
Yuanjia Zhang
2025-12-29 14:13:11 +08:00
committed by GitHub
parent 01916ad835
commit 03bc2507ca
8 changed files with 63 additions and 8 deletions

View File

@ -142,6 +142,9 @@ func MatchSQLBindingForPlanCache(sctx sessionctx.Context, stmtNode ast.StmtNode,
// MatchSQLBinding returns the matched binding for this statement.
func MatchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (binding *Binding, matched bool, scope string) {
defer func(begin time.Time) {
sctx.GetSessionVars().DurationOptimizer.BindingMatch = time.Since(begin)
}(time.Now())
return matchSQLBinding(sctx, stmtNode, nil)
}

View File

@ -95,7 +95,7 @@ func TestPrepareAndCompleteSlowLogItemsForRules(t *testing.T) {
sessVars.CurrentDB = "testdb"
sessVars.DurationParse = time.Second
sessVars.DurationCompile = 2 * time.Second
sessVars.DurationOptimization = 3 * time.Second
sessVars.DurationOptimizer.Total = 3 * time.Second
sessVars.DurationWaitTS = 4 * time.Second
sessVars.StmtCtx.ExecRetryCount = 2
sessVars.StmtCtx.ExecSuccess = true

View File

@ -633,6 +633,10 @@ func getTiFlashServerMinLogicalCores(ctx context.Context, sctx base.PlanContext,
failpoint.Return(false, 0)
}
})
defer func(begin time.Time) {
// if there are any network jitters, this could take a long time.
sctx.GetSessionVars().DurationOptimizer.TiFlashInfoFetch = time.Since(begin)
}(time.Now())
rows, err := infoschema.FetchClusterServerInfoWithoutPrivilegeCheck(ctx, sctx.GetSessionVars(), serversInfo, diagnosticspb.ServerInfoType_HardwareInfo, false)
if err != nil {
return false, 0
@ -979,6 +983,10 @@ func normalizeOptimize(ctx context.Context, flag uint64, logic base.LogicalPlan)
}
func logicalOptimize(ctx context.Context, flag uint64, logic base.LogicalPlan) (base.LogicalPlan, error) {
defer func(begin time.Time) {
logic.SCtx().GetSessionVars().DurationOptimizer.LogicalOpt = time.Since(begin)
}(time.Now())
var err error
var againRuleList []base.LogicalOptRule
for i, rule := range logicalRuleList {
@ -1017,9 +1025,15 @@ func isLogicalRuleDisabled(r base.LogicalOptRule) bool {
}
func physicalOptimize(logic base.LogicalPlan) (plan base.PhysicalPlan, cost float64, err error) {
begin := time.Now()
defer func() {
logic.SCtx().GetSessionVars().DurationOptimizer.PhysicalOpt = time.Since(begin)
}()
if _, _, err := logic.RecursiveDeriveStats(nil); err != nil {
return nil, 0, err
}
// if there are too many indexes, this process might take a relatively long time, track its time cost.
logic.SCtx().GetSessionVars().DurationOptimizer.StatsDerive = time.Since(begin)
preparePossibleProperties(logic)

View File

@ -273,6 +273,10 @@ func SyncWaitStatsLoad(plan base.LogicalPlan) error {
if len(stmtCtx.StatsLoad.NeededItems) <= 0 {
return nil
}
defer func(begin time.Time) {
// track the time spent in sync wait stats load, which might take a long time.
plan.SCtx().GetSessionVars().DurationOptimizer.StatsSyncWait = time.Since(begin)
}(time.Now())
err := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx)
if err != nil {
stmtCtx.IsSyncStatsFailed = true

View File

@ -502,7 +502,7 @@ func optimize(ctx context.Context, sctx planctx.PlanContext, node *resolve.NodeW
finalPlan, cost, err := core.DoOptimize(ctx, sctx, builder.GetOptFlag(), logic)
// TODO: capture plan replayer here if it matches sql and plan digest
sessVars.DurationOptimization = time.Since(beginOpt)
sessVars.DurationOptimizer.Total = time.Since(beginOpt)
return finalPlan, names, cost, err
}

View File

@ -1257,8 +1257,16 @@ type SessionVars struct {
// RewritePhaseInfo records all information about the rewriting phase.
RewritePhaseInfo
// DurationOptimization is the duration of optimizing a query.
DurationOptimization time.Duration
// DurationOptimizer aggregates timing metrics used by the optimizer.
DurationOptimizer struct {
Total time.Duration // total time spent in query optimization
BindingMatch time.Duration // time spent matching plan bindings
StatsSyncWait time.Duration // time spent waiting for stats load to complete
LogicalOpt time.Duration // time spent in logical optimization
PhysicalOpt time.Duration // time spent in physical optimization
StatsDerive time.Duration // time spent deriving/estimating statistics
TiFlashInfoFetch time.Duration // time spent fetching TiFlash replica information
}
// DurationWaitTS is the duration of waiting for a snapshot TS
DurationWaitTS time.Duration

View File

@ -235,7 +235,7 @@ func TestSlowLogFormat(t *testing.T) {
# Parse_time: 0.00000001
# Compile_time: 0.00000001
# Rewrite_time: 0.000000003 Preproc_subqueries: 2 Preproc_subqueries_time: 0.000000002
# Optimize_time: 0.00000001
# Optimize_time: 0.00000001 Opt_logical: 0.00000001 Opt_physical: 0.00000001 Opt_binding_match: 0.00000001 Opt_stats_sync_wait: 0.00000001 Opt_stats_derive: 0.00000001
# Wait_TS: 0.000000003
# Process_time: 2 Wait_time: 60 Backoff_time: 0.001 Request_count: 2 Process_keys: 20001 Total_keys: 10000
# DB: test
@ -289,7 +289,13 @@ func TestSlowLogFormat(t *testing.T) {
ruDetails := util.NewRUDetailsWith(50.0, 100.56, 134*time.Millisecond)
seVar.DurationParse = time.Duration(10)
seVar.DurationCompile = time.Duration(10)
seVar.DurationOptimization = time.Duration(10)
seVar.DurationOptimizer.Total = time.Duration(10)
seVar.DurationOptimizer.BindingMatch = time.Duration(10)
seVar.DurationOptimizer.StatsSyncWait = time.Duration(10)
seVar.DurationOptimizer.LogicalOpt = time.Duration(10)
seVar.DurationOptimizer.PhysicalOpt = time.Duration(10)
seVar.DurationOptimizer.StatsDerive = time.Duration(10)
seVar.DurationOptimizer.TiFlashInfoFetch = time.Duration(10)
seVar.DurationWaitTS = time.Duration(3)
logItems := &variable.SlowQueryLogItems{
TxnTS: txnTS,

View File

@ -161,6 +161,17 @@ const (
SlowLogRewriteTimeStr = "Rewrite_time"
// SlowLogOptimizeTimeStr is the optimization time.
SlowLogOptimizeTimeStr = "Optimize_time"
// SlowLogOptimizeLogicalOpt is the logical optimization time.
SlowLogOptimizeLogicalOpt = "Opt_logical"
// SlowLogOptimizePhysicalOpt is the physical optimization time.
SlowLogOptimizePhysicalOpt = "Opt_physical"
// SlowLogOptimizeBindingMatch is the binding match time.
SlowLogOptimizeBindingMatch = "Opt_binding_match"
// SlowLogOptimizeStatsSyncWait is the stats sync wait time.
SlowLogOptimizeStatsSyncWait = "Opt_stats_sync_wait"
// SlowLogOptimizeStatsDerive is the stats derive time.
SlowLogOptimizeStatsDerive = "Opt_stats_derive"
// SlowLogWaitTSTimeStr is the time of waiting TS.
SlowLogWaitTSTimeStr = "Wait_TS"
// SlowLogDBStr is slow log field name.
@ -389,7 +400,16 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string {
}
buf.WriteString("\n")
writeSlowLogItem(&buf, SlowLogOptimizeTimeStr, strconv.FormatFloat(s.DurationOptimization.Seconds(), 'f', -1, 64))
// optimizer time
buf.WriteString(SlowLogRowPrefixStr)
buf.WriteString(SlowLogOptimizeTimeStr + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.Total.Seconds(), 'f', -1, 64) + " ")
buf.WriteString(SlowLogOptimizeLogicalOpt + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.LogicalOpt.Seconds(), 'f', -1, 64) + " ")
buf.WriteString(SlowLogOptimizePhysicalOpt + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.PhysicalOpt.Seconds(), 'f', -1, 64) + " ")
buf.WriteString(SlowLogOptimizeBindingMatch + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.BindingMatch.Seconds(), 'f', -1, 64) + " ")
buf.WriteString(SlowLogOptimizeStatsSyncWait + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.StatsSyncWait.Seconds(), 'f', -1, 64) + " ")
buf.WriteString(SlowLogOptimizeStatsDerive + SlowLogSpaceMarkStr + strconv.FormatFloat(s.DurationOptimizer.StatsDerive.Seconds(), 'f', -1, 64))
buf.WriteString("\n")
writeSlowLogItem(&buf, SlowLogWaitTSTimeStr, strconv.FormatFloat(s.DurationWaitTS.Seconds(), 'f', -1, 64))
if execDetailStr := logItems.ExecDetail.String(); len(execDetailStr) > 0 {
@ -704,7 +724,7 @@ var SlowLogRuleFieldAccessors = map[string]SlowLogFieldAccessor{
strings.ToLower(SlowLogOptimizeTimeStr): {
Parse: parseFloat64,
Match: func(seVars *SessionVars, _ *SlowQueryLogItems, threshold any) bool {
return matchGE(threshold, seVars.DurationOptimization.Seconds())
return matchGE(threshold, seVars.DurationOptimizer.Total.Seconds())
},
},
strings.ToLower(SlowLogWaitTSTimeStr): {