Skip to content

Commit

Permalink
planner: enrich diagnosis info for plan-cache (#39457)
Browse files Browse the repository at this point in the history
close #39147
  • Loading branch information
qw4990 authored Dec 1, 2022
1 parent 9a7341e commit 7b91e6a
Show file tree
Hide file tree
Showing 12 changed files with 86 additions and 8 deletions.
1 change: 1 addition & 0 deletions expression/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
6 changes: 6 additions & 0 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 (
Expand Down Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 <cmp> 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
Expand Down
1 change: 1 addition & 0 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
16 changes: 13 additions & 3 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
Expand Down Expand Up @@ -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"))
}
Expand Down Expand Up @@ -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"))
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}
}

Expand Down
32 changes: 32 additions & 0 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}
6 changes: 5 additions & 1 deletion planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 = ""
Expand Down
25 changes: 21 additions & 4 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
package core

import (
"fmt"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/ast"
Expand All @@ -27,28 +29,29 @@ 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,
cacheable: true,
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:
Expand All @@ -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.
Expand All @@ -70,61 +74,71 @@ 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.DeleteStmt:
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.UpdateStmt:
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
}
}
case *ast.GroupByClause:
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
}
}
case *ast.Limit:
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:
Expand All @@ -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
}
}
Expand Down
3 changes: 3 additions & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}
}
}
Expand Down

0 comments on commit 7b91e6a

Please sign in to comment.