From 43ab4ba90b2249c8d8cf761f0fa9fcf8d218a5b0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 15:21:22 +0800 Subject: [PATCH 01/11] update Signed-off-by: Weizhen Wang --- .../handle/cache/internal/lru/BUILD.bazel | 9 +- .../handle/cache/internal/lru/bench_test.go | 31 +++++ .../cache/internal/lru/lru_cache_test.go | 114 ++++-------------- .../cache/internal/mapcache/BUILD.bazel | 11 +- .../cache/internal/mapcache/bench_test.go | 29 +++++ .../cache/internal/mapcache/map_cache.go | 17 +++ .../cache/internal/testutil/BUILD.bazel | 14 +++ .../cache/internal/testutil/testutil.go | 77 ++++++++++++ 8 files changed, 209 insertions(+), 93 deletions(-) create mode 100644 statistics/handle/cache/internal/lru/bench_test.go create mode 100644 statistics/handle/cache/internal/mapcache/bench_test.go create mode 100644 statistics/handle/cache/internal/testutil/BUILD.bazel create mode 100644 statistics/handle/cache/internal/testutil/testutil.go diff --git a/statistics/handle/cache/internal/lru/BUILD.bazel b/statistics/handle/cache/internal/lru/BUILD.bazel index fdd4b956d12be..dee46aedf48ee 100644 --- a/statistics/handle/cache/internal/lru/BUILD.bazel +++ b/statistics/handle/cache/internal/lru/BUILD.bazel @@ -15,15 +15,16 @@ go_library( go_test( name = "lru_test", timeout = "short", - srcs = ["lru_cache_test.go"], + srcs = [ + "bench_test.go", + "lru_cache_test.go", + ], embed = [":lru"], flaky = True, shard_count = 8, deps = [ - "//parser/model", - "//parser/mysql", "//statistics", - "//types", + "//statistics/handle/cache/internal/testutil", "@com_github_stretchr_testify//require", ], ) diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go new file mode 100644 index 0000000000000..5d035ba60a9ce --- /dev/null +++ b/statistics/handle/cache/internal/lru/bench_test.go @@ -0,0 +1,31 @@ +package lru + +import ( + "sync" + "testing" + + "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" +) + +const defaultSize int64 = 1000 + +func BenchmarkLruPut(b *testing.B) { + var ( + wg sync.WaitGroup + c = NewStatsLruCache(defaultSize) + ) + 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) + }(i) + } + + wg.Wait() + + b.StopTimer() +} diff --git a/statistics/handle/cache/internal/lru/lru_cache_test.go b/statistics/handle/cache/internal/lru/lru_cache_test.go index 8b81b22816aa1..e8f684514c1d3 100644 --- a/statistics/handle/cache/internal/lru/lru_cache_test.go +++ b/statistics/handle/cache/internal/lru/lru_cache_test.go @@ -17,10 +17,8 @@ package lru import ( "testing" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" "github.com/stretchr/testify/require" ) @@ -30,75 +28,11 @@ var ( mockHistMemoryUsage = int64(289) ) -// each column and index consumes 4 bytes memory -func newMockStatisticsTable(columns int, indices int, withCMS, withTopN, withHist bool) *statistics.Table { - t := &statistics.Table{} - t.Columns = make(map[int64]*statistics.Column) - t.Indices = make(map[int64]*statistics.Index) - for i := 1; i <= columns; i++ { - t.Columns[int64(i)] = &statistics.Column{ - Info: &model.ColumnInfo{ID: int64(i)}, - StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), - } - if withCMS { - t.Columns[int64(i)].CMSketch = statistics.NewCMSketch(1, 1) - } - if withTopN { - t.Columns[int64(i)].TopN = statistics.NewTopN(1) - t.Columns[int64(i)].TopN.AppendTopN([]byte{}, 1) - } - if withHist { - t.Columns[int64(i)].Histogram = *statistics.NewHistogram(0, 10, 0, 0, types.NewFieldType(mysql.TypeBlob), 1, 0) - } - } - for i := 1; i <= indices; i++ { - t.Indices[int64(i)] = &statistics.Index{ - Info: &model.IndexInfo{ID: int64(i)}, - StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), - } - if withCMS { - t.Indices[int64(i)].CMSketch = statistics.NewCMSketch(1, 1) - } - if withTopN { - t.Indices[int64(i)].TopN = statistics.NewTopN(1) - t.Indices[int64(i)].TopN.AppendTopN([]byte{}, 1) - } - if withHist { - t.Indices[int64(i)].Histogram = *statistics.NewHistogram(0, 10, 0, 0, types.NewFieldType(mysql.TypeBlob), 1, 0) - } - } - return t -} - -func mockTableAppendColumn(t *statistics.Table) { - index := int64(len(t.Columns) + 1) - t.Columns[index] = &statistics.Column{ - Info: &model.ColumnInfo{ID: index}, - CMSketch: statistics.NewCMSketch(1, 1), - } -} - -func mockTableAppendIndex(t *statistics.Table) { - index := int64(len(t.Indices) + 1) - t.Indices[index] = &statistics.Index{ - Info: &model.IndexInfo{ID: index}, - CMSketch: statistics.NewCMSketch(1, 1), - } -} - -func mockTableRemoveColumn(t *statistics.Table) { - delete(t.Columns, int64(len(t.Columns))) -} - -func mockTableRemoveIndex(t *statistics.Table) { - delete(t.Indices, int64(len(t.Indices))) -} - func TestLRUPutGetDel(t *testing.T) { capacity := int64(100) lru := NewStatsLruCache(capacity) require.Equal(t, capacity, lru.capacity()) - mockTable := newMockStatisticsTable(1, 1, true, false, false) + mockTable := testutil.NewMockStatisticsTable(1, 1, true, false, false) mockTableID := int64(1) lru.Put(mockTableID, mockTable, false) v, ok := lru.Get(mockTableID, false) @@ -123,7 +57,7 @@ func TestLRUPutGetDel(t *testing.T) { func TestLRUEvict(t *testing.T) { capacity := int64(24) lru := NewStatsLruCache(capacity) - t1 := newMockStatisticsTable(2, 0, true, false, false) + t1 := testutil.NewMockStatisticsTable(2, 0, true, false, false) require.Equal(t, t1.MemoryUsage().TotalIdxTrackingMemUsage(), int64(0)) require.Equal(t, t1.MemoryUsage().TotalColTrackingMemUsage(), 2*mockCMSMemoryUsage) @@ -133,21 +67,23 @@ func TestLRUEvict(t *testing.T) { require.Equal(t, lru.Cost(), t1.MemoryUsage().TotalTrackingMemUsage()) // Put t2, assert TotalMemUsage and TotalColTrackingMemUsage - t2 := newMockStatisticsTable(2, 1, true, false, false) + t2 := testutil.NewMockStatisticsTable(2, 1, true, false, false) lru.Put(int64(2), t2, false) require.Equal(t, lru.Cost(), 4*mockCMSMemoryUsage+1*mockCMSMemoryUsage) // Put t3, a column of t1 should be evicted - t3 := newMockStatisticsTable(1, 1, true, false, false) + t3 := testutil.NewMockStatisticsTable(1, 1, true, false, false) lru.Put(int64(3), t3, false) + require.Equal(t, lru.Len(), 3) require.Equal(t, t1.MemoryUsage().TotalColTrackingMemUsage(), mockCMSMemoryUsage) require.Equal(t, lru.TotalCost(), t1.MemoryUsage().TotalMemUsage+t2.MemoryUsage().TotalMemUsage+t3.MemoryUsage().TotalMemUsage) require.Equal(t, lru.Cost(), 4*mockCMSMemoryUsage+2*mockCMSMemoryUsage) // Put t4, all indices' cmsketch of other tables should be evicted - t4 := newMockStatisticsTable(3, 3, true, false, false) + t4 := testutil.NewMockStatisticsTable(3, 3, true, false, false) lru.Put(int64(4), t4, false) + require.Equal(t, lru.Len(), 4) require.Equal(t, t1.MemoryUsage().TotalTrackingMemUsage(), int64(0)) require.Equal(t, t2.MemoryUsage().TotalTrackingMemUsage(), int64(0)) @@ -163,7 +99,7 @@ func TestLRUCopy(t *testing.T) { lru := NewStatsLruCache(1000) tables := make([]*statistics.Table, 0) for i := 0; i < 5; i++ { - tables = append(tables, newMockStatisticsTable(1, 1, true, false, false)) + tables = append(tables, testutil.NewMockStatisticsTable(1, 1, true, false, false)) } // insert 1,2,3 into old lru @@ -199,32 +135,32 @@ func TestLRUCopy(t *testing.T) { func TestLRUFreshMemUsage(t *testing.T) { lru := NewStatsLruCache(1000) - t1 := newMockStatisticsTable(1, 1, true, false, false) - t2 := newMockStatisticsTable(2, 2, true, false, false) - t3 := newMockStatisticsTable(3, 3, true, false, false) + t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) + t2 := testutil.NewMockStatisticsTable(2, 2, true, false, false) + t3 := testutil.NewMockStatisticsTable(3, 3, true, false, false) lru.Put(int64(1), t1, false) lru.Put(int64(2), t2, false) lru.Put(int64(3), t3, false) require.Equal(t, lru.Cost(), 6*mockCMSMemoryUsage+6*mockCMSMemoryUsage) - mockTableAppendColumn(t1) + testutil.MockTableAppendColumn(t1) lru.Put(int64(1), t1, false) require.Equal(t, lru.Cost(), 6*mockCMSMemoryUsage+7*mockCMSMemoryUsage) - mockTableAppendIndex(t1) + testutil.MockTableAppendIndex(t1) lru.Put(int64(1), t1, false) require.Equal(t, lru.Cost(), 7*mockCMSMemoryUsage+7*mockCMSMemoryUsage) - mockTableRemoveColumn(t1) + testutil.MockTableRemoveColumn(t1) lru.Put(int64(1), t1, false) require.Equal(t, lru.Cost(), 7*mockCMSMemoryUsage+6*mockCMSMemoryUsage) - mockTableRemoveIndex(t1) + testutil.MockTableRemoveIndex(t1) lru.Put(int64(1), t1, false) require.Equal(t, lru.Cost(), 6*mockCMSMemoryUsage+6*mockCMSMemoryUsage) } func TestLRUPutTooBig(t *testing.T) { lru := NewStatsLruCache(1) - mockTable := newMockStatisticsTable(1, 1, true, false, false) + mockTable := testutil.NewMockStatisticsTable(1, 1, true, false, false) // put mockTable, the index should be evicted lru.Put(int64(1), mockTable, false) _, ok := lru.Get(int64(1), false) @@ -236,9 +172,10 @@ func TestLRUPutTooBig(t *testing.T) { func TestCacheLen(t *testing.T) { capacity := int64(12) stats := NewStatsLruCache(capacity) - t1 := newMockStatisticsTable(2, 1, true, false, false) + + t1 := testutil.NewMockStatisticsTable(2, 1, true, false, false) stats.Put(int64(1), t1, false) - t2 := newMockStatisticsTable(1, 1, true, false, false) + t2 := testutil.NewMockStatisticsTable(1, 1, true, false, false) // put t2, t1 should be evicted 2 items and still exists in the list stats.Put(int64(2), t2, false) require.Equal(t, stats.lru.cache.Len(), 3) @@ -246,8 +183,9 @@ func TestCacheLen(t *testing.T) { require.Equal(t, stats.Len(), 2) // put t3, t1/t2 should be evicted all items and disappeared from the list - t3 := newMockStatisticsTable(2, 1, true, false, false) + t3 := testutil.NewMockStatisticsTable(2, 1, true, false, false) stats.Put(int64(3), t3, false) + require.Equal(t, stats.lru.cache.Len(), 3) require.Equal(t, t1.MemoryUsage().TotalTrackingMemUsage(), int64(0)) require.Equal(t, t2.MemoryUsage().TotalTrackingMemUsage(), int64(0)) @@ -257,9 +195,9 @@ func TestCacheLen(t *testing.T) { func TestLRUMove(t *testing.T) { capacity := int64(100) s := NewStatsLruCache(capacity) - t1 := newMockStatisticsTable(1, 1, true, false, false) + t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) t1ID := int64(1) - t2 := newMockStatisticsTable(1, 1, true, false, false) + t2 := testutil.NewMockStatisticsTable(1, 1, true, false, false) t2ID := int64(2) s.Put(t1ID, t1, false) s.Put(t2ID, t2, false) @@ -275,7 +213,7 @@ func TestLRUMove(t *testing.T) { func TestLRUEvictPolicy(t *testing.T) { capacity := int64(999) s := NewStatsLruCache(capacity) - t1 := newMockStatisticsTable(1, 0, true, true, true) + t1 := testutil.NewMockStatisticsTable(1, 0, true, true, true) s.Put(1, t1, false) require.Equal(t, s.TotalCost(), mockCMSMemoryUsage+mockTopNMemoryUsage+mockHistMemoryUsage) require.Equal(t, s.Cost(), mockCMSMemoryUsage+mockTopNMemoryUsage+mockHistMemoryUsage) @@ -308,7 +246,7 @@ func TestLRUEvictPolicy(t *testing.T) { require.True(t, t1.Columns[1].IsAllEvicted()) s = NewStatsLruCache(capacity) - t2 := newMockStatisticsTable(0, 1, true, true, true) + t2 := testutil.NewMockStatisticsTable(0, 1, true, true, true) s.Put(2, t2, false) require.Equal(t, s.TotalCost(), mockCMSMemoryUsage+mockTopNMemoryUsage+mockHistMemoryUsage) require.Equal(t, s.Cost(), mockCMSMemoryUsage+mockTopNMemoryUsage+mockHistMemoryUsage) diff --git a/statistics/handle/cache/internal/mapcache/BUILD.bazel b/statistics/handle/cache/internal/mapcache/BUILD.bazel index 00d2fc65e3421..bed8982d7d294 100644 --- a/statistics/handle/cache/internal/mapcache/BUILD.bazel +++ b/statistics/handle/cache/internal/mapcache/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "mapcache", @@ -10,3 +10,12 @@ go_library( "//statistics/handle/cache/internal", ], ) + +go_test( + name = "mapcache_test", + timeout = "short", + srcs = ["bench_test.go"], + embed = [":mapcache"], + flaky = True, + deps = ["//statistics/handle/cache/internal/testutil"], +) diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go new file mode 100644 index 0000000000000..abc0d1e577bc5 --- /dev/null +++ b/statistics/handle/cache/internal/mapcache/bench_test.go @@ -0,0 +1,29 @@ +package mapcache + +import ( + "sync" + "testing" + + "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" +) + +const defaultSize int64 = 1000 + +func BenchmarkMapCachePut(b *testing.B) { + var ( + wg sync.WaitGroup + c = NewMapCache() + ) + + 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) + }(i) + } + wg.Wait() + b.StopTimer() +} diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index bd4b8897c2265..12424c7e28068 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -15,6 +15,8 @@ package mapcache import ( + "sync" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache/internal" ) @@ -35,6 +37,7 @@ func (c cacheItem) copy() cacheItem { // MapCache is a cache based on map. type MapCache struct { + mu sync.RWMutex tables map[int64]cacheItem memUsage int64 } @@ -49,12 +52,16 @@ func NewMapCache() *MapCache { // Get implements StatsCacheInner func (m *MapCache) Get(k int64, _ bool) (*statistics.Table, bool) { + m.mu.RLock() + defer m.mu.RUnlock() v, ok := m.tables[k] return v.value, ok } // Put implements StatsCacheInner func (m *MapCache) Put(k int64, v *statistics.Table, _ bool) { + m.mu.Lock() + defer m.mu.Unlock() item, ok := m.tables[k] if ok { oldCost := item.cost @@ -77,6 +84,8 @@ func (m *MapCache) Put(k int64, v *statistics.Table, _ bool) { // Del implements StatsCacheInner func (m *MapCache) Del(k int64) { + m.mu.Lock() + defer m.mu.Unlock() item, ok := m.tables[k] if !ok { return @@ -87,11 +96,15 @@ func (m *MapCache) Del(k int64) { // Cost implements StatsCacheInner func (m *MapCache) Cost() int64 { + m.mu.RLock() + defer m.mu.RUnlock() return m.memUsage } // Keys implements StatsCacheInner func (m *MapCache) Keys() []int64 { + m.mu.RLock() + defer m.mu.RUnlock() ks := make([]int64, 0, len(m.tables)) for k := range m.tables { ks = append(ks, k) @@ -101,6 +114,8 @@ func (m *MapCache) Keys() []int64 { // Values implements StatsCacheInner func (m *MapCache) Values() []*statistics.Table { + m.mu.RLock() + defer m.mu.RUnlock() vs := make([]*statistics.Table, 0, len(m.tables)) for _, v := range m.tables { vs = append(vs, v.value) @@ -110,6 +125,8 @@ func (m *MapCache) Values() []*statistics.Table { // Map implements StatsCacheInner func (m *MapCache) Map() map[int64]*statistics.Table { + m.mu.RLock() + defer m.mu.RUnlock() t := make(map[int64]*statistics.Table, len(m.tables)) for k, v := range m.tables { t[k] = v.value diff --git a/statistics/handle/cache/internal/testutil/BUILD.bazel b/statistics/handle/cache/internal/testutil/BUILD.bazel new file mode 100644 index 0000000000000..161e074e54ac6 --- /dev/null +++ b/statistics/handle/cache/internal/testutil/BUILD.bazel @@ -0,0 +1,14 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "testutil", + srcs = ["testutil.go"], + importpath = "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil", + visibility = ["//statistics/handle/cache:__subpackages__"], + deps = [ + "//parser/model", + "//parser/mysql", + "//statistics", + "//types", + ], +) diff --git a/statistics/handle/cache/internal/testutil/testutil.go b/statistics/handle/cache/internal/testutil/testutil.go new file mode 100644 index 0000000000000..bfb19691fee51 --- /dev/null +++ b/statistics/handle/cache/internal/testutil/testutil.go @@ -0,0 +1,77 @@ +package testutil + +import ( + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/types" +) + +// NewMockStatisticsTable creates a mock statistics table with given columns and indices. +// each column and index consumes 4 bytes memory +func NewMockStatisticsTable(columns int, indices int, withCMS, withTopN, withHist bool) *statistics.Table { + t := &statistics.Table{} + t.Columns = make(map[int64]*statistics.Column) + t.Indices = make(map[int64]*statistics.Index) + for i := 1; i <= columns; i++ { + t.Columns[int64(i)] = &statistics.Column{ + Info: &model.ColumnInfo{ID: int64(i)}, + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + } + if withCMS { + t.Columns[int64(i)].CMSketch = statistics.NewCMSketch(1, 1) + } + if withTopN { + t.Columns[int64(i)].TopN = statistics.NewTopN(1) + t.Columns[int64(i)].TopN.AppendTopN([]byte{}, 1) + } + if withHist { + t.Columns[int64(i)].Histogram = *statistics.NewHistogram(0, 10, 0, 0, types.NewFieldType(mysql.TypeBlob), 1, 0) + } + } + for i := 1; i <= indices; i++ { + t.Indices[int64(i)] = &statistics.Index{ + Info: &model.IndexInfo{ID: int64(i)}, + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + } + if withCMS { + t.Indices[int64(i)].CMSketch = statistics.NewCMSketch(1, 1) + } + if withTopN { + t.Indices[int64(i)].TopN = statistics.NewTopN(1) + t.Indices[int64(i)].TopN.AppendTopN([]byte{}, 1) + } + if withHist { + t.Indices[int64(i)].Histogram = *statistics.NewHistogram(0, 10, 0, 0, types.NewFieldType(mysql.TypeBlob), 1, 0) + } + } + return t +} + +// MockTableAppendColumn appends a column to the table. +func MockTableAppendColumn(t *statistics.Table) { + index := int64(len(t.Columns) + 1) + t.Columns[index] = &statistics.Column{ + Info: &model.ColumnInfo{ID: index}, + CMSketch: statistics.NewCMSketch(1, 1), + } +} + +// MockTableAppendIndex appends an index to the table. +func MockTableAppendIndex(t *statistics.Table) { + index := int64(len(t.Indices) + 1) + t.Indices[index] = &statistics.Index{ + Info: &model.IndexInfo{ID: index}, + CMSketch: statistics.NewCMSketch(1, 1), + } +} + +// MockTableRemoveColumn removes the last column of the table. +func MockTableRemoveColumn(t *statistics.Table) { + delete(t.Columns, int64(len(t.Columns))) +} + +// MockTableRemoveIndex removes the last index of the table. +func MockTableRemoveIndex(t *statistics.Table) { + delete(t.Indices, int64(len(t.Indices))) +} From 1e8dff33122f6380b16b9a452c6db27d347fb793 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 10 Jul 2023 23:48:49 +0800 Subject: [PATCH 02/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/internal/lru/bench_test.go | 14 ++++++++++++++ .../handle/cache/internal/mapcache/bench_test.go | 14 ++++++++++++++ .../handle/cache/internal/testutil/testutil.go | 14 ++++++++++++++ 3 files changed, 42 insertions(+) diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go index 5d035ba60a9ce..306ea73361f34 100644 --- a/statistics/handle/cache/internal/lru/bench_test.go +++ b/statistics/handle/cache/internal/lru/bench_test.go @@ -1,3 +1,17 @@ +// 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 lru import ( diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go index abc0d1e577bc5..dd06326499588 100644 --- a/statistics/handle/cache/internal/mapcache/bench_test.go +++ b/statistics/handle/cache/internal/mapcache/bench_test.go @@ -1,3 +1,17 @@ +// 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 mapcache import ( diff --git a/statistics/handle/cache/internal/testutil/testutil.go b/statistics/handle/cache/internal/testutil/testutil.go index bfb19691fee51..674b8d61da863 100644 --- a/statistics/handle/cache/internal/testutil/testutil.go +++ b/statistics/handle/cache/internal/testutil/testutil.go @@ -1,3 +1,17 @@ +// 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 testutil import ( From e1a549671c02062b1e23a4ad5392341e214db200 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 11 Jul 2023 00:00:04 +0800 Subject: [PATCH 03/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/internal/mapcache/map_cache.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index 12424c7e28068..1c8c79370857f 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -37,9 +37,9 @@ func (c cacheItem) copy() cacheItem { // MapCache is a cache based on map. type MapCache struct { - mu sync.RWMutex tables map[int64]cacheItem memUsage int64 + mu sync.RWMutex } // NewMapCache creates a new map cache. From 47642949a6f7f4f4a61ce71336b2e218a4b7fb61 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 11 Jul 2023 11:37:16 +0800 Subject: [PATCH 04/11] update Signed-off-by: Weizhen Wang --- .../handle/cache/internal/lru/bench_test.go | 27 +++++++++++++++++++ .../cache/internal/mapcache/bench_test.go | 27 +++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go index 306ea73361f34..786a473b59e4d 100644 --- a/statistics/handle/cache/internal/lru/bench_test.go +++ b/statistics/handle/cache/internal/lru/bench_test.go @@ -43,3 +43,30 @@ func BenchmarkLruPut(b *testing.B) { b.StopTimer() } + +func BenchmarkLruPutGet(b *testing.B) { + var ( + wg sync.WaitGroup + c = NewStatsLruCache(defaultSize) + ) + 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) + }(i) + } + for i := 0; i < b.N; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.Get(int64(i)) + }(i) + } + wg.Wait() + + b.StopTimer() +} diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go index dd06326499588..627088c40b013 100644 --- a/statistics/handle/cache/internal/mapcache/bench_test.go +++ b/statistics/handle/cache/internal/mapcache/bench_test.go @@ -41,3 +41,30 @@ func BenchmarkMapCachePut(b *testing.B) { wg.Wait() b.StopTimer() } + +func BenchmarkMapCachePutGet(b *testing.B) { + var ( + wg sync.WaitGroup + c = NewMapCache() + ) + 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) + }(i) + } + for i := 0; i < b.N; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.Get(int64(i)) + }(i) + } + wg.Wait() + + b.StopTimer() +} From 7c24a6d9d19a36b0daf2656dde5b9a0613f3fb47 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 15:35:14 +0800 Subject: [PATCH 05/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/internal/lru/bench_test.go | 6 +++--- statistics/handle/cache/internal/mapcache/bench_test.go | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go index 786a473b59e4d..d33d468bd3b4c 100644 --- a/statistics/handle/cache/internal/lru/bench_test.go +++ b/statistics/handle/cache/internal/lru/bench_test.go @@ -35,7 +35,7 @@ func BenchmarkLruPut(b *testing.B) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1) + c.Put(int64(i), t1, true) }(i) } @@ -56,14 +56,14 @@ func BenchmarkLruPutGet(b *testing.B) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1) + 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)) + c.Get(int64(i), true) }(i) } wg.Wait() diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go index 627088c40b013..67aff77fa979d 100644 --- a/statistics/handle/cache/internal/mapcache/bench_test.go +++ b/statistics/handle/cache/internal/mapcache/bench_test.go @@ -35,7 +35,7 @@ func BenchmarkMapCachePut(b *testing.B) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1) + c.Put(int64(i), t1, true) }(i) } wg.Wait() @@ -54,14 +54,14 @@ func BenchmarkMapCachePutGet(b *testing.B) { go func(i int) { defer wg.Done() t1 := testutil.NewMockStatisticsTable(1, 1, true, false, false) - c.Put(int64(i), t1) + 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)) + c.Get(int64(i), true) }(i) } wg.Wait() From 7bc89afa69059167033432396462e9400b25f388 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 16:30:06 +0800 Subject: [PATCH 06/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/BUILD.bazel | 15 +++++- statistics/handle/cache/bench_test.go | 75 +++++++++++++++++++++++++++ 2 files changed, 89 insertions(+), 1 deletion(-) create mode 100644 statistics/handle/cache/bench_test.go diff --git a/statistics/handle/cache/BUILD.bazel b/statistics/handle/cache/BUILD.bazel index 5d7914aa27328..4466b2f505873 100644 --- a/statistics/handle/cache/BUILD.bazel +++ b/statistics/handle/cache/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "cache", @@ -24,3 +24,16 @@ go_library( "//util/syncutil", ], ) + +go_test( + name = "cache_test", + timeout = "short", + srcs = ["bench_test.go"], + embed = [":cache"], + flaky = True, + deps = [ + "//config", + "//statistics", + "//statistics/handle/cache/internal/testutil", + ], +) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go new file mode 100644 index 0000000000000..9c652072ef66f --- /dev/null +++ b/statistics/handle/cache/bench_test.go @@ -0,0 +1,75 @@ +// 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 cache + +import ( + "math/rand" + "sync" + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" +) + +func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + var ( + wg sync.WaitGroup + c = NewStatsCachePointer() + ) + b.ResetTimer() + for i := 0; i < b.N; i++ { + wg.Add(1) + go func() { + 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)) + }() + } + wg.Wait() + b.StopTimer() +} + +func BenchmarkStatsCacheMapCacheCopyAndUpdate(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = false + }) + var ( + wg sync.WaitGroup + c = NewStatsCachePointer() + ) + b.ResetTimer() + for i := 0; i < b.N; i++ { + wg.Add(1) + go func() { + 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)) + }() + } + wg.Wait() + b.StopTimer() +} From 353876c4216e5d8a578817e71eb52ba96b34604c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 16:32:29 +0800 Subject: [PATCH 07/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/bench_test.go | 8 ++++++++ statistics/handle/cache/internal/lru/bench_test.go | 8 ++++++++ .../handle/cache/internal/mapcache/bench_test.go | 10 ++++++++-- 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 9c652072ef66f..77e58d31fdcec 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" + "github.com/pingcap/tidb/util/benchdaily" ) func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { @@ -73,3 +74,10 @@ func BenchmarkStatsCacheMapCacheCopyAndUpdate(b *testing.B) { wg.Wait() b.StopTimer() } + +func TestBenchDaily(t *testing.T) { + benchdaily.Run( + BenchmarkStatsCacheLRUCopyAndUpdate, + BenchmarkStatsCacheMapCacheCopyAndUpdate, + ) +} diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go index d33d468bd3b4c..c0c72c9a8fa6f 100644 --- a/statistics/handle/cache/internal/lru/bench_test.go +++ b/statistics/handle/cache/internal/lru/bench_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" + "github.com/pingcap/tidb/util/benchdaily" ) const defaultSize int64 = 1000 @@ -70,3 +71,10 @@ func BenchmarkLruPutGet(b *testing.B) { b.StopTimer() } + +func TestBenchDaily(t *testing.T) { + benchdaily.Run( + BenchmarkLruPut, + BenchmarkLruPutGet, + ) +} diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go index 67aff77fa979d..103c32908b408 100644 --- a/statistics/handle/cache/internal/mapcache/bench_test.go +++ b/statistics/handle/cache/internal/mapcache/bench_test.go @@ -19,10 +19,9 @@ import ( "testing" "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" + "github.com/pingcap/tidb/util/benchdaily" ) -const defaultSize int64 = 1000 - func BenchmarkMapCachePut(b *testing.B) { var ( wg sync.WaitGroup @@ -68,3 +67,10 @@ func BenchmarkMapCachePutGet(b *testing.B) { b.StopTimer() } + +func TestBenchDaily(t *testing.T) { + benchdaily.Run( + BenchmarkMapCachePut, + BenchmarkMapCachePutGet, + ) +} From a260818091d1c2d48a9a7a744341fef0a1433e35 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 16:33:17 +0800 Subject: [PATCH 08/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/BUILD.bazel | 1 + statistics/handle/cache/internal/lru/BUILD.bazel | 3 ++- statistics/handle/cache/internal/mapcache/BUILD.bazel | 5 ++++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/statistics/handle/cache/BUILD.bazel b/statistics/handle/cache/BUILD.bazel index 4466b2f505873..4b8ec8e927242 100644 --- a/statistics/handle/cache/BUILD.bazel +++ b/statistics/handle/cache/BUILD.bazel @@ -35,5 +35,6 @@ go_test( "//config", "//statistics", "//statistics/handle/cache/internal/testutil", + "//util/benchdaily", ], ) diff --git a/statistics/handle/cache/internal/lru/BUILD.bazel b/statistics/handle/cache/internal/lru/BUILD.bazel index dee46aedf48ee..6508049e2c5e4 100644 --- a/statistics/handle/cache/internal/lru/BUILD.bazel +++ b/statistics/handle/cache/internal/lru/BUILD.bazel @@ -21,10 +21,11 @@ go_test( ], embed = [":lru"], flaky = True, - shard_count = 8, + shard_count = 9, deps = [ "//statistics", "//statistics/handle/cache/internal/testutil", + "//util/benchdaily", "@com_github_stretchr_testify//require", ], ) diff --git a/statistics/handle/cache/internal/mapcache/BUILD.bazel b/statistics/handle/cache/internal/mapcache/BUILD.bazel index bed8982d7d294..cbcd20cc7bae4 100644 --- a/statistics/handle/cache/internal/mapcache/BUILD.bazel +++ b/statistics/handle/cache/internal/mapcache/BUILD.bazel @@ -17,5 +17,8 @@ go_test( srcs = ["bench_test.go"], embed = [":mapcache"], flaky = True, - deps = ["//statistics/handle/cache/internal/testutil"], + deps = [ + "//statistics/handle/cache/internal/testutil", + "//util/benchdaily", + ], ) From e4757b2cdc530e064f62c8b2f6606622b8e1610c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 17:02:25 +0800 Subject: [PATCH 09/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/bench_test.go | 39 +++++++++------------------ 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go index 77e58d31fdcec..f307275bdd255 100644 --- a/statistics/handle/cache/bench_test.go +++ b/statistics/handle/cache/bench_test.go @@ -25,16 +25,8 @@ import ( "github.com/pingcap/tidb/util/benchdaily" ) -func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { - restore := config.RestoreFunc() - defer restore() - config.UpdateGlobal(func(conf *config.Config) { - conf.Performance.EnableStatsCacheMemQuota = true - }) - var ( - wg sync.WaitGroup - c = NewStatsCachePointer() - ) +func benchCopyAndUpdate(b *testing.B, c *StatsCachePointer) { + var wg sync.WaitGroup b.ResetTimer() for i := 0; i < b.N; i++ { wg.Add(1) @@ -50,29 +42,22 @@ func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { b.StopTimer() } +func BenchmarkStatsCacheLRUCopyAndUpdate(b *testing.B) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + benchCopyAndUpdate(b, NewStatsCachePointer()) +} + func BenchmarkStatsCacheMapCacheCopyAndUpdate(b *testing.B) { restore := config.RestoreFunc() defer restore() config.UpdateGlobal(func(conf *config.Config) { conf.Performance.EnableStatsCacheMemQuota = false }) - var ( - wg sync.WaitGroup - c = NewStatsCachePointer() - ) - b.ResetTimer() - for i := 0; i < b.N; i++ { - wg.Add(1) - go func() { - 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)) - }() - } - wg.Wait() - b.StopTimer() + benchCopyAndUpdate(b, NewStatsCachePointer()) } func TestBenchDaily(t *testing.T) { From 13a91860a8d83112352793d015d7e36157704bb0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 17:07:13 +0800 Subject: [PATCH 10/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/internal/lru/lru_cache.go | 11 ----------- .../handle/cache/internal/mapcache/map_cache.go | 11 ----------- 2 files changed, 22 deletions(-) diff --git a/statistics/handle/cache/internal/lru/lru_cache.go b/statistics/handle/cache/internal/lru/lru_cache.go index 65e4590498753..c832c9af98cce 100644 --- a/statistics/handle/cache/internal/lru/lru_cache.go +++ b/statistics/handle/cache/internal/lru/lru_cache.go @@ -244,17 +244,6 @@ func (s *StatsInnerCache) Values() []*statistics.Table { return r } -// Map implements statsCacheInner -func (s *StatsInnerCache) Map() map[int64]*statistics.Table { - s.RLock() - defer s.RUnlock() - r := make(map[int64]*statistics.Table, len(s.elements)) - for k, v := range s.elements { - r[k] = v.tbl - } - return r -} - // Len implements statsCacheInner func (s *StatsInnerCache) Len() int { s.RLock() diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index 1c8c79370857f..423230626a730 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -123,17 +123,6 @@ func (m *MapCache) Values() []*statistics.Table { return vs } -// Map implements StatsCacheInner -func (m *MapCache) Map() map[int64]*statistics.Table { - m.mu.RLock() - defer m.mu.RUnlock() - t := make(map[int64]*statistics.Table, len(m.tables)) - for k, v := range m.tables { - t[k] = v.value - } - return t -} - // Len implements StatsCacheInner func (m *MapCache) Len() int { return len(m.tables) From 12cfd29e274ec3469537992d177be3b05f8c81f6 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 12 Jul 2023 17:39:53 +0800 Subject: [PATCH 11/11] update Signed-off-by: Weizhen Wang --- statistics/handle/cache/internal/BUILD.bazel | 16 ++- .../handle/cache/internal/bench_test.go | 100 ++++++++++++++++++ .../handle/cache/internal/lru/BUILD.bazel | 8 +- .../handle/cache/internal/lru/bench_test.go | 80 -------------- .../cache/internal/mapcache/BUILD.bazel | 14 +-- .../cache/internal/mapcache/bench_test.go | 76 ------------- .../cache/internal/mapcache/map_cache.go | 15 --- 7 files changed, 118 insertions(+), 191 deletions(-) create mode 100644 statistics/handle/cache/internal/bench_test.go delete mode 100644 statistics/handle/cache/internal/lru/bench_test.go delete mode 100644 statistics/handle/cache/internal/mapcache/bench_test.go diff --git a/statistics/handle/cache/internal/BUILD.bazel b/statistics/handle/cache/internal/BUILD.bazel index 7cd17f787dcf1..2eeef4a40e005 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") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "cache", @@ -15,3 +15,17 @@ 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/lru", + "//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 new file mode 100644 index 0000000000000..f1957032a429b --- /dev/null +++ b/statistics/handle/cache/internal/bench_test.go @@ -0,0 +1,100 @@ +// 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/lru" + "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: "LRU", + newFunc: func() internal.StatsCacheInner { + return lru.NewStatsLruCache(defaultSize) + }, + }, + { + name: "mapcache", + newFunc: func() internal.StatsCacheInner { + return mapcache.NewMapCache() + }, + }, +} + +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/lru/BUILD.bazel b/statistics/handle/cache/internal/lru/BUILD.bazel index 6508049e2c5e4..3c7f7a6a1cd50 100644 --- a/statistics/handle/cache/internal/lru/BUILD.bazel +++ b/statistics/handle/cache/internal/lru/BUILD.bazel @@ -15,17 +15,13 @@ go_library( go_test( name = "lru_test", timeout = "short", - srcs = [ - "bench_test.go", - "lru_cache_test.go", - ], + srcs = ["lru_cache_test.go"], embed = [":lru"], flaky = True, - shard_count = 9, + shard_count = 8, deps = [ "//statistics", "//statistics/handle/cache/internal/testutil", - "//util/benchdaily", "@com_github_stretchr_testify//require", ], ) diff --git a/statistics/handle/cache/internal/lru/bench_test.go b/statistics/handle/cache/internal/lru/bench_test.go deleted file mode 100644 index c0c72c9a8fa6f..0000000000000 --- a/statistics/handle/cache/internal/lru/bench_test.go +++ /dev/null @@ -1,80 +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 lru - -import ( - "sync" - "testing" - - "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" - "github.com/pingcap/tidb/util/benchdaily" -) - -const defaultSize int64 = 1000 - -func BenchmarkLruPut(b *testing.B) { - var ( - wg sync.WaitGroup - c = NewStatsLruCache(defaultSize) - ) - 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 BenchmarkLruPutGet(b *testing.B) { - var ( - wg sync.WaitGroup - c = NewStatsLruCache(defaultSize) - ) - 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( - BenchmarkLruPut, - BenchmarkLruPutGet, - ) -} diff --git a/statistics/handle/cache/internal/mapcache/BUILD.bazel b/statistics/handle/cache/internal/mapcache/BUILD.bazel index cbcd20cc7bae4..00d2fc65e3421 100644 --- a/statistics/handle/cache/internal/mapcache/BUILD.bazel +++ b/statistics/handle/cache/internal/mapcache/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 = "mapcache", @@ -10,15 +10,3 @@ go_library( "//statistics/handle/cache/internal", ], ) - -go_test( - name = "mapcache_test", - timeout = "short", - srcs = ["bench_test.go"], - embed = [":mapcache"], - flaky = True, - deps = [ - "//statistics/handle/cache/internal/testutil", - "//util/benchdaily", - ], -) diff --git a/statistics/handle/cache/internal/mapcache/bench_test.go b/statistics/handle/cache/internal/mapcache/bench_test.go deleted file mode 100644 index 103c32908b408..0000000000000 --- a/statistics/handle/cache/internal/mapcache/bench_test.go +++ /dev/null @@ -1,76 +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 mapcache - -import ( - "sync" - "testing" - - "github.com/pingcap/tidb/statistics/handle/cache/internal/testutil" - "github.com/pingcap/tidb/util/benchdaily" -) - -func BenchmarkMapCachePut(b *testing.B) { - var ( - wg sync.WaitGroup - c = NewMapCache() - ) - - 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 BenchmarkMapCachePutGet(b *testing.B) { - var ( - wg sync.WaitGroup - c = NewMapCache() - ) - 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( - BenchmarkMapCachePut, - BenchmarkMapCachePutGet, - ) -} diff --git a/statistics/handle/cache/internal/mapcache/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index 423230626a730..315e3bc2ef6a6 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -15,8 +15,6 @@ package mapcache import ( - "sync" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache/internal" ) @@ -39,7 +37,6 @@ func (c cacheItem) copy() cacheItem { type MapCache struct { tables map[int64]cacheItem memUsage int64 - mu sync.RWMutex } // NewMapCache creates a new map cache. @@ -52,16 +49,12 @@ func NewMapCache() *MapCache { // Get implements StatsCacheInner func (m *MapCache) Get(k int64, _ bool) (*statistics.Table, bool) { - m.mu.RLock() - defer m.mu.RUnlock() v, ok := m.tables[k] return v.value, ok } // Put implements StatsCacheInner func (m *MapCache) Put(k int64, v *statistics.Table, _ bool) { - m.mu.Lock() - defer m.mu.Unlock() item, ok := m.tables[k] if ok { oldCost := item.cost @@ -84,8 +77,6 @@ func (m *MapCache) Put(k int64, v *statistics.Table, _ bool) { // Del implements StatsCacheInner func (m *MapCache) Del(k int64) { - m.mu.Lock() - defer m.mu.Unlock() item, ok := m.tables[k] if !ok { return @@ -96,15 +87,11 @@ func (m *MapCache) Del(k int64) { // Cost implements StatsCacheInner func (m *MapCache) Cost() int64 { - m.mu.RLock() - defer m.mu.RUnlock() return m.memUsage } // Keys implements StatsCacheInner func (m *MapCache) Keys() []int64 { - m.mu.RLock() - defer m.mu.RUnlock() ks := make([]int64, 0, len(m.tables)) for k := range m.tables { ks = append(ks, k) @@ -114,8 +101,6 @@ func (m *MapCache) Keys() []int64 { // Values implements StatsCacheInner func (m *MapCache) Values() []*statistics.Table { - m.mu.RLock() - defer m.mu.RUnlock() vs := make([]*statistics.Table, 0, len(m.tables)) for _, v := range m.tables { vs = append(vs, v.value)