diff --git a/config/config.go b/config/config.go index 0417287b7067c..c130630b3f687 100644 --- a/config/config.go +++ b/config/config.go @@ -998,7 +998,7 @@ var defaultConf = Config{ StatsLoadQueueSize: 1000, AnalyzePartitionConcurrencyQuota: 16, PlanReplayerDumpWorkerConcurrency: 1, - EnableStatsCacheMemQuota: false, + EnableStatsCacheMemQuota: true, RunAutoAnalyze: true, EnableLoadFMSketch: false, LiteInitStats: true, diff --git a/planner/cascades/optimize_test.go b/planner/cascades/optimize_test.go index 104e1bd59b3e2..55727427e132e 100644 --- a/planner/cascades/optimize_test.go +++ b/planner/cascades/optimize_test.go @@ -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) @@ -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) @@ -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) @@ -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() @@ -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() diff --git a/planner/cascades/stringer_test.go b/planner/cascades/stringer_test.go index 2c28c4cf8e52a..5fe587b890518 100644 --- a/planner/cascades/stringer_test.go +++ b/planner/cascades/stringer_test.go @@ -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 { diff --git a/planner/cascades/transformation_rules_test.go b/planner/cascades/transformation_rules_test.go index a57100a253d92..fe47adbbee379 100644 --- a/planner/cascades/transformation_rules_test.go +++ b/planner/cascades/transformation_rules_test.go @@ -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) @@ -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 { diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 249f210050c56..82844ce4def83 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -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) @@ -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) diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index ae3564dd27755..06a5ea12a56fe 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -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" @@ -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) diff --git a/planner/core/expression_test.go b/planner/core/expression_test.go index 93ba50d2d5011..1057461e60b17 100644 --- a/planner/core/expression_test.go +++ b/planner/core/expression_test.go @@ -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" @@ -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) @@ -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) @@ -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) diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index 7eac1e0f7e6d5..9b7caa3baef5a 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -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" @@ -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) @@ -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) @@ -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, diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index 44c3e123b30cd..58e739e096161 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -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" @@ -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() @@ -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() { @@ -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() } } diff --git a/planner/core/logical_plan_trace_test.go b/planner/core/logical_plan_trace_test.go index 7233b49cb24e1..da14575e93256 100644 --- a/planner/core/logical_plan_trace_test.go +++ b/planner/core/logical_plan_trace_test.go @@ -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" @@ -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) @@ -428,6 +435,8 @@ func TestSingleRuleTraceStep(t *testing.T) { } } require.True(t, assert) + do := domain.GetDomain(sctx) + do.StatsHandle().Close() } } diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index 164ed95f2dae8..78898e96fe7fc 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -21,6 +21,7 @@ import ( "strings" "testing" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" @@ -80,10 +81,15 @@ func createPlannerSuite() (s *plannerSuite) { return } +func (p *plannerSuite) Close() { + domain.GetDomain(p.ctx).StatsHandle().Close() +} + func TestPredicatePushDown(t *testing.T) { var input, output []string planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for ith, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -106,6 +112,7 @@ func TestImplicitCastNotNullFlag(t *testing.T) { ca := "select count(*) from t3 group by a having bit_and(b) > 1;" comment := fmt.Sprintf("for %s", ca) s := createPlannerSuite() + defer s.Close() stmt, err := s.p.ParseOneStmt(ca, "", "") require.NoError(t, err, comment) p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) @@ -123,6 +130,7 @@ func TestEliminateProjectionUnderUnion(t *testing.T) { ca := "Select a from t3 join ( (select 127 as IDD from t3) union all (select 1 as IDD from t3) ) u on t3.b = u.IDD;" comment := fmt.Sprintf("for %s", ca) s := createPlannerSuite() + defer s.Close() stmt, err := s.p.ParseOneStmt(ca, "", "") require.NoError(t, err, comment) p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) @@ -146,6 +154,7 @@ func TestJoinPredicatePushDown(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -185,6 +194,7 @@ func TestOuterWherePredicatePushDown(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -230,6 +240,7 @@ func TestSimplifyOuterJoin(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -270,6 +281,7 @@ func TestAntiSemiJoinConstFalse(t *testing.T) { } s := createPlannerSuite() + defer s.Close() ctx := context.Background() for _, ca := range tests { comment := fmt.Sprintf("for %s", ca.sql) @@ -297,6 +309,7 @@ func TestDeriveNotNullConds(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -326,6 +339,7 @@ func TestDeriveNotNullConds(t *testing.T) { func TestExtraPKNotNullFlag(t *testing.T) { sql := "select count(*) from t3" s := createPlannerSuite() + defer s.Close() ctx := context.Background() comment := fmt.Sprintf("for %s", sql) stmt, err := s.p.ParseOneStmt(sql, "", "") @@ -396,6 +410,7 @@ func TestGroupByWhenNotExistCols(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() for _, test := range sqlTests { sql := test.sql p, err := buildLogicPlan4GroupBy(s, t, sql) @@ -409,6 +424,7 @@ func TestDupRandJoinCondsPushDown(t *testing.T) { sql := "select * from t as t1 join t t2 on t1.a > rand() and t1.a > rand()" comment := fmt.Sprintf("for %s", sql) s := createPlannerSuite() + defer s.Close() stmt, err := s.p.ParseOneStmt(sql, "", "") require.NoError(t, err, comment) p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) @@ -472,6 +488,7 @@ func TestTablePartition(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, ca := range input { comment := fmt.Sprintf("for %s", ca.SQL) @@ -497,6 +514,7 @@ func TestSubquery(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for ith, ca := range input { comment := fmt.Sprintf("for %s", ca) @@ -523,6 +541,7 @@ func TestPlanBuilder(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() s.ctx.GetSessionVars().CostModelVersion = modelVer1 ctx := context.Background() for i, ca := range input { @@ -551,6 +570,7 @@ func TestJoinReOrder(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, tt := range input { comment := fmt.Sprintf("for %s", tt) @@ -575,6 +595,7 @@ func TestEagerAggregation(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() s.ctx.GetSessionVars().AllowAggPushDown = true defer func() { @@ -604,6 +625,7 @@ func TestColumnPruning(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, tt := range input { comment := fmt.Sprintf("case:%v sql:\"%s\"", i, tt) @@ -632,6 +654,7 @@ func TestSortByItemsPruning(t *testing.T) { }) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, tt := range input { comment := fmt.Sprintf("for %s", tt) @@ -661,6 +684,7 @@ func TestProjectionEliminator(t *testing.T) { } s := createPlannerSuite() + defer s.Close() ctx := context.Background() for ith, tt := range tests { comment := fmt.Sprintf("for %s", tt.sql) @@ -677,6 +701,7 @@ func TestProjectionEliminator(t *testing.T) { func TestCS3389(t *testing.T) { s := createPlannerSuite() + defer s.Close() ctx := context.Background() stmt, err := s.p.ParseOneStmt("select count(*) from t where a in (select b from t2 where a is null);", "", "") require.NoError(t, err) @@ -699,6 +724,9 @@ func TestCS3389(t *testing.T) { func TestAllocID(t *testing.T) { ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() pA := DataSource{}.Init(ctx, 0) pB := DataSource{}.Init(ctx, 0) require.Equal(t, pB.ID(), pA.ID()+1) @@ -922,6 +950,7 @@ func TestValidate(t *testing.T) { } s := createPlannerSuite() + defer s.Close() ctx := context.Background() for _, tt := range tests { sql := tt.sql @@ -975,6 +1004,7 @@ func TestUniqueKeyInfo(t *testing.T) { }) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for ith, tt := range input { comment := fmt.Sprintf("for %s %d", tt, ith) @@ -997,6 +1027,7 @@ func TestAggPrune(t *testing.T) { planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.Background() for i, tt := range input { comment := fmt.Sprintf("for %s", tt) @@ -1415,8 +1446,14 @@ func TestVisitInfo(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 _, tt := range tests { comment := fmt.Sprintf("for %s", tt.sql) stmt, err := s.p.ParseOneStmt(tt.sql, "", "") @@ -1432,6 +1469,8 @@ func TestVisitInfo(t *testing.T) { require.NoError(t, err, comment) checkVisitInfo(t, builder.visitInfo, tt.ans, comment) + + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -1498,6 +1537,7 @@ func TestUnion(t *testing.T) { } planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range input { comment := fmt.Sprintf("case:%v sql:%s", i, tt) @@ -1514,6 +1554,7 @@ func TestUnion(t *testing.T) { }) if output[i].Err { require.Error(t, err) + domain.GetDomain(sctx).StatsHandle().Close() continue } require.NoError(t, err, comment) @@ -1524,13 +1565,21 @@ func TestUnion(t *testing.T) { }) require.NoError(t, err) require.Equal(t, output[i].Best, ToString(p), comment) + domain.GetDomain(sctx).StatsHandle().Close() } } func TestTopNPushDown(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 + }) var input, output []string planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range input { comment := fmt.Sprintf("case:%v sql:%s", i, tt) @@ -1549,6 +1598,8 @@ func TestTopNPushDown(t *testing.T) { output[i] = ToString(p) }) require.Equal(t, output[i], ToString(p), comment) + + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -1587,6 +1638,7 @@ func TestNameResolver(t *testing.T) { } s := createPlannerSuite() + defer s.Close() ctx := context.Background() for _, test := range tests { comment := fmt.Sprintf("for %s", test.sql) @@ -1604,10 +1656,17 @@ func TestNameResolver(t *testing.T) { } func TestOuterJoinEliminator(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 + }) var input, output []string planSuiteUnexportedData.LoadTestCases(t, &input, &output) s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range input { comment := fmt.Sprintf("case:%v sql:%s", i, tt) @@ -1627,6 +1686,8 @@ func TestOuterJoinEliminator(t *testing.T) { output[i] = planString }) require.Equal(t, output[i], planString, comment) + + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -1645,6 +1706,7 @@ func TestSelectView(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range tests { comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) @@ -1652,12 +1714,14 @@ func TestSelectView(t *testing.T) { require.NoError(t, err, comment) err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) require.NoError(t, err) - builder, _ := NewPlanBuilder().Init(MockContext(), s.is, &hint.BlockHintProcessor{}) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) require.NoError(t, err) p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) require.NoError(t, err) require.Equal(t, tt.best, ToString(p), comment) + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -1667,6 +1731,12 @@ type plannerSuiteWithOptimizeVars struct { } func TestWindowFunction(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 := new(plannerSuiteWithOptimizeVars) s.plannerSuite = createPlannerSuite() @@ -1683,9 +1753,14 @@ func TestWindowFunction(t *testing.T) { } func TestWindowParallelFunction(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 := new(plannerSuiteWithOptimizeVars) s.plannerSuite = createPlannerSuite() - s.optimizeVars = map[string]string{ variable.TiDBWindowConcurrency: "4", variable.TiDBCostModelVersion: "1", @@ -1739,6 +1814,9 @@ func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) } sctx := MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() for k, v := range s.optimizeVars { if err = sctx.GetSessionVars().SetSystemVar(k, v); err != nil { return nil, nil, err @@ -1837,6 +1915,7 @@ func TestSkylinePruning(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range tests { comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) @@ -1850,6 +1929,7 @@ func TestSkylinePruning(t *testing.T) { p, err := builder.Build(ctx, stmt) if err != nil { require.EqualError(t, err, tt.result, comment) + domain.GetDomain(sctx).StatsHandle().Close() continue } require.NoError(t, err, comment) @@ -1884,6 +1964,7 @@ func TestSkylinePruning(t *testing.T) { } paths := ds.skylinePruning(byItemsToProperty(byItems)) require.Equal(t, tt.result, pathsName(paths), comment) + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -1911,6 +1992,7 @@ func TestFastPlanContextTables(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() s.ctx.GetSessionVars().SnapshotInfoschema = s.is for _, tt := range tests { stmt, err := s.p.ParseOneStmt(tt.sql, "", "") @@ -1942,6 +2024,7 @@ func TestUpdateEQCond(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range tests { comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) @@ -1957,18 +2040,23 @@ func TestUpdateEQCond(t *testing.T) { p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) require.NoError(t, err) require.Equal(t, tt.best, ToString(p), comment) + domain.GetDomain(sctx).StatsHandle().Close() } } func TestConflictedJoinTypeHints(t *testing.T) { sql := "select /*+ INL_JOIN(t1) HASH_JOIN(t1) */ * from t t1, t t2 where t1.e = t2.e" s := createPlannerSuite() + defer s.Close() ctx := context.TODO() stmt, err := s.p.ParseOneStmt(sql, "", "") require.NoError(t, err) err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) require.NoError(t, err) sctx := MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) p, err := builder.Build(ctx, stmt) @@ -1985,6 +2073,7 @@ func TestConflictedJoinTypeHints(t *testing.T) { func TestSimplyOuterJoinWithOnlyOuterExpr(t *testing.T) { s := createPlannerSuite() + defer s.Close() sql := "select * from t t1 right join t t0 ON TRUE where CONCAT_WS(t0.e=t0.e, 0, NULL) IS NULL" ctx := context.TODO() stmt, err := s.p.ParseOneStmt(sql, "", "") @@ -1992,6 +2081,9 @@ func TestSimplyOuterJoinWithOnlyOuterExpr(t *testing.T) { err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) require.NoError(t, err) sctx := MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) p, err := builder.Build(ctx, stmt) @@ -2038,6 +2130,7 @@ func TestResolvingCorrelatedAggregate(t *testing.T) { } s := createPlannerSuite() + defer s.Close() ctx := context.TODO() for i, tt := range tests { comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) @@ -2081,6 +2174,7 @@ func TestFastPathInvalidBatchPointGet(t *testing.T) { }, } s := createPlannerSuite() + defer s.Close() for i, tc := range tt { comment := fmt.Sprintf("case:%v sql:%s", i, tc.sql) stmt, err := s.p.ParseOneStmt(tc.sql, "", "") @@ -2098,6 +2192,7 @@ func TestFastPathInvalidBatchPointGet(t *testing.T) { func TestTraceFastPlan(t *testing.T) { s := createPlannerSuite() + defer s.Close() s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true defer func() { s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = false @@ -2122,6 +2217,7 @@ func TestWindowLogicalPlanAmbiguous(t *testing.T) { // The ambiguous logical plan which contains window function can usually be found in 100 iterations. iterations := 100 s := createPlannerSuite() + defer s.Close() for i := 0; i < iterations; i++ { stmt, err := s.p.ParseOneStmt(sql, "", "") require.NoError(t, err) @@ -2159,6 +2255,7 @@ func TestRemoveOrderbyInSubquery(t *testing.T) { } s := createPlannerSuite() + defer s.Close() s.ctx.GetSessionVars().RemoveOrderbyInSubquery = true ctx := context.TODO() for i, tt := range tests { @@ -2176,6 +2273,7 @@ func TestRollupExpand(t *testing.T) { sql := "select count(a) from t group by a, b with rollup" comment := fmt.Sprintf("for %s", sql) s := createPlannerSuite() + defer s.Close() stmt, err := s.p.ParseOneStmt(sql, "", "") require.NoError(t, err, comment) diff --git a/planner/core/optimizer_test.go b/planner/core/optimizer_test.go index 59afdf9e9afba..90da2e886bce8 100644 --- a/planner/core/optimizer_test.go +++ b/planner/core/optimizer_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" @@ -163,7 +164,9 @@ func TestHandleFineGrainedShuffle(t *testing.T) { const expStreamCount int64 = 8 sctx := MockContext() sctx.GetSessionVars().TiFlashFineGrainedShuffleStreamCount = expStreamCount - + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() start := func(p PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) { handleFineGrainedShuffle(nil, sctx, tableReader) check(p, expStreamCount, expChildCount, curChildCount) @@ -401,6 +404,9 @@ func TestHandleFineGrainedShuffle(t *testing.T) { // Test for core.prunePhysicalColumns() func TestPrunePhysicalColumns(t *testing.T) { sctx := MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() col0 := &expression.Column{ UniqueID: sctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldType(mysql.TypeLonglong), diff --git a/planner/core/physical_plan_trace_test.go b/planner/core/physical_plan_trace_test.go index 5390fc20af0bd..2344ffe204b90 100644 --- a/planner/core/physical_plan_trace_test.go +++ b/planner/core/physical_plan_trace_test.go @@ -105,6 +105,7 @@ func TestPhysicalOptimizeWithTraceEnabled(t *testing.T) { require.NotNil(t, otrace) physicalList := getList(otrace) require.True(t, checkList(physicalList, testcase.physicalList)) + domain.GetDomain(sctx).StatsHandle().Close() } } @@ -148,6 +149,9 @@ func TestPhysicalOptimizerTrace(t *testing.T) { err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) require.NoError(t, err) sctx := core.MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true sctx.GetSessionVars().AllowAggPushDown = true builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) @@ -211,6 +215,9 @@ func TestPhysicalOptimizerTraceChildrenNotDuplicated(t *testing.T) { err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) require.NoError(t, err) sctx := core.MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) diff --git a/planner/core/plan_cache_lru_test.go b/planner/core/plan_cache_lru_test.go index 518252293a522..5b3e0a5309e3c 100644 --- a/planner/core/plan_cache_lru_test.go +++ b/planner/core/plan_cache_lru_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hack" @@ -50,6 +51,9 @@ func TestLRUPCPut(t *testing.T) { // test initialize mockCtx := MockContext() mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true + defer func() { + domain.GetDomain(mockCtx).StatsHandle().Close() + }() lruA := NewLRUPlanCache(0, 0, 0, mockCtx, false) require.Equal(t, lruA.capacity, uint(100)) @@ -137,6 +141,9 @@ func TestLRUPCPut(t *testing.T) { func TestLRUPCGet(t *testing.T) { mockCtx := MockContext() mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true + defer func() { + domain.GetDomain(mockCtx).StatsHandle().Close() + }() lru := NewLRUPlanCache(3, 0, 0, mockCtx, false) keys := make([]*planCacheKey, 5) @@ -202,6 +209,9 @@ func TestLRUPCGet(t *testing.T) { func TestLRUPCDelete(t *testing.T) { mockCtx := MockContext() mockCtx.GetSessionVars().EnablePlanCacheForParamLimit = true + defer func() { + domain.GetDomain(mockCtx).StatsHandle().Close() + }() lru := NewLRUPlanCache(3, 0, 0, mockCtx, false) keys := make([]*planCacheKey, 3) @@ -250,8 +260,11 @@ func TestLRUPCDelete(t *testing.T) { } func TestLRUPCDeleteAll(t *testing.T) { - lru := NewLRUPlanCache(3, 0, 0, MockContext(), false) - + ctx := MockContext() + lru := NewLRUPlanCache(3, 0, 0, ctx, false) + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() keys := make([]*planCacheKey, 3) vals := make([]*PlanCacheValue, 3) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, @@ -287,7 +300,11 @@ func TestLRUPCDeleteAll(t *testing.T) { func TestLRUPCSetCapacity(t *testing.T) { maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value) - lru := NewLRUPlanCache(5, 0, 0, MockContext(), false) + ctx := MockContext() + lru := NewLRUPlanCache(5, 0, 0, ctx, false) + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() lru.onEvict = func(key kvcache.Key, value kvcache.Value) { maxMemDroppedKv[key] = value } @@ -355,8 +372,11 @@ func TestLRUPCSetCapacity(t *testing.T) { } func TestIssue37914(t *testing.T) { - lru := NewLRUPlanCache(3, 0.1, 1, MockContext(), false) - + ctx := MockContext() + lru := NewLRUPlanCache(3, 0.1, 1, ctx, false) + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)} key := &planCacheKey{database: strconv.FormatInt(int64(1), 10)} opts := &utilpc.PlanCacheMatchOpts{ @@ -371,7 +391,11 @@ func TestIssue37914(t *testing.T) { } func TestIssue38244(t *testing.T) { - lru := NewLRUPlanCache(3, 0, 0, MockContext(), false) + ctx := MockContext() + lru := NewLRUPlanCache(3, 0, 0, ctx, false) + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() require.Equal(t, uint(3), lru.capacity) keys := make([]*planCacheKey, 5) @@ -401,6 +425,9 @@ func TestIssue38244(t *testing.T) { func TestLRUPlanCacheMemoryUsage(t *testing.T) { pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)} ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() ctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor = true lru := NewLRUPlanCache(3, 0, 0, ctx, false) evict := make(map[kvcache.Key]kvcache.Value) diff --git a/planner/core/plan_cache_param_test.go b/planner/core/plan_cache_param_test.go index a187ba892aa7c..a70f2210823dd 100644 --- a/planner/core/plan_cache_param_test.go +++ b/planner/core/plan_cache_param_test.go @@ -121,7 +121,7 @@ func TestParameterize(t *testing.T) { } func TestGetParamSQLFromASTConcurrently(t *testing.T) { - n := 100 + n := 50 sqls := make([]string, 0, n) for i := 0; i < n; i++ { sqls = append(sqls, fmt.Sprintf(`insert into t values (%d, %d, %d)`, i*3+0, i*3+1, i*3+2)) @@ -137,7 +137,7 @@ func TestGetParamSQLFromASTConcurrently(t *testing.T) { for i := 0; i < n; i++ { wg.Add(1) go func(id int) { - for i := 0; i < 1000; i++ { + for i := 0; i < 100; i++ { _, vals, err := GetParamSQLFromAST(stmts[id]) require.Nil(t, err) require.Equal(t, len(vals), 3) diff --git a/planner/core/plan_cache_utils_test.go b/planner/core/plan_cache_utils_test.go index f1332408fc2ad..93b3afbe31572 100644 --- a/planner/core/plan_cache_utils_test.go +++ b/planner/core/plan_cache_utils_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/stretchr/testify/require" @@ -30,6 +31,9 @@ func TestCacheKey(t *testing.T) { ctx.GetSessionVars().TimeZone = time.UTC ctx.GetSessionVars().ConnectionID = 0 ctx.GetSessionVars().InRestrictedSQL = false + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() variable.RestrictedReadOnly.Store(false) variable.VarTiDBSuperReadOnly.Store(false) key, err := NewPlanCacheKey(ctx.GetSessionVars(), "", "test", 1, 1, "", 0) diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index 0f51159595835..62463ae70cfbc 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -136,6 +136,9 @@ func optimize(t *testing.T, sql string, p *parser.Parser, ctx sessionctx.Context err = plannercore.Preprocess(context.Background(), ctx, stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) require.NoError(t, err) sctx := plannercore.MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true sctx.GetSessionVars().EnableNewCostInterface = true sctx.GetSessionVars().CostModelVersion = 1 diff --git a/planner/core/plan_replayer_capture_test.go b/planner/core/plan_replayer_capture_test.go index 6778cdba20bbf..b89b1e634a677 100644 --- a/planner/core/plan_replayer_capture_test.go +++ b/planner/core/plan_replayer_capture_test.go @@ -70,6 +70,9 @@ func getTableStats(sql string, t *testing.T, ctx sessionctx.Context, dom *domain sctx.GetSessionVars().EnablePlanReplayerCapture = true builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() plan, err := builder.Build(context.TODO(), stmt) require.NoError(t, err) _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 1177027aaf2c3..09a55a37d41a0 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -729,7 +730,9 @@ func TestBuildFinalModeAggregation(t *testing.T) { } ctx := core.MockContext() - + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() aggCol := &expression.Column{ Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong), diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 5b95414a05943..f08861727500b 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -23,6 +23,7 @@ import ( "unsafe" "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -118,7 +119,11 @@ func TestGetPathByIndexName(t *testing.T) { } func TestRewriterPool(t *testing.T) { - builder, _ := NewPlanBuilder().Init(MockContext(), nil, &hint.BlockHintProcessor{}) + ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() + builder, _ := NewPlanBuilder().Init(ctx, nil, &hint.BlockHintProcessor{}) // Make sure PlanBuilder.getExpressionRewriter() provides clean rewriter from pool. // First, pick one rewriter from the pool and make it dirty. @@ -166,6 +171,9 @@ func TestDisableFold(t *testing.T) { } ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() for _, c := range cases { st, err := parser.New().ParseOneStmt(c.SQL, "", "") require.NoError(t, err) @@ -645,7 +653,11 @@ func TestHandleAnalyzeOptionsV1AndV2(t *testing.T) { } func TestGetFullAnalyzeColumnsInfo(t *testing.T) { - pb, _ := NewPlanBuilder().Init(MockContext(), nil, &hint.BlockHintProcessor{}) + ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() + pb, _ := NewPlanBuilder().Init(ctx, nil, &hint.BlockHintProcessor{}) // Create a new TableName instance. tableName := &ast.TableName{ diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index 760941c552d50..926ad80194c9c 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -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/model" @@ -162,6 +163,10 @@ func TestDPReorderTPCHQ5(t *testing.T) { statsMap := makeStatsMapForTPCHQ5() ctx := MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() ctx.GetSessionVars().PlanID.Store(-1) joinGroups := make([]LogicalPlan, 0, 6) joinGroups = append(joinGroups, newDataSource(ctx, "lineitem", 59986052)) @@ -207,6 +212,9 @@ func TestDPReorderAllCartesian(t *testing.T) { statsMap := makeStatsMapForTPCHQ5() ctx := MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() ctx.GetSessionVars().PlanID.Store(-1) joinGroup := make([]LogicalPlan, 0, 4) diff --git a/planner/implementation/BUILD.bazel b/planner/implementation/BUILD.bazel index 01f47d77f27f9..da204b5b15c60 100644 --- a/planner/implementation/BUILD.bazel +++ b/planner/implementation/BUILD.bazel @@ -32,6 +32,7 @@ go_test( embed = [":implementation"], flaky = True, deps = [ + "//domain", "//planner/core", "//planner/memo", "//testkit/testsetup", diff --git a/planner/implementation/base_test.go b/planner/implementation/base_test.go index ce5ee9c299c0d..265ba730f8936 100644 --- a/planner/implementation/base_test.go +++ b/planner/implementation/base_test.go @@ -17,6 +17,7 @@ package implementation import ( "testing" + "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/memo" "github.com/stretchr/testify/require" @@ -26,6 +27,9 @@ import ( func TestBaseImplementation(t *testing.T) { defer view.Stop() sctx := plannercore.MockContext() + defer func() { + domain.GetDomain(sctx).StatsHandle().Close() + }() p := plannercore.PhysicalLimit{}.Init(sctx, nil, 0, nil) impl := &baseImpl{plan: p} require.Equal(t, p, impl.GetPlan()) diff --git a/planner/memo/expr_iterator_test.go b/planner/memo/expr_iterator_test.go index 4930371bb84f2..37282457f4413 100644 --- a/planner/memo/expr_iterator_test.go +++ b/planner/memo/expr_iterator_test.go @@ -17,6 +17,7 @@ package memo import ( "testing" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" "github.com/stretchr/testify/require" @@ -27,7 +28,10 @@ func TestNewExprIterFromGroupElem(t *testing.T) { defer view.Stop() ctx := plannercore.MockContext() schema := expression.NewSchema() - + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() g0 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{}.Init(ctx, 0)), schema) g0.Insert(NewGroupExpr(plannercore.LogicalLimit{}.Init(ctx, 0))) g0.Insert(NewGroupExpr(plannercore.LogicalProjection{}.Init(ctx, 0))) @@ -70,7 +74,10 @@ func TestExprIterNext(t *testing.T) { defer view.Stop() ctx := plannercore.MockContext() schema := expression.NewSchema() - + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() g0 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewZero()}}.Init(ctx, 0)), schema) g0.Insert(NewGroupExpr(plannercore.LogicalLimit{Count: 1}.Init(ctx, 0))) g0.Insert(NewGroupExpr(plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) @@ -118,7 +125,10 @@ func TestExprIterReset(t *testing.T) { defer view.Stop() ctx := plannercore.MockContext() schema := expression.NewSchema() - + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() g0 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewZero()}}.Init(ctx, 0)), schema) g0.Insert(NewGroupExpr(plannercore.LogicalLimit{Count: 1}.Init(ctx, 0))) g0.Insert(NewGroupExpr(plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) @@ -191,7 +201,10 @@ func TestExprIterWithEngineType(t *testing.T) { defer view.Stop() ctx := plannercore.MockContext() schema := expression.NewSchema() - + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() g1 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)), schema).SetEngineType(EngineTiFlash) g1.Insert(NewGroupExpr(plannercore.LogicalLimit{Count: 1}.Init(ctx, 0))) g1.Insert(NewGroupExpr(plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) diff --git a/planner/memo/group_test.go b/planner/memo/group_test.go index e601fea20536e..5120924868f4c 100644 --- a/planner/memo/group_test.go +++ b/planner/memo/group_test.go @@ -64,6 +64,10 @@ func TestGroupDelete(t *testing.T) { func TestGroupDeleteAll(t *testing.T) { ctx := plannercore.MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() expr := NewGroupExpr(plannercore.LogicalSelection{}.Init(ctx, 0)) g := NewGroupWithSchema(expr, expression.NewSchema()) require.True(t, g.Insert(NewGroupExpr(plannercore.LogicalLimit{}.Init(ctx, 0)))) @@ -96,6 +100,10 @@ func TestGroupFingerPrint(t *testing.T) { is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()}) ctx := plannercore.MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt1, is) require.NoError(t, err) logic1, ok := plan.(plannercore.LogicalPlan) @@ -146,6 +154,10 @@ func TestGroupFingerPrint(t *testing.T) { func TestGroupGetFirstElem(t *testing.T) { ctx := plannercore.MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() expr0 := NewGroupExpr(plannercore.LogicalProjection{}.Init(ctx, 0)) expr1 := NewGroupExpr(plannercore.LogicalLimit{}.Init(ctx, 0)) expr2 := NewGroupExpr(plannercore.LogicalProjection{}.Init(ctx, 0)) @@ -175,7 +187,12 @@ func (impl *fakeImpl) AttachChildren(...Implementation) Implementation { return func (impl *fakeImpl) GetCostLimit(float64, ...Implementation) float64 { return 0 } func TestGetInsertGroupImpl(t *testing.T) { - g := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalLimit{}.Init(plannercore.MockContext(), 0)), expression.NewSchema()) + ctx := plannercore.MockContext() + g := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalLimit{}.Init(ctx, 0)), expression.NewSchema()) + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() emptyProp := &property.PhysicalProperty{} require.Nil(t, g.GetImpl(emptyProp)) @@ -211,6 +228,10 @@ func TestEngineTypeSet(t *testing.T) { func TestFirstElemAfterDelete(t *testing.T) { ctx := plannercore.MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() oldExpr := NewGroupExpr(plannercore.LogicalLimit{Count: 10}.Init(ctx, 0)) g := NewGroupWithSchema(oldExpr, expression.NewSchema()) newExpr := NewGroupExpr(plannercore.LogicalLimit{Count: 20}.Init(ctx, 0)) @@ -228,6 +249,10 @@ func TestBuildKeyInfo(t *testing.T) { variable.EnableMDL.Store(false) p := parser.New() ctx := plannercore.MockContext() + defer func() { + do := domain.GetDomain(ctx) + do.StatsHandle().Close() + }() is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()}) domain.GetDomain(ctx).MockInfoCacheAndLoadInfoSchema(is) diff --git a/planner/util/BUILD.bazel b/planner/util/BUILD.bazel index 00fd783b89041..32807d3579a45 100644 --- a/planner/util/BUILD.bazel +++ b/planner/util/BUILD.bazel @@ -32,6 +32,7 @@ go_test( embed = [":util"], flaky = True, deps = [ + "//domain", "//parser/model", "//planner/core", "//testkit/testsetup", diff --git a/planner/util/path_test.go b/planner/util/path_test.go index d3603845aa737..39d1cf03b09d4 100644 --- a/planner/util/path_test.go +++ b/planner/util/path_test.go @@ -17,6 +17,7 @@ package util_test import ( "testing" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/util" @@ -79,6 +80,10 @@ func TestCompareCol2Len(t *testing.T) { func TestOnlyPointRange(t *testing.T) { sctx := core.MockContext() + defer func() { + do := domain.GetDomain(sctx) + do.StatsHandle().Close() + }() nullDatum := types.MinNotNullDatum() nullDatum.SetNull() nullPointRange := ranger.Range{ diff --git a/session/tidb_test.go b/session/tidb_test.go index fdc7036d1be39..aed1826b8e8d1 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -18,6 +18,7 @@ import ( "context" "testing" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/planner/core" @@ -65,7 +66,9 @@ func TestSysSessionPoolGoroutineLeak(t *testing.T) { func TestParseErrorWarn(t *testing.T) { ctx := core.MockContext() - + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() nodes, err := Parse(ctx, "select /*+ adf */ 1") require.NoError(t, err) require.Len(t, nodes, 1) diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 77fa58c0fe053..400cf9ca68708 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -39,6 +39,7 @@ import ( func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache *cache.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { physicalID := row.GetInt64(1) + // The table is read-only. Please do not modify it. table, ok := h.getTableByPhysicalID(is, physicalID) if !ok { logutil.BgLogger().Debug("unknown physical ID in stats meta table, maybe it has been dropped", zap.Int64("ID", physicalID)) @@ -164,6 +165,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache *cach for row := iter.Begin(); row != iter.End(); row = iter.Next() { tblID, statsVer := row.GetInt64(0), row.GetInt64(8) table, ok := cache.GetFromInternal(tblID) + table = table.Copy() if !ok { continue } @@ -282,6 +284,7 @@ func (*Handle) initStatsTopN4Chunk(cache *cache.StatsCache, iter *chunk.Iterator if !ok { continue } + table = table.Copy() idx, ok := table.Indices[row.GetInt64(1)] if !ok || (idx.CMSketch == nil && idx.StatsVer <= statistics.Version1) { continue @@ -380,6 +383,7 @@ func (*Handle) initStatsBuckets4Chunk(cache *cache.StatsCache, iter *chunk.Itera if !ok { continue } + table = table.Copy() var lower, upper types.Datum var hist *statistics.Histogram if isIndex > 0 { diff --git a/statistics/handle/cache/internal/lfu/lfu_cache.go b/statistics/handle/cache/internal/lfu/lfu_cache.go index f8084bf73496b..4421280f2f2fa 100644 --- a/statistics/handle/cache/internal/lfu/lfu_cache.go +++ b/statistics/handle/cache/internal/lfu/lfu_cache.go @@ -34,6 +34,7 @@ type LFU struct { cache *ristretto.Cache resultKeySet *keySetShard cost atomic.Int64 + closed atomic.Bool closeOnce sync.Once } @@ -149,6 +150,9 @@ func (s *LFU) dropMemory(item *ristretto.Item) { // so it should not be processed. return } + if s.closed.Load() { + return + } // We do not need to calculate the cost during onEvict, // because the onexit function is also called when the evict event occurs. // TODO(hawkingrei): not copy the useless part. @@ -177,6 +181,9 @@ func (s *LFU) onExit(val any) { // and in the second invocation, the value is empty, so it should not be processed. return } + if s.closed.Load() { + return + } s.addCost( -1 * val.(*statistics.Table).MemoryUsage().TotalTrackingMemUsage()) } @@ -210,6 +217,7 @@ func (s *LFU) metrics() *ristretto.Metrics { // Close implements statsCacheInner func (s *LFU) Close() { s.closeOnce.Do(func() { + s.closed.Store(true) s.Clear() s.cache.Close() s.cache.Wait() diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index bd56020d6591d..661640897b636 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -97,7 +97,6 @@ func TestConversion(t *testing.T) { tbl := h.GetTableStats(tableInfo.Meta()) requireTableEqual(t, loadTbl, tbl) - cleanStats(tk, dom) var wg util.WaitGroupWrapper wg.Run(func() { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 5c994ef261cab..df5d9eeb34b29 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -2023,6 +2023,6 @@ func (h *Handle) SetStatsCacheCapacity(c int64) { // Close stops the background func (h *Handle) Close() { - h.statsCache.Load().Close() h.gpool.Close() + h.statsCache.Load().Close() } diff --git a/statistics/handle/handletest/handle_test.go b/statistics/handle/handletest/handle_test.go index 3fa5b2cbf7ee4..9a36c1bbfe1cc 100644 --- a/statistics/handle/handletest/handle_test.go +++ b/statistics/handle/handletest/handle_test.go @@ -119,6 +119,9 @@ func TestVersion(t *testing.T) { require.NoError(t, err) tableInfo1 := tbl1.Meta() h, err := handle.NewHandle(testKit.Session(), testKit2.Session(), time.Millisecond, do.SysSessionPool(), do.SysProcTracker(), do.GetAutoAnalyzeProcID) + defer func() { + h.Close() + }() require.NoError(t, err) unit := oracle.ComposeTS(1, 0) testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) diff --git a/statistics/handle/handletest/statstest/stats_test.go b/statistics/handle/handletest/statstest/stats_test.go index 9acb8d770ad61..468c1161664b8 100644 --- a/statistics/handle/handletest/statstest/stats_test.go +++ b/statistics/handle/handletest/statstest/stats_test.go @@ -174,6 +174,7 @@ func testInitStatsMemTrace(t *testing.T) { h := dom.StatsHandle() is := dom.InfoSchema() h.Clear() + require.Equal(t, h.GetMemConsumed(), int64(0)) require.NoError(t, h.InitStats(is)) var memCostTot int64 @@ -268,7 +269,7 @@ func TestInitStatsVer2(t *testing.T) { require.Equal(t, uint8(0x33), cols[2].LastAnalyzePos.GetBytes()[0]) require.Equal(t, uint8(0x33), cols[3].LastAnalyzePos.GetBytes()[0]) h.Clear() - require.NoError(t, h.Update(is)) + require.NoError(t, h.InitStats(is)) table1 := h.GetTableStats(tbl.Meta()) internal.AssertTableEqual(t, table0, table1) h.SetLease(0) diff --git a/util/ranger/BUILD.bazel b/util/ranger/BUILD.bazel index 2c695ed98d24d..59bbbc6f8f3c3 100644 --- a/util/ranger/BUILD.bazel +++ b/util/ranger/BUILD.bazel @@ -50,6 +50,7 @@ go_test( deps = [ ":ranger", "//config", + "//domain", "//expression", "//parser/ast", "//parser/model", diff --git a/util/ranger/types_test.go b/util/ranger/types_test.go index 2a25694dbbbf9..d09c7e65b8b89 100644 --- a/util/ranger/types_test.go +++ b/util/ranger/types_test.go @@ -18,6 +18,7 @@ import ( "math" "testing" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" @@ -136,7 +137,9 @@ func TestRange(t *testing.T) { }, } for _, v := range isPointTests { - require.Equal(t, v.isPoint, v.ran.IsPoint(core.MockContext())) + ctx := core.MockContext() + require.Equal(t, v.isPoint, v.ran.IsPoint(ctx)) + domain.GetDomain(ctx).StatsHandle().Close() } }