From fd0b6da2b4fb57475783b03dd0cc34944ff2a9c5 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 27 Jul 2023 12:27:19 +0800 Subject: [PATCH 1/4] statscache: improve stats cache benchmark Signed-off-by: Weizhen Wang --- statistics/handle/bootstrap.go | 14 +-- statistics/handle/cache/bench_test.go | 59 +++++++++- statistics/handle/cache/internal/BUILD.bazel | 16 +-- .../handle/cache/internal/bench_test.go | 101 ------------------ statistics/handle/cache/internal/inner.go | 2 +- .../handle/cache/internal/lfu/lfu_cache.go | 7 +- .../cache/internal/lfu/lfu_cache_test.go | 28 ++--- .../cache/internal/mapcache/map_cache.go | 2 +- statistics/handle/cache/statscache.go | 11 ++ statistics/handle/cache/statscacheinner.go | 31 ++---- statistics/handle/handle.go | 9 +- 11 files changed, 102 insertions(+), 178 deletions(-) delete mode 100644 statistics/handle/cache/internal/bench_test.go diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 5fdbc888c3f68..77fa58c0fe053 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -58,7 +58,7 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache *cache.Stat Version: row.GetUint64(0), Name: getFullTableName(is, tableInfo), } - cache.PutFromInternal(physicalID, tbl) // put this table again since it is updated + cache.Put(physicalID, tbl) // put this table again since it is updated } } @@ -156,7 +156,7 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache * } table.Columns[hist.ID] = col } - cache.PutFromInternal(tblID, table) // put this table again since it is updated + cache.Put(tblID, table) // put this table again since it is updated } } @@ -225,7 +225,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache *cach lastAnalyzePos.Copy(&col.LastAnalyzePos) table.Columns[hist.ID] = col } - cache.PutFromInternal(tblID, table) // put this table again since it is updated + cache.Put(tblID, table) // put this table again since it is updated } } @@ -293,7 +293,7 @@ func (*Handle) initStatsTopN4Chunk(cache *cache.StatsCache, iter *chunk.Iterator data := make([]byte, len(row.GetBytes(2))) copy(data, row.GetBytes(2)) idx.TopN.AppendTopN(data, row.GetUint64(3)) - cache.PutFromInternal(table.PhysicalID, table) // put this table again since it is updated + cache.Put(table.PhysicalID, table) // put this table again since it is updated } for idx := range affectedIndexes { idx.TopN.Sort() @@ -346,7 +346,7 @@ func (*Handle) initStatsFMSketch4Chunk(cache *cache.StatsCache, iter *chunk.Iter colStats.FMSketch = fms } } - cache.PutFromInternal(table.PhysicalID, table) // put this table again since it is updated + cache.Put(table.PhysicalID, table) // put this table again since it is updated } } @@ -418,7 +418,7 @@ func (*Handle) initStatsBuckets4Chunk(cache *cache.StatsCache, iter *chunk.Itera } } hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(3), row.GetInt64(4), row.GetInt64(7)) - cache.PutFromInternal(tableID, table) // put this table again since it is updated + cache.Put(tableID, table) // put this table again since it is updated } } @@ -456,7 +456,7 @@ func (h *Handle) initStatsBuckets(cache *cache.StatsCache) error { } col.PreCalculateScalar() } - cache.PutFromInternal(table.PhysicalID, table) // put this table again since it is updated + cache.Put(table.PhysicalID, table) // put this table again since it is updated } return nil } diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 7514153cbb792..22f90da2a941d 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -34,15 +34,36 @@ func benchCopyAndUpdate(b *testing.B, c *StatsCachePointer) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) t1.PhysicalID = rand.Int63() - cache := c.Load() - c.Replace(cache.CopyAndUpdate([]*statistics.Table{t1}, nil)) + c.UpdateStatsCache(c.Load(), []*statistics.Table{t1}, nil) }() } wg.Wait() b.StopTimer() } -func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { +func benchPutGet(b *testing.B, c *StatsCachePointer) { + var wg sync.WaitGroup + b.ResetTimer() + for i := 0; i < b.N; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) + c.UpdateStatsCache(c.Load(), []*statistics.Table{t1}, nil) + }(i) + } + for i := 0; i < b.N; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.Load().GetFromUser(int64(i)) + }(i) + } + wg.Wait() + b.StopTimer() +} + +func BenchmarkStatsCacheLFUCopyAndUpdate(b *testing.B) { restore := config.RestoreFunc() defer restore() config.UpdateGlobal(func(conf *config.Config) { @@ -68,9 +89,37 @@ func BenchmarkStatsCacheMapCacheCopyAndUpdate(b *testing.B) { benchCopyAndUpdate(b, cache) } -func TestBenchDaily(t *testing.T) { +func BenchmarkLFUCachePutGet(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + cache, err := NewStatsCachePointer() + if err != nil { + b.Fail() + } + benchPutGet(b, cache) +} + +func BenchmarkMapCachePutGet(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = false + }) + cache, err := NewStatsCachePointer() + if err != nil { + b.Fail() + } + benchPutGet(b, cache) +} + +func TestBenchDaily(*testing.T) { benchdaily.Run( - BenchmarkStatsCacheLRUCopyAndUpdate, + BenchmarkStatsCacheLFUCopyAndUpdate, BenchmarkStatsCacheMapCacheCopyAndUpdate, + BenchmarkLFUCachePutGet, + BenchmarkMapCachePutGet, ) } diff --git a/statistics/handle/cache/internal/BUILD.bazel b/statistics/handle/cache/internal/BUILD.bazel index 340a0b45d8229..7cd17f787dcf1 100644 --- a/statistics/handle/cache/internal/BUILD.bazel +++ b/statistics/handle/cache/internal/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "cache", @@ -15,17 +15,3 @@ go_library( visibility = ["//statistics/handle/cache:__subpackages__"], deps = ["//statistics"], ) - -go_test( - name = "internal_test", - timeout = "short", - srcs = ["bench_test.go"], - flaky = True, - deps = [ - ":internal", - "//statistics/handle/cache/internal/lfu", - "//statistics/handle/cache/internal/mapcache", - "//statistics/handle/cache/internal/testutil", - "//util/benchdaily", - ], -) diff --git a/statistics/handle/cache/internal/bench_test.go b/statistics/handle/cache/internal/bench_test.go deleted file mode 100644 index 7a3943ee9cc39..0000000000000 --- a/statistics/handle/cache/internal/bench_test.go +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package internal_test - -import ( - "sync" - "testing" - - "github.com/pingcap/tidb/statistics/handle/cache/internal" - "github.com/pingcap/tidb/statistics/handle/cache/internal/lfu" - "github.com/pingcap/tidb/statistics/handle/cache/internal/mapcache" - "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" - "github.com/pingcap/tidb/util/benchdaily" -) - -const defaultSize int64 = 1000 - -var cases = []struct { - name string - newFunc func() internal.StatsCacheInner -}{ - { - name: "mapcache", - newFunc: func() internal.StatsCacheInner { - return mapcache.NewMapCache() - }, - }, - { - name: "LFU", - newFunc: func() internal.StatsCacheInner { - result, _ := lfu.NewLFU(defaultSize) - return result - }, - }, -} - -func BenchmarkCachePut(b *testing.B) { - for _, cs := range cases { - b.Run(cs.name, func(b *testing.B) { - var wg sync.WaitGroup - c := cs.newFunc() - b.ResetTimer() - for i := 0; i < b.N; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1, true) - }(i) - } - wg.Wait() - b.StopTimer() - }) - } -} - -func BenchmarkCachePutGet(b *testing.B) { - for _, cs := range cases { - b.Run(cs.name, func(b *testing.B) { - var wg sync.WaitGroup - c := cs.newFunc() - b.ResetTimer() - for i := 0; i < b.N; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1, true) - }(i) - } - for i := 0; i < b.N; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - c.Get(int64(i), true) - }(i) - } - wg.Wait() - b.StopTimer() - }) - } -} - -func TestBenchDaily(t *testing.T) { - benchdaily.Run( - BenchmarkCachePut, - BenchmarkCachePutGet, - ) -} diff --git a/statistics/handle/cache/internal/inner.go b/statistics/handle/cache/internal/inner.go index 9005025fc37cf..a4009f9ce3fca 100644 --- a/statistics/handle/cache/internal/inner.go +++ b/statistics/handle/cache/internal/inner.go @@ -26,7 +26,7 @@ type StatsCacheInner interface { // Get gets the cache. Get(tid int64, moveFront bool) (*statistics.Table, bool) // Put puts a cache. - Put(tid int64, tbl *statistics.Table, moveLRUFront bool) bool + Put(tid int64, tbl *statistics.Table) bool // Del deletes a cache. Del(int64) // Cost returns the memory usage of the cache. diff --git a/statistics/handle/cache/internal/lfu/lfu_cache.go b/statistics/handle/cache/internal/lfu/lfu_cache.go index 7c1d3bab80542..72f76a7493e0f 100644 --- a/statistics/handle/cache/internal/lfu/lfu_cache.go +++ b/statistics/handle/cache/internal/lfu/lfu_cache.go @@ -73,12 +73,7 @@ func (s *LFU) Get(tid int64, _ bool) (*statistics.Table, bool) { } // Put implements statsCacheInner -func (s *LFU) Put(tblID int64, tbl *statistics.Table, _ bool) bool { - return s.put(tblID, tbl) -} - -// Put implements statsCacheInner -func (s *LFU) put(tblID int64, tbl *statistics.Table) bool { +func (s *LFU) Put(tblID int64, tbl *statistics.Table) bool { ok := s.cache.Set(tblID, tbl, tbl.MemoryUsage().TotalTrackingMemUsage()) if ok { // NOTE: `s.cache` and `s.resultKeySet` may be inconsistent since the update operation is not atomic, but it's acceptable for our scenario s.resultKeySet.Add(tblID) diff --git a/statistics/handle/cache/internal/lfu/lfu_cache_test.go b/statistics/handle/cache/internal/lfu/lfu_cache_test.go index 72132046fdc6b..ca8154650ee28 100644 --- a/statistics/handle/cache/internal/lfu/lfu_cache_test.go +++ b/statistics/handle/cache/internal/lfu/lfu_cache_test.go @@ -34,7 +34,7 @@ func TestLFUPutGetDel(t *testing.T) { require.NoError(t, err) mockTable := testutil.NewMockStatisticsTable(1, 1, true, false, false) mockTableID := int64(1) - lfu.Put(mockTableID, mockTable, false) + lfu.Put(mockTableID, mockTable) lfu.wait() lfu.Del(mockTableID) v, ok := lfu.Get(mockTableID, false) @@ -51,23 +51,23 @@ func TestLFUFreshMemUsage(t *testing.T) { t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) t2 := testutil.NewMockStatisticsTable(2, 2, true, false, false) t3 := testutil.NewMockStatisticsTable(3, 3, true, false, false) - lfu.Put(int64(1), t1, false) - lfu.Put(int64(2), t2, false) - lfu.Put(int64(3), t3, false) + lfu.Put(int64(1), t1) + lfu.Put(int64(2), t2) + lfu.Put(int64(3), t3) require.Equal(t, lfu.Cost(), 6*mockCMSMemoryUsage+6*mockCMSMemoryUsage) t4 := testutil.NewMockStatisticsTable(2, 1, true, false, false) - lfu.Put(int64(1), t4, false) + lfu.Put(int64(1), t4) require.Equal(t, lfu.Cost(), 6*mockCMSMemoryUsage+7*mockCMSMemoryUsage) t5 := testutil.NewMockStatisticsTable(2, 2, true, false, false) - lfu.Put(int64(1), t5, false) + lfu.Put(int64(1), t5) require.Equal(t, lfu.Cost(), 7*mockCMSMemoryUsage+7*mockCMSMemoryUsage) t6 := testutil.NewMockStatisticsTable(1, 2, true, false, false) - lfu.Put(int64(1), t6, false) + lfu.Put(int64(1), t6) require.Equal(t, lfu.Cost(), 7*mockCMSMemoryUsage+6*mockCMSMemoryUsage) t7 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - lfu.Put(int64(1), t7, false) + lfu.Put(int64(1), t7) require.Equal(t, lfu.Cost(), 6*mockCMSMemoryUsage+6*mockCMSMemoryUsage) lfu.wait() require.Equal(t, uint64(lfu.Cost()), lfu.metrics().CostAdded()-lfu.metrics().CostEvicted()) @@ -78,7 +78,7 @@ func TestLFUPutTooBig(t *testing.T) { require.NoError(t, err) mockTable := testutil.NewMockStatisticsTable(1, 1, true, false, false) // put mockTable, the index should be evicted - lfu.Put(int64(1), mockTable, false) + lfu.Put(int64(1), mockTable) _, ok := lfu.Get(int64(1), false) require.False(t, ok) lfu.wait() @@ -91,17 +91,17 @@ func TestCacheLen(t *testing.T) { require.NoError(t, err) t1 := testutil.NewMockStatisticsTable(2, 1, true, false, false) require.Equal(t, int64(12), t1.MemoryUsage().TotalTrackingMemUsage()) - lfu.Put(int64(1), t1, false) + lfu.Put(int64(1), t1) t2 := testutil.NewMockStatisticsTable(1, 1, true, false, false) // put t2, t1 should be evicted 2 items and still exists in the list - lfu.Put(int64(2), t2, false) + lfu.Put(int64(2), t2) lfu.wait() require.Equal(t, lfu.Len(), 1) require.Equal(t, uint64(lfu.Cost()), lfu.metrics().CostAdded()-lfu.metrics().CostEvicted()) // put t3, t1/t2 should be evicted all items and disappeared from the list t3 := testutil.NewMockStatisticsTable(2, 1, true, false, false) - lfu.Put(int64(3), t3, false) + lfu.Put(int64(3), t3) lfu.wait() require.Equal(t, lfu.Len(), 1) require.Equal(t, uint64(lfu.Cost()), lfu.metrics().CostAdded()-lfu.metrics().CostEvicted()) @@ -118,7 +118,7 @@ func TestLFUCachePutGetWithManyConcurrency(t *testing.T) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - lfu.Put(int64(i), t1, true) + lfu.Put(int64(i), t1) }(i) go func(i int) { defer wg.Done() @@ -144,7 +144,7 @@ func TestLFUCachePutGetWithManyConcurrency2(t *testing.T) { defer wg.Done() for n := 0; n < 1000; n++ { t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - lfu.Put(int64(n), t1, true) + lfu.Put(int64(n), t1) } }() } diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index 48c5302ef5d72..2a22e2764a5d4 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -54,7 +54,7 @@ func (m *MapCache) Get(k int64, _ bool) (*statistics.Table, bool) { } // Put implements StatsCacheInner -func (m *MapCache) Put(k int64, v *statistics.Table, _ bool) bool { +func (m *MapCache) Put(k int64, v *statistics.Table) bool { item, ok := m.tables[k] if ok { oldCost := item.cost diff --git a/statistics/handle/cache/statscache.go b/statistics/handle/cache/statscache.go index 816ee6d3180b7..644f1f4c0eee8 100644 --- a/statistics/handle/cache/statscache.go +++ b/statistics/handle/cache/statscache.go @@ -17,6 +17,8 @@ package cache import ( "sync/atomic" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache/internal/metrics" ) @@ -46,3 +48,12 @@ func (s *StatsCachePointer) Replace(newCache *StatsCache) { s.Store(newCache) metrics.CostGauge.Set(float64(newCache.Cost())) } + +// UpdateStatsCache updates the cache with the new cache. +func (s *StatsCachePointer) UpdateStatsCache(newCache *StatsCache, tables []*statistics.Table, deletedIDs []int64, opts ...TableStatsOpt) { + if enableQuota := config.GetGlobalConfig().Performance.EnableStatsCacheMemQuota; enableQuota { + s.Load().Update(tables, deletedIDs, opts...) + } else { + s.Replace(newCache.CopyAndUpdate(tables, deletedIDs, opts...)) + } +} diff --git a/statistics/handle/cache/statscacheinner.go b/statistics/handle/cache/statscacheinner.go index 164f8c4738052..9b834ad3ad187 100644 --- a/statistics/handle/cache/statscacheinner.go +++ b/statistics/handle/cache/statscacheinner.go @@ -112,18 +112,13 @@ func (sc *StatsCache) GetFromInternal(id int64) (*statistics.Table, bool) { return sc.getCache(id, false) } -// PutFromUser puts the table statistics to the cache from query. -func (sc *StatsCache) PutFromUser(id int64, t *statistics.Table) { - sc.put(id, t, false) +// Put puts the table statistics to the cache from query. +func (sc *StatsCache) Put(id int64, t *statistics.Table) { + sc.put(id, t) } -// PutFromInternal puts the table statistics to the cache from internal. -func (sc *StatsCache) PutFromInternal(id int64, t *statistics.Table) { - sc.put(id, t, false) -} - -func (sc *StatsCache) putCache(id int64, t *statistics.Table, moveLRUFront bool) bool { - ok := sc.c.Put(id, t, moveLRUFront) +func (sc *StatsCache) putCache(id int64, t *statistics.Table) bool { + ok := sc.c.Put(id, t) if ok { return ok } @@ -133,8 +128,8 @@ func (sc *StatsCache) putCache(id int64, t *statistics.Table, moveLRUFront bool) } // Put puts the table statistics to the cache. -func (sc *StatsCache) put(id int64, t *statistics.Table, moveLRUFront bool) { - ok := sc.putCache(id, t, moveLRUFront) +func (sc *StatsCache) put(id int64, t *statistics.Table) { + ok := sc.putCache(id, t) if !ok { logutil.BgLogger().Warn("fail to put the stats cache", zap.Int64("id", id)) return @@ -189,11 +184,7 @@ func (sc *StatsCache) CopyAndUpdate(tables []*statistics.Table, deletedIDs []int newCache.maxTblStatsVer.Store(sc.maxTblStatsVer.Load()) for _, tbl := range tables { id := tbl.PhysicalID - if option.byQuery { - newCache.c.Put(id, tbl, true) - } else { - newCache.c.Put(id, tbl, false) - } + newCache.c.Put(id, tbl) } for _, id := range deletedIDs { newCache.c.Del(id) @@ -216,11 +207,7 @@ func (sc *StatsCache) Update(tables []*statistics.Table, deletedIDs []int64, opt } for _, tbl := range tables { id := tbl.PhysicalID - if option.byQuery { - sc.c.Put(id, tbl, true) - } else { - sc.c.Put(id, tbl, false) - } + sc.c.Put(id, tbl) } for _, id := range deletedIDs { sc.c.Del(id) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index bf813835f6789..a4af5e01e57d1 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1072,12 +1072,9 @@ func (h *Handle) initStatsCache(newCache *cache.StatsCache) { // If it is in the COW mode. it overrides the global statsCache with a new one, it may fail // if the global statsCache has been modified by others already. // Callers should add retry loop if necessary. -func (h *Handle) updateStatsCache(newCache *cache.StatsCache, tables []*statistics.Table, deletedIDs []int64, opts ...cache.TableStatsOpt) (updated bool) { - if enableQuota := config.GetGlobalConfig().Performance.EnableStatsCacheMemQuota; enableQuota { - h.statsCache.Load().Update(tables, deletedIDs, opts...) - } else { - h.statsCache.Replace(newCache.CopyAndUpdate(tables, deletedIDs, opts...)) - } +func (h *Handle) updateStatsCache(newCache *cache.StatsCache, tables []*statistics.Table, deletedIDs []int64, + opts ...cache.TableStatsOpt) (updated bool) { + h.statsCache.UpdateStatsCache(newCache, tables, deletedIDs, opts...) return true } From a362466492ec51dbcd0a1e7b2475aaad192a0ac0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 27 Jul 2023 12:32:30 +0800 Subject: [PATCH 2/4] statscache: improve stats cache benchmark Signed-off-by: Weizhen Wang --- statistics/handle/cache/bench_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 22f90da2a941d..1af5bc6e559b0 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -49,6 +49,7 @@ func benchPutGet(b *testing.B, c *StatsCachePointer) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) + t1.PhysicalID = rand.Int63() c.UpdateStatsCache(c.Load(), []*statistics.Table{t1}, nil) }(i) } From 3e9c993fb537f3a335b4d3f59237aa842bc3f556 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 27 Jul 2023 13:50:24 +0800 Subject: [PATCH 3/4] statscache: improve stats cache benchmark Signed-off-by: Weizhen Wang --- statistics/handle/cache/bench_test.go | 53 +++++++++++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 1af5bc6e559b0..3f701ad40e9d2 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -64,6 +64,31 @@ func benchPutGet(b *testing.B, c *StatsCachePointer) { b.StopTimer() } +func benchGet(b *testing.B, c *StatsCachePointer) { + var w sync.WaitGroup + for i := 0; i < b.N; i++ { + w.Add(1) + go func(i int) { + defer w.Done() + t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) + t1.PhysicalID = rand.Int63() + c.UpdateStatsCache(c.Load(), []*statistics.Table{t1}, nil) + }(i) + } + w.Done() + b.ResetTimer() + var wg sync.WaitGroup + for i := 0; i < b.N; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.Load().GetFromUser(int64(i)) + }(i) + } + wg.Wait() + b.StopTimer() +} + func BenchmarkStatsCacheLFUCopyAndUpdate(b *testing.B) { restore := config.RestoreFunc() defer restore() @@ -116,11 +141,39 @@ func BenchmarkMapCachePutGet(b *testing.B) { benchPutGet(b, cache) } +func BenchmarkLFUCacheGet(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + cache, err := NewStatsCachePointer() + if err != nil { + b.Fail() + } + benchGet(b, cache) +} + +func BenchmarkMapCacheGet(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = false + }) + cache, err := NewStatsCachePointer() + if err != nil { + b.Fail() + } + benchGet(b, cache) +} + func TestBenchDaily(*testing.T) { benchdaily.Run( BenchmarkStatsCacheLFUCopyAndUpdate, BenchmarkStatsCacheMapCacheCopyAndUpdate, BenchmarkLFUCachePutGet, BenchmarkMapCachePutGet, + BenchmarkLFUCacheGet, + BenchmarkMapCacheGet, ) } From 5d87364f757551792fa5fb7f65617fe0e5eaca4d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 27 Jul 2023 13:54:22 +0800 Subject: [PATCH 4/4] statscache: improve stats cache benchmark Signed-off-by: Weizhen Wang --- statistics/handle/cache/bench_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 3f701ad40e9d2..d178ba47e801b 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -75,7 +75,7 @@ func benchGet(b *testing.B, c *StatsCachePointer) { c.UpdateStatsCache(c.Load(), []*statistics.Table{t1}, nil) }(i) } - w.Done() + w.Wait() b.ResetTimer() var wg sync.WaitGroup for i := 0; i < b.N; i++ {