diff --git a/pkg/bindinfo/binding.go b/pkg/bindinfo/binding.go index 8d67fbfb2d..97ebb70a75 100644 --- a/pkg/bindinfo/binding.go +++ b/pkg/bindinfo/binding.go @@ -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) } diff --git a/pkg/executor/adapter_test.go b/pkg/executor/adapter_test.go index 8442a7de18..9783d76790 100644 --- a/pkg/executor/adapter_test.go +++ b/pkg/executor/adapter_test.go @@ -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 diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 619c581768..97a98e953f 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -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) diff --git a/pkg/planner/core/rule/rule_collect_plan_stats.go b/pkg/planner/core/rule/rule_collect_plan_stats.go index a3ea1e1836..70f129a13a 100644 --- a/pkg/planner/core/rule/rule_collect_plan_stats.go +++ b/pkg/planner/core/rule/rule_collect_plan_stats.go @@ -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 diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 85d6575b4b..405efe1b52 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -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 } diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index b3a85a4a41..bbfb3b867c 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -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 diff --git a/pkg/sessionctx/variable/session_test.go b/pkg/sessionctx/variable/session_test.go index 004f086427..118e462d2a 100644 --- a/pkg/sessionctx/variable/session_test.go +++ b/pkg/sessionctx/variable/session_test.go @@ -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, diff --git a/pkg/sessionctx/variable/slow_log.go b/pkg/sessionctx/variable/slow_log.go index faff820959..5cc68cedb1 100644 --- a/pkg/sessionctx/variable/slow_log.go +++ b/pkg/sessionctx/variable/slow_log.go @@ -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): {