From 8a2047ea267cfa0036c678130eee595e1868b16a Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 16 Dec 2024 23:54:51 +0800 Subject: [PATCH] statistics: get right max table id when to init stats (#58280) (#58297) close pingcap/tidb#58284 --- .bazelversion | 1 + pkg/statistics/handle/bootstrap.go | 7 +- .../handle/handletest/initstats/BUILD.bazel | 15 ++++ .../handletest/initstats/load_stats_test.go | 85 +++++++++++++++++++ 4 files changed, 107 insertions(+), 1 deletion(-) create mode 100644 .bazelversion create mode 100644 pkg/statistics/handle/handletest/initstats/BUILD.bazel create mode 100644 pkg/statistics/handle/handletest/initstats/load_stats_test.go diff --git a/.bazelversion b/.bazelversion new file mode 100644 index 0000000000000..815da58b7a9ed --- /dev/null +++ b/.bazelversion @@ -0,0 +1 @@ +7.4.1 diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 272078ca764ff..885db2cf17fae 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -44,6 +44,11 @@ const initStatsStep = int64(500) var maxTidRecord MaxTidRecord +// GetMaxTidRecordForTest gets the max tid record for test. +func GetMaxTidRecordForTest() int64 { + return maxTidRecord.tid.Load() +} + // MaxTidRecord is to record the max tid. type MaxTidRecord struct { mu sync.Mutex @@ -81,7 +86,7 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache statstypes. maxTidRecord.mu.Lock() defer maxTidRecord.mu.Unlock() if maxTidRecord.tid.Load() < maxPhysicalID { - maxTidRecord.tid.Store(physicalID) + maxTidRecord.tid.Store(maxPhysicalID) } } diff --git a/pkg/statistics/handle/handletest/initstats/BUILD.bazel b/pkg/statistics/handle/handletest/initstats/BUILD.bazel new file mode 100644 index 0000000000000..9bed8ddf07d5d --- /dev/null +++ b/pkg/statistics/handle/handletest/initstats/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "initstats_test", + timeout = "short", + srcs = ["load_stats_test.go"], + flaky = True, + deps = [ + "//pkg/config", + "//pkg/parser/model", + "//pkg/statistics/handle", + "//pkg/testkit", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/statistics/handle/handletest/initstats/load_stats_test.go b/pkg/statistics/handle/handletest/initstats/load_stats_test.go new file mode 100644 index 0000000000000..f425b230596c1 --- /dev/null +++ b/pkg/statistics/handle/handletest/initstats/load_stats_test.go @@ -0,0 +1,85 @@ +// Copyright 2024 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 initstats + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/statistics/handle" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func TestConcurrentlyInitStatsWithoutMemoryLimit(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.LiteInitStats = false + conf.Performance.ConcurrentlyInitStats = true + }) + testConcurrentlyInitStats(t) +} + +func testConcurrentlyInitStats(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(c))") + 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(c))", i)) + tk.MustExec(fmt.Sprintf("insert into t%v select * from t1", i)) + tk.MustExec(fmt.Sprintf("analyze table t%v all columns", i)) + } + h := dom.StatsHandle() + is := dom.InfoSchema() + h.Clear() + require.Equal(t, h.MemConsumed(), int64(0)) + require.NoError(t, h.InitStats(is)) + for i := 1; i < 10; i++ { + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i))) + require.NoError(t, err) + stats, ok := h.StatsCache.Get(tbl.Meta().ID) + require.True(t, ok) + for _, col := range stats.Columns { + require.True(t, col.IsAllEvicted()) + require.False(t, col.IsFullLoad()) + } + } + for i := 1; i < 10; i++ { + tk.MustQuery(fmt.Sprintf("explain select * from t%v where a = 1", i)).CheckNotContain("pseudo") + } + for i := 1; i < 10; i++ { + tk.MustQuery(fmt.Sprintf("explain select * from t%v where b = 1", i)).CheckNotContain("pseudo") + } + for i := 1; i < 10; i++ { + tk.MustQuery(fmt.Sprintf("explain select * from t%v where c >= 1", i)).CheckNotContain("pseudo") + } + for i := 1; i < 10; i++ { + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i))) + require.NoError(t, err) + stats, ok := h.StatsCache.Get(tbl.Meta().ID) + require.True(t, ok) + for _, col := range stats.Columns { + require.True(t, col.IsFullLoad()) + require.False(t, col.IsAllEvicted()) + } + } + require.Equal(t, int64(120), handle.GetMaxTidRecordForTest()) +}