diff --git a/expression/BUILD.bazel b/expression/BUILD.bazel index fc1752ef19e63..c7304642c544a 100644 --- a/expression/BUILD.bazel +++ b/expression/BUILD.bazel @@ -111,6 +111,7 @@ go_library( "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_tipb//go-tipb", + "@com_github_pkg_errors//:errors", "@com_github_tikv_client_go_v2//oracle", "@org_golang_x_exp//slices", "@org_golang_x_tools//container/intsets", diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 3baa86b635ba6..4411f4b6b90ff 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" + "github.com/pkg/errors" ) var ( @@ -1575,6 +1576,11 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express // here and skip this refine operation in all other cases for safety. if (arg0IsInt && !arg0IsCon && arg1IsString && arg1IsCon) || (arg1IsInt && !arg1IsCon && arg0IsString && arg0IsCon) { ctx.GetSessionVars().StmtCtx.SkipPlanCache = true + if arg1IsString { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg1.String())) + } else { // arg0IsString + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg0.String())) + } RemoveMutableConst(ctx, args) } else { return args diff --git a/expression/builtin_other.go b/expression/builtin_other.go index 0f4bd85d45b43..c62278c2bd101 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -166,6 +166,7 @@ func (c *inFunctionClass) verifyArgs(ctx sessionctx.Context, args []Expression) if constant.Value.GetInt64() < 0 { if MaybeOverOptimized4PlanCache(ctx, args) { ctx.GetSessionVars().StmtCtx.SkipPlanCache = true + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: Bit Column in (%v)", constant.Value.GetInt64())) } continue } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 522dedacca24a..d0ca6e6f8e4cf 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1554,6 +1554,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field // To keep the result be compatible with MySQL, refine `int non-constant str constant` // here and skip this refine operation in all other cases for safety. er.sctx.GetSessionVars().StmtCtx.SkipPlanCache = true + er.sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", c.String())) expression.RemoveMutableConst(er.sctx, []expression.Expression{c}) } else { continue diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 52eab05629d59..12f298e6cd2ba 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -903,6 +903,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // We should uncache the tableDual plan. if expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) { ds.ctx.GetSessionVars().StmtCtx.SkipPlanCache = true + ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: get a TableDual plan")) } dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index f9c58ace525f8..ce4b299cdf328 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7781,7 +7781,10 @@ func TestPlanCacheForTableRangeFallback(t *testing.T) { tk.MustExec("prepare stmt from 'select * from t where a in (?, ?, ?, ?, ?) and b > 1'") tk.MustExec("set @a=10, @b=20, @c=30, @d=40, @e=50") tk.MustExec("execute stmt using @a, @b, @c, @d, @e") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 10 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 10 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt using @a, @b, @c, @d, @e") // The plan with range fallback is not cached. tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) @@ -7828,7 +7831,9 @@ func TestPlanCacheForIndexRangeFallback(t *testing.T) { tk.MustExec("prepare stmt2 from 'select * from t where a in (?, ?, ?, ?, ?) and b in (?, ?, ?, ?, ?)'") tk.MustExec("set @a='aa', @b='bb', @c='cc', @d='dd', @e='ee', @f='ff', @g='gg', @h='hh', @i='ii', @j='jj'") tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 1330 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1330 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } @@ -7984,7 +7989,12 @@ func TestPlanCacheForIndexJoinRangeFallback(t *testing.T) { tk.MustExec("prepare stmt2 from 'select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.d where t1.b in (?, ?, ?, ?, ?)'") tk.MustExec("set @a='a', @b='b', @c='c', @d='d', @e='e'") tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 1275 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1275 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long", + "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d96f70462cc33..8b281cbc1b7cb 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -668,6 +668,7 @@ func setupFineGrainedShuffleInternal(plan PhysicalPlan, helper *fineGrainedShuff func checkPlanCacheable(sctx sessionctx.Context, plan PhysicalPlan) { if sctx.GetSessionVars().StmtCtx.UseCache && useTiFlash(plan) { sctx.GetSessionVars().StmtCtx.SkipPlanCache = true + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: TiFlash plan is un-cacheable")) } } diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index d1f51bc296c07..868f35229ce6e 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -176,3 +176,35 @@ func TestIssue38710(t *testing.T) { require.Equal(t, 2, len(rows.Rows())) tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // can not use the cache because the types for @a and @b are not equal to the cached plan } + +func TestPlanCacheDiagInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, key(a), key(b))") + + tk.MustExec("prepare stmt from 'select * from t where a in (select a from t)'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has sub-queries is un-cacheable")) + + tk.MustExec("prepare stmt from 'select /*+ ignore_plan_cache() */ * from t'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: ignore plan cache by hint")) + + tk.MustExec("prepare stmt from 'select * from t limit ?'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable")) + + tk.MustExec("prepare stmt from 'select * from t limit ?, 1'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?, 10' is un-cacheable")) + + tk.MustExec("prepare stmt from 'select * from t order by ?'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'order by ?' is un-cacheable")) + + tk.MustExec("prepare stmt from 'select * from t where a=?'") + tk.MustExec("set @a='123'") + tk.MustExec("execute stmt using @a") // '123' -> 123 + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: '123' may be converted to INT")) + + tk.MustExec("prepare stmt from 'select * from t where a=? and a=?'") + tk.MustExec("set @a=1, @b=1") + tk.MustExec("execute stmt using @a, @b") // a=1 and a=1 -> a=1 + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: some parameters may be overwritten")) +} diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 0abecafaf7ec8..4229e2b134f06 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -117,7 +117,11 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, if !vars.EnablePreparedPlanCache { prepared.UseCache = false } else { - prepared.UseCache = CacheableWithCtx(sctx, stmt, ret.InfoSchema) + cacheable, reason := CacheableWithCtx(sctx, stmt, ret.InfoSchema) + prepared.UseCache = cacheable + if !cacheable { + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: " + reason)) + } selectStmtNode, normalizedSQL4PC, digest4PC, err = ExtractSelectAndNormalizeDigest(stmt, vars.CurrentDB) if err != nil || selectStmtNode == nil { normalizedSQL4PC = "" diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 0e4e28250ad4c..55b1e45cd0e99 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -15,6 +15,8 @@ package core import ( + "fmt" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/ast" @@ -27,20 +29,21 @@ import ( // Cacheable checks whether the input ast is cacheable with empty session context, which is mainly for testing. func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { - return CacheableWithCtx(nil, node, is) + c, _ := CacheableWithCtx(nil, node, is) + return c } // CacheableWithCtx checks whether the input ast 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 { +func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { _, isSelect := node.(*ast.SelectStmt) _, isUpdate := node.(*ast.UpdateStmt) _, isInsert := node.(*ast.InsertStmt) _, isDelete := node.(*ast.DeleteStmt) _, isSetOpr := node.(*ast.SetOprStmt) if !(isSelect || isUpdate || isInsert || isDelete || isSetOpr) { - return false + return false, "not a SELECT/UPDATE/INSERT/DELETE/SET statement" } checker := cacheableChecker{ sctx: sctx, @@ -48,7 +51,7 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info schema: is, } node.Accept(&checker) - return checker.cacheable + return checker.cacheable, checker.reason } // cacheableChecker checks whether a query's plan can be cached, querys that: @@ -61,6 +64,7 @@ type cacheableChecker struct { sctx sessionctx.Context cacheable bool schema infoschema.InfoSchema + reason string // reason why cannot use plan-cache } // Enter implements Visitor interface. @@ -70,6 +74,7 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren for _, hints := range node.TableHints { if hints.HintName.L == HintIgnorePlanCache { checker.cacheable = false + checker.reason = "ignore plan cache by hint" return in, true } } @@ -77,6 +82,7 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren for _, hints := range node.TableHints { if hints.HintName.L == HintIgnorePlanCache { checker.cacheable = false + checker.reason = "ignore plan cache by hint" return in, true } } @@ -84,21 +90,25 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren for _, hints := range node.TableHints { if hints.HintName.L == HintIgnorePlanCache { checker.cacheable = false + checker.reason = "ignore plan cache by hint" return in, true } } case *ast.VariableExpr, *ast.ExistsSubqueryExpr, *ast.SubqueryExpr: checker.cacheable = false + checker.reason = "query has sub-queries is un-cacheable" return in, true case *ast.FuncCallExpr: if _, found := expression.UnCacheableFunctions[node.FnName.L]; found { checker.cacheable = false + checker.reason = fmt.Sprintf("query has '%v' is un-cacheable", node.FnName.L) return in, true } case *ast.OrderByClause: for _, item := range node.Items { if _, isParamMarker := item.Expr.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false + checker.reason = "query has 'order by ?' is un-cacheable" return in, true } } @@ -106,6 +116,7 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren for _, item := range node.Items { if _, isParamMarker := item.Expr.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false + checker.reason = "query has 'group by ?' is un-cacheable" return in, true } } @@ -113,18 +124,21 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren if node.Count != nil { if _, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false + checker.reason = "query has 'limit ?' is un-cacheable" return in, true } } if node.Offset != nil { if _, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false + checker.reason = "query has 'limit ?, 10' is un-cacheable" return in, true } } case *ast.FrameBound: if _, ok := node.Expr.(*driver.ParamMarkerExpr); ok { checker.cacheable = false + checker.reason = "query has ? in window function frames is un-cacheable" return in, true } case *ast.TableName: @@ -138,14 +152,17 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren } */ 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" return in, true } } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index c8fdc2ce32f2e..ec30fcb5fee35 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -1093,6 +1093,9 @@ func (sc *StatementContext) RecordRangeFallback(rangeMaxSize int64) { // If range fallback happens, it means ether the query is unreasonable(for example, several long IN lists) or tidb_opt_range_max_size is too small // and the generated plan is probably suboptimal. In that case we don't put it into plan cache. sc.SkipPlanCache = true + if sc.UseCache { + sc.AppendWarning(errors.Errorf("skip plan-cache: in-list is too long")) + } if !sc.RangeFallback { sc.AppendWarning(errors.Errorf("Memory capacity of %v bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", rangeMaxSize)) sc.RangeFallback = true diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index cbeed5b6364de..596d27c27d9d8 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -623,6 +623,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a=@x and a=@y` --> `a=@x if @x==@y` sctx.GetSessionVars().StmtCtx.SkipPlanCache = true + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: some parameters may be overwritten")) } } }