bindinfo: support multiply bindings for a sql (#12851)
This commit is contained in:
committed by
pingcap-github-bot
parent
adcd0ac7bf
commit
bb2e349fea
@ -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")
|
||||
|
||||
@ -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]))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -850,7 +850,7 @@ func (do *Domain) globalBindHandleWorkerLoop() {
|
||||
if !variable.TiDBOptOn(variable.CapturePlanBaseline.GetVal()) {
|
||||
continue
|
||||
}
|
||||
do.bindHandle.CaptureBaselines(do.InfoSchema())
|
||||
do.bindHandle.CaptureBaselines()
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user