From 5dd6f12f4422c8cc23d225261d1335bbbd31fa1f Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 14 Feb 2023 13:02:01 +0800 Subject: [PATCH] planner: small refactor about plan cache checker (#41357) ref pingcap/tidb#36598 --- planner/core/plan_cache.go | 111 +------------------------ planner/core/plan_cacheable_checker.go | 63 ++++++++++++-- 2 files changed, 59 insertions(+), 115 deletions(-) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 7e8d691aaa1ae..df09f65b92f0e 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -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 } @@ -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. @@ -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 @@ -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 diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 3da0f285cd9bf..46cbb7a816939 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -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" @@ -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) { @@ -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 @@ -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, "" +}