From f2c6c3ccf14941aa241a8514f717b4dc47144085 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 19 Nov 2024 12:56:03 +0100 Subject: [PATCH 1/3] This is an automated cherry-pick of #57473 Signed-off-by: ti-chi-bot --- pkg/planner/core/issuetest/BUILD.bazel | 27 ++++ .../core/issuetest/planner_issue_test.go | 136 ++++++++++++++++++ planner/core/logical_plan_builder.go | 17 +++ 3 files changed, 180 insertions(+) create mode 100644 pkg/planner/core/issuetest/BUILD.bazel create mode 100644 pkg/planner/core/issuetest/planner_issue_test.go diff --git a/pkg/planner/core/issuetest/BUILD.bazel b/pkg/planner/core/issuetest/BUILD.bazel new file mode 100644 index 0000000000000..2fa7a2e123983 --- /dev/null +++ b/pkg/planner/core/issuetest/BUILD.bazel @@ -0,0 +1,27 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "issuetest_test", + timeout = "short", + srcs = [ + "main_test.go", + "planner_issue_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + race = "on", + shard_count = 4, + deps = [ + "//pkg/parser", + "//pkg/planner", + "//pkg/planner/core", + "//pkg/planner/core/base", + "//pkg/planner/core/resolve", + "//pkg/testkit", + "//pkg/testkit/testdata", + "//pkg/testkit/testmain", + "//pkg/testkit/testsetup", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/planner/core/issuetest/planner_issue_test.go b/pkg/planner/core/issuetest/planner_issue_test.go new file mode 100644 index 0000000000000..fda17d6cfce17 --- /dev/null +++ b/pkg/planner/core/issuetest/planner_issue_test.go @@ -0,0 +1,136 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package issuetest + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/planner" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/resolve" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +// It's a case for Columns in tableScan and indexScan with double reader +func TestIssue43461(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int, index b(b), index b_c(b, c)) partition by hash(a) partitions 4;") + tk.MustExec("analyze table t") + + stmt, err := parser.New().ParseOneStmt("select * from t use index(b) where b > 1 order by b limit 1", "", "") + require.NoError(t, err) + + nodeW := resolve.NewNodeW(stmt) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), nodeW, domain.InfoSchema()) + require.NoError(t, err) + require.NotNil(t, p) + + var idxLookUpPlan *core.PhysicalIndexLookUpReader + var ok bool + + for { + idxLookUpPlan, ok = p.(*core.PhysicalIndexLookUpReader) + if ok { + break + } + p = p.(base.PhysicalPlan).Children()[0] + } + require.True(t, ok) + + is := idxLookUpPlan.IndexPlans[0].(*core.PhysicalIndexScan) + ts := idxLookUpPlan.TablePlans[0].(*core.PhysicalTableScan) + + require.NotEqual(t, is.Columns, ts.Columns) +} + +func Test53726(t *testing.T) { + // test for RemoveUnnecessaryFirstRow + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t7(c int); ") + tk.MustExec("insert into t7 values (575932053), (-258025139);") + tk.MustQuery("select distinct cast(c as decimal), cast(c as signed) from t7"). + Sort().Check(testkit.Rows("-258025139 -258025139", "575932053 575932053")) + tk.MustQuery("explain select distinct cast(c as decimal), cast(c as signed) from t7"). + Check(testkit.Rows( + "HashAgg_8 8000.00 root group by:Column#7, Column#8, funcs:firstrow(Column#7)->Column#3, funcs:firstrow(Column#8)->Column#4", + "└─TableReader_9 8000.00 root data:HashAgg_4", + " └─HashAgg_4 8000.00 cop[tikv] group by:cast(test.t7.c, bigint(22) BINARY), cast(test.t7.c, decimal(10,0) BINARY), ", + " └─TableFullScan_7 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo")) + + tk.MustExec("analyze table t7 all columns") + tk.MustQuery("select distinct cast(c as decimal), cast(c as signed) from t7"). + Sort(). + Check(testkit.Rows("-258025139 -258025139", "575932053 575932053")) + tk.MustQuery("explain select distinct cast(c as decimal), cast(c as signed) from t7"). + Check(testkit.Rows( + "HashAgg_6 2.00 root group by:Column#11, Column#12, funcs:firstrow(Column#11)->Column#3, funcs:firstrow(Column#12)->Column#4", + "└─Projection_12 2.00 root cast(test.t7.c, decimal(10,0) BINARY)->Column#11, cast(test.t7.c, bigint(22) BINARY)->Column#12", + " └─TableReader_11 2.00 root data:TableFullScan_10", + " └─TableFullScan_10 2.00 cop[tikv] table:t7 keep order:false")) +} + +func TestIssue54535(t *testing.T) { + // test for tidb_enable_inl_join_inner_multi_pattern system variable + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set session tidb_enable_inl_join_inner_multi_pattern='ON'") + tk.MustExec("create table ta(a1 int, a2 int, a3 int, index idx_a(a1))") + tk.MustExec("create table tb(b1 int, b2 int, b3 int, index idx_b(b1))") + tk.MustExec("analyze table ta") + tk.MustExec("analyze table tb") + + tk.MustQuery("explain SELECT /*+ inl_join(tmp) */ * FROM ta, (SELECT b1, COUNT(b3) AS cnt FROM tb GROUP BY b1, b2) as tmp where ta.a1 = tmp.b1"). + Check(testkit.Rows( + "Projection_9 9990.00 root test.ta.a1, test.ta.a2, test.ta.a3, test.tb.b1, Column#9", + "└─IndexJoin_16 9990.00 root inner join, inner:HashAgg_14, outer key:test.ta.a1, inner key:test.tb.b1, equal cond:eq(test.ta.a1, test.tb.b1)", + " ├─TableReader_43(Build) 9990.00 root data:Selection_42", + " │ └─Selection_42 9990.00 cop[tikv] not(isnull(test.ta.a1))", + " │ └─TableFullScan_41 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─HashAgg_14(Probe) 79840080.00 root group by:test.tb.b1, test.tb.b2, funcs:count(Column#11)->Column#9, funcs:firstrow(test.tb.b1)->test.tb.b1", + " └─IndexLookUp_15 79840080.00 root ", + " ├─Selection_12(Build) 9990.00 cop[tikv] not(isnull(test.tb.b1))", + " │ └─IndexRangeScan_10 10000.00 cop[tikv] table:tb, index:idx_b(b1) range: decided by [eq(test.tb.b1, test.ta.a1)], keep order:false, stats:pseudo", + " └─HashAgg_13(Probe) 79840080.00 cop[tikv] group by:test.tb.b1, test.tb.b2, funcs:count(test.tb.b3)->Column#11", + " └─TableRowIDScan_11 9990.00 cop[tikv] table:tb keep order:false, stats:pseudo")) + // test for issues/55169 + tk.MustExec("create table t1(col_1 int, index idx_1(col_1));") + tk.MustExec("create table t2(col_1 int, col_2 int, index idx_2(col_1));") + tk.MustQuery("select /*+ inl_join(tmp) */ * from t1 inner join (select col_1, group_concat(col_2) from t2 group by col_1) tmp on t1.col_1 = tmp.col_1;").Check(testkit.Rows()) + tk.MustQuery("select /*+ inl_join(tmp) */ * from t1 inner join (select col_1, group_concat(distinct col_2 order by col_2) from t2 group by col_1) tmp on t1.col_1 = tmp.col_1;").Check(testkit.Rows()) +} + +func TestIssue53175(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t(a int)`) + tk.MustExec(`set @@sql_mode = default`) + tk.MustQuery(`select @@sql_mode REGEXP 'ONLY_FULL_GROUP_BY'`).Check(testkit.Rows("1")) + tk.MustContainErrMsg(`select * from t group by null`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") + tk.MustExec(`create view v as select * from t group by null`) + tk.MustContainErrMsg(`select * from v`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") + tk.MustExec(`set @@sql_mode = ''`) + tk.MustQuery(`select * from t group by null`) + tk.MustQuery(`select * from v`) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 6b1acac4355a3..6c3f6aef81210 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3101,7 +3101,24 @@ func tblInfoFromCol(from ast.ResultSetNode, name *types.FieldName) *model.TableI tableList = extractTableList(from, tableList, true) for _, field := range tableList { if field.Name.L == name.TblName.L { +<<<<<<< HEAD:planner/core/logical_plan_builder.go return field.TableInfo +======= + tnW := b.resolveCtx.GetTableName(field) + if tnW != nil { + return tnW.TableInfo + } + // when the Select is inside a view, it's not pre-processed, tnW is nil. + if b.isCreateView { + // Ignore during create + return nil + } + tblInfo, err := b.is.TableInfoByName(name.DBName, name.TblName) + if err != nil { + return nil + } + return tblInfo +>>>>>>> d0150c0a686 (planner: ONLY_FULL_GROUP_BY sql_mode was not working with VIEWs (#57473)):pkg/planner/core/logical_plan_builder.go } } return nil From 5432f66a55c4b5c8c7c19679d046d6fb95c054e7 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 7 Feb 2025 13:19:20 +0100 Subject: [PATCH 2/3] Manual merge fixes --- planner/core/integration_test.go | 15 ++++++++++++ planner/core/logical_plan_builder.go | 34 ++++++++++++---------------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index cdcc4bfbac1e9..4f97a7a4bbde8 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -8805,3 +8805,18 @@ func TestIssue40285(t *testing.T) { tk.MustExec("CREATE TABLE t(col1 enum('p5', '9a33x') NOT NULL DEFAULT 'p5',col2 tinyblob DEFAULT NULL) ENGINE = InnoDB DEFAULT CHARSET = latin1 COLLATE = latin1_bin;") tk.MustQuery("(select last_value(col1) over () as r0 from t) union all (select col2 as r0 from t);") } + +func TestIssue53175(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t(a int)`) + tk.MustExec(`set @@sql_mode = default`) + tk.MustQuery(`select @@sql_mode REGEXP 'ONLY_FULL_GROUP_BY'`).Check(testkit.Rows("1")) + tk.MustContainErrMsg(`select * from t group by null`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") + tk.MustExec(`create view v as select * from t group by null`) + tk.MustContainErrMsg(`select * from v`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") + tk.MustExec(`set @@sql_mode = ''`) + tk.MustQuery(`select * from t group by null`) + tk.MustQuery(`select * from v`) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 6c3f6aef81210..ffca81ff5eb52 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3096,29 +3096,23 @@ func (g *gbyResolver) Leave(inNode ast.Node) (ast.Node, bool) { return inNode, true } -func tblInfoFromCol(from ast.ResultSetNode, name *types.FieldName) *model.TableInfo { +func (b *PlanBuilder) tblInfoFromCol(from ast.ResultSetNode, name *types.FieldName) *model.TableInfo { var tableList []*ast.TableName tableList = extractTableList(from, tableList, true) for _, field := range tableList { if field.Name.L == name.TblName.L { -<<<<<<< HEAD:planner/core/logical_plan_builder.go - return field.TableInfo -======= - tnW := b.resolveCtx.GetTableName(field) - if tnW != nil { - return tnW.TableInfo - } - // when the Select is inside a view, it's not pre-processed, tnW is nil. + if field.TableInfo != nil { + return field.TableInfo + } if b.isCreateView { - // Ignore during create return nil } - tblInfo, err := b.is.TableInfoByName(name.DBName, name.TblName) + tbl, err := b.is.TableByName(name.DBName, name.TblName) if err != nil { return nil } + tblInfo := tbl.Meta() return tblInfo ->>>>>>> d0150c0a686 (planner: ONLY_FULL_GROUP_BY sql_mode was not working with VIEWs (#57473)):pkg/planner/core/logical_plan_builder.go } } return nil @@ -3174,7 +3168,7 @@ func buildWhereFuncDepend(p LogicalPlan, where ast.ExprNode) (map[*types.FieldNa return colDependMap, nil } -func buildJoinFuncDepend(p LogicalPlan, from ast.ResultSetNode) (map[*types.FieldName]*types.FieldName, error) { +func (b *PlanBuilder) buildJoinFuncDepend(p LogicalPlan, from ast.ResultSetNode) (map[*types.FieldName]*types.FieldName, error) { switch x := from.(type) { case *ast.Join: if x.On == nil { @@ -3190,7 +3184,7 @@ func buildJoinFuncDepend(p LogicalPlan, from ast.ResultSetNode) (map[*types.Fiel if lCol == nil || rCol == nil { continue } - lTbl := tblInfoFromCol(x.Left, lCol) + lTbl := b.tblInfoFromCol(x.Left, lCol) if lTbl == nil { lCol, rCol = rCol, lCol } @@ -3396,7 +3390,7 @@ func extractSingeValueColNamesFromWhere(p LogicalPlan, where ast.ExprNode, gbyOr } } -func (*PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *ast.SelectStmt) error { +func (b *PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *ast.SelectStmt) error { gbyOrSingleValueColNames := make(map[*types.FieldName]struct{}, len(sel.Fields.Fields)) gbyExprs := make([]ast.ExprNode, 0, len(sel.Fields.Fields)) for _, byItem := range sel.GroupBy.Items { @@ -3443,13 +3437,13 @@ func (*PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *ast. if err != nil { return err } - joinDepends, err := buildJoinFuncDepend(p, sel.From.TableRefs) + joinDepends, err := b.buildJoinFuncDepend(p, sel.From.TableRefs) if err != nil { return err } tblMap := make(map[*model.TableInfo]struct{}, len(notInGbyOrSingleValueColNames)) for name, errExprLoc := range notInGbyOrSingleValueColNames { - tblInfo := tblInfoFromCol(sel.From.TableRefs, name) + tblInfo := b.tblInfoFromCol(sel.From.TableRefs, name) if tblInfo == nil { continue } @@ -3471,7 +3465,7 @@ func (*PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *ast. return nil } -func (*PlanBuilder) checkOnlyFullGroupByWithOutGroupClause(p LogicalPlan, sel *ast.SelectStmt) error { +func (b *PlanBuilder) checkOnlyFullGroupByWithOutGroupClause(p LogicalPlan, sel *ast.SelectStmt) error { resolver := colResolverForOnlyFullGroupBy{ firstOrderByAggColIdx: -1, } @@ -3506,7 +3500,7 @@ func (*PlanBuilder) checkOnlyFullGroupByWithOutGroupClause(p LogicalPlan, sel *a return err } - joinDepends, err := buildJoinFuncDepend(p, sel.From.TableRefs) + joinDepends, err := b.buildJoinFuncDepend(p, sel.From.TableRefs) if err != nil { return err } @@ -3520,7 +3514,7 @@ func (*PlanBuilder) checkOnlyFullGroupByWithOutGroupClause(p LogicalPlan, sel *a if _, ok := singleValueColNames[fieldName]; ok { continue } - tblInfo := tblInfoFromCol(sel.From.TableRefs, fieldName) + tblInfo := b.tblInfoFromCol(sel.From.TableRefs, fieldName) if tblInfo == nil { continue } From c14615825e3eacc08c264a1ec3377c9eb635e567 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 7 Feb 2025 13:24:58 +0100 Subject: [PATCH 3/3] Removed test files from bad merge --- pkg/planner/core/issuetest/BUILD.bazel | 27 ---- .../core/issuetest/planner_issue_test.go | 136 ------------------ 2 files changed, 163 deletions(-) delete mode 100644 pkg/planner/core/issuetest/BUILD.bazel delete mode 100644 pkg/planner/core/issuetest/planner_issue_test.go diff --git a/pkg/planner/core/issuetest/BUILD.bazel b/pkg/planner/core/issuetest/BUILD.bazel deleted file mode 100644 index 2fa7a2e123983..0000000000000 --- a/pkg/planner/core/issuetest/BUILD.bazel +++ /dev/null @@ -1,27 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_test") - -go_test( - name = "issuetest_test", - timeout = "short", - srcs = [ - "main_test.go", - "planner_issue_test.go", - ], - data = glob(["testdata/**"]), - flaky = True, - race = "on", - shard_count = 4, - deps = [ - "//pkg/parser", - "//pkg/planner", - "//pkg/planner/core", - "//pkg/planner/core/base", - "//pkg/planner/core/resolve", - "//pkg/testkit", - "//pkg/testkit/testdata", - "//pkg/testkit/testmain", - "//pkg/testkit/testsetup", - "@com_github_stretchr_testify//require", - "@org_uber_go_goleak//:goleak", - ], -) diff --git a/pkg/planner/core/issuetest/planner_issue_test.go b/pkg/planner/core/issuetest/planner_issue_test.go deleted file mode 100644 index fda17d6cfce17..0000000000000 --- a/pkg/planner/core/issuetest/planner_issue_test.go +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package issuetest - -import ( - "context" - "testing" - - "github.com/pingcap/tidb/pkg/parser" - "github.com/pingcap/tidb/pkg/planner" - "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/planner/core/base" - "github.com/pingcap/tidb/pkg/planner/core/resolve" - "github.com/pingcap/tidb/pkg/testkit" - "github.com/stretchr/testify/require" -) - -// It's a case for Columns in tableScan and indexScan with double reader -func TestIssue43461(t *testing.T) { - store, domain := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t(a int, b int, c int, index b(b), index b_c(b, c)) partition by hash(a) partitions 4;") - tk.MustExec("analyze table t") - - stmt, err := parser.New().ParseOneStmt("select * from t use index(b) where b > 1 order by b limit 1", "", "") - require.NoError(t, err) - - nodeW := resolve.NewNodeW(stmt) - p, _, err := planner.Optimize(context.TODO(), tk.Session(), nodeW, domain.InfoSchema()) - require.NoError(t, err) - require.NotNil(t, p) - - var idxLookUpPlan *core.PhysicalIndexLookUpReader - var ok bool - - for { - idxLookUpPlan, ok = p.(*core.PhysicalIndexLookUpReader) - if ok { - break - } - p = p.(base.PhysicalPlan).Children()[0] - } - require.True(t, ok) - - is := idxLookUpPlan.IndexPlans[0].(*core.PhysicalIndexScan) - ts := idxLookUpPlan.TablePlans[0].(*core.PhysicalTableScan) - - require.NotEqual(t, is.Columns, ts.Columns) -} - -func Test53726(t *testing.T) { - // test for RemoveUnnecessaryFirstRow - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t7(c int); ") - tk.MustExec("insert into t7 values (575932053), (-258025139);") - tk.MustQuery("select distinct cast(c as decimal), cast(c as signed) from t7"). - Sort().Check(testkit.Rows("-258025139 -258025139", "575932053 575932053")) - tk.MustQuery("explain select distinct cast(c as decimal), cast(c as signed) from t7"). - Check(testkit.Rows( - "HashAgg_8 8000.00 root group by:Column#7, Column#8, funcs:firstrow(Column#7)->Column#3, funcs:firstrow(Column#8)->Column#4", - "└─TableReader_9 8000.00 root data:HashAgg_4", - " └─HashAgg_4 8000.00 cop[tikv] group by:cast(test.t7.c, bigint(22) BINARY), cast(test.t7.c, decimal(10,0) BINARY), ", - " └─TableFullScan_7 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo")) - - tk.MustExec("analyze table t7 all columns") - tk.MustQuery("select distinct cast(c as decimal), cast(c as signed) from t7"). - Sort(). - Check(testkit.Rows("-258025139 -258025139", "575932053 575932053")) - tk.MustQuery("explain select distinct cast(c as decimal), cast(c as signed) from t7"). - Check(testkit.Rows( - "HashAgg_6 2.00 root group by:Column#11, Column#12, funcs:firstrow(Column#11)->Column#3, funcs:firstrow(Column#12)->Column#4", - "└─Projection_12 2.00 root cast(test.t7.c, decimal(10,0) BINARY)->Column#11, cast(test.t7.c, bigint(22) BINARY)->Column#12", - " └─TableReader_11 2.00 root data:TableFullScan_10", - " └─TableFullScan_10 2.00 cop[tikv] table:t7 keep order:false")) -} - -func TestIssue54535(t *testing.T) { - // test for tidb_enable_inl_join_inner_multi_pattern system variable - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set session tidb_enable_inl_join_inner_multi_pattern='ON'") - tk.MustExec("create table ta(a1 int, a2 int, a3 int, index idx_a(a1))") - tk.MustExec("create table tb(b1 int, b2 int, b3 int, index idx_b(b1))") - tk.MustExec("analyze table ta") - tk.MustExec("analyze table tb") - - tk.MustQuery("explain SELECT /*+ inl_join(tmp) */ * FROM ta, (SELECT b1, COUNT(b3) AS cnt FROM tb GROUP BY b1, b2) as tmp where ta.a1 = tmp.b1"). - Check(testkit.Rows( - "Projection_9 9990.00 root test.ta.a1, test.ta.a2, test.ta.a3, test.tb.b1, Column#9", - "└─IndexJoin_16 9990.00 root inner join, inner:HashAgg_14, outer key:test.ta.a1, inner key:test.tb.b1, equal cond:eq(test.ta.a1, test.tb.b1)", - " ├─TableReader_43(Build) 9990.00 root data:Selection_42", - " │ └─Selection_42 9990.00 cop[tikv] not(isnull(test.ta.a1))", - " │ └─TableFullScan_41 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", - " └─HashAgg_14(Probe) 79840080.00 root group by:test.tb.b1, test.tb.b2, funcs:count(Column#11)->Column#9, funcs:firstrow(test.tb.b1)->test.tb.b1", - " └─IndexLookUp_15 79840080.00 root ", - " ├─Selection_12(Build) 9990.00 cop[tikv] not(isnull(test.tb.b1))", - " │ └─IndexRangeScan_10 10000.00 cop[tikv] table:tb, index:idx_b(b1) range: decided by [eq(test.tb.b1, test.ta.a1)], keep order:false, stats:pseudo", - " └─HashAgg_13(Probe) 79840080.00 cop[tikv] group by:test.tb.b1, test.tb.b2, funcs:count(test.tb.b3)->Column#11", - " └─TableRowIDScan_11 9990.00 cop[tikv] table:tb keep order:false, stats:pseudo")) - // test for issues/55169 - tk.MustExec("create table t1(col_1 int, index idx_1(col_1));") - tk.MustExec("create table t2(col_1 int, col_2 int, index idx_2(col_1));") - tk.MustQuery("select /*+ inl_join(tmp) */ * from t1 inner join (select col_1, group_concat(col_2) from t2 group by col_1) tmp on t1.col_1 = tmp.col_1;").Check(testkit.Rows()) - tk.MustQuery("select /*+ inl_join(tmp) */ * from t1 inner join (select col_1, group_concat(distinct col_2 order by col_2) from t2 group by col_1) tmp on t1.col_1 = tmp.col_1;").Check(testkit.Rows()) -} - -func TestIssue53175(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`create table t(a int)`) - tk.MustExec(`set @@sql_mode = default`) - tk.MustQuery(`select @@sql_mode REGEXP 'ONLY_FULL_GROUP_BY'`).Check(testkit.Rows("1")) - tk.MustContainErrMsg(`select * from t group by null`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") - tk.MustExec(`create view v as select * from t group by null`) - tk.MustContainErrMsg(`select * from v`, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") - tk.MustExec(`set @@sql_mode = ''`) - tk.MustQuery(`select * from t group by null`) - tk.MustQuery(`select * from v`) -}