Skip to content

Commit

Permalink
planner: remove unnecessary method StatsCache.FreshMemUsage (pingc…
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Jul 11, 2023
1 parent 551b044 commit bd59ea3
Show file tree
Hide file tree
Showing 4 changed files with 50 additions and 17 deletions.
15 changes: 9 additions & 6 deletions statistics/handle/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache *cache.Stat
Version: row.GetUint64(0),
Name: getFullTableName(is, tableInfo),
}
cache.Put(physicalID, tbl)
cache.Put(physicalID, tbl) // put this table again since it is updated
}
}

Expand Down Expand Up @@ -153,7 +153,7 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache *
}
table.Columns[hist.ID] = col
}
cache.Put(tblID, table)
cache.Put(tblID, table) // put this table again since it is updated
}
}

Expand Down Expand Up @@ -222,7 +222,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache *cach
lastAnalyzePos.Copy(&col.LastAnalyzePos)
table.Columns[hist.ID] = col
}
cache.Put(tblID, table)
cache.Put(tblID, table) // put this table again since it is updated
}
}

Expand Down Expand Up @@ -290,6 +290,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.Put(table.PhysicalID, table) // put this table again since it is updated
}
for idx := range affectedIndexes {
idx.TopN.Sort()
Expand Down Expand Up @@ -342,6 +343,7 @@ func (*Handle) initStatsFMSketch4Chunk(cache *cache.StatsCache, iter *chunk.Iter
colStats.FMSketch = fms
}
}
cache.Put(table.PhysicalID, table) // put this table again since it is updated
}
}

Expand Down Expand Up @@ -413,6 +415,7 @@ func (*Handle) initStatsBuckets4Chunk(cache *cache.StatsCache, iter *chunk.Itera
}
}
hist.AppendBucketWithNDV(&lower, &upper, row.GetInt64(3), row.GetInt64(4), row.GetInt64(7))
cache.Put(tableID, table) // put this table again since it is updated
}
}

Expand All @@ -436,7 +439,8 @@ func (h *Handle) initStatsBuckets(cache *cache.StatsCache) error {
}
h.initStatsBuckets4Chunk(cache, iter)
}
for _, table := range cache.Values() {
tables := cache.Values()
for _, table := range tables {
for _, idx := range table.Indices {
for i := 1; i < idx.Len(); i++ {
idx.Buckets[i].Count += idx.Buckets[i-1].Count
Expand All @@ -449,6 +453,7 @@ func (h *Handle) initStatsBuckets(cache *cache.StatsCache) error {
}
col.PreCalculateScalar()
}
cache.Put(table.PhysicalID, table) // put this table again since it is updated
}
return nil
}
Expand All @@ -475,7 +480,6 @@ func (h *Handle) InitStatsLite(is infoschema.InfoSchema) (err error) {
if err != nil {
return errors.Trace(err)
}
cache.FreshMemUsage()
h.updateStatsCache(cache)
return nil
}
Expand Down Expand Up @@ -530,7 +534,6 @@ func (h *Handle) InitStats(is infoschema.InfoSchema) (err error) {
}
}
}
cache.FreshMemUsage()
h.updateStatsCache(cache)
return nil
}
Expand Down
10 changes: 0 additions & 10 deletions statistics/handle/cache/statscacheinner.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,16 +113,6 @@ func (sc *StatsCache) Values() []*statistics.Table {
return sc.c.Values()
}

// FreshMemUsage refreshes the memory usage of the cache.
// Values in StatsCache should be read-only, but when initializing the cache, some values can be modified.
// To make the memory cost more accurate, we need to refresh the memory usage of the cache after finishing the initialization.
func (sc *StatsCache) FreshMemUsage() {
values := sc.c.Values()
for _, v := range values {
sc.c.Put(v.PhysicalID, v)
}
}

// Cost returns the memory usage of the cache.
func (sc *StatsCache) Cost() int64 {
return sc.c.Cost()
Expand Down
2 changes: 1 addition & 1 deletion statistics/handle/handletest/statstest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 7,
shard_count = 8,
deps = [
"//config",
"//parser/model",
Expand Down
40 changes: 40 additions & 0 deletions statistics/handle/handletest/statstest/stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
package statstest

import (
"fmt"
"testing"
"time"

Expand Down Expand Up @@ -157,6 +158,45 @@ func TestStatsStoreAndLoad(t *testing.T) {
internal.AssertTableEqual(t, statsTbl1, statsTbl2)
}

func testInitStatsMemTrace(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t1 (a int, b int, c int, primary key(a), key idx(b))")
tk.MustExec("insert into t1 values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,7,8)")
tk.MustExec("analyze table t1")
for i := 2; i < 10; i++ {
tk.MustExec(fmt.Sprintf("create table t%v (a int, b int, c int, primary key(a), key idx(b))", i))
tk.MustExec(fmt.Sprintf("insert into t%v select * from t1", i))
tk.MustExec(fmt.Sprintf("analyze table t%v", i))
}
h := dom.StatsHandle()
is := dom.InfoSchema()
h.Clear()
require.NoError(t, h.InitStats(is))

var memCostTot int64
for i := 1; i < 10; i++ {
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i)))
require.NoError(t, err)
tStats := h.GetTableStats(tbl.Meta())
memCostTot += tStats.MemoryUsage().TotalMemUsage
}

require.Equal(t, h.GetMemConsumed(), memCostTot)
}

func TestInitStatsMemTrace(t *testing.T) {
originValue := config.GetGlobalConfig().Performance.LiteInitStats
defer func() {
config.GetGlobalConfig().Performance.LiteInitStats = originValue
}()
for _, v := range []bool{false, true} {
config.GetGlobalConfig().Performance.LiteInitStats = v
testInitStatsMemTrace(t)
}
}

func TestInitStats(t *testing.T) {
originValue := config.GetGlobalConfig().Performance.LiteInitStats
defer func() {
Expand Down

0 comments on commit bd59ea3

Please sign in to comment.