diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 3eb32472170e5..cf22589980380 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -816,12 +816,12 @@ Projection 1.00 root Column#7 └─TableDual(Probe) 1.00 root rows:1 explain format = 'brief' select count(a) from t group by b order by (select count(a)); id estRows task access object operator info -Sort 8000.00 root Column#4 +Sort 8000.00 root Column#5 └─HashJoin 8000.00 root CARTESIAN left outer join ├─TableDual(Build) 1.00 root rows:1 - └─HashAgg(Probe) 8000.00 root group by:test.t.b, funcs:count(Column#8)->Column#4 + └─HashAgg(Probe) 8000.00 root group by:test.t.b, funcs:count(Column#9)->Column#5 └─TableReader 8000.00 root data:HashAgg - └─HashAgg 8000.00 cop[tikv] group by:test.t.b, funcs:count(test.t.a)->Column#8 + └─HashAgg 8000.00 cop[tikv] group by:test.t.b, funcs:count(test.t.a)->Column#9 └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select (select sum(count(a))) from t; id estRows task access object operator info @@ -835,15 +835,15 @@ Projection 1.00 root Column#5 └─TableDual 1.00 root rows:1 explain format = 'brief' select sum(a), (select sum(a)), count(a) from t group by b order by (select count(a)); id estRows task access object operator info -Projection 8000.00 root Column#4, Column#4, Column#5 -└─Sort 8000.00 root Column#5 +Projection 8000.00 root Column#5, Column#5, Column#6 +└─Sort 8000.00 root Column#6 └─HashJoin 8000.00 root CARTESIAN left outer join ├─TableDual(Build) 1.00 root rows:1 └─HashJoin(Probe) 8000.00 root CARTESIAN left outer join ├─TableDual(Build) 1.00 root rows:1 - └─HashAgg(Probe) 8000.00 root group by:test.t.b, funcs:sum(Column#13)->Column#4, funcs:count(Column#14)->Column#5 + └─HashAgg(Probe) 8000.00 root group by:test.t.b, funcs:sum(Column#14)->Column#5, funcs:count(Column#15)->Column#6 └─TableReader 8000.00 root data:HashAgg - └─HashAgg 8000.00 cop[tikv] group by:test.t.b, funcs:sum(test.t.a)->Column#13, funcs:count(test.t.a)->Column#14 + └─HashAgg 8000.00 cop[tikv] group by:test.t.b, funcs:sum(test.t.a)->Column#14, funcs:count(test.t.a)->Column#15 └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo drop table if exists t; create table t(a tinyint, b smallint, c mediumint, d int, e bigint); diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 769e9782ace39..a370e3ce7eea8 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -5226,3 +5226,344 @@ func (s *testIntegrationSuite) TestIssue31202(c *C) { "└─TableFullScan 10000.00 cop[tikv] table:t31202 keep order:false, stats:pseudo")) tk.MustExec("drop table if exists t31202") } +<<<<<<< HEAD +======= + +func TestNaturalJoinUpdateSameTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database natural_join_update") + defer tk.MustExec("drop database natural_join_update") + tk.MustExec("use natural_join_update") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustExec("update t1 as a natural join t1 b SET a.a = 2, b.b = 3") + tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2 3", "2 3")) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (a int primary key, b int)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 b SET a.a = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (a int, b int) partition by hash (a) partitions 3") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 b SET a.a = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (A int, b int) partition by hash (b) partitions 3") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + _, err := tk.Exec(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`) + require.Error(t, err) + require.Regexp(t, ".planner:1706.Primary key/partition key update is not allowed since the table is updated both as 'a' and 'B'.", err.Error()) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (A int, b int) partition by RANGE COLUMNS (b) (partition `pNeg` values less than (0),partition `pPos` values less than MAXVALUE)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") +} + +func TestAggPushToCopForCachedTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(`create table t32157( + process_code varchar(8) NOT NULL, + ctrl_class varchar(2) NOT NULL, + ctrl_type varchar(1) NOT NULL, + oper_no varchar(12) DEFAULT NULL, + modify_date datetime DEFAULT NULL, + d_c_flag varchar(2) NOT NULL, + PRIMARY KEY (process_code,ctrl_class,d_c_flag));`) + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '1', '10000', '2016-06-29 00:00:00', 'C')") + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '0', '0000', '2016-06-01 00:00:00', 'D')") + tk.MustExec("alter table t32157 cache") + + tk.MustQuery("explain format = 'brief' select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:count(1)->Column#8]\n" + + "[└─UnionScan 10.00 root eq(test.t32157.process_code, \"GDEP0071\")]\n" + + "[ └─TableReader 10.00 root data:Selection]\n" + + "[ └─Selection 10.00 cop[tikv] eq(test.t32157.process_code, \"GDEP0071\")]\n" + + "[ └─TableFullScan 10000.00 cop[tikv] table:t32157 keep order:false, stats:pseudo")) + + var readFromCacheNoPanic bool + for i := 0; i < 10; i++ { + tk.MustQuery("select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows("2")) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + readFromCacheNoPanic = true + break + } + } + require.True(t, readFromCacheNoPanic) + + tk.MustExec("drop table if exists t31202") +} + +func TestIssue31240(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t31240(a int, b int);") + tk.MustExec("set @@tidb_allow_mpp = 0") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31240", L: "t31240"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table if exists t31240") +} + +func TestIssue32632(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `partsupp` (" + + " `PS_PARTKEY` bigint(20) NOT NULL," + + "`PS_SUPPKEY` bigint(20) NOT NULL," + + "`PS_AVAILQTY` bigint(20) NOT NULL," + + "`PS_SUPPLYCOST` decimal(15,2) NOT NULL," + + "`PS_COMMENT` varchar(199) NOT NULL," + + "PRIMARY KEY (`PS_PARTKEY`,`PS_SUPPKEY`) /*T![clustered_index] NONCLUSTERED */)") + tk.MustExec("CREATE TABLE `supplier` (" + + "`S_SUPPKEY` bigint(20) NOT NULL," + + "`S_NAME` char(25) NOT NULL," + + "`S_ADDRESS` varchar(40) NOT NULL," + + "`S_NATIONKEY` bigint(20) NOT NULL," + + "`S_PHONE` char(15) NOT NULL," + + "`S_ACCTBAL` decimal(15,2) NOT NULL," + + "`S_COMMENT` varchar(101) NOT NULL," + + "PRIMARY KEY (`S_SUPPKEY`) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("analyze table partsupp;") + tk.MustExec("analyze table supplier;") + tk.MustExec("set @@tidb_enforce_mpp = 1") + + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "partsupp", L: "partsupp"}) + require.NoError(t, err) + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "supplier", L: "supplier"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + h := dom.StatsHandle() + statsTbl1 := h.GetTableStats(tbl1.Meta()) + statsTbl1.Count = 800000 + statsTbl2 := h.GetTableStats(tbl2.Meta()) + statsTbl2.Count = 10000 + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table if exists partsupp") + tk.MustExec("drop table if exists supplier") +} + +func TestTiFlashPartitionTableScan(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@tidb_enforce_mpp = on") + tk.MustExec("set @@tidb_allow_batch_cop = 2") + tk.MustExec("drop table if exists rp_t;") + tk.MustExec("drop table if exists hp_t;") + tk.MustExec("create table rp_t(a int) partition by RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21));") + tk.MustExec("create table hp_t(a int) partition by hash(a) partitions 4;") + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "rp_t", L: "rp_t"}) + require.NoError(t, err) + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "hp_t", L: "hp_t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table rp_t;") + tk.MustExec("drop table hp_t;") +} + +func TestIssue33175(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create table t (id bigint(45) unsigned not null, c varchar(20), primary key(id));") + tk.MustExec("insert into t values (9734095886065816707, 'a'), (10353107668348738101, 'b'), (0, 'c');") + tk.MustExec("begin") + tk.MustExec("insert into t values (33, 'd');") + tk.MustQuery("select max(id) from t;").Check(testkit.Rows("10353107668348738101")) + tk.MustExec("rollback") + + tk.MustExec("alter table t cache") + for { + tk.MustQuery("select max(id) from t;").Check(testkit.Rows("10353107668348738101")) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + break + } + } + + // // With subquery, like the original issue case. + for { + tk.MustQuery("select * from t where id > (select max(id) from t where t.id > 0);").Check(testkit.Rows()) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + break + } + } + + // Test order by desc / asc. + tk.MustQuery("select id from t order by id desc;").Check(testkit.Rows( + "10353107668348738101", + "9734095886065816707", + "0")) + + tk.MustQuery("select id from t order by id asc;").Check(testkit.Rows( + "0", + "9734095886065816707", + "10353107668348738101")) + + tk.MustExec("alter table t nocache") + tk.MustExec("drop table t") + + // Cover more code that use union scan + // TableReader/IndexReader/IndexLookup + for idx, q := range []string{ + "create temporary table t (id bigint unsigned, c int default null, index(id))", + "create temporary table t (id bigint unsigned primary key)", + } { + tk.MustExec(q) + tk.MustExec("insert into t(id) values (1), (3), (9734095886065816707), (9734095886065816708)") + tk.MustQuery("select min(id) from t").Check(testkit.Rows("1")) + tk.MustQuery("select max(id) from t").Check(testkit.Rows("9734095886065816708")) + tk.MustQuery("select id from t order by id asc").Check(testkit.Rows( + "1", "3", "9734095886065816707", "9734095886065816708")) + tk.MustQuery("select id from t order by id desc").Check(testkit.Rows( + "9734095886065816708", "9734095886065816707", "3", "1")) + if idx == 0 { + tk.MustQuery("select * from t order by id asc").Check(testkit.Rows( + "1 ", + "3 ", + "9734095886065816707 ", + "9734095886065816708 ")) + tk.MustQuery("select * from t order by id desc").Check(testkit.Rows( + "9734095886065816708 ", + "9734095886065816707 ", + "3 ", + "1 ")) + } + tk.MustExec("drop table t") + } + + // More and more test + tk.MustExec("create global temporary table `tmp1` (id bigint unsigned primary key) on commit delete rows;") + tk.MustExec("begin") + tk.MustExec("insert into tmp1 values (0),(1),(2),(65536),(9734095886065816707),(9734095886065816708);") + tk.MustQuery("select * from tmp1 where id <= 65534 or (id > 65535 and id < 9734095886065816700) or id >= 9734095886065816707 order by id desc;").Check(testkit.Rows( + "9734095886065816708", "9734095886065816707", "65536", "2", "1", "0")) + + tk.MustQuery("select * from tmp1 where id <= 65534 or (id > 65535 and id < 9734095886065816700) or id >= 9734095886065816707 order by id asc;").Check(testkit.Rows( + "0", "1", "2", "65536", "9734095886065816707", "9734095886065816708")) + + tk.MustExec("create global temporary table `tmp2` (id bigint primary key) on commit delete rows;") + tk.MustExec("begin") + tk.MustExec("insert into tmp2 values(-2),(-1),(0),(1),(2);") + tk.MustQuery("select * from tmp2 where id <= -1 or id > 0 order by id desc;").Check(testkit.Rows("2", "1", "-1", "-2")) + tk.MustQuery("select * from tmp2 where id <= -1 or id > 0 order by id asc;").Check(testkit.Rows("-2", "-1", "1", "2")) +} + +func TestIssue33042(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t1(id int primary key, col1 int)") + tk.MustExec("create table t2(id int primary key, col1 int)") + tk.MustQuery("explain format='brief' SELECT /*+ merge_join(t1, t2)*/ * FROM (t1 LEFT JOIN t2 ON t1.col1=t2.id) order by t2.id;").Check( + testkit.Rows( + "Sort 12500.00 root test.t2.id", + "└─MergeJoin 12500.00 root left outer join, left key:test.t1.col1, right key:test.t2.id", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo", + " └─Sort(Probe) 10000.00 root test.t1.col1", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + ), + ) +} + +func TestIssue29663(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (a int, b int)") + tk.MustExec("create table t2 (c int, d int)") + tk.MustExec("insert into t1 values(1, 1), (1,2),(2,1),(2,2)") + tk.MustExec("insert into t2 values(1, 3), (1,4),(2,5),(2,6)") + + tk.MustQuery("explain select one.a from t1 one order by (select two.d from t2 two where two.c = one.b)").Check(testkit.Rows( + "Projection_16 10000.00 root test.t1.a", + "└─Sort_17 10000.00 root test.t2.d", + " └─Apply_20 10000.00 root CARTESIAN left outer join", + " ├─TableReader_22(Build) 10000.00 root data:TableFullScan_21", + " │ └─TableFullScan_21 10000.00 cop[tikv] table:one keep order:false, stats:pseudo", + " └─MaxOneRow_23(Probe) 1.00 root ", + " └─TableReader_26 2.00 root data:Selection_25", + " └─Selection_25 2.00 cop[tikv] eq(test.t2.c, test.t1.b)", + " └─TableFullScan_24 2000.00 cop[tikv] table:two keep order:false, stats:pseudo")) +} +>>>>>>> 334508e13... planner: fix order by sub-query couldn't find outer correlated columns (#33640) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index a67dc8da7808a..6f37265324a14 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2166,7 +2166,7 @@ func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, o // resolveHavingAndOrderBy will process aggregate functions and resolve the columns that don't exist in select fields. // If we found some columns that are not in select fields, we will append it to select fields and update the colMapper. // When we rewrite the order by / having expression, we will find column in map at first. -func (b *PlanBuilder) resolveHavingAndOrderBy(sel *ast.SelectStmt, p LogicalPlan) ( +func (b *PlanBuilder) resolveHavingAndOrderBy(ctx context.Context, sel *ast.SelectStmt, p LogicalPlan) ( map[*ast.AggregateFuncExpr]int, map[*ast.AggregateFuncExpr]int, error) { extractor := &havingWindowAndOrderbyExprResolver{ p: p, @@ -2206,6 +2206,50 @@ func (b *PlanBuilder) resolveHavingAndOrderBy(sel *ast.SelectStmt, p LogicalPlan } } sel.Fields.Fields = extractor.selectFields + // this part is used to fetch correlated column from sub-query item in order-by clause, and append the origin + // auxiliary select filed in select list, otherwise, sub-query itself won't get the name resolved in outer schema. + if sel.OrderBy != nil { + for _, byItem := range sel.OrderBy.Items { + if _, ok := byItem.Expr.(*ast.SubqueryExpr); ok { + // correlated agg will be extracted completely latter. + _, np, err := b.rewrite(ctx, byItem.Expr, p, nil, true) + if err != nil { + return nil, nil, errors.Trace(err) + } + correlatedCols := ExtractCorrelatedCols4LogicalPlan(np) + for _, cone := range correlatedCols { + var colName *ast.ColumnName + for idx, pone := range p.Schema().Columns { + if cone.UniqueID == pone.UniqueID { + pname := p.OutputNames()[idx] + colName = &ast.ColumnName{ + Schema: pname.DBName, + Table: pname.TblName, + Name: pname.ColName, + } + break + } + } + if colName != nil { + columnNameExpr := &ast.ColumnNameExpr{Name: colName} + for _, field := range sel.Fields.Fields { + if c, ok := field.Expr.(*ast.ColumnNameExpr); ok && colMatch(c.Name, columnNameExpr.Name) { + // deduplicate select fields: don't append it once it already has one. + columnNameExpr = nil + break + } + } + if columnNameExpr != nil { + sel.Fields.Fields = append(sel.Fields.Fields, &ast.SelectField{ + Auxiliary: true, + Expr: columnNameExpr, + }) + } + } + } + } + } + } return havingAggMapper, extractor.aggMapper, nil } @@ -2375,7 +2419,7 @@ func (r *correlatedAggregateResolver) resolveSelect(sel *ast.SelectStmt) (err er } } - _, _, err = r.b.resolveHavingAndOrderBy(sel, p) + _, _, err = r.b.resolveHavingAndOrderBy(r.ctx, sel, p) if err != nil { return err } @@ -3603,7 +3647,7 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L // We must resolve having and order by clause before build projection, // because when the query is "select a+1 as b from t having sum(b) < 0", we must replace sum(b) to sum(a+1), // which only can be done before building projection and extracting Agg functions. - havingMap, orderMap, err = b.resolveHavingAndOrderBy(sel, p) + havingMap, orderMap, err = b.resolveHavingAndOrderBy(ctx, sel, p) if err != nil { return nil, err } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 0136545eff430..0b9e85177df49 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1496,7 +1496,11 @@ func (s *testPlanSuite) TestUnion(c *C) { c.Assert(err, NotNil) continue } +<<<<<<< HEAD c.Assert(err, IsNil) +======= + require.NoError(t, err, comment) +>>>>>>> 334508e13... planner: fix order by sub-query couldn't find outer correlated columns (#33640) p := plan.(LogicalPlan) p, err = logicalOptimize(ctx, builder.optFlag, p) s.testData.OnRecord(func() {