Skip to content

Commit 8c97a5a

Browse files
authored
planner, expr: eval readonly user var during plan phase (#54462) (#58540)
close #52742
1 parent 9029a53 commit 8c97a5a

File tree

10 files changed

+108
-8
lines changed

10 files changed

+108
-8
lines changed

pkg/executor/select.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1205,6 +1205,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
12051205
vars.DurationWaitTS = 0
12061206
vars.CurrInsertBatchExtraCols = nil
12071207
vars.CurrInsertValues = chunk.Row{}
1208+
ctx.GetPlanCtx().Reset()
12081209

12091210
return
12101211
}

pkg/expression/builtin_other.go

Lines changed: 31 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"github.com/pingcap/tidb/pkg/types"
2828
"github.com/pingcap/tidb/pkg/util/chunk"
2929
"github.com/pingcap/tidb/pkg/util/collate"
30+
"github.com/pingcap/tidb/pkg/util/intest"
3031
"github.com/pingcap/tidb/pkg/util/set"
3132
"github.com/pingcap/tidb/pkg/util/stringutil"
3233
"github.com/pingcap/tipb/go-tipb"
@@ -989,11 +990,39 @@ func BuildGetVarFunction(ctx BuildContext, expr Expression, retType *types.Field
989990
if builtinRetTp := f.getRetTp(); builtinRetTp.GetType() != mysql.TypeUnspecified || retType.GetType() == mysql.TypeUnspecified {
990991
retType = builtinRetTp
991992
}
992-
return &ScalarFunction{
993+
sf := &ScalarFunction{
993994
FuncName: model.NewCIStr(ast.GetVar),
994995
RetType: retType,
995996
Function: f,
996-
}, nil
997+
}
998+
return convertReadonlyVarToConst(ctx, sf), nil
999+
}
1000+
1001+
// convertReadonlyVarToConst tries to convert the readonly user variables to constants.
1002+
func convertReadonlyVarToConst(ctx BuildContext, getVar *ScalarFunction) Expression {
1003+
arg0, isConst := getVar.GetArgs()[0].(*Constant)
1004+
if !isConst || arg0.DeferredExpr != nil {
1005+
return getVar
1006+
}
1007+
varName := arg0.Value.GetString()
1008+
isReadonly := ctx.IsReadonlyUserVar(varName)
1009+
if !isReadonly {
1010+
return getVar
1011+
}
1012+
v, err := getVar.Eval(ctx.GetEvalCtx(), chunk.Row{})
1013+
if err != nil {
1014+
intest.Assert(false, "readonly user variable should not meet error when executing.")
1015+
return getVar
1016+
}
1017+
d, ok := ctx.GetEvalCtx().GetUserVarsReader().GetUserVarVal(varName)
1018+
if ok && d.Kind() == types.KindBinaryLiteral {
1019+
v.SetBinaryLiteral(v.GetBytes())
1020+
}
1021+
return &Constant{
1022+
Value: v,
1023+
RetType: getVar.RetType,
1024+
DeferredExpr: nil,
1025+
}
9971026
}
9981027

9991028
type getVarFunctionClass struct {

pkg/expression/exprctx/context.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -128,6 +128,8 @@ type BuildContext interface {
128128
// ConnectionID indicates the connection ID of the current session.
129129
// If the context is not in a session, it should return 0.
130130
ConnectionID() uint64
131+
// IsReadonlyUserVar checks whether the user variable is readonly.
132+
IsReadonlyUserVar(name string) bool
131133
}
132134

133135
// ExprContext contains full context for expression building and evaluating.

pkg/expression/exprstatic/exprctx.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -299,6 +299,12 @@ func (ctx *ExprContext) GetStaticConvertibleEvalContext() exprctx.StaticConverti
299299
return ctx.evalCtx
300300
}
301301

302+
// IsReadonlyUserVar implements the `BuildContext.IsReadonlyUserVar`.
303+
// This method always returns false for simplicity, ensuring the safest behavior across all scenarios.
304+
func (ctx *ExprContext) IsReadonlyUserVar(name string) bool {
305+
return false
306+
}
307+
302308
// MakeExprContextStatic converts the `exprctx.StaticConvertibleExprContext` to `ExprContext`.
303309
func MakeExprContextStatic(ctx exprctx.StaticConvertibleExprContext) *ExprContext {
304310
staticEvalContext := MakeEvalContextStatic(ctx.GetStaticConvertibleEvalContext())

pkg/expression/sessionexpr/sessionctx.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -137,6 +137,13 @@ func (ctx *ExprContext) ConnectionID() uint64 {
137137
return ctx.sctx.GetSessionVars().ConnectionID
138138
}
139139

140+
// IsReadonlyUserVar checks whether the user variable is readonly.
141+
func (ctx *ExprContext) IsReadonlyUserVar(name string) bool {
142+
m := ctx.sctx.GetPlanCtx().GetReadonlyUserVarMap()
143+
_, ok := m[name]
144+
return ok
145+
}
146+
140147
// IntoStatic turns the ExprContext into a ExprContext.
141148
func (ctx *ExprContext) IntoStatic() *exprstatic.ExprContext {
142149
return exprstatic.MakeExprContextStatic(ctx)

pkg/planner/core/plan_cache.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,11 @@ func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers [
6262
vars := sctx.GetSessionVars()
6363
vars.PlanCacheParams.Reset()
6464
for i, usingParam := range params {
65-
val, err := usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})
65+
var (
66+
val types.Datum
67+
err error
68+
)
69+
val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})
6670
if err != nil {
6771
return err
6872
}

