From cfbabfa705dbbc97c8e1e00af7f2d2620406304c Mon Sep 17 00:00:00 2001 From: YangKeao Date: Mon, 22 Apr 2024 21:04:39 +0800 Subject: [PATCH] stmtctx: use a standalone `RangeFallbackHandler` (#52738) close pingcap/tidb#52737 --- pkg/executor/adapter.go | 2 +- pkg/executor/executor.go | 4 +- pkg/expression/contextimpl/sessionctx.go | 2 +- pkg/expression/contextimpl/sessionctx_test.go | 2 +- pkg/expression/expression_test.go | 4 +- pkg/expression/util.go | 2 +- pkg/extension/function_test.go | 4 +- pkg/planner/core/BUILD.bazel | 2 + .../core/exhaust_physical_plans_test.go | 13 +- pkg/planner/core/logical_plan_builder.go | 2 +- pkg/planner/core/plan_cache.go | 27 +-- pkg/planner/core/point_get_plan.go | 2 +- pkg/planner/core/rule_partition_processor.go | 2 +- pkg/session/session.go | 4 +- pkg/sessionctx/stmtctx/stmtctx.go | 73 ++------ pkg/util/context/BUILD.bazel | 2 + pkg/util/context/plancache.go | 175 ++++++++++++++++++ pkg/util/mock/context.go | 4 +- pkg/util/ranger/BUILD.bazel | 1 + pkg/util/ranger/context/context.go | 11 +- pkg/util/ranger/ranger_test.go | 13 +- 21 files changed, 248 insertions(+), 103 deletions(-) create mode 100644 pkg/util/context/plancache.go diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index cc0a56728cde5..5d9a51dd7c775 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -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) diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index deaa229b2c144..16aa229f51bfe 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -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 diff --git a/pkg/expression/contextimpl/sessionctx.go b/pkg/expression/contextimpl/sessionctx.go index 474ba51b241ff..6b59a15226eaf 100644 --- a/pkg/expression/contextimpl/sessionctx.go +++ b/pkg/expression/contextimpl/sessionctx.go @@ -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. diff --git a/pkg/expression/contextimpl/sessionctx_test.go b/pkg/expression/contextimpl/sessionctx_test.go index 0bb4c3bd979e0..371f9ad774ca8 100644 --- a/pkg/expression/contextimpl/sessionctx_test.go +++ b/pkg/expression/contextimpl/sessionctx_test.go @@ -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()) diff --git a/pkg/expression/expression_test.go b/pkg/expression/expression_test.go index 906131d0a15a0..361d9e88e86fa 100644 --- a/pkg/expression/expression_test.go +++ b/pkg/expression/expression_test.go @@ -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()) @@ -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) { diff --git a/pkg/expression/util.go b/pkg/expression/util.go index d966b90b89166..08751ec3a57a6 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -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) diff --git a/pkg/extension/function_test.go b/pkg/extension/function_test.go index d6d40d91e3abd..9635144912ab2 100644 --- a/pkg/extension/function_test.go +++ b/pkg/extension/function_test.go @@ -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()) } } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index c30e812a32cb6..23d4c096d98c4 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -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", @@ -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", diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index 6b281d4a9f533..80d1af2d99cab 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -16,6 +16,7 @@ package core import ( "fmt" + "strings" "testing" "github.com/pingcap/tidb/pkg/domain" @@ -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" ) @@ -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) { diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 808e1bfe4e80b..cb4b1bcb2e7a8 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -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 { diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index a5b4473e96917..3386bfba8ee3c 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -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" @@ -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 { @@ -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 @@ -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 } @@ -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 } @@ -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) @@ -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 } @@ -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 @@ -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 ( diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 5abcf89e8d389..ebb56f91a8e35 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -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 } diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index 6d185d02649ca..a2f9b1895fc84 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -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{}) diff --git a/pkg/session/session.go b/pkg/session/session.go index c761b0399f9bf..b5285509a6904 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -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, } }) diff --git a/pkg/sessionctx/stmtctx/stmtctx.go b/pkg/sessionctx/stmtctx/stmtctx.go index 8426f70f5f82c..64050ca66a6e0 100644 --- a/pkg/sessionctx/stmtctx/stmtctx.go +++ b/pkg/sessionctx/stmtctx/stmtctx.go @@ -158,7 +158,7 @@ type StatementContext struct { // copying this object will make the copied TypeCtx field to refer a wrong `AppendWarnings` func. _ nocopy.NoCopy - _ constructor.Constructor `ctor:"NewStmtCtx,NewStmtCtxWithTimeZone,Reset"` + _ constructor.Constructor `ctor:"NewStmtCtxWithTimeZone,Reset"` ctxID uint64 @@ -209,13 +209,13 @@ type StatementContext struct { InSetSessionStatesStmt bool InPreparedPlanBuilding bool InShowWarning bool - UseCache bool - ForcePlanCache bool // force the optimizer to use plan cache even if there is risky optimization, see #49736. - CacheType PlanCacheType - PlanCacheUnqualified string // why this query is not supported by the plan cache - BatchCheck bool - IgnoreExplainIDSuffix bool - MultiSchemaInfo *model.MultiSchemaInfo + + contextutil.PlanCacheTracker + contextutil.RangeFallbackHandler + + BatchCheck bool + IgnoreExplainIDSuffix bool + MultiSchemaInfo *model.MultiSchemaInfo // If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction // or is affected by the tidb_read_staleness session variable, then the statement will be makred as isStaleness // in stmtCtx @@ -411,9 +411,6 @@ type StatementContext struct { // ColRefFromPlan mark the column ref used by assignment in update statement. ColRefFromUpdatePlan []int64 - // RangeFallback indicates that building complete ranges exceeds the memory limit so it falls back to less accurate ranges such as full range. - RangeFallback bool - // IsExplainAnalyzeDML is true if the statement is "explain analyze DML executors", before responding the explain // results to the client, the transaction should be committed first. See issue #37373 for more details. IsExplainAnalyzeDML bool @@ -468,6 +465,8 @@ func NewStmtCtxWithTimeZone(tz *time.Location) *StatementContext { } sc.typeCtx = types.NewContext(types.DefaultStmtFlags, tz, sc) sc.errCtx = newErrCtx(sc.typeCtx, defaultErrLevels, sc) + sc.PlanCacheTracker = contextutil.NewPlanCacheTracker(sc) + sc.RangeFallbackHandler = contextutil.NewRangeFallbackHandler(&sc.PlanCacheTracker, sc) return sc } @@ -478,6 +477,8 @@ func (sc *StatementContext) Reset() { } sc.typeCtx = types.NewContext(types.DefaultStmtFlags, time.UTC, sc) sc.errCtx = newErrCtx(sc.typeCtx, defaultErrLevels, sc) + sc.PlanCacheTracker = contextutil.NewPlanCacheTracker(sc) + sc.RangeFallbackHandler = contextutil.NewRangeFallbackHandler(&sc.PlanCacheTracker, sc) } // CtxID returns the context id of the statement @@ -759,19 +760,6 @@ const ( SessionNonPrepared ) -// SetSkipPlanCache sets to skip the plan cache and records the reason. -func (sc *StatementContext) SetSkipPlanCache(reason error) { - if !sc.UseCache { - return // avoid unnecessary warnings - } - - if sc.ForcePlanCache { - sc.AppendWarning(errors.NewNoStackErrorf("force plan-cache: may use risky cached plan: %s", reason.Error())) - return - } - sc.setSkipPlanCache(reason) -} - // SetHintWarning sets the hint warning and records the reason. func (sc *StatementContext) SetHintWarning(reason string) { sc.AppendWarning(plannererrors.ErrInternal.FastGen(reason)) @@ -782,30 +770,6 @@ func (sc *StatementContext) SetHintWarningFromError(reason error) { sc.AppendWarning(reason) } -// ForceSetSkipPlanCache sets to skip the plan cache and records the reason. -func (sc *StatementContext) ForceSetSkipPlanCache(reason error) { - if sc.CacheType == DefaultNoCache { - return - } - sc.setSkipPlanCache(reason) -} - -func (sc *StatementContext) setSkipPlanCache(reason error) { - sc.UseCache = false - sc.PlanCacheUnqualified = reason.Error() - switch sc.CacheType { - case DefaultNoCache: - sc.AppendWarning(errors.NewNoStackError("unknown cache type")) - case SessionPrepared: - sc.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: %s", reason.Error())) - case SessionNonPrepared: - if sc.InExplainStmt && sc.ExplainFormat == "plan_cache" { - // use "plan_cache" rather than types.ExplainFormatPlanCache to avoid import cycle - sc.AppendWarning(errors.NewNoStackErrorf("skip non-prepared plan-cache: %s", reason.Error())) - } - } -} - // TableEntry presents table in db. type TableEntry struct { DB string @@ -1163,19 +1127,6 @@ func (sc *StatementContext) GetLockWaitStartTime() time.Time { return time.Unix(0, startTime) } -// RecordRangeFallback records range fallback. -func (sc *StatementContext) RecordRangeFallback(rangeMaxSize int64) { - // If range fallback happens, it means ether the query is unreasonable(for example, several long IN lists) or tidb_opt_range_max_size is too small - // and the generated plan is probably suboptimal. In that case we don't put it into plan cache. - if sc.UseCache { - sc.SetSkipPlanCache(errors.NewNoStackError("in-list is too long")) - } - if !sc.RangeFallback { - sc.AppendWarning(errors.NewNoStackErrorf("Memory capacity of %v bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", rangeMaxSize)) - sc.RangeFallback = true - } -} - // UseDynamicPartitionPrune indicates whether dynamic partition is used during the query func (sc *StatementContext) UseDynamicPartitionPrune() bool { return sc.UseDynamicPruneMode diff --git a/pkg/util/context/BUILD.bazel b/pkg/util/context/BUILD.bazel index 2a362524e55d7..ae8d1a9c24eb8 100644 --- a/pkg/util/context/BUILD.bazel +++ b/pkg/util/context/BUILD.bazel @@ -4,8 +4,10 @@ go_library( name = "context", srcs = [ "context.go", + "plancache.go", "warn.go", ], importpath = "github.com/pingcap/tidb/pkg/util/context", visibility = ["//visibility:public"], + deps = ["@com_github_pingcap_errors//:errors"], ) diff --git a/pkg/util/context/plancache.go b/pkg/util/context/plancache.go new file mode 100644 index 0000000000000..bb8db9be71c3d --- /dev/null +++ b/pkg/util/context/plancache.go @@ -0,0 +1,175 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package context + +import ( + "sync" + + "github.com/pingcap/errors" +) + +// PlanCacheType is the flag of plan cache +type PlanCacheType int + +const ( + // DefaultNoCache no cache + DefaultNoCache PlanCacheType = iota + // SessionPrepared session prepared plan cache + SessionPrepared + // SessionNonPrepared session non-prepared plan cache + SessionNonPrepared +) + +// PlanCacheTracker PlanCacheTrackerf `PlanCacheTracker`. `PlanCacheTracker` is thread-safe. +type PlanCacheTracker struct { + mu sync.Mutex + + useCache bool + cacheType PlanCacheType + planCacheUnqualified string + forcePlanCache bool // force the optimizer to use plan cache even if there is risky optimization, see #49736. + alwaysWarnSkipCache bool + + warnHandler WarnHandler +} + +// ForceSetSkipPlanCache forces to skip plan cache. +func (h *PlanCacheTracker) ForceSetSkipPlanCache(err error) { + h.mu.Lock() + defer h.mu.Unlock() + + if h.cacheType == DefaultNoCache { + return + } + h.setSkipPlanCache(err) +} + +// SetSkipPlanCache sets to skip plan cache. +func (h *PlanCacheTracker) SetSkipPlanCache(err error) { + h.mu.Lock() + defer h.mu.Unlock() + + if !h.useCache { + return + } + + if h.forcePlanCache { + h.warnHandler.AppendWarning(errors.NewNoStackErrorf("force plan-cache: may use risky cached plan: %s", err.Error())) + return + } + + h.setSkipPlanCache(err) +} + +func (h *PlanCacheTracker) setSkipPlanCache(reason error) { + h.useCache = false + h.planCacheUnqualified = reason.Error() + + switch h.cacheType { + case DefaultNoCache: + h.warnHandler.AppendWarning(errors.NewNoStackError("unknown cache type")) + case SessionPrepared: + h.warnHandler.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: %s", reason.Error())) + case SessionNonPrepared: + if h.alwaysWarnSkipCache { + // use "plan_cache" rather than types.ExplainFormatPlanCache to avoid import cycle + h.warnHandler.AppendWarning(errors.NewNoStackErrorf("skip non-prepared plan-cache: %s", reason.Error())) + } + } +} + +// SetAlwaysWarnSkipCache sets whether to always warn when skip plan cache. By default, for `SessionNonPrepared`, we don't warn +// when skip plan cache. But in some cases, we want to warn even for `SessionNonPrepared`. +func (h *PlanCacheTracker) SetAlwaysWarnSkipCache(alwaysWarnSkipCache bool) { + h.mu.Lock() + defer h.mu.Unlock() + + h.alwaysWarnSkipCache = alwaysWarnSkipCache +} + +// SetCacheType sets the cache type. +func (h *PlanCacheTracker) SetCacheType(cacheType PlanCacheType) { + h.mu.Lock() + defer h.mu.Unlock() + + h.cacheType = cacheType +} + +// SetForcePlanCache sets whether to force the optimizer to use plan cache even if there is risky optimization. +func (h *PlanCacheTracker) SetForcePlanCache(forcePlanCache bool) { + h.mu.Lock() + defer h.mu.Unlock() + + h.forcePlanCache = forcePlanCache +} + +// EnablePlanCache sets to use plan cache. +func (h *PlanCacheTracker) EnablePlanCache() { + h.mu.Lock() + defer h.mu.Unlock() + + h.useCache = true +} + +// UseCache returns whether to use plan cache. +func (h *PlanCacheTracker) UseCache() bool { + h.mu.Lock() + defer h.mu.Unlock() + + return h.useCache +} + +// PlanCacheUnqualified returns the reason of why the plan cache is unqualified +func (h *PlanCacheTracker) PlanCacheUnqualified() string { + h.mu.Lock() + defer h.mu.Unlock() + + return h.planCacheUnqualified +} + +// NewPlanCacheTracker creates a new PlanCacheTracker. +func NewPlanCacheTracker(warnHandler WarnHandler) PlanCacheTracker { + return PlanCacheTracker{ + warnHandler: warnHandler, + } +} + +// RangeFallbackHandler is used to handle range fallback. +// If there are too many ranges, it'll fallback and add a warning. +// RangeFallbackHandler is thread-safe. +type RangeFallbackHandler struct { + planCacheTracker *PlanCacheTracker + + warnHandler WarnHandler + reportRangeFallbackWarning sync.Once +} + +// RecordRangeFallback records the range fallback event. +func (h *RangeFallbackHandler) RecordRangeFallback(rangeMaxSize int64) { + // If range fallback happens, it means ether the query is unreasonable(for example, several long IN lists) or tidb_opt_range_max_size is too small + // and the generated plan is probably suboptimal. In that case we don't put it into plan cache. + h.planCacheTracker.SetSkipPlanCache(errors.NewNoStackError("in-list is too long")) + h.reportRangeFallbackWarning.Do(func() { + h.warnHandler.AppendWarning(errors.NewNoStackErrorf("Memory capacity of %v bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", rangeMaxSize)) + }) +} + +// NewRangeFallbackHandler creates a new RangeFallbackHandler. +func NewRangeFallbackHandler(planCacheTracker *PlanCacheTracker, warnHandler WarnHandler) RangeFallbackHandler { + return RangeFallbackHandler{ + planCacheTracker: planCacheTracker, + warnHandler: warnHandler, + } +} diff --git a/pkg/util/mock/context.go b/pkg/util/mock/context.go index 6bb0d35432ca8..4e1266e0c8448 100644 --- a/pkg/util/mock/context.go +++ b/pkg/util/mock/context.go @@ -281,8 +281,8 @@ func (c *Context) GetRangerCtx() *rangerctx.RangerContext { OptPrefixIndexSingleScan: c.GetSessionVars().OptPrefixIndexSingleScan, OptimizerFixControl: c.GetSessionVars().OptimizerFixControl, - // TODO: avoid using the whole `StmtCtx` here. - RangeFallbackHandler: c.GetSessionVars().StmtCtx, + PlanCacheTracker: &c.GetSessionVars().StmtCtx.PlanCacheTracker, + RangeFallbackHandler: &c.GetSessionVars().StmtCtx.RangeFallbackHandler, } } diff --git a/pkg/util/ranger/BUILD.bazel b/pkg/util/ranger/BUILD.bazel index 4a675f0bf4415..c2cd514b0a5df 100644 --- a/pkg/util/ranger/BUILD.bazel +++ b/pkg/util/ranger/BUILD.bazel @@ -61,6 +61,7 @@ go_test( "//pkg/testkit/testsetup", "//pkg/types", "//pkg/util/collate", + "//pkg/util/context", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], diff --git a/pkg/util/ranger/context/context.go b/pkg/util/ranger/context/context.go index 696f3b111bcb7..626e924981e7c 100644 --- a/pkg/util/ranger/context/context.go +++ b/pkg/util/ranger/context/context.go @@ -21,20 +21,13 @@ import ( contextutil "github.com/pingcap/tidb/pkg/util/context" ) -// RangeFallbackHandler is used to handle range fallback. -// If there are too many ranges, it'll fallback and add a warning. -type RangeFallbackHandler interface { - RecordRangeFallback(rangeMaxSize int64) - SetSkipPlanCache(err error) - contextutil.WarnHandler -} - // RangerContext is the context used to build range. type RangerContext struct { TypeCtx types.Context ErrCtx errctx.Context ExprCtx exprctx.BuildContext - RangeFallbackHandler + *contextutil.RangeFallbackHandler + *contextutil.PlanCacheTracker OptimizerFixControl map[uint64]string UseCache bool InPreparedPlanBuilding bool diff --git a/pkg/util/ranger/ranger_test.go b/pkg/util/ranger/ranger_test.go index af570c7e47f21..c599fc80d52f8 100644 --- a/pkg/util/ranger/ranger_test.go +++ b/pkg/util/ranger/ranger_test.go @@ -17,6 +17,7 @@ package ranger_test import ( "context" "fmt" + "strings" "testing" "github.com/pingcap/tidb/pkg/config" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/collate" + contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/ranger" "github.com/stretchr/testify/require" ) @@ -1846,8 +1848,15 @@ func checkDetachRangeResult(t *testing.T, res *ranger.DetachRangeResult, expecte } func checkRangeFallbackAndReset(t *testing.T, sctx sessionctx.Context, expectedRangeFallback bool) { - require.Equal(t, expectedRangeFallback, sctx.GetSessionVars().StmtCtx.RangeFallback) - sctx.GetSessionVars().StmtCtx.RangeFallback = false + stmtCtx := sctx.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 TestRangeFallbackForDetachCondAndBuildRangeForIndex(t *testing.T) {