diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index b768ef4edd394..aab1a82f1b6be 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -19,7 +19,6 @@ import ( "fmt" "os" "testing" - "time" . "github.com/pingcap/check" "github.com/pingcap/parser" @@ -27,6 +26,7 @@ import ( "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" @@ -34,6 +34,7 @@ import ( "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) { @@ -146,13 +147,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") @@ -203,6 +212,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) @@ -235,13 +250,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) @@ -283,6 +306,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) { @@ -318,6 +346,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", @@ -329,6 +358,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") diff --git a/bindinfo/cache.go b/bindinfo/cache.go index a4c2785eb9c64..5b74a8c316832 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -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" ) @@ -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() + } +} diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 3e3da3b90ea09..2e5124b95f990 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -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" @@ -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 @@ -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) @@ -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. @@ -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) } @@ -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) @@ -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) diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index f343b3ca8e24d..f52c7d0f92e22 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/types" ) @@ -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) { @@ -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) } @@ -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 diff --git a/executor/compiler.go b/executor/compiler.go index 7ac5f1eef3367..9190763fe68fd 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -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) } } @@ -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 diff --git a/metrics/bindinfo.go b/metrics/bindinfo.go new file mode 100644 index 0000000000000..958bd110c2b23 --- /dev/null +++ b/metrics/bindinfo.go @@ -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}) +) diff --git a/metrics/metrics.go b/metrics/metrics.go index ba6a3ce73736d..a31d5ec3837c6 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -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. @@ -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) diff --git a/session/session.go b/session/session.go index 2d58ac5f2258a..bca00fdf4e7fa 100644 --- a/session/session.go +++ b/session/session.go @@ -1364,6 +1364,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 {