Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

planner: prepared plan cache support cached plan with placeholder in limit clause #40196

Merged
merged 48 commits into from
Jan 17, 2023
Merged
Show file tree
Hide file tree
Changes from 14 commits
Commits
Show all changes
48 commits
Select commit Hold shift + click to select a range
eba19fe
commit
fzzf678 Dec 27, 2022
47c4f51
pass stmtNode
fzzf678 Dec 27, 2022
544fb69
fix ut
fzzf678 Dec 28, 2022
9ce7ca2
fix ut
fzzf678 Dec 28, 2022
7d90645
Merge branch 'master' into planCache_limit
fzzf678 Dec 28, 2022
505e9c2
Merge remote-tracking branch 'upstream/master' into planCache_limit
fzzf678 Dec 28, 2022
2c8f7b6
Merge branch 'planCache_limit' of https://github.com/fzzf678/tidb int…
fzzf678 Dec 28, 2022
9a759ee
commit
fzzf678 Dec 28, 2022
83f7a9d
Update plan_cache_utils.go
fzzf678 Dec 28, 2022
b5f17a4
safe value
fzzf678 Dec 28, 2022
07f327f
Merge branch 'master' into planCache_limit
fzzf678 Dec 28, 2022
a7a5844
fix
fzzf678 Dec 28, 2022
d6482e4
Merge branch 'planCache_limit' of https://github.com/fzzf678/tidb int…
fzzf678 Dec 28, 2022
6188f82
Update plan_cache_test.go
fzzf678 Dec 28, 2022
b5b90ed
Merge branch 'master' into planCache_limit
fzzf678 Jan 3, 2023
fbe23e5
unify error message
fzzf678 Jan 3, 2023
190bc29
check limit argument
fzzf678 Jan 3, 2023
f1f73db
unify warning message
fzzf678 Jan 3, 2023
b75e63e
Merge remote-tracking branch 'upstream/master' into planCache_limit
fzzf678 Jan 3, 2023
ed829d1
fix ut
fzzf678 Jan 3, 2023
8244f14
revert
fzzf678 Jan 4, 2023
03cee83
Merge remote-tracking branch 'upstream/master' into planCache_limit
fzzf678 Jan 4, 2023
8258da0
only_int
fzzf678 Jan 4, 2023
51d06a4
Update plan_cache_utils.go
fzzf678 Jan 4, 2023
5d83cea
only int in limit stmt
fzzf678 Jan 5, 2023
aae94e1
fix
fzzf678 Jan 6, 2023
0d6335a
replace int by uint in cache key
fzzf678 Jan 9, 2023
ab99bb7
Merge branch 'master' into planCache_limit
fzzf678 Jan 9, 2023
4305ba1
Update plan_cache_utils_test.go
fzzf678 Jan 9, 2023
cc040cb
Merge branch 'planCache_limit' of https://github.com/fzzf678/tidb int…
fzzf678 Jan 9, 2023
61b1547
Update prepared_test.go
fzzf678 Jan 9, 2023
cd6ca23
Update prepared_test.go
fzzf678 Jan 9, 2023
e921a81
Update plan_cache_test.go
fzzf678 Jan 9, 2023
9c4df35
Update plan_cache_utils.go
fzzf678 Jan 9, 2023
d9eadbd
Merge remote-tracking branch 'upstream/master' into planCache_limit
fzzf678 Jan 11, 2023
9dc74fb
use exist function
fzzf678 Jan 11, 2023
de1d441
Update plan_cache_test.go
fzzf678 Jan 12, 2023
4c05386
move limit params
fzzf678 Jan 13, 2023
67f83dc
Update plan_cache_lru_test.go
fzzf678 Jan 13, 2023
18c5b47
Merge branch 'master' into planCache_limit
fzzf678 Jan 13, 2023
a9ba0e6
fix ut
fzzf678 Jan 13, 2023
e1cf8a0
Merge branch 'planCache_limit' of https://github.com/fzzf678/tidb int…
fzzf678 Jan 13, 2023
e1b33a4
Update plan_cache_lru.go
fzzf678 Jan 13, 2023
8bf7f8b
Merge branch 'master' into planCache_limit
fzzf678 Jan 13, 2023
5a985d7
Merge branch 'master' into planCache_limit
fzzf678 Jan 13, 2023
64c82da
Merge branch 'master' into planCache_limit
fzzf678 Jan 13, 2023
b877d21
fix
fzzf678 Jan 16, 2023
54edf40
Merge branch 'master' into planCache_limit
ti-chi-bot Jan 17, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,8 +204,9 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error {
delete(vars.PreparedStmtNameToID, e.Name)
if e.ctx.GetSessionVars().EnablePreparedPlanCache {
bindSQL, _ := plannercore.GetBindSQL4PlanCache(e.ctx, preparedObj)
limitOffsetAndCount, _ := plannercore.ExtractLimitFromAst(prepared.Stmt)
cacheKey, err := plannercore.NewPlanCacheKey(vars, preparedObj.StmtText, preparedObj.StmtDB, prepared.SchemaVersion,
0, bindSQL)
0, bindSQL, limitOffsetAndCount)
if err != nil {
return err
}
Expand Down
12 changes: 9 additions & 3 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,

var bindSQL string
var ignorePlanCache = false
var canBeCached = true

// In rc or for update read, we need the latest schema version to decide whether we need to
// rebuild the plan. So we set this value in rc or for update read. In other cases, let it be 0.
Expand All @@ -144,8 +145,12 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
// up-to-date schema version which can lead plan cache miss and thus, the plan will be rebuilt.
latestSchemaVersion = domain.GetDomain(sctx).InfoSchema().SchemaMetaVersion()
}
var limitOffsetAndCount []int64
if limitOffsetAndCount, canBeCached = ExtractLimitFromAst(stmt.PreparedAst.Stmt); !canBeCached {
sctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("plan with limit count more than 10000 can't be cached"))
qw4990 marked this conversation as resolved.
Show resolved Hide resolved
}
if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), stmt.StmtText,
stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil {
stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL, limitOffsetAndCount); err != nil {
return nil, nil, err
}
}
Expand All @@ -165,7 +170,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
}
}

