Skip to content

Commit

Permalink
statcache: add bench for LRU/mapCache (#45277)
Browse files Browse the repository at this point in the history
ref #45281
  • Loading branch information
hawkingrei authored Jul 12, 2023
1 parent a84c1e8 commit ce0e9ed
Show file tree
Hide file tree
Showing 10 changed files with 330 additions and 113 deletions.
16 changes: 15 additions & 1 deletion statistics/handle/cache/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
Expand All @@ -24,3 +24,17 @@ 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",
"//util/benchdaily",
],
)
68 changes: 68 additions & 0 deletions statistics/handle/cache/bench_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// 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"
"github.com/pingcap/tidb/util/benchdaily"
)

func benchCopyAndUpdate(b *testing.B, c *StatsCachePointer) {
var wg sync.WaitGroup
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 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
})
benchCopyAndUpdate(b, NewStatsCachePointer())
}

func TestBenchDaily(t *testing.T) {
benchdaily.Run(
BenchmarkStatsCacheLRUCopyAndUpdate,
BenchmarkStatsCacheMapCacheCopyAndUpdate,
)
}
16 changes: 15 additions & 1 deletion statistics/handle/cache/internal/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
Expand All @@ -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",
],
)
100 changes: 100 additions & 0 deletions statistics/handle/cache/internal/bench_test.go
Original file line number Diff line number Diff line change
@@ -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,
)
}
4 changes: 1 addition & 3 deletions statistics/handle/cache/internal/lru/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,8 @@ go_test(
flaky = True,
shard_count = 8,
deps = [
"//parser/model",
"//parser/mysql",
"//statistics",
"//types",
"//statistics/handle/cache/internal/testutil",
"@com_github_stretchr_testify//require",
],
)
11 changes: 0 additions & 11 deletions statistics/handle/cache/internal/lru/lru_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Loading

0 comments on commit ce0e9ed

Please sign in to comment.