Skip to content

Commit

Permalink
Merge branch 'master' into gogctuner_value
Browse files Browse the repository at this point in the history
  • Loading branch information
hawkingrei authored Nov 3, 2022
2 parents 90bc5f8 + 510003d commit 6261687
Show file tree
Hide file tree
Showing 22 changed files with 227 additions and 28 deletions.
4 changes: 3 additions & 1 deletion domain/plan_replayer.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"sync"
"time"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)
Expand All @@ -39,7 +40,8 @@ type dumpFileGcChecker struct {
// GetPlanReplayerDirName returns plan replayer directory path.
// The path is related to the process id.
func GetPlanReplayerDirName() string {
return filepath.Join(os.TempDir(), "replayer", strconv.Itoa(os.Getpid()))
tidbLogDir := filepath.Dir(config.GetGlobalConfig().Log.File.Filename)
return filepath.Join(tidbLogDir, "replayer")
}

func parseTime(s string) (time.Time, error) {
Expand Down
23 changes: 23 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6072,12 +6072,15 @@ func TestGlobalMemoryControl(t *testing.T) {

tk1 := testkit.NewTestKit(t, store)
tracker1 := tk1.Session().GetSessionVars().MemTracker
tracker1.FallbackOldAndSetNewAction(&memory.PanicOnExceed{})

tk2 := testkit.NewTestKit(t, store)
tracker2 := tk2.Session().GetSessionVars().MemTracker
tracker2.FallbackOldAndSetNewAction(&memory.PanicOnExceed{})

tk3 := testkit.NewTestKit(t, store)
tracker3 := tk3.Session().GetSessionVars().MemTracker
tracker3.FallbackOldAndSetNewAction(&memory.PanicOnExceed{})

sm := &testkit.MockSessionManager{
PS: []*util.ProcessInfo{tk1.Session().ShowProcess(), tk2.Session().ShowProcess(), tk3.Session().ShowProcess()},
Expand Down Expand Up @@ -6176,3 +6179,23 @@ func TestCompileOutOfMemoryQuota(t *testing.T) {
err := tk.ExecToErr("select t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a")
require.Contains(t, err.Error(), "Out Of Memory Quota!")
}

func TestSessionRootTrackerDetach(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
defer tk.MustExec("set global tidb_mem_oom_action = DEFAULT")
tk.MustExec("set global tidb_mem_oom_action='CANCEL'")
tk.MustExec("use test")
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec("create table t1(a int, c int, index idx(a))")
tk.MustExec("set tidb_mem_quota_query=10")
err := tk.ExecToErr("select /*+hash_join(t1)*/ t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a")
require.Contains(t, err.Error(), "Out Of Memory Quota!")
tk.MustExec("set tidb_mem_quota_query=1000")
rs, err := tk.Exec("select /*+hash_join(t1)*/ t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a")
require.NoError(t, err)
require.NotNil(t, tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false))
err = rs.Close()
require.NoError(t, err)
require.Nil(t, tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false))
}
2 changes: 1 addition & 1 deletion executor/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1030,7 +1030,7 @@ func TestPreparePlanCache4Function(t *testing.T) {
tk.MustExec("set @a = 0, @b = 1, @c = 2, @d = null;")
tk.MustQuery("execute stmt using @a, @b;").Check(testkit.Rows("<nil> 2", "0 0", "1 1", "2 2"))
tk.MustQuery("execute stmt using @c, @d;").Check(testkit.Rows("<nil> 1", "0 2", "1 2", "2 0"))
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0"))
}

func TestPreparePlanCache4DifferentSystemVars(t *testing.T) {
Expand Down
4 changes: 2 additions & 2 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -334,7 +334,7 @@ func TestPrepareWithAggregation(t *testing.T) {
tk.MustExec(fmt.Sprintf(`set @@tidb_enable_prepared_plan_cache=%v`, flag))

se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket, tk.Session()),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down Expand Up @@ -599,7 +599,7 @@ func TestPrepareDealloc(t *testing.T) {
tk.MustExec(`set @@tidb_enable_prepared_plan_cache=true`)

se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: plannercore.NewLRUPlanCache(3, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
PreparedPlanCache: plannercore.NewLRUPlanCache(3, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket, tk.Session()),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down
18 changes: 18 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2023,3 +2023,21 @@ func TestSetTiFlashFastScanVariable(t *testing.T) {
tk.MustExec("set GLOBAL tiflash_fastscan=OFF;")
tk.MustQuery("select @@global.tiflash_fastscan").Check(testkit.Rows("0"))
}

func TestSetPlanCacheMemoryMonitor(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_prepared_plan_cache_memory_monitor=OFF;")
tk.MustQuery("select @@session.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_prepared_plan_cache_memory_monitor=1;")
tk.MustQuery("select @@session.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_prepared_plan_cache_memory_monitor=off;")
tk.MustQuery("select @@global.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("0"))
}
5 changes: 5 additions & 0 deletions expression/builtin_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,11 @@ func InferType4ControlFuncs(ctx sessionctx.Context, funcName string, lexp, rexp
}
flen := maxlen(lhsFlen, rhsFlen) + resultFieldType.GetDecimal() + 1 // account for -1 len fields
resultFieldType.SetFlenUnderLimit(flen)
} else if evalType == types.ETString {
lhsLen, rhsLen := lhs.GetFlen(), rhs.GetFlen()
if lhsLen != types.UnspecifiedLength && rhsLen != types.UnspecifiedLength {
resultFieldType.SetFlen(mathutil.Max(lhsLen, rhsLen))
}
} else {
resultFieldType.SetFlen(maxlen(lhs.GetFlen(), rhs.GetFlen()))
}
Expand Down
3 changes: 2 additions & 1 deletion expression/integration_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3795,10 +3795,11 @@ func TestPreparePlanCacheOnCachedTable(t *testing.T) {

tk := testkit.NewTestKit(t, store)
tk.MustExec("set tidb_enable_prepared_plan_cache=ON")
tk.Session()

var err error
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket, tk.Session()),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down
11 changes: 11 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7805,3 +7805,14 @@ func TestJSONExtractRange(t *testing.T) {
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[0 to last]')`).Check(testkit.Rows("[1, 2, 3, 4]"))
tk.MustQuery(`select json_extract('[{"a": [1,2,3,4]}]', '$[0].a[0 to 2]')`).Check(testkit.Rows("[1, 2, 3]"))
}

func TestIfNullParamMarker(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t (c1 varchar(100), c2 varchar(128));")
tk.MustExec(`prepare pr1 from "insert into t values(ifnull(?,' '),ifnull(?,' '))";`)
tk.MustExec(`set @a='1',@b=repeat('x', 80);`)
// Should not report 'Data too long for column' error.
tk.MustExec(`execute pr1 using @a,@b;`)
}
2 changes: 1 addition & 1 deletion infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func newTestKitWithRoot(t *testing.T, store kv.Storage) *testkit.TestKit {
func newTestKitWithPlanCache(t *testing.T, store kv.Storage) *testkit.TestKit {
tk := testkit.NewTestKit(t, store)
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{PreparedPlanCache: plannercore.NewLRUPlanCache(100,
0.1, math.MaxUint64, plannercore.PickPlanFromBucket)})
0.1, math.MaxUint64, plannercore.PickPlanFromBucket, tk.Session())})
require.NoError(t, err)
tk.SetSession(se)
tk.RefreshConnectionID()
Expand Down
23 changes: 19 additions & 4 deletions planner/core/plan_cache_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/kvcache"
Expand Down Expand Up @@ -61,12 +62,13 @@ type LRUPlanCache struct {
guard float64

memoryUsageTotal int64
sctx sessionctx.Context
}

// NewLRUPlanCache creates a PCLRUCache object, whose capacity is "capacity".
// NOTE: "capacity" should be a positive value.
func NewLRUPlanCache(capacity uint, guard float64, quota uint64,
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType) (*list.Element, bool)) *LRUPlanCache {
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType) (*list.Element, bool), sctx sessionctx.Context) *LRUPlanCache {
if capacity < 1 {
capacity = 100
logutil.BgLogger().Info("capacity of LRU cache is less than 1, will use default value(100) init cache")
Expand All @@ -79,6 +81,7 @@ func NewLRUPlanCache(capacity uint, guard float64, quota uint64,
pickFromBucket: pickFromBucket,
quota: quota,
guard: guard,
sctx: sctx,
}
}

Expand Down Expand Up @@ -203,7 +206,9 @@ func (l *LRUPlanCache) Close() {
if l == nil {
return
}
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(l.memoryUsageTotal))
if l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(l.memoryUsageTotal))
}
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Sub(float64(l.size))
}

