Skip to content

Commit

Permalink
metrics: add different labels for restricted SQL and general SQL (#7631)
Browse files Browse the repository at this point in the history
The newly added label is `sql_type`, it has value: `general` and `internal`,
users can use this label to filter metrics they have interest.

Also, add keyword `[INTERNAL]` for restricted SQL printed in slow log for
convenient identification.
  • Loading branch information
eurekaka committed Sep 7, 2018
1 parent 3f5bd3b commit 8c44f56
Show file tree
Hide file tree
Showing 10 changed files with 56 additions and 23 deletions.
15 changes: 14 additions & 1 deletion distsql/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package distsql
import (
"github.com/juju/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -54,6 +55,10 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie
}, nil
}

label := metrics.LblGeneral
if sctx.GetSessionVars().InRestrictedSQL {
label = metrics.LblInternal
}
return &selectResult{
label: "dag",
resp: resp,
Expand All @@ -63,21 +68,28 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie
fieldTypes: fieldTypes,
ctx: sctx,
feedback: fb,
sqlType: label,
}, nil
}

// Analyze do a analyze request.
func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables) (SelectResult, error) {
func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables,
isRestrict bool) (SelectResult, error) {
resp := client.Send(ctx, kvReq, vars)
if resp == nil {
return nil, errors.New("client returns nil response")
}
label := metrics.LblGeneral
if isRestrict {
label = metrics.LblInternal
}
result := &selectResult{
label: "analyze",
resp: resp,
results: make(chan resultWithErr, kvReq.Concurrency),
closed: make(chan struct{}),
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
sqlType: label,
}
return result, nil
}
Expand All @@ -94,6 +106,7 @@ func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv
results: make(chan resultWithErr, kvReq.Concurrency),
closed: make(chan struct{}),
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
sqlType: metrics.LblGeneral,
}
return result, nil
}
4 changes: 3 additions & 1 deletion distsql/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ func (s *testSuite) TestSelectNormal(c *C) {
result, ok := response.(*selectResult)
c.Assert(ok, IsTrue)
c.Assert(result.label, Equals, "dag")
c.Assert(result.sqlType, Equals, "general")
c.Assert(result.rowLen, Equals, len(colTypes))

response.Fetch(context.TODO())
Expand Down Expand Up @@ -143,12 +144,13 @@ func (s *testSuite) TestAnalyze(c *C) {
Build()
c.Assert(err, IsNil)

response, err := Analyze(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars)
response, err := Analyze(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars, true)
c.Assert(err, IsNil)

result, ok := response.(*selectResult)
c.Assert(ok, IsTrue)
c.Assert(result.label, Equals, "analyze")
c.Assert(result.sqlType, Equals, "internal")

response.Fetch(context.TODO())

Expand Down
3 changes: 2 additions & 1 deletion distsql/select_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ type selectResult struct {

feedback *statistics.QueryFeedback
partialCount int64 // number of partial results.
sqlType string
}

func (r *selectResult) Fetch(ctx context.Context) {
Expand All @@ -78,7 +79,7 @@ func (r *selectResult) fetch(ctx context.Context) {
defer func() {
close(r.results)
duration := time.Since(startTime)
metrics.DistSQLQueryHistgram.WithLabelValues(r.label).Observe(duration.Seconds())
metrics.DistSQLQueryHistgram.WithLabelValues(r.label, r.sqlType).Observe(duration.Seconds())
}()
for {
resultSubset, err := r.resp.Next(ctx)
Expand Down
14 changes: 9 additions & 5 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,15 +354,19 @@ func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) {
if len(sessVars.StmtCtx.IndexIDs) > 0 {
indexIDs = strings.Replace(fmt.Sprintf("index_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1)
}
user := a.Ctx.GetSessionVars().User
user := sessVars.User
var internal string
if sessVars.InRestrictedSQL {
internal = "[INTERNAL] "
}
if costTime < threshold {
logutil.SlowQueryLogger.Debugf(
"[QUERY] cost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v",
costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql)
"[QUERY] %vcost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v",
internal, costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql)
} else {
logutil.SlowQueryLogger.Warnf(
"[SLOW_QUERY] cost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v",
costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql)
"[SLOW_QUERY] %vcost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v",
internal, costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql)
}
}

Expand Down
4 changes: 2 additions & 2 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ func (e *AnalyzeIndexExec) open() error {
SetConcurrency(e.concurrency).
Build()
ctx := context.TODO()
e.result, err = distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars)
e.result, err = distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -295,7 +295,7 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe
return nil, errors.Trace(err)
}
ctx := context.TODO()
result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars)
result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion metrics/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ var (
Name: "handle_query_duration_seconds",
Help: "Bucketed histogram of processing time (s) of handled queries.",
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13),
}, []string{LblType})
}, []string{LblType, LblSQLType})

