Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/executor/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -1205,6 +1205,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
vars.DurationWaitTS = 0
vars.CurrInsertBatchExtraCols = nil
vars.CurrInsertValues = chunk.Row{}
ctx.GetPlanCtx().Reset()

return
}
Expand Down
33 changes: 31 additions & 2 deletions pkg/expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -1068,11 +1069,39 @@ 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 convertReadonlyVarToConst(ctx, sf), nil
}

// 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
}
varName := arg0.Value.GetString()
isReadonly := ctx.IsReadonlyUserVar(varName)
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(varName)
if ok && d.Kind() == types.KindBinaryLiteral {
v.SetBinaryLiteral(v.GetBytes())
}
return &Constant{
Value: v,
RetType: getVar.RetType,
DeferredExpr: nil,
}
}

type getVarFunctionClass struct {
Expand Down
2 changes: 2 additions & 0 deletions pkg/expression/exprctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,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.
Expand Down
6 changes: 6 additions & 0 deletions pkg/expression/exprstatic/exprctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,12 @@ func (ctx *ExprContext) GetStaticConvertibleEvalContext() exprctx.StaticConverti
return ctx.evalCtx
}

// 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
}

// MakeExprContextStatic converts the `exprctx.StaticConvertibleExprContext` to `ExprContext`.
func MakeExprContextStatic(ctx exprctx.StaticConvertibleExprContext) *ExprContext {
staticEvalContext := MakeEvalContextStatic(ctx.GetStaticConvertibleEvalContext())
Expand Down
7 changes: 7 additions & 0 deletions pkg/expression/sessionexpr/sessionctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,13 @@ 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]
return ok
}

// IntoStatic turns the ExprContext into a ExprContext.
func (ctx *ExprContext) IntoStatic() *exprstatic.ExprContext {
return exprstatic.MakeExprContextStatic(ctx)
Expand Down
6 changes: 5 additions & 1 deletion pkg/planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,11 @@ 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{})
var (
val types.Datum
err error
)
val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})
Comment on lines +67 to +71
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
var (
val types.Datum
err error
)
val, err = usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})
val, err := usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{})

if err != nil {
return err
}
Expand Down
20 changes: 20 additions & 0 deletions pkg/planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,8 @@ 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),
varsMutable: make(map[string]struct{}),
varsReadonly: make(map[string]struct{}),
resolveCtx: node.GetResolveContext(),
}
for _, optFn := range preprocessOpt {
Expand All @@ -144,6 +146,10 @@ 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.GetPlanCtx().SetReadonlyUserVarMap(v.varsReadonly)
if len(v.varsReadonly) > 0 {
sctx.GetSessionVars().StmtCtx.SetSkipPlanCache("read-only variables are used")
}
return errors.Trace(v.err)
}

Expand Down Expand Up @@ -236,6 +242,9 @@ type preprocessor struct {

staleReadProcessor staleread.Processor

varsMutable map[string]struct{}
varsReadonly map[string]struct{}

// values that may be returned
*PreprocessorReturn
err error
Expand Down Expand Up @@ -419,6 +428,17 @@ 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.varsMutable[node.Name] = struct{}{}
delete(p.varsReadonly, node.Name)
} else if p.stmtTp == TypeSelect {
// Only check the variable in select statement.
_, ok := p.varsMutable[node.Name]
if !ok {
p.varsReadonly[node.Name] = struct{}{}
}
}
default:
p.flag &= ^parentIsJoin
}
Expand Down
15 changes: 15 additions & 0 deletions pkg/planner/planctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,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.
Expand All @@ -82,6 +88,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(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
Expand Down
16 changes: 16 additions & 0 deletions pkg/planner/plannersession/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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
}
10 changes: 5 additions & 5 deletions tests/integrationtest/r/index_merge.result
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down