Skip to content

Commit

Permalink
metrics: add metrics for bind info (#10921) (#11467)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored and zz-jason committed Jul 29, 2019
1 parent 384be78 commit 58aae12
Show file tree
Hide file tree
Showing 8 changed files with 136 additions and 12 deletions.
38 changes: 35 additions & 3 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,21 +19,22 @@ import (
"fmt"
"os"
"testing"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/parser"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
dto "github.com/prometheus/client_model/go"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -145,13 +146,21 @@ func (s *testSuite) TestGlobalBinding(c *C) {
tk.MustExec("create table t1(i int, s varchar(20))")
tk.MustExec("create index index_t on t(i,s)")

metrics.BindTotalGauge.Reset()
metrics.BindMemoryUsage.Reset()

_, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100")
c.Assert(err, IsNil, Commentf("err %v", err))

time.Sleep(time.Second * 1)
_, err = tk.Exec("create global binding for select * from t where i>99 using select * from t use index(index_t) where i>99")
c.Assert(err, IsNil)

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

sql, hash := parser.NormalizeDigest("select * from t where i > ?")

bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test")
Expand Down Expand Up @@ -202,6 +211,12 @@ func (s *testSuite) TestGlobalBinding(c *C) {
bindData = s.domain.BindHandle().GetBindRecord(hash, sql, "test")
c.Check(bindData, IsNil)

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
// From newly created global bind handle.
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

bindHandle = bindinfo.NewBindHandle(tk.Se)
err = bindHandle.Update(true)
c.Check(err, IsNil)
Expand Down Expand Up @@ -234,13 +249,21 @@ func (s *testSuite) TestSessionBinding(c *C) {
tk.MustExec("create table t1(i int, s varchar(20))")
tk.MustExec("create index index_t on t(i,s)")

metrics.BindTotalGauge.Reset()
metrics.BindMemoryUsage.Reset()

_, err := tk.Exec("create session binding for select * from t where i>100 using select * from t use index(index_t) where i>100")
c.Assert(err, IsNil, Commentf("err %v", err))

time.Sleep(time.Second * 1)
_, err = tk.Exec("create session binding for select * from t where i>99 using select * from t use index(index_t) where i>99")
c.Assert(err, IsNil)

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

handle := tk.Se.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
bindData := handle.GetBindRecord("select * from t where i > ?", "test")
c.Check(bindData, NotNil)
Expand Down Expand Up @@ -282,6 +305,11 @@ func (s *testSuite) TestSessionBinding(c *C) {
c.Check(bindData, NotNil)
c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?")
c.Check(bindData.Status, Equals, "deleted")

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
}

func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {
Expand Down Expand Up @@ -317,6 +345,7 @@ func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {

tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id")

metrics.BindUsageCounter.Reset()
tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows(
"MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id",
"├─Sort_11 9990.00 root test.t1.id:asc",
Expand All @@ -328,6 +357,9 @@ func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {
" └─Selection_13 9990.00 cop not(isnull(test.t2.id))",
" └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo",
))
pb := &dto.Metric{}
metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Write(pb)
c.Assert(pb.GetCounter().GetValue(), Equals, float64(1))

tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id")

Expand Down
19 changes: 19 additions & 0 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,10 @@
package bindinfo

import (
"unsafe"

"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
)
Expand Down Expand Up @@ -64,3 +67,19 @@ func newBindRecord(row chunk.Row) *BindRecord {
Collation: row.GetString(7),
}
}

// size calculates the memory size of a bind meta.
func (m *BindRecord) size() float64 {
res := len(m.OriginalSQL) + len(m.BindSQL) + len(m.Db) + len(m.Status) + 2*int(unsafe.Sizeof(m.CreateTime)) + len(m.Charset) + len(m.Collation)
return float64(res)
}

func (m *BindRecord) updateMetrics(scope string, inc bool) {
if inc {
metrics.BindMemoryUsage.WithLabelValues(scope, m.Status).Add(float64(m.size()))
metrics.BindTotalGauge.WithLabelValues(scope, m.Status).Inc()
} else {
metrics.BindMemoryUsage.WithLabelValues(scope, m.Status).Sub(float64(m.size()))
metrics.BindTotalGauge.WithLabelValues(scope, m.Status).Dec()
}
}
20 changes: 13 additions & 7 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -123,9 +124,10 @@ func (h *BindHandle) Update(fullLoad bool) (err error) {
continue
}

newCache.removeStaleBindMetas(hash, meta)
newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal)
if meta.Status == Using {
newCache[hash] = append(newCache[hash], meta)
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, meta.Status).Add(meta.size())
}
}
return nil
Expand Down Expand Up @@ -254,6 +256,7 @@ func (h *BindHandle) DropInvalidBindRecord() {

if time.Since(invalidBindRecord.droppedTime) > 6*time.Second {
delete(invalidBindRecordMap, key)
invalidBindRecord.bindRecord.updateMetrics(metrics.ScopeGlobal, false)
}
}
h.invalidBindRecordMap.Store(invalidBindRecordMap)
Expand All @@ -275,6 +278,7 @@ func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) {
bindRecord: invalidBindRecord,
}
h.invalidBindRecordMap.Store(newMap)
invalidBindRecord.updateMetrics(metrics.ScopeGlobal, true)
}

// Size return the size of bind info cache.
Expand Down Expand Up @@ -320,8 +324,9 @@ func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) {
// removed from the cache after this operation.
func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) {
newCache := h.bindInfo.Value.Load().(cache).copy()
newCache.removeStaleBindMetas(hash, meta)
newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal)
newCache[hash] = append(newCache[hash], meta)
meta.updateMetrics(metrics.ScopeGlobal, true)
h.bindInfo.Value.Store(newCache)
}

