planner: track more details about time spent on query optimization (#65096)
close pingcap/tidb#64375
This commit is contained in:
@ -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)
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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): {
|
||||
|
||||
Reference in New Issue
Block a user