Skip to content

Commit

Permalink
expression: refactor like and ilike cache logic (#49676)
Browse files Browse the repository at this point in the history
close #49674
  • Loading branch information
lcwangchao authored Dec 25, 2023
1 parent 946bcfc commit 4e9f006
Show file tree
Hide file tree
Showing 7 changed files with 137 additions and 70 deletions.
8 changes: 5 additions & 3 deletions pkg/expression/builtin_func_param.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,8 +78,9 @@ func buildStringParam(ctx EvalContext, bf *baseBuiltinFunc, idx int, input *chun
return &pa, false, nil
}

// Check if this is a const value
if bf.args[idx].ConstItem(ctx.GetSessionVars().StmtCtx.UseCache) {
// Check if this is a const value.
// funcParam will not be shared between evaluations, so we just need it to be const in one ctx.
if bf.args[idx].ConstItem(false) {
// Initialize the const
var isConstNull bool
pa.defaultStrVal, isConstNull, err = bf.args[idx].EvalString(ctx, chunk.Row{})
Expand Down Expand Up @@ -111,7 +112,8 @@ 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.UseCache) {
// funcParam will not be shared between evaluations, so we just need it to be const in one ctx.
if bf.args[idx].ConstItem(false) {
// Initialize the const
var isConstNull bool
pa.defaultIntVal, isConstNull, err = bf.args[idx].EvalInt(ctx, chunk.Row{})
Expand Down
42 changes: 18 additions & 24 deletions pkg/expression/builtin_ilike.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,11 @@
package expression

import (
"sync"

"github.com/pingcap/tidb/pkg/sessionctx"
"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/stringutil"
"github.com/pingcap/tipb/go-tipb"
)
Expand All @@ -47,25 +46,21 @@ func (c *ilikeFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi
return nil, err
}
bf.tp.SetFlen(1)
sig := &builtinIlikeSig{bf, nil, false, sync.Once{}}
sig := &builtinIlikeSig{baseBuiltinFunc: bf}
sig.setPbCode(tipb.ScalarFuncSig_IlikeSig)
return sig, nil
}

type builtinIlikeSig struct {
baseBuiltinFunc
// pattern and isMemorizedPattern is not serialized with builtinIlikeSig, treat them as a cache to accelerate
// pattern is not serialized with builtinIlikeSig, treat them as a cache to accelerate
// the evaluation of builtinIlikeSig.
pattern collate.WildcardPattern
isMemorizedPattern bool
once sync.Once
patternCache builtinFuncCache[collate.WildcardPattern]
}

func (b *builtinIlikeSig) Clone() builtinFunc {
newSig := &builtinIlikeSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
newSig.pattern = b.pattern
newSig.isMemorizedPattern = b.isMemorizedPattern
return newSig
}

Expand Down Expand Up @@ -99,22 +94,21 @@ func (b *builtinIlikeSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool,
valStr = string(valStrBytes)
patternStr = string(patternStrBytes)

memorization := func() {
if b.pattern == nil {
b.pattern = collate.ConvertAndGetBinCollation(b.collation).Pattern()
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
}
var pattern collate.WildcardPattern
if b.args[1].ConstItem(false) && b.args[2].ConstItem(false) {
pattern, err = b.patternCache.getOrInitCache(ctx, func() (collate.WildcardPattern, error) {
ret := collate.ConvertAndGetBinCollation(b.collation).Pattern()
ret.Compile(patternStr, byte(escape))
return ret, nil
})

intest.AssertNoError(err)
if err != nil {
return 0, true, err
}
}
// Only be executed once to achieve thread-safe
b.once.Do(memorization)
if !b.isMemorizedPattern {
// Must not use b.pattern to avoid data race
pattern := collate.ConvertAndGetBinCollation(b.collation).Pattern()
} else {
pattern = collate.ConvertAndGetBinCollation(b.collation).Pattern()
pattern.Compile(patternStr, byte(escape))
return boolToInt64(pattern.DoMatch(valStr)), false, nil
}
return boolToInt64(b.pattern.DoMatch(valStr)), false, nil
return boolToInt64(pattern.DoMatch(valStr)), false, nil
}
35 changes: 19 additions & 16 deletions pkg/expression/builtin_ilike_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/collate"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/stringutil"
)

Expand Down Expand Up @@ -52,21 +53,23 @@ func (b *builtinIlikeSig) canMemorize(param *funcParam) bool {
return param.getCol() == nil
}

func (b *builtinIlikeSig) tryToMemorize(param *funcParam, escape int64) {
func (b *builtinIlikeSig) tryToVecMemorize(ctx EvalContext, param *funcParam, escape int64) (collate.WildcardPattern, bool) {
if !b.canMemorize(param) {
return
return nil, false
}

memorization := func() {
if b.pattern == nil {
b.pattern = collate.ConvertAndGetBinCollation(b.collation).Pattern()
b.pattern.Compile(param.getStringVal(0), byte(escape))
b.isMemorizedPattern = true
}
pattern, err := b.patternCache.getOrInitCache(ctx, func() (collate.WildcardPattern, error) {
pattern := collate.ConvertAndGetBinCollation(b.collation).Pattern()
pattern.Compile(param.getStringVal(0), byte(escape))
return pattern, nil
})

intest.AssertNoError(err)
if err != nil {
return nil, false
}

// Only be executed once to achieve thread-safe
b.once.Do(memorization)
return pattern, true
}

func (b *builtinIlikeSig) getEscape(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) (int64, bool, error) {
Expand Down Expand Up @@ -149,15 +152,15 @@ func (b *builtinIlikeSig) constVec(pattern collate.WildcardPattern, expr string,
return nil
}

func (b *builtinIlikeSig) ilikeWithMemorization(exprParam *funcParam, rowNum int, result *chunk.Column) error {
func (b *builtinIlikeSig) ilikeWithMemorization(pattern collate.WildcardPattern, exprParam *funcParam, rowNum int, result *chunk.Column) error {
result.ResizeInt64(rowNum, false)
result.MergeNulls(exprParam.getCol())
i64s := result.Int64s()
for i := 0; i < rowNum; i++ {
if result.IsNull(i) {
continue
}
match := b.pattern.DoMatch(exprParam.getStringVal(i))
match := pattern.DoMatch(exprParam.getStringVal(i))
i64s[i] = boolToInt64(match)
}
return nil
Expand Down Expand Up @@ -196,11 +199,11 @@ func (b *builtinIlikeSig) vecEvalInt(ctx EvalContext, input *chunk.Chunk, result
b.lowerExpr(params[0], rowNum)
escape = b.lowerPattern(params[1], rowNum, escape)

b.tryToMemorize(params[1], escape)
if !b.isMemorizedPattern {
pattern := collate.ConvertAndGetBinCollation(b.collation).Pattern()
pattern, ok := b.tryToVecMemorize(ctx, params[1], escape)
if !ok {
pattern = collate.ConvertAndGetBinCollation(b.collation).Pattern()
return b.ilikeWithoutMemorization(pattern, params, rowNum, escape, result)
}

return b.ilikeWithMemorization(params[0], rowNum, result)
return b.ilikeWithMemorization(pattern, params[0], rowNum, result)
}
41 changes: 17 additions & 24 deletions pkg/expression/builtin_like.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,11 @@
package expression

import (
"sync"

"github.com/pingcap/tidb/pkg/sessionctx"
"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/tipb/go-tipb"
)

Expand All @@ -46,25 +45,21 @@ func (c *likeFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio
return nil, err
}
bf.tp.SetFlen(1)
sig := &builtinLikeSig{bf, nil, false, sync.Once{}}
sig := &builtinLikeSig{baseBuiltinFunc: bf}
sig.setPbCode(tipb.ScalarFuncSig_LikeSig)
return sig, nil
}

type builtinLikeSig struct {
baseBuiltinFunc
// pattern and isMemorizedPattern is not serialized with builtinLikeSig, treat them as a cache to accelerate
// pattern is not serialized with builtinLikeSig, treat them as a cache to accelerate
// the evaluation of builtinLikeSig.
pattern collate.WildcardPattern
isMemorizedPattern bool
once sync.Once
patternCache builtinFuncCache[collate.WildcardPattern]
}

func (b *builtinLikeSig) Clone() builtinFunc {
newSig := &builtinLikeSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
newSig.pattern = b.pattern
newSig.isMemorizedPattern = b.isMemorizedPattern
return newSig
}

Expand All @@ -84,22 +79,20 @@ func (b *builtinLikeSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool, e
if isNull || err != nil {
return 0, isNull, err
}
memorization := func() {
if b.pattern == nil {
b.pattern = b.collator().Pattern()
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
}
var pattern collate.WildcardPattern
if b.args[1].ConstItem(false) && b.args[2].ConstItem(false) {
pattern, err = b.patternCache.getOrInitCache(ctx, func() (collate.WildcardPattern, error) {
ret := b.collator().Pattern()
ret.Compile(patternStr, byte(escape))
return ret, nil
})
intest.AssertNoError(err)
if err != nil {
return 0, true, err
}
}
// Only be executed once to achieve thread-safe
b.once.Do(memorization)
if !b.isMemorizedPattern {
// Must not use b.pattern to avoid data race
pattern := b.collator().Pattern()
} else {
pattern = b.collator().Pattern()
pattern.Compile(patternStr, byte(escape))
return boolToInt64(pattern.DoMatch(valStr)), false, nil
}
return boolToInt64(b.pattern.DoMatch(valStr)), false, nil
return boolToInt64(pattern.DoMatch(valStr)), false, nil
}
5 changes: 2 additions & 3 deletions pkg/expression/distsql_builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ package expression
import (
"fmt"
"strconv"
"sync"
"time"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -659,9 +658,9 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti
case tipb.ScalarFuncSig_UUID:
f = &builtinUUIDSig{base}
case tipb.ScalarFuncSig_LikeSig:
f = &builtinLikeSig{base, nil, false, sync.Once{}}
f = &builtinLikeSig{baseBuiltinFunc: base}
case tipb.ScalarFuncSig_IlikeSig:
f = &builtinIlikeSig{base, nil, false, sync.Once{}}
f = &builtinIlikeSig{baseBuiltinFunc: base}
case tipb.ScalarFuncSig_RegexpSig:
f = &builtinRegexpLikeFuncSig{regexpBaseFuncSig{baseBuiltinFunc: base}}
case tipb.ScalarFuncSig_RegexpUTF8Sig:
Expand Down
47 changes: 47 additions & 0 deletions tests/integrationtest/r/expression/plan_cache.result
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,53 @@ admin reload expr_pushdown_blacklist;
set tidb_enable_prepared_plan_cache=default;
set tidb_enable_vectorized_expression=default;
set tidb_enable_prepared_plan_cache=ON;
drop table if exists t1;
create table t1 (a varchar(40));
insert into t1 values ('C1'),('R1'),('c1'),('r1');
insert into mysql.expr_pushdown_blacklist values('like', 'tikv,tiflash,tidb', 'for test');
insert into mysql.expr_pushdown_blacklist values('ilike', 'tikv,tiflash,tidb', 'for test');
admin reload expr_pushdown_blacklist;
set tidb_enable_vectorized_expression=ON;
prepare stmt1 from 'select a from t1 where a like ? order by a asc';
prepare stmt2 from 'select a from t1 where a ilike ? order by a asc';
set @a='%C%';
execute stmt1 using @a;
a
C1
execute stmt2 using @a;
a
C1
c1
set @a='%R%';
execute stmt1 using @a;
a
R1
execute stmt2 using @a;
a
R1
r1
set tidb_enable_vectorized_expression=OFF;
set @a='%C%';
execute stmt1 using @a;
a
C1
execute stmt2 using @a;
a
C1
c1
set @a='%R%';
execute stmt1 using @a;
a
R1
execute stmt2 using @a;
a
R1
r1
delete from mysql.expr_pushdown_blacklist where name in ('like', 'ilike') and store_type = 'tikv,tiflash,tidb' and reason = 'for test';
admin reload expr_pushdown_blacklist;
set tidb_enable_prepared_plan_cache=default;
set tidb_enable_vectorized_expression=default;
set tidb_enable_prepared_plan_cache=ON;
drop table if exists t;
create table t(col_int int);
insert into t values(null);
Expand Down
29 changes: 29 additions & 0 deletions tests/integrationtest/t/expression/plan_cache.test
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,35 @@ admin reload expr_pushdown_blacklist;
set tidb_enable_prepared_plan_cache=default;
set tidb_enable_vectorized_expression=default;

# TestCacheLikeAndIlike
set tidb_enable_prepared_plan_cache=ON;
drop table if exists t1;
create table t1 (a varchar(40));
insert into t1 values ('C1'),('R1'),('c1'),('r1');
insert into mysql.expr_pushdown_blacklist values('like', 'tikv,tiflash,tidb', 'for test');
insert into mysql.expr_pushdown_blacklist values('ilike', 'tikv,tiflash,tidb', 'for test');
admin reload expr_pushdown_blacklist;
set tidb_enable_vectorized_expression=ON;
prepare stmt1 from 'select a from t1 where a like ? order by a asc';
prepare stmt2 from 'select a from t1 where a ilike ? order by a asc';
set @a='%C%';
execute stmt1 using @a;
execute stmt2 using @a;
set @a='%R%';
execute stmt1 using @a;
execute stmt2 using @a;
set tidb_enable_vectorized_expression=OFF;
set @a='%C%';
execute stmt1 using @a;
execute stmt2 using @a;
set @a='%R%';
execute stmt1 using @a;
execute stmt2 using @a;
delete from mysql.expr_pushdown_blacklist where name in ('like', 'ilike') and store_type = 'tikv,tiflash,tidb' and reason = 'for test';
admin reload expr_pushdown_blacklist;
set tidb_enable_prepared_plan_cache=default;
set tidb_enable_vectorized_expression=default;

# TestCacheRefineArgs
set tidb_enable_prepared_plan_cache=ON;
drop table if exists t;
Expand Down

0 comments on commit 4e9f006

Please sign in to comment.