Expand All @@ -334,18 +339,19 @@ func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) {
h.bindInfo.Unlock()
}()

newCache.removeDeletedBindMeta(hash, meta)
newCache.removeDeletedBindMeta(hash, meta, metrics.ScopeGlobal)
}

// removeDeletedBindMeta removes all the BindMeta which originSQL and db are the same with the parameter's meta.
func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta) {
func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) {
metas, ok := c[hash]
if !ok {
return
}

for i := len(metas) - 1; i >= 0; i-- {
if meta.isSame(meta) {
if metas[i].isSame(meta) {
metas[i].updateMetrics(scope, false)
metas = append(metas[:i], metas[i+1:]...)
if len(metas) == 0 {
delete(c, hash)
Expand All @@ -356,15 +362,15 @@ func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta) {
}

// removeStaleBindMetas removes all the stale BindMeta in the cache.
func (c cache) removeStaleBindMetas(hash string, meta *BindMeta) {
func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) {
metas, ok := c[hash]
if !ok {
return
}

// remove stale bindMetas.
for i := len(metas) - 1; i >= 0; i-- {
if metas[i].isStale(meta) {
metas[i].updateMetrics(scope, false)
metas = append(metas[:i], metas[i+1:]...)
if len(metas) == 0 {
delete(c, hash)
Expand Down
15 changes: 13 additions & 2 deletions bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (

"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
)

Expand All @@ -38,8 +39,9 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle {
// removed from the cache after this operation.
func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) {
// Make sure there is only one goroutine writes the cache.
h.ch.removeStaleBindMetas(hash, meta)
h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession)
h.ch[hash] = append(h.ch[hash], meta)
meta.updateMetrics(metrics.ScopeSession, true)
}

func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) {
Expand Down Expand Up @@ -74,7 +76,7 @@ func (h *SessionHandle) DropBindRecord(record *BindRecord) {
meta := &BindMeta{BindRecord: record}
meta.Status = deleted
hash := parser.DigestHash(record.OriginalSQL)
h.ch.removeDeletedBindMeta(hash, meta)
h.ch.removeDeletedBindMeta(hash, meta, metrics.ScopeSession)
h.appendBindMeta(hash, meta)
}

Expand All @@ -100,6 +102,15 @@ func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) {
return bindRecords
}

// Close closes the session handle.
func (h *SessionHandle) Close() {
for _, bindRecords := range h.ch {
for _, bindRecord := range bindRecords {
bindRecord.updateMetrics(metrics.ScopeSession, false)
}
}
}

// sessionBindInfoKeyType is a dummy type to avoid naming collision in context.
type sessionBindInfoKeyType int

Expand Down
2 changes: 2 additions & 0 deletions executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -404,6 +404,7 @@ func addHintForSelect(hash, normdOrigSQL string, ctx sessionctx.Context, stmt as
return stmt
}
if bindRecord.Status == bindinfo.Using {
metrics.BindUsageCounter.WithLabelValues(metrics.ScopeSession).Inc()
return bindinfo.BindHint(stmt, bindRecord.Ast)
}
}
Expand All @@ -413,6 +414,7 @@ func addHintForSelect(hash, normdOrigSQL string, ctx sessionctx.Context, stmt as
bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, "")
}
if bindRecord != nil {
metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Inc()
return bindinfo.BindHint(stmt, bindRecord.Ast)
}
return stmt
Expand Down
43 changes: 43 additions & 0 deletions metrics/bindinfo.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
// 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 metrics

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

// bindinfo metrics.
var (
BindUsageCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "bindinfo",
Name: "bind_usage_counter",
Help: "Counter of query using sql bind",
}, []string{LableScope})

BindTotalGauge = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: "tidb",
Subsystem: "bindinfo",
Name: "bind_total_gauge",
Help: "Total number of sql bind",
}, []string{LableScope, LblType})

BindMemoryUsage = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: "tidb",
Subsystem: "bindinfo",
Name: "bind_memory_usage",
Help: "Memory usage of sql bind",
}, []string{LableScope, LblType})
)
7 changes: 7 additions & 0 deletions metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,10 @@ const (

opSucc = "ok"
opFailed = "err"

LableScope = "scope"
ScopeGlobal = "global"
ScopeSession = "session"
)

// RetLabel returns "ok" when err == nil and "err" when err != nil.
Expand All @@ -58,6 +62,9 @@ func RegisterMetrics() {
prometheus.MustRegister(AutoAnalyzeHistogram)
prometheus.MustRegister(AutoIDHistogram)
prometheus.MustRegister(BatchAddIdxHistogram)
prometheus.MustRegister(BindUsageCounter)
prometheus.MustRegister(BindTotalGauge)
prometheus.MustRegister(BindMemoryUsage)
prometheus.MustRegister(CampaignOwnerCounter)
prometheus.MustRegister(ConnGauge)
prometheus.MustRegister(PreparedStmtGauge)
Expand Down
4 changes: 4 additions & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1318,6 +1318,10 @@ func (s *session) Close() {
if s.statsCollector != nil {
s.statsCollector.Delete()
}
bindValue := s.Value(bindinfo.SessionBindInfoKeyType)
if bindValue != nil {
bindValue.(*bindinfo.SessionHandle).Close()
}
ctx := context.TODO()
s.RollbackTxn(ctx)
if s.sessionVars != nil {
Expand Down

0 comments on commit 58aae12

Please sign in to comment.