Skip to content

Commit

Permalink
stmtctx: use a standalone RangeFallbackHandler (#52738)
Browse files Browse the repository at this point in the history
close #52737
  • Loading branch information
YangKeao committed Apr 22, 2024
1 parent 62f3aea commit cfbabfa
Show file tree
Hide file tree
Showing 21 changed files with 248 additions and 103 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1959,7 +1959,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
RUDetail: ruDetail,
ResourceGroupName: sessVars.StmtCtx.ResourceGroupName,

PlanCacheUnqualified: sessVars.StmtCtx.PlanCacheUnqualified,
PlanCacheUnqualified: sessVars.StmtCtx.PlanCacheUnqualified(),
}
if a.retryCount > 0 {
stmtExecInfo.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime)
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2243,8 +2243,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.IndexUsageCollector = ctx.NewStmtIndexUsageCollector()
}

sc.PlanCacheUnqualified = ""
sc.ForcePlanCache = fixcontrol.GetBoolWithDefault(vars.OptimizerFixControl, fixcontrol.Fix49736, false)
sc.SetForcePlanCache(fixcontrol.GetBoolWithDefault(vars.OptimizerFixControl, fixcontrol.Fix49736, false))
sc.SetAlwaysWarnSkipCache(sc.InExplainStmt && sc.ExplainFormat == "plan_cache")
sc.TblInfo2UnionScan = make(map[*model.TableInfo]bool)
errCount, warnCount := vars.StmtCtx.NumErrorWarnings()
vars.SysErrorCount = errCount
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/contextimpl/sessionctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func (ctx *ExprCtxExtendedImpl) Rng() *mathutil.MysqlRng {
// IsUseCache indicates whether to cache the build expression in plan cache.
// If SetSkipPlanCache is invoked, it should return false.
func (ctx *ExprCtxExtendedImpl) IsUseCache() bool {
return ctx.sctx.GetSessionVars().StmtCtx.UseCache
return ctx.sctx.GetSessionVars().StmtCtx.UseCache()
}

// SetSkipPlanCache sets to skip the plan cache and records the reason.
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/contextimpl/sessionctx_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -282,7 +282,7 @@ func TestSessionBuildContext(t *testing.T) {
require.Same(t, vars.Rng, impl.Rng())

// PlanCache
vars.StmtCtx.UseCache = true
vars.StmtCtx.EnablePlanCache()
require.True(t, impl.IsUseCache())
impl.SetSkipPlanCache(errors.New("mockReason"))
require.False(t, impl.IsUseCache())
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/expression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ func TestEvaluateExprWithNullAndParameters(t *testing.T) {
schema := tableInfoToSchemaForTest(tblInfo)
col0 := schema.Columns[0]

ctx.GetSessionVars().StmtCtx.UseCache = true
ctx.GetSessionVars().StmtCtx.EnablePlanCache()

// cases for parameters
ltWithoutParam, err := newFunctionForTest(ctx, ast.LT, col0, NewOne())
Expand All @@ -76,7 +76,7 @@ func TestEvaluateExprWithNullAndParameters(t *testing.T) {
res = EvaluateExprWithNull(ctx, schema, ltWithParam)
_, isConst := res.(*Constant)
require.True(t, isConst) // this expression is evaluated and skip-plan cache flag is set.
require.True(t, !ctx.GetSessionVars().StmtCtx.UseCache)
require.True(t, !ctx.GetSessionVars().StmtCtx.UseCache())
}

func TestEvaluateExprWithNullNoChangeRetType(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1165,7 +1165,7 @@ func DatumToConstant(d types.Datum, tp byte, flag uint) *Constant {

// ParamMarkerExpression generate a getparam function expression.
func ParamMarkerExpression(ctx variable.SessionVarsProvider, v *driver.ParamMarkerExpr, needParam bool) (*Constant, error) {
useCache := ctx.GetSessionVars().StmtCtx.UseCache
useCache := ctx.GetSessionVars().StmtCtx.UseCache()
isPointExec := ctx.GetSessionVars().StmtCtx.PointExec
tp := types.NewFieldType(mysql.TypeUnspecified)
types.InferParamTypeFromDatum(&v.Datum, tp)
Expand Down
4 changes: 2 additions & 2 deletions pkg/extension/function_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -514,12 +514,12 @@ func TestShouldNotOptimizeExtensionFunc(t *testing.T) {
"my_func2()",
} {
ctx := mock.NewContext()
ctx.GetSessionVars().StmtCtx.UseCache = true
ctx.GetSessionVars().StmtCtx.EnablePlanCache()
expr, err := expression.ParseSimpleExpr(ctx, exprStr)
require.NoError(t, err)
scalar, ok := expr.(*expression.ScalarFunction)
require.True(t, ok)
require.Equal(t, expression.ConstNone, scalar.ConstLevel())
require.False(t, ctx.GetSessionVars().StmtCtx.UseCache)
require.False(t, ctx.GetSessionVars().StmtCtx.UseCache())
}
}
2 changes: 2 additions & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,7 @@ go_library(
"//pkg/util/chunk",
"//pkg/util/codec",
"//pkg/util/collate",
"//pkg/util/context",
"//pkg/util/dbterror",
"//pkg/util/dbterror/exeerrors",
"//pkg/util/dbterror/plannererrors",
Expand Down Expand Up @@ -274,6 +275,7 @@ go_test(
"//pkg/util/benchdaily",
"//pkg/util/chunk",
"//pkg/util/collate",
"//pkg/util/context",
"//pkg/util/dbterror",
"//pkg/util/dbterror/plannererrors",
"//pkg/util/hack",
Expand Down
13 changes: 11 additions & 2 deletions pkg/planner/core/exhaust_physical_plans_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package core

import (
"fmt"
"strings"
"testing"

"github.com/pingcap/tidb/pkg/domain"
Expand All @@ -29,6 +30,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/types"
contextutil "github.com/pingcap/tidb/pkg/util/context"
"github.com/pingcap/tidb/pkg/util/ranger"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -341,8 +343,15 @@ func TestIndexJoinAnalyzeLookUpFilters(t *testing.T) {
}

func checkRangeFallbackAndReset(t *testing.T, ctx base.PlanContext, expectedRangeFallback bool) {
require.Equal(t, expectedRangeFallback, ctx.GetSessionVars().StmtCtx.RangeFallback)
ctx.GetSessionVars().StmtCtx.RangeFallback = false
stmtCtx := ctx.GetSessionVars().StmtCtx
hasRangeFallbackWarn := false
for _, warn := range stmtCtx.GetWarnings() {
hasRangeFallbackWarn = hasRangeFallbackWarn || strings.Contains(warn.Err.Error(), "'tidb_opt_range_max_size' exceeded when building ranges")
}
require.Equal(t, expectedRangeFallback, hasRangeFallbackWarn)
stmtCtx.PlanCacheTracker = contextutil.NewPlanCacheTracker(stmtCtx)
stmtCtx.RangeFallbackHandler = contextutil.NewRangeFallbackHandler(&stmtCtx.PlanCacheTracker, stmtCtx)
stmtCtx.SetWarnings(nil)
}

func TestRangeFallbackForAnalyzeLookUpFilters(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1312,7 +1312,7 @@ func (b *PlanBuilder) buildSelection(ctx context.Context, p LogicalPlan, where a
expressions = append(expressions, expr)
}
cnfExpres := make([]expression.Expression, 0)
useCache := b.ctx.GetSessionVars().StmtCtx.UseCache
useCache := b.ctx.GetSessionVars().StmtCtx.UseCache()
for _, expr := range expressions {
cnfItems := expression.SplitCNFItems(expr)
for _, item := range cnfItems {
Expand Down
27 changes: 15 additions & 12 deletions pkg/planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
driver "github.com/pingcap/tidb/pkg/types/parser_driver"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/collate"
contextutil "github.com/pingcap/tidb/pkg/util/context"
"github.com/pingcap/tidb/pkg/util/dbterror/plannererrors"
"github.com/pingcap/tidb/pkg/util/kvcache"
utilpc "github.com/pingcap/tidb/pkg/util/plancache"
Expand Down Expand Up @@ -158,19 +159,21 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
stmtCtx := sessVars.StmtCtx
cacheEnabled := false
if isNonPrepared {
stmtCtx.CacheType = stmtctx.SessionNonPrepared
stmtCtx.SetCacheType(contextutil.SessionNonPrepared)
cacheEnabled = sctx.GetSessionVars().EnableNonPreparedPlanCache // plan-cache might be disabled after prepare.
} else {
stmtCtx.CacheType = stmtctx.SessionPrepared
stmtCtx.SetCacheType(contextutil.SessionPrepared)
cacheEnabled = sctx.GetSessionVars().EnablePreparedPlanCache
}
stmtCtx.UseCache = stmt.StmtCacheable && cacheEnabled
if stmt.StmtCacheable && cacheEnabled {
stmtCtx.EnablePlanCache()
}
if stmt.UncacheableReason != "" {
stmtCtx.ForceSetSkipPlanCache(errors.NewNoStackError(stmt.UncacheableReason))
}

var bindSQL string
if stmtCtx.UseCache {
if stmtCtx.UseCache() {
var ignoreByBinding bool
bindSQL, ignoreByBinding = bindinfo.MatchSQLBindingForPlanCache(sctx, stmt.PreparedAst.Stmt, &stmt.BindingInfo)
if ignoreByBinding {
Expand All @@ -182,7 +185,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
// rebuild the plan. So we set this value in rc or for update read. In other cases, let it be 0.
var latestSchemaVersion int64

if stmtCtx.UseCache {
if stmtCtx.UseCache() {
if sctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || stmt.ForUpdateRead {
// In Rc or ForUpdateRead, we should check if the information schema has been changed since
// last time. If it changed, we should rebuild the plan. Here, we use a different and more
Expand All @@ -195,7 +198,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
}
}

if stmtCtx.UseCache && stmt.PointGet.Plan != nil { // special code path for fast point plan
if stmtCtx.UseCache() && stmt.PointGet.Plan != nil { // special code path for fast point plan
if plan, names, ok, err := getCachedPointPlan(stmt, sessVars, stmtCtx); ok {
return plan, names, err
}
Expand All @@ -205,7 +208,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
if err != nil {
return nil, nil, err
}
if stmtCtx.UseCache { // for non-point plans
if stmtCtx.UseCache() { // for non-point plans
if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, matchOpts); err != nil || ok {
return plan, names, err
}
Expand Down Expand Up @@ -323,14 +326,14 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
}

// check whether this plan is cacheable.
if stmtCtx.UseCache {
if stmtCtx.UseCache() {
if cacheable, reason := isPlanCacheable(sctx.GetPlanCtx(), p, len(matchOpts.ParamTypes), len(matchOpts.LimitOffsetAndCount), matchOpts.HasSubQuery); !cacheable {
stmtCtx.SetSkipPlanCache(errors.Errorf(reason))
}
}

// put this plan into the plan cache.
if stmtCtx.UseCache {
if stmtCtx.UseCache() {
// rebuild key to exclude kv.TiFlash when stmt is not read only
if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) {
delete(sessVars.IsolationReadEngines, kv.TiFlash)
Expand All @@ -353,7 +356,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
// RebuildPlan4CachedPlan will rebuild this plan under current user parameters.
func RebuildPlan4CachedPlan(p base.Plan) (ok bool) {
sc := p.SCtx().GetSessionVars().StmtCtx
if !sc.UseCache {
if !sc.UseCache() {
return false // plan-cache is disabled for this query
}

Expand All @@ -363,7 +366,7 @@ func RebuildPlan4CachedPlan(p base.Plan) (ok bool) {
sc.AppendWarning(errors.NewNoStackErrorf("skip plan-cache: plan rebuild failed, %s", err.Error()))
return false // fail to rebuild ranges
}
if !sc.UseCache {
if !sc.UseCache() {
// in this case, the UseCache flag changes from `true` to `false`, then there must be some
// over-optimized operations were triggered, return `false` for safety here.
return false
Expand Down Expand Up @@ -773,7 +776,7 @@ func CheckPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infosche
// short paths for these executions, currently "point select" and "point update"
func tryCachePointPlan(_ context.Context, sctx base.PlanContext,
stmt *PlanCacheStmt, p base.Plan, names types.NameSlice) error {
if !sctx.GetSessionVars().StmtCtx.UseCache {
if !sctx.GetSessionVars().StmtCtx.UseCache() {
return nil
}
var (
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,7 +348,7 @@ func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) bool {
// - This should NOT be cached and should already be having PartitionIdx set!
// 2) Converted to PointGet from checkTblIndexForPointPlan
// and it does not have the PartitionIdx set
if !p.SCtx().GetSessionVars().StmtCtx.UseCache &&
if !p.SCtx().GetSessionVars().StmtCtx.UseCache() &&
p.PartitionIdx != nil {
return false
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1536,7 +1536,7 @@ func (p *rangePruner) extractDataForPrune(sctx base.PlanContext, expr expression
// the constExpr may not a really constant when coming here.
// Suppose the partition expression is 'a + b' and we have a condition 'a = 2',
// the constExpr is '2 + b' after the replacement which we can't evaluate.
if !expression.ConstExprConsiderPlanCache(constExpr, sctx.GetSessionVars().StmtCtx.UseCache) {
if !expression.ConstExprConsiderPlanCache(constExpr, sctx.GetSessionVars().StmtCtx.UseCache()) {
return ret, false
}
c, isNull, err := constExpr.EvalInt(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})
Expand Down
4 changes: 2 additions & 2 deletions pkg/session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2599,8 +2599,8 @@ func (s *session) GetRangerCtx() *rangerctx.RangerContext {
OptPrefixIndexSingleScan: s.GetSessionVars().OptPrefixIndexSingleScan,
OptimizerFixControl: s.GetSessionVars().OptimizerFixControl,

// TODO: avoid using the whole `StmtCtx` here.
RangeFallbackHandler: s.GetSessionVars().StmtCtx,
PlanCacheTracker: &s.GetSessionVars().StmtCtx.PlanCacheTracker,
RangeFallbackHandler: &s.GetSessionVars().StmtCtx.RangeFallbackHandler,
}
})

Expand Down

0 comments on commit cfbabfa

Please sign in to comment.