bindinfo: support multiply bindings for a sql (#12851)

This commit is contained in:
Haibin Xie
2019-10-29 14:40:23 +08:00
committed by pingcap-github-bot
parent adcd0ac7bf
commit bb2e349fea
9 changed files with 394 additions and 250 deletions

View File

@ -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")

View File

@ -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]))
}
}
}

View File

@ -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))
}

View File

@ -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)
}
}
}

View File

@ -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))
}

View File

@ -850,7 +850,7 @@ func (do *Domain) globalBindHandleWorkerLoop() {
if !variable.TiDBOptOn(variable.CapturePlanBaseline.GetVal()) {
continue
}
do.bindHandle.CaptureBaselines(do.InfoSchema())
do.bindHandle.CaptureBaselines()
}
}
}()

View File

@ -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)
}

View File

@ -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
}

View File

@ -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
}