Skip to content

Commit

Permalink
planner: Adjust the log level and returned value when `cacheableCheck…
Browse files Browse the repository at this point in the history
…er` check `*ast.TableName` nodes (#46831) (#46876)

close #46760
  • Loading branch information
ti-chi-bot authored Sep 12, 2023
1 parent 10a4f67 commit be57cf0
Show file tree
Hide file tree
Showing 4 changed files with 97 additions and 64 deletions.
2 changes: 2 additions & 0 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1288,13 +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 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"))

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 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"))
Expand Down
5 changes: 5 additions & 0 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,11 @@ import (
"go.uber.org/zap"
)

var (
// PlanCacheKeyTestIssue46760 is only for test.
PlanCacheKeyTestIssue46760 struct{}
)

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
Expand Down
125 changes: 63 additions & 62 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,12 @@
package core

import (
"context"
"errors"
"fmt"
"strings"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/ast"
Expand All @@ -27,16 +31,23 @@ import (
"go.uber.org/zap"
)

// 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.
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 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) {
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)
Expand All @@ -46,6 +57,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,
Expand All @@ -61,6 +73,7 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info
// will not be cached currently.
// NOTE: we can add more rules in the future.
type cacheableChecker struct {
ctx context.Context
sctx sessionctx.Context
cacheable bool
schema infoschema.InfoSchema
Expand Down Expand Up @@ -158,26 +171,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
}
}
Expand Down Expand Up @@ -262,18 +257,7 @@ func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, sk
return in, true
case *ast.TableName:
if checker.schema != nil {
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, _ = checkTableCacheable(nil, checker.sctx, checker.schema, node, true)
}
}
return in, false
Expand All @@ -284,40 +268,57 @@ func (checker *generalPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok
return in, checker.cacheable
}

func hasGeneratedCol(schema infoschema.InfoSchema, tn *ast.TableName) bool {
tb, err := schema.TableByName(tn.Schema, tn.Name)
// 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)

failpoint.Inject("TestIssue46760", func(val failpoint.Value) {
if val.(bool) {
err = errors.New("mock error")
}
})
if err != nil {
logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err))
return false
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 true
return false, "query accesses generated columns is un-cacheable"
}
}
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, "query accesses temporary tables is un-cacheable"
}
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
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 false

return true, ""
}
29 changes: 27 additions & 2 deletions planner/core/plan_cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,10 @@
package core_test

import (
"context"
"testing"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
Expand All @@ -29,6 +31,28 @@ import (
"github.com/stretchr/testify/require"
)

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<?'`)
tk.MustExec(`set @a=1`)
tk.MustQuery(`execute st using @a`).Check(testkit.Rows())
tk.MustQuery(`execute st using @a`).Check(testkit.Rows())
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<?'`)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/TestIssue46760"))
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: find table test.t failed: mock error"))
tk.MustExec(`set @a=1`)
tk.MustQuery(`execute st using @a`).Check(testkit.Rows())
tk.MustQuery(`execute st using @a`).Check(testkit.Rows())
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0"))
}

func TestCacheable(t *testing.T) {
store := testkit.CreateMockStore(t)

Expand Down Expand Up @@ -255,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)")
Expand Down Expand Up @@ -299,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))
}
}

0 comments on commit be57cf0

Please sign in to comment.