From c7003d8d65fdf0dff2a196cc8fc18230bbc7bb1f Mon Sep 17 00:00:00 2001 From: Will DeVries <1624341+wddevries@users.noreply.github.com> Date: Tue, 29 Jul 2025 22:50:37 -0700 Subject: [PATCH] planner: Implement ignore_plan_cache hint for non-prepared statements. (#62641) close pingcap/tidb#62634 --- pkg/bindinfo/binding.go | 3 +- pkg/planner/core/plan_cache_test.go | 51 +++++++++++++++++++ pkg/planner/core/plan_cache_utils.go | 5 +- pkg/planner/core/plan_cacheable_checker.go | 32 ------------ .../core/plan_cacheable_checker_test.go | 16 ------ pkg/planner/optimize.go | 8 +++ pkg/util/hint/hint.go | 33 +++++++----- pkg/util/stmtsummary/v2/tests/table_test.go | 6 +-- .../r/planner/core/plan_cache.result | 4 +- .../planner/core/tests/prepare/prepare.result | 6 +++ .../t/planner/core/tests/prepare/prepare.test | 2 + 11 files changed, 92 insertions(+), 74 deletions(-) diff --git a/pkg/bindinfo/binding.go b/pkg/bindinfo/binding.go index 09f1e4836c..168ae6f577 100644 --- a/pkg/bindinfo/binding.go +++ b/pkg/bindinfo/binding.go @@ -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 } diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go index ae6af40327..afa5091929 100644 --- a/pkg/planner/core/plan_cache_test.go +++ b/pkg/planner/core/plan_cache_test.go @@ -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")) +} diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index 67d5427390..96532b148d 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -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. diff --git a/pkg/planner/core/plan_cacheable_checker.go b/pkg/planner/core/plan_cacheable_checker.go index fcf8188e92..25d0e76d21 100644 --- a/pkg/planner/core/plan_cacheable_checker.go +++ b/pkg/planner/core/plan_cacheable_checker.go @@ -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 diff --git a/pkg/planner/core/plan_cacheable_checker_test.go b/pkg/planner/core/plan_cacheable_checker_test.go index a4a7afe348..86759ef383 100644 --- a/pkg/planner/core/plan_cacheable_checker_test.go +++ b/pkg/planner/core/plan_cacheable_checker_test.go @@ -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)) diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index b547e60b93..7b3d4f36af 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -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 { diff --git a/pkg/util/hint/hint.go b/pkg/util/hint/hint.go index 6cb3dba22a..175f163a22 100644 --- a/pkg/util/hint/hint.go +++ b/pkg/util/hint/hint.go @@ -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() } diff --git a/pkg/util/stmtsummary/v2/tests/table_test.go b/pkg/util/stmtsummary/v2/tests/table_test.go index 78596d6fbc..2133843987 100644 --- a/pkg/util/stmtsummary/v2/tests/table_test.go +++ b/pkg/util/stmtsummary/v2/tests/table_test.go @@ -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")) diff --git a/tests/integrationtest/r/planner/core/plan_cache.result b/tests/integrationtest/r/planner/core/plan_cache.result index 4945100a70..0e0a35b412 100644 --- a/tests/integrationtest/r/planner/core/plan_cache.result +++ b/tests/integrationtest/r/planner/core/plan_cache.result @@ -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; diff --git a/tests/integrationtest/r/planner/core/tests/prepare/prepare.result b/tests/integrationtest/r/planner/core/tests/prepare/prepare.result index 49931b996b..dc11bff8b4 100644 --- a/tests/integrationtest/r/planner/core/tests/prepare/prepare.result +++ b/tests/integrationtest/r/planner/core/tests/prepare/prepare.result @@ -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 diff --git a/tests/integrationtest/t/planner/core/tests/prepare/prepare.test b/tests/integrationtest/t/planner/core/tests/prepare/prepare.test index b7c0d2f5c6..800863c5bb 100644 --- a/tests/integrationtest/t/planner/core/tests/prepare/prepare.test +++ b/tests/integrationtest/t/planner/core/tests/prepare/prepare.test @@ -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;