From 58341c34de26d36c6aa42fba7e043869427bbfe1 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 14 Oct 2019 15:48:03 +0800 Subject: [PATCH] *: record query start time to session variables (#11822) (#12676) --- executor/adapter.go | 16 +++++++-------- executor/adapter_test.go | 37 ++++++++++++++++++++++++++++++++++ executor/prepared.go | 2 +- session/session.go | 11 ++++++---- sessionctx/stmtctx/stmtctx.go | 10 --------- sessionctx/variable/session.go | 9 +++++++++ 6 files changed, 62 insertions(+), 23 deletions(-) create mode 100644 executor/adapter_test.go diff --git a/executor/adapter.go b/executor/adapter.go index 569076aa8a26a..5770809444aaf 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -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) @@ -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 @@ -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 @@ -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, @@ -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, @@ -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, diff --git a/executor/adapter_test.go b/executor/adapter_test.go new file mode 100644 index 0000000000000..32f288e28d8f4 --- /dev/null +++ b/executor/adapter_test.go @@ -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) +} diff --git a/executor/prepared.go b/executor/prepared.go index e5951899e1ae1..15c5fadde3f4b 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -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} diff --git a/session/session.go b/session/session.go index f0597d318c584..3c2be3cd66cdb 100644 --- a/session/session.go +++ b/session/session.go @@ -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() @@ -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) @@ -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()) @@ -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. @@ -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) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 36e96e5f7f564..9d742aef26eae 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -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 @@ -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 diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 174f7cb3acfed..eeb2a93c49dc8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -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.