From 3c599cd04777a0b26f1856880a428d2e1fff1bc7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Tue, 18 Jun 2024 16:23:47 +0800 Subject: [PATCH] statistics: do not load unnecessary index statistics (#54060) close pingcap/tidb#54022 --- pkg/statistics/handle/handle_hist.go | 4 +- pkg/statistics/handle/storage/read.go | 19 +++-- tests/realtikvtest/statisticstest/BUILD.bazel | 2 + .../statisticstest/statistics_test.go | 75 +++++++++++++++++++ 4 files changed, 93 insertions(+), 7 deletions(-) diff --git a/pkg/statistics/handle/handle_hist.go b/pkg/statistics/handle/handle_hist.go index 3a55bcc1e9dcd..2b1763a8bcd14 100644 --- a/pkg/statistics/handle/handle_hist.go +++ b/pkg/statistics/handle/handle_hist.go @@ -368,9 +368,9 @@ func (*Handle) readStatsForOneItem(sctx sessionctx.Context, item model.TableItem return nil, errors.Trace(err) } if len(rows) == 0 { - logutil.BgLogger().Error("fail to get stats version for this histogram", zap.Int64("table_id", item.TableID), + 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", item.TableID), zap.Int64("hist_id", item.ID), zap.Bool("is_index", item.IsIndex)) - return nil, errors.Trace(fmt.Errorf("fail to get stats version for this histogram, table_id:%v, hist_id:%v, is_index:%v", item.TableID, item.ID, item.IsIndex)) + return nil, 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, hist_id:%v, is_index:%v", item.TableID, item.ID, item.IsIndex)) } statsVer := rows[0].GetInt64(0) if item.IsIndex { diff --git a/pkg/statistics/handle/storage/read.go b/pkg/statistics/handle/storage/read.go index bacf43abef60f..3947efe424f66 100644 --- a/pkg/statistics/handle/storage/read.go +++ b/pkg/statistics/handle/storage/read.go @@ -540,8 +540,8 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache util.StatsCa 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{ @@ -576,7 +576,16 @@ func loadNeededIndexHistograms(sctx sessionctx.Context, statsCache util.StatsCac return nil } index, ok := tbl.Indices[idx.ID] - if !ok { + // Double check if the index is really needed to load. + // 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 } @@ -600,8 +609,8 @@ func loadNeededIndexHistograms(sctx sessionctx.Context, statsCache util.StatsCac 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, StatsVer: rows[0].GetInt64(0), diff --git a/tests/realtikvtest/statisticstest/BUILD.bazel b/tests/realtikvtest/statisticstest/BUILD.bazel index b9ee7985a9952..7130eb10ca422 100644 --- a/tests/realtikvtest/statisticstest/BUILD.bazel +++ b/tests/realtikvtest/statisticstest/BUILD.bazel @@ -10,6 +10,8 @@ go_test( flaky = True, race = "on", deps = [ + "//pkg/parser/model", + "//pkg/statistics", "//pkg/testkit", "//tests/realtikvtest", "@com_github_stretchr_testify//require", diff --git a/tests/realtikvtest/statisticstest/statistics_test.go b/tests/realtikvtest/statisticstest/statistics_test.go index 5535a387e73d4..0f488afc692e2 100644 --- a/tests/realtikvtest/statisticstest/statistics_test.go +++ b/tests/realtikvtest/statisticstest/statistics_test.go @@ -15,9 +15,13 @@ package statisticstest import ( + "context" "fmt" "testing" + "time" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" @@ -232,3 +236,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") + } +}