Skip to content

Commit

Permalink
server: add metrics for duration of each session execute stage (#1751)
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Sep 23, 2016
1 parent 09906f7 commit 50fa99a
Show file tree
Hide file tree
Showing 4 changed files with 60 additions and 2 deletions.
2 changes: 1 addition & 1 deletion executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ type Compiler struct {
// If it is supported to use new plan and executer, it optimizes the node to
// a plan, and we wrap the plan in an adapter as stmt.Statement.
// If it is not supported, the node will be converted to old statement.
func (c *Compiler) Compile(ctx context.Context, node ast.StmtNode) (ast.Statement, error) {
func (c Compiler) Compile(ctx context.Context, node ast.StmtNode) (ast.Statement, error) {
ast.SetFlag(node)
if _, ok := node.(*ast.UpdateStmt); ok {
sVars := variable.GetSessionVars(ctx)
Expand Down
51 changes: 51 additions & 0 deletions metrics.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
// Copyright 2016 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 tidb

import (
"github.com/prometheus/client_golang/prometheus"
)

var (
sessionExecuteParseDuration = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "session_execute_parse_duration",
Help: "Bucketed histogram of processing time (s) in parse SQL.",
Buckets: prometheus.LinearBuckets(0.00004, 0.00001, 13),
})
sessionExecuteCompileDuration = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "session_execute_compile_duration",
Help: "Bucketed histogram of processing time (s) in query optimize.",
Buckets: prometheus.LinearBuckets(0.00004, 0.00001, 13),
})
sessionExecuteRunDuration = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "session_execute_run_duration",
Help: "Bucketed histogram of processing time (s) in running executor.",
Buckets: prometheus.ExponentialBuckets(0.0001, 2, 13),
})
)

func init() {
prometheus.MustRegister(sessionExecuteParseDuration)
prometheus.MustRegister(sessionExecuteCompileDuration)
prometheus.MustRegister(sessionExecuteRunDuration)
}
7 changes: 7 additions & 0 deletions session.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,30 +445,37 @@ func (s *session) Execute(sql string) ([]ast.RecordSet, error) {
if err := s.checkSchemaValidOrRollback(); err != nil {
return nil, errors.Trace(err)
}
startTS := time.Now()
charset, collation := getCtxCharsetInfo(s)
rawStmts, err := s.ParseSQL(sql, charset, collation)
if err != nil {
log.Warnf("compiling %s, error: %v", sql, err)
return nil, errors.Trace(err)
}
sessionExecuteParseDuration.Observe(time.Since(startTS).Seconds())

var rs []ast.RecordSet
ph := sessionctx.GetDomain(s).PerfSchema()
for i, rst := range rawStmts {
startTS := time.Now()
st, err1 := Compile(s, rst)
if err1 != nil {
log.Errorf("Syntax error: %s", sql)
log.Errorf("Error occurs at %s.", err1)
return nil, errors.Trace(err1)
}
sessionExecuteCompileDuration.Observe(time.Since(startTS).Seconds())
id := variable.GetSessionVars(s).ConnectionID
s.stmtState = ph.StartStatement(sql, id, perfschema.CallerNameSessionExecute, rawStmts[i])

startTS = time.Now()
r, err := runStmt(s, st)
ph.EndStatement(s.stmtState)
if err != nil {
log.Warnf("session:%v, err:%v", s, err)
return nil, errors.Trace(err)
}
sessionExecuteRunDuration.Observe(time.Since(startTS).Seconds())
if r != nil {
rs = append(rs, r)
}
Expand Down
2 changes: 1 addition & 1 deletion tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ func Parse(ctx context.Context, src string) ([]ast.StmtNode, error) {

// Compile is safe for concurrent use by multiple goroutines.
func Compile(ctx context.Context, rawStmt ast.StmtNode) (ast.Statement, error) {
compiler := &executor.Compiler{}
compiler := executor.Compiler{}
st, err := compiler.Compile(ctx, rawStmt)
if err != nil {
return nil, errors.Trace(err)
Expand Down

0 comments on commit 50fa99a

Please sign in to comment.