Skip to content

Commit

Permalink
planner: small refactor about plan cache checker (#41357)
Browse files Browse the repository at this point in the history
ref #36598
  • Loading branch information
qw4990 committed Feb 14, 2023
1 parent 8cf2ca8 commit 71ed267
Show file tree
Hide file tree
Showing 2 changed files with 59 additions and 115 deletions.
111 changes: 4 additions & 107 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,

paramTypes := parseParamTypes(sctx, params)

if stmtCtx.UseCache && stmtAst.CachedPlan != nil { // for point query plan
if stmtCtx.UseCache && stmtAst.CachedPlan != nil { // special code path for fast point plan
if plan, names, ok, err := getCachedPointPlan(stmtAst, sessVars, stmtCtx); ok {
return plan, names, err
}
Expand Down Expand Up @@ -287,7 +287,9 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared

// check whether this plan is cacheable.
if stmtCtx.UseCache {
checkPlanCacheability(sctx, p, len(paramTypes), len(limitParams))
if cacheable, reason := isPlanCacheable(sctx, p, len(paramTypes), len(limitParams)); !cacheable {
stmtCtx.SetSkipPlanCache(errors.Errorf(reason))
}
}

// put this plan into the plan cache.
Expand All @@ -311,54 +313,6 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
return p, names, err
}

// checkPlanCacheability checks whether this plan is cacheable and set to skip plan cache if it's uncacheable.
func checkPlanCacheability(sctx sessionctx.Context, p Plan, paramNum int, limitParamNum int) {
stmtCtx := sctx.GetSessionVars().StmtCtx
var pp PhysicalPlan
switch x := p.(type) {
case *Insert:
pp = x.SelectPlan
case *Update:
pp = x.SelectPlan
case *Delete:
pp = x.SelectPlan
case PhysicalPlan:
pp = x
default:
stmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: unexpected un-cacheable plan %v", p.ExplainID().String()))
return
}
if pp == nil { // simple DML statements
return
}

if useTiFlash(pp) {
stmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: TiFlash plan is un-cacheable"))
return
}

// We only cache the tableDual plan when the number of parameters are zero.
if containTableDual(pp) && paramNum > 0 {
stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: get a TableDual plan"))
return
}

if containShuffleOperator(pp) {
stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: get a Shuffle plan"))
return
}

if accessMVIndexWithIndexMerge(pp) {
stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: the plan with IndexMerge accessing Multi-Valued Index is un-cacheable"))
return
}

// before cache the param limit plan, check switch
if limitParamNum != 0 && !sctx.GetSessionVars().EnablePlanCacheForParamLimit {
stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off"))
}
}

// RebuildPlan4CachedPlan will rebuild this plan under current user parameters.
func RebuildPlan4CachedPlan(p Plan) error {
sc := p.SCtx().GetSessionVars().StmtCtx
Expand Down Expand Up @@ -728,63 +682,6 @@ func tryCachePointPlan(_ context.Context, sctx sessionctx.Context,
return err
}

func containTableDual(p PhysicalPlan) bool {
_, isTableDual := p.(*PhysicalTableDual)
if isTableDual {
return true
}
childContainTableDual := false
for _, child := range p.Children() {
childContainTableDual = childContainTableDual || containTableDual(child)
}
return childContainTableDual
}

func containShuffleOperator(p PhysicalPlan) bool {
if _, isShuffle := p.(*PhysicalShuffle); isShuffle {
return true
}
if _, isShuffleRecv := p.(*PhysicalShuffleReceiverStub); isShuffleRecv {
return true
}
return false
}

func accessMVIndexWithIndexMerge(p PhysicalPlan) bool {
if idxMerge, ok := p.(*PhysicalIndexMergeReader); ok {
if idxMerge.AccessMVIndex {
return true
}
}

for _, c := range p.Children() {
if accessMVIndexWithIndexMerge(c) {
return true
}
}
return false
}

// useTiFlash used to check whether the plan use the TiFlash engine.
func useTiFlash(p PhysicalPlan) bool {
switch x := p.(type) {
case *PhysicalTableReader:
switch x.StoreType {
case kv.TiFlash:
return true
default:
return false
}
default:
if len(p.Children()) > 0 {
for _, plan := range p.Children() {
return useTiFlash(plan)
}
}
}
return false
}

// GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key.
func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string, bool) {
useBinding := sctx.GetSessionVars().UsePlanBaselines
Expand Down
63 changes: 55 additions & 8 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
Expand All @@ -27,13 +28,13 @@ 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.
func Cacheable(node ast.Node, is infoschema.InfoSchema) bool {
c, _ := CacheableWithCtx(nil, node, is)
return c
}

// CacheableWithCtx checks whether the input ast is cacheable.
// CacheableWithCtx 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) {
Expand All @@ -54,12 +55,7 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info
return checker.cacheable, checker.reason
}

// cacheableChecker checks whether a query's plan can be cached, querys that:
// 1. have ExistsSubqueryExpr, or
// 2. have VariableExpr
//
// will not be cached currently.
// NOTE: we can add more rules in the future.
// cacheableChecker checks whether a query can be cached:
type cacheableChecker struct {
sctx sessionctx.Context
cacheable bool
Expand Down Expand Up @@ -320,3 +316,54 @@ func isPartitionTable(schema infoschema.InfoSchema, tn *ast.TableName) bool {
}
return false
}

// isPlanCacheable returns whether this plan is cacheable and the reason if not.
func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum int) (cacheable bool, reason string) {
var pp PhysicalPlan
switch x := p.(type) {
case *Insert:
pp = x.SelectPlan
case *Update:
pp = x.SelectPlan
case *Delete:
pp = x.SelectPlan
case PhysicalPlan:
pp = x
default:
return false, fmt.Sprintf("skip plan-cache: unexpected un-cacheable plan %v", p.ExplainID().String())
}
if pp == nil { // simple DML statements
return true, ""
}
if limitParamNum != 0 && !sctx.GetSessionVars().EnablePlanCacheForParamLimit {
return false, "skip plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off"
}
return isPhysicalPlanCacheable(sctx, pp, paramNum, limitParamNum)
}

// isPhysicalPlanCacheable returns whether this physical plan is cacheable and return the reason if not.
func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, limitParamNum int) (cacheable bool, reason string) {
switch x := p.(type) {
case *PhysicalTableDual:
if paramNum > 0 {
return false, "skip plan-cache: get a TableDual plan"
}
case *PhysicalTableReader:
if x.StoreType == kv.TiFlash {
return false, "skip plan-cache: TiFlash plan is un-cacheable"
}
case *PhysicalShuffle, *PhysicalShuffleReceiverStub:
return false, "skip plan-cache: get a Shuffle plan"
case *PhysicalIndexMergeReader:
if x.AccessMVIndex {
return false, "skip plan-cache: the plan with IndexMerge accessing Multi-Valued Index is un-cacheable"
}
}

for _, c := range p.Children() {
if cacheable, reason = isPhysicalPlanCacheable(sctx, c, paramNum, limitParamNum); !cacheable {
return cacheable, reason
}
}
return true, ""
}

0 comments on commit 71ed267

Please sign in to comment.