Skip to content

Commit

Permalink
expression: use a bool arg for Expression.ConstItem (pingcap#49495)
Browse files Browse the repository at this point in the history
  • Loading branch information
lcwangchao authored Dec 15, 2023
1 parent 4238341 commit 074af10
Show file tree
Hide file tree
Showing 22 changed files with 42 additions and 55 deletions.
2 changes: 1 addition & 1 deletion pkg/expression/aggregation/base_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ func (a *baseFuncDesc) typeInfer4ApproxPercentile(ctx sessionctx.Context) error
return errors.New("APPROX_PERCENTILE should take 2 arguments")
}

if !a.Args[1].ConstItem(ctx.GetSessionVars().StmtCtx) {
if !a.Args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
return errors.New("APPROX_PERCENTILE should take a constant expression as percentage argument")
}
percent, isNull, err := a.Args[1].EvalInt(ctx, chunk.Row{})
Expand Down
13 changes: 1 addition & 12 deletions pkg/expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,11 +52,8 @@ type baseBuiltinFunc struct {
pbCode tipb.ScalarFuncSig
ctor collate.Collator

childrenVectorized bool
childrenReversed bool

childrenVectorized bool
childrenVectorizedOnce *sync.Once
childrenReversedOnce *sync.Once

collationInfo
}
Expand Down Expand Up @@ -120,7 +117,6 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, funcName string, args []Expressi
bf := baseBuiltinFunc{
bufAllocator: newLocalColumnPool(),
childrenVectorizedOnce: new(sync.Once),
childrenReversedOnce: new(sync.Once),

args: args,
tp: tp,
Expand Down Expand Up @@ -207,7 +203,6 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Ex
bf = baseBuiltinFunc{
bufAllocator: newLocalColumnPool(),
childrenVectorizedOnce: new(sync.Once),
childrenReversedOnce: new(sync.Once),

args: args,
tp: fieldType,
Expand Down Expand Up @@ -269,7 +264,6 @@ func newBaseBuiltinFuncWithFieldTypes(ctx sessionctx.Context, funcName string, a
bf = baseBuiltinFunc{
bufAllocator: newLocalColumnPool(),
childrenVectorizedOnce: new(sync.Once),
childrenReversedOnce: new(sync.Once),

args: args,
tp: fieldType,
Expand All @@ -289,7 +283,6 @@ func newBaseBuiltinFuncWithFieldType(tp *types.FieldType, args []Expression) (ba
bf := baseBuiltinFunc{
bufAllocator: newLocalColumnPool(),
childrenVectorizedOnce: new(sync.Once),
childrenReversedOnce: new(sync.Once),

args: args,
tp: tp,
Expand Down Expand Up @@ -414,7 +407,6 @@ func (b *baseBuiltinFunc) cloneFrom(from *baseBuiltinFunc) {
b.pbCode = from.pbCode
b.bufAllocator = newLocalColumnPool()
b.childrenVectorizedOnce = new(sync.Once)
b.childrenReversedOnce = new(sync.Once)
b.ctor = from.ctor
}

Expand Down Expand Up @@ -1020,9 +1012,6 @@ func (b *baseBuiltinFunc) MemoryUsage() (sum int64) {
if b.childrenVectorizedOnce != nil {
sum += onceSize
}
if b.childrenReversedOnce != nil {
sum += onceSize
}
for _, e := range b.args {
sum += e.MemoryUsage()
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -2233,7 +2233,7 @@ func WrapWithCastAsDecimal(ctx sessionctx.Context, expr Expression) Expression {
tp.AddFlag(expr.GetType().GetFlag() & (mysql.UnsignedFlag | mysql.NotNullFlag))
castExpr := BuildCastFunction(ctx, expr, tp)
// For const item, we can use find-grained precision and scale by the result.
if castExpr.ConstItem(ctx.GetSessionVars().StmtCtx) {
if castExpr.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isnull, err := castExpr.EvalDecimal(ctx, chunk.Row{})
if !isnull && err == nil {
precision, frac := val.PrecisionAndFrac()
Expand Down
8 changes: 4 additions & 4 deletions pkg/expression/builtin_encryption_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func (b *builtinAesDecryptSig) vecEvalString(ctx EvalContext, input *chunk.Chunk
}

isWarning := !b.ivRequired && len(b.args) == 3
isConstKey := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx)
isConstKey := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)

var key []byte
if isConstKey {
Expand Down Expand Up @@ -158,7 +158,7 @@ func (b *builtinAesEncryptIVSig) vecEvalString(ctx EvalContext, input *chunk.Chu
return errors.Errorf("unsupported block encryption mode - %v", b.modeName)
}

isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx)
isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)
var key []byte
if isConst {
key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize)
Expand Down Expand Up @@ -331,7 +331,7 @@ func (b *builtinAesDecryptIVSig) vecEvalString(ctx EvalContext, input *chunk.Chu
return errors.Errorf("unsupported block encryption mode - %v", b.modeName)
}

isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx)
isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)
var key []byte
if isConst {
key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize)
Expand Down Expand Up @@ -672,7 +672,7 @@ func (b *builtinAesEncryptSig) vecEvalString(ctx EvalContext, input *chunk.Chunk
}

isWarning := !b.ivRequired && len(b.args) == 3
isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx)
isConst := b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)
var key []byte
if isConst {
key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize)
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/builtin_func_param.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ func buildStringParam(ctx EvalContext, bf *baseBuiltinFunc, idx int, input *chun
}

// Check if this is a const value
if bf.args[idx].ConstItem(ctx.GetSessionVars().StmtCtx) {
if bf.args[idx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
// Initialize the const
var isConstNull bool
pa.defaultStrVal, isConstNull, err = bf.args[idx].EvalString(ctx, chunk.Row{})
Expand Down Expand Up @@ -111,7 +111,7 @@ func buildIntParam(ctx EvalContext, bf *baseBuiltinFunc, idx int, input *chunk.C
}

// Check if this is a const value
if bf.args[idx].ConstItem(ctx.GetSessionVars().StmtCtx) {
if bf.args[idx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
// Initialize the const
var isConstNull bool
pa.defaultIntVal, isConstNull, err = bf.args[idx].EvalInt(ctx, chunk.Row{})
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_ilike.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func (b *builtinIlikeSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool,
memorization := func() {
if b.pattern == nil {
b.pattern = collate.ConvertAndGetBinCollation(b.collation).Pattern()
if b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx) && b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) && b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
b.pattern.Compile(patternStr, byte(escape))
b.isMemorizedPattern = true
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_ilike_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ func (b *builtinIlikeSig) getEscape(ctx EvalContext, input *chunk.Chunk, result
rowNum := input.NumRows()
escape := int64('\\')

if !b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx) {
if !b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
return escape, true, errors.Errorf("escape should be const")
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_like.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func (b *builtinLikeSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool, e
memorization := func() {
if b.pattern == nil {
b.pattern = b.collator().Pattern()
if b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx) && b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[1].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) && b.args[2].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
b.pattern.Compile(patternStr, byte(escape))
b.isMemorizedPattern = true
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ func (b *builtinInIntSig) buildHashMapForConstArgs(ctx sessionctx.Context) error
b.nonConstArgsIdx = make([]int, 0)
b.hashSet = make(map[int64]bool, len(b.args)-1)
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalInt(ctx, chunk.Row{})
if err != nil {
return err
Expand Down Expand Up @@ -290,7 +290,7 @@ func (b *builtinInStringSig) buildHashMapForConstArgs(ctx sessionctx.Context) er
b.hashSet = set.NewStringSet()
collator := collate.GetCollator(b.collation)
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalString(ctx, chunk.Row{})
if err != nil {
return err
Expand Down Expand Up @@ -363,7 +363,7 @@ func (b *builtinInRealSig) buildHashMapForConstArgs(ctx sessionctx.Context) erro
b.nonConstArgsIdx = make([]int, 0)
b.hashSet = set.NewFloat64Set()
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalReal(ctx, chunk.Row{})
if err != nil {
return err
Expand Down Expand Up @@ -434,7 +434,7 @@ func (b *builtinInDecimalSig) buildHashMapForConstArgs(ctx sessionctx.Context) e
b.nonConstArgsIdx = make([]int, 0)
b.hashSet = set.NewStringSet()
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalDecimal(ctx, chunk.Row{})
if err != nil {
return err
Expand Down Expand Up @@ -514,7 +514,7 @@ func (b *builtinInTimeSig) buildHashMapForConstArgs(ctx sessionctx.Context) erro
b.nonConstArgsIdx = make([]int, 0)
b.hashSet = make(map[types.CoreTime]struct{}, len(b.args)-1)
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalTime(ctx, chunk.Row{})
if err != nil {
return err
Expand Down Expand Up @@ -585,7 +585,7 @@ func (b *builtinInDurationSig) buildHashMapForConstArgs(ctx sessionctx.Context)
b.nonConstArgsIdx = make([]int, 0)
b.hashSet = make(map[time.Duration]struct{}, len(b.args)-1)
for i := 1; i < len(b.args); i++ {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx) {
if b.args[i].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
val, isNull, err := b.args[i].EvalDuration(ctx, chunk.Row{})
if err != nil {
return err
Expand Down
6 changes: 3 additions & 3 deletions pkg/expression/builtin_regexp.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,7 @@ func (re *regexpBaseFuncSig) genRegexp(pat string, matchType string) (*regexp.Re
//
// return true: need, false: needless
func (re *regexpBaseFuncSig) canMemorize(ctx EvalContext, matchTypeIdx int) bool {
return re.args[patternIdx].ConstItem(ctx.GetSessionVars().StmtCtx) && (len(re.args) <= matchTypeIdx || re.args[matchTypeIdx].ConstItem(ctx.GetSessionVars().StmtCtx))
return re.args[patternIdx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) && (len(re.args) <= matchTypeIdx || re.args[matchTypeIdx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
}

func (re *regexpBaseFuncSig) initMemoizedRegexp(params []*funcParam, matchTypeIdx int) error {
Expand Down Expand Up @@ -1251,7 +1251,7 @@ func getInstructions(repl []byte) ([]Instruction, error) {
}

func (re *builtinRegexpReplaceFuncSig) canInstructionsMemorized(ctx EvalContext) bool {
return re.args[replacementIdx].ConstItem(ctx.GetSessionVars().StmtCtx)
return re.args[replacementIdx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)
}

func (re *builtinRegexpReplaceFuncSig) evalString(ctx EvalContext, row chunk.Row) (string, bool, error) {
Expand Down Expand Up @@ -1473,7 +1473,7 @@ func (re *builtinRegexpReplaceFuncSig) vecEvalString(ctx EvalContext, input *chu
buffers := getBuffers(params)

instructions := make([]Instruction, 0)
isReplConst := re.baseBuiltinFunc.args[2].ConstItem(ctx.GetSessionVars().StmtCtx)
isReplConst := re.baseBuiltinFunc.args[2].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache)
if isReplConst {
// repl is const
instructions, err = getInstructions([]byte(params[2].getStringVal(0)))
Expand Down
5 changes: 2 additions & 3 deletions pkg/expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/pingcap/tidb/pkg/parser/charset"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/codec"
Expand Down Expand Up @@ -173,7 +172,7 @@ func (col *CorrelatedColumn) IsCorrelated() bool {
}

// ConstItem implements Expression interface.
func (col *CorrelatedColumn) ConstItem(_ *stmtctx.StatementContext) bool {
func (col *CorrelatedColumn) ConstItem(_ bool) bool {
return false
}

Expand Down Expand Up @@ -516,7 +515,7 @@ func (col *Column) IsCorrelated() bool {
}

// ConstItem implements Expression interface.
func (col *Column) ConstItem(_ *stmtctx.StatementContext) bool {
func (col *Column) ConstItem(_ bool) bool {
return false
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func TestColumn(t *testing.T) {
require.True(t, corCol.EqualColumn(corCol))
require.False(t, corCol.EqualColumn(invalidCorCol))
require.True(t, corCol.IsCorrelated())
require.False(t, corCol.ConstItem(nil))
require.False(t, corCol.ConstItem(false))
require.True(t, col.EqualColumn(corCol.Decorrelate(schema)))
require.True(t, invalidCorCol.EqualColumn(invalidCorCol.Decorrelate(schema)))

Expand Down
5 changes: 2 additions & 3 deletions pkg/expression/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (

"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/codec"
Expand Down Expand Up @@ -423,8 +422,8 @@ func (c *Constant) IsCorrelated() bool {
}

// ConstItem implements Expression interface.
func (c *Constant) ConstItem(sc *stmtctx.StatementContext) bool {
return !sc.UseCache || (c.DeferredExpr == nil && c.ParamMarker == nil)
func (c *Constant) ConstItem(acrossCtx bool) bool {
return !acrossCtx || (c.DeferredExpr == nil && c.ParamMarker == nil)
}

// Decorrelate implements Expression interface.
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (s *basePropConstSolver) insertCol(col *Column) {
// tryToUpdateEQList tries to update the eqList. When the eqList has store this column with a different constant, like
// a = 1 and a = 2, we set the second return value to false.
func (s *basePropConstSolver) tryToUpdateEQList(col *Column, con *Constant) (bool, bool) {
if con.ConstItem(s.ctx.GetSessionVars().StmtCtx) && con.Value.IsNull() {
if con.ConstItem(s.ctx.GetSessionVars().StmtCtx.UseCache) && con.Value.IsNull() {
return false, true
}
id := s.getColID(col)
Expand Down
4 changes: 3 additions & 1 deletion pkg/expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,14 +140,16 @@ type Expression interface {
IsCorrelated() bool

// ConstItem checks if this expression is constant item, regardless of query evaluation state.
// If the argument `acrossCtxs` is true,
// it will check if this expression returns a constant value even across multiple contexts.
// An expression is constant item if it:
// refers no tables.
// refers no correlated column.
// refers no subqueries that refers any tables.
// refers no non-deterministic functions.
// refers no statement parameters.
// refers no param markers when prepare plan cache is enabled.
ConstItem(sc *stmtctx.StatementContext) bool
ConstItem(acrossCtx bool) bool

// Decorrelate try to decorrelate the expression by schema.
Decorrelate(schema *Schema) Expression
Expand Down
10 changes: 5 additions & 5 deletions pkg/expression/expression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ func TestConstant(t *testing.T) {
ctx := createContext(t)
sc := stmtctx.NewStmtCtxWithTimeZone(time.Local)
require.False(t, NewZero().IsCorrelated())
require.True(t, NewZero().ConstItem(sc))
require.True(t, NewZero().ConstItem(sc.UseCache))
require.True(t, NewZero().Decorrelate(nil).Equal(ctx, NewZero()))
require.Equal(t, []byte{0x0, 0x8, 0x0}, NewZero().HashCode())
require.False(t, NewZero().Equal(ctx, NewOne()))
Expand Down Expand Up @@ -136,13 +136,13 @@ func TestIsBinaryLiteral(t *testing.T) {
func TestConstItem(t *testing.T) {
ctx := createContext(t)
sf := newFunctionWithMockCtx(ast.Rand)
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx))
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
sf = newFunctionWithMockCtx(ast.UUID)
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx))
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
sf = newFunctionWithMockCtx(ast.GetParam, NewOne())
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx))
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
sf = newFunctionWithMockCtx(ast.Abs, NewOne())
require.True(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx))
require.True(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
}

func TestVectorizable(t *testing.T) {
Expand Down
5 changes: 2 additions & 3 deletions pkg/expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
Expand Down Expand Up @@ -350,13 +349,13 @@ func (sf *ScalarFunction) IsCorrelated() bool {
}

// ConstItem implements Expression interface.
func (sf *ScalarFunction) ConstItem(sc *stmtctx.StatementContext) bool {
func (sf *ScalarFunction) ConstItem(acrossCtx bool) bool {
// Note: some unfoldable functions are deterministic, we use unFoldableFunctions here for simplification.
if _, ok := unFoldableFunctions[sf.FuncName.L]; ok {
return false
}
for _, arg := range sf.GetArgs() {
if !arg.ConstItem(sc) {
if !arg.ConstItem(acrossCtx) {
return false
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/scalar_function_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func TestScalarFunction(t *testing.T) {
require.NoError(t, err)
require.EqualValues(t, []byte{0x22, 0x6c, 0x74, 0x28, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x23, 0x31, 0x2c, 0x20, 0x31, 0x29, 0x22}, res)
require.False(t, sf.IsCorrelated())
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx))
require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx.UseCache))
require.True(t, sf.Decorrelate(nil).Equal(ctx, sf))
require.EqualValues(t, []byte{0x3, 0x4, 0x6c, 0x74, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x5, 0xbf, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}, sf.HashCode())

Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -571,7 +571,7 @@ func (m *MockExpr) GetType() *types.FieldType { return m
func (m *MockExpr) Clone() Expression { return nil }
func (m *MockExpr) Equal(ctx EvalContext, e Expression) bool { return false }
func (m *MockExpr) IsCorrelated() bool { return false }
func (m *MockExpr) ConstItem(_ *stmtctx.StatementContext) bool { return false }
func (m *MockExpr) ConstItem(_ bool) bool { return false }
func (m *MockExpr) Decorrelate(schema *Schema) Expression { return m }
func (m *MockExpr) ResolveIndices(schema *Schema) (Expression, error) { return m, nil }
func (m *MockExpr) resolveIndices(schema *Schema) error { return nil }
Expand Down
Loading

0 comments on commit 074af10

Please sign in to comment.