diff --git a/executor/adapter.go b/executor/adapter.go index 9acfc94441bc3..2908dccf0d37f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -151,9 +151,7 @@ type ExecStmt struct { StmtNode ast.StmtNode - Ctx sessionctx.Context - // StartTime stands for the starting time when executing the statement. - StartTime time.Time + Ctx sessionctx.Context isPreparedStmt bool isSelectForUpdate bool retryCount uint @@ -187,6 +185,11 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { + startTime := time.Now() + defer func() { + a.Ctx.GetSessionVars().StmtCtx.DurationCompile = time.Since(startTime) + }() + is := GetInfoSchema(a.Ctx) a.InfoSchema = is if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { @@ -204,7 +207,6 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { // 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. func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { - a.StartTime = time.Now() sctx := a.Ctx if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { oriStats, _ := sctx.GetSessionVars().GetSystemVar(variable.TiDBBuildStatsConcurrency) @@ -596,7 +598,7 @@ func (a *ExecStmt) logAudit() { audit := plugin.DeclareAuditManifest(p.Manifest) if audit.OnGeneralEvent != nil { cmd := mysql.Command2Str[byte(atomic.LoadUint32(&a.Ctx.GetSessionVars().CommandValue))] - ctx := context.WithValue(context.Background(), plugin.ExecStartTimeCtxKey, a.StartTime) + ctx := context.WithValue(context.Background(), plugin.ExecStartTimeCtxKey, a.Ctx.GetSessionVars().StmtCtx.StartTime) audit.OnGeneralEvent(ctx, sessVars, plugin.Log, cmd) } return nil @@ -614,7 +616,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { return } cfg := config.GetGlobalConfig() - costTime := time.Since(a.StartTime) + costTime := time.Since(a.Ctx.GetSessionVars().StmtCtx.StartTime) threshold := time.Duration(atomic.LoadUint64(&cfg.Log.SlowThreshold)) * time.Millisecond if costTime < threshold && level > zapcore.DebugLevel { return @@ -638,10 +640,36 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() if costTime < threshold { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, digest, statsInfos, copTaskInfo, memMax, succ, sql)) + logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(&variable.SlowQueryLogItems{ + TxnTS: txnTS, + SQL: sql, + Digest: digest, + TimeTotal: costTime, + TimeParse: a.Ctx.GetSessionVars().StmtCtx.DurationParse, + TimeCompile: a.Ctx.GetSessionVars().StmtCtx.DurationCompile, + IndexNames: indexNames, + StatsInfos: statsInfos, + CopTasks: copTaskInfo, + ExecDetail: execDetail, + MemMax: memMax, + Succ: succ, + })) } else { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, digest, statsInfos, copTaskInfo, memMax, succ, sql)) + logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(&variable.SlowQueryLogItems{ + TxnTS: txnTS, + SQL: sql, + Digest: digest, + TimeTotal: costTime, + TimeParse: a.Ctx.GetSessionVars().StmtCtx.DurationParse, + TimeCompile: a.Ctx.GetSessionVars().StmtCtx.DurationCompile, + IndexNames: indexNames, + StatsInfos: statsInfos, + CopTasks: copTaskInfo, + ExecDetail: execDetail, + MemMax: memMax, + Succ: succ, + })) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -652,7 +680,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{ SQL: sql, Digest: digest, - Start: a.StartTime, + Start: a.Ctx.GetSessionVars().StmtCtx.StartTime, Duration: costTime, Detail: sessVars.StmtCtx.GetExecDetails(), Succ: succ, diff --git a/executor/prepared.go b/executor/prepared.go index 2171845d86d3f..854e3b02a6d03 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -17,6 +17,7 @@ import ( "context" "math" "sort" + "time" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -266,6 +267,10 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { // CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement. func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, ID uint32, args ...interface{}) (sqlexec.Statement, error) { + startTime := time.Now() + defer func() { + sctx.GetSessionVars().StmtCtx.DurationCompile = time.Since(startTime) + }() execStmt := &ast.ExecuteStmt{ExecID: ID} if err := ResetContextOfStmt(sctx, execStmt); err != nil { return nil, err diff --git a/session/session.go b/session/session.go index 14c2d1e3fd3dd..c6c9852817869 100644 --- a/session/session.go +++ b/session/session.go @@ -1009,6 +1009,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec // Step1: Compile query string to abstract syntax trees(ASTs). startTS := time.Now() + s.GetSessionVars().StmtCtx.StartTime = startTS stmtNodes, warns, err := s.ParseSQL(ctx, sql, charsetInfo, collation) if err != nil { s.rollbackOnError(ctx) @@ -1017,11 +1018,13 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec zap.String("sql", sql)) return nil, util.SyntaxError(err) } + durParse := time.Since(startTS) + s.GetSessionVars().StmtCtx.DurationParse = durParse isInternal := s.isInternal() if isInternal { - sessionExecuteParseDurationInternal.Observe(time.Since(startTS).Seconds()) + sessionExecuteParseDurationInternal.Observe(durParse.Seconds()) } else { - sessionExecuteParseDurationGeneral.Observe(time.Since(startTS).Seconds()) + sessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) } var tempStmtNodes []ast.StmtNode @@ -1055,10 +1058,12 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec } s.handleInvalidBindRecord(ctx, stmtNode) } + durCompile := time.Since(startTS) + s.GetSessionVars().StmtCtx.DurationCompile = durCompile if isInternal { - sessionExecuteCompileDurationInternal.Observe(time.Since(startTS).Seconds()) + sessionExecuteCompileDurationInternal.Observe(durCompile.Seconds()) } else { - sessionExecuteCompileDurationGeneral.Observe(time.Since(startTS).Seconds()) + sessionExecuteCompileDurationGeneral.Observe(durCompile.Seconds()) } s.currentPlan = stmt.Plan @@ -1222,6 +1227,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args . } s.PrepareTxnCtx(ctx) + s.sessionVars.StmtCtx.StartTime = time.Now() st, err := executor.CompileExecutePreparedStmt(ctx, s, stmtID, args...) if err != nil { return nil, err diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 2812ffea60bd7..680484ae0e24e 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -75,6 +75,13 @@ type StatementContext struct { // prefix in a strict way, only extract 0-9 and (+ or - in first bit). CastStrToIntStrict bool + // StartTime is the query start time. + StartTime time.Time + // DurationParse is the duration of pasing SQL string to AST. + DurationParse time.Duration + // DurationCompile is the duration of compiling AST to execution plan. + DurationCompile time.Duration + // mu struct holds variables that change during execution. mu struct { sync.Mutex @@ -420,6 +427,9 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.Unlock() sc.TableIDs = sc.TableIDs[:0] sc.IndexNames = sc.IndexNames[:0] + sc.StartTime = time.Now() + sc.DurationCompile = time.Duration(0) + sc.DurationParse = time.Duration(0) } // MergeExecDetails merges a single region execution details into self, used to print diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 6e70ca85dbee1..aa83a0c39954d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1017,6 +1017,23 @@ const ( SlowLogSucc = "Succ" ) +// SlowQueryLogItems is a collection of items that should be included in the +// slow query log. +type SlowQueryLogItems struct { + TxnTS uint64 + SQL string + Digest string + TimeTotal time.Duration + TimeParse time.Duration + TimeCompile time.Duration + IndexNames string + StatsInfos map[string]uint64 + CopTasks *stmtctx.CopTasksDetails + ExecDetail execdetails.ExecDetails + MemMax int64 + Succ bool +} + // SlowLogFormat uses for formatting slow log. // The slow log output is like below: // # Time: 2019-04-28T15:24:04.309074+08:00 @@ -1036,36 +1053,38 @@ const ( // # Memory_max: 4096 // # Succ: true // select * from t_slim; -func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexNames string, digest string, - statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, memMax int64, succ bool, sql string) string { +func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { var buf bytes.Buffer - execDetailStr := execDetail.String() - buf.WriteString(SlowLogRowPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") + + writeSlowLogItem(&buf, SlowLogTxnStartTSStr, strconv.FormatUint(logItems.TxnTS, 10)) if s.User != nil { - buf.WriteString(SlowLogRowPrefixStr + SlowLogUserStr + SlowLogSpaceMarkStr + s.User.String() + "\n") + writeSlowLogItem(&buf, SlowLogUserStr, s.User.String()) } if s.ConnectionID != 0 { - buf.WriteString(SlowLogRowPrefixStr + SlowLogConnIDStr + SlowLogSpaceMarkStr + strconv.FormatUint(s.ConnectionID, 10) + "\n") + writeSlowLogItem(&buf, SlowLogConnIDStr, strconv.FormatUint(s.ConnectionID, 10)) } - buf.WriteString(SlowLogRowPrefixStr + SlowLogQueryTimeStr + SlowLogSpaceMarkStr + strconv.FormatFloat(costTime.Seconds(), 'f', -1, 64) + "\n") - if len(execDetailStr) > 0 { + writeSlowLogItem(&buf, SlowLogQueryTimeStr, strconv.FormatFloat(logItems.TimeTotal.Seconds(), 'f', -1, 64)) + + if execDetailStr := logItems.ExecDetail.String(); len(execDetailStr) > 0 { buf.WriteString(SlowLogRowPrefixStr + execDetailStr + "\n") } + if len(s.CurrentDB) > 0 { - buf.WriteString(SlowLogRowPrefixStr + SlowLogDBStr + SlowLogSpaceMarkStr + s.CurrentDB + "\n") + writeSlowLogItem(&buf, SlowLogDBStr, s.CurrentDB) } - if len(indexNames) > 0 { - buf.WriteString(SlowLogRowPrefixStr + SlowLogIndexNamesStr + SlowLogSpaceMarkStr + indexNames + "\n") + if len(logItems.IndexNames) > 0 { + writeSlowLogItem(&buf, SlowLogIndexNamesStr, logItems.IndexNames) } - buf.WriteString(SlowLogRowPrefixStr + SlowLogIsInternalStr + SlowLogSpaceMarkStr + strconv.FormatBool(s.InRestrictedSQL) + "\n") - if len(digest) > 0 { - buf.WriteString(SlowLogRowPrefixStr + SlowLogDigestStr + SlowLogSpaceMarkStr + digest + "\n") + + writeSlowLogItem(&buf, SlowLogIsInternalStr, strconv.FormatBool(s.InRestrictedSQL)) + if len(logItems.Digest) > 0 { + writeSlowLogItem(&buf, SlowLogDigestStr, logItems.Digest) } - if len(statsInfos) > 0 { + if len(logItems.StatsInfos) > 0 { buf.WriteString(SlowLogRowPrefixStr + SlowLogStatsInfoStr + SlowLogSpaceMarkStr) firstComma := false vStr := "" - for k, v := range statsInfos { + for k, v := range logItems.StatsInfos { if v == 0 { vStr = "pseudo" } else { @@ -1081,29 +1100,33 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe } buf.WriteString("\n") } - if copTasks != nil { - buf.WriteString(SlowLogRowPrefixStr + SlowLogNumCopTasksStr + SlowLogSpaceMarkStr + strconv.FormatInt(int64(copTasks.NumCopTasks), 10) + "\n") + if logItems.CopTasks != nil { + writeSlowLogItem(&buf, SlowLogNumCopTasksStr, strconv.FormatInt(int64(logItems.CopTasks.NumCopTasks), 10)) buf.WriteString(SlowLogRowPrefixStr + fmt.Sprintf("%v%v%v %v%v%v %v%v%v %v%v%v", - SlowLogCopProcAvg, SlowLogSpaceMarkStr, copTasks.AvgProcessTime.Seconds(), - SlowLogCopProcP90, SlowLogSpaceMarkStr, copTasks.P90ProcessTime.Seconds(), - SlowLogCopProcMax, SlowLogSpaceMarkStr, copTasks.MaxProcessTime.Seconds(), - SlowLogCopProcAddr, SlowLogSpaceMarkStr, copTasks.MaxProcessAddress) + "\n") + SlowLogCopProcAvg, SlowLogSpaceMarkStr, logItems.CopTasks.AvgProcessTime.Seconds(), + SlowLogCopProcP90, SlowLogSpaceMarkStr, logItems.CopTasks.P90ProcessTime.Seconds(), + SlowLogCopProcMax, SlowLogSpaceMarkStr, logItems.CopTasks.MaxProcessTime.Seconds(), + SlowLogCopProcAddr, SlowLogSpaceMarkStr, logItems.CopTasks.MaxProcessAddress) + "\n") buf.WriteString(SlowLogRowPrefixStr + fmt.Sprintf("%v%v%v %v%v%v %v%v%v %v%v%v", - SlowLogCopWaitAvg, SlowLogSpaceMarkStr, copTasks.AvgWaitTime.Seconds(), - SlowLogCopWaitP90, SlowLogSpaceMarkStr, copTasks.P90WaitTime.Seconds(), - SlowLogCopWaitMax, SlowLogSpaceMarkStr, copTasks.MaxWaitTime.Seconds(), - SlowLogCopWaitAddr, SlowLogSpaceMarkStr, copTasks.MaxWaitAddress) + "\n") + SlowLogCopWaitAvg, SlowLogSpaceMarkStr, logItems.CopTasks.AvgWaitTime.Seconds(), + SlowLogCopWaitP90, SlowLogSpaceMarkStr, logItems.CopTasks.P90WaitTime.Seconds(), + SlowLogCopWaitMax, SlowLogSpaceMarkStr, logItems.CopTasks.MaxWaitTime.Seconds(), + SlowLogCopWaitAddr, SlowLogSpaceMarkStr, logItems.CopTasks.MaxWaitAddress) + "\n") } - if memMax > 0 { - buf.WriteString(SlowLogRowPrefixStr + SlowLogMemMax + SlowLogSpaceMarkStr + strconv.FormatInt(memMax, 10) + "\n") + if logItems.MemMax > 0 { + writeSlowLogItem(&buf, SlowLogMemMax, strconv.FormatInt(logItems.MemMax, 10)) } - buf.WriteString(SlowLogRowPrefixStr + SlowLogSucc + SlowLogSpaceMarkStr + strconv.FormatBool(succ) + "\n") - if len(sql) == 0 { - sql = ";" - } - buf.WriteString(sql) - if sql[len(sql)-1] != ';' { + + writeSlowLogItem(&buf, SlowLogSucc, strconv.FormatBool(logItems.Succ)) + + buf.WriteString(logItems.SQL) + if len(logItems.SQL) == 0 || logItems.SQL[len(logItems.SQL)-1] != ';' { buf.WriteString(";") } return buf.String() } + +// writeSlowLogItem writes a slow log item in the form of: "# ${key}:${value}" +func writeSlowLogItem(buf *bytes.Buffer, key, value string) { + buf.WriteString(SlowLogRowPrefixStr + key + SlowLogSpaceMarkStr + value + "\n") +} diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 6a134f95b1361..3359115c281fc 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" ) @@ -137,6 +138,19 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { select * from t;` sql := "select * from t" digest := parser.DigestHash(sql) - logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[t1:a,t2:b]", digest, statsInfos, copTasks, memMax, true, sql) + logString := seVar.SlowLogFormat(&variable.SlowQueryLogItems{ + TxnTS: txnTS, + SQL: sql, + Digest: digest, + TimeTotal: costTime, + TimeParse: time.Duration(10), + TimeCompile: time.Duration(10), + IndexNames: "[t1:a,t2:b]", + StatsInfos: statsInfos, + CopTasks: copTasks, + ExecDetail: execDetail, + MemMax: memMax, + Succ: true, + }) c.Assert(logString, Equals, resultString) }