DistSQLScanKeysPartialHistogram = prometheus.NewHistogram(
prometheus.HistogramOpts{
Expand Down
4 changes: 2 additions & 2 deletions metrics/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,14 @@ import (

// Metrics
var (
QueryDurationHistogram = prometheus.NewHistogram(
QueryDurationHistogram = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "handle_query_duration_seconds",
Help: "Bucketed histogram of processing time (s) of handled queries.",
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22),
})
}, []string{LblSQLType})

QueryTotalCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Expand Down
15 changes: 9 additions & 6 deletions metrics/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,30 +17,30 @@ import "github.com/prometheus/client_golang/prometheus"

// Session metrics.
var (
SessionExecuteParseDuration = prometheus.NewHistogram(
SessionExecuteParseDuration = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "session",
Name: "parse_duration_seconds",
Help: "Bucketed histogram of processing time (s) in parse SQL.",
Buckets: prometheus.LinearBuckets(0.00004, 0.00001, 13),
})
SessionExecuteCompileDuration = prometheus.NewHistogram(
}, []string{LblSQLType})
SessionExecuteCompileDuration = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "session",
Name: "compile_duration_seconds",
Help: "Bucketed histogram of processing time (s) in query optimize.",
Buckets: prometheus.LinearBuckets(0.00004, 0.00001, 13),
})
SessionExecuteRunDuration = prometheus.NewHistogram(
}, []string{LblSQLType})
SessionExecuteRunDuration = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: "tidb",
Subsystem: "session",
Name: "execute_duration_seconds",
Help: "Bucketed histogram of processing time (s) in running executor.",
Buckets: prometheus.ExponentialBuckets(0.0001, 2, 13),
})
}, []string{LblSQLType})
SchemaLeaseErrorCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Expand Down Expand Up @@ -107,6 +107,9 @@ const (
LblRollback = "rollback"
LblType = "type"
LblResult = "result"
LblSQLType = "sql_type"
LblGeneral = "general"
LblInternal = "internal"
)

func init() {
Expand Down
2 changes: 1 addition & 1 deletion server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -569,7 +569,7 @@ func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) {
} else {
metrics.QueryTotalCounter.WithLabelValues(label, "OK").Inc()
}
metrics.QueryDurationHistogram.Observe(time.Since(startTime).Seconds())
metrics.QueryDurationHistogram.WithLabelValues(metrics.LblGeneral).Observe(time.Since(startTime).Seconds())
}

// dispatch handles client request based on command which is the first byte of the data.
Expand Down
16 changes: 13 additions & 3 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -542,6 +542,7 @@ func (s *session) ExecRestrictedSQL(sctx sessionctx.Context, sql string) ([]chun
defer s.sysSessionPool().Put(tmp)
metrics.SessionRestrictedSQLCounter.Inc()

startTime := time.Now()
recordSets, err := se.Execute(ctx, sql)
if err != nil {
return nil, nil, errors.Trace(err)
Expand All @@ -566,6 +567,7 @@ func (s *session) ExecRestrictedSQL(sctx sessionctx.Context, sql string) ([]chun
fields = rs.Fields()
}
}
metrics.QueryDurationHistogram.WithLabelValues(metrics.LblInternal).Observe(time.Since(startTime).Seconds())
return rows, fields, nil
}

Expand Down Expand Up @@ -732,7 +734,11 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode
}
return nil, errors.Trace(err)
}
metrics.SessionExecuteRunDuration.Observe(time.Since(startTime).Seconds())
label := metrics.LblGeneral
if s.sessionVars.InRestrictedSQL {
label = metrics.LblInternal
}
metrics.SessionExecuteRunDuration.WithLabelValues(label).Observe(time.Since(startTime).Seconds())

if recordSet != nil {
recordSets = append(recordSets, recordSet)
Expand Down Expand Up @@ -766,7 +772,11 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.Rec
log.Warnf("con:%d parse error:\n%v\n%s", connID, err, sql)
return nil, errors.Trace(err)
}
metrics.SessionExecuteParseDuration.Observe(time.Since(startTS).Seconds())
label := metrics.LblGeneral
if s.sessionVars.InRestrictedSQL {
label = metrics.LblInternal
}
metrics.SessionExecuteParseDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds())

compiler := executor.Compiler{Ctx: s}
for _, stmtNode := range stmtNodes {
Expand All @@ -784,7 +794,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.Rec
log.Warnf("con:%d compile error:\n%v\n%s", connID, err, sql)
return nil, errors.Trace(err)
}
metrics.SessionExecuteCompileDuration.Observe(time.Since(startTS).Seconds())
metrics.SessionExecuteCompileDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds())

// Step3: Execute the physical plan.
if recordSets, err = s.executeStatement(ctx, connID, stmtNode, stmt, recordSets); err != nil {
Expand Down

0 comments on commit 8c44f56

Please sign in to comment.