return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, ignorePlanCache, cacheKey,
return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, ignorePlanCache && canBeCached, cacheKey,
latestSchemaVersion, paramNum, paramTypes, bindSQL)
}

Expand Down Expand Up @@ -284,8 +289,9 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
// rebuild key to exclude kv.TiFlash when stmt is not read only
if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) {
delete(sessVars.IsolationReadEngines, kv.TiFlash)
limitOffsetAndCount, _ := ExtractLimitFromAst(stmt.PreparedAst.Stmt)
if cacheKey, err = NewPlanCacheKey(sessVars, stmt.StmtText, stmt.StmtDB,
stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil {
stmtAst.SchemaVersion, latestSchemaVersion, bindSQL, limitOffsetAndCount); err != nil {
return nil, nil, err
}
sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{}
Expand Down
51 changes: 45 additions & 6 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -283,12 +283,6 @@ func TestPlanCacheDiagInfo(t *testing.T) {
tk.MustExec("prepare stmt from 'select /*+ ignore_plan_cache() */ * from t'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: ignore plan cache by hint"))

tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable"))

tk.MustExec("prepare stmt from 'select * from t limit ?, 1'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?, 10' is un-cacheable"))

tk.MustExec("prepare stmt from 'select * from t order by ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'order by ?' is un-cacheable"))

Expand All @@ -302,3 +296,48 @@ func TestPlanCacheDiagInfo(t *testing.T) {
tk.MustExec("execute stmt using @a, @b") // a=1 and a=1 -> a=1
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: some parameters may be overwritten"))
}

func TestPlanCacheWithLimit(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, key(a))")

testCases := []struct {
sql string
params []int
}{
{"prepare stmt from 'select * from t limit ?'", []int{1}},
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add some cases using PointGet + Limit.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All fixed, PTAL again

{"prepare stmt from 'select * from t limit ?, ?'", []int{1, 2}},
{"prepare stmt from 'delete from t order by a limit ?'", []int{1}},
{"prepare stmt from 'insert into t select * from t order by a desc limit ?'", []int{1}},
{"prepare stmt from 'insert into t select * from t order by a desc limit ?, ?'", []int{1, 2}},
{"prepare stmt from 'update t set a = 1 limit ?'", []int{1}},
{" prepare stmt from '(select * from t order by a limit ?) union (select * from t order by a desc limit ?)';", []int{1, 2}},
}

