Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

metrics: add different labels for restricted SQL and general SQL #7631

Merged
merged 4 commits into from
Sep 7, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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