Skip to content

Commit

Permalink
config: enable-stats-cache-mem-quota is set to enable by default (#46130
Browse files Browse the repository at this point in the history
)

close #46158
  • Loading branch information
hawkingrei authored Aug 30, 2023
1 parent a84159f commit 99738a3
Show file tree
Hide file tree
Showing 36 changed files with 338 additions and 30 deletions.
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -998,7 +998,7 @@ var defaultConf = Config{
StatsLoadQueueSize: 1000,
AnalyzePartitionConcurrencyQuota: 16,
PlanReplayerDumpWorkerConcurrency: 1,
EnableStatsCacheMemQuota: false,
EnableStatsCacheMemQuota: true,
RunAutoAnalyze: true,
EnableLoadFMSketch: false,
LiteInitStats: true,
Expand Down
15 changes: 15 additions & 0 deletions planner/cascades/optimize_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ import (
func TestImplGroupZeroCost(t *testing.T) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)

Expand All @@ -57,6 +60,9 @@ func TestImplGroupZeroCost(t *testing.T) {
func TestInitGroupSchema(t *testing.T) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)

Expand All @@ -79,6 +85,9 @@ func TestInitGroupSchema(t *testing.T) {
func TestFillGroupStats(t *testing.T) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)

Expand All @@ -100,6 +109,9 @@ func TestFillGroupStats(t *testing.T) {
func TestPreparePossibleProperties(t *testing.T) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)
optimizer := NewOptimizer()
Expand Down Expand Up @@ -192,6 +204,9 @@ func (rule *fakeTransformation) OnTransform(old *memo.ExprIter) (newExprs []*mem
func TestAppliedRuleSet(t *testing.T) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)
optimizer := NewOptimizer()
Expand Down
3 changes: 3 additions & 0 deletions planner/cascades/stringer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,9 @@ func TestGroupStringer(t *testing.T) {

p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)
for i, sql := range input {
Expand Down
6 changes: 6 additions & 0 deletions planner/cascades/transformation_rules_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,9 @@ func testGroupToString(t *testing.T, input []string, output []struct {
}, optimizer *Optimizer) {
p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)

Expand Down Expand Up @@ -84,6 +87,9 @@ func TestAggPushDownGather(t *testing.T) {

p := parser.New()
ctx := plannercore.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()})
domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is)
for i, sql := range input {
Expand Down
2 changes: 2 additions & 0 deletions planner/core/collect_column_stats_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,7 @@ func TestCollectPredicateColumns(t *testing.T) {
}

s := createPlannerSuite()
defer s.Close()
ctx := context.Background()
for _, tt := range tests {
comment := fmt.Sprintf("sql: %s", tt.sql)
Expand Down Expand Up @@ -325,6 +326,7 @@ func TestCollectHistNeededColumns(t *testing.T) {
}

s := createPlannerSuite()
defer s.Close()
ctx := context.Background()
for _, tt := range tests {
comment := fmt.Sprintf("sql: %s", tt.sql)
Expand Down
6 changes: 5 additions & 1 deletion planner/core/exhaust_physical_plans_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"fmt"
"testing"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
Expand Down Expand Up @@ -56,7 +57,10 @@ type indexJoinContext struct {

func prepareForAnalyzeLookUpFilters() *indexJoinContext {
ctx := MockContext()

defer func() {
do := domain.GetDomain(ctx)
do.StatsHandle().Close()
}()
ctx.GetSessionVars().PlanID.Store(-1)
joinNode := LogicalJoin{}.Init(ctx, 0)
dataSourceNode := DataSource{}.Init(ctx, 0)
Expand Down
13 changes: 12 additions & 1 deletion planner/core/expression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"fmt"
"testing"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
Expand All @@ -42,6 +43,9 @@ type testCase struct {

func runTests(t *testing.T, tests []testCase) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
for _, tt := range tests {
expr := parseExpr(t, tt.exprStr)
val, err := evalAstExpr(ctx, expr)
Expand Down Expand Up @@ -91,6 +95,10 @@ func TestCaseWhen(t *testing.T) {
WhenClauses: []*ast.WhenClause{whenClause},
}
ctx := MockContext()
defer func() {
do := domain.GetDomain(ctx)
do.StatsHandle().Close()
}()
v, err := evalAstExpr(ctx, caseExpr)
require.NoError(t, err)
require.Equal(t, types.NewDatum(int64(1)), v)
Expand All @@ -109,7 +117,10 @@ func TestCast(t *testing.T) {
}

ctx := MockContext()

defer func() {
do := domain.GetDomain(ctx)
do.StatsHandle().Close()
}()
ast.SetFlag(expr)
v, err := evalAstExpr(ctx, expr)
require.NoError(t, err)
Expand Down
10 changes: 10 additions & 0 deletions planner/core/find_best_task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"fmt"
"testing"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -135,6 +136,9 @@ func (p *mockPhysicalPlan4Test) MemoryUsage() (sum int64) {

func TestCostOverflow(t *testing.T) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
// Plan Tree: mockPlan -> mockDataSource
mockPlan := mockLogicalPlan4Test{costOverflow: true}.Init(ctx)
mockDS := mockDataSource{}.Init(ctx)
Expand All @@ -149,6 +153,9 @@ func TestCostOverflow(t *testing.T) {

func TestEnforcedProperty(t *testing.T) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
// PlanTree : mockLogicalPlan -> mockDataSource
mockPlan := mockLogicalPlan4Test{}.Init(ctx)
mockDS := mockDataSource{}.Init(ctx)
Expand Down Expand Up @@ -183,6 +190,9 @@ func TestEnforcedProperty(t *testing.T) {

func TestHintCannotFitProperty(t *testing.T) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
// PlanTree : mockLogicalPlan -> mockDataSource
mockPlan0 := mockLogicalPlan4Test{
hasHintForPlan2: true,
Expand Down
8 changes: 7 additions & 1 deletion planner/core/indexmerge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"context"
"testing"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/model"
Expand Down Expand Up @@ -62,6 +63,9 @@ func TestIndexMergePathGeneration(t *testing.T) {
indexMergeSuiteData.LoadTestCases(t, &input, &output)
ctx := context.TODO()
sctx := MockContext()
defer func() {
domain.GetDomain(sctx).StatsHandle().Close()
}()
is := infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockView()})

parser := parser.New()
Expand All @@ -71,7 +75,8 @@ func TestIndexMergePathGeneration(t *testing.T) {
require.NoErrorf(t, err, "case:%v sql:%s", i, tc)
err = Preprocess(context.Background(), sctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err)
builder, _ := NewPlanBuilder().Init(MockContext(), is, &hint.BlockHintProcessor{})
sctx := MockContext()
builder, _ := NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{})
p, err := builder.Build(ctx, stmt)
if err != nil {
testdata.OnRecord(func() {
Expand Down Expand Up @@ -102,5 +107,6 @@ func TestIndexMergePathGeneration(t *testing.T) {
output[i] = result
})
require.Equalf(t, output[i], result, "case:%v sql:%s", i, tc)
domain.GetDomain(sctx).StatsHandle().Close()
}
}
11 changes: 10 additions & 1 deletion planner/core/logical_plan_trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"fmt"
"testing"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/util/hint"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -392,8 +393,14 @@ func TestSingleRuleTraceStep(t *testing.T) {
},
},
}

restore := config.RestoreFunc()
defer restore()
config.UpdateGlobal(func(conf *config.Config) {
// if true, test will too slow to run.
conf.Performance.EnableStatsCacheMemQuota = false
})
s := createPlannerSuite()
defer s.Close()
for i, tc := range tt {
sql := tc.sql
comment := fmt.Sprintf("case:%v sql:%s", i, sql)
Expand Down Expand Up @@ -428,6 +435,8 @@ func TestSingleRuleTraceStep(t *testing.T) {
}
}
require.True(t, assert)
do := domain.GetDomain(sctx)
do.StatsHandle().Close()
}
}

Expand Down
Loading

0 comments on commit 99738a3

Please sign in to comment.