Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#46831
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
qw4990 authored and ti-chi-bot committed Sep 11, 2023
1 parent 9d5d781 commit 997a7a4
Show file tree
Hide file tree
Showing 6 changed files with 109 additions and 104 deletions.
14 changes: 8 additions & 6 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 (" +
Expand All @@ -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 (" +
Expand All @@ -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) {
Expand Down
2 changes: 2 additions & 0 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ import (
var (
// PlanCacheKeyTestIssue43667 is for test.
PlanCacheKeyTestIssue43667 struct{}
// PlanCacheKeyTestIssue46760 is only for test.
PlanCacheKeyTestIssue46760 struct{}
)

// SetParameterValuesIntoSCtx sets these parameters into session context.
Expand Down
8 changes: 4 additions & 4 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1994,8 +1994,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",
Expand All @@ -2005,8 +2005,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",
Expand Down
2 changes: 1 addition & 1 deletion planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,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
}
Expand Down
166 changes: 73 additions & 93 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,15 @@
package core

import (
"context"
"errors"
"fmt"
"math"
<<<<<<< HEAD
"strconv"
=======
"strings"
>>>>>>> bc80772052f (planner: Adjust the log level and returned value when `cacheableChecker` check `*ast.TableName` nodes (#46831))
"sync"

"github.com/pingcap/tidb/expression"
Expand All @@ -32,20 +38,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)
Expand All @@ -55,6 +69,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 @@ -67,6 +82,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
Expand Down Expand Up @@ -186,26 +202,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 @@ -497,39 +495,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
}
Expand All @@ -555,20 +521,6 @@ func (checker *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
Expand All @@ -585,30 +537,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
Expand Down Expand Up @@ -701,3 +629,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, ""
}
21 changes: 21 additions & 0 deletions planner/core/plan_cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
package core_test

import (
"context"
"fmt"
"strings"
"testing"
Expand Down Expand Up @@ -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<?'`)
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{}{})
tk.MustExecWithContext(ctx, `prepare st from 'select * from t where a<?'`)
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip prepared 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)
mockCtx := mock.NewContext()
Expand Down

0 comments on commit 997a7a4

Please sign in to comment.