Skip to content

Commit

Permalink
*: record query start time to session variables (#11822) (#12676)
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored and sre-bot committed Oct 14, 2019
1 parent df41474 commit 58341c3
Show file tree
Hide file tree
Showing 6 changed files with 62 additions and 23 deletions.
16 changes: 8 additions & 8 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool {
func (a *ExecStmt) RebuildPlan() (int64, error) {
startTime := time.Now()
defer func() {
a.Ctx.GetSessionVars().StmtCtx.DurationCompile = time.Since(startTime)
a.Ctx.GetSessionVars().DurationCompile = time.Since(startTime)
}()

is := GetInfoSchema(a.Ctx)
Expand Down Expand Up @@ -383,7 +383,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.Ctx.GetSessionVars().StmtCtx.StartTime)
ctx := context.WithValue(context.Background(), plugin.ExecStartTimeCtxKey, a.Ctx.GetSessionVars().StartTime)
audit.OnGeneralEvent(ctx, sessVars, plugin.Log, cmd)
}
return nil
Expand All @@ -397,7 +397,7 @@ func (a *ExecStmt) logAudit() {
func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
level := log.GetLevel()
cfg := config.GetGlobalConfig()
costTime := time.Since(a.Ctx.GetSessionVars().StmtCtx.StartTime)
costTime := time.Since(a.Ctx.GetSessionVars().StartTime)
threshold := time.Duration(atomic.LoadUint64(&cfg.Log.SlowThreshold)) * time.Millisecond
if costTime < threshold && level > zapcore.DebugLevel {
return
Expand Down Expand Up @@ -427,8 +427,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
SQL: sql,
Digest: digest,
TimeTotal: costTime,
TimeParse: a.Ctx.GetSessionVars().StmtCtx.DurationParse,
TimeCompile: a.Ctx.GetSessionVars().StmtCtx.DurationCompile,
TimeParse: a.Ctx.GetSessionVars().DurationParse,
TimeCompile: a.Ctx.GetSessionVars().DurationCompile,
IndexNames: indexNames,
StatsInfos: statsInfos,
CopTasks: copTaskInfo,
Expand All @@ -443,8 +443,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
SQL: sql,
Digest: digest,
TimeTotal: costTime,
TimeParse: a.Ctx.GetSessionVars().StmtCtx.DurationParse,
TimeCompile: a.Ctx.GetSessionVars().StmtCtx.DurationCompile,
TimeParse: a.Ctx.GetSessionVars().DurationParse,
TimeCompile: a.Ctx.GetSessionVars().DurationCompile,
IndexNames: indexNames,
StatsInfos: statsInfos,
CopTasks: copTaskInfo,
Expand All @@ -462,7 +462,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{
SQL: sql,
Digest: digest,
Start: a.Ctx.GetSessionVars().StmtCtx.StartTime,
Start: a.Ctx.GetSessionVars().StartTime,
Duration: costTime,
Detail: sessVars.StmtCtx.GetExecDetails(),
Succ: succ,
Expand Down
37 changes: 37 additions & 0 deletions executor/adapter_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor_test

import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/util/testkit"
)

func (s *testSuite) TestQueryTime(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

costTime := time.Since(tk.Se.GetSessionVars().StartTime)
c.Assert(costTime < 1*time.Second, IsTrue)

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")
tk.MustExec("insert into t values(1), (1), (1), (1), (1)")
tk.MustExec("select * from t t1 join t t2 on t1.a = t2.a")

costTime = time.Since(tk.Se.GetSessionVars().StartTime)
c.Assert(costTime < 1*time.Second, IsTrue)
}
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,7 @@ func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error {
func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (sqlexec.Statement, error) {
startTime := time.Now()
defer func() {
ctx.GetSessionVars().StmtCtx.DurationCompile = time.Since(startTime)
ctx.GetSessionVars().DurationCompile = time.Since(startTime)
}()

execStmt := &ast.ExecuteStmt{ExecID: ID}
Expand Down
11 changes: 7 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -519,6 +519,9 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) {
for i, sr := range nh.history {
st := sr.st
s.sessionVars.StmtCtx = sr.stmtCtx
s.sessionVars.StartTime = time.Now()
s.sessionVars.DurationCompile = time.Duration(0)
s.sessionVars.DurationParse = time.Duration(0)
s.sessionVars.StmtCtx.ResetForRetry()
s.sessionVars.PreparedParams = s.sessionVars.PreparedParams[:0]
schemaVersion, err = st.RebuildPlan()
Expand Down Expand Up @@ -866,7 +869,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
s.GetSessionVars().StartTime = startTS
stmtNodes, warns, err := s.ParseSQL(ctx, sql, charsetInfo, collation)
if err != nil {
s.rollbackOnError(ctx)
Expand All @@ -876,7 +879,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec
return nil, errors.Trace(err)
}
durParse := time.Since(startTS)
s.GetSessionVars().StmtCtx.DurationParse = durParse
s.GetSessionVars().DurationParse = durParse
label := s.getSQLLabel()
metrics.SessionExecuteParseDuration.WithLabelValues(label).Observe(durParse.Seconds())

Expand All @@ -900,7 +903,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec
}

durCompile := time.Since(startTS)
s.GetSessionVars().StmtCtx.DurationCompile = durCompile
s.GetSessionVars().DurationCompile = durCompile
metrics.SessionExecuteCompileDuration.WithLabelValues(label).Observe(durCompile.Seconds())

// Step3: Execute the physical plan.
Expand Down Expand Up @@ -1015,7 +1018,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args .
}

s.PrepareTxnCtx(ctx)
s.sessionVars.StmtCtx.StartTime = time.Now()
s.sessionVars.StartTime = time.Now()
st, err := executor.CompileExecutePreparedStmt(s, stmtID, args...)
if err != nil {
return nil, errors.Trace(err)
Expand Down
10 changes: 0 additions & 10 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,13 +72,6 @@ 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
Expand Down Expand Up @@ -309,9 +302,6 @@ 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
Expand Down
9 changes: 9 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,6 +339,15 @@ type SessionVars struct {

// AllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not.
AllowRemoveAutoInc bool

// StartTime is the start time of the last query.
StartTime time.Time

// DurationParse is the duration of pasing SQL string to AST of the last query.
DurationParse time.Duration

// DurationCompile is the duration of compiling AST to execution plan of the last query.
DurationCompile time.Duration
}

// ConnectionInfo present connection used by audit.
Expand Down

0 comments on commit 58341c3

Please sign in to comment.