From bb2e349fea71426b973eb8bb570d940c596f8b02 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 29 Oct 2019 14:40:23 +0800 Subject: [PATCH] bindinfo: support multiply bindings for a sql (#12851) --- bindinfo/bind_test.go | 75 ++++++------ bindinfo/cache.go | 186 ++++++++++++++++++++++++------ bindinfo/handle.go | 230 ++++++++++++++++++++----------------- bindinfo/session_handle.go | 61 +++++----- domain/db_test.go | 2 +- domain/domain.go | 2 +- executor/bind.go | 15 ++- executor/show.go | 24 ++-- planner/optimize.go | 49 ++++---- 9 files changed, 394 insertions(+), 250 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 14f66af3a0..d4a0316109 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -128,18 +128,19 @@ func (s *testSuite) TestBindParse(c *C) { bindData := bindHandle.GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t") - c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t)") + bind := bindData.Bindings[0] + c.Check(bind.BindSQL, Equals, "select * from t use index(index_t)") c.Check(bindData.Db, Equals, "test") - c.Check(bindData.Status, Equals, "using") - c.Check(bindData.Charset, Equals, "utf8mb4") - c.Check(bindData.Collation, Equals, "utf8mb4_bin") - c.Check(bindData.CreateTime, NotNil) - c.Check(bindData.UpdateTime, NotNil) + c.Check(bind.Status, Equals, "using") + c.Check(bind.Charset, Equals, "utf8mb4") + c.Check(bind.Collation, Equals, "utf8mb4_bin") + c.Check(bind.CreateTime, NotNil) + c.Check(bind.UpdateTime, NotNil) // Test fields with quotes or slashes. - sql = `CREATE GLOBAL BINDING FOR select * from t where a BETWEEN "a" and "b" USING select * from t use index(idx) where a BETWEEN "a\nb\rc\td\0e" and "x"` + sql = `CREATE GLOBAL BINDING FOR select * from t where i BETWEEN "a" and "b" USING select * from t use index(index_t) where i BETWEEN "a\nb\rc\td\0e" and "x"` tk.MustExec(sql) - tk.MustExec(`DROP global binding for select * from t use index(idx) where a BETWEEN "a\nb\rc\td\0e" and "x"`) + tk.MustExec(`DROP global binding for select * from t use index(idx) where i BETWEEN "a\nb\rc\td\0e" and "x"`) } func (s *testSuite) TestGlobalBinding(c *C) { @@ -172,13 +173,14 @@ func (s *testSuite) TestGlobalBinding(c *C) { bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") - c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") + bind := bindData.Bindings[0] + c.Check(bind.BindSQL, Equals, "select * from t use index(index_t) where i>99") c.Check(bindData.Db, Equals, "test") - c.Check(bindData.Status, Equals, "using") - c.Check(bindData.Charset, NotNil) - c.Check(bindData.Collation, NotNil) - c.Check(bindData.CreateTime, NotNil) - c.Check(bindData.UpdateTime, NotNil) + c.Check(bind.Status, Equals, "using") + c.Check(bind.Charset, NotNil) + c.Check(bind.Collation, NotNil) + c.Check(bind.CreateTime, NotNil) + c.Check(bind.UpdateTime, NotNil) rs, err := tk.Exec("show global bindings") c.Assert(err, IsNil) @@ -204,13 +206,14 @@ func (s *testSuite) TestGlobalBinding(c *C) { bindData = bindHandle.GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") - c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") + bind = bindData.Bindings[0] + c.Check(bind.BindSQL, Equals, "select * from t use index(index_t) where i>99") c.Check(bindData.Db, Equals, "test") - c.Check(bindData.Status, Equals, "using") - c.Check(bindData.Charset, NotNil) - c.Check(bindData.Collation, NotNil) - c.Check(bindData.CreateTime, NotNil) - c.Check(bindData.UpdateTime, NotNil) + c.Check(bind.Status, Equals, "using") + c.Check(bind.Charset, NotNil) + c.Check(bind.Collation, NotNil) + c.Check(bind.CreateTime, NotNil) + c.Check(bind.UpdateTime, NotNil) _, err = tk.Exec("DROP global binding for select * from t where i>100") c.Check(err, IsNil) @@ -274,13 +277,14 @@ func (s *testSuite) TestSessionBinding(c *C) { bindData := handle.GetBindRecord("select * from t where i > ?", "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") - c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") + bind := bindData.Bindings[0] + c.Check(bind.BindSQL, Equals, "select * from t use index(index_t) where i>99") c.Check(bindData.Db, Equals, "test") - c.Check(bindData.Status, Equals, "using") - c.Check(bindData.Charset, NotNil) - c.Check(bindData.Collation, NotNil) - c.Check(bindData.CreateTime, NotNil) - c.Check(bindData.UpdateTime, NotNil) + c.Check(bind.Status, Equals, "using") + c.Check(bind.Charset, NotNil) + c.Check(bind.Collation, NotNil) + c.Check(bind.CreateTime, NotNil) + c.Check(bind.UpdateTime, NotNil) rs, err := tk.Exec("show global bindings") c.Assert(err, IsNil) @@ -310,7 +314,7 @@ func (s *testSuite) TestSessionBinding(c *C) { bindData = handle.GetBindRecord("select * from t where i > ?", "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") - c.Check(bindData.Status, Equals, "deleted") + c.Check(len(bindData.Bindings), Equals, 0) metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(0)) @@ -380,13 +384,14 @@ func (s *testSuite) TestErrorBind(c *C) { bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") - c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>100") + bind := bindData.Bindings[0] + c.Check(bind.BindSQL, Equals, "select * from t use index(index_t) where i>100") c.Check(bindData.Db, Equals, "test") - c.Check(bindData.Status, Equals, "using") - c.Check(bindData.Charset, NotNil) - c.Check(bindData.Collation, NotNil) - c.Check(bindData.CreateTime, NotNil) - c.Check(bindData.UpdateTime, NotNil) + c.Check(bind.Status, Equals, "using") + c.Check(bind.Charset, NotNil) + c.Check(bind.Collation, NotNil) + c.Check(bind.CreateTime, NotNil) + c.Check(bind.UpdateTime, NotNil) tk.MustExec("drop index index_t on t") _, err = tk.Exec("select * from t where i > 10") @@ -434,11 +439,11 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") - s.domain.BindHandle().CaptureBaselines(s.domain.InfoSchema()) + s.domain.BindHandle().CaptureBaselines() tk.MustQuery("show global bindings").Check(testkit.Rows()) tk.MustExec("select * from t") tk.MustExec("select * from t") - s.domain.BindHandle().CaptureBaselines(s.domain.InfoSchema()) + s.domain.BindHandle().CaptureBaselines() rows := tk.MustQuery("show global bindings").Rows() c.Assert(len(rows), Equals, 1) c.Assert(rows[0][0], Equals, "select * from t") diff --git a/bindinfo/cache.go b/bindinfo/cache.go index 12e9a88e06..9ae5d49f06 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -14,11 +14,14 @@ package bindinfo import ( + "context" "unsafe" + "github.com/pingcap/parser" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" ) const ( @@ -30,56 +33,165 @@ const ( Invalid = "invalid" ) -// BindMeta stores the basic bind info and bindSql astNode. -type BindMeta struct { - *BindRecord - // HintSet stores the set of hints of binding sql. - *HintsSet -} - -// cache is a k-v map, key is original sql, value is a slice of BindMeta. -type cache map[string][]*BindMeta - -// BindRecord represents a sql bind record retrieved from the storage. -type BindRecord struct { - OriginalSQL string - BindSQL string - Db string +// Binding stores the basic bind hint info. +type Binding struct { + BindSQL string // Status represents the status of the binding. It can only be one of the following values: // 1. deleted: BindRecord is deleted, can not be used anymore. - // 2. using: BindRecord is in the normal active mode. + // 2. using: Binding is in the normal active mode. Status string CreateTime types.Time UpdateTime types.Time Charset string Collation string + // Hint is the parsed hints, it is used to bind hints to stmt node. + Hint *HintsSet + // id is the string form of all hints. It is used to uniquely identify different hints. + id string } -func newBindRecord(row chunk.Row) *BindRecord { - return &BindRecord{ - OriginalSQL: row.GetString(0), - BindSQL: row.GetString(1), - Db: row.GetString(2), - Status: row.GetString(3), - CreateTime: row.GetTime(4), - UpdateTime: row.GetTime(5), - Charset: row.GetString(6), - Collation: row.GetString(7), +// cache is a k-v map, key is original sql, value is a slice of BindRecord. +type cache map[string][]*BindRecord + +// BindRecord represents a sql bind record retrieved from the storage. +type BindRecord struct { + OriginalSQL string + Db string + + Bindings []Binding +} + +// FirstUsingBinding returns the first binding that has status 'using'. +// TODO: Remove it when we support using all bindings to adjust the plan. +func (br *BindRecord) FirstUsingBinding() *Binding { + for _, binding := range br.Bindings { + if binding.Status == Using { + return &binding + } } + return nil } -// 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) +func (br *BindRecord) prepareHintsForUsing(sctx sessionctx.Context, is infoschema.InfoSchema) error { + p := parser.New() + for i, bind := range br.Bindings { + if bind.Status != Using || bind.Hint != nil { + continue + } + stmtNode, err := p.ParseOneStmt(bind.BindSQL, bind.Charset, bind.Collation) + if err != nil { + return err + } + hints, err := GenHintsFromSQL(context.TODO(), sctx, stmtNode, is) + if err != nil { + return err + } + br.Bindings[i].Hint = CollectHint(stmtNode) + br.Bindings[i].id = hints + } + return nil +} + +// `merge` merges two BindRecord. It will replace old bindings with new bindings if there are new updates. +func merge(lBindRecord, rBindRecord *BindRecord) *BindRecord { + if lBindRecord == nil { + return rBindRecord + } + if rBindRecord == nil { + return lBindRecord + } + result := lBindRecord.shallowCopy() + for _, rbind := range rBindRecord.Bindings { + found := false + for j, lbind := range lBindRecord.Bindings { + if lbind.id == rbind.id { + found = true + if rbind.UpdateTime.Compare(lbind.UpdateTime) >= 0 { + result.Bindings[j] = rbind + } + break + } + } + if !found { + result.Bindings = append(result.Bindings, rbind) + } + } + return result +} + +func (br *BindRecord) remove(deleted *BindRecord) *BindRecord { + // Delete all bindings. + if len(deleted.Bindings) == 0 { + return &BindRecord{OriginalSQL: br.OriginalSQL, Db: br.Db} + } + result := br.shallowCopy() + for _, deletedBind := range deleted.Bindings { + for i, bind := range result.Bindings { + if bind.id == deletedBind.id { + result.Bindings = append(result.Bindings[:i], result.Bindings[i+1:]...) + break + } + } + } + return result +} + +// shallowCopy shallow copies the BindRecord. +func (br *BindRecord) shallowCopy() *BindRecord { + result := BindRecord{ + OriginalSQL: br.OriginalSQL, + Db: br.Db, + Bindings: make([]Binding, len(br.Bindings)), + } + copy(result.Bindings, br.Bindings) + return &result +} + +func (br *BindRecord) isSame(other *BindRecord) bool { + return br.OriginalSQL == other.OriginalSQL && br.Db == other.Db +} + +var statusIndex = map[string]int{ + Using: 0, + deleted: 1, + Invalid: 2, +} + +func (br *BindRecord) metrics() ([]float64, []int) { + sizes := make([]float64, len(statusIndex)) + count := make([]int, len(statusIndex)) + if br == nil { + return sizes, count + } + commonLength := float64(len(br.OriginalSQL) + len(br.Db)) + // We treat it as deleted if there are no bindings. It could only occur in session handles. + if len(br.Bindings) == 0 { + sizes[statusIndex[deleted]] = commonLength + count[statusIndex[deleted]] = 1 + return sizes, count + } + // Make the common length counted in the first binding. + sizes[statusIndex[br.Bindings[0].Status]] = commonLength + for _, binding := range br.Bindings { + sizes[statusIndex[binding.Status]] += binding.size() + count[statusIndex[binding.Status]] += 1 + } + return sizes, count +} + +// size calculates the memory size of a bind info. +func (m *Binding) size() float64 { + res := len(m.BindSQL) + 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() +func updateMetrics(scope string, before *BindRecord, after *BindRecord, sizeOnly bool) { + beforeSize, beforeCount := before.metrics() + afterSize, afterCount := after.metrics() + for status, index := range statusIndex { + metrics.BindMemoryUsage.WithLabelValues(scope, status).Add(afterSize[index] - beforeSize[index]) + if !sizeOnly { + metrics.BindTotalGauge.WithLabelValues(scope, status).Add(float64(afterCount[index] - beforeCount[index])) + } } } diff --git a/bindinfo/handle.go b/bindinfo/handle.go index aba4438b08..203be6de84 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" @@ -120,27 +121,34 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { }() for _, row := range rows { - hash, meta, err := h.newBindMeta(newBindRecord(row)) + hash, meta, err := h.newBindRecord(row) // Update lastUpdateTime to the newest one. - if meta.UpdateTime.Compare(h.lastUpdateTime) > 0 { - h.lastUpdateTime = meta.UpdateTime + if meta.Bindings[0].UpdateTime.Compare(h.lastUpdateTime) > 0 { + h.lastUpdateTime = meta.Bindings[0].UpdateTime } if err != nil { logutil.BgLogger().Error("update bindinfo failed", zap.Error(err)) continue } - 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()) + oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) + newRecord := merge(oldRecord, meta) + if meta.FirstUsingBinding() != nil { + newCache.setBindRecord(hash, newRecord) + } else { + newCache.removeDeletedBindRecord(hash, oldRecord) } + updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db), true) } return nil } -// AddBindRecord adds a BindRecord to the storage and BindMeta to the cache. -func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { +// AddBindRecord adds a BindRecord to the storage and BindRecord to the cache. +func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSchema, record *BindRecord) (err error) { + err = record.prepareHintsForUsing(sctx, is) + if err != nil { + return err + } exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() _, err = exec.Execute(context.TODO(), "BEGIN") @@ -165,15 +173,8 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { // Make sure there is only one goroutine writes the cache and use parser. h.bindInfo.Lock() - // update the BindMeta to the cache. - hash, meta, err1 := h.newBindMeta(record) - if err1 != nil { - err = err1 - h.bindInfo.Unlock() - return - } - - h.appendBindMeta(hash, meta) + // update the BindRecord to the cache. + h.appendBindRecord(parser.DigestHash(record.OriginalSQL), record) h.bindInfo.Unlock() }() @@ -187,20 +188,25 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { if err1 != nil { return err1 } - record.CreateTime = types.Time{ - Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), - Type: mysql.TypeDatetime, - Fsp: 3, - } - record.UpdateTime = record.CreateTime - record.Status = Using + for i := range record.Bindings { + record.Bindings[i].CreateTime = types.Time{ + Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), + Type: mysql.TypeDatetime, + Fsp: 3, + } + record.Bindings[i].UpdateTime = record.Bindings[0].CreateTime + record.Bindings[i].Status = Using - // insert the BindRecord to the storage. - _, err = exec.Execute(context.TODO(), h.insertBindInfoSQL(record)) - return err + // insert the BindRecord to the storage. + _, err = exec.Execute(context.TODO(), h.insertBindInfoSQL(record.OriginalSQL, record.Db, record.Bindings[i])) + if err != nil { + return err + } + } + return nil } -// DropBindRecord drops a BindRecord to the storage and BindMeta int the cache. +// DropBindRecord drops a BindRecord to the storage and BindRecord int the cache. func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() @@ -225,8 +231,7 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { return } - hash, meta := newBindMetaWithoutHints(record) - h.removeBindMeta(hash, meta) + err = h.removeBindRecord(parser.DigestHash(record.OriginalSQL), record) }() txn, err1 := h.sctx.Context.Txn(true) @@ -239,10 +244,12 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { Type: mysql.TypeDatetime, Fsp: 3, } - record.Status = deleted - record.UpdateTime = updateTs + for i := range record.Bindings { + record.Bindings[i].Status = deleted + record.Bindings[i].UpdateTime = updateTs + } - _, err = exec.Execute(context.TODO(), h.logicalDeleteBindInfoSQL(record.OriginalSQL, record.Db, updateTs)) + _, err = exec.Execute(context.TODO(), h.logicalDeleteBindInfoSQL(record, updateTs)) return err } @@ -261,7 +268,7 @@ func (h *BindHandle) DropInvalidBindRecord() { if time.Since(invalidBindRecord.droppedTime) > 6*time.Second { delete(invalidBindRecordMap, key) - invalidBindRecord.bindRecord.updateMetrics(metrics.ScopeGlobal, false) + updateMetrics(metrics.ScopeGlobal, invalidBindRecord.bindRecord, nil, false) } } h.invalidBindRecordMap.Store(invalidBindRecordMap) @@ -283,7 +290,7 @@ func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { bindRecord: invalidBindRecord, } h.invalidBindRecordMap.Store(newMap) - invalidBindRecord.updateMetrics(metrics.ScopeGlobal, true) + updateMetrics(metrics.ScopeGlobal, nil, invalidBindRecord, false) } // Size return the size of bind info cache. @@ -295,13 +302,13 @@ func (h *BindHandle) Size() int { return size } -// GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. -func (h *BindHandle) GetBindRecord(hash, normdOrigSQL, db string) *BindMeta { +// GetBindRecord return the BindRecord of the (normdOrigSQL,db) if BindRecord exist. +func (h *BindHandle) GetBindRecord(hash, normdOrigSQL, db string) *BindRecord { return h.bindInfo.Load().(cache).getBindRecord(hash, normdOrigSQL, db) } // GetAllBindRecord return all bind record in cache. -func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindMeta) { +func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindRecord) { bindRecordMap := h.bindInfo.Load().(cache) for _, bindRecord := range bindRecordMap { bindRecords = append(bindRecords, bindRecord...) @@ -309,46 +316,60 @@ func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindMeta) { return bindRecords } -func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { - hash = parser.DigestHash(record.OriginalSQL) - stmtNodes, _, err := h.bindInfo.parser.Parse(record.BindSQL, record.Charset, record.Collation) +func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { + hint := Binding{ + BindSQL: row.GetString(1), + Status: row.GetString(3), + CreateTime: row.GetTime(4), + UpdateTime: row.GetTime(5), + Charset: row.GetString(6), + Collation: row.GetString(7), + } + bindRecord := &BindRecord{ + OriginalSQL: row.GetString(0), + Db: row.GetString(2), + Bindings: []Binding{hint}, + } + hash := parser.DigestHash(bindRecord.OriginalSQL) + h.sctx.Lock() + defer h.sctx.Unlock() + err := h.sctx.RefreshTxnCtx(context.TODO()) if err != nil { return "", nil, err } - meta = &BindMeta{BindRecord: record, HintsSet: CollectHint(stmtNodes[0])} - return hash, meta, nil + h.sctx.GetSessionVars().CurrentDB = bindRecord.Db + err = bindRecord.prepareHintsForUsing(h.sctx.Context, h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema)) + return hash, bindRecord, err } -func newBindMetaWithoutHints(record *BindRecord) (hash string, meta *BindMeta) { - hash = parser.DigestHash(record.OriginalSQL) - meta = &BindMeta{BindRecord: record} - return hash, meta -} - -// appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are +// appendBindRecord addes the BindRecord to the cache, all the stale BindRecords are // removed from the cache after this operation. -func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { +func (h *BindHandle) appendBindRecord(hash string, meta *BindRecord) { newCache := h.bindInfo.Value.Load().(cache).copy() - newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal) - newCache[hash] = append(newCache[hash], meta) - meta.updateMetrics(metrics.ScopeGlobal, true) + oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) + newRecord := merge(oldRecord, meta) + newCache.setBindRecord(hash, meta) h.bindInfo.Value.Store(newCache) + updateMetrics(metrics.ScopeGlobal, oldRecord, newRecord, false) } -// removeBindMeta removes the BindMeta from the cache. -func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) { +// removeBindRecord removes the BindRecord from the cache. +func (h *BindHandle) removeBindRecord(hash string, meta *BindRecord) error { h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() + oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) defer func() { h.bindInfo.Value.Store(newCache) h.bindInfo.Unlock() + updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db), false) }() - newCache.removeDeletedBindMeta(hash, meta, metrics.ScopeGlobal) + newCache.removeDeletedBindRecord(hash, meta) + return nil } -// 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) { +// removeDeletedBindRecord removes all the BindRecord which originSQL and db are the same with the parameter's meta. +func (c cache) removeDeletedBindRecord(hash string, meta *BindRecord) { metas, ok := c[hash] if !ok { return @@ -356,8 +377,10 @@ func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) for i := len(metas) - 1; i >= 0; i-- { if metas[i].isSame(meta) { - metas[i].updateMetrics(scope, false) - metas = append(metas[:i], metas[i+1:]...) + metas[i] = metas[i].remove(meta) + if len(metas[i].Bindings) == 0 { + metas = append(metas[:i], metas[i+1:]...) + } if len(metas) == 0 { delete(c, hash) return @@ -366,23 +389,15 @@ func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) } } -// removeStaleBindMetas removes all the stale BindMeta in the cache. -func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) { - metas, ok := c[hash] - if !ok { - return - } - - 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) - return - } +func (c cache) setBindRecord(hash string, meta *BindRecord) { + metas := c[hash] + for i := range metas { + if metas[i].Db == meta.Db && metas[i].OriginalSQL == meta.OriginalSQL { + metas[i] = meta + return } } + c[hash] = append(c[hash], meta) } func (c cache) copy() cache { @@ -401,7 +416,7 @@ func copyInvalidBindRecordMap(oldMap map[string]*invalidBindRecordMap) map[strin return newMap } -func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindMeta { +func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindRecord { bindRecords := c[hash] if bindRecords != nil { for _, bindRecord := range bindRecords { @@ -413,16 +428,6 @@ func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindMeta { return nil } -// isStale checks whether this BindMeta is stale compared with the other BindMeta. -func (m *BindMeta) isStale(other *BindMeta) bool { - return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db && - m.UpdateTime.Compare(other.UpdateTime) <= 0 -} - -func (m *BindMeta) isSame(other *BindMeta) bool { - return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db -} - func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db string) string { return fmt.Sprintf( "DELETE FROM mysql.bind_info WHERE original_sql='%s' AND default_db='%s'", @@ -431,25 +436,33 @@ func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db string) string { ) } -func (h *BindHandle) insertBindInfoSQL(record *BindRecord) string { +func (h *BindHandle) insertBindInfoSQL(orignalSQL string, db string, info Binding) string { return fmt.Sprintf(`INSERT INTO mysql.bind_info VALUES ('%s', '%s', '%s', '%s', '%s', '%s','%s', '%s')`, - record.OriginalSQL, - record.BindSQL, - record.Db, - record.Status, - record.CreateTime, - record.UpdateTime, - record.Charset, - record.Collation, + orignalSQL, + info.BindSQL, + db, + info.Status, + info.CreateTime, + info.UpdateTime, + info.Charset, + info.Collation, ) } -func (h *BindHandle) logicalDeleteBindInfoSQL(normdOrigSQL, db string, updateTs types.Time) string { - return fmt.Sprintf(`UPDATE mysql.bind_info SET status='%s',update_time='%s' WHERE original_sql='%s' and default_db='%s'`, +func (h *BindHandle) logicalDeleteBindInfoSQL(record *BindRecord, updateTs types.Time) string { + sql := fmt.Sprintf(`UPDATE mysql.bind_info SET status='%s',update_time='%s' WHERE original_sql='%s' and default_db='%s'`, deleted, updateTs, - normdOrigSQL, - db) + record.OriginalSQL, + record.Db) + if len(record.Bindings) == 0 { + return sql + } + bindings := make([]string, 0, len(record.Bindings)) + for _, bind := range record.Bindings { + bindings = append(bindings, fmt.Sprintf(`'%s'`, bind.BindSQL)) + } + return sql + fmt.Sprintf(" and bind_sql in (%s)", strings.Join(bindings, ",")) } // GenHintsFromSQL is used to generate hints from SQL. @@ -457,7 +470,7 @@ func (h *BindHandle) logicalDeleteBindInfoSQL(normdOrigSQL, db string, updateTs var GenHintsFromSQL func(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (string, error) // CaptureBaselines is used to automatically capture plan baselines. -func (h *BindHandle) CaptureBaselines(is infoschema.InfoSchema) { +func (h *BindHandle) CaptureBaselines() { schemas, sqls := stmtsummary.StmtSummaryByDigestMap.GetMoreThanOnceSelect() for i := range sqls { stmt, err := h.parser4Baseline.ParseOneStmt(sqls[i], "", "") @@ -466,7 +479,7 @@ func (h *BindHandle) CaptureBaselines(is infoschema.InfoSchema) { continue } normalizedSQL, digiest := parser.NormalizeDigest(sqls[i]) - if r := h.GetBindRecord(digiest, normalizedSQL, schemas[i]); r != nil && r.Status == Using { + if r := h.GetBindRecord(digiest, normalizedSQL, schemas[i]); r != nil && r.FirstUsingBinding() != nil { continue } h.sctx.Lock() @@ -474,7 +487,7 @@ func (h *BindHandle) CaptureBaselines(is infoschema.InfoSchema) { var hints string if err == nil { h.sctx.GetSessionVars().CurrentDB = schemas[i] - hints, err = GenHintsFromSQL(context.TODO(), h.sctx.Context, stmt, is) + hints, err = GenHintsFromSQL(context.TODO(), h.sctx.Context, stmt, h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema)) } h.sctx.Unlock() if err != nil { @@ -486,7 +499,14 @@ func (h *BindHandle) CaptureBaselines(is infoschema.InfoSchema) { continue } bindsql := strings.Replace(normalizedSQL, "select", fmt.Sprintf("select /*+ %s*/", hints), 1) - err = h.AddBindRecord(&BindRecord{OriginalSQL: sqls[i], BindSQL: bindsql, Db: schemas[i], Status: Using}) + binding := Binding{ + BindSQL: bindsql, + Status: Using, + Hint: CollectHint(stmt), + id: hints, + } + // We don't need to pass the `sctx` and `is` because they are used to generate hints and we already filled hints in. + err = h.AddBindRecord(nil, nil, &BindRecord{OriginalSQL: sqls[i], Db: schemas[i], Bindings: []Binding{binding}}) if err != nil { logutil.BgLogger().Info("capture baseline failed", zap.String("SQL", sqls[i]), zap.Error(err)) } diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index 5b41ae6be2..321fa5ed8d 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -18,7 +18,9 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" ) @@ -35,53 +37,50 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { return sessionHandle } -// appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are +// appendBindRecord adds the BindRecord 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) appendBindRecord(hash string, meta *BindRecord) { // 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) - meta.updateMetrics(metrics.ScopeSession, true) -} - -func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { - hash = parser.DigestHash(record.OriginalSQL) - stmtNodes, _, err := h.parser.Parse(record.BindSQL, record.Charset, record.Collation) - if err != nil { - return "", nil, err - } - meta = &BindMeta{BindRecord: record, HintsSet: CollectHint(stmtNodes[0])} - return hash, meta, nil + oldRecord := h.ch.getBindRecord(hash, meta.OriginalSQL, meta.Db) + newRecord := merge(oldRecord, meta) + h.ch.setBindRecord(hash, newRecord) + updateMetrics(metrics.ScopeSession, oldRecord, newRecord, false) } // AddBindRecord new a BindRecord with BindMeta, add it to the cache. -func (h *SessionHandle) AddBindRecord(record *BindRecord) error { - record.CreateTime = types.Time{ - Time: types.FromGoTime(time.Now()), - Type: mysql.TypeDatetime, - Fsp: 3, +func (h *SessionHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSchema, record *BindRecord) error { + for i := range record.Bindings { + record.Bindings[i].CreateTime = types.Time{ + Time: types.FromGoTime(time.Now()), + Type: mysql.TypeDatetime, + Fsp: 3, + } + record.Bindings[i].UpdateTime = record.Bindings[i].CreateTime } - record.UpdateTime = record.CreateTime + err := record.prepareHintsForUsing(sctx, is) // update the BindMeta to the cache. - hash, meta, err := h.newBindMeta(record) if err == nil { - h.appendBindMeta(hash, meta) + h.appendBindRecord(parser.DigestHash(record.OriginalSQL), record) } return err } // DropBindRecord drops a BindRecord in the cache. func (h *SessionHandle) DropBindRecord(record *BindRecord) { - meta := &BindMeta{BindRecord: record} - meta.Status = deleted - hash := parser.DigestHash(record.OriginalSQL) - h.ch.removeDeletedBindMeta(hash, meta, metrics.ScopeSession) - h.appendBindMeta(hash, meta) + oldRecord := h.GetBindRecord(record.OriginalSQL, record.Db) + var newRecord *BindRecord + if oldRecord != nil { + newRecord = oldRecord.remove(record) + } else { + newRecord = record + } + h.ch.setBindRecord(parser.DigestHash(record.OriginalSQL), newRecord) + updateMetrics(metrics.ScopeSession, oldRecord, newRecord, false) } // GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. -func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { +func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindRecord { hash := parser.DigestHash(normdOrigSQL) bindRecords := h.ch[hash] if bindRecords != nil { @@ -95,7 +94,7 @@ func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { } // GetAllBindRecord return all session bind info. -func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { +func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindRecord) { for _, bindRecord := range h.ch { bindRecords = append(bindRecords, bindRecord...) } @@ -106,7 +105,7 @@ func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { func (h *SessionHandle) Close() { for _, bindRecords := range h.ch { for _, bindRecord := range bindRecords { - bindRecord.updateMetrics(metrics.ScopeSession, false) + updateMetrics(metrics.ScopeSession, bindRecord, nil, false) } } } diff --git a/domain/db_test.go b/domain/db_test.go index 2b6c78a68e..ca0b3ef737 100644 --- a/domain/db_test.go +++ b/domain/db_test.go @@ -54,7 +54,7 @@ func (ts *dbTestSuite) TestIntegration(c *C) { // for BindHandle se.Execute(context.Background(), "use test") se.Execute(context.Background(), "drop table if exists t") - se.Execute(context.Background(), "create table t(i int, s varchar(20), index(i, s))") + se.Execute(context.Background(), "create table t(i int, s varchar(20), index index_t(i, s))") _, err = se.Execute(context.Background(), "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)) } diff --git a/domain/domain.go b/domain/domain.go index 0d5aeb11ce..67a9d2358e 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -850,7 +850,7 @@ func (do *Domain) globalBindHandleWorkerLoop() { if !variable.TiDBOptOn(variable.CapturePlanBaseline.GetVal()) { continue } - do.bindHandle.CaptureBaselines(do.InfoSchema()) + do.bindHandle.CaptureBaselines() } } }() diff --git a/executor/bind.go b/executor/bind.go index da007420ca..49c065da57 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -64,17 +64,20 @@ func (e *SQLBindExec) dropSQLBind() error { } func (e *SQLBindExec) createSQLBind() error { + bindInfo := bindinfo.Binding{ + BindSQL: e.bindSQL, + Charset: e.charset, + Collation: e.collation, + Status: bindinfo.Using, + } record := &bindinfo.BindRecord{ OriginalSQL: e.normdOrigSQL, - BindSQL: e.bindSQL, Db: e.ctx.GetSessionVars().CurrentDB, - Charset: e.charset, - Collation: e.collation, - Status: bindinfo.Using, + Bindings: []bindinfo.Binding{bindInfo}, } if !e.isGlobal { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - return handle.AddBindRecord(record) + return handle.AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record) } - return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(record) + return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record) } diff --git a/executor/show.go b/executor/show.go index 57832f392f..59ad17b51b 100644 --- a/executor/show.go +++ b/executor/show.go @@ -187,7 +187,7 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { } func (e *ShowExec) fetchShowBind() error { - var bindRecords []*bindinfo.BindMeta + var bindRecords []*bindinfo.BindRecord if !e.GlobalScope { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) bindRecords = handle.GetAllBindRecord() @@ -195,16 +195,18 @@ func (e *ShowExec) fetchShowBind() error { bindRecords = domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() } for _, bindData := range bindRecords { - e.appendRow([]interface{}{ - bindData.OriginalSQL, - bindData.BindSQL, - bindData.Db, - bindData.Status, - bindData.CreateTime, - bindData.UpdateTime, - bindData.Charset, - bindData.Collation, - }) + for _, hint := range bindData.Bindings { + e.appendRow([]interface{}{ + bindData.OriginalSQL, + hint.BindSQL, + bindData.Db, + hint.Status, + hint.CreateTime, + hint.UpdateTime, + hint.Charset, + hint.Collation, + }) + } } return nil } diff --git a/planner/optimize.go b/planner/optimize.go index 4fd0c74772..b6b7171017 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -146,15 +146,14 @@ func addHintForSelect(ctx sessionctx.Context, stmt ast.StmtNode, normdOrigSQL, h sessionHandle := ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) bindRecord := sessionHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) if bindRecord != nil { - if bindRecord.Status == bindinfo.Invalid { + binding := bindRecord.FirstUsingBinding() + if binding == nil { return nil } - if bindRecord.Status == bindinfo.Using { - metrics.BindUsageCounter.WithLabelValues(metrics.ScopeSession).Inc() - oriHint := bindinfo.CollectHint(stmt) - bindinfo.BindHint(stmt, bindRecord.HintsSet) - return oriHint - } + metrics.BindUsageCounter.WithLabelValues(metrics.ScopeSession).Inc() + oriHint := bindinfo.CollectHint(stmt) + bindinfo.BindHint(stmt, binding.Hint) + return oriHint } globalHandle := domain.GetDomain(ctx).BindHandle() bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, ctx.GetSessionVars().CurrentDB) @@ -164,7 +163,7 @@ func addHintForSelect(ctx sessionctx.Context, stmt ast.StmtNode, normdOrigSQL, h if bindRecord != nil { metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Inc() oriHint := bindinfo.CollectHint(stmt) - bindinfo.BindHint(stmt, bindRecord.HintsSet) + bindinfo.BindHint(stmt, bindRecord.FirstUsingBinding().Hint) return oriHint } return nil @@ -176,36 +175,36 @@ func handleInvalidBindRecord(ctx context.Context, sctx sessionctx.Context, stmtN return } sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - bindMeta := sessionHandle.GetBindRecord(normdOrigSQL, sctx.GetSessionVars().CurrentDB) - if bindMeta != nil { - bindMeta.Status = bindinfo.Invalid + bindRecord := sessionHandle.GetBindRecord(normdOrigSQL, sctx.GetSessionVars().CurrentDB) + if bindRecord != nil { + bindRecord.FirstUsingBinding().Status = bindinfo.Invalid return } globalHandle := domain.GetDomain(sctx).BindHandle() - bindMeta = globalHandle.GetBindRecord(hash, normdOrigSQL, sctx.GetSessionVars().CurrentDB) - if bindMeta == nil { - bindMeta = globalHandle.GetBindRecord(hash, normdOrigSQL, "") + bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, sctx.GetSessionVars().CurrentDB) + if bindRecord == nil { + bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, "") } - if bindMeta != nil { + if bindRecord != nil { + binding := *bindRecord.FirstUsingBinding() + binding.Status = bindinfo.Invalid record := &bindinfo.BindRecord{ - OriginalSQL: bindMeta.OriginalSQL, - BindSQL: bindMeta.BindSQL, + OriginalSQL: bindRecord.OriginalSQL, Db: sctx.GetSessionVars().CurrentDB, - Charset: bindMeta.Charset, - Collation: bindMeta.Collation, - Status: bindinfo.Invalid, + Bindings: []bindinfo.Binding{binding}, } - err := sessionHandle.AddBindRecord(record) + err := sessionHandle.AddBindRecord(nil, nil, record) if err != nil { logutil.Logger(ctx).Warn("handleInvalidBindRecord failed", zap.Error(err)) } globalHandle := domain.GetDomain(sctx).BindHandle() dropBindRecord := &bindinfo.BindRecord{ - OriginalSQL: bindMeta.OriginalSQL, - Db: bindMeta.Db, + OriginalSQL: bindRecord.OriginalSQL, + Db: bindRecord.Db, + Bindings: []bindinfo.Binding{binding}, } globalHandle.AddDropInvalidBindTask(dropBindRecord) } @@ -248,7 +247,11 @@ func GenHintsFromSQL(ctx context.Context, sctx sessionctx.Context, node ast.Node if err != nil { return "", err } + oldValue := sctx.GetSessionVars().UsePlanBaselines + // Disable baseline to avoid binding hints. + sctx.GetSessionVars().UsePlanBaselines = false p, err := Optimize(ctx, sctx, node, is) + sctx.GetSessionVars().UsePlanBaselines = oldValue if err != nil { return "", err }