planner: add monitor for non-prepared plan cache (#41752)

ref pingcap/tidb#36598
This commit is contained in:
fzzf678
2023-03-07 19:41:12 +08:00
committed by GitHub
parent 29adb0aa86
commit 18f1ecdfcd
12 changed files with 100 additions and 50 deletions

View File

@ -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)

View File

@ -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)

View File

@ -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()

View File

@ -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
}

View File

@ -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
}

View File

@ -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

View File

@ -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)
}
}

View File

@ -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

View File

@ -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)
}

View File

@ -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)

View File

@ -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.

View File

@ -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)