Skip to content

Commit

Permalink
Merge branch 'master' into insert-sub-query-throw-error
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Dec 28, 2021
2 parents eacbe2c + 57b2908 commit 550bd1d
Show file tree
Hide file tree
Showing 7 changed files with 110 additions and 32 deletions.
11 changes: 11 additions & 0 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,17 @@ func (br *BindRecord) HasUsingBinding() bool {
return false
}

// FindUsingBinding gets the using binding.
// There is at most one binding that can be used now
func (br *BindRecord) FindUsingBinding() *Binding {
for _, binding := range br.Bindings {
if binding.Status == Using {
return &binding
}
}
return nil
}

// FindBinding find bindings in BindRecord.
func (br *BindRecord) FindBinding(hint string) *Binding {
for i := range br.Bindings {
Expand Down
23 changes: 17 additions & 6 deletions executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,10 +200,19 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, vars.InRestrictedSQL)
}

var (
normalizedSQL4PC, normalizedSQL4PCHash string
selectStmtNode ast.StmtNode
)
if !plannercore.PreparedPlanCacheEnabled() {
prepared.UseCache = false
} else {
prepared.UseCache = plannercore.CacheableWithCtx(e.ctx, stmt, ret.InfoSchema)
selectStmtNode, normalizedSQL4PC, normalizedSQL4PCHash, err = planner.ExtractSelectAndNormalizeDigest(stmt, e.ctx.GetSessionVars().CurrentDB)
if err != nil || selectStmtNode == nil {
normalizedSQL4PC = ""
normalizedSQL4PCHash = ""
}
}

// We try to build the real statement of preparedStmt.
Expand Down Expand Up @@ -231,12 +240,14 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
}

preparedObj := &plannercore.CachedPrepareStmt{
PreparedAst: prepared,
VisitInfos: destBuilder.GetVisitInfo(),
NormalizedSQL: normalizedSQL,
SQLDigest: digest,
ForUpdateRead: destBuilder.GetIsForUpdateRead(),
SnapshotTSEvaluator: ret.SnapshotTSEvaluator,
PreparedAst: prepared,
VisitInfos: destBuilder.GetVisitInfo(),
NormalizedSQL: normalizedSQL,
SQLDigest: digest,
ForUpdateRead: destBuilder.GetIsForUpdateRead(),
SnapshotTSEvaluator: ret.SnapshotTSEvaluator,
NormalizedSQL4PC: normalizedSQL4PC,
NormalizedSQL4PCHash: normalizedSQL4PCHash,
}
return vars.AddPreparedStmt(e.ID, preparedObj)
}
Expand Down
34 changes: 34 additions & 0 deletions executor/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1372,3 +1372,37 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) {
require.Equal(t, "select * from `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL)
require.Equal(t, "", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedPlan)
}

func TestPreparePC4Binding(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
orgEnable := plannercore.PreparedPlanCacheEnabled()
defer func() {
plannercore.SetPreparedPlanCache(orgEnable)
}()
plannercore.SetPreparedPlanCache(true) // requires plan cache enable
tk := testkit.NewTestKit(t, store)
tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890"))
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")

tk.MustExec("prepare stmt from \"select * from t\"")
require.Equal(t, 1, len(tk.Session().GetSessionVars().PreparedStmts))
require.Equal(t, "select * from `test` . `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL4PC)

tk.MustQuery("execute stmt")
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0"))
tk.MustQuery("execute stmt")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustExec("create binding for select * from t using select * from t")
res := tk.MustQuery("show session bindings")
require.Equal(t, 1, len(res.Rows()))

tk.MustQuery("execute stmt")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
tk.MustQuery("execute stmt")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("execute stmt")
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))
}
22 changes: 12 additions & 10 deletions planner/core/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,16 +206,18 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta

// CachedPrepareStmt store prepared ast from PrepareExec and other related fields
type CachedPrepareStmt struct {
PreparedAst *ast.Prepared
VisitInfos []visitInfo
ColumnInfos interface{}
Executor interface{}
NormalizedSQL string
NormalizedPlan string
SQLDigest *parser.Digest
PlanDigest *parser.Digest
ForUpdateRead bool
SnapshotTSEvaluator func(sessionctx.Context) (uint64, error)
PreparedAst *ast.Prepared
VisitInfos []visitInfo
ColumnInfos interface{}
Executor interface{}
NormalizedSQL string
NormalizedPlan string
SQLDigest *parser.Digest
PlanDigest *parser.Digest
ForUpdateRead bool
SnapshotTSEvaluator func(sessionctx.Context) (uint64, error)
NormalizedSQL4PC string
NormalizedSQL4PCHash string
}