for _, testCase := range testCases {
tk.MustExec(testCase.sql)
tk.MustExec("set @a = 1")
var using []string
for i, p := range testCase.params {
tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p))
using = append(using, fmt.Sprintf("@a%d", i))
}

tk.MustExec("execute stmt using " + strings.Join(using, ", "))
tk.MustExec("execute stmt using " + strings.Join(using, ", "))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustExec("set @a0 = 10086")
tk.MustExec("execute stmt using " + strings.Join(using, ", "))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
}

// error case
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustExec("set @a = 10001")
tk.MustExec("execute stmt using @a")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 plan with limit count more than 10000 can't be cached"))
}
52 changes: 51 additions & 1 deletion planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,7 @@ type planCacheKey struct {
inRestrictedSQL bool
restrictedReadOnly bool
TiDBSuperReadOnly bool
limitOffsetAndCount []int64

memoryUsage int64 // Do not include in hash
hash []byte
Expand Down Expand Up @@ -226,6 +227,9 @@ func (key *planCacheKey) Hash() []byte {
key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.inRestrictedSQL))...)
key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.restrictedReadOnly))...)
key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.TiDBSuperReadOnly))...)
for _, l := range key.limitOffsetAndCount {
key.hash = codec.EncodeInt(key.hash, l)
}
}
return key.hash
}
Expand Down Expand Up @@ -267,7 +271,7 @@ func SetPstmtIDSchemaVersion(key kvcache.Key, stmtText string, schemaVersion int
// Note: lastUpdatedSchemaVersion will only be set in the case of rc or for update read in order to
// differentiate the cache key. In other cases, it will be 0.
func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, schemaVersion int64,
lastUpdatedSchemaVersion int64, bindSQL string) (kvcache.Key, error) {
lastUpdatedSchemaVersion int64, bindSQL string, offsetAndCount []int64) (kvcache.Key, error) {
if stmtText == "" {
return nil, errors.New("no statement text")
}
Expand Down Expand Up @@ -295,6 +299,7 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string,
inRestrictedSQL: sessionVars.InRestrictedSQL,
restrictedReadOnly: variable.RestrictedReadOnly.Load(),
TiDBSuperReadOnly: variable.VarTiDBSuperReadOnly.Load(),
limitOffsetAndCount: offsetAndCount,
}
for k, v := range sessionVars.IsolationReadEngines {
key.isolationReadEngines[k] = v
Expand Down Expand Up @@ -445,3 +450,48 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa
}
return nil, ErrStmtNotFound
}

type limitExtractor struct {
cacheable bool // For safety considerations, check if limit count less than 10000
offsetAndCount []int64
}

// Enter implements Visitor interface.
func (checker *limitExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch node := in.(type) {
case *ast.Limit:
if node.Count != nil {
if count, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker {
countNum := count.GetInt64()
qw4990 marked this conversation as resolved.
Show resolved Hide resolved
checker.offsetAndCount = append(checker.offsetAndCount, countNum)
if countNum > 10000 {
checker.cacheable = false
}
}
}
if node.Offset != nil {
if offset, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker {
checker.offsetAndCount = append(checker.offsetAndCount, offset.GetInt64())
}
}
}
return in, false
}

// Leave implements Visitor interface.
func (checker *limitExtractor) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, checker.cacheable
}

