Skip to content

Commit

Permalink
statistics: enhance error logging for histogram stats retrieval and a…
Browse files Browse the repository at this point in the history
…dd test for index stats loading

Signed-off-by: Rustin170506 <techregister@pm.me>
  • Loading branch information
Rustin170506 committed Feb 20, 2025
1 parent 0576380 commit e9a6da8
Show file tree
Hide file tree
Showing 3 changed files with 91 additions and 5 deletions.
19 changes: 14 additions & 5 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -1135,8 +1135,8 @@ func (h *Handle) loadNeededColumnHistograms(reader *statsReader, col model.Table
return errors.Trace(err)
}
if len(rows) == 0 {
logutil.BgLogger().Error("fail to get stats version for this histogram", zap.Int64("table_id", col.TableID), zap.Int64("hist_id", col.ID))
return errors.Trace(fmt.Errorf("fail to get stats version for this histogram, table_id:%v, hist_id:%v", col.TableID, col.ID))
logutil.BgLogger().Error("fail to get stats version for this histogram, normally this wouldn't happen, please check if this column or index has a histogram record in `mysql.stats_histogram`", zap.Int64("table_id", col.TableID), zap.Int64("column_id", col.ID))
return errors.Trace(fmt.Errorf("fail to get stats version for this histogram, normally this wouldn't happen, please check if this column or index has a histogram record in `mysql.stats_histogram`, table_id:%v, column_id:%v", col.TableID, col.ID))
}
statsVer := rows[0].GetInt64(0)
colHist := &statistics.Column{
Expand Down Expand Up @@ -1173,8 +1173,17 @@ func (h *Handle) loadNeededIndexHistograms(reader *statsReader, idx model.TableI
if !ok {
return nil
}
// Double check if the index is really needed to load.
index, ok := tbl.Indices[idx.ID]
if !ok {
// If we don't do this it might cause a memory leak.
// See: https://github.com/pingcap/tidb/issues/54022
if !ok || !index.IsLoadNeeded() {
if !index.IsLoadNeeded() {
logutil.BgLogger().Warn(
"Although the index stats is not required to load, an attempt is still made to load it, skip it",
zap.Int64("table_id", idx.TableID), zap.Int64("hist_id", idx.ID),
)
}
statistics.HistogramNeededItems.Delete(idx)
return nil
}
Expand All @@ -1198,8 +1207,8 @@ func (h *Handle) loadNeededIndexHistograms(reader *statsReader, idx model.TableI
return errors.Trace(err)
}
if len(rows) == 0 {
logutil.BgLogger().Error("fail to get stats version for this histogram", zap.Int64("table_id", idx.TableID), zap.Int64("hist_id", idx.ID))
return errors.Trace(fmt.Errorf("fail to get stats version for this histogram, table_id:%v, hist_id:%v", idx.TableID, idx.ID))
logutil.BgLogger().Error("fail to get stats version for this histogram, normally this wouldn't happen, please check if this column or index has a histogram record in `mysql.stats_histogram`", zap.Int64("table_id", idx.TableID), zap.Int64("index_id", idx.ID))
return errors.Trace(fmt.Errorf("fail to get stats version for this histogram, normally this wouldn't happen, please check if this column or index has a histogram record in `mysql.stats_histogram`, table_id:%v, index_id:%v", idx.TableID, idx.ID))
}
idxHist := &statistics.Index{Histogram: *hg, CMSketch: cms, TopN: topN, FMSketch: fms,
Info: index.Info, ErrorRate: index.ErrorRate, StatsVer: rows[0].GetInt64(0),
Expand Down
2 changes: 2 additions & 0 deletions tests/realtikvtest/statisticstest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ go_test(
flaky = True,
race = "on",
deps = [
"//parser/model",
"//statistics",
"//statistics/handle",
"//testkit",
"//tests/realtikvtest",
Expand Down
75 changes: 75 additions & 0 deletions tests/realtikvtest/statisticstest/statistics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,13 @@
package statisticstest

import (
"context"
"fmt"
"testing"
"time"

"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/tests/realtikvtest"
Expand Down Expand Up @@ -221,3 +225,74 @@ func checkFMSketch(tk *testkit.TestKit) {
tk.MustQuery(`SHOW STATS_HISTOGRAMS WHERE TABLE_NAME='employees' and partition_name="global" and column_name="id"`).CheckAt([]int{6}, [][]any{
{"14"}})
}

func TestNoNeedIndexStatsLoading(t *testing.T) {
store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists t;")
// 1. Create a table and the statsHandle.Update(do.InfoSchema()) will load this table into the stats cache.
tk.MustExec("create table if not exists t(a int, b int, index ia(a));")
// 2. Drop the stats of the stats, it will clean up all system table records for this table.
tk.MustExec("drop stats t;")
// 3. Insert some data and wait for the modify_count and the count is not null in the mysql.stats_meta.
tk.MustExec("insert into t value(1,1), (2,2);")
h := dom.StatsHandle()
require.NoError(t, h.DumpStatsDeltaToKV(true))
require.Eventually(t, func() bool {
rows := tk.MustQuery("show stats_meta").Rows()
return len(rows) > 0
}, 1*time.Minute, 2*time.Millisecond)
require.NoError(t, h.Update(dom.InfoSchema()))
// 4. Try to select some data from this table by ID, it would trigger an async load.
tk.MustExec("set tidb_opt_objective='determinate';")
tk.MustQuery("select * from t where a = 1 and b = 1;").Check(testkit.Rows("1 1"))
table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
checkTableIDInItems(t, table.Meta().ID)
}

func checkTableIDInItems(t *testing.T, tableID int64) {
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute)
defer cancel()

ticker := time.NewTicker(2 * time.Millisecond)
defer ticker.Stop()

done := make(chan bool)

// First, confirm that the table ID is in the items.
items := statistics.HistogramNeededItems.AllItems()
for _, item := range items {
if item.TableID == tableID {
// Then, continuously check until it no longer exists or timeout.
go func() {
for {
select {
case <-ticker.C:
items := statistics.HistogramNeededItems.AllItems()
found := false
for _, item := range items {
if item.TableID == tableID {
found = true
}
}
if !found {
done <- true
}
case <-ctx.Done():
return
}
}
}()
break
}
}

select {
case <-done:
t.Log("Table ID has been removed from items")
case <-ctx.Done():
t.Fatal("Timeout: Table ID was not removed from items within the time limit")
}
}

0 comments on commit e9a6da8

Please sign in to comment.