Expand Down Expand Up @@ -259,7 +264,8 @@ func PickPlanFromBucket(bucket map[*list.Element]struct{}, paramTypes []*types.F

// updateInstanceMetric update the memory usage and plan num for show in grafana
func (l *LRUPlanCache) updateInstanceMetric(in, out *planCacheEntry) {
if l == nil {
updateInstancePlanNum(in, out)
if l == nil || !l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
return
}

Expand All @@ -270,10 +276,19 @@ func (l *LRUPlanCache) updateInstanceMetric(in, out *planCacheEntry) {
} else if in != nil { // put plan
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Add(float64(in.MemoryUsage()))
l.memoryUsageTotal += in.MemoryUsage()
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Add(1)
} else { // delete plan
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(out.MemoryUsage()))
l.memoryUsageTotal -= out.MemoryUsage()
}
}

// updateInstancePlanNum update the plan num
func updateInstancePlanNum(in, out *planCacheEntry) {
if in != nil && out != nil { // replace plan
return
} else if in != nil { // put plan
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Add(1)
} else { // delete plan
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Sub(1)
}
}
20 changes: 11 additions & 9 deletions planner/core/plan_cache_lru_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,11 @@ func randomPlanCacheValue(types []*types.FieldType) *PlanCacheValue {

func TestLRUPCPut(t *testing.T) {
// test initialize
lruA := NewLRUPlanCache(0, 0, 0, PickPlanFromBucket)
lruA := NewLRUPlanCache(0, 0, 0, PickPlanFromBucket, MockContext())
require.Equal(t, lruA.capacity, uint(100))

maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, MockContext())
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
maxMemDroppedKv[key] = value
}
Expand Down Expand Up @@ -121,7 +121,7 @@ func TestLRUPCPut(t *testing.T) {
}

