diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index bf2dc51efcf99..73a9ec02f40dd 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -1235,9 +1235,9 @@ func TestCTE4PlanCache(t *testing.T) { tk.MustExec("set @a=5, @b=4, @c=2, @d=1;") tk.MustQuery("execute stmt using @d, @a").Check(testkit.Rows("1", "2", "3", "4", "5")) tk.MustQuery("execute stmt using @d, @b").Check(testkit.Rows("1", "2", "3", "4")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) tk.MustQuery("execute stmt using @c, @b").Check(testkit.Rows("2", "3", "4")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) // Two seed parts. tk.MustExec("prepare stmt from 'with recursive cte1 as (" + @@ -1250,7 +1250,7 @@ func TestCTE4PlanCache(t *testing.T) { tk.MustExec("set @a=10, @b=2;") tk.MustQuery("execute stmt using @a").Check(testkit.Rows("1", "2", "2", "3", "3", "4", "4", "5", "5", "6", "6", "7", "7", "8", "8", "9", "9", "10", "10")) tk.MustQuery("execute stmt using @b").Check(testkit.Rows("1", "2", "2")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) // Two recursive parts. tk.MustExec("prepare stmt from 'with recursive cte1 as (" + @@ -1265,25 +1265,27 @@ func TestCTE4PlanCache(t *testing.T) { tk.MustExec("set @a=1, @b=2, @c=3, @d=4, @e=5;") tk.MustQuery("execute stmt using @c, @b, @e;").Check(testkit.Rows("1", "2", "2", "3", "3", "3", "4", "4", "5", "5", "5", "6", "6")) tk.MustQuery("execute stmt using @b, @a, @d;").Check(testkit.Rows("1", "2", "2", "2", "3", "3", "3", "4", "4", "4")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(a int);") 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.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")) 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")) + //tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: PhysicalApply plan is un-cacheable")) 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("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("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) } func TestValidity4PlanCache(t *testing.T) { diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 6702a4c582a97..715de6bfdf546 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -46,6 +46,8 @@ import ( var ( // PlanCacheKeyTestIssue43667 is only for test. PlanCacheKeyTestIssue43667 struct{} + // PlanCacheKeyTestIssue46760 is only for test. + PlanCacheKeyTestIssue46760 struct{} ) // SetParameterValuesIntoSCtx sets these parameters into session context. diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index c8fd334482c7d..84f58dcf0d1ac 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -2357,8 +2357,8 @@ func TestNonPreparedPlanExplainWarning(t *testing.T) { "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: queries that access partitioning table are not supported", - "skip non-prepared plan-cache: queries that access partitioning table 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", @@ -2368,8 +2368,8 @@ func TestNonPreparedPlanExplainWarning(t *testing.T) { "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: queries that have generated columns are not supported", - "skip non-prepared plan-cache: queries that have generated columns are not supported", + "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", diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 56a0f38af0c61..db7267e0aba61 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -138,7 +138,7 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, reason = "plan cache is disabled" } else { if isPrepStmt { - cacheable, reason = CacheableWithCtx(sctx, paramStmt, ret.InfoSchema) + cacheable, reason = IsASTCacheable(ctx, sctx, paramStmt, ret.InfoSchema) } else { cacheable = true // it is already checked here } diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 3fcc7d007f881..8fca84aa58809 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -15,8 +15,11 @@ package core import ( + "context" + "errors" "fmt" "math" + "strings" "sync" "github.com/pingcap/tidb/expression" @@ -31,20 +34,28 @@ import ( "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/filter" + "github.com/pingcap/tidb/util/intest" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) // 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. func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { - c, _ := CacheableWithCtx(nil, node, is) + c, _ := IsASTCacheable(nil, nil, node, is) return c } // 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) { + return IsASTCacheable(nil, sctx, node, is) +} + +// IsASTCacheable checks whether the input ast(query) is cacheable. // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect -func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { +func IsASTCacheable(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { _, isSelect := node.(*ast.SelectStmt) _, isUpdate := node.(*ast.UpdateStmt) _, isInsert := node.(*ast.InsertStmt) @@ -54,6 +65,7 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info return false, "not a SELECT/UPDATE/INSERT/DELETE/SET statement" } checker := cacheableChecker{ + ctx: ctx, sctx: sctx, cacheable: true, schema: is, @@ -66,6 +78,7 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info // cacheableChecker checks whether a query can be cached: type cacheableChecker struct { + ctx context.Context sctx sessionctx.Context cacheable bool schema infoschema.InfoSchema @@ -185,26 +198,8 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren } case *ast.TableName: if checker.schema != nil { - if isPartitionTable(checker.schema, node) { - // Temporary disable prepared plan cache until https://github.com/pingcap/tidb/issues/33031 - // is fixed and additional tests with dynamic partition prune mode has been added. - /* - if checker.sctx != nil && checker.sctx.GetSessionVars().UseDynamicPartitionPrune() { - return in, false // dynamic-mode for partition tables can use plan-cache - } - */ - checker.cacheable = false - checker.reason = "query accesses partitioned tables is un-cacheable" - return in, true - } - if hasGeneratedCol(checker.schema, node) { - checker.cacheable = false - checker.reason = "query accesses generated columns is un-cacheable" - return in, true - } - if isTempTable(checker.schema, node) { - checker.cacheable = false - checker.reason = "query accesses temporary tables is un-cacheable" + checker.cacheable, checker.reason = checkTableCacheable(checker.ctx, checker.sctx, checker.schema, node, false) + if !checker.cacheable { return in, true } } @@ -496,39 +491,7 @@ func (checker *nonPreparedPlanCacheableChecker) Enter(in ast.Node) (out ast.Node return in, !checker.cacheable } if checker.schema != nil { - tb, err := checker.schema.TableByName(node.Schema, node.Name) - if err != nil { - checker.cacheable = false - checker.reason = "table cannot be found in schema" - return in, !checker.cacheable - } - if tb.Meta().GetPartitionInfo() != nil { - checker.cacheable = false - checker.reason = "queries that access partitioning table are not supported" - return in, !checker.cacheable - } - for _, col := range tb.Cols() { - if col.IsGenerated() { - checker.cacheable = false - checker.reason = "queries that have generated columns are not supported" - return in, !checker.cacheable - } - } - if tb.Meta().TempTableType != model.TempTableNone { - checker.cacheable = false - checker.reason = "queries that access temporary tables are not supported" - return in, !checker.cacheable - } - if tb.Meta().IsView() { - checker.cacheable = false - checker.reason = "queries that access views are not supported" - return in, !checker.cacheable - } - if !tb.Type().IsNormalTable() { - checker.cacheable = false - checker.reason = "queries that access in-memory tables" - return in, !checker.cacheable - } + checker.cacheable, checker.reason = checkTableCacheable(nil, checker.sctx, checker.schema, node, true) } return in, !checker.cacheable } @@ -554,20 +517,6 @@ func (*nonPreparedPlanCacheableChecker) isFilterNode(node ast.Node) bool { return false } -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 getColType(schema infoschema.InfoSchema, tbl *ast.TableName, col *ast.ColumnName) (colType byte, found bool) { if tbl == nil { return 0, false @@ -584,30 +533,6 @@ func getColType(schema infoschema.InfoSchema, tbl *ast.TableName, col *ast.Colum return 0, 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 -} - // 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 @@ -696,3 +621,55 @@ func getMaxParamLimit(sctx sessionctx.Context) int { 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 + if tableSchema.L == "" { + tableSchema.O = sctx.GetSessionVars().CurrentDB + 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") + } + if err != nil { + sql := sctx.GetSessionVars().StmtCtx.OriginalSQL + if len(sql) > 256 { + sql = sql[:256] + } + logutil.BgLogger().Warn("find table failed", zap.Error(err), zap.String("sql", sql), + zap.String("table_schema", tableSchema.O), zap.String("table_name", node.Name.O)) + return false, fmt.Sprintf("find table %s.%s failed: %s", tableSchema, node.Name, err.Error()) + } + + if tb.Meta().GetPartitionInfo() != nil { + // Temporary disable prepared plan cache until https://github.com/pingcap/tidb/issues/33031 + // is fixed and additional tests with dynamic partition prune mode has been added. + /* + if checker.sctx != nil && checker.sctx.GetSessionVars().UseDynamicPartitionPrune() { + return in, false // dynamic-mode for partition tables can use plan-cache + } + */ + return false, "query accesses partitioned tables is un-cacheable" + } + for _, col := range tb.Cols() { + if col.IsGenerated() { + return false, "query accesses generated columns is un-cacheable" + } + } + if tb.Meta().TempTableType != model.TempTableNone { + return false, "query accesses temporary tables is un-cacheable" + } + + if isNonPrep { // non-prep plan cache is stricter + if tb.Meta().IsView() { + return false, "queries that access views are not supported" + } + if !tb.Type().IsNormalTable() { + return false, "queries that access in-memory tables" + } + } + + return true, "" +} diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index 70ff91872d36d..edd3509d93ea5 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -15,6 +15,7 @@ package core_test import ( + "context" "fmt" "strings" "testing" @@ -89,6 +90,26 @@ func TestFixControl44823(t *testing.T) { 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) + tk.MustExec("use test") + tk.MustExec(`create table t (a int)`) + tk.MustExec(`prepare st from 'select * from t where a