Skip to content

Commit

Permalink
planner: make tidb_enable_plan_cache_for_subquery can take effective …
Browse files Browse the repository at this point in the history
…immediately (#41612)

ref #40219
  • Loading branch information
fzzf678 authored Mar 17, 2023
1 parent 3ea3647 commit df92b9f
Show file tree
Hide file tree
Showing 7 changed files with 168 additions and 111 deletions.
93 changes: 0 additions & 93 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,96 +806,3 @@ func TestIssue38323(t *testing.T) {
tk.MustExec("set @a = 1;")
tk.MustQuery("execute stmt using @a, @a").Check(tk.MustQuery("explain select * from t where 1 = id and 1 = k group by id, k").Rows())
}

func TestSetPlanCacheLimitSwitch(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = 1;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = off;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = ON;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = '';", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of ''")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = 11;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of '11'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = enabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'enabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = disabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'disabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = open;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'open'")
}

func TestPlanCacheLimitSwitchEffective(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, key(a))")

checkIfCached := func(res string) {
tk.MustExec("set @a = 1")
tk.MustExec("execute stmt using @a")
tk.MustExec("execute stmt using @a")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(res))
}

// before prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query has 'limit ?' is un-cacheable"))
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")

// after prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("execute stmt using @a")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off"))
tk.MustExec("deallocate prepare stmt")

// after execute
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
checkIfCached("1")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")
}

func TestSetPlanCacheSubquerySwitch(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = OFF;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = 1;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_subquery = off;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_subquery = ON;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = '';", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of ''")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = 11;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of '11'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = enabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'enabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = disabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'disabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = open;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'open'")
}
2 changes: 1 addition & 1 deletion planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared

// check whether this plan is cacheable.
if stmtCtx.UseCache {
if cacheable, reason := isPlanCacheable(sctx, p, len(matchOpts.ParamTypes), len(matchOpts.LimitOffsetAndCount)); !cacheable {
if cacheable, reason := isPlanCacheable(sctx, p, len(matchOpts.ParamTypes), len(matchOpts.LimitOffsetAndCount), matchOpts.HasSubQuery); !cacheable {
stmtCtx.SetSkipPlanCache(errors.Errorf(reason))
}
}
Expand Down
4 changes: 4 additions & 0 deletions planner/core/plan_cache_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,10 @@ func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, matchOp
// offset and key slice matched, but it is a plan with param limit and the switch is disabled
continue
}
// check subquery switch state
if plan.matchOpts.HasSubQuery && !l.sctx.GetSessionVars().EnablePlanCacheForSubquery {
continue
}
return k, true
}
return nil, false
Expand Down
132 changes: 132 additions & 0 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1061,6 +1061,72 @@ func TestIssue41032(t *testing.T) {
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1"))
}

func TestSetPlanCacheLimitSwitch(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = 1;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = off;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = ON;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = '';", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of ''")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = 11;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of '11'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = enabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'enabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = disabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'disabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_param_limit = open;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_param_limit' can't be set to the value of 'open'")
}

func TestPlanCacheLimitSwitchEffective(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, key(a))")

checkIfCached := func(res string) {
tk.MustExec("set @a = 1")
tk.MustExec("execute stmt using @a")
tk.MustExec("execute stmt using @a")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(res))
}

// before prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query has 'limit ?' is un-cacheable"))
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")

// after prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("execute stmt using @a")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off"))
tk.MustExec("deallocate prepare stmt")

// after execute
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
checkIfCached("1")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")
}

func TestPlanCacheWithLimit(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -1124,6 +1190,72 @@ func TestPlanCacheMemoryTable(t *testing.T) {
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) // plan accessing memory tables cannot hit the cache
}

func TestSetPlanCacheSubquerySwitch(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = OFF;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = 1;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_subquery = off;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_subquery = ON;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_subquery").Check(testkit.Rows("1"))

tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = '';", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of ''")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = 11;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of '11'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = enabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'enabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = disabled;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'disabled'")
tk.MustGetErrMsg("set @@global.tidb_enable_plan_cache_for_subquery = open;", "[variable:1231]Variable 'tidb_enable_plan_cache_for_subquery' can't be set to the value of 'open'")
}

func TestPlanCacheSubQuerySwitchEffective(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, key(a))")
tk.MustExec("create table s(a int, key(a))")

checkIfCached := func(res string) {
tk.MustExec("execute stmt")
tk.MustExec("execute stmt")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(res))
}

// before prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = OFF")
tk.MustExec("prepare stmt from 'select * from t where a in (select a from s)'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query has sub-queries is un-cacheable"))
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")

// after prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = ON")
tk.MustExec("prepare stmt from 'select * from t where a in (select a from s)'")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = OFF")
checkIfCached("0")
tk.MustExec("execute stmt")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: the switch 'tidb_enable_plan_cache_for_subquery' is off"))
tk.MustExec("deallocate prepare stmt")

// after execute
tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = ON")
tk.MustExec("prepare stmt from 'select * from t where a in (select a from s)'")
checkIfCached("1")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_subquery = OFF")
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")
}

