planner: Implement ignore_plan_cache hint for non-prepared statements. (#62641)
close pingcap/tidb#62634
This commit is contained in:
@ -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
|
||||
}
|
||||
|
||||
@ -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"))
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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))
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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()
|
||||
}
|
||||
|
||||
@ -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"))
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
Reference in New Issue
Block a user