// ExtractLimitFromAst extract limit offset and count from ast for plan cache key encode
func ExtractLimitFromAst(node ast.Node) ([]int64, bool) {
if node == nil {
return []int64{}, true
}
checker := limitExtractor{
cacheable: true,
offsetAndCount: []int64{},
}
node.Accept(&checker)
return checker.offsetAndCount, checker.cacheable
}
8 changes: 4 additions & 4 deletions planner/core/plan_cache_utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,20 +32,20 @@ func TestCacheKey(t *testing.T) {
ctx.GetSessionVars().InRestrictedSQL = false
variable.RestrictedReadOnly.Store(false)
variable.VarTiDBSuperReadOnly.Store(false)
key, err := NewPlanCacheKey(ctx.GetSessionVars(), "", "test", 1, 1, "")
key, err := NewPlanCacheKey(ctx.GetSessionVars(), "", "test", 1, 1, "", []int64{})
if err.Error() != "no statement text" {
t.Fail() // no statement text
}
key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "", 1, 1, "")
key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "", 1, 1, "", []int64{})
if err != nil {
t.Fail() // schema can be nil
}
key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1,
"select /*+ ignore_plan_cache() */ * from t")
"select /*+ ignore_plan_cache() */ * from t", []int64{})
if err != nil {
t.Fail()
}
key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1, "")
key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1, "", []int64{})
if err != nil {
t.Fail()
}
Expand Down
15 changes: 0 additions & 15 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,21 +135,6 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren
return in, true
}
}
case *ast.Limit:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about just commenting these lines out instead of removing them? Since we'll add a switch later.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good idea

if node.Count != nil {
if _, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker {
checker.cacheable = false
checker.reason = "query has 'limit ?' is un-cacheable"
return in, true
}
}
if node.Offset != nil {
if _, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker {
checker.cacheable = false
checker.reason = "query has 'limit ?, 10' is un-cacheable"
return in, true
}
}
case *ast.FrameBound:
if _, ok := node.Expr.(*driver.ParamMarkerExpr); ok {
checker.cacheable = false
Expand Down
12 changes: 6 additions & 6 deletions planner/core/plan_cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func TestCacheable(t *testing.T) {
TableRefs: tableRefsClause,
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{
Offset: &driver.ParamMarkerExpr{},
Expand All @@ -96,7 +96,7 @@ func TestCacheable(t *testing.T) {
TableRefs: tableRefsClause,
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{}
stmt = &ast.DeleteStmt{
Expand Down Expand Up @@ -139,7 +139,7 @@ func TestCacheable(t *testing.T) {
TableRefs: tableRefsClause,
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{
Offset: &driver.ParamMarkerExpr{},
Expand All @@ -148,7 +148,7 @@ func TestCacheable(t *testing.T) {
TableRefs: tableRefsClause,
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{}
stmt = &ast.UpdateStmt{
Expand Down Expand Up @@ -188,15 +188,15 @@ func TestCacheable(t *testing.T) {
stmt = &ast.SelectStmt{
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{
Offset: &driver.ParamMarkerExpr{},
}
stmt = &ast.SelectStmt{
Limit: limitStmt,
}
require.False(t, core.Cacheable(stmt, is))
require.True(t, core.Cacheable(stmt, is))

limitStmt = &ast.Limit{}
stmt = &ast.SelectStmt{
Expand Down
3 changes: 2 additions & 1 deletion server/driver_tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,8 +171,9 @@ func (ts *TiDBStatement) Close() error {
return errors.Errorf("invalid PlanCacheStmt type")
}
bindSQL, _ := core.GetBindSQL4PlanCache(ts.ctx, preparedObj)
limitOffsetAndCount, _ := core.ExtractLimitFromAst(preparedObj.PreparedAst.Stmt)
cacheKey, err := core.NewPlanCacheKey(ts.ctx.GetSessionVars(), preparedObj.StmtText, preparedObj.StmtDB,
preparedObj.PreparedAst.SchemaVersion, 0, bindSQL)
preparedObj.PreparedAst.SchemaVersion, 0, bindSQL, limitOffsetAndCount)
if err != nil {
return err
}
Expand Down
3 changes: 2 additions & 1 deletion session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -382,8 +382,9 @@ func (s *session) cleanRetryInfo() {
preparedAst = preparedObj.PreparedAst
stmtText, stmtDB = preparedObj.StmtText, preparedObj.StmtDB
bindSQL, _ := plannercore.GetBindSQL4PlanCache(s, preparedObj)
limitOffsetAndCount, _ := plannercore.ExtractLimitFromAst(preparedAst.Stmt)
cacheKey, err = plannercore.NewPlanCacheKey(s.sessionVars, stmtText, stmtDB, preparedAst.SchemaVersion,
0, bindSQL)
0, bindSQL, limitOffsetAndCount)
if err != nil {
logutil.Logger(s.currentCtx).Warn("clean cached plan failed", zap.Error(err))
return
Expand Down