pkg/planner/core/preprocess.go

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -132,6 +132,8 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node *resolve.Node
132132
tableAliasInJoin: make([]map[string]any, 0),
133133
preprocessWith: &preprocessWith{cteCanUsed: make([]string, 0), cteBeforeOffset: make([]int, 0)},
134134
staleReadProcessor: staleread.NewStaleReadProcessor(ctx, sctx),
135+
varsMutable: make(map[string]struct{}),
136+
varsReadonly: make(map[string]struct{}),
135137
resolveCtx: node.GetResolveContext(),
136138
}
137139
for _, optFn := range preprocessOpt {
@@ -144,6 +146,10 @@ func Preprocess(ctx context.Context, sctx sessionctx.Context, node *resolve.Node
144146
node.Node.Accept(&v)
145147
// InfoSchema must be non-nil after preprocessing
146148
v.ensureInfoSchema()
149+
sctx.GetPlanCtx().SetReadonlyUserVarMap(v.varsReadonly)
150+
if len(v.varsReadonly) > 0 {
151+
sctx.GetSessionVars().StmtCtx.SetSkipPlanCache("read-only variables are used")
152+
}
147153
return errors.Trace(v.err)
148154
}
149155

@@ -236,6 +242,9 @@ type preprocessor struct {
236242

237243
staleReadProcessor staleread.Processor
238244

245+
varsMutable map[string]struct{}
246+
varsReadonly map[string]struct{}
247+
239248
// values that may be returned
240249
*PreprocessorReturn
241250
err error
@@ -419,6 +428,17 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
419428
}
420429
case *ast.AnalyzeTableStmt:
421430
p.flag |= inAnalyze
431+
case *ast.VariableExpr:
432+
if node.Value != nil {
433+
p.varsMutable[node.Name] = struct{}{}
434+
delete(p.varsReadonly, node.Name)
435+
} else if p.stmtTp == TypeSelect {
436+
// Only check the variable in select statement.
437+
_, ok := p.varsMutable[node.Name]
438+
if !ok {
439+
p.varsReadonly[node.Name] = struct{}{}
440+
}
441+
}
422442
default:
423443
p.flag &= ^parentIsJoin
424444
}

pkg/planner/planctx/context.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,12 @@ type PlanContext interface {
7272
GetRangerCtx() *rangerctx.RangerContext
7373
// GetBuildPBCtx returns the context used in `ToPB` method.
7474
GetBuildPBCtx() *BuildPBContext
75+
// SetReadonlyUserVarMap sets the readonly user variable map.
76+
SetReadonlyUserVarMap(readonlyUserVars map[string]struct{})
77+
// GetReadonlyUserVarMap gets the readonly user variable map.
78+
GetReadonlyUserVarMap() map[string]struct{}
79+
// Reset reset the local context.
80+
Reset()
7581
}
7682

7783
// EmptyPlanContextExtended is used to provide some empty implementations for PlanContext.
@@ -82,6 +88,15 @@ type EmptyPlanContextExtended struct{}
8288
// AdviseTxnWarmup advises the txn to warm up.
8389
func (EmptyPlanContextExtended) AdviseTxnWarmup() error { return nil }
8490

91+
// SetReadonlyUserVarMap sets the readonly user variable map.
92+
func (EmptyPlanContextExtended) SetReadonlyUserVarMap(map[string]struct{}) {}
93+
94+
// GetReadonlyUserVarMap gets the readonly user variable map.
95+
func (EmptyPlanContextExtended) GetReadonlyUserVarMap() map[string]struct{} { return nil }
96+
97+
// Reset implements the PlanContext interface.
98+
func (EmptyPlanContextExtended) Reset() {}
99+
85100
// BuildPBContext is used to build the `*tipb.Executor` according to the plan.
86101
type BuildPBContext struct {
87102
ExprCtx exprctx.BuildContext

pkg/planner/plannersession/context.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ var _ planctx.PlanContext = struct {
3131
type PlanCtxExtended struct {
3232
sctx sessionctx.Context
3333
nullRejectCheckExprCtx *exprctx.NullRejectCheckExprContext
34+
readonlyUserVars map[string]struct{}
3435
}
3536

3637
// NewPlanCtxExtended creates a new PlanCtxExtended.
@@ -54,3 +55,18 @@ func (ctx *PlanCtxExtended) GetNullRejectCheckExprCtx() exprctx.ExprContext {
5455
func (ctx *PlanCtxExtended) AdviseTxnWarmup() error {
5556
return sessiontxn.GetTxnManager(ctx.sctx).AdviseWarmup()
5657
}
58+
59+
// SetReadonlyUserVarMap sets the readonly user variable map.
60+
func (ctx *PlanCtxExtended) SetReadonlyUserVarMap(readonlyUserVars map[string]struct{}) {
61+
ctx.readonlyUserVars = readonlyUserVars
62+
}
63+
64+
// GetReadonlyUserVarMap gets the readonly user variable map.
65+
func (ctx *PlanCtxExtended) GetReadonlyUserVarMap() map[string]struct{} {
66+
return ctx.readonlyUserVars
67+
}
68+
69+
// Reset resets the local
70+
func (ctx *PlanCtxExtended) Reset() {
71+
ctx.readonlyUserVars = nil
72+
}

tests/integrationtest/r/index_merge.result

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -781,11 +781,11 @@ set @a = 1;
781781
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;
782782
id estRows task access object operator info
783783
Sort_5 4433.77 root index_merge.t1.c1
784-
└─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))))
785-
└─IndexMerge_12 5542.21 root type: union
786-
├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo
787-
├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo
788-
└─TableRowIDScan_11(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo
784+
└─IndexMerge_12 4433.77 root type: union
785+
├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo
786+
├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo
787+
└─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))))
788+
└─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo
789789
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;
790790
c1 c2 c3 c4 c5
791791
1 1 1 1 1

0 commit comments

Comments
 (0)