planner: add monitor for non-prepared plan cache (#41752)
ref pingcap/tidb#36598
This commit is contained in:
@ -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, tk.Session()),
|
||||
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
tk.SetSession(se)
|
||||
@ -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, tk.Session()),
|
||||
PreparedPlanCache: plannercore.NewLRUPlanCache(3, 0.1, math.MaxUint64, tk.Session(), false),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
tk.SetSession(se)
|
||||
|
||||
@ -3790,7 +3790,7 @@ func TestPreparePlanCacheOnCachedTable(t *testing.T) {
|
||||
|
||||
var err error
|
||||
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
|
||||
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session()),
|
||||
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
tk.SetSession(se)
|
||||
|
||||
@ -56,7 +56,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, tk.Session())})
|
||||
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false)})
|
||||
require.NoError(t, err)
|
||||
tk.SetSession(se)
|
||||
tk.RefreshConnectionID()
|
||||
|
||||
@ -7839,7 +7839,7 @@
|
||||
"expr": "tidb_server_plan_cache_instance_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}",
|
||||
"hide": false,
|
||||
"interval": "",
|
||||
"legendFormat": "{{instance}}",
|
||||
"legendFormat": "{{instance}}{{type}}",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
@ -7945,11 +7945,11 @@
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "tidb_server_plan_cache_instance_plan_num_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}\n",
|
||||
"expr": "tidb_server_plan_cache_instance_plan_num_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}",
|
||||
"format": "time_series",
|
||||
"interval": "",
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{instance}}",
|
||||
"legendFormat": "{{instance}}{{type}}",
|
||||
"refId": "A",
|
||||
"step": 30
|
||||
}
|
||||
|
||||
@ -21,11 +21,16 @@ import (
|
||||
|
||||
// planner core metrics vars
|
||||
var (
|
||||
PlanCacheCounter prometheus.Counter
|
||||
PlanCacheMissCounter prometheus.Counter
|
||||
|
||||
PseudoEstimationNotAvailable prometheus.Counter
|
||||
PseudoEstimationOutdate prometheus.Counter
|
||||
PseudoEstimationNotAvailable prometheus.Counter
|
||||
PseudoEstimationOutdate prometheus.Counter
|
||||
preparedPlanCacheHitCounter prometheus.Counter
|
||||
nonPreparedPlanCacheHitCounter prometheus.Counter
|
||||
preparedPlanCacheMissCounter prometheus.Counter
|
||||
nonPreparedPlanCacheMissCounter prometheus.Counter
|
||||
preparedPlanCacheInstancePlanNumCounter prometheus.Gauge
|
||||
nonPreparedPlanCacheInstancePlanNumCounter prometheus.Gauge
|
||||
preparedPlanCacheInstanceMemoryUsage prometheus.Gauge
|
||||
nonPreparedPlanCacheInstanceMemoryUsage prometheus.Gauge
|
||||
)
|
||||
|
||||
func init() {
|
||||
@ -34,8 +39,47 @@ func init() {
|
||||
|
||||
// InitMetricsVars init planner core metrics vars.
|
||||
func InitMetricsVars() {
|
||||
PlanCacheCounter = metrics.PlanCacheCounter.WithLabelValues("prepare")
|
||||
PlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("cache_miss")
|
||||
PseudoEstimationNotAvailable = metrics.PseudoEstimation.WithLabelValues("nodata")
|
||||
PseudoEstimationOutdate = metrics.PseudoEstimation.WithLabelValues("outdate")
|
||||
// plan cache metrics
|
||||
preparedPlanCacheHitCounter = metrics.PlanCacheCounter.WithLabelValues("prepared")
|
||||
nonPreparedPlanCacheHitCounter = metrics.PlanCacheCounter.WithLabelValues("non-prepared")
|
||||
preparedPlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("prepared")
|
||||
nonPreparedPlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("non-prepared")
|
||||
preparedPlanCacheInstancePlanNumCounter = metrics.PlanCacheInstancePlanNumCounter.WithLabelValues(" prepared")
|
||||
nonPreparedPlanCacheInstancePlanNumCounter = metrics.PlanCacheInstancePlanNumCounter.WithLabelValues(" non-prepared")
|
||||
preparedPlanCacheInstanceMemoryUsage = metrics.PlanCacheInstanceMemoryUsage.WithLabelValues(" prepared")
|
||||
nonPreparedPlanCacheInstanceMemoryUsage = metrics.PlanCacheInstanceMemoryUsage.WithLabelValues(" non-prepared")
|
||||
}
|
||||
|
||||
// GetPlanCacheHitCounter get different plan cache hit counter
|
||||
func GetPlanCacheHitCounter(isNonPrepared bool) prometheus.Counter {
|
||||
if isNonPrepared {
|
||||
return nonPreparedPlanCacheHitCounter
|
||||
}
|
||||
return preparedPlanCacheHitCounter
|
||||
}
|
||||
|
||||
// GetPlanCacheMissCounter get different plan cache miss counter
|
||||
func GetPlanCacheMissCounter(isNonPrepared bool) prometheus.Counter {
|
||||
if isNonPrepared {
|
||||
return nonPreparedPlanCacheMissCounter
|
||||
}
|
||||
return preparedPlanCacheMissCounter
|
||||
}
|
||||
|
||||
// GetPlanCacheInstanceNumCounter get different plan counter of plan cache
|
||||
func GetPlanCacheInstanceNumCounter(isNonPrepared bool) prometheus.Gauge {
|
||||
if isNonPrepared {
|
||||
return nonPreparedPlanCacheInstancePlanNumCounter
|
||||
}
|
||||
return preparedPlanCacheInstancePlanNumCounter
|
||||
}
|
||||
|
||||
// GetPlanCacheInstanceMemoryUsage get different plan memory usage counter of plan cache
|
||||
func GetPlanCacheInstanceMemoryUsage(isNonPrepared bool) prometheus.Gauge {
|
||||
if isNonPrepared {
|
||||
return nonPreparedPlanCacheInstanceMemoryUsage
|
||||
}
|
||||
return preparedPlanCacheInstanceMemoryUsage
|
||||
}
|
||||
|
||||
@ -207,7 +207,8 @@ func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmt
|
||||
if metrics.ResettablePlanCacheCounterFortTest {
|
||||
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
|
||||
} else {
|
||||
core_metrics.PlanCacheCounter.Inc()
|
||||
// only for prepared plan cache
|
||||
core_metrics.GetPlanCacheHitCounter(false).Inc()
|
||||
}
|
||||
sessVars.FoundInPlanCache = true
|
||||
stmtCtx.PointExec = true
|
||||
@ -248,7 +249,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache
|
||||
if metrics.ResettablePlanCacheCounterFortTest {
|
||||
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
|
||||
} else {
|
||||
core_metrics.PlanCacheCounter.Inc()
|
||||
core_metrics.GetPlanCacheHitCounter(isNonPrepared).Inc()
|
||||
}
|
||||
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
|
||||
return cachedVal.Plan, cachedVal.OutPutNames, true, nil
|
||||
@ -263,7 +264,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
|
||||
sessVars := sctx.GetSessionVars()
|
||||
stmtCtx := sessVars.StmtCtx
|
||||
|
||||
core_metrics.PlanCacheMissCounter.Inc()
|
||||
core_metrics.GetPlanCacheMissCounter(isNonPrepared).Inc()
|
||||
sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = true
|
||||
p, names, err := OptimizeAstNode(ctx, sctx, stmtAst.Stmt, is)
|
||||
sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = false
|
||||
|
||||
@ -18,7 +18,7 @@ import (
|
||||
"sync"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/tidb/metrics"
|
||||
core_metrics "github.com/pingcap/tidb/planner/core/metrics"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
"github.com/pingcap/tidb/util/hack"
|
||||
"github.com/pingcap/tidb/util/kvcache"
|
||||
@ -60,23 +60,25 @@ type LRUPlanCache struct {
|
||||
|
||||
memoryUsageTotal int64
|
||||
sctx sessionctx.Context
|
||||
isNonPrepared bool
|
||||
}
|
||||
|
||||
// NewLRUPlanCache creates a PCLRUCache object, whose capacity is "capacity".
|
||||
// NOTE: "capacity" should be a positive value.
|
||||
func NewLRUPlanCache(capacity uint, guard float64, quota uint64, sctx sessionctx.Context) *LRUPlanCache {
|
||||
func NewLRUPlanCache(capacity uint, guard float64, quota uint64, sctx sessionctx.Context, isNonPrepared bool) *LRUPlanCache {
|
||||
if capacity < 1 {
|
||||
capacity = 100
|
||||
logutil.BgLogger().Info("capacity of LRU cache is less than 1, will use default value(100) init cache")
|
||||
}
|
||||
return &LRUPlanCache{
|
||||
capacity: capacity,
|
||||
size: 0,
|
||||
buckets: make(map[string]map[*list.Element]struct{}, 1), //Generally one query has one plan
|
||||
lruList: list.New(),
|
||||
quota: quota,
|
||||
guard: guard,
|
||||
sctx: sctx,
|
||||
capacity: capacity,
|
||||
size: 0,
|
||||
buckets: make(map[string]map[*list.Element]struct{}, 1), //Generally one query has one plan
|
||||
lruList: list.New(),
|
||||
quota: quota,
|
||||
guard: guard,
|
||||
sctx: sctx,
|
||||
isNonPrepared: isNonPrepared,
|
||||
}
|
||||
}
|
||||
|
||||
@ -202,9 +204,9 @@ func (l *LRUPlanCache) Close() {
|
||||
return
|
||||
}
|
||||
if l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
|
||||
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(l.memoryUsageTotal))
|
||||
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(l.memoryUsageTotal))
|
||||
}
|
||||
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Sub(float64(l.size))
|
||||
core_metrics.GetPlanCacheInstanceNumCounter(l.isNonPrepared).Sub(float64(l.size))
|
||||
}
|
||||
|
||||
// removeOldest removes the oldest element from the cache.
|
||||
@ -287,31 +289,31 @@ func checkUint64SliceIfEqual(a, b []uint64) bool {
|
||||
|
||||
// updateInstanceMetric update the memory usage and plan num for show in grafana
|
||||
func (l *LRUPlanCache) updateInstanceMetric(in, out *planCacheEntry) {
|
||||
updateInstancePlanNum(in, out)
|
||||
updateInstancePlanNum(in, out, l.isNonPrepared)
|
||||
if l == nil || !l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
|
||||
return
|
||||
}
|
||||
|
||||
if in != nil && out != nil { // replace plan
|
||||
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(out.MemoryUsage()))
|
||||
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Add(float64(in.MemoryUsage()))
|
||||
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(out.MemoryUsage()))
|
||||
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Add(float64(in.MemoryUsage()))
|
||||
l.memoryUsageTotal += in.MemoryUsage() - out.MemoryUsage()
|
||||
} else if in != nil { // put plan
|
||||
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Add(float64(in.MemoryUsage()))
|
||||
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Add(float64(in.MemoryUsage()))
|
||||
l.memoryUsageTotal += in.MemoryUsage()
|
||||
} else { // delete plan
|
||||
metrics.PlanCacheInstanceMemoryUsage.WithLabelValues("instance").Sub(float64(out.MemoryUsage()))
|
||||
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(out.MemoryUsage()))
|
||||
l.memoryUsageTotal -= out.MemoryUsage()
|
||||
}
|
||||
}
|
||||
|
||||
// updateInstancePlanNum update the plan num
|
||||
func updateInstancePlanNum(in, out *planCacheEntry) {
|
||||
func updateInstancePlanNum(in, out *planCacheEntry, isNonPrepared bool) {
|
||||
if in != nil && out != nil { // replace plan
|
||||
return
|
||||
} else if in != nil { // put plan
|
||||
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Add(1)
|
||||
core_metrics.GetPlanCacheInstanceNumCounter(isNonPrepared).Add(1)
|
||||
} else { // delete plan
|
||||
metrics.PlanCacheInstancePlanNumCounter.WithLabelValues("plan_num").Sub(1)
|
||||
core_metrics.GetPlanCacheInstanceNumCounter(isNonPrepared).Sub(1)
|
||||
}
|
||||
}
|
||||
|
||||
@ -50,11 +50,11 @@ func TestLRUPCPut(t *testing.T) {
|
||||
// test initialize
|
||||
mockCtx := MockContext()
|
||||
mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true
|
||||
lruA := NewLRUPlanCache(0, 0, 0, mockCtx)
|
||||
lruA := NewLRUPlanCache(0, 0, 0, mockCtx, false)
|
||||
require.Equal(t, lruA.capacity, uint(100))
|
||||
|
||||
maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx)
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx, false)
|
||||
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
|
||||
maxMemDroppedKv[key] = value
|
||||
}
|
||||
@ -137,7 +137,7 @@ func TestLRUPCPut(t *testing.T) {
|
||||
func TestLRUPCGet(t *testing.T) {
|
||||
mockCtx := MockContext()
|
||||
mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx)
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx, false)
|
||||
|
||||
keys := make([]*planCacheKey, 5)
|
||||
vals := make([]*PlanCacheValue, 5)
|
||||
@ -202,7 +202,7 @@ func TestLRUPCGet(t *testing.T) {
|
||||
func TestLRUPCDelete(t *testing.T) {
|
||||
mockCtx := MockContext()
|
||||
mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx)
|
||||
lru := NewLRUPlanCache(3, 0, 0, mockCtx, false)
|
||||
|
||||
keys := make([]*planCacheKey, 3)
|
||||
vals := make([]*PlanCacheValue, 3)
|
||||
@ -250,7 +250,7 @@ func TestLRUPCDelete(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestLRUPCDeleteAll(t *testing.T) {
|
||||
lru := NewLRUPlanCache(3, 0, 0, MockContext())
|
||||
lru := NewLRUPlanCache(3, 0, 0, MockContext(), false)
|
||||
|
||||
keys := make([]*planCacheKey, 3)
|
||||
vals := make([]*PlanCacheValue, 3)
|
||||
@ -287,7 +287,7 @@ func TestLRUPCDeleteAll(t *testing.T) {
|
||||
|
||||
func TestLRUPCSetCapacity(t *testing.T) {
|
||||
maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
|
||||
lru := NewLRUPlanCache(5, 0, 0, MockContext())
|
||||
lru := NewLRUPlanCache(5, 0, 0, MockContext(), false)
|
||||
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
|
||||
maxMemDroppedKv[key] = value
|
||||
}
|
||||
@ -355,7 +355,7 @@ func TestLRUPCSetCapacity(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestIssue37914(t *testing.T) {
|
||||
lru := NewLRUPlanCache(3, 0.1, 1, MockContext())
|
||||
lru := NewLRUPlanCache(3, 0.1, 1, MockContext(), false)
|
||||
|
||||
pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
|
||||
key := &planCacheKey{database: strconv.FormatInt(int64(1), 10)}
|
||||
@ -371,7 +371,7 @@ func TestIssue37914(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestIssue38244(t *testing.T) {
|
||||
lru := NewLRUPlanCache(3, 0, 0, MockContext())
|
||||
lru := NewLRUPlanCache(3, 0, 0, MockContext(), false)
|
||||
require.Equal(t, uint(3), lru.capacity)
|
||||
|
||||
keys := make([]*planCacheKey, 5)
|
||||
@ -402,7 +402,7 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) {
|
||||
pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
|
||||
ctx := MockContext()
|
||||
ctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor = true
|
||||
lru := NewLRUPlanCache(3, 0, 0, ctx)
|
||||
lru := NewLRUPlanCache(3, 0, 0, ctx, false)
|
||||
evict := make(map[kvcache.Key]kvcache.Value)
|
||||
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
|
||||
evict[key] = value
|
||||
|
||||
@ -79,7 +79,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, tk.Session())
|
||||
lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, tk.Session(), false)
|
||||
require.NotNil(t, lru)
|
||||
}
|
||||
|
||||
|
||||
@ -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, tk.Session()),
|
||||
PreparedPlanCache: core.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
tk2 := testkit.NewTestKitWithSession(t, store, se2)
|
||||
|
||||
@ -421,7 +421,7 @@ func (s *session) GetPlanCache(isNonPrepared bool) sessionctx.PlanCache {
|
||||
}
|
||||
if s.nonPreparedPlanCache == nil { // lazy construction
|
||||
s.nonPreparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().NonPreparedPlanCacheSize),
|
||||
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s)
|
||||
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, true)
|
||||
}
|
||||
return s.nonPreparedPlanCache
|
||||
}
|
||||
@ -432,7 +432,7 @@ func (s *session) GetPlanCache(isNonPrepared bool) sessionctx.PlanCache {
|
||||
}
|
||||
if s.preparedPlanCache == nil { // lazy construction
|
||||
s.preparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().PreparedPlanCacheSize),
|
||||
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s)
|
||||
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, false)
|
||||
}
|
||||
return s.preparedPlanCache
|
||||
}
|
||||
@ -2568,6 +2568,9 @@ func (s *session) Close() {
|
||||
if s.preparedPlanCache != nil {
|
||||
s.preparedPlanCache.Close()
|
||||
}
|
||||
if s.nonPreparedPlanCache != nil {
|
||||
s.nonPreparedPlanCache.Close()
|
||||
}
|
||||
}
|
||||
|
||||
// GetSessionVars implements the context.Context interface.
|
||||
|
||||
@ -539,7 +539,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, tk.Session()),
|
||||
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
tk.SetSession(se)
|
||||
|
||||
Reference in New Issue
Block a user