From 1543af78e383391ad90cd019f8191122dad7a852 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 24 Jun 2019 14:32:25 +0800 Subject: [PATCH 1/5] metrics: add metrics for bind info --- bindinfo/bind_test.go | 38 +++++++++++++++++++++++++++++--- bindinfo/cache.go | 7 ++++++ bindinfo/handle.go | 44 ++++++++++++++++++++++++++------------ bindinfo/session_handle.go | 40 +++++++++++++++++++++++++++++----- executor/compiler.go | 2 ++ metrics/bindinfo.go | 43 +++++++++++++++++++++++++++++++++++++ metrics/metrics.go | 6 ++++++ session/session.go | 4 ++++ 8 files changed, 162 insertions(+), 22 deletions(-) create mode 100644 metrics/bindinfo.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index d6e1a12a9a5f2..bbd7f2f038dfd 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) { @@ -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).Write(pb) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).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") @@ -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).Write(pb) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(0)) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).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) @@ -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).Write(pb) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).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) @@ -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).Write(pb) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(0)) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Write(pb) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(118)) } func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) { @@ -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", @@ -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") diff --git a/bindinfo/cache.go b/bindinfo/cache.go index a4c2785eb9c64..666c8b8b0fb51 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -14,6 +14,8 @@ package bindinfo import ( + "unsafe" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -64,3 +66,8 @@ func newBindRecord(row chunk.Row) *BindRecord { Collation: row.GetString(7), } } + +func (m *BindMeta) 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) +} diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 3e3da3b90ea09..7f48589f88cd3 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).Add(meta.size()) } } return nil @@ -165,7 +167,9 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return } - h.appendBindMeta(hash, meta) + if h.appendBindMeta(hash, meta) { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Inc() + } h.bindInfo.Unlock() }() @@ -219,7 +223,9 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { } hash, meta := newBindMetaWithoutAst(record) - h.removeBindMeta(hash, meta) + if h.removeBindMeta(hash, meta) { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Dec() + } }() txn, err1 := h.sctx.Context.Txn(true) @@ -318,15 +324,17 @@ func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { // appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { +func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) bool { newCache := h.bindInfo.Value.Load().(cache).copy() - newCache.removeStaleBindMetas(hash, meta) + removed := newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) newCache[hash] = append(newCache[hash], meta) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Add(float64(meta.size())) h.bindInfo.Value.Store(newCache) + return !removed } // removeBindMeta removes the BindMeta from the cache. -func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) { +func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) bool { h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() defer func() { @@ -334,44 +342,52 @@ func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) { h.bindInfo.Unlock() }() - newCache.removeDeletedBindMeta(hash, meta) + return 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) bool { metas, ok := c[hash] if !ok { - return + return false } + var removed bool for i := len(metas) - 1; i >= 0; i-- { - if meta.isSame(meta) { + if metas[i].isSame(meta) { + metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) + removed = true metas = append(metas[:i], metas[i+1:]...) if len(metas) == 0 { delete(c, hash) - return + return removed } } } + return removed } // 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) bool { metas, ok := c[hash] if !ok { - return + return false } // remove stale bindMetas. + var removed bool for i := len(metas) - 1; i >= 0; i-- { if metas[i].isStale(meta) { + metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) + removed = true metas = append(metas[:i], metas[i+1:]...) if len(metas) == 0 { delete(c, hash) - return + return removed } } } + return removed } func (c cache) copy() cache { diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index f343b3ca8e24d..56ede785a2183 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" ) @@ -36,10 +37,12 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { // appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) { +func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) bool { // Make sure there is only one goroutine writes the cache. - h.ch.removeStaleBindMetas(hash, meta) + removed := h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession) h.ch[hash] = append(h.ch[hash], meta) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Add(float64(meta.size())) + return !removed } func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { @@ -63,8 +66,12 @@ func (h *SessionHandle) AddBindRecord(record *BindRecord) error { // update the BindMeta to the cache. hash, meta, err := h.newBindMeta(record) - if err == nil { - h.appendBindMeta(hash, meta) + if err != nil { + return err + } + + if h.appendBindMeta(hash, meta) && record.Status == Using { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Inc() } return err } @@ -74,7 +81,11 @@ func (h *SessionHandle) DropBindRecord(record *BindRecord) { meta := &BindMeta{BindRecord: record} meta.Status = deleted hash := parser.DigestHash(record.OriginalSQL) - h.ch.removeDeletedBindMeta(hash, meta) + matchRecord := h.GetBindRecord(record.OriginalSQL, record.Db) + if matchRecord != nil && matchRecord.Status == Using { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Dec() + } + h.ch.removeDeletedBindMeta(hash, meta, metrics.ScopeSession) h.appendBindMeta(hash, meta) } @@ -100,6 +111,25 @@ func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { return bindRecords } +// Close closes the session handle. +func (h *SessionHandle) Close() { + totalNum, totalSize := float64(0), float64(0) + for _, bindRecords := range h.ch { + for _, bindRecord := range bindRecords { + if bindRecord.Status == Using { + totalNum++ + } + totalSize += bindRecord.size() + } + } + if totalNum > 0 { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Sub(totalNum) + } + if totalSize > 0 { + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Sub(totalSize) + } +} + // 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 7c0c6a6a4cf97..fc33fb423a58c 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..429b6e232004a --- /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{LblType}) + + BindTotalGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "bindinfo", + Name: "bind_total_gauge", + Help: "Total number of sql bind", + }, []string{LblType}) + + BindMemoryUsage = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "bindinfo", + Name: "bind_memory_usage", + Help: "Memory usage of sql bind", + }, []string{LblType}) +) diff --git a/metrics/metrics.go b/metrics/metrics.go index 7b4b97ca1859c..e7e8b8d75e00f 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -41,6 +41,9 @@ const ( opSucc = "ok" opFailed = "err" + + ScopeGlobal = "global" + ScopeSession = "session" ) // RetLabel returns "ok" when err == nil and "err" when err != nil. @@ -58,6 +61,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 46c5dee202ebc..4633adcb9319c 100644 --- a/session/session.go +++ b/session/session.go @@ -1348,6 +1348,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 { From c9015a182a8a9666d48b76e763e8c7b9bcbf0a84 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 3 Jul 2019 18:46:36 +0800 Subject: [PATCH 2/5] address comments --- bindinfo/handle.go | 43 +++++++++++++++++--------------------- bindinfo/session_handle.go | 20 +++++++----------- 2 files changed, 26 insertions(+), 37 deletions(-) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 7f48589f88cd3..41641dfbf6c32 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -167,9 +167,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return } - if h.appendBindMeta(hash, meta) { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Inc() - } + h.appendBindMeta(hash, meta) h.bindInfo.Unlock() }() @@ -223,9 +221,7 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { } hash, meta := newBindMetaWithoutAst(record) - if h.removeBindMeta(hash, meta) { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Dec() - } + h.removeBindMeta(hash, meta) }() txn, err1 := h.sctx.Context.Txn(true) @@ -324,17 +320,17 @@ func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { // appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) bool { +func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { newCache := h.bindInfo.Value.Load().(cache).copy() - removed := newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) + newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) newCache[hash] = append(newCache[hash], meta) metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Add(float64(meta.size())) + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Inc() h.bindInfo.Value.Store(newCache) - return !removed } // removeBindMeta removes the BindMeta from the cache. -func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) bool { +func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) { h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() defer func() { @@ -342,52 +338,51 @@ func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) bool { h.bindInfo.Unlock() }() - return newCache.removeDeletedBindMeta(hash, meta, metrics.ScopeGlobal) + 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, scope string) bool { +func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) { metas, ok := c[hash] if !ok { - return false + return } - var removed bool for i := len(metas) - 1; i >= 0; i-- { if metas[i].isSame(meta) { metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) - removed = true + if metas[i].Status == Using { + metrics.BindTotalGauge.WithLabelValues(scope).Dec() + } metas = append(metas[:i], metas[i+1:]...) if len(metas) == 0 { delete(c, hash) - return removed + return } } } - return removed } // removeStaleBindMetas removes all the stale BindMeta in the cache. -func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) bool { +func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) { metas, ok := c[hash] if !ok { - return false + return } - // remove stale bindMetas. - var removed bool for i := len(metas) - 1; i >= 0; i-- { if metas[i].isStale(meta) { metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) - removed = true + if metas[i].Status == Using { + metrics.BindTotalGauge.WithLabelValues(scope).Sub(1) + } metas = append(metas[:i], metas[i+1:]...) if len(metas) == 0 { delete(c, hash) - return removed + return } } } - return removed } func (c cache) copy() cache { diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index 56ede785a2183..7bc62582f3fab 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -37,12 +37,14 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { // appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) bool { +func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) { // Make sure there is only one goroutine writes the cache. - removed := h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession) + h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession) h.ch[hash] = append(h.ch[hash], meta) metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Add(float64(meta.size())) - return !removed + if meta.Status == Using { + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Inc() + } } func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { @@ -66,12 +68,8 @@ func (h *SessionHandle) AddBindRecord(record *BindRecord) error { // update the BindMeta to the cache. hash, meta, err := h.newBindMeta(record) - if err != nil { - return err - } - - if h.appendBindMeta(hash, meta) && record.Status == Using { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Inc() + if err == nil { + h.appendBindMeta(hash, meta) } return err } @@ -81,10 +79,6 @@ func (h *SessionHandle) DropBindRecord(record *BindRecord) { meta := &BindMeta{BindRecord: record} meta.Status = deleted hash := parser.DigestHash(record.OriginalSQL) - matchRecord := h.GetBindRecord(record.OriginalSQL, record.Db) - if matchRecord != nil && matchRecord.Status == Using { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Dec() - } h.ch.removeDeletedBindMeta(hash, meta, metrics.ScopeSession) h.appendBindMeta(hash, meta) } From 718db71ba3793cefa3db9b5a28f70c3a5132b9f0 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 9 Jul 2019 15:10:49 +0800 Subject: [PATCH 3/5] add comments --- bindinfo/cache.go | 1 + 1 file changed, 1 insertion(+) diff --git a/bindinfo/cache.go b/bindinfo/cache.go index 666c8b8b0fb51..846b28418958a 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -67,6 +67,7 @@ func newBindRecord(row chunk.Row) *BindRecord { } } +// size calculates the memory size of a bind meta. func (m *BindMeta) 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) From 6f843f7bae6c38376325bfab65bc5369ac5f85ba Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 25 Jul 2019 15:31:26 +0800 Subject: [PATCH 4/5] address comments --- bindinfo/bind_test.go | 18 +++++++++--------- bindinfo/cache.go | 13 ++++++++++++- bindinfo/handle.go | 17 ++++++----------- bindinfo/session_handle.go | 17 ++--------------- metrics/bindinfo.go | 6 +++--- metrics/metrics.go | 1 + 6 files changed, 33 insertions(+), 39 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index bbd7f2f038dfd..2959b77149d6b 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -156,9 +156,9 @@ func (s *testSuite) TestGlobalBinding(c *C) { c.Assert(err, IsNil) pb := &dto.Metric{} - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Write(pb) + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Write(pb) + 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 > ?") @@ -211,9 +211,9 @@ func (s *testSuite) TestGlobalBinding(c *C) { bindData = s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, IsNil) - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Write(pb) + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(0)) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Write(pb) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) // From newly created global bind handle. c.Assert(pb.GetGauge().GetValue(), Equals, float64(161)) @@ -259,9 +259,9 @@ func (s *testSuite) TestSessionBinding(c *C) { c.Assert(err, IsNil) pb := &dto.Metric{} - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Write(pb) + metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Write(pb) + 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) @@ -306,10 +306,10 @@ func (s *testSuite) TestSessionBinding(c *C) { c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") c.Check(bindData.Status, Equals, "deleted") - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Write(pb) + 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)) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Write(pb) - c.Assert(pb.GetGauge().GetValue(), Equals, float64(118)) } func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) { diff --git a/bindinfo/cache.go b/bindinfo/cache.go index 846b28418958a..5b74a8c316832 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -17,6 +17,7 @@ import ( "unsafe" "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) @@ -68,7 +69,17 @@ func newBindRecord(row chunk.Row) *BindRecord { } // size calculates the memory size of a bind meta. -func (m *BindMeta) size() float64 { +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 41641dfbf6c32..a4100349b2867 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -127,7 +127,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) if meta.Status == Using { newCache[hash] = append(newCache[hash], meta) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Add(meta.size()) + metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, meta.Status).Add(meta.size()) } } return nil @@ -257,6 +257,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) } @@ -277,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. @@ -324,8 +326,7 @@ func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { newCache := h.bindInfo.Value.Load().(cache).copy() newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) newCache[hash] = append(newCache[hash], meta) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal).Add(float64(meta.size())) - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal).Inc() + meta.updateMetrics(metrics.ScopeGlobal, true) h.bindInfo.Value.Store(newCache) } @@ -350,10 +351,7 @@ func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) for i := len(metas) - 1; i >= 0; i-- { if metas[i].isSame(meta) { - metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) - if metas[i].Status == Using { - metrics.BindTotalGauge.WithLabelValues(scope).Dec() - } + metas[i].updateMetrics(scope, false) metas = append(metas[:i], metas[i+1:]...) if len(metas) == 0 { delete(c, hash) @@ -372,10 +370,7 @@ func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) { for i := len(metas) - 1; i >= 0; i-- { if metas[i].isStale(meta) { - metrics.BindMemoryUsage.WithLabelValues(scope).Sub(metas[i].size()) - if metas[i].Status == Using { - metrics.BindTotalGauge.WithLabelValues(scope).Sub(1) - } + 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 7bc62582f3fab..f52c7d0f92e22 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -41,10 +41,7 @@ func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) { // Make sure there is only one goroutine writes the cache. h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession) h.ch[hash] = append(h.ch[hash], meta) - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Add(float64(meta.size())) - if meta.Status == Using { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Inc() - } + meta.updateMetrics(metrics.ScopeSession, true) } func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { @@ -107,21 +104,11 @@ func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { // Close closes the session handle. func (h *SessionHandle) Close() { - totalNum, totalSize := float64(0), float64(0) for _, bindRecords := range h.ch { for _, bindRecord := range bindRecords { - if bindRecord.Status == Using { - totalNum++ - } - totalSize += bindRecord.size() + bindRecord.updateMetrics(metrics.ScopeSession, false) } } - if totalNum > 0 { - metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession).Sub(totalNum) - } - if totalSize > 0 { - metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession).Sub(totalSize) - } } // sessionBindInfoKeyType is a dummy type to avoid naming collision in context. diff --git a/metrics/bindinfo.go b/metrics/bindinfo.go index 429b6e232004a..958bd110c2b23 100644 --- a/metrics/bindinfo.go +++ b/metrics/bindinfo.go @@ -23,7 +23,7 @@ var ( Subsystem: "bindinfo", Name: "bind_usage_counter", Help: "Counter of query using sql bind", - }, []string{LblType}) + }, []string{LableScope}) BindTotalGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -31,7 +31,7 @@ var ( Subsystem: "bindinfo", Name: "bind_total_gauge", Help: "Total number of sql bind", - }, []string{LblType}) + }, []string{LableScope, LblType}) BindMemoryUsage = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -39,5 +39,5 @@ var ( Subsystem: "bindinfo", Name: "bind_memory_usage", Help: "Memory usage of sql bind", - }, []string{LblType}) + }, []string{LableScope, LblType}) ) diff --git a/metrics/metrics.go b/metrics/metrics.go index e7e8b8d75e00f..b2f31e3e5eb9f 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -42,6 +42,7 @@ const ( opSucc = "ok" opFailed = "err" + LableScope = "scope" ScopeGlobal = "global" ScopeSession = "session" ) From e4a54a25d4c2825e47565fb3e64846f284a1aa6e Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Fri, 26 Jul 2019 13:20:35 +0800 Subject: [PATCH 5/5] address comments --- bindinfo/handle.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index a4100349b2867..2e5124b95f990 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -256,8 +256,8 @@ func (h *BindHandle) DropInvalidBindRecord() { if time.Since(invalidBindRecord.droppedTime) > 6*time.Second { delete(invalidBindRecordMap, key) + invalidBindRecord.bindRecord.updateMetrics(metrics.ScopeGlobal, false) } - invalidBindRecord.bindRecord.updateMetrics(metrics.ScopeGlobal, false) } h.invalidBindRecordMap.Store(invalidBindRecordMap) }