func TestPlanCacheWithSubquery(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down
41 changes: 25 additions & 16 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -432,7 +432,7 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa
return nil, ErrStmtNotFound
}

type limitExtractor struct {
type matchOptsExtractor struct {
cacheable bool // For safety considerations, check if limit count less than 10000
offsetAndCount []uint64
unCacheableReason string
Expand All @@ -441,7 +441,7 @@ type limitExtractor struct {
}

// Enter implements Visitor interface.
func (checker *limitExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
func (checker *matchOptsExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch node := in.(type) {
case *ast.Limit:
if node.Count != nil {
Expand All @@ -451,12 +451,13 @@ func (checker *limitExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bo
if val > 10000 {
checker.cacheable = false
checker.unCacheableReason = "limit count more than 10000"
return in, true
return in, !checker.cacheable
}
checker.offsetAndCount = append(checker.offsetAndCount, val)
} else {
checker.cacheable = false
checker.paramTypeErr = ErrWrongArguments.GenWithStackByArgs("LIMIT")
return in, true
return in, !checker.cacheable
}
}
}
Expand All @@ -466,28 +467,32 @@ func (checker *limitExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bo
if typeExpected {
checker.offsetAndCount = append(checker.offsetAndCount, val)
} else {
checker.cacheable = false
checker.paramTypeErr = ErrWrongArguments.GenWithStackByArgs("LIMIT")
return in, true
return in, !checker.cacheable
}
}
}
case *ast.SubqueryExpr, *ast.ExistsSubqueryExpr:
checker.hasSubQuery = true
}
return in, false
}

// Leave implements Visitor interface.
func (checker *limitExtractor) Leave(in ast.Node) (out ast.Node, ok bool) {
func (checker *matchOptsExtractor) 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, sctx sessionctx.Context) ([]uint64, error) {
func extractMatchOptsFromAST(node ast.Node, sctx sessionctx.Context) (*utilpc.PlanCacheMatchOpts, error) {
if node == nil {
return nil, nil
return nil, errors.New("AST node is nil")
}
checker := limitExtractor{
checker := matchOptsExtractor{
cacheable: true,
offsetAndCount: []uint64{},
hasSubQuery: false,
}
node.Accept(&checker)
if checker.paramTypeErr != nil {
Expand All @@ -496,20 +501,24 @@ func ExtractLimitFromAst(node ast.Node, sctx sessionctx.Context) ([]uint64, erro
if sctx != nil && !checker.cacheable {
sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New(checker.unCacheableReason))
}
return checker.offsetAndCount, nil

return &utilpc.PlanCacheMatchOpts{
LimitOffsetAndCount: checker.offsetAndCount,
HasSubQuery: checker.hasSubQuery,
}, nil
}

// GetMatchOpts get options to fetch plan or generate new plan
// we can add more options here
func GetMatchOpts(sctx sessionctx.Context, node ast.Node, params []expression.Expression) (*utilpc.PlanCacheMatchOpts, error) {
limitParams, err := ExtractLimitFromAst(node, sctx)
// get limit params and has sub query indicator
matchOpts, err := extractMatchOptsFromAST(node, sctx)
if err != nil {
return nil, err
}
paramTypes := parseParamTypes(sctx, params)
return &utilpc.PlanCacheMatchOpts{
ParamTypes: paramTypes,
LimitOffsetAndCount: limitParams,
}, nil
// get param types
matchOpts.ParamTypes = parseParamTypes(sctx, params)
return matchOpts, nil
}

// CheckTypesCompatibility4PC compares FieldSlice with []*types.FieldType
Expand Down
5 changes: 4 additions & 1 deletion planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -435,7 +435,7 @@ func isPartitionTable(schema infoschema.InfoSchema, tn *ast.TableName) bool {
}

// isPlanCacheable returns whether this plan is cacheable and the reason if not.
func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum int) (cacheable bool, reason string) {
func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum int, hasSubQuery bool) (cacheable bool, reason string) {
var pp PhysicalPlan
switch x := p.(type) {
case *Insert:
Expand All @@ -455,6 +455,9 @@ func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum in
if limitParamNum != 0 && !sctx.GetSessionVars().EnablePlanCacheForParamLimit {
return false, "the switch 'tidb_enable_plan_cache_for_param_limit' is off"
}
if hasSubQuery && !sctx.GetSessionVars().EnablePlanCacheForSubquery {
return false, "the switch 'tidb_enable_plan_cache_for_subquery' is off"
}
return isPhysicalPlanCacheable(sctx, pp, paramNum, limitParamNum, false)
}

Expand Down
2 changes: 2 additions & 0 deletions util/plancache/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,4 +26,6 @@ type PlanCacheMatchOpts struct {
// limitOffsetAndCount stores all the offset and key parameters extract from limit statement
// only used for cache and pick plan with parameters in limit
LimitOffsetAndCount []uint64
// HasSubQuery indicate whether this query has sub query
HasSubQuery bool
}

0 comments on commit df92b9f

Please sign in to comment.