diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 6159dc7c00054..cdcffdb0addd0 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -23,42 +23,21 @@ import ( "strings" "testing" - . "github.com/pingcap/check" - "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testAnalyzeSuite{}) - -type testAnalyzeSuite struct { - testData testutil.TestData -} - -func (s *testAnalyzeSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "analyze_suite") - c.Assert(err, IsNil) -} - -func (s *testAnalyzeSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) error { +func loadTableStats(fileName string, dom *domain.Domain) error { statsPath := filepath.Join("testdata", fileName) bytes, err := os.ReadFile(statsPath) if err != nil { @@ -77,15 +56,10 @@ func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) e return nil } -func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestExplainAnalyze(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by tk.MustExec("create table t1(a int, b int, c int, key idx(a, b))") @@ -94,151 +68,134 @@ func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) { tk.MustExec("insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5)") tk.MustExec("analyze table t1, t2") rs := tk.MustQuery("explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1") - c.Assert(len(rs.Rows()), Equals, 10) + require.Len(t, rs.Rows(), 10) for _, row := range rs.Rows() { - c.Assert(len(row), Equals, 9) + require.Len(t, row, 9) execInfo := row[5].(string) - c.Assert(strings.Contains(execInfo, "time"), Equals, true) - c.Assert(strings.Contains(execInfo, "loops"), Equals, true) + require.Contains(t, execInfo, "time") + require.Contains(t, execInfo, "loops") if strings.Contains(row[0].(string), "Reader") || strings.Contains(row[0].(string), "IndexLookUp") { - c.Assert(strings.Contains(execInfo, "cop_task"), Equals, true) + require.Contains(t, execInfo, "cop_task") } } } // TestCBOWithoutAnalyze tests the plan with stats that only have count info. -func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCBOWithoutAnalyze(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t1 (a int)") testKit.MustExec("create table t2 (a int)") h := dom.StatsHandle() - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)") testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestStraightJoin(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestStraightJoin(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") h := dom.StatsHandle() for _, tblName := range []string{"t1", "t2", "t3", "t4"} { testKit.MustExec(fmt.Sprintf("create table %s (a int)", tblName)) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) } var input []string var output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestTableDual(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() +func TestTableDual(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() - testKit := testkit.NewTestKit(c, store) + testKit := testkit.NewTestKit(t, store) testKit.MustExec(`use test`) h := dom.StatsHandle() testKit.MustExec(`create table t(a int)`) testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - statistics.RatioOfPseudoEstimate.Store(0.7) - }() +func TestEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) statistics.RatioOfPseudoEstimate.Store(10.0) + defer statistics.RatioOfPseudoEstimate.Store(0.7) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") h := dom.StatsHandle() - err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") for i := 1; i <= 8; i++ { testKit.MustExec("delete from t where a = ?", i) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } @@ -255,15 +212,10 @@ func constructInsertSQL(i, n int) string { return sql } -func (s *testAnalyzeSuite) TestIndexRead(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIndexRead(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("set @@session.tidb_executor_concurrency = 4;") testKit.MustExec("set @@session.tidb_hash_join_concurrency = 5;") testKit.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") @@ -280,78 +232,69 @@ func (s *testAnalyzeSuite) TestIndexRead(c *C) { // This stats is generated by following format: // fill (a, b, c, e) as (i*100+j, i, i+j, i*100+j), i and j is dependent and range of this two are [0, 99]. - err = s.loadTableStats("analyzesSuiteTestIndexReadT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzesSuiteTestIndexReadT.json", dom)) for i := 1; i < 16; i++ { testKit.MustExec(fmt.Sprintf("insert into t1 values(%v, %v)", i, i)) } testKit.MustExec("analyze table t1") - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestEmptyTable(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestEmptyTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1") testKit.MustExec("create table t (c1 int)") testKit.MustExec("create table t1 (c1 int)") testKit.MustExec("analyze table t, t1") var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestAnalyze(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1, t2, t3") testKit.MustExec("create table t (a int, b int)") @@ -382,63 +325,58 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { testKit.MustExec("analyze table t4") testKit.MustExec("create view v as select * from t") - _, err = testKit.Exec("analyze table v") - c.Assert(err.Error(), Equals, "analyze view v is not supported now.") + _, err := testKit.Exec("analyze table v") + require.EqualError(t, err, "analyze view v is not supported now.") testKit.MustExec("drop view v") testKit.MustExec("create sequence seq") _, err = testKit.Exec("analyze table seq") - c.Assert(err.Error(), Equals, "analyze sequence seq is not supported now.") + require.EqualError(t, err, "analyze sequence seq is not supported now.") testKit.MustExec("drop sequence seq") var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] err = executor.ResetContextOfStmt(ctx, stmt) - c.Assert(err, IsNil) + require.NoError(t, err) ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestOutdatedAnalyze(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx(a))") for i := 0; i < 10; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d,%d)", i, i)) } h := dom.StatsHandle() - err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []struct { SQL string EnablePseudoForOutdatedStats bool @@ -450,30 +388,26 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { RatioOfPseudoEstimate float64 Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - testKit.Se.GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) + testKit.Session().GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) statistics.RatioOfPseudoEstimate.Store(tt.RatioOfPseudoEstimate) plan := testKit.MustQuery(tt.SQL) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt.SQL output[i].EnablePseudoForOutdatedStats = tt.EnablePseudoForOutdatedStats output[i].RatioOfPseudoEstimate = tt.RatioOfPseudoEstimate - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestNullCount(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestNullCount(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int, b int, index idx(a))") @@ -481,33 +415,29 @@ func (s *testAnalyzeSuite) TestNullCount(c *C) { testKit.MustExec("analyze table t") var input []string var output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i := 0; i < 2; i++ { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) }) testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) } h := dom.StatsHandle() h.Clear() - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.Update(dom.InfoSchema())) for i := 2; i < 4; i++ { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) }) testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCorrelatedEstimation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by tk.MustExec("create table t(a int, b int, c int, index idx(c,b,a))") @@ -517,25 +447,21 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { input []string output [][]string ) - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { rs := tk.MustQuery(tt) - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(rs.Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(rs.Rows()) }) rs.Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestInconsistentEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int, index ab(a,b), index ac(a,c))") tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)") @@ -547,52 +473,29 @@ func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { // Force using the histogram to estimate. tk.MustExec("update mysql.stats_histograms set stats_ver = 0") dom.StatsHandle().Clear() - err = dom.StatsHandle().Update(dom.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := tk.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { - store, err := mockstore.NewMockStore() - if err != nil { - return nil, nil, errors.Trace(err) - } - - session.SetSchemaLease(0) - session.DisableStats4Test() - - dom, err := session.BootstrapSession(store) - if err != nil { - return nil, nil, err - } - - dom.SetStatsUpdating(true) - return store, dom, errors.Trace(err) -} - func BenchmarkOptimize(b *testing.B) { - c := &C{} - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() + store, clean := testkit.CreateMockStore(b) + defer clean() - testKit := testkit.NewTestKit(c, store) + testKit := testkit.NewTestKit(b, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int primary key, b int, c varchar(200), d datetime DEFAULT CURRENT_TIMESTAMP, e int, ts timestamp DEFAULT CURRENT_TIMESTAMP)") @@ -686,35 +589,30 @@ func BenchmarkOptimize(b *testing.B) { }, } for _, tt := range tests { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt.sql) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(b, err) + require.Len(b, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(b, err) b.Run(tt.sql, func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { _, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(b, err) } b.ReportAllocs() }) } } -func (s *testAnalyzeSuite) TestIssue9562(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIssue9562(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") var input [][]string @@ -722,16 +620,17 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { SQL []string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -741,15 +640,10 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { } } -func (s *testAnalyzeSuite) TestIssue9805(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIssue9805(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec(` @@ -775,15 +669,10 @@ func (s *testAnalyzeSuite) TestIssue9805(c *C) { tk.MustQuery("explain analyze select /*+ TIDB_INLJ(t2) */ t1.id, t2.a from t1 join t2 on t1.a = t2.d where t1.b = 't2' and t1.d = 4") } -func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLimitCrossEstimation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@session.tidb_executor_concurrency = 4;") tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") @@ -796,16 +685,17 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { SQL []string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -815,21 +705,15 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { } } -func (s *testAnalyzeSuite) TestLowSelIndexGreedySearch(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLowSelIndexGreedySearch(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec(`set tidb_opt_limit_push_down_threshold=0`) testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a varchar(32) default null, b varchar(10) default null, c varchar(12) default null, d varchar(32) default null, e bigint(10) default null, key idx1 (d,a), key idx2 (a,c), key idx3 (c,b), key idx4 (e))") - err = s.loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom)) var input []string var output []struct { SQL string @@ -838,59 +722,52 @@ func (s *testAnalyzeSuite) TestLowSelIndexGreedySearch(c *C) { // The test purposes are: // - index `idx2` runs much faster than `idx4` experimentally; // - estimated row count of IndexLookUp should be 0; - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestUpdateProjEliminate(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestUpdateProjEliminate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("explain update t t1, (select distinct b from t) t2 set t1.b = t2.b") } -func (s *testAnalyzeSuite) TestTiFlashCostModel(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestTiFlashCostModel(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int, b int, c int, primary key(a))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) - c.Assert(err, IsNil) + require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} var input, output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { if j == len(ts)-1 { - output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -900,45 +777,35 @@ func (s *testAnalyzeSuite) TestTiFlashCostModel(c *C) { } } -func (s *testAnalyzeSuite) TestIndexEqualUnknown(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIndexEqualUnknown(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1") - testKit.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly testKit.MustExec("CREATE TABLE t(a bigint(20) NOT NULL, b bigint(20) NOT NULL, c bigint(20) NOT NULL, PRIMARY KEY (a,c,b), KEY (b))") - err = s.loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom)) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestLimitIndexEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLimitIndexEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -946,31 +813,27 @@ func (s *testAnalyzeSuite) TestLimitIndexEstimation(c *C) { tk.MustExec("set session tidb_enable_extended_stats = on") // Values in column a are from 1 to 1000000, values in column b are from 1000000 to 1, // these 2 columns are strictly correlated in reverse order. - err = s.loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom)) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestBatchPointGetTablePartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t1,t2,t3,t4,t5,t6") @@ -1051,7 +914,7 @@ func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { "1 2", )) - testKit.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn testKit.MustExec("create table t3(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") testKit.MustExec("insert into t3 values(1,1),(1,2),(2,1),(2,2)") @@ -1209,14 +1072,10 @@ func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { } // TestAppendIntPkToIndexTailForRangeBuilding tests for issue25219 https://github.com/pingcap/tidb/issues/25219. -func (s *testAnalyzeSuite) TestAppendIntPkToIndexTailForRangeBuilding(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestAppendIntPkToIndexTailForRangeBuilding(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t25219(a int primary key, col3 int, col1 int, index idx(col3))") tk.MustExec("insert into t25219 values(1, 1, 1)") diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 709c3d48f7a15..5baa8dc7aa9f4 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -43,6 +43,7 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "partition_pruner") testDataMap.LoadTestSuiteData("testdata", "plan_suite") testDataMap.LoadTestSuiteData("testdata", "integration_suite") + testDataMap.LoadTestSuiteData("testdata", "analyze_suite") indexMergeSuiteData = testDataMap["index_merge_suite"] @@ -98,3 +99,7 @@ func GetPlanSuiteData() testdata.TestData { func GetIntegrationSuiteData() testdata.TestData { return testDataMap["integration_suite"] } + +func GetAnalyzeSuiteData() testdata.TestData { + return testDataMap["analyze_suite"] +}