diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 8eac7816a4aba..3983019a1b20f 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -1288,20 +1288,15 @@ func TestCTE4PlanCache(t *testing.T) { tk.MustExec("insert into t1 values(1);") tk.MustExec("insert into t1 values(2);") tk.MustExec("prepare stmt from 'SELECT * FROM t1 dt WHERE EXISTS(WITH RECURSIVE qn AS (SELECT a*? AS b UNION ALL SELECT b+? FROM qn WHERE b=?) SELECT * FROM qn WHERE b=a);';") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: find table test.qn failed: [schema:1146]Table 'test.qn' doesn't exist")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has sub-queries is un-cacheable")) tk.MustExec("set @a=1, @b=2, @c=3, @d=4, @e=5, @f=0;") tk.MustQuery("execute stmt using @f, @a, @f").Check(testkit.Rows("1")) tk.MustQuery("execute stmt using @a, @b, @a").Sort().Check(testkit.Rows("1", "2")) tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) -<<<<<<< HEAD -======= - tk.MustQuery("execute stmt using @a, @b, @a").Sort().Check(testkit.Rows("1", "2")) - //tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: PhysicalApply plan is un-cacheable")) ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) tk.MustExec("prepare stmt from 'with recursive c(p) as (select ?), cte(a, b) as (select 1, 1 union select a+?, 1 from cte, c where a < ?) select * from cte order by 1, 2;';") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: find table test.cte failed: [schema:1146]Table 'test.cte' doesn't exist")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has sub-queries is un-cacheable")) tk.MustQuery("execute stmt using @a, @a, @e;").Check(testkit.Rows("1 1", "2 1", "3 1", "4 1", "5 1")) tk.MustQuery("execute stmt using @b, @b, @c;").Check(testkit.Rows("1 1", "3 1")) tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index f86f85f5ba9d5..958f65f44c75a 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -42,53 +42,12 @@ import ( "go.uber.org/zap" ) -<<<<<<< HEAD -func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { -======= var ( - // PlanCacheKeyTestIssue43667 is only for test. - PlanCacheKeyTestIssue43667 struct{} // PlanCacheKeyTestIssue46760 is only for test. PlanCacheKeyTestIssue46760 struct{} ) -// SetParameterValuesIntoSCtx sets these parameters into session context. -func SetParameterValuesIntoSCtx(sctx sessionctx.Context, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { - vars := sctx.GetSessionVars() - vars.PlanCacheParams.Reset() - for i, usingParam := range params { - val, err := usingParam.Eval(chunk.Row{}) - if err != nil { - return err - } - if isGetVarBinaryLiteral(sctx, usingParam) { - binVal, convErr := val.ToBytes() - if convErr != nil { - return convErr - } - val.SetBinaryLiteral(binVal) - } - if markers != nil { - param := markers[i].(*driver.ParamMarkerExpr) - param.Datum = val - param.InExecute = true - } - vars.PlanCacheParams.Append(val) - } - if vars.StmtCtx.EnableOptimizerDebugTrace && len(vars.PlanCacheParams.AllParamValues()) > 0 { - vals := vars.PlanCacheParams.AllParamValues() - valStrs := make([]string, len(vals)) - for i, val := range vals { - valStrs[i] = val.String() - } - debugtrace.RecordAnyValuesWithNames(sctx, "Parameter datums for EXECUTE", valStrs) - } - vars.PlanCacheParams.SetForNonPrepCache(isNonPrep) - return nil -} - -func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) +func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { vars := sctx.GetSessionVars() stmtAst := stmt.PreparedAst vars.StmtCtx.StmtType = stmtAst.StmtType diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 19fdc496457d6..0d73bef965d81 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -518,73 +518,12 @@ func TestIssue41828(t *testing.T) { KEY U_M_COL4 (COL1,COL2), KEY U_M_COL5 (COL3,COL2))`) -<<<<<<< HEAD tk.MustExec(`INSERT INTO IDT_MULTI15840STROBJSTROBJ VALUES ('zzz',1047,'6115-06-05'),('zzz',-23221,'4250-09-03'),('zzz',27138,'1568-07-30'),('zzz',-30903,'6753-08-21'),('zzz',-26875,'6117-10-10')`) tk.MustExec(`prepare stmt from 'select * from IDT_MULTI15840STROBJSTROBJ where col3 <=> ? or col1 in (?, ?, ?) and col2 not between ? and ?'`) tk.MustExec(`set @a="0051-12-23", @b="none", @c="none", @d="none", @e=-32757, @f=-32757`) tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f`).Check(testkit.Rows()) tk.MustExec(`set @a="9795-01-10", @b="aa", @c="aa", @d="aa", @e=31928, @f=31928`) tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f`).Check(testkit.Rows()) -======= - reasons := []string{ - "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", - "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", - "skip non-prepared plan-cache: queries that have sub-queries are not supported", - "skip non-prepared plan-cache: query accesses partitioned tables is un-cacheable", - "skip non-prepared plan-cache: query accesses partitioned tables is un-cacheable", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", - "skip non-prepared plan-cache: access tables in system schema", - "skip non-prepared plan-cache: query accesses generated columns is un-cacheable", - "skip non-prepared plan-cache: query accesses generated columns is un-cacheable", - "skip non-prepared plan-cache: queries that access views are not supported", - "skip non-prepared plan-cache: query has null constants", - "skip non-prepared plan-cache: some parameters may be overwritten when constant propagation", - } - - all := append(supported, unsupported...) - - explainFormats := []string{ - types.ExplainFormatBrief, - types.ExplainFormatDOT, - types.ExplainFormatHint, - types.ExplainFormatROW, - types.ExplainFormatVerbose, - types.ExplainFormatTraditional, - types.ExplainFormatBinary, - types.ExplainFormatTiDBJSON, - types.ExplainFormatCostTrace, - } - // all cases no warnings use other format - for _, q := range all { - tk.MustExec("explain " + q) - tk.MustQuery("show warnings").Check(testkit.Rows()) - tk.MustExec("explain " + q) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - } - for _, format := range explainFormats { - for _, q := range all { - tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) - //tk.MustQuery("show warnings").Check(testkit.Rows()) - tk.MustQuery("show warnings").CheckNotContain("plan cache") - tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - } - } - - // unsupported case with warning use 'plan_cache' format - for idx, q := range unsupported { - tk.MustExec("explain format = 'plan_cache'" + q) - warn := tk.MustQuery("show warnings").Rows()[0] - require.Equal(t, reasons[idx], warn[2]) - } ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) } func TestIssue42150(t *testing.T) { diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 42de526f19141..ba35b76afc87d 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -118,16 +118,8 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, if !vars.EnablePreparedPlanCache { prepared.UseCache = false } else { -<<<<<<< HEAD cacheable, reason := CacheableWithCtx(sctx, stmt, ret.InfoSchema) prepared.UseCache = cacheable -======= - if isPrepStmt { - cacheable, reason = IsASTCacheable(ctx, sctx, paramStmt, ret.InfoSchema) - } else { - cacheable = true // it is already checked here - } ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) if !cacheable { sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: " + reason)) } diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index a6b1016bca671..ac19a6fe17071 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -18,42 +18,26 @@ import ( "context" "errors" "fmt" -<<<<<<< HEAD -======= - "math" "strings" - "sync" ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" driver "github.com/pingcap/tidb/types/parser_driver" -<<<<<<< HEAD -======= - "github.com/pingcap/tidb/util/filter" - "github.com/pingcap/tidb/util/intest" ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) -<<<<<<< HEAD -// Cacheable checks whether the input ast is cacheable with empty session context, which is mainly for testing. -======= // Cacheable checks whether the input ast(query) is cacheable with empty session context, which is mainly for testing. // TODO: only for test, remove this function later on. ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { c, _ := IsASTCacheable(nil, nil, node, is) return c } -<<<<<<< HEAD -// CacheableWithCtx checks whether the input ast is cacheable. -======= // CacheableWithCtx checks whether the input ast(query) is cacheable. // TODO: only for test, remove this function later on. func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { @@ -61,7 +45,6 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info } // IsASTCacheable checks whether the input ast(query) is cacheable. ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect func IsASTCacheable(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { @@ -74,18 +57,10 @@ func IsASTCacheable(ctx context.Context, sctx sessionctx.Context, node ast.Node, return false, "not a SELECT/UPDATE/INSERT/DELETE/SET statement" } checker := cacheableChecker{ -<<<<<<< HEAD + ctx: ctx, sctx: sctx, cacheable: true, schema: is, -======= - ctx: ctx, - sctx: sctx, - cacheable: true, - schema: is, - sumInListLen: 0, - maxNumParam: getMaxParamLimit(sctx), ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) } node.Accept(&checker) return checker.cacheable, checker.reason @@ -282,22 +257,7 @@ func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, sk return in, true case *ast.TableName: if checker.schema != nil { -<<<<<<< HEAD - if isPartitionTable(checker.schema, node) { - checker.cacheable = false - return in, true - } - if hasGeneratedCol(checker.schema, node) { - checker.cacheable = false - return in, true - } - if isTempTable(checker.schema, node) { - checker.cacheable = false - return in, true - } -======= - checker.cacheable, checker.reason = checkTableCacheable(nil, checker.sctx, checker.schema, node, true) ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) + checker.cacheable, _ = checkTableCacheable(nil, checker.sctx, checker.schema, node, true) } } return in, false @@ -308,157 +268,6 @@ func (checker *generalPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok return in, checker.cacheable } -<<<<<<< HEAD -func hasGeneratedCol(schema infoschema.InfoSchema, tn *ast.TableName) bool { - tb, err := schema.TableByName(tn.Schema, tn.Name) - if err != nil { - logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) - return false - } - for _, col := range tb.Cols() { - if col.IsGenerated() { - return true - } - } - return false -} - -func isTempTable(schema infoschema.InfoSchema, tn *ast.TableName) bool { - tb, err := schema.TableByName(tn.Schema, tn.Name) - if err != nil { - logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) - return false - } - if tb.Meta().TempTableType != model.TempTableNone { - return true - } - return false -} - -func isPartitionTable(schema infoschema.InfoSchema, tn *ast.TableName) bool { - tb, err := schema.TableByName(tn.Schema, tn.Name) - if err != nil { - logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) - return false - } - if tb.Meta().GetPartitionInfo() != nil { - return true - } - return false -======= -func (*nonPreparedPlanCacheableChecker) isFilterNode(node ast.Node) bool { - switch node.(type) { - case *ast.BetweenExpr, *ast.PatternInExpr, *ast.BinaryOperationExpr: - return true - } - return false -} - -func getColType(schema infoschema.InfoSchema, tbl *ast.TableName, col *ast.ColumnName) (colType byte, found bool) { - if tbl == nil { - return 0, false - } - tb, err := schema.TableByName(tbl.Schema, tbl.Name) - if err != nil { - return 0, false - } - for _, c := range tb.Cols() { - if c.Name.L == col.Name.L { - return c.GetType(), true - } - } - return 0, false -} - -// isPlanCacheable returns whether this plan is cacheable and the reason if not. -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: - pp = x.SelectPlan - case *Update: - pp = x.SelectPlan - case *Delete: - pp = x.SelectPlan - case PhysicalPlan: - pp = x - default: - return false, fmt.Sprintf("unexpected un-cacheable plan %v", p.ExplainID().String()) - } - if pp == nil { // simple DML statements - return true, "" - } - 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" - } - if sctx.GetSessionVars().PlanCacheMaxPlanSize > 0 && uint64(pp.MemoryUsage()) > sctx.GetSessionVars().PlanCacheMaxPlanSize { // to save memory - return false, "plan is too large(decided by the variable @@tidb_plan_cache_max_plan_size)" - } - return isPhysicalPlanCacheable(sctx, pp, paramNum, limitParamNum, false) -} - -// isPhysicalPlanCacheable returns whether this physical plan is cacheable and return the reason if not. -func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { - var subPlans []PhysicalPlan - switch x := p.(type) { - case *PhysicalTableDual: - if paramNum > 0 { - return false, "get a TableDual plan" - } - case *PhysicalTableReader: - if x.StoreType == kv.TiFlash { - return false, "TiFlash plan is un-cacheable" - } - case *PhysicalShuffle, *PhysicalShuffleReceiverStub: - return false, "get a Shuffle plan" - case *PhysicalMemTable: - return false, "PhysicalMemTable plan is un-cacheable" - case *PhysicalIndexMergeReader: - if x.AccessMVIndex { - return false, "the plan with IndexMerge accessing Multi-Valued Index is un-cacheable" - } - underIndexMerge = true - subPlans = append(subPlans, x.partialPlans...) - case *PhysicalIndexScan: - if underIndexMerge && x.isFullScan() { - return false, "IndexMerge plan with full-scan is un-cacheable" - } - case *PhysicalTableScan: - if underIndexMerge && x.isFullScan() { - return false, "IndexMerge plan with full-scan is un-cacheable" - } - case *PhysicalApply: - return false, "PhysicalApply plan is un-cacheable" - } - - subPlans = append(subPlans, p.Children()...) - for _, c := range subPlans { - if cacheable, reason = isPhysicalPlanCacheable(sctx, c, paramNum, limitParamNum, underIndexMerge); !cacheable { - return cacheable, reason - } - } - return true, "" -} - -// getMaxParamLimit returns the maximum number of parameters for a query that can be cached in the Plan Cache. -func getMaxParamLimit(sctx sessionctx.Context) int { - v := 200 - if sctx == nil || sctx.GetSessionVars() == nil || sctx.GetSessionVars().OptimizerFixControl == nil { - return v - } - n := fixcontrol.GetIntWithDefault(sctx.GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix44823, int64(v)) - if n == 0 { - v = math.MaxInt32 // no limitation - } else if n > 0 { - v = int(n) - } - - return v -} - // checkTableCacheable checks whether a query accessing this table is cacheable. func checkTableCacheable(ctx context.Context, sctx sessionctx.Context, schema infoschema.InfoSchema, node *ast.TableName, isNonPrep bool) (cacheable bool, reason string) { tableSchema := node.Schema @@ -467,9 +276,12 @@ func checkTableCacheable(ctx context.Context, sctx sessionctx.Context, schema in tableSchema.L = strings.ToLower(tableSchema.O) } tb, err := schema.TableByName(tableSchema, node.Name) - if intest.InTest && ctx != nil && ctx.Value(PlanCacheKeyTestIssue46760) != nil { - err = errors.New("mock error") - } + + failpoint.Inject("TestIssue46760", func(val failpoint.Value) { + if val.(bool) { + err = errors.New("mock error") + } + }) if err != nil { sql := sctx.GetSessionVars().StmtCtx.OriginalSQL if len(sql) > 256 { @@ -509,5 +321,4 @@ func checkTableCacheable(ctx context.Context, sctx sessionctx.Context, schema in } return true, "" ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) } diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index 90cc9dc2251c4..bdddaf9c27000 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -15,14 +15,10 @@ package core_test import ( -<<<<<<< HEAD -======= "context" - "fmt" - "strings" ->>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser" @@ -35,64 +31,6 @@ import ( "github.com/stretchr/testify/require" ) -<<<<<<< HEAD -======= -func TestFixControl44823(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`create table t (a int)`) - var va []string - for i := 0; i < 201; i++ { - tk.MustExec(fmt.Sprintf(`set @a%v = %v`, i, i)) - va = append(va, fmt.Sprintf("@a%v", i)) - } - - // prepared plan cache - tk.MustExec(fmt.Sprintf(`prepare st from 'select * from t where a in (%v?)'`, strings.Repeat("?,", 200))) - tk.MustQuery(`show warnings`).Check(testkit.Rows(`Warning 1105 skip prepared plan-cache: too many values in in-list`)) - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - - tk.MustExec(`set @@tidb_opt_fix_control = "44823:250"`) - tk.MustExec(fmt.Sprintf(`prepare st from 'select * from t where a in (%v?)'`, strings.Repeat("?,", 200))) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can hit - - tk.MustExec(`set @@tidb_opt_fix_control = "44823:0"`) - tk.MustExec(fmt.Sprintf(`prepare st from 'select * from t where a in (%v?)'`, strings.Repeat("?,", 200))) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustExec(fmt.Sprintf(`execute st using %v`, strings.Join(va, ","))) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - - // non prepared plan cache - values := make([]string, 0, 201) - for i := 0; i < 201; i++ { - values = append(values, fmt.Sprintf("%v", i)) - } - query := fmt.Sprintf("select * from t where a in (%v)", strings.Join(values, ",")) - tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) - - tk.MustExec(`set @@tidb_opt_fix_control = ""`) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - - tk.MustExec(`set @@tidb_opt_fix_control = "44823:250"`) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - - tk.MustExec(`set @@tidb_opt_fix_control = "44823:0"`) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(query).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) -} - func TestIssue46760(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -105,15 +43,16 @@ func TestIssue46760(t *testing.T) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) ctx := context.WithValue(context.Background(), core.PlanCacheKeyTestIssue46760, struct{}{}) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/TestIssue46760", "return(true)")) tk.MustExecWithContext(ctx, `prepare st from 'select * from t where a>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831)) func TestCacheable(t *testing.T) { store := testkit.CreateMockStore(t) @@ -340,6 +279,7 @@ func TestGeneralPlanCacheable(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec(`create table t (a int, b int, c int, d int, key(a), key(b))`) tk.MustExec("create table t1(a int, b int, index idx_b(b)) partition by range(a) ( partition p0 values less than (6), partition p1 values less than (11) )") tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") tk.MustExec("create table t3(a int, b int)") @@ -384,12 +324,12 @@ func TestGeneralPlanCacheable(t *testing.T) { for _, q := range unsupported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.False(t, core.GeneralPlanCacheable(stmt, is)) + require.False(t, core.GeneralPlanCacheableWithCtx(tk.Session(), stmt, is)) } for _, q := range supported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.True(t, core.GeneralPlanCacheable(stmt, is)) + require.True(t, core.GeneralPlanCacheableWithCtx(tk.Session(), stmt, is)) } }