func TestLRUPCGet(t *testing.T) {
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, MockContext())

keys := make([]*planCacheKey, 5)
vals := make([]*PlanCacheValue, 5)
Expand Down Expand Up @@ -167,7 +167,7 @@ func TestLRUPCGet(t *testing.T) {
}

func TestLRUPCDelete(t *testing.T) {
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, MockContext())

keys := make([]*planCacheKey, 3)
vals := make([]*PlanCacheValue, 3)
Expand Down Expand Up @@ -196,7 +196,7 @@ func TestLRUPCDelete(t *testing.T) {
}

func TestLRUPCDeleteAll(t *testing.T) {
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, MockContext())

keys := make([]*planCacheKey, 3)
vals := make([]*PlanCacheValue, 3)
Expand All @@ -223,7 +223,7 @@ func TestLRUPCDeleteAll(t *testing.T) {

func TestLRUPCSetCapacity(t *testing.T) {
maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
lru := NewLRUPlanCache(5, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(5, 0, 0, PickPlanFromBucket, MockContext())
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
maxMemDroppedKv[key] = value
}
Expand Down Expand Up @@ -285,7 +285,7 @@ func TestLRUPCSetCapacity(t *testing.T) {
}

func TestIssue37914(t *testing.T) {
lru := NewLRUPlanCache(3, 0.1, 1, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0.1, 1, PickPlanFromBucket, MockContext())

pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
key := &planCacheKey{database: strconv.FormatInt(int64(1), 10)}
Expand All @@ -297,7 +297,7 @@ func TestIssue37914(t *testing.T) {
}

func TestIssue38244(t *testing.T) {
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, MockContext())
require.Equal(t, uint(3), lru.capacity)

keys := make([]*planCacheKey, 5)
Expand All @@ -322,7 +322,9 @@ func TestIssue38244(t *testing.T) {

func TestLRUPlanCacheMemoryUsage(t *testing.T) {
pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket)
ctx := MockContext()
ctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor = true
lru := NewLRUPlanCache(3, 0, 0, PickPlanFromBucket, ctx)
evict := make(map[kvcache.Key]kvcache.Value)
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
evict[key] = value
Expand Down
29 changes: 28 additions & 1 deletion planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ func TestInitLRUWithSystemVar(t *testing.T) {
tk.MustQuery("select @@session.tidb_prepared_plan_cache_size").Check(testkit.Rows("1"))
sessionVar := tk.Session().GetSessionVars()

lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, plannercore.PickPlanFromBucket)
lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, plannercore.PickPlanFromBucket, tk.Session())
require.NotNil(t, lru)
}

Expand Down Expand Up @@ -149,3 +149,30 @@ func TestIssue38533(t *testing.T) {
tk.MustExec(`execute st using @a, @a`)
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
}

func TestIssue38710(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists UK_NO_PRECISION_19392;")
tk.MustExec("CREATE TABLE `UK_NO_PRECISION_19392` (\n `COL1` bit(1) DEFAULT NULL,\n `COL2` varchar(20) COLLATE utf8mb4_bin DEFAULT NULL,\n `COL4` datetime DEFAULT NULL,\n `COL3` bigint DEFAULT NULL,\n `COL5` float DEFAULT NULL,\n UNIQUE KEY `UK_COL1` (`COL1`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;")
tk.MustExec("INSERT INTO `UK_NO_PRECISION_19392` VALUES (0x00,'缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈','9294-12-26 06:50:40',-3088380202191555887,-3.33294e38),(NULL,'仲膩蕦圓猴洠飌镂喵疎偌嫺荂踖Ƕ藨蜿諪軁笞','1746-08-30 18:04:04',-4016793239832666288,-2.52633e38),(0x01,'冑溜畁脊乤纊繳蟥哅稐奺躁悼貘飗昹槐速玃沮','1272-01-19 23:03:27',-8014797887128775012,1.48868e38);\n")
tk.MustExec(`prepare stmt from 'select * from UK_NO_PRECISION_19392 where col1 between ? and ? or col3 = ? or col2 in (?, ?, ?);';`)
tk.MustExec("set @a=0x01, @b=0x01, @c=-3088380202191555887, @d=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\", @e=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\", @f=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\";")
rows := tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`) // can not be cached because @a = @b
require.Equal(t, 2, len(rows.Rows()))

tk.MustExec(`set @a=NULL, @b=NULL, @c=-4016793239832666288, @d="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @e="仲膩蕦圓猴洠飌镂喵疎偌嫺荂踖Ƕ藨蜿諪軁笞", @f="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈";`)
rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))

rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustExec(`set @a=0x01, @b=0x01, @c=-3088380202191555887, @d="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @e="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @f="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈";`)
rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // can not use the cache because the types for @a and @b are not equal to the cached plan
}
4 changes: 1 addition & 3 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,9 +322,7 @@ func (s FieldSlice) CheckTypesCompatibility4PC(tps []*types.FieldType) bool {
// string types will show up here, and (2) we don't need flen and decimal to be matched exactly to use plan cache
tpEqual := (s[i].GetType() == tps[i].GetType()) ||
(s[i].GetType() == mysql.TypeVarchar && tps[i].GetType() == mysql.TypeVarString) ||
(s[i].GetType() == mysql.TypeVarString && tps[i].GetType() == mysql.TypeVarchar) ||
// TypeNull should be considered the same as other types.
(s[i].GetType() == mysql.TypeNull || tps[i].GetType() == mysql.TypeNull)
(s[i].GetType() == mysql.TypeVarString && tps[i].GetType() == mysql.TypeVarchar)
if !tpEqual || s[i].GetCharset() != tps[i].GetCharset() || s[i].GetCollate() != tps[i].GetCollate() ||
(s[i].EvalType() == types.ETInt && mysql.HasUnsignedFlag(s[i].GetFlag()) != mysql.HasUnsignedFlag(tps[i].GetFlag())) {
return false
Expand Down
2 changes: 1 addition & 1 deletion planner/core/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1339,7 +1339,7 @@ func TestPlanCacheSwitchDB(t *testing.T) {

// DB is not specified
se2, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: core.NewLRUPlanCache(100, 0.1, math.MaxUint64, core.PickPlanFromBucket),
PreparedPlanCache: core.NewLRUPlanCache(100, 0.1, math.MaxUint64, core.PickPlanFromBucket, tk.Session()),
})
require.NoError(t, err)
tk2 := testkit.NewTestKitWithSession(t, store, se2)
Expand Down
Loading

0 comments on commit 6261687

Please sign in to comment.