planner: Implement ignore_plan_cache hint for non-prepared statements. (#62641)

close pingcap/tidb#62634
This commit is contained in:
Will DeVries
2025-07-29 22:50:37 -07:00
committed by GitHub
parent dc664bfe96
commit c7003d8d65
11 changed files with 92 additions and 74 deletions

View File

@ -104,11 +104,10 @@ type BindingMatchInfo struct {
}
// MatchSQLBindingForPlanCache matches binding for plan cache.
func MatchSQLBindingForPlanCache(sctx sessionctx.Context, stmtNode ast.StmtNode, info *BindingMatchInfo) (bindingSQL string, ignoreBinding bool) {
func MatchSQLBindingForPlanCache(sctx sessionctx.Context, stmtNode ast.StmtNode, info *BindingMatchInfo) (bindingSQL string) {
binding, matched, _ := matchSQLBinding(sctx, stmtNode, info)
if matched {
bindingSQL = binding.BindSQL
ignoreBinding = binding.Hint.ContainTableHint(hint.HintIgnorePlanCache)
}
return
}

View File

@ -1778,3 +1778,54 @@ func TestNonPreparedPlanSupportsSetVar(t *testing.T) {
tk.MustExec(`select /*+ set_var(max_execution_time=2000) */ * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 1"))
}
func TestSupportForIgnorePlanCacheHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec(`create table t (pk int, a int, primary key(pk))`)
tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1;`)
tk.MustExec(`select /*+ ignore_plan_cache() */ * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 0"))
tk.MustExec(`select /*+ ignore_plan_cache() */ * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 0"))
tk.MustExec(`CREATE BINDING FOR select * from t where pk >= ? USING select * from t where pk >= ?`)
tk.MustExec(`select /*+ ignore_plan_cache() */ * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
tk.MustExec(`select /*+ ignore_plan_cache() */ * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
tk.MustExec(`DROP BINDING FOR select * from t where pk >= ?`)
tk.MustExec(`CREATE BINDING FOR select * from t where pk >= ? USING select /*+ ignore_plan_cache() */ * from t where pk >= ?`)
tk.MustExec(`select * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
tk.MustExec(`select * from t where pk >= 1`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
tk.MustExec(`DROP BINDING FOR select * from t where pk >= ?`)
tk.MustExec(`prepare st from 'select * from t where pk >= ?'`)
tk.MustExec(`set @a=4`)
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 0"))
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 1"))
tk.MustExec(`prepare st from 'select /*+ ignore_plan_cache() */ * from t where pk >= ?'`)
tk.MustExec(`set @a=4`)
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 0"))
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("0 0"))
tk.MustExec(`CREATE BINDING FOR select * from t where pk >= ? USING select /*+ ignore_plan_cache() */ * from t where pk >= ?`)
tk.MustExec(`prepare st from 'select * from t where pk >= ?'`)
tk.MustExec(`set @a=4`)
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
tk.MustExec(`execute st using @a`)
tk.MustQuery(`select @@last_plan_from_binding, @@last_plan_from_cache`).Check(testkit.Rows("1 0"))
}

View File

@ -256,10 +256,7 @@ func hashInt64Uint64Map(b []byte, m map[int64]uint64) []byte {
// differentiate the cache key. In other cases, it will be 0.
// All information that might affect the plan should be considered in this function.
func NewPlanCacheKey(sctx sessionctx.Context, stmt *PlanCacheStmt) (key, binding string, cacheable bool, reason string, err error) {
binding, ignored := bindinfo.MatchSQLBindingForPlanCache(sctx, stmt.PreparedAst.Stmt, &stmt.BindingInfo)
if ignored {
return "", binding, false, "ignore plan cache by binding", nil
}
binding = bindinfo.MatchSQLBindingForPlanCache(sctx, stmt.PreparedAst.Stmt, &stmt.BindingInfo)
// 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.

View File

@ -35,7 +35,6 @@ import (
"github.com/pingcap/tidb/pkg/types"
driver "github.com/pingcap/tidb/pkg/types/parser_driver"
"github.com/pingcap/tidb/pkg/util/filter"
h "github.com/pingcap/tidb/pkg/util/hint"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/logutil"
"go.uber.org/zap"
@ -90,30 +89,6 @@ type cacheableChecker struct {
// Enter implements Visitor interface.
func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch node := in.(type) {
case *ast.SelectStmt:
for _, hints := range node.TableHints {
if hints.HintName.L == h.HintIgnorePlanCache {
checker.cacheable = false
checker.reason = "ignore plan cache by hint"
return in, true
}
}
case *ast.DeleteStmt:
for _, hints := range node.TableHints {
if hints.HintName.L == h.HintIgnorePlanCache {
checker.cacheable = false
checker.reason = "ignore plan cache by hint"
return in, true
}
}
case *ast.UpdateStmt:
for _, hints := range node.TableHints {
if hints.HintName.L == h.HintIgnorePlanCache {
checker.cacheable = false
checker.reason = "ignore plan cache by hint"
return in, true
}
}
case *ast.InsertStmt:
if node.Select == nil {
nRows := len(node.Lists)
@ -127,13 +102,6 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren
return in, true
}
}
for _, hints := range node.TableHints {
if hints.HintName.L == h.HintIgnorePlanCache {
checker.cacheable = false
checker.reason = "ignore plan cache by hint"
return in, true
}
}
case *ast.PatternInExpr:
checker.sumInListLen += len(node.List)
if checker.sumInListLen > checker.maxNumParam { // to save memory

View File

@ -29,7 +29,6 @@ import (
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/testkit"
driver "github.com/pingcap/tidb/pkg/types/parser_driver"
"github.com/pingcap/tidb/pkg/util/hint"
"github.com/pingcap/tidb/pkg/util/mock"
"github.com/stretchr/testify/require"
)
@ -196,11 +195,6 @@ func TestCacheable(t *testing.T) {
c, _ = core.CacheableWithCtx(mockCtx, stmt, is)
require.True(t, c)
stmt.(*ast.DeleteStmt).TableHints = append(stmt.(*ast.DeleteStmt).TableHints, &ast.TableOptimizerHint{
HintName: ast.NewCIStr(hint.HintIgnorePlanCache),
})
require.False(t, core.Cacheable(stmt, is))
// test UpdateStmt
whereExpr = &ast.FuncCallExpr{}
stmt = &ast.UpdateStmt{
@ -252,11 +246,6 @@ func TestCacheable(t *testing.T) {
c, _ = core.CacheableWithCtx(mockCtx, stmt, is)
require.True(t, c)
stmt.(*ast.UpdateStmt).TableHints = append(stmt.(*ast.UpdateStmt).TableHints, &ast.TableOptimizerHint{
HintName: ast.NewCIStr(hint.HintIgnorePlanCache),
})
require.False(t, core.Cacheable(stmt, is))
// test SelectStmt
whereExpr = &ast.FuncCallExpr{}
stmt = &ast.SelectStmt{
@ -317,11 +306,6 @@ func TestCacheable(t *testing.T) {
}
require.True(t, core.Cacheable(stmt, is))
stmt.(*ast.SelectStmt).TableHints = append(stmt.(*ast.SelectStmt).TableHints, &ast.TableOptimizerHint{
HintName: ast.NewCIStr(hint.HintIgnorePlanCache),
})
require.False(t, core.Cacheable(stmt, is))
boundExpr := &ast.FrameBound{Expr: &driver.ParamMarkerExpr{}}
require.False(t, core.Cacheable(boundExpr, is))

View File

@ -235,6 +235,10 @@ func optimizeNoCache(ctx context.Context, sctx sessionctx.Context, node *resolve
sessVars.StmtCtx.AppendWarning(warn)
}
if sessVars.StmtCtx.StmtHints.IgnorePlanCache {
sessVars.StmtCtx.SetSkipPlanCache("ignore_plan_cache hint used in SQL query")
}
for name, val := range sessVars.StmtCtx.StmtHints.SetVars {
oldV, err := sessVars.SetSystemVarWithOldStateAsRet(name, val)
if err != nil {
@ -302,6 +306,10 @@ func optimizeNoCache(ctx context.Context, sctx sessionctx.Context, node *resolve
sessVars.CurrentDB, byte(kv.ReplicaReadFollower))
sessVars.StmtCtx.StmtHints = curStmtHints
if sessVars.StmtCtx.StmtHints.IgnorePlanCache {
sessVars.StmtCtx.SetSkipPlanCache("ignore_plan_cache hint used in SQL binding")
}
for name, val := range sessVars.StmtCtx.StmtHints.SetVars {
oldV, err := sessVars.SetSystemVarWithOldStateAsRet(name, val)
if err != nil {

View File

@ -219,6 +219,8 @@ type StmtHints struct {
// -1 for disable.
ForceNthPlan int64
ResourceGroup string
// Do not store plan in either plan cache.
IgnorePlanCache bool
// Hint flags
HasAllowInSubqToJoinAndAggHint bool
@ -265,6 +267,7 @@ func (sh *StmtHints) Clone() *StmtHints {
EnableCascadesPlanner: sh.EnableCascadesPlanner,
ForceNthPlan: sh.ForceNthPlan,
ResourceGroup: sh.ResourceGroup,
IgnorePlanCache: sh.IgnorePlanCache,
HasAllowInSubqToJoinAndAggHint: sh.HasAllowInSubqToJoinAndAggHint,
HasMemQuotaHint: sh.HasMemQuotaHint,
HasReplicaReadHint: sh.HasReplicaReadHint,
@ -308,31 +311,31 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
setVarsOffs := make([]int, 0, len(hints))
for i, hint := range hints {
switch hint.HintName.L {
case "memory_quota":
case HintMemoryQuota:
hintOffs[hint.HintName.L] = i
memoryQuotaHintCnt++
case "resource_group":
hintOffs[hint.HintName.L] = i
resourceGroupHintCnt++
case "use_toja":
case HintUseToja:
hintOffs[hint.HintName.L] = i
useToJAHintCnt++
case "use_cascades":
hintOffs[hint.HintName.L] = i
useCascadesHintCnt++
case "no_index_merge":
case HintNoIndexMerge:
hintOffs[hint.HintName.L] = i
noIndexMergeHintCnt++
case "read_consistent_replica":
hintOffs[hint.HintName.L] = i
readReplicaHintCnt++
case "max_execution_time":
case HintMaxExecutionTime:
hintOffs[hint.HintName.L] = i
maxExecutionTimeCnt++
case "nth_plan":
forceNthPlanCnt++
forceNthPlan = hint
case "straight_join":
case HintStraightJoin:
hintOffs[hint.HintName.L] = i
straightJoinHintCnt++
case "hypo_index":
@ -395,6 +398,8 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
}
setVars[setVarHint.VarName] = setVarHint.Value
setVarsOffs = append(setVarsOffs, i)
case HintIgnorePlanCache:
stmtHints.IgnorePlanCache = true
}
}
stmtHints.OriginalTableHints = hints
@ -402,14 +407,14 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
// Handle MEMORY_QUOTA
if memoryQuotaHintCnt != 0 {
memoryQuotaHint := hints[hintOffs["memory_quota"]]
memoryQuotaHint := hints[hintOffs[HintMemoryQuota]]
if memoryQuotaHintCnt > 1 {
warn := errors.NewNoStackErrorf("MEMORY_QUOTA() is defined more than once, only the last definition takes effect: MEMORY_QUOTA(%v)", memoryQuotaHint.HintData.(int64))
warns = append(warns, warn)
}
// Executor use MemoryQuota <= 0 to indicate no memory limit, here use < 0 to handle hint syntax error.
if memoryQuota := memoryQuotaHint.HintData.(int64); memoryQuota < 0 {
delete(hintOffs, "memory_quota")
delete(hintOffs, HintMemoryQuota)
warn := errors.NewNoStackError("The use of MEMORY_QUOTA hint is invalid, valid usage: MEMORY_QUOTA(10 MB) or MEMORY_QUOTA(10 GB)")
warns = append(warns, warn)
} else {
@ -423,7 +428,7 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
}
// Handle USE_TOJA
if useToJAHintCnt != 0 {
useToJAHint := hints[hintOffs["use_toja"]]
useToJAHint := hints[hintOffs[HintUseToja]]
if useToJAHintCnt > 1 {
warn := errors.NewNoStackErrorf("USE_TOJA() is defined more than once, only the last definition takes effect: USE_TOJA(%v)", useToJAHint.HintData.(bool))
warns = append(warns, warn)
@ -468,7 +473,7 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
}
// Handle MAX_EXECUTION_TIME
if maxExecutionTimeCnt != 0 {
maxExecutionTime := hints[hintOffs["max_execution_time"]]
maxExecutionTime := hints[hintOffs[HintMaxExecutionTime]]
if maxExecutionTimeCnt > 1 {
warn := errors.NewNoStackErrorf("MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(%v)", maxExecutionTime.HintData.(uint64))
warns = append(warns, warn)
@ -513,7 +518,7 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint,
// isStmtHint checks whether this hint is a statement-level hint.
func isStmtHint(h *ast.TableOptimizerHint) bool {
switch h.HintName.L {
case "max_execution_time", "memory_quota", "resource_group":
case HintMaxExecutionTime, HintMemoryQuota, "resource_group":
return true
default:
return false
@ -587,11 +592,11 @@ func (hint *HintedIndex) Match(dbName, tblName ast.CIStr) bool {
func (hint *HintedIndex) HintTypeString() string {
switch hint.IndexHint.HintType {
case ast.HintUse:
return "use_index"
return HintUseIndex
case ast.HintIgnore:
return "ignore_index"
return HintIgnoreIndex
case ast.HintForce:
return "force_index"
return HintForceIndex
}
return ""
}
@ -1090,7 +1095,7 @@ func collectUnmatchedIndexHintWarning(indexHints []HintedIndex, usedForIndexMerg
if !hint.Matched {
var hintTypeString string
if usedForIndexMerge {
hintTypeString = "use_index_merge"
hintTypeString = HintIndexMerge
} else {
hintTypeString = hint.HintTypeString()
}

View File

@ -548,7 +548,7 @@ func TestPlanCacheUnqualified2(t *testing.T) {
tk.MustExec(`execute st`)
tk.MustQuery(`select digest_text, exec_count, plan_cache_unqualified, plan_cache_unqualified_last_reason
from information_schema.statements_summary where digest_text like '%select%from%t_ignore_unqualified_test%'`).Sort().Check(testkit.Rows(
"select * from `t_ignore_unqualified_test` 1 1 ignore plan cache by hint"))
"select * from `t_ignore_unqualified_test` 1 1 ignore_plan_cache hint used in SQL query"))
// queries containing non-deterministic functions
tk.MustExec(`create table t_non_deterministic_1_unqualified_test (a int, b int)`)
@ -619,7 +619,7 @@ func TestPlanCacheUnqualified(t *testing.T) {
tk.MustQuery(`select digest_text, exec_count, plan_cache_unqualified, plan_cache_unqualified_last_reason
from information_schema.statements_summary where plan_cache_unqualified > 0`).Sort().Check(testkit.Rows(
"select * from `t1` 2 2 ignore plan cache by hint",
"select * from `t1` 2 2 ignore_plan_cache hint used in SQL query",
"select * from `t1` where `a` <= ? 4 4 '123' may be converted to INT",
"select * from `t1` where `t1` . `a` > ( select ? from `t2` where `t2` . `b` < ? ) 3 3 query has uncorrelated sub-queries is un-cacheable",
"select database ( ) from `t1` 2 2 query has 'database' is un-cacheable"))
@ -630,7 +630,7 @@ func TestPlanCacheUnqualified(t *testing.T) {
}
tk.MustQuery(`select digest_text, exec_count, plan_cache_unqualified, plan_cache_unqualified_last_reason
from information_schema.statements_summary where plan_cache_unqualified > 0`).Sort().Check(testkit.Rows(
"select * from `t1` 102 102 ignore plan cache by hint",
"select * from `t1` 102 102 ignore_plan_cache hint used in SQL query",
"select * from `t1` where `a` <= ? 4 4 '123' may be converted to INT",
"select * from `t1` where `t1` . `a` > ( select ? from `t2` where `t2` . `b` < ? ) 3 3 query has uncorrelated sub-queries is un-cacheable",
"select database ( ) from `t1` 102 102 query has 'database' is un-cacheable"))

View File

@ -1166,8 +1166,6 @@ select @@last_plan_from_cache;
drop table if exists t;
create table t (a int, b int, key(a), key(b));
prepare stmt from 'select /*+ ignore_plan_cache() */ * from t';
Level Code Message
Warning 1105 skip prepared plan-cache: ignore plan cache by hint
prepare stmt from 'select * from t order by ?';
Level Code Message
Warning 1105 skip prepared plan-cache: query has 'order by ?' is un-cacheable
@ -1195,7 +1193,7 @@ create binding for select * from t where a<1 using select /*+ ignore_plan_cache(
execute st using @a;
a
Level Code Message
Warning 1105 skip prepared plan-cache: ignore plan cache by binding
Warning 1105 skip prepared plan-cache: ignore_plan_cache hint used in SQL binding
prepare st from 'select * from t where a>?';
set @a=1;
execute st using @a;

View File

@ -506,6 +506,12 @@ id
select @@last_plan_from_cache;
@@last_plan_from_cache
0
execute stmt2 using @doma;
id
1
select @@last_plan_from_cache;
@@last_plan_from_cache
0
execute stmt using @doma;
id
1

View File

@ -320,6 +320,8 @@ execute stmt using @doma;
select @@last_plan_from_cache;
execute stmt2 using @doma;
select @@last_plan_from_cache;
execute stmt2 using @doma;
select @@last_plan_from_cache;
execute stmt using @doma;
select @@last_plan_from_cache;
select * from t where id=1;