Skip to content

Commit 678b713

Browse files
authored
statistics: get right max table id when to init stats (#58280) (#58298)
close #58284
1 parent 7ce96f9 commit 678b713

File tree

4 files changed

+139
-10
lines changed

4 files changed

+139
-10
lines changed

.bazelversion

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
7.4.1

pkg/statistics/handle/bootstrap.go

Lines changed: 18 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,11 @@ const (
5151

5252
var maxTidRecord MaxTidRecord
5353

54+
// GetMaxTidRecordForTest gets the max tid record for test.
55+
func GetMaxTidRecordForTest() int64 {
56+
return maxTidRecord.tid.Load()
57+
}
58+
5459
// MaxTidRecord is to record the max tid.
5560
type MaxTidRecord struct {
5661
mu sync.Mutex
@@ -84,7 +89,7 @@ func (*Handle) initStatsMeta4Chunk(cache statstypes.StatsCache, iter *chunk.Iter
8489
maxTidRecord.mu.Lock()
8590
defer maxTidRecord.mu.Unlock()
8691
if maxTidRecord.tid.Load() < maxPhysicalID {
87-
maxTidRecord.tid.Store(physicalID)
92+
maxTidRecord.tid.Store(maxPhysicalID)
8893
}
8994
}
9095

@@ -352,7 +357,7 @@ func (h *Handle) initStatsHistogramsByPaging(is infoschema.InfoSchema, cache sta
352357
if req.NumRows() == 0 {
353358
break
354359
}
355-
h.initStatsHistograms4Chunk(is, cache, iter, isFullCache(cache, totalMemory))
360+
h.initStatsHistograms4Chunk(is, cache, iter, IsFullCacheFunc(cache, totalMemory))
356361
}
357362
return nil
358363
}
@@ -376,7 +381,7 @@ func (h *Handle) initStatsHistogramsConcurrency(is infoschema.InfoSchema, cache
376381
}
377382

378383
func (*Handle) initStatsTopN4Chunk(cache statstypes.StatsCache, iter *chunk.Iterator4Chunk, totalMemory uint64) {
379-
if isFullCache(cache, totalMemory) {
384+
if IsFullCacheFunc(cache, totalMemory) {
380385
return
381386
}
382387
affectedIndexes := make(map[*statistics.Index]struct{})
@@ -483,20 +488,20 @@ func (h *Handle) initStatsTopNByPaging(cache statstypes.StatsCache, task initsta
483488
}
484489

485490
func (h *Handle) initStatsTopNConcurrency(cache statstypes.StatsCache, totalMemory uint64) error {
486-
if isFullCache(cache, totalMemory) {
491+
if IsFullCacheFunc(cache, totalMemory) {
487492
return nil
488493
}
489494
var maxTid = maxTidRecord.tid.Load()
490495
tid := int64(0)
491496
ls := initstats.NewRangeWorker("TopN", func(task initstats.Task) error {
492-
if isFullCache(cache, totalMemory) {
497+
if IsFullCacheFunc(cache, totalMemory) {
493498
return nil
494499
}
495500
return h.initStatsTopNByPaging(cache, task, totalMemory)
496501
}, uint64(maxTid), uint64(initStatsStep), initStatsPercentageInterval)
497502
ls.LoadStats()
498503
for tid <= maxTid {
499-
if isFullCache(cache, totalMemory) {
504+
if IsFullCacheFunc(cache, totalMemory) {
500505
break
501506
}
502507
ls.SendTask(initstats.Task{
@@ -657,7 +662,7 @@ func initStatsBucketsSQLGen(isPaging bool) string {
657662
}
658663

659664
func (h *Handle) initStatsBuckets(cache statstypes.StatsCache, totalMemory uint64) error {
660-
if isFullCache(cache, totalMemory) {
665+
if IsFullCacheFunc(cache, totalMemory) {
661666
return nil
662667
}
663668
if config.GetGlobalConfig().Performance.ConcurrentlyInitStats {
@@ -728,13 +733,13 @@ func (h *Handle) initStatsBucketsByPaging(cache statstypes.StatsCache, task init
728733
}
729734

730735
func (h *Handle) initStatsBucketsConcurrency(cache statstypes.StatsCache, totalMemory uint64) error {
731-
if isFullCache(cache, totalMemory) {
736+
if IsFullCacheFunc(cache, totalMemory) {
732737
return nil
733738
}
734739
var maxTid = maxTidRecord.tid.Load()
735740
tid := int64(0)
736741
ls := initstats.NewRangeWorker("bucket", func(task initstats.Task) error {
737-
if isFullCache(cache, totalMemory) {
742+
if IsFullCacheFunc(cache, totalMemory) {
738743
return nil
739744
}
740745
return h.initStatsBucketsByPaging(cache, task)
@@ -746,7 +751,7 @@ func (h *Handle) initStatsBucketsConcurrency(cache statstypes.StatsCache, totalM
746751
EndTid: tid + initStatsStep,
747752
})
748753
tid += initStatsStep
749-
if isFullCache(cache, totalMemory) {
754+
if IsFullCacheFunc(cache, totalMemory) {
750755
break
751756
}
752757
}
@@ -851,6 +856,9 @@ func (h *Handle) InitStats(ctx context.Context, is infoschema.InfoSchema) (err e
851856
return nil
852857
}
853858

859+
// IsFullCacheFunc is whether the cache is full or not. but we can only change it when to test
860+
var IsFullCacheFunc = isFullCache
861+
854862
func isFullCache(cache statstypes.StatsCache, total uint64) bool {
855863
memQuota := variable.StatsCacheMemQuota.Load()
856864
return (uint64(cache.MemConsumed()) >= total/4) || (cache.MemConsumed() >= memQuota && memQuota != 0)
Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
load("@io_bazel_rules_go//go:def.bzl", "go_test")
2+
3+
go_test(
4+
name = "initstats_test",
5+
timeout = "short",
6+
srcs = ["load_stats_test.go"],
7+
flaky = True,
8+
deps = [
9+
"//pkg/config",
10+
"//pkg/parser/model",
11+
"//pkg/statistics/handle",
12+
"//pkg/statistics/handle/types",
13+
"//pkg/testkit",
14+
"@com_github_stretchr_testify//require",
15+
],
16+
)
Lines changed: 104 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,104 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package initstats
16+
17+
import (
18+
"context"
19+
"fmt"
20+
"testing"
21+
22+
"github.com/pingcap/tidb/pkg/config"
23+
"github.com/pingcap/tidb/pkg/parser/model"
24+
"github.com/pingcap/tidb/pkg/statistics/handle"
25+
"github.com/pingcap/tidb/pkg/statistics/handle/types"
26+
"github.com/pingcap/tidb/pkg/testkit"
27+
"github.com/stretchr/testify/require"
28+
)
29+
30+
func TestConcurrentlyInitStatsWithMemoryLimit(t *testing.T) {
31+
restore := config.RestoreFunc()
32+
defer restore()
33+
config.UpdateGlobal(func(conf *config.Config) {
34+
conf.Performance.LiteInitStats = false
35+
conf.Performance.ConcurrentlyInitStats = true
36+
})
37+
handle.IsFullCacheFunc = func(cache types.StatsCache, total uint64) bool {
38+
return true
39+
}
40+
testConcurrentlyInitStats(t)
41+
}
42+
43+
func TestConcurrentlyInitStatsWithoutMemoryLimit(t *testing.T) {
44+
restore := config.RestoreFunc()
45+
defer restore()
46+
config.UpdateGlobal(func(conf *config.Config) {
47+
conf.Performance.LiteInitStats = false
48+
conf.Performance.ConcurrentlyInitStats = true
49+
})
50+
handle.IsFullCacheFunc = func(cache types.StatsCache, total uint64) bool {
51+
return false
52+
}
53+
testConcurrentlyInitStats(t)
54+
}
55+
56+
func testConcurrentlyInitStats(t *testing.T) {
57+
store, dom := testkit.CreateMockStoreAndDomain(t)
58+
tk := testkit.NewTestKit(t, store)
59+
tk.MustExec("use test")
60+
tk.MustExec("set global tidb_analyze_column_options='ALL'")
61+
tk.MustExec("create table t1 (a int, b int, c int, primary key(c))")
62+
tk.MustExec("insert into t1 values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,7,8)")
63+
tk.MustExec("analyze table t1")
64+
for i := 2; i < 10; i++ {
65+
tk.MustExec(fmt.Sprintf("create table t%v (a int, b int, c int, primary key(c))", i))
66+
tk.MustExec(fmt.Sprintf("insert into t%v select * from t1", i))
67+
tk.MustExec(fmt.Sprintf("analyze table t%v all columns", i))
68+
}
69+
h := dom.StatsHandle()
70+
is := dom.InfoSchema()
71+
h.Clear()
72+
require.Equal(t, h.MemConsumed(), int64(0))
73+
require.NoError(t, h.InitStats(context.Background(), is))
74+
for i := 1; i < 10; i++ {
75+
tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i)))
76+
require.NoError(t, err)
77+
stats, ok := h.StatsCache.Get(tbl.Meta().ID)
78+
require.True(t, ok)
79+
for _, col := range stats.GetColSlice() {
80+
require.True(t, col.IsAllEvicted())
81+
require.False(t, col.IsFullLoad())
82+
}
83+
}
84+
for i := 1; i < 10; i++ {
85+
tk.MustQuery(fmt.Sprintf("explain select * from t%v where a = 1", i)).CheckNotContain("pseudo")
86+
}
87+
for i := 1; i < 10; i++ {
88+
tk.MustQuery(fmt.Sprintf("explain select * from t%v where b = 1", i)).CheckNotContain("pseudo")
89+
}
90+
for i := 1; i < 10; i++ {
91+
tk.MustQuery(fmt.Sprintf("explain select * from t%v where c >= 1", i)).CheckNotContain("pseudo")
92+
}
93+
for i := 1; i < 10; i++ {
94+
tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i)))
95+
require.NoError(t, err)
96+
stats, ok := h.StatsCache.Get(tbl.Meta().ID)
97+
require.True(t, ok)
98+
for _, col := range stats.GetColSlice() {
99+
require.True(t, col.IsFullLoad())
100+
require.False(t, col.IsAllEvicted())
101+
}
102+
}
103+
require.Equal(t, int64(126), handle.GetMaxTidRecordForTest())
104+
}

0 commit comments

Comments
 (0)