diff --git a/pkg/expression/builtin.go b/pkg/expression/builtin.go index a82d8ee96999a..0e2f8e007566c 100644 --- a/pkg/expression/builtin.go +++ b/pkg/expression/builtin.go @@ -28,6 +28,7 @@ import ( "slices" "strings" "sync" + "sync/atomic" "unsafe" "github.com/gogo/protobuf/proto" @@ -40,6 +41,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/tipb/go-tipb" ) @@ -1017,3 +1019,46 @@ func (b *baseBuiltinFunc) MemoryUsage() (sum int64) { } return } + +type builtinFuncCacheItem[T any] struct { + ctxID uint64 + item T +} + +type builtinFuncCache[T any] struct { + sync.Mutex + cached atomic.Pointer[builtinFuncCacheItem[T]] +} + +func (c *builtinFuncCache[T]) getCache(ctxID uint64) (v T, ok bool) { + if p := c.cached.Load(); p != nil && p.ctxID == ctxID { + return p.item, true + } + return v, false +} + +func (c *builtinFuncCache[T]) getOrInitCache(ctx EvalContext, constructCache func() (T, error)) (T, error) { + intest.Assert(constructCache != nil) + ctxID := ctx.GetSessionVars().StmtCtx.CtxID() + if item, ok := c.getCache(ctxID); ok { + return item, nil + } + + c.Lock() + defer c.Unlock() + if item, ok := c.getCache(ctxID); ok { + return item, nil + } + + item, err := constructCache() + if err != nil { + var def T + return def, err + } + + c.cached.Store(&builtinFuncCacheItem[T]{ + ctxID: ctxID, + item: item, + }) + return item, nil +} diff --git a/pkg/expression/builtin_regexp.go b/pkg/expression/builtin_regexp.go index d3210f4474fc1..dc60891aaa285 100644 --- a/pkg/expression/builtin_regexp.go +++ b/pkg/expression/builtin_regexp.go @@ -66,6 +66,99 @@ var validMatchType = set.NewStringSet( flagS, // The . character matches line terminators ) +type regexpNewBaseFuncSig struct { + baseBuiltinFunc + memorizedRegexp builtinFuncCache[regexpMemorizedSig] +} + +// check binary collation, not xxx_bin collation! +func (re *regexpNewBaseFuncSig) isBinaryCollation() bool { + return re.collation == charset.CollationBin && re.charset == charset.CharsetBin +} + +func (re *regexpNewBaseFuncSig) clone() *regexpNewBaseFuncSig { + newSig := ®expNewBaseFuncSig{} + newSig.cloneFrom(&re.baseBuiltinFunc) + return newSig +} + +// we can memorize the regexp when: +// 1. pattern and match type are constant +// 2. pattern is const and there is no match type argument +// +// return true: need, false: needless +func (re *regexpNewBaseFuncSig) canMemorizeRegexp(matchTypeIdx int) bool { + // If the pattern and match type are both constants, we can cache the regexp into memory. + // Notice that the above two arguments are not required to be constant across contexts because the cache is only + // valid when the two context ids are the same. + return re.args[patternIdx].ConstItem(false) && + (len(re.args) <= matchTypeIdx || re.args[matchTypeIdx].ConstItem(false)) +} + +// buildRegexp builds a new `*regexp.Regexp` from the pattern and matchType +func (re *regexpNewBaseFuncSig) buildRegexp(pattern string, matchType string) (reg *regexp.Regexp, err error) { + matchType, err = getRegexpMatchType(matchType, re.collation) + if err != nil { + return nil, err + } + + if len(matchType) == 0 { + reg, err = regexp.Compile(pattern) + } else { + reg, err = regexp.Compile(fmt.Sprintf("(?%s)%s", matchType, pattern)) + } + + if err != nil { + return nil, ErrRegexp.GenWithStackByArgs(err) + } + + return reg, nil +} + +// getRegexp returns the Regexp which can be used by the current function. +// If the pattern and matchType arguments are both constant, the `*regexp.Regexp` object will be cached in memory. +// The next call of `getRegexp` will return the cached regexp if it is present and the context id is equal +func (re *regexpNewBaseFuncSig) getRegexp(ctx EvalContext, pattern string, matchType string, matchTypeIdx int) (*regexp.Regexp, error) { + if !re.canMemorizeRegexp(matchTypeIdx) { + return re.buildRegexp(pattern, matchType) + } + + sig, err := re.memorizedRegexp.getOrInitCache(ctx, func() (ret regexpMemorizedSig, err error) { + ret.memorizedRegexp, ret.memorizedErr = re.buildRegexp(pattern, matchType) + return + }) + + if err != nil { + return nil, err + } + + return sig.memorizedRegexp, sig.memorizedErr +} + +func (re *regexpNewBaseFuncSig) tryVecMemorizedRegexp(ctx EvalContext, params []*funcParam, matchTypeIdx int, nRows int) (*regexp.Regexp, bool, error) { + // Check memorization + if nRows == 0 || !re.canMemorizeRegexp(matchTypeIdx) { + return nil, false, nil + } + + pattern := params[patternIdx].getStringVal(0) + if len(pattern) == 0 { + return nil, false, ErrRegexp.GenWithStackByArgs(emptyPatternErr) + } + + matchType := params[matchTypeIdx].getStringVal(0) + sig, err := re.memorizedRegexp.getOrInitCache(ctx, func() (ret regexpMemorizedSig, err error) { + ret.memorizedRegexp, ret.memorizedErr = re.buildRegexp(pattern, matchType) + return + }) + + if err != nil { + return nil, false, err + } + + return sig.memorizedRegexp, true, sig.memorizedErr +} + type regexpBaseFuncSig struct { baseBuiltinFunc regexpMemorizedSig @@ -89,11 +182,11 @@ func (re *regexpBaseFuncSig) clone() *regexpBaseFuncSig { // If characters specifying contradictory options are specified // within match_type, the rightmost one takes precedence. -func (re *regexpBaseFuncSig) getMatchType(userInputMatchType string) (string, error) { +func getRegexpMatchType(userInputMatchType string, collation string) (string, error) { flag := "" matchTypeSet := set.NewStringSet() - if collate.IsCICollation(re.baseBuiltinFunc.collation) { + if collate.IsCICollation(collation) { matchTypeSet.Insert(flagI) } @@ -126,7 +219,7 @@ func (re *regexpBaseFuncSig) getMatchType(userInputMatchType string) (string, er // To get a unified compile interface in initMemoizedRegexp, we need to process many things in genCompile func (re *regexpBaseFuncSig) genCompile(matchType string) (func(string) (*regexp.Regexp, error), error) { - matchType, err := re.getMatchType(matchType) + matchType, err := getRegexpMatchType(matchType, re.baseBuiltinFunc.collation) if err != nil { return nil, err } @@ -389,7 +482,7 @@ func (c *regexpSubstrFunctionClass) getFunction(ctx sessionctx.Context, args []E argType := args[0].GetType() bf.tp.SetFlen(argType.GetFlen()) sig := builtinRegexpSubstrFuncSig{ - regexpBaseFuncSig: regexpBaseFuncSig{baseBuiltinFunc: bf}, + regexpNewBaseFuncSig: regexpNewBaseFuncSig{baseBuiltinFunc: bf}, } sig.setPbCode(tipb.ScalarFuncSig_RegexpSubstrSig) @@ -401,7 +494,7 @@ func (c *regexpSubstrFunctionClass) getFunction(ctx sessionctx.Context, args []E } type builtinRegexpSubstrFuncSig struct { - regexpBaseFuncSig + regexpNewBaseFuncSig } func (re *builtinRegexpSubstrFuncSig) vectorized() bool { @@ -410,7 +503,7 @@ func (re *builtinRegexpSubstrFuncSig) vectorized() bool { func (re *builtinRegexpSubstrFuncSig) Clone() builtinFunc { newSig := &builtinRegexpSubstrFuncSig{} - newSig.regexpBaseFuncSig = *re.regexpBaseFuncSig.clone() + newSig.regexpNewBaseFuncSig = *re.regexpNewBaseFuncSig.clone() return newSig } @@ -499,44 +592,15 @@ func (re *builtinRegexpSubstrFuncSig) evalString(ctx EvalContext, row chunk.Row) } } - memorize := func() { - compile, err := re.genCompile(matchType) - if err != nil { - re.memorizedErr = err - return - } - re.memorize(compile, pat) - } - - if re.canMemorize(ctx, regexpSubstrMatchTypeIdx) { - re.once.Do(memorize) // Avoid data race - } - - if !re.isMemorizedRegexpInitialized() { - compile, err := re.genCompile(matchType) - if err != nil { - return "", true, ErrRegexp.GenWithStackByArgs(err) - } - reg, err := compile(pat) - if err != nil { - return "", true, ErrRegexp.GenWithStackByArgs(err) - } - - if re.isBinaryCollation() { - return re.findBinString(reg, bexpr, occurrence) - } - return re.findString(reg, expr, occurrence) - } - - if re.memorizedErr != nil { - return "", true, ErrRegexp.GenWithStackByArgs(re.memorizedErr) + reg, err := re.getRegexp(ctx, pat, matchType, regexpSubstrMatchTypeIdx) + if err != nil { + return "", true, err } if re.isBinaryCollation() { - return re.findBinString(re.memorizedRegexp, bexpr, occurrence) + return re.findBinString(reg, bexpr, occurrence) } - - return re.findString(re.memorizedRegexp, expr, occurrence) + return re.findString(reg, expr, occurrence) } // REGEXP_SUBSTR(expr, pat[, pos[, occurrence[, match_type]]]) @@ -599,7 +663,7 @@ func (re *builtinRegexpSubstrFuncSig) vecEvalString(ctx EvalContext, input *chun } // Check memorization - err = re.tryToMemorize(ctx, params, regexpSubstrMatchTypeIdx, n) + reg, memorized, err := re.tryVecMemorizedRegexp(ctx, params, regexpSubstrMatchTypeIdx, n) if err != nil { return err } @@ -647,11 +711,13 @@ func (re *builtinRegexpSubstrFuncSig) vecEvalString(ctx EvalContext, input *chun occurrence = 1 } - // Get match type and generate regexp - matchType := params[4].getStringVal(i) - reg, err := re.genRegexp(params[1].getStringVal(i), matchType) - if err != nil { - return err + if !memorized { + // Get pattern and match type and then generate regexp + pattern := params[1].getStringVal(i) + matchType := params[4].getStringVal(i) + if reg, err = re.buildRegexp(pattern, matchType); err != nil { + return err + } } // Find string diff --git a/pkg/expression/builtin_regexp_test.go b/pkg/expression/builtin_regexp_test.go index 41665dfcf73ee..8b2b99b0dc83c 100644 --- a/pkg/expression/builtin_regexp_test.go +++ b/pkg/expression/builtin_regexp_test.go @@ -1224,3 +1224,80 @@ func TestRegexpReplaceVec(t *testing.T) { testVectorizedBuiltinFunc(t, vecBuiltinRegexpReplaceCases) } + +func TestRegexpCache(t *testing.T) { + ctx := createContext(t) + + // if the pattern or match type is not constant, it should not be cached + sig := regexpNewBaseFuncSig{} + sig.args = []Expression{&Column{}, &Column{}, &Constant{}} + reg, err := sig.getRegexp(ctx, "abc", "", 2) + require.NoError(t, err) + require.Equal(t, "abc", reg.String()) + + reg, err = sig.getRegexp(ctx, "def", "", 2) + require.NoError(t, err) + require.Equal(t, "def", reg.String()) + + reg, ok, err := sig.tryVecMemorizedRegexp(ctx, []*funcParam{ + {defaultStrVal: "x"}, + {defaultStrVal: "aaa"}, + {defaultStrVal: ""}, + }, 2, 1) + require.Nil(t, reg) + require.False(t, ok) + require.NoError(t, err) + + _, ok = sig.memorizedRegexp.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.False(t, ok) + + sig.args = []Expression{&Column{}, &Constant{}, &Column{}} + reg, err = sig.getRegexp(ctx, "bbb", "", 2) + require.NoError(t, err) + require.Equal(t, "bbb", reg.String()) + + reg, ok, err = sig.tryVecMemorizedRegexp(ctx, []*funcParam{ + {defaultStrVal: "x"}, + {defaultStrVal: "aaa"}, + {defaultStrVal: ""}, + }, 2, 1) + require.Nil(t, reg) + require.False(t, ok) + require.NoError(t, err) + + _, ok = sig.memorizedRegexp.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.False(t, ok) + + // if pattern and match type are both constant, it should be cached + sig = regexpNewBaseFuncSig{} + sig.args = []Expression{&Column{}, &Constant{ParamMarker: &ParamMarker{}}, &Constant{ParamMarker: &ParamMarker{}}} + reg, err = sig.getRegexp(ctx, "ccc", "", 2) + require.NoError(t, err) + require.Equal(t, "ccc", reg.String()) + + reg2, err := sig.getRegexp(ctx, "ddd", "", 2) + require.NoError(t, err) + require.Same(t, reg, reg2) + require.Equal(t, "ccc", reg2.String()) + + sig = regexpNewBaseFuncSig{} + sig.args = []Expression{&Column{}, &Constant{ParamMarker: &ParamMarker{}}, &Constant{ParamMarker: &ParamMarker{}}} + reg, ok, err = sig.tryVecMemorizedRegexp(ctx, []*funcParam{ + {defaultStrVal: "x"}, + {defaultStrVal: "ddd"}, + {defaultStrVal: ""}, + }, 2, 1) + require.Equal(t, "ddd", reg.String()) + require.True(t, ok) + require.NoError(t, err) + + reg2, ok, err = sig.tryVecMemorizedRegexp(ctx, []*funcParam{ + {defaultStrVal: "x"}, + {defaultStrVal: "eee"}, + {defaultStrVal: ""}, + }, 2, 1) + require.Same(t, reg, reg2) + require.Equal(t, "ddd", reg2.String()) + require.True(t, ok) + require.NoError(t, err) +} diff --git a/pkg/expression/builtin_test.go b/pkg/expression/builtin_test.go index 8bf4d1f0fa1de..ca71c5af03b21 100644 --- a/pkg/expression/builtin_test.go +++ b/pkg/expression/builtin_test.go @@ -17,8 +17,11 @@ package expression import ( "reflect" "sync" + "sync/atomic" "testing" + "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/model" @@ -161,6 +164,100 @@ func TestDisplayName(t *testing.T) { require.Equal(t, "other_unknown_func", GetDisplayName("other_unknown_func")) } +func TestBuiltinFuncCacheConcurrency(t *testing.T) { + cache := builtinFuncCache[int]{} + ctx := createContext(t) + + var invoked atomic.Int64 + construct := func() (int, error) { + invoked.Add(1) + time.Sleep(time.Millisecond) + return 100 + int(invoked.Load()), nil + } + + var wg sync.WaitGroup + concurrency := 8 + wg.Add(concurrency) + for i := 0; i < concurrency; i++ { + go func() { + defer wg.Done() + v, err := cache.getOrInitCache(ctx, construct) + // all goroutines should get the same value + require.NoError(t, err) + require.Equal(t, 101, v) + }() + } + + wg.Wait() + // construct will only be called once even in concurrency + require.Equal(t, int64(1), invoked.Load()) +} + +func TestBuiltinFuncCache(t *testing.T) { + cache := builtinFuncCache[int]{} + ctx := createContext(t) + + // ok should be false when no cache present + v, ok := cache.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.Equal(t, 0, v) + require.False(t, ok) + + // getCache should not init cache + v, ok = cache.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.Equal(t, 0, v) + require.False(t, ok) + + var invoked atomic.Int64 + returnError := false + construct := func() (int, error) { + invoked.Add(1) + if returnError { + return 128, errors.New("mockError") + } + return 100 + int(invoked.Load()), nil + } + + // the first getOrInitCache should init cache + v, err := cache.getOrInitCache(ctx, construct) + require.NoError(t, err) + require.Equal(t, 101, v) + require.Equal(t, int64(1), invoked.Load()) + + // get should return the cache + v, ok = cache.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.Equal(t, 101, v) + require.True(t, ok) + + // the second should use the cached one + v, err = cache.getOrInitCache(ctx, construct) + require.NoError(t, err) + require.Equal(t, 101, v) + require.Equal(t, int64(1), invoked.Load()) + + // if ctxID changed, should re-init cache + ctx = createContext(t) + v, err = cache.getOrInitCache(ctx, construct) + require.NoError(t, err) + require.Equal(t, 102, v) + require.Equal(t, int64(2), invoked.Load()) + v, ok = cache.getCache(ctx.GetSessionVars().StmtCtx.CtxID()) + require.Equal(t, 102, v) + require.True(t, ok) + + // error should be returned + ctx = createContext(t) + returnError = true + v, err = cache.getOrInitCache(ctx, construct) + require.Equal(t, 0, v) + require.EqualError(t, err, "mockError") + + // error should not be cached + returnError = false + v, err = cache.getOrInitCache(ctx, construct) + require.NoError(t, err) + require.Equal(t, 104, v) +} + // newFunctionForTest creates a new ScalarFunction using funcName and arguments, // it is different from expression.NewFunction which needs an additional retType argument. func newFunctionForTest(ctx sessionctx.Context, funcName string, args ...Expression) (Expression, error) { diff --git a/pkg/expression/distsql_builtin.go b/pkg/expression/distsql_builtin.go index 3d041644e58f6..67460a3d9dbc6 100644 --- a/pkg/expression/distsql_builtin.go +++ b/pkg/expression/distsql_builtin.go @@ -669,7 +669,7 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti case tipb.ScalarFuncSig_RegexpLikeSig: f = &builtinRegexpLikeFuncSig{regexpBaseFuncSig{base, regexpMemorizedSig{nil, nil}, sync.Once{}}} case tipb.ScalarFuncSig_RegexpSubstrSig: - f = &builtinRegexpSubstrFuncSig{regexpBaseFuncSig{base, regexpMemorizedSig{nil, nil}, sync.Once{}}} + f = &builtinRegexpSubstrFuncSig{regexpNewBaseFuncSig{baseBuiltinFunc: base}} case tipb.ScalarFuncSig_RegexpInStrSig: f = &builtinRegexpInStrFuncSig{regexpBaseFuncSig{base, regexpMemorizedSig{nil, nil}, sync.Once{}}} case tipb.ScalarFuncSig_RegexpReplaceSig: diff --git a/pkg/sessionctx/stmtctx/BUILD.bazel b/pkg/sessionctx/stmtctx/BUILD.bazel index b9602e725fe91..20ecfbdc9f3c2 100644 --- a/pkg/sessionctx/stmtctx/BUILD.bazel +++ b/pkg/sessionctx/stmtctx/BUILD.bazel @@ -41,7 +41,7 @@ go_test( ], embed = [":stmtctx"], flaky = True, - shard_count = 10, + shard_count = 11, deps = [ "//pkg/kv", "//pkg/sessionctx/variable", diff --git a/pkg/sessionctx/stmtctx/stmtctx.go b/pkg/sessionctx/stmtctx/stmtctx.go index b833da451d2ea..bccfbb4f70831 100644 --- a/pkg/sessionctx/stmtctx/stmtctx.go +++ b/pkg/sessionctx/stmtctx/stmtctx.go @@ -147,6 +147,8 @@ func (rf *ReferenceCount) UnFreeze() { atomic.StoreInt32((*int32)(rf), ReferenceCountNoReference) } +var stmtCtxIDGenerator atomic.Uint64 + // StatementContext contains variables for a statement. // It should be reset before executing a statement. type StatementContext struct { @@ -156,6 +158,8 @@ type StatementContext struct { _ constructor.Constructor `ctor:"NewStmtCtx,NewStmtCtxWithTimeZone,Reset"` + ctxID uint64 + // typeCtx is used to indicate how to make the type conversation. typeCtx types.Context @@ -425,15 +429,15 @@ type StatementContext struct { // NewStmtCtx creates a new statement context func NewStmtCtx() *StatementContext { - sc := &StatementContext{} - sc.typeCtx = types.NewContext(types.DefaultStmtFlags, time.UTC, sc) - return sc + return NewStmtCtxWithTimeZone(time.UTC) } // NewStmtCtxWithTimeZone creates a new StatementContext with the given timezone func NewStmtCtxWithTimeZone(tz *time.Location) *StatementContext { intest.AssertNotNil(tz) - sc := &StatementContext{} + sc := &StatementContext{ + ctxID: stmtCtxIDGenerator.Add(1), + } sc.typeCtx = types.NewContext(types.DefaultStmtFlags, tz, sc) return sc } @@ -441,10 +445,16 @@ func NewStmtCtxWithTimeZone(tz *time.Location) *StatementContext { // Reset resets a statement context func (sc *StatementContext) Reset() { *sc = StatementContext{ + ctxID: stmtCtxIDGenerator.Add(1), typeCtx: types.NewContext(types.DefaultStmtFlags, time.UTC, sc), } } +// CtxID returns the context id of the statement +func (sc *StatementContext) CtxID() uint64 { + return sc.ctxID +} + // TimeZone returns the timezone of the type context func (sc *StatementContext) TimeZone() *time.Location { intest.AssertNotNil(sc) diff --git a/pkg/sessionctx/stmtctx/stmtctx_test.go b/pkg/sessionctx/stmtctx/stmtctx_test.go index 24c60d9375c34..a93a269915f42 100644 --- a/pkg/sessionctx/stmtctx/stmtctx_test.go +++ b/pkg/sessionctx/stmtctx/stmtctx_test.go @@ -386,6 +386,28 @@ func TestResetStmtCtx(t *testing.T) { require.Equal(t, "err2", warnings[0].Err.Error()) } +func TestStmtCtxID(t *testing.T) { + sc := stmtctx.NewStmtCtx() + currentID := sc.CtxID() + + cases := []struct { + fn func() *stmtctx.StatementContext + }{ + {func() *stmtctx.StatementContext { return stmtctx.NewStmtCtx() }}, + {func() *stmtctx.StatementContext { return stmtctx.NewStmtCtxWithTimeZone(time.Local) }}, + {func() *stmtctx.StatementContext { + sc.Reset() + return sc + }}, + } + + for _, c := range cases { + ctxID := c.fn().CtxID() + require.Greater(t, ctxID, currentID) + currentID = ctxID + } +} + func BenchmarkErrCtx(b *testing.B) { sc := stmtctx.NewStmtCtx() diff --git a/tests/integrationtest/r/expression/plan_cache.result b/tests/integrationtest/r/expression/plan_cache.result index 8559e2da02230..b7b825f5d495e 100644 --- a/tests/integrationtest/r/expression/plan_cache.result +++ b/tests/integrationtest/r/expression/plan_cache.result @@ -94,16 +94,49 @@ set tidb_enable_prepared_plan_cache=ON; drop table if exists t1; create table t1 (a varchar(40)); insert into t1 values ('C1'),('R1'); +insert into mysql.expr_pushdown_blacklist values('regexp_like', 'tikv,tiflash,tidb', 'for test'); +insert into mysql.expr_pushdown_blacklist values('regexp_substr', '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 rlike ?'; +prepare stmt2 from 'select a, regexp_substr(a, ?) from t1 order by a asc'; set @a='^C.*'; execute stmt1 using @a; a C1 +execute stmt2 using @a; +a regexp_substr(a, ?) +C1 C1 +R1 NULL set @a='^R.*'; execute stmt1 using @a; a R1 +execute stmt2 using @a; +a regexp_substr(a, ?) +C1 NULL +R1 R1 +set tidb_enable_vectorized_expression=OFF; +set @a='^C.*'; +execute stmt1 using @a; +a +C1 +execute stmt2 using @a; +a regexp_substr(a, ?) +C1 C1 +R1 NULL +set @a='^R.*'; +execute stmt1 using @a; +a +R1 +execute stmt2 using @a; +a regexp_substr(a, ?) +C1 NULL +R1 R1 +delete from mysql.expr_pushdown_blacklist where name like 'regexp%' 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); diff --git a/tests/integrationtest/t/expression/plan_cache.test b/tests/integrationtest/t/expression/plan_cache.test index 2d211d1187cf7..a81b6673aed35 100644 --- a/tests/integrationtest/t/expression/plan_cache.test +++ b/tests/integrationtest/t/expression/plan_cache.test @@ -67,12 +67,29 @@ set tidb_enable_prepared_plan_cache=ON; drop table if exists t1; create table t1 (a varchar(40)); insert into t1 values ('C1'),('R1'); +insert into mysql.expr_pushdown_blacklist values('regexp_like', 'tikv,tiflash,tidb', 'for test'); +insert into mysql.expr_pushdown_blacklist values('regexp_substr', '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 rlike ?'; +prepare stmt2 from 'select a, regexp_substr(a, ?) from t1 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 like 'regexp%' 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;