Skip to content

Commit

Permalink
statscache: update config and metrics for stats cache (#45581)
Browse files Browse the repository at this point in the history
close #45367
  • Loading branch information
hawkingrei authored Jul 26, 2023
1 parent 7e18f79 commit 93a880a
Show file tree
Hide file tree
Showing 20 changed files with 1,528 additions and 2,117 deletions.
3 changes: 2 additions & 1 deletion domain/domain_sysvars.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@ import (
// because the domain is not available. Instead a noop func is specified,
// which is overwritten here.
func (do *Domain) initDomainSysVars() {
variable.SetStatsCacheCapacity.Store(do.setStatsCacheCapacity)
setStatsCacheCapacityFunc := do.setStatsCacheCapacity
variable.SetStatsCacheCapacity.Store(&setStatsCacheCapacityFunc)
pdClientDynamicOptionFunc := do.setPDClientDynamicOption
variable.SetPDClientDynamicOption.Store(&pdClientDynamicOptionFunc)

Expand Down
14 changes: 7 additions & 7 deletions metrics/grafana/tidb.json
Original file line number Diff line number Diff line change
Expand Up @@ -14768,7 +14768,7 @@
"dashLength": 10,
"dashes": false,
"datasource": "${DS_TEST-CLUSTER}",
"description": "TiDB managing stats cache by lru",
"description": "TiDB managing stats cache",
"editable": true,
"error": false,
"fieldConfig": {
Expand Down Expand Up @@ -14819,7 +14819,7 @@
"targets": [
{
"exemplar": true,
"expr": "tidb_statistics_stats_cache_lru_val{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"track\"}",
"expr": "tidb_statistics_stats_cache_val{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"track\"}",
"format": "time_series",
"interval": "",
"intervalFactor": 1,
Expand All @@ -14828,7 +14828,7 @@
},
{
"exemplar": true,
"expr": "tidb_statistics_stats_cache_lru_val{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"capacity\"}",
"expr": "tidb_statistics_stats_cache_val{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"capacity\"}",
"hide": true,
"interval": "",
"legendFormat": "capacity--{{instance}}",
Expand All @@ -14839,7 +14839,7 @@
"timeFrom": null,
"timeRegions": [],
"timeShift": null,
"title": "Stats Cache LRU Cost",
"title": "Stats Cache Cost",
"tooltip": {
"msResolution": true,
"shared": true,
Expand Down Expand Up @@ -14883,7 +14883,7 @@
"dashLength": 10,
"dashes": false,
"datasource": "${DS_TEST-CLUSTER}",
"description": "TiDB managing stats cache by lru",
"description": "TiDB managing stats cache",
"fieldConfig": {
"defaults": {},
"overrides": []
Expand Down Expand Up @@ -14928,7 +14928,7 @@
"targets": [
{
"exemplar": true,
"expr": "sum(rate(tidb_statistics_stats_cache_lru_op{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)",
"expr": "sum(rate(tidb_statistics_stats_cache_op{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)",
"format": "time_series",
"interval": "",
"intervalFactor": 2,
Expand All @@ -14941,7 +14941,7 @@
"timeFrom": null,
"timeRegions": [],
"timeShift": null,
"title": "Stats Cache LRU OPS",
"title": "Stats Cache OPS",
"tooltip": {
"shared": true,
"sort": 0,
Expand Down
4 changes: 2 additions & 2 deletions metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -232,8 +232,8 @@ func RegisterMetrics() {
prometheus.MustRegister(PessimisticDMLDurationByAttempt)
prometheus.MustRegister(ResourceGroupQueryTotalCounter)
prometheus.MustRegister(MemoryUsage)
prometheus.MustRegister(StatsCacheLRUCounter)
prometheus.MustRegister(StatsCacheLRUGauge)
prometheus.MustRegister(StatsCacheCounter)
prometheus.MustRegister(StatsCacheGauge)
prometheus.MustRegister(StatsHealthyGauge)
prometheus.MustRegister(TxnStatusEnteringCounter)
prometheus.MustRegister(TxnDurationHistogram)
Expand Down
16 changes: 8 additions & 8 deletions metrics/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@ var (
SyncLoadTimeoutCounter prometheus.Counter
SyncLoadHistogram prometheus.Histogram
ReadStatsHistogram prometheus.Histogram
StatsCacheLRUCounter *prometheus.CounterVec
StatsCacheLRUGauge *prometheus.GaugeVec
StatsCacheCounter *prometheus.CounterVec
StatsCacheGauge *prometheus.GaugeVec
StatsHealthyGauge *prometheus.GaugeVec

HistoricalStatsCounter *prometheus.CounterVec
Expand Down Expand Up @@ -153,19 +153,19 @@ func InitStatsMetrics() {
Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h
})

StatsCacheLRUCounter = NewCounterVec(
StatsCacheCounter = NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "statistics",
Name: "stats_cache_lru_op",
Help: "Counter of lru for statsCache operation",
Name: "stats_cache_op",
Help: "Counter for statsCache operation",
}, []string{LblType})

StatsCacheLRUGauge = NewGaugeVec(prometheus.GaugeOpts{
StatsCacheGauge = NewGaugeVec(prometheus.GaugeOpts{
Namespace: "tidb",
Subsystem: "statistics",
Name: "stats_cache_lru_val",
Help: "gauge of stats cache lru value",
Name: "stats_cache_val",
Help: "gauge of stats cache value",
}, []string{LblType})

StatsHealthyGauge = NewGaugeVec(prometheus.GaugeOpts{
Expand Down
3 changes: 2 additions & 1 deletion sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1088,7 +1088,8 @@ var defaultSysVars = []*SysVar{
oldv := StatsCacheMemQuota.Load()
if v != oldv {
StatsCacheMemQuota.Store(v)
SetStatsCacheCapacity.Load().(func(int64))(v)
SetStatsCacheCapacityFunc := SetStatsCacheCapacity.Load()
(*SetStatsCacheCapacityFunc)(v)
}
return nil
},
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1490,7 +1490,7 @@ var (
// GetMemQuotaAnalyze is the func registered by global/subglobal tracker to get memory quota.
GetMemQuotaAnalyze func() int64 = nil
// SetStatsCacheCapacity is the func registered by domain to set statsCache memory quota.
SetStatsCacheCapacity atomic.Value
SetStatsCacheCapacity atomic.Pointer[func(int64)]
// SetPDClientDynamicOption is the func registered by domain
SetPDClientDynamicOption atomic.Pointer[func(string, string)]
// SwitchMDL is the func registered by DDL to switch MDL.
Expand Down
1 change: 1 addition & 0 deletions statistics/handle/cache/internal/lfu/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//statistics/handle/cache/internal",
"//statistics/handle/cache/internal/metrics",
"//util/intest",
"//util/mathutil",
"//util/memory",
"@com_github_dgraph_io_ristretto//:ristretto",
"@org_golang_x_exp//maps",
Expand Down
4 changes: 3 additions & 1 deletion statistics/handle/cache/internal/lfu/lfu_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/statistics/handle/cache/internal"
"github.com/pingcap/tidb/statistics/handle/cache/internal/metrics"
"github.com/pingcap/tidb/util/intest"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
)

Expand All @@ -47,8 +48,9 @@ func NewLFU(totalMemCost int64) (*LFU, error) {
if intest.InTest {
bufferItems = 1
}

cache, err := ristretto.NewCache(&ristretto.Config{
NumCounters: 1024 * 1024 * 1024, // TODO(hawkingrei): make it configurable for NumCounters and MaxCost
NumCounters: mathutil.Max(totalMemCost/128*2, 10), // assume the cost per table stats is 128
MaxCost: totalMemCost,
BufferItems: bufferItems,
OnEvict: result.onEvict,
Expand Down
27 changes: 0 additions & 27 deletions statistics/handle/cache/internal/lru/BUILD.bazel

This file was deleted.

Loading

0 comments on commit 93a880a

Please sign in to comment.