From 6ba612504c08da8bb10d364590bdf1b6b5ce4fd0 Mon Sep 17 00:00:00 2001 From: Yiding Date: Fri, 5 Jul 2024 05:12:50 +0800 Subject: [PATCH 1/8] planner, expr: eval readonly user var during plan phase --- pkg/executor/executor.go | 1 + pkg/expression/builtin_other.go | 5 ++-- pkg/expression/constant_fold.go | 26 +++++++++++++++++++++ pkg/expression/context/context.go | 4 ++++ pkg/expression/contextsession/sessionctx.go | 25 ++++++++++++++++++-- pkg/expression/contextstatic/evalctx.go | 13 +++++++++++ pkg/expression/function_traits.go | 2 +- pkg/expression/util.go | 4 ++++ pkg/planner/core/plan_cache.go | 26 ++++++++++++++------- pkg/planner/core/preprocess.go | 19 +++++++++++++++ tests/integrationtest/r/index_merge.result | 10 ++++---- 11 files changed, 116 insertions(+), 19 deletions(-) diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 3b06e7c518f1c..8169f4aa51afa 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -2085,6 +2085,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.DurationWaitTS = 0 vars.CurrInsertBatchExtraCols = nil vars.CurrInsertValues = chunk.Row{} + ctx.GetExprCtx().GetEvalCtx().ResetReadonlyVarMap() return } diff --git a/pkg/expression/builtin_other.go b/pkg/expression/builtin_other.go index 14e0b7600ef19..569bdd8676e3d 100644 --- a/pkg/expression/builtin_other.go +++ b/pkg/expression/builtin_other.go @@ -950,11 +950,12 @@ func BuildGetVarFunction(ctx BuildContext, expr Expression, retType *types.Field if builtinRetTp := f.getRetTp(); builtinRetTp.GetType() != mysql.TypeUnspecified || retType.GetType() == mysql.TypeUnspecified { retType = builtinRetTp } - return &ScalarFunction{ + sf := &ScalarFunction{ FuncName: model.NewCIStr(ast.GetVar), RetType: retType, Function: f, - }, nil + } + return FoldConstant(ctx, sf), nil } type getVarFunctionClass struct { diff --git a/pkg/expression/constant_fold.go b/pkg/expression/constant_fold.go index c33e4c6a8b820..941de87c0dbb2 100644 --- a/pkg/expression/constant_fold.go +++ b/pkg/expression/constant_fold.go @@ -15,6 +15,7 @@ package expression import ( + "github.com/pingcap/tidb/pkg/expression/contextopt" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -33,6 +34,7 @@ func init() { ast.Ifnull: ifNullFoldHandler, ast.Case: caseWhenHandler, ast.IsNull: isNullHandler, + ast.GetVar: getVarHandler, } } @@ -159,6 +161,30 @@ func caseWhenHandler(ctx BuildContext, expr *ScalarFunction) (Expression, bool) return expr, isDeferredConst } +func getVarHandler(ctx BuildContext, expr *ScalarFunction) (Expression, bool) { + arg0 := expr.GetArgs()[0] + c, ok := arg0.(*Constant) + if !ok { + return expr, false + } + if !ctx.GetEvalCtx().IsReadonlyVar(c.Value.GetString()) { + return expr, false + } + v, err := expr.Eval(ctx.GetEvalCtx(), chunk.Row{}) + if err != nil { + return expr, false + } + sessVars, err := contextopt.SessionVarsPropReader{}.GetSessionVars(ctx.GetEvalCtx()) + if err != nil { + return expr, false + } + d, ok := sessVars.GetUserVarVal(c.Value.GetString()) + if ok && d.Kind() == types.KindBinaryLiteral { + v.SetBinaryLiteral(v.GetBytes()) + } + return &Constant{Value: v, RetType: expr.RetType, DeferredExpr: expr}, false +} + func foldConstant(ctx BuildContext, expr Expression) (Expression, bool) { switch x := expr.(type) { case *ScalarFunction: diff --git a/pkg/expression/context/context.go b/pkg/expression/context/context.go index 36c623c213680..e6e0c411d3f86 100644 --- a/pkg/expression/context/context.go +++ b/pkg/expression/context/context.go @@ -83,6 +83,10 @@ type EvalContext interface { GetOptionalPropSet() OptionalEvalPropKeySet // GetOptionalPropProvider gets the optional property provider by key GetOptionalPropProvider(OptionalEvalPropKey) (OptionalEvalPropProvider, bool) + + ResetReadonlyVarMap() + SetReadonlyVarMap(map[string]struct{}) + IsReadonlyVar(name string) bool } // BuildContext is used to build an expression diff --git a/pkg/expression/contextsession/sessionctx.go b/pkg/expression/contextsession/sessionctx.go index eb864cb7f8d0a..8a87ecf03dd77 100644 --- a/pkg/expression/contextsession/sessionctx.go +++ b/pkg/expression/contextsession/sessionctx.go @@ -138,8 +138,9 @@ func (ctx *SessionExprContext) ConnectionID() uint64 { // SessionEvalContext implements the `expression.EvalContext` interface to provide evaluation context in session. type SessionEvalContext struct { - sctx sessionctx.Context - props contextopt.OptionalEvalPropProviders + sctx sessionctx.Context + props contextopt.OptionalEvalPropProviders + readonlyVars map[string]struct{} } // NewSessionEvalContext creates a new SessionEvalContext. @@ -156,9 +157,29 @@ func NewSessionEvalContext(sctx sessionctx.Context) *SessionEvalContext { ctx.setOptionalProp(contextopt.DDLOwnerInfoProvider(sctx.IsDDLOwner)) // When EvalContext is created from a session, it should contain all the optional properties. intest.Assert(ctx.props.PropKeySet().IsFull()) + ctx.readonlyVars = make(map[string]struct{}) return ctx } +// ResetReadonlyVarMap resets the readonly vars map. +func (ctx *SessionEvalContext) ResetReadonlyVarMap() { + ctx.readonlyVars = nil +} + +// SetReadonlyVarMap sets the readonly vars map. +func (ctx *SessionEvalContext) SetReadonlyVarMap(vars map[string]struct{}) { + ctx.readonlyVars = vars +} + +// IsReadonlyVar checks whether the variable is readonly. +func (ctx *SessionEvalContext) IsReadonlyVar(name string) bool { + if ctx.readonlyVars == nil { + return false + } + _, ok := ctx.readonlyVars[name] + return ok +} + func (ctx *SessionEvalContext) setOptionalProp(prop exprctx.OptionalEvalPropProvider) { intest.AssertFunc(func() bool { return !ctx.props.Contains(prop.Desc().Key()) diff --git a/pkg/expression/contextstatic/evalctx.go b/pkg/expression/contextstatic/evalctx.go index 0ef002780c8c6..a2906725e70fe 100644 --- a/pkg/expression/contextstatic/evalctx.go +++ b/pkg/expression/contextstatic/evalctx.go @@ -218,6 +218,19 @@ type StaticEvalContext struct { staticEvalCtxState } +// ResetReadonlyVarMap implements context.EvalContext. +func (ctx *StaticEvalContext) ResetReadonlyVarMap() { +} + +// SetReadonlyVarMap implements context.EvalContext. +func (ctx *StaticEvalContext) SetReadonlyVarMap(map[string]struct{}) { +} + +// IsReadonlyVar implements context.EvalContext. +func (ctx *StaticEvalContext) IsReadonlyVar(name string) bool { + return false +} + // NewStaticEvalContext creates a new `StaticEvalContext` with the given options. func NewStaticEvalContext(opt ...StaticEvalCtxOption) *StaticEvalContext { ctx := &StaticEvalContext{ diff --git a/pkg/expression/function_traits.go b/pkg/expression/function_traits.go index bdd3e9acbbddf..22111275d6f9d 100644 --- a/pkg/expression/function_traits.go +++ b/pkg/expression/function_traits.go @@ -55,7 +55,6 @@ var unFoldableFunctions = map[string]struct{}{ ast.RowFunc: {}, ast.Values: {}, ast.SetVar: {}, - ast.GetVar: {}, ast.GetParam: {}, ast.Benchmark: {}, ast.DayName: {}, @@ -208,6 +207,7 @@ var AllowedPartition4UnaryOpMap = map[opcode.Op]struct{}{ // inequalFunctions stores functions which cannot be propagated from column equal condition. var inequalFunctions = map[string]struct{}{ ast.IsNull: {}, + ast.GetVar: {}, } // mutableEffectsFunctions stores functions which are mutable or have side effects, specifically, diff --git a/pkg/expression/util.go b/pkg/expression/util.go index 8f874258ecd81..3481a875069f7 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -1307,6 +1307,10 @@ func IsRuntimeConstExpr(expr Expression) bool { if _, ok := unFoldableFunctions[x.FuncName.L]; ok { return false } + // If the GetVar is not folded, it's not readonly and can't be treated as a constant. + if x.FuncName.L == ast.GetVar { + return false + } for _, arg := range x.GetArgs() { if !IsRuntimeConstExpr(arg) { return false diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 5acdd475457d1..8dc53de663559 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -58,16 +58,24 @@ func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers [ vars := sctx.GetSessionVars() vars.PlanCacheParams.Reset() for i, usingParam := range params { - val, err := usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) - if err != nil { - return err - } - if isGetVarBinaryLiteral(sctx, usingParam) { - binVal, convErr := val.ToBytes() - if convErr != nil { - return convErr + var ( + val types.Datum + err error + ) + if c, ok := usingParam.(*expression.Constant); ok { + val = c.Value + } else { + val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) + if err != nil { + return err + } + if isGetVarBinaryLiteral(sctx, usingParam) { + binVal, convErr := val.ToBytes() + if convErr != nil { + return convErr + } + val.SetBinaryLiteral(binVal) } - val.SetBinaryLiteral(binVal) } if markers != nil { param := markers[i].(*driver.ParamMarkerExpr) diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index 4e32a40099bab..1bc91ec416492 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -127,6 +127,8 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node ast.Node, pre tableAliasInJoin: make([]map[string]any, 0), preprocessWith: &preprocessWith{cteCanUsed: make([]string, 0), cteBeforeOffset: make([]int, 0)}, staleReadProcessor: staleread.NewStaleReadProcessor(ctx, sctx), + varsChanged: make(map[string]struct{}), + varsReadonly: make(map[string]struct{}), } for _, optFn := range preprocessOpt { optFn(&v) @@ -138,6 +140,10 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node ast.Node, pre node.Accept(&v) // InfoSchema must be non-nil after preprocessing v.ensureInfoSchema() + sctx.GetExprCtx().GetEvalCtx().SetReadonlyVarMap(v.varsReadonly) + if len(v.varsReadonly) > 0 { + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache("read-only variables are used") + } return errors.Trace(v.err) } @@ -229,6 +235,9 @@ type preprocessor struct { staleReadProcessor staleread.Processor + varsChanged map[string]struct{} + varsReadonly map[string]struct{} + // values that may be returned *PreprocessorReturn err error @@ -410,6 +419,16 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { } case *ast.AnalyzeTableStmt: p.flag |= inAnalyze + case *ast.VariableExpr: + if node.Value != nil { + p.varsChanged[node.Name] = struct{}{} + delete(p.varsReadonly, node.Name) + } else { + _, ok := p.varsChanged[node.Name] + if !ok { + p.varsReadonly[node.Name] = struct{}{} + } + } default: p.flag &= ^parentIsJoin } diff --git a/tests/integrationtest/r/index_merge.result b/tests/integrationtest/r/index_merge.result index 516f5558e1942..f03bf777c10ca 100644 --- a/tests/integrationtest/r/index_merge.result +++ b/tests/integrationtest/r/index_merge.result @@ -781,11 +781,11 @@ set @a = 1; explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and length(substring(sqrt(c3), @a, 1)) = char_length(if(c1, c2, c3)) order by 1; id estRows task access object operator info Sort_5 4433.77 root index_merge.t1.c1 -└─Selection_8 4433.77 root eq(length(substring(cast(sqrt(cast(index_merge.t1.c3, double BINARY)), var_string(5)), getvar("a"), 1)), char_length(cast(if(index_merge.t1.c1, index_merge.t1.c2, index_merge.t1.c3), var_string(20)))) - └─IndexMerge_12 5542.21 root type: union - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─TableRowIDScan_11(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexMerge_12 4433.77 root type: union + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─Selection_11(Probe) 4433.77 cop[tikv] eq(length(substring(cast(sqrt(cast(index_merge.t1.c3, double BINARY)), var_string(5)), 1, 1)), char_length(cast(if(index_merge.t1.c1, index_merge.t1.c2, index_merge.t1.c3), var_string(20)))) + └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and length(substring(sqrt(c3), @a, 1)) = char_length(if(c1, c2, c3)) order by 1; c1 c2 c3 c4 c5 1 1 1 1 1 From ae03a7a6bce2a09563a86a6a81458fe61f49f649 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 23 Oct 2024 01:09:45 +0800 Subject: [PATCH 2/8] refine the impl --- pkg/executor/executor.go | 2 +- pkg/expression/builtin_other.go | 29 +++++++++++++++++++++++- pkg/expression/constant_fold.go | 26 --------------------- pkg/expression/exprctx/context.go | 6 ++--- pkg/expression/exprstatic/evalctx.go | 16 ------------- pkg/expression/exprstatic/exprctx.go | 5 ++++ pkg/expression/function_traits.go | 1 - pkg/expression/sessionexpr/sessionctx.go | 28 +++++++---------------- pkg/expression/util.go | 4 ---- pkg/lightning/backend/kv/context.go | 5 ++++ pkg/planner/core/plan_cache.go | 1 + pkg/planner/core/preprocess.go | 2 +- pkg/planner/planctx/context.go | 15 ++++++++++++ pkg/planner/plannersession/context.go | 16 +++++++++++++ 14 files changed, 82 insertions(+), 74 deletions(-) diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 2e93e080d60ef..355b948259867 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -2144,7 +2144,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.DurationWaitTS = 0 vars.CurrInsertBatchExtraCols = nil vars.CurrInsertValues = chunk.Row{} - ctx.GetExprCtx().GetEvalCtx().ResetReadonlyVarMap() + ctx.GetPlanCtx().Reset() return } diff --git a/pkg/expression/builtin_other.go b/pkg/expression/builtin_other.go index 51a30e546d3ce..581f54de5835c 100644 --- a/pkg/expression/builtin_other.go +++ b/pkg/expression/builtin_other.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/set" "github.com/pingcap/tidb/pkg/util/stringutil" "github.com/pingcap/tipb/go-tipb" @@ -994,7 +995,33 @@ func BuildGetVarFunction(ctx BuildContext, expr Expression, retType *types.Field RetType: retType, Function: f, } - return FoldConstant(ctx, sf), nil + return optimizeReadonlyVar(ctx, sf), nil +} + +// optimizeReadonlyVar tries to convert the readonly user variables to constants. +func optimizeReadonlyVar(ctx BuildContext, getVar *ScalarFunction) Expression { + arg0, isConst := getVar.GetArgs()[0].(*Constant) + if !isConst || arg0.DeferredExpr != nil { + return getVar + } + isReadonly := ctx.IsReadonlyUserVar(arg0.Value.GetString()) + if !isReadonly { + return getVar + } + v, err := getVar.Eval(ctx.GetEvalCtx(), chunk.Row{}) + if err != nil { + intest.Assert(false, "readonly user variable should not meet error when executing.") + return getVar + } + d, ok := ctx.GetEvalCtx().GetUserVarsReader().GetUserVarVal(arg0.Value.GetString()) + if ok && d.Kind() == types.KindBinaryLiteral { + v.SetBinaryLiteral(v.GetBytes()) + } + return &Constant{ + Value: v, + RetType: getVar.RetType, + DeferredExpr: nil, + } } type getVarFunctionClass struct { diff --git a/pkg/expression/constant_fold.go b/pkg/expression/constant_fold.go index ea8fe7d4f7102..c33e4c6a8b820 100644 --- a/pkg/expression/constant_fold.go +++ b/pkg/expression/constant_fold.go @@ -15,7 +15,6 @@ package expression import ( - "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -34,7 +33,6 @@ func init() { ast.Ifnull: ifNullFoldHandler, ast.Case: caseWhenHandler, ast.IsNull: isNullHandler, - ast.GetVar: getVarHandler, } } @@ -161,30 +159,6 @@ func caseWhenHandler(ctx BuildContext, expr *ScalarFunction) (Expression, bool) return expr, isDeferredConst } -func getVarHandler(ctx BuildContext, expr *ScalarFunction) (Expression, bool) { - arg0 := expr.GetArgs()[0] - c, ok := arg0.(*Constant) - if !ok { - return expr, false - } - if !ctx.GetEvalCtx().IsReadonlyVar(c.Value.GetString()) { - return expr, false - } - v, err := expr.Eval(ctx.GetEvalCtx(), chunk.Row{}) - if err != nil { - return expr, false - } - sessVars, err := expropt.SessionVarsPropReader{}.GetSessionVars(ctx.GetEvalCtx()) - if err != nil { - return expr, false - } - d, ok := sessVars.GetUserVarVal(c.Value.GetString()) - if ok && d.Kind() == types.KindBinaryLiteral { - v.SetBinaryLiteral(v.GetBytes()) - } - return &Constant{Value: v, RetType: expr.RetType, DeferredExpr: expr}, false -} - func foldConstant(ctx BuildContext, expr Expression) (Expression, bool) { switch x := expr.(type) { case *ScalarFunction: diff --git a/pkg/expression/exprctx/context.go b/pkg/expression/exprctx/context.go index 3ea8d65dd0eec..e752af29c6a74 100644 --- a/pkg/expression/exprctx/context.go +++ b/pkg/expression/exprctx/context.go @@ -92,10 +92,6 @@ type EvalContext interface { GetOptionalPropSet() OptionalEvalPropKeySet // GetOptionalPropProvider gets the optional property provider by key GetOptionalPropProvider(OptionalEvalPropKey) (OptionalEvalPropProvider, bool) - - ResetReadonlyVarMap() - SetReadonlyVarMap(map[string]struct{}) - IsReadonlyVar(name string) bool } // BuildContext is used to build an expression @@ -132,6 +128,8 @@ type BuildContext interface { // ConnectionID indicates the connection ID of the current session. // If the context is not in a session, it should return 0. ConnectionID() uint64 + // IsReadonlyUserVar checks whether the user variable is readonly. + IsReadonlyUserVar(name string) bool } // ExprContext contains full context for expression building and evaluating. diff --git a/pkg/expression/exprstatic/evalctx.go b/pkg/expression/exprstatic/evalctx.go index 278a6489aff6d..3a196d14992d3 100644 --- a/pkg/expression/exprstatic/evalctx.go +++ b/pkg/expression/exprstatic/evalctx.go @@ -391,22 +391,6 @@ func (ctx *EvalContext) GetParamValue(idx int) (types.Datum, error) { return ctx.paramList[idx], nil } -// ResetReadonlyVarMap resets the readonly variable map. -func (ctx *EvalContext) ResetReadonlyVarMap() { - ctx.readonlyVarMap = make(map[string]struct{}) -} - -// SetReadonlyVarMap sets the readonly variable map. -func (ctx *EvalContext) SetReadonlyVarMap(m map[string]struct{}) { - ctx.readonlyVarMap = m -} - -// IsReadonlyVar checks whether the variable is readonly. -func (ctx *EvalContext) IsReadonlyVar(name string) bool { - _, ok := ctx.readonlyVarMap[name] - return ok -} - var _ exprctx.StaticConvertibleEvalContext = &EvalContext{} // AllParamValues implements context.StaticConvertibleEvalContext. diff --git a/pkg/expression/exprstatic/exprctx.go b/pkg/expression/exprstatic/exprctx.go index 347625af5538c..0d488135e6ef2 100644 --- a/pkg/expression/exprstatic/exprctx.go +++ b/pkg/expression/exprstatic/exprctx.go @@ -299,6 +299,11 @@ func (ctx *ExprContext) GetStaticConvertibleEvalContext() exprctx.StaticConverti return ctx.evalCtx } +// IsReadonlyUserVar implements the `BuildContext.IsReadonlyUserVar` and should always return false. +func (ctx *ExprContext) IsReadonlyUserVar(name string) bool { + return false +} + // MakeExprContextStatic converts the `exprctx.StaticConvertibleExprContext` to `ExprContext`. func MakeExprContextStatic(ctx exprctx.StaticConvertibleExprContext) *ExprContext { staticEvalContext := MakeEvalContextStatic(ctx.GetStaticConvertibleEvalContext()) diff --git a/pkg/expression/function_traits.go b/pkg/expression/function_traits.go index 22111275d6f9d..021451ea38e6f 100644 --- a/pkg/expression/function_traits.go +++ b/pkg/expression/function_traits.go @@ -207,7 +207,6 @@ var AllowedPartition4UnaryOpMap = map[opcode.Op]struct{}{ // inequalFunctions stores functions which cannot be propagated from column equal condition. var inequalFunctions = map[string]struct{}{ ast.IsNull: {}, - ast.GetVar: {}, } // mutableEffectsFunctions stores functions which are mutable or have side effects, specifically, diff --git a/pkg/expression/sessionexpr/sessionctx.go b/pkg/expression/sessionexpr/sessionctx.go index c967324a34605..283c82bf947dc 100644 --- a/pkg/expression/sessionexpr/sessionctx.go +++ b/pkg/expression/sessionexpr/sessionctx.go @@ -137,6 +137,12 @@ func (ctx *ExprContext) ConnectionID() uint64 { return ctx.sctx.GetSessionVars().ConnectionID } +func (ctx *ExprContext) IsReadonlyUserVar(name string) bool { + m := ctx.sctx.GetPlanCtx().GetReadonlyUserVarMap() + _, ok := m[name] + return ok +} + // IntoStatic turns the ExprContext into a ExprContext. func (ctx *ExprContext) IntoStatic() *exprstatic.ExprContext { return exprstatic.MakeExprContextStatic(ctx) @@ -144,9 +150,8 @@ func (ctx *ExprContext) IntoStatic() *exprstatic.ExprContext { // EvalContext implements the `expression.EvalContext` interface to provide evaluation context in session. type EvalContext struct { - sctx sessionctx.Context - props expropt.OptionalEvalPropProviders - readonlyVars map[string]struct{} + sctx sessionctx.Context + props expropt.OptionalEvalPropProviders } // NewEvalContext creates a new EvalContext. @@ -164,26 +169,9 @@ func NewEvalContext(sctx sessionctx.Context) *EvalContext { ctx.setOptionalProp(expropt.PrivilegeCheckerProvider(func() expropt.PrivilegeChecker { return ctx })) // When EvalContext is created from a session, it should contain all the optional properties. intest.Assert(ctx.props.PropKeySet().IsFull()) - ctx.readonlyVars = make(map[string]struct{}) return ctx } -// ResetReadonlyVarMap resets the readonly vars map. -func (ctx *EvalContext) ResetReadonlyVarMap() { - ctx.readonlyVars = make(map[string]struct{}) -} - -// SetReadonlyVarMap sets the readonly vars map. -func (ctx *EvalContext) SetReadonlyVarMap(vars map[string]struct{}) { - ctx.readonlyVars = vars -} - -// IsReadonlyVar checks whether the variable is readonly. -func (ctx *EvalContext) IsReadonlyVar(name string) bool { - _, ok := ctx.readonlyVars[name] - return ok -} - func (ctx *EvalContext) setOptionalProp(prop exprctx.OptionalEvalPropProvider) { intest.AssertFunc(func() bool { return !ctx.props.Contains(prop.Desc().Key()) diff --git a/pkg/expression/util.go b/pkg/expression/util.go index bf2a9f4415fb2..b74322c3f8c88 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -1312,10 +1312,6 @@ func IsRuntimeConstExpr(expr Expression) bool { if _, ok := unFoldableFunctions[x.FuncName.L]; ok { return false } - // If the GetVar is not folded, it's not readonly and can't be treated as a constant. - if x.FuncName.L == ast.GetVar { - return false - } for _, arg := range x.GetArgs() { if !IsRuntimeConstExpr(arg) { return false diff --git a/pkg/lightning/backend/kv/context.go b/pkg/lightning/backend/kv/context.go index 8b5bd42deee55..1666d91e7f110 100644 --- a/pkg/lightning/backend/kv/context.go +++ b/pkg/lightning/backend/kv/context.go @@ -110,6 +110,11 @@ func (ctx *litExprContext) unsetUserVar(varName string) { ctx.userVars.UnsetUserVar(varName) } +// GetUserVar implements the `exprctx.BuildContext` interface. +func (ctx *litExprContext) IsReadonlyUserVar(name string) bool { + return false +} + var _ table.MutateContext = &litTableMutateContext{} // litTableMutateContext implements the `table.MutateContext` interface for lightning import. diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 22193161ce69f..59aec72af6d68 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -65,6 +65,7 @@ func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers [ ) if c, ok := usingParam.(*expression.Constant); ok { val = c.Value + intest.Assert(c.DeferredExpr == nil, "folded readonly user variable should not contain deferred expression") } else { val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) if err != nil { diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index bc52225d13724..fd3aa9ef5405b 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -146,7 +146,7 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node *resolve.Node node.Node.Accept(&v) // InfoSchema must be non-nil after preprocessing v.ensureInfoSchema() - sctx.GetExprCtx().GetEvalCtx().SetReadonlyVarMap(v.varsReadonly) + sctx.GetPlanCtx().SetReadonlyUserVarMap(v.varsReadonly) if len(v.varsReadonly) > 0 { sctx.GetSessionVars().StmtCtx.SetSkipPlanCache("read-only variables are used") } diff --git a/pkg/planner/planctx/context.go b/pkg/planner/planctx/context.go index 7a61c2a7e08e9..2af1823efd1d1 100644 --- a/pkg/planner/planctx/context.go +++ b/pkg/planner/planctx/context.go @@ -70,6 +70,12 @@ type PlanContext interface { GetRangerCtx() *rangerctx.RangerContext // GetBuildPBCtx returns the context used in `ToPB` method. GetBuildPBCtx() *BuildPBContext + // SetReadonlyUserVarMap sets the readonly user variable map. + SetReadonlyUserVarMap(readonlyUserVars map[string]struct{}) + // GetReadonlyUserVarMap gets the readonly user variable map. + GetReadonlyUserVarMap() map[string]struct{} + // Reset reset the local context. + Reset() } // EmptyPlanContextExtended is used to provide some empty implementations for PlanContext. @@ -80,6 +86,15 @@ type EmptyPlanContextExtended struct{} // AdviseTxnWarmup advises the txn to warm up. func (EmptyPlanContextExtended) AdviseTxnWarmup() error { return nil } +// SetReadonlyUserVarMap sets the readonly user variable map. +func (EmptyPlanContextExtended) SetReadonlyUserVarMap(readonlyUserVars map[string]struct{}) {} + +// GetReadonlyUserVarMap gets the readonly user variable map. +func (EmptyPlanContextExtended) GetReadonlyUserVarMap() map[string]struct{} { return nil } + +// Reset implements the PlanContext interface. +func (EmptyPlanContextExtended) Reset() {} + // BuildPBContext is used to build the `*tipb.Executor` according to the plan. type BuildPBContext struct { ExprCtx exprctx.BuildContext diff --git a/pkg/planner/plannersession/context.go b/pkg/planner/plannersession/context.go index 35cdd8017be76..c8c2764d0929e 100644 --- a/pkg/planner/plannersession/context.go +++ b/pkg/planner/plannersession/context.go @@ -31,6 +31,7 @@ var _ planctx.PlanContext = struct { type PlanCtxExtended struct { sctx sessionctx.Context nullRejectCheckExprCtx *exprctx.NullRejectCheckExprContext + readonlyUserVars map[string]struct{} } // NewPlanCtxExtended creates a new PlanCtxExtended. @@ -54,3 +55,18 @@ func (ctx *PlanCtxExtended) GetNullRejectCheckExprCtx() exprctx.ExprContext { func (ctx *PlanCtxExtended) AdviseTxnWarmup() error { return sessiontxn.GetTxnManager(ctx.sctx).AdviseWarmup() } + +// SetReadonlyUserVarMap sets the readonly user variable map. +func (ctx *PlanCtxExtended) SetReadonlyUserVarMap(readonlyUserVars map[string]struct{}) { + ctx.readonlyUserVars = readonlyUserVars +} + +// GetReadonlyUserVarMap gets the readonly user variable map. +func (ctx *PlanCtxExtended) GetReadonlyUserVarMap() map[string]struct{} { + return ctx.readonlyUserVars +} + +// Reset resets the local +func (ctx *PlanCtxExtended) Reset() { + ctx.readonlyUserVars = nil +} From d92be4f76397e019fab4b77eba09aa2587be7e65 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 23 Oct 2024 02:40:30 +0800 Subject: [PATCH 3/8] limit the scope --- pkg/expression/sessionexpr/sessionctx.go | 1 + pkg/planner/core/preprocess.go | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/expression/sessionexpr/sessionctx.go b/pkg/expression/sessionexpr/sessionctx.go index 283c82bf947dc..0ad74841543b8 100644 --- a/pkg/expression/sessionexpr/sessionctx.go +++ b/pkg/expression/sessionexpr/sessionctx.go @@ -137,6 +137,7 @@ func (ctx *ExprContext) ConnectionID() uint64 { return ctx.sctx.GetSessionVars().ConnectionID } +// IsReadonlyUserVar checks whether the user variable is readonly. func (ctx *ExprContext) IsReadonlyUserVar(name string) bool { m := ctx.sctx.GetPlanCtx().GetReadonlyUserVarMap() _, ok := m[name] diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index fd3aa9ef5405b..969962bbb0ffc 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -432,7 +432,8 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if node.Value != nil { p.varsChanged[node.Name] = struct{}{} delete(p.varsReadonly, node.Name) - } else { + } else if p.stmtTp == TypeSelect { + // Only check the variable in select statement. _, ok := p.varsChanged[node.Name] if !ok { p.varsReadonly[node.Name] = struct{}{} From 97e1a351c3c4f5077f32a9cc381a5c85286c73a6 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 23 Oct 2024 03:15:05 +0800 Subject: [PATCH 4/8] revert unexpected change --- pkg/expression/function_traits.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/expression/function_traits.go b/pkg/expression/function_traits.go index 021451ea38e6f..bdd3e9acbbddf 100644 --- a/pkg/expression/function_traits.go +++ b/pkg/expression/function_traits.go @@ -55,6 +55,7 @@ var unFoldableFunctions = map[string]struct{}{ ast.RowFunc: {}, ast.Values: {}, ast.SetVar: {}, + ast.GetVar: {}, ast.GetParam: {}, ast.Benchmark: {}, ast.DayName: {}, From 4e79cfbb8f4508a639a3ba1ee7748a583c0f22d0 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 23 Oct 2024 20:46:21 +0800 Subject: [PATCH 5/8] address comments and add some comments --- pkg/expression/exprstatic/exprctx.go | 3 +++ pkg/lightning/backend/kv/context.go | 5 ----- pkg/planner/core/plan_cache.go | 1 + 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/pkg/expression/exprstatic/exprctx.go b/pkg/expression/exprstatic/exprctx.go index 0d488135e6ef2..81d693721ec4f 100644 --- a/pkg/expression/exprstatic/exprctx.go +++ b/pkg/expression/exprstatic/exprctx.go @@ -300,6 +300,9 @@ func (ctx *ExprContext) GetStaticConvertibleEvalContext() exprctx.StaticConverti } // IsReadonlyUserVar implements the `BuildContext.IsReadonlyUserVar` and should always return false. +// We return false because the static context is only used during execution phase. +// But we check whether the user variable is readonly in planning phase. +// So just return false for static context. func (ctx *ExprContext) IsReadonlyUserVar(name string) bool { return false } diff --git a/pkg/lightning/backend/kv/context.go b/pkg/lightning/backend/kv/context.go index 1af7ed404f34e..8b5bd42deee55 100644 --- a/pkg/lightning/backend/kv/context.go +++ b/pkg/lightning/backend/kv/context.go @@ -110,11 +110,6 @@ func (ctx *litExprContext) unsetUserVar(varName string) { ctx.userVars.UnsetUserVar(varName) } -// GetUserVar implements the `exprctx.BuildContext` interface. -func (*litExprContext) IsReadonlyUserVar(string) bool { - return false -} - var _ table.MutateContext = &litTableMutateContext{} // litTableMutateContext implements the `table.MutateContext` interface for lightning import. diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 59aec72af6d68..d2d7850daa052 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -64,6 +64,7 @@ func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers [ err error ) if c, ok := usingParam.(*expression.Constant); ok { + // If the variable is readonly, it's folded to a constant value. val = c.Value intest.Assert(c.DeferredExpr == nil, "folded readonly user variable should not contain deferred expression") } else { From 2698c575ff278f6811dbc705429bf96bea0cee8a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 23 Oct 2024 23:39:10 +0800 Subject: [PATCH 6/8] Update pkg/expression/exprstatic/exprctx.go MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 王超 --- pkg/expression/exprstatic/exprctx.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pkg/expression/exprstatic/exprctx.go b/pkg/expression/exprstatic/exprctx.go index 81d693721ec4f..4c3e2e3d7b043 100644 --- a/pkg/expression/exprstatic/exprctx.go +++ b/pkg/expression/exprstatic/exprctx.go @@ -299,10 +299,8 @@ func (ctx *ExprContext) GetStaticConvertibleEvalContext() exprctx.StaticConverti return ctx.evalCtx } -// IsReadonlyUserVar implements the `BuildContext.IsReadonlyUserVar` and should always return false. -// We return false because the static context is only used during execution phase. -// But we check whether the user variable is readonly in planning phase. -// So just return false for static context. +// IsReadonlyUserVar implements the `BuildContext.IsReadonlyUserVar`. +// This method always returns false for simplicity, ensuring the safest behavior across all scenarios. func (ctx *ExprContext) IsReadonlyUserVar(name string) bool { return false } From 155a816ecdc8b2517d2142fde67c053790e010b3 Mon Sep 17 00:00:00 2001 From: Yiding Date: Tue, 5 Nov 2024 01:15:03 +0800 Subject: [PATCH 7/8] try to simplify some codes --- pkg/planner/core/plan_cache.go | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index d1bc80b833f1e..377108fdb2b00 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -64,22 +64,16 @@ func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers [ val types.Datum err error ) - if c, ok := usingParam.(*expression.Constant); ok { - // If the variable is readonly, it's folded to a constant value. - val = c.Value - intest.Assert(c.DeferredExpr == nil, "folded readonly user variable should not contain deferred expression") - } else { - val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) - if err != nil { - return err - } - if isGetVarBinaryLiteral(sctx, usingParam) { - binVal, convErr := val.ToBytes() - if convErr != nil { - return convErr - } - val.SetBinaryLiteral(binVal) + val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) + if err != nil { + return err + } + if isGetVarBinaryLiteral(sctx, usingParam) { + binVal, convErr := val.ToBytes() + if convErr != nil { + return convErr } + val.SetBinaryLiteral(binVal) } if markers != nil { param := markers[i].(*driver.ParamMarkerExpr) From 8aeca9d3f42d359480c9c0c3b05f8b613e87494f Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 25 Dec 2024 20:52:33 +0800 Subject: [PATCH 8/8] address comments --- pkg/expression/builtin_other.go | 11 ++++++----- pkg/planner/core/preprocess.go | 8 ++++---- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/pkg/expression/builtin_other.go b/pkg/expression/builtin_other.go index 14c9661287f13..bdc6afd3c9298 100644 --- a/pkg/expression/builtin_other.go +++ b/pkg/expression/builtin_other.go @@ -1074,16 +1074,17 @@ func BuildGetVarFunction(ctx BuildContext, expr Expression, retType *types.Field RetType: retType, Function: f, } - return optimizeReadonlyVar(ctx, sf), nil + return convertReadonlyVarToConst(ctx, sf), nil } -// optimizeReadonlyVar tries to convert the readonly user variables to constants. -func optimizeReadonlyVar(ctx BuildContext, getVar *ScalarFunction) Expression { +// convertReadonlyVarToConst tries to convert the readonly user variables to constants. +func convertReadonlyVarToConst(ctx BuildContext, getVar *ScalarFunction) Expression { arg0, isConst := getVar.GetArgs()[0].(*Constant) if !isConst || arg0.DeferredExpr != nil { return getVar } - isReadonly := ctx.IsReadonlyUserVar(arg0.Value.GetString()) + varName := arg0.Value.GetString() + isReadonly := ctx.IsReadonlyUserVar(varName) if !isReadonly { return getVar } @@ -1092,7 +1093,7 @@ func optimizeReadonlyVar(ctx BuildContext, getVar *ScalarFunction) Expression { intest.Assert(false, "readonly user variable should not meet error when executing.") return getVar } - d, ok := ctx.GetEvalCtx().GetUserVarsReader().GetUserVarVal(arg0.Value.GetString()) + d, ok := ctx.GetEvalCtx().GetUserVarsReader().GetUserVarVal(varName) if ok && d.Kind() == types.KindBinaryLiteral { v.SetBinaryLiteral(v.GetBytes()) } diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index 98b2def29e501..a1a870f7ead5b 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -132,7 +132,7 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node *resolve.Node tableAliasInJoin: make([]map[string]any, 0), preprocessWith: &preprocessWith{cteCanUsed: make([]string, 0), cteBeforeOffset: make([]int, 0)}, staleReadProcessor: staleread.NewStaleReadProcessor(ctx, sctx), - varsChanged: make(map[string]struct{}), + varsMutable: make(map[string]struct{}), varsReadonly: make(map[string]struct{}), resolveCtx: node.GetResolveContext(), } @@ -242,7 +242,7 @@ type preprocessor struct { staleReadProcessor staleread.Processor - varsChanged map[string]struct{} + varsMutable map[string]struct{} varsReadonly map[string]struct{} // values that may be returned @@ -430,11 +430,11 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { p.flag |= inAnalyze case *ast.VariableExpr: if node.Value != nil { - p.varsChanged[node.Name] = struct{}{} + p.varsMutable[node.Name] = struct{}{} delete(p.varsReadonly, node.Name) } else if p.stmtTp == TypeSelect { // Only check the variable in select statement. - _, ok := p.varsChanged[node.Name] + _, ok := p.varsMutable[node.Name] if !ok { p.varsReadonly[node.Name] = struct{}{} }