diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index fdf58bc95b..9cbacdbe6c 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -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) diff --git a/expression/integration_serial_test/integration_serial_test.go b/expression/integration_serial_test/integration_serial_test.go index 213d4b4660..fe92887b10 100644 --- a/expression/integration_serial_test/integration_serial_test.go +++ b/expression/integration_serial_test/integration_serial_test.go @@ -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) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index fbd554fbc8..15ba3f1d5c 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -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() diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 8c19fe9a56..97da7c8e3a 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -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 } diff --git a/planner/core/metrics/metrics.go b/planner/core/metrics/metrics.go index 1e8e108d5f..461d319321 100644 --- a/planner/core/metrics/metrics.go +++ b/planner/core/metrics/metrics.go @@ -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 } diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index f416edbd13..69d3ecbbef 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -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 diff --git a/planner/core/plan_cache_lru.go b/planner/core/plan_cache_lru.go index ad07c174dc..298e9abdf3 100644 --- a/planner/core/plan_cache_lru.go +++ b/planner/core/plan_cache_lru.go @@ -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) } } diff --git a/planner/core/plan_cache_lru_test.go b/planner/core/plan_cache_lru_test.go index f6ed09a8e6..518252293a 100644 --- a/planner/core/plan_cache_lru_test.go +++ b/planner/core/plan_cache_lru_test.go @@ -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 diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 03352132e0..60ce400140 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -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) } diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index a4050951d8..fc35cf3ec2 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -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) diff --git a/session/session.go b/session/session.go index b71cc70316..6faeeb0d63 100644 --- a/session/session.go +++ b/session/session.go @@ -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. diff --git a/util/stmtsummary/v2/tests/table_test.go b/util/stmtsummary/v2/tests/table_test.go index 830624a3f7..9c47cbcd59 100644 --- a/util/stmtsummary/v2/tests/table_test.go +++ b/util/stmtsummary/v2/tests/table_test.go @@ -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)