// GetPreparedStmt extract the prepared statement from the execute statement.
Expand Down
34 changes: 33 additions & 1 deletion planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -390,6 +391,37 @@ func (e *Execute) setFoundInPlanCache(sctx sessionctx.Context, opt bool) error {
return err
}

// GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key.
func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareStmt) string {
useBinding := sctx.GetSessionVars().UsePlanBaselines
if !useBinding || preparedStmt.PreparedAst.Stmt == nil || preparedStmt.NormalizedSQL4PC == "" || preparedStmt.NormalizedSQL4PCHash == "" {
return ""
}
if sctx.Value(bindinfo.SessionBindInfoKeyType) == nil {
return ""
}
sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
bindRecord := sessionHandle.GetBindRecord(preparedStmt.NormalizedSQL4PC, "")
if bindRecord != nil {
usingBinding := bindRecord.FindUsingBinding()
if usingBinding != nil {
return usingBinding.BindSQL
}
}
globalHandle := domain.GetDomain(sctx).BindHandle()
if globalHandle == nil {
return ""
}
bindRecord = globalHandle.GetBindRecord(preparedStmt.NormalizedSQL4PCHash, preparedStmt.NormalizedSQL4PC, "")
if bindRecord != nil {
usingBinding := bindRecord.FindUsingBinding()
if usingBinding != nil {
return usingBinding.BindSQL
}
}
return ""
}

func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt) error {
var cacheKey kvcache.Key
sessVars := sctx.GetSessionVars()
Expand All @@ -413,7 +445,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context,

var bindSQL string
if prepared.UseCache {
bindSQL = GetBindSQL4PlanCache(sctx, prepared.Stmt)
bindSQL = GetBindSQL4PlanCache(sctx, preparedStmt)
cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion)
}
tps := make([]*types.FieldType, len(e.UsingVars))
Expand Down
3 changes: 0 additions & 3 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,6 @@ import (
// OptimizeAstNode optimizes the query to a physical plan directly.
var OptimizeAstNode func(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (Plan, types.NameSlice, error)

// GetBindSQL4PlanCache get the bindSQL for the ast.StmtNode
var GetBindSQL4PlanCache func(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindSQL string)

// AllowCartesianProduct means whether tidb allows cartesian join without equal conditions.
var AllowCartesianProduct = atomic.NewBool(true)

Expand Down
15 changes: 3 additions & 12 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,15 +80,6 @@ func GetExecuteForUpdateReadIS(node ast.Node, sctx sessionctx.Context) infoschem
return nil
}

// GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key.
func GetBindSQL4PlanCache(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindSQL string) {
bindRecord, _, match := matchSQLBinding(sctx, stmtNode)
if match {
bindSQL = bindRecord.Bindings[0].BindSQL
}
return bindSQL
}

func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindRecord *bindinfo.BindRecord, scope string, matched bool) {
useBinding := sctx.GetSessionVars().UsePlanBaselines
if !useBinding || stmtNode == nil {
Expand Down Expand Up @@ -391,7 +382,8 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
return finalPlan, names, cost, err
}

func extractSelectAndNormalizeDigest(stmtNode ast.StmtNode, specifiledDB string) (ast.StmtNode, string, string, error) {
// ExtractSelectAndNormalizeDigest extract the select statement and normalize it.
func ExtractSelectAndNormalizeDigest(stmtNode ast.StmtNode, specifiledDB string) (ast.StmtNode, string, string, error) {
switch x := stmtNode.(type) {
case *ast.ExplainStmt:
// This function is only used to find bind record.
Expand Down Expand Up @@ -446,7 +438,7 @@ func getBindRecord(ctx sessionctx.Context, stmt ast.StmtNode) (*bindinfo.BindRec
if ctx.Value(bindinfo.SessionBindInfoKeyType) == nil {
return nil, "", nil
}
stmtNode, normalizedSQL, hash, err := extractSelectAndNormalizeDigest(stmt, ctx.GetSessionVars().CurrentDB)
stmtNode, normalizedSQL, hash, err := ExtractSelectAndNormalizeDigest(stmt, ctx.GetSessionVars().CurrentDB)
if err != nil || stmtNode == nil {
return nil, "", err
}
Expand Down Expand Up @@ -700,6 +692,5 @@ func setFoundInBinding(sctx sessionctx.Context, opt bool, bindSQL string) error

func init() {
plannercore.OptimizeAstNode = Optimize
plannercore.GetBindSQL4PlanCache = GetBindSQL4PlanCache
plannercore.IsReadOnly = IsReadOnly
}

0 comments on commit 550bd1d

Please sign in to comment.