diff --git a/statistics/handle/cache/BUILD.bazel b/statistics/handle/cache/BUILD.bazel index 5d7914aa27328..4b8ec8e927242 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,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", + ], +) diff --git a/statistics/handle/cache/bench_test.go b/statistics/handle/cache/bench_test.go new file mode 100644 index 0000000000000..f307275bdd255 --- /dev/null +++ b/statistics/handle/cache/bench_test.go @@ -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, + ) +} 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 fdd4b956d12be..3c7f7a6a1cd50 100644 --- a/statistics/handle/cache/internal/lru/BUILD.bazel +++ b/statistics/handle/cache/internal/lru/BUILD.bazel @@ -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", ], ) 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/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/map_cache.go b/statistics/handle/cache/internal/mapcache/map_cache.go index bd4b8897c2265..315e3bc2ef6a6 100644 --- a/statistics/handle/cache/internal/mapcache/map_cache.go +++ b/statistics/handle/cache/internal/mapcache/map_cache.go @@ -108,15 +108,6 @@ func (m *MapCache) Values() []*statistics.Table { return vs } -// Map implements StatsCacheInner -func (m *MapCache) Map() map[int64]*statistics.Table { - 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) 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..674b8d61da863 --- /dev/null +++ b/statistics/handle/cache/internal/testutil/testutil.go @@ -0,0 +1,91 @@ +// 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 ( + "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))) +}