planner: using the funcdep to check the only_full_group_by (#33567)

ref pingcap/tidb#29766
This commit is contained in:
Yiding Cui
2022-04-21 00:44:04 +08:00
committed by GitHub
parent ebffff011f
commit 571d97bb16
6 changed files with 840 additions and 63 deletions

View File

@ -701,6 +701,7 @@ func TestOnlyFullGroupBy(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'")
tk.MustExec("set @@session.tidb_enable_new_only_full_group_by_check = 'on';")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int not null primary key, b int not null, c int default null, d int not null, unique key I_b_c (b,c), unique key I_b_d (b,d))")
tk.MustExec("create table x(a int not null primary key, b int not null, c int default null, d int not null, unique key I_b_c (b,c), unique key I_b_d (b,d))")
@ -742,16 +743,22 @@ func TestOnlyFullGroupBy(t *testing.T) {
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err)
err = tk.ExecToErr("select count(b), c from t")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err)
tk.MustQuery("select count(b), any_value(c) from t")
tk.MustQuery("select count(b), any_value(c) + 2 from t")
err = tk.ExecToErr("select distinct a, b, count(a) from t")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err)
// test compatible with sql_mode = ONLY_FULL_GROUP_BY
tk.MustQuery("select a from t group by a,b,c")
tk.MustQuery("select b from t group by b")
err = tk.ExecToErr("select b*rand() from t group by b")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
tk.MustQuery("select b as e from t group by b")
tk.MustQuery("select b+c from t group by b+c")
tk.MustQuery("select b+c, min(a) from t group by b+c, b-c")
tk.MustQuery("select b+c, min(a) from t group by b, c")
tk.MustQuery("select b+c from t group by b,c")
err = tk.ExecToErr("select b+c from (select b, b+rand() as c from t) t group by b")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
tk.MustQuery("select b between c and d from t group by b,c,d")
tk.MustQuery("select case b when 1 then c when 2 then d else d end from t group by b,c,d")
tk.MustQuery("select c > (select b from t) from t group by c")
@ -779,8 +786,7 @@ func TestOnlyFullGroupBy(t *testing.T) {
tk.MustQuery("select t.*, x.* from t, x where t.b = x.b and t.d = x.d group by t.b, t.d")
tk.MustQuery("select t.*, x.* from t, x where t.b = x.a group by t.b, t.d")
tk.MustQuery("select t.b, x.* from t, x where t.b = x.a group by t.b")
err = tk.ExecToErr("select t.*, x.* from t, x where t.c = x.a group by t.b, t.c")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
tk.MustQuery("select t.*, x.* from t, x where t.c = x.a group by t.b, t.c")
// test functional dependency derived from keys in join
tk.MustQuery("select t.*, x.* from t inner join x on t.a = x.a group by t.a")
tk.MustQuery("select t.*, x.* from t inner join x on (t.b = x.b and t.d = x.d) group by t.b, x.d")
@ -794,11 +800,10 @@ func TestOnlyFullGroupBy(t *testing.T) {
err = tk.ExecToErr("select t.b, x.* from t right join x on t.b = x.b group by t.b, x.d")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
// FixMe: test functional dependency of derived table
// tk.MustQuery("select * from (select * from t) as e group by a")
// tk.MustQuery("select * from (select * from t) as e group by b,d")
// err = tk.ExecToErr("select * from (select * from t) as e group by b,c")
// require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
tk.MustQuery("select * from (select * from t) as e group by a")
tk.MustQuery("select * from (select * from t) as e group by b,d")
err = tk.ExecToErr("select * from (select * from t) as e group by b,c")
require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err)
// test order by
tk.MustQuery("select c from t group by c,d order by d")

View File

@ -56,6 +56,8 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/plancodec"
"github.com/pingcap/tidb/util/set"
)
@ -1321,6 +1323,109 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields
}
}
proj.SetChildren(p)
// delay the only-full-group-by-check in create view statement to later query.
if !b.isCreateView && b.ctx.GetSessionVars().OptimizerEnableNewOnlyFullGroupByCheck && b.ctx.GetSessionVars().SQLMode.HasOnlyFullGroupBy() {
fds := proj.ExtractFD()
// Projection -> Children -> ...
// Let the projection itself to evaluate the whole FD, which will build the connection
// 1: from select-expr to registered-expr
// 2: from base-column to select-expr
// After that
if fds.HasAggBuilt {
for offset, expr := range proj.Exprs[:len(fields)] {
// skip the auxiliary column in agg appended to select fields, which mainly comes from two kind of cases:
// 1: having agg(t.a), this will append t.a to the select fields, if it isn't here.
// 2: order by agg(t.a), this will append t.a to the select fields, if it isn't here.
if fields[offset].AuxiliaryColInAgg {
continue
}
item := fd.NewFastIntSet()
switch x := expr.(type) {
case *expression.Column:
item.Insert(int(x.UniqueID))
case *expression.ScalarFunction:
if expression.CheckFuncInExpr(x, ast.AnyValue) {
continue
}
scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode(p.SCtx().GetSessionVars().StmtCtx))))
if !ok {
logutil.BgLogger().Warn("Error occurred while maintaining the functional dependency")
continue
}
item.Insert(scalarUniqueID)
default:
}
// Rule #1, if there are no group cols, the col in the order by shouldn't be limited.
if fds.GroupByCols.Only1Zero() && fields[offset].AuxiliaryColInOrderBy {
continue
}
// Rule #2, if select fields are constant, it's ok.
if item.SubsetOf(fds.ConstantCols()) {
continue
}
// Rule #3, if select fields are subset of group by items, it's ok.
if item.SubsetOf(fds.GroupByCols) {
continue
}
// Rule #4, if select fields are dependencies of Strict FD with determinants in group-by items, it's ok.
// lax FD couldn't be done here, eg: for unique key (b), index key NULL & NULL are different rows with
// uncertain other column values.
strictClosure := fds.ClosureOfStrict(fds.GroupByCols)
if item.SubsetOf(strictClosure) {
continue
}
// locate the base col that are not in (constant list / group by list / strict fd closure) for error show.
baseCols := expression.ExtractColumns(expr)
errShowCol := baseCols[0]
for _, col := range baseCols {
colSet := fd.NewFastIntSet(int(col.UniqueID))
if !colSet.SubsetOf(strictClosure) {
errShowCol = col
break
}
}
// better use the schema alias name firstly if any.
name := ""
for idx, schemaCol := range proj.Schema().Columns {
if schemaCol.UniqueID == errShowCol.UniqueID {
name = proj.names[idx].String()
break
}
}
if name == "" {
name = errShowCol.OrigName
}
// Only1Zero is to judge whether it's no-group-by-items case.
if !fds.GroupByCols.Only1Zero() {
return nil, nil, 0, ErrFieldNotInGroupBy.GenWithStackByArgs(offset+1, ErrExprInSelect, name)
}
return nil, nil, 0, ErrMixOfGroupFuncAndFields.GenWithStackByArgs(offset+1, name)
}
if fds.GroupByCols.Only1Zero() {
// maxOneRow is delayed from agg's ExtractFD logic since some details listed in it.
projectionUniqueIDs := fd.NewFastIntSet()
for _, expr := range proj.Exprs {
switch x := expr.(type) {
case *expression.Column:
projectionUniqueIDs.Insert(int(x.UniqueID))
case *expression.ScalarFunction:
scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode(p.SCtx().GetSessionVars().StmtCtx))))
if !ok {
logutil.BgLogger().Warn("Error occurred while maintaining the functional dependency")
continue
}
projectionUniqueIDs.Insert(scalarUniqueID)
}
}
fds.MaxOneRow(projectionUniqueIDs)
}
// for select * from view (include agg), outer projection don't have to check select list with the inner group-by flag.
fds.HasAggBuilt = false
}
}
return proj, proj.Exprs, oldLen, nil
}
@ -2022,6 +2127,7 @@ func (a *havingWindowAndOrderbyExprResolver) resolveFromPlan(v *ast.ColumnNameEx
}
}
if err != nil || idx < 0 {
// nowhere to be found.
return -1, err
}
}
@ -2585,6 +2691,22 @@ func (b *PlanBuilder) resolveCorrelatedAggregates(ctx context.Context, sel *ast.
}
correlatedAggMap := make(map[*ast.AggregateFuncExpr]int)
for _, aggFunc := range correlatedAggList {
colMap := make(map[*types.FieldName]struct{}, len(p.Schema().Columns))
allColFromAggExprNode(p, aggFunc, colMap)
for k := range colMap {
colName := &ast.ColumnName{
Schema: k.DBName,
Table: k.TblName,
Name: k.ColName,
}
// Add the column referred in the agg func into the select list. So that we can resolve the agg func correctly.
// And we need set the AuxiliaryColInAgg to true to help our only_full_group_by checker work correctly.
sel.Fields.Fields = append(sel.Fields.Fields, &ast.SelectField{
Auxiliary: true,
AuxiliaryColInAgg: true,
Expr: &ast.ColumnNameExpr{Name: colName},
})
}
correlatedAggMap[aggFunc] = len(sel.Fields.Fields)
sel.Fields.Fields = append(sel.Fields.Fields, &ast.SelectField{
Auxiliary: true,
@ -3153,6 +3275,28 @@ func (c *colResolverForOnlyFullGroupBy) Leave(node ast.Node) (ast.Node, bool) {
return node, true
}
type aggColNameResolver struct {
colNameResolver
}
func (c *aggColNameResolver) Enter(inNode ast.Node) (ast.Node, bool) {
switch inNode.(type) {
case *ast.ColumnNameExpr:
return inNode, true
}
return inNode, false
}
func allColFromAggExprNode(p LogicalPlan, n ast.Node, names map[*types.FieldName]struct{}) {
extractor := &aggColNameResolver{
colNameResolver: colNameResolver{
p: p,
names: names,
},
}
n.Accept(extractor)
}
type colNameResolver struct {
p LogicalPlan
names map[*types.FieldName]struct{}
@ -3641,7 +3785,7 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L
}
}
if b.ctx.GetSessionVars().SQLMode.HasOnlyFullGroupBy() && sel.From != nil {
if b.ctx.GetSessionVars().SQLMode.HasOnlyFullGroupBy() && sel.From != nil && !b.ctx.GetSessionVars().OptimizerEnableNewOnlyFullGroupByCheck {
err = b.checkOnlyFullGroupBy(p, sel)
if err != nil {
return nil, err

View File

@ -31,7 +31,6 @@ import (
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/ranger"
"go.uber.org/zap"
@ -186,6 +185,8 @@ func (p *LogicalJoin) ExtractFD() *fd.FDSet {
switch p.JoinType {
case InnerJoin:
return p.extractFDForInnerJoin(nil)
case LeftOuterJoin, RightOuterJoin:
return p.extractFDForOuterJoin(nil)
case SemiJoin:
return p.extractFDForSemiJoin(nil)
default:
@ -238,8 +239,8 @@ func (p *LogicalJoin) extractFDForInnerJoin(filtersFromApply []expression.Expres
fds.HashCodeToUniqueID = rightFD.HashCodeToUniqueID
} else {
for k, v := range rightFD.HashCodeToUniqueID {
// If there's same constant in the different subquery, we might go into this IF branch.
if _, ok := fds.HashCodeToUniqueID[k]; ok {
logutil.BgLogger().Warn("Error occurred while maintaining functional dependency")
continue
}
fds.HashCodeToUniqueID[k] = v
@ -253,6 +254,100 @@ func (p *LogicalJoin) extractFDForInnerJoin(filtersFromApply []expression.Expres
return fds
}
func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expression) *fd.FDSet {
outerFD, innerFD := p.children[0].ExtractFD(), p.children[1].ExtractFD()
innerCondition := p.RightConditions
outerCondition := p.LeftConditions
outerCols, innerCols := fd.NewFastIntSet(), fd.NewFastIntSet()
for _, col := range p.children[0].Schema().Columns {
outerCols.Insert(int(col.UniqueID))
}
for _, col := range p.children[1].Schema().Columns {
innerCols.Insert(int(col.UniqueID))
}
if p.JoinType == RightOuterJoin {
innerFD, outerFD = outerFD, innerFD
innerCondition = p.LeftConditions
outerCondition = p.RightConditions
innerCols, outerCols = outerCols, innerCols
}
eqCondSlice := expression.ScalarFuncs2Exprs(p.EqualConditions)
allConds := append(eqCondSlice, p.OtherConditions...)
allConds = append(allConds, innerCondition...)
allConds = append(allConds, outerCondition...)
allConds = append(allConds, filtersFromApply...)
notNullColsFromFilters := extractNotNullFromConds(allConds, p)
filterFD := &fd.FDSet{HashCodeToUniqueID: make(map[string]int)}
constUniqueIDs := extractConstantCols(allConds, p.SCtx(), filterFD)
equivUniqueIDs := extractEquivalenceCols(allConds, p.SCtx(), filterFD)
filterFD.AddConstants(constUniqueIDs)
equivOuterUniqueIDs := fd.NewFastIntSet()
equivAcrossNum := 0
for _, equiv := range equivUniqueIDs {
filterFD.AddEquivalence(equiv[0], equiv[1])
if equiv[0].SubsetOf(outerCols) && equiv[1].SubsetOf(innerCols) {
equivOuterUniqueIDs.UnionWith(equiv[0])
equivAcrossNum++
continue
}
if equiv[0].SubsetOf(innerCols) && equiv[1].SubsetOf(outerCols) {
equivOuterUniqueIDs.UnionWith(equiv[1])
equivAcrossNum++
}
}
filterFD.MakeNotNull(notNullColsFromFilters)
// pre-perceive the filters for the convenience judgement of 3.3.1.
var opt fd.ArgOpts
if equivAcrossNum > 0 {
// find the equivalence FD across left and right cols.
var outConditionCols []*expression.Column
if len(outerCondition) != 0 {
outConditionCols = append(outConditionCols, expression.ExtractColumnsFromExpressions(nil, outerCondition, nil)...)
}
if len(p.OtherConditions) != 0 {
// other condition may contain right side cols, it doesn't affect the judgement of intersection of non-left-equiv cols.
outConditionCols = append(outConditionCols, expression.ExtractColumnsFromExpressions(nil, p.OtherConditions, nil)...)
}
outerConditionUniqueIDs := fd.NewFastIntSet()
for _, col := range outConditionCols {
outerConditionUniqueIDs.Insert(int(col.UniqueID))
}
// judge whether left filters is on non-left-equiv cols.
if outerConditionUniqueIDs.Intersects(outerCols.Difference(equivOuterUniqueIDs)) {
opt.SkipFDRule331 = true
}
} else {
// if there is none across equivalence condition, skip rule 3.3.1.
opt.SkipFDRule331 = true
}
opt.OnlyInnerFilter = len(eqCondSlice) == 0 && len(outerCondition) == 0
if opt.OnlyInnerFilter {
// if one of the inner condition is constant false, the inner side are all null, left make constant all of that.
for _, one := range innerCondition {
if c, ok := one.(*expression.Constant); ok && c.DeferredExpr == nil && c.ParamMarker == nil {
if isTrue, err := c.Value.ToBool(p.ctx.GetSessionVars().StmtCtx); err == nil {
if isTrue == 0 {
// c is false
opt.InnerIsFalse = true
}
}
}
}
}
fds := outerFD
fds.MakeOuterJoin(innerFD, filterFD, outerCols, innerCols, &opt)
p.fdSet = fds
return fds
}
// GetJoinKeys extracts join keys(columns) from EqualConditions. It returns left join keys, right
// join keys and an `isNullEQ` array which means the `joinKey[i]` is a `NullEQ` function. The `hasNullEQ`
// means whether there is a `NullEQ` of a join key.
@ -472,28 +567,6 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet {
fds.MakeNotNull(notnullColsUniqueIDs)
// select max(a) from t group by b, we should project both `a` & `b` to maintain the FD down here, even if select-fields only contain `a`.
fds.ProjectCols(outputColsUniqueIDs.Union(fds.GroupByCols))
if fds.GroupByCols.Only1Zero() {
// maxOneRow is delayed from agg's ExtractFD logic since some details listed in it.
projectionUniqueIDs := fd.NewFastIntSet()
for _, expr := range p.Exprs {
switch x := expr.(type) {
case *expression.Column:
projectionUniqueIDs.Insert(int(x.UniqueID))
case *expression.ScalarFunction:
scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode(p.SCtx().GetSessionVars().StmtCtx))))
if !ok {
logutil.BgLogger().Warn("Error occurred while maintaining the functional dependency")
continue
}
projectionUniqueIDs.Insert(scalarUniqueID)
}
}
fds.MaxOneRow(projectionUniqueIDs)
}
// for select * from view (include agg), outer projection don't have to check select list with the inner group-by flag.
fds.HasAggBuilt = false
// just trace it down in every operator for test checking.
p.fdSet = fds
return fds
@ -874,7 +947,7 @@ func (p *LogicalSelection) ExtractFD() *fd.FDSet {
// join's schema will miss t2.a while join.full schema has. since selection
// itself doesn't contain schema, extracting schema should tell them apart.
var columns []*expression.Column
if join, ok := p.children[0].(*LogicalJoin); ok {
if join, ok := p.children[0].(*LogicalJoin); ok && join.fullSchema != nil {
columns = join.fullSchema.Columns
} else {
columns = p.Schema().Columns
@ -971,6 +1044,8 @@ func (la *LogicalApply) ExtractFD() *fd.FDSet {
switch la.JoinType {
case InnerJoin:
return la.extractFDForInnerJoin(eqCond)
case LeftOuterJoin, RightOuterJoin:
return la.extractFDForOuterJoin(eqCond)
case SemiJoin:
return la.extractFDForSemiJoin(eqCond)
default:

View File

@ -19,8 +19,6 @@ import (
"fmt"
"testing"
"github.com/stretchr/testify/require"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
@ -28,6 +26,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/hint"
"github.com/stretchr/testify/require"
)
func testGetIS(t *testing.T, ctx sessionctx.Context) infoschema.InfoSchema {
@ -46,7 +45,6 @@ func TestFDSet_ExtractFD(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set sql_mode = ''")
tk.MustExec("set @@session.tidb_enable_new_only_full_group_by_check = 'on';")
tk.MustExec("create table t1(a int key, b int, c int, unique(b,c))")
tk.MustExec("create table t2(m int key, n int, p int, unique(m,n))")
@ -91,31 +89,31 @@ func TestFDSet_ExtractFD(t *testing.T) {
sql: "select b+1, sum(a) from t1 group by(b)",
// The final ones are b -> (b+1), b -> sum(a)
best: "DataScan(t1)->Aggr(sum(test.t1.a),firstrow(test.t1.b))->Projection",
fd: "{(1)-->(2,3), (2,3)~~>(1)} >>> {(2)-->(4)} >>> {(2)-->(4,5)}",
fd: "{(1)-->(2,3), (2,3)~~>(1)} >>> {(1)-->(2,3), (2,3)~~>(1), (2)-->(4)} >>> {(2)-->(4,5)}",
},
{
sql: "select b+1, b, sum(a) from t1 group by(b)",
// The final ones are b -> (b+1), b -> sum(a)
best: "DataScan(t1)->Aggr(sum(test.t1.a),firstrow(test.t1.b))->Projection",
fd: "{(1)-->(2,3), (2,3)~~>(1)} >>> {(2)-->(4)} >>> {(2)-->(4,5)}",
fd: "{(1)-->(2,3), (2,3)~~>(1)} >>> {(1)-->(2,3), (2,3)~~>(1), (2)-->(4)} >>> {(2)-->(4,5)}",
},
// test for table x1 and x2
{
sql: "select a from x1 group by a,b,c",
best: "DataScan(x1)->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2,3), (2,3)~~>(1)}",
},
{
sql: "select b from x1 group by b",
best: "DataScan(x1)->Aggr(firstrow(test.x1.b))->Projection",
// b --> b is natural existed, so it won't exist in fd.
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {}",
},
{
sql: "select b as e from x1 group by b",
best: "DataScan(x1)->Aggr(firstrow(test.x1.b))->Projection",
// b --> b is naturally existed, so it won't exist in fd.
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {}",
},
{
sql: "select b+c from x1 group by b+c",
@ -125,23 +123,23 @@ func TestFDSet_ExtractFD(t *testing.T) {
// b+c is an expr assigned with new plan ID when building upper-layer projection.
// when extracting FD after build phase is done, we should be able to recognize a+b in lower-layer group by item with the same unique ID.
// that's why we introduce session variable MapHashCode2UniqueID4ExtendedCol in.
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(2,3)-->(5)} >>> {(2,3)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3), (2,3)-->(5)} >>> {(2,3)-->(5)}",
},
{
sql: "select b+c, min(a) from x1 group by b+c, b-c",
best: "DataScan(x1)->Aggr(min(test.x1.a),firstrow(test.x1.b),firstrow(test.x1.c))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(2,3)-->(6,8), (6,8)-->(5)} >>> {(2,3)-->(6,8), (6,8)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3), (2,3)-->(6,7), (6,7)-->(5)} >>> {(2,3)-->(6,7), (6,7)-->(5)}",
},
{
sql: "select b+c, min(a) from x1 group by b, c",
best: "DataScan(x1)->Aggr(min(test.x1.a),firstrow(test.x1.b),firstrow(test.x1.c))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(2,3)-->(5)} >>> {(2,3)-->(5,6)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3), (2,3)-->(5)} >>> {(2,3)-->(5,6)}",
},
{
sql: "select b+c from x1 group by b,c",
best: "DataScan(x1)->Aggr(firstrow(test.x1.b),firstrow(test.x1.c))->Projection",
// b --> b is naturally existed, so it won't exist in fd.
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {(2,3)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(2,3)-->(5)}",
},
{
sql: "select case b when 1 then c when 2 then d else d end from x1 group by b,c,d",
@ -152,23 +150,23 @@ func TestFDSet_ExtractFD(t *testing.T) {
// scalar sub query will be substituted with constant datum.
sql: "select c > (select b from x1) from x1 group by c",
best: "DataScan(x1)->Aggr(firstrow(test.x1.c))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {(3)-->(15)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(3)-->(15)}",
},
{
sql: "select exists (select * from x1) from x1 group by d",
best: "DataScan(x1)->Aggr(firstrow(1))->Projection",
// 14 is added in the logicAgg pruning process cause all the columns of agg has been pruned.
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {()-->(13)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {()-->(13)}",
},
{
sql: "select c is null from x1 group by c",
best: "DataScan(x1)->Aggr(firstrow(test.x1.c))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {(3)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(3)-->(5)}",
},
{
sql: "select c is true from x1 group by c",
best: "DataScan(x1)->Aggr(firstrow(test.x1.c))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {(3)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(3)-->(5)}",
},
{
sql: "select (c+b)*d from x1 group by c,b,d",
@ -184,7 +182,7 @@ func TestFDSet_ExtractFD(t *testing.T) {
{
sql: "select b like '%a' from x1 group by b",
best: "DataScan(x1)->Aggr(firstrow(test.x1.b))->Projection",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {} >>> {(2)-->(5)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(2)-->(5)}",
},
// test functional dependency on primary key
{
@ -206,7 +204,7 @@ func TestFDSet_ExtractFD(t *testing.T) {
// c = d derives:
// 1: c and d are not null, make lax FD (2,3)~~>(1,4) to be strict one.
// 2: c and d are equivalent.
fd: "{(1)-->(2-4), (2,3)-->(1,4), (2,4)-->(1,3), (3,4)==(3,4)} >>> {(1)-->(2-4), (2,3)-->(1,4), (2,4)-->(1,3), (3,4)==(3,4)} >>> {(1)-->(2-4), (2,3)-->(1,4), (2,4)-->(1,3), (3,4)==(3,4)}",
fd: "{(1)-->(2-4), (2,3)~~>(1,4), (2,4)-->(1,3)} >>> {(1)-->(2-4), (2,3)-->(1,4), (2,4)-->(1,3), (3,4)==(3,4)} >>> {(1)-->(2-4), (2,3)-->(1,4), (2,4)-->(1,3), (3,4)==(3,4)}",
},
}
@ -287,7 +285,7 @@ func TestFDSet_ExtractFDForApply(t *testing.T) {
sql: "select * from X where exists (select * from Y where m=a and b=1)",
best: "Join{DataScan(X)->DataScan(Y)}(test.x.a,test.y.m)->Projection", // semi join
// b=1 is semi join's left condition which should be conserved.
fd: "{(1)-->(3-5), (2,3)~~>(1,4,5), ()-->(2)} >>> {(1)-->(3-5), (2,3)~~>(1,4,5), ()-->(2)}",
fd: "{(1)-->(2-5), (2,3)~~>(1,4,5)} >>> {(1)-->(2-5), (2,3)~~>(1,4,5)}",
},
{
sql: "select * from (select b,c,d,e from X) X1 where exists (select * from Y where p=q and n=1) ",
@ -297,7 +295,7 @@ func TestFDSet_ExtractFDForApply(t *testing.T) {
{
sql: "select * from (select b,c,d,e from X) X1 where exists (select * from Y where p=b and n=1) ",
best: "Join{DataScan(X)->DataScan(Y)}(test.x.b,test.y.p)->Projection", // semi join
fd: "{(1)-->(2-5), (2,3)~~>(1,4,5)} >>> {(2,3)~~>(4,5)}",
fd: "{(2,3)~~>(4,5)} >>> {(2,3)~~>(4,5)}",
},
{
sql: "select * from X where exists (select m, p, q from Y where n=a and p=1)",
@ -330,3 +328,52 @@ func TestFDSet_ExtractFDForApply(t *testing.T) {
require.Equal(t, tt.fd, plannercore.FDToString(p.(plannercore.LogicalPlan)), comment)
}
}
func TestFDSet_MakeOuterJoin(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
par := parser.New()
par.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true})
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_enable_new_only_full_group_by_check = 'on';")
tk.MustExec("CREATE TABLE X (a INT PRIMARY KEY, b INT, c INT, d INT, e INT)")
tk.MustExec("CREATE UNIQUE INDEX uni ON X (b, c)")
tk.MustExec("CREATE TABLE Y (m INT, n INT, p INT, q INT, PRIMARY KEY (m, n))")
tests := []struct {
sql string
best string
fd string
}{
{
sql: "select * from X left outer join (select *, p+q from Y) Y1 ON true",
best: "Join{DataScan(X)->DataScan(Y)->Projection}->Projection",
fd: "{(1)-->(2-5), (2,3)~~>(1,4,5), (6,7)-->(8,9,11), (8,9)-->(11), (1,6,7)-->(2-5,8,9,11)} >>> {(1)-->(2-5), (2,3)~~>(1,4,5), (6,7)-->(8,9,11), (8,9)-->(11), (1,6,7)-->(2-5,8,9,11)}",
},
}
ctx := context.TODO()
is := testGetIS(t, tk.Session())
for i, tt := range tests {
comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql)
stmt, err := par.ParseOneStmt(tt.sql, "", "")
require.NoError(t, err, comment)
tk.Session().GetSessionVars().PlanID = 0
tk.Session().GetSessionVars().PlanColumnID = 0
err = plannercore.Preprocess(tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err, comment)
tk.Session().PrepareTSFuture(ctx)
builder, _ := plannercore.NewPlanBuilder().Init(tk.Session(), is, &hint.BlockHintProcessor{})
// extract FD to every OP
p, err := builder.Build(ctx, stmt)
require.NoError(t, err, comment)
p, err = plannercore.LogicalOptimizeTest(ctx, builder.GetOptFlag(), p.(plannercore.LogicalPlan))
require.NoError(t, err, comment)
require.Equal(t, tt.best, plannercore.ToString(p), comment)
// extract FD to every OP
p.(plannercore.LogicalPlan).ExtractFD()
require.Equal(t, tt.fd, plannercore.FDToString(p.(plannercore.LogicalPlan)), comment)
}
}

View File

@ -549,6 +549,273 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) {
// maintain a key?
}
// MakeOuterJoin generates the records the fdSet of the outer join.
//
// We always take the left side as the row-supplying side and the right side as the null-supplying side. (swap it if not)
// As we know, the outer join would generate null extended rows compared with the inner join.
// So we cannot do the same thing directly with the inner join. This function deals with the special cases of the outer join.
//
// Knowledge:
// 1: the filter condition related to the lhs column won't filter predicate-allowed rows and refuse null rows (left rows always remain)
// 2: the filter condition related to the rhs column won't filter NULL rows, although the filter has the not-null attribute. (null-appending happened after that)
//
// Notification:
// 1: the origin FD from the left side (rows-supplying) over the result of outer join filtered are preserved because
// it may be duplicated by multi-matching, but actually, they are the same left rows (don't violate FD definition).
//
// 2: the origin FD from the right side (nulls-supplying) over the result of outer join filtered may not be valid anymore.
//
// <1> strict FD may be wakened as a lax one. But if at least one non-NULL column is part of the determinant, the
// strict FD can be preserved.
// a b | c d e
// ------+----------------
// 1 1 | 1 NULL 1
// 1 2 | NULL NULL NULL
// 2 1 | NULL NULL NULL
// left join with (a,b) * (c,d,e) on (a=c and b=1), if there is a strict FD {d} -> {e} on the rhs. After supplied
// with null values, {d} -> {e} are degraded to a lax one {d} ~~> {e} as you see. the origin and supplied null value
// for d column determine different dependency. NULL -> 1 and NULL -> NULL which breaks strict FD definition.
//
// If the determinant contains at least a not null column for example c here, FD like {c,d} -> {e} can survive
// after the left join. Because you can not find two same key, one from the origin rows and the other one from the
// supplied rows.
//
// for lax FD, the supplied rows of null values don't affect lax FD itself. So we can keep it.
//
// <2> The FDSet should remove constant FD since null values may be substituted for some unmatched left rows. NULL is not a
// constant anymore.
//
// <3> equivalence FD should be removed since substituted null values are not equal to the other substituted null value.
//
// 3: the newly added FD from filters should take some consideration as below:
//
// <1> strict/lax FD: join key filter conditions can not produce new strict/lax FD yet (knowledge: 1&2).
//
// <2> constant FD from the join conditions is only used for checking other FD. We cannot keep itself.
// a b | c d
// -------+---------
// 1 1 | 1 1
// 1 2 | NULL NULL
// left join with (a,b) * (c,d) on (a=c and d=1), some rhs rows will be substituted with null values, and FD on rhs
// {d=1} are lost.
//
// a b | c d
// -------+---------
// 1 1 | 1 1
// 1 2 | NULL NULL
// left join with (a,b) * (c,d) on (a=c and b=1), it only gives the pass to the first matching, lhs other rows are still
// kept and appended with null values. So the FD on rhs {b=1} are not applicable to lhs rows.
//
// above all: constant FD are lost
//
// <3.1> equivalence FD: when the left join conditions only contain equivalence FD (EFD for short below) across left and right
// cols and no other `LEFT` condition on the (left-side cols except the cols in EFD's from) to filter the left join results. We can maintain the strict
// FD from EFD's `from` side to EFD's `to` side over the left join result.
// a b | c d e
// ------+----------------
// 1 1 | 1 NULL 1
// 1 2 | NULL NULL NULL
// 2 1 | NULL NULL NULL
// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=1), other b=1 will filter the result, causing the left row (1, 2)
// miss matched with right row (1, null 1) by a=c, consequently leading the left row appended as (1,2,null,null,null), which
// will break the FD: {a} -> {c} for key a=1 with different c=1/null.
// a b | c d e
// ------+----------------
// 1 1 | NULL NULL NULL
// 2 1 | NULL NULL NULL
// Pos eg: if the filter is on EFD's `from` cols, it's ok. Let's say: (a,b) * (c,d,e) on (a=c and a=2), a=2 only won't leading
// same key a with matched c and mismatched NULL, neg case result is changed as above, so strict FD {a} -> {c} can exist.
// a b | c d e
// ------+----------------
// 1 1 | 1 NULL 1
// 1 2 | NULL NULL NULL
// 2 1 | NULL NULL NULL
// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=c), two EFD here, where b=c can also refuse some rows joined by a=c,
// consequently applying it with NULL as (1 2 | NULL NULL NULL), leading the same key a has different value 1/NULL. But
// macroscopically, we can combine {a,b} together as the strict FD's from side, so new FD {a,b} -> {c} is secured. For case
// of (a=c and b=ce), the FD is {a, b} -> {c, e}
//
// conclusion: without this kind of limited left conditions to judge the join match, we can say: FD {a} -> {c} exists.
//
// <3.2> equivalence FD: when the determinant and dependencies from an equivalence FD of join condition are each covering a strict
// FD of the left / right side. After joining, we can extend the left side strict FD's dependencies to all cols.
// a b | c d e
// ------+----------------
// 1 1 | 1 NULL 1
// 2 2 | NULL NULL NULL
// 3 1 | NULL NULL NULL
// left join with (a,b) * (c,d,e) on (a=c and b=1). Supposing that left `a` are strict Key and right `c` are strict Key too.
// Key means the strict FD can determine all cols from that table.
// case 1: left join matched
// one left row match one / multi rows from right side, since `c` is strict determine all cols from right table, so
// {a} == {b} --> {all cols in right}, according to the transitive rule of strict FD, we get {a} --> {all cols in right}
// case 2: left join miss match
// miss matched rows from left side are unique originally, even appended with NULL value from right side, they are still
// strictly determine themselves and even the all rows after left join.
// conclusion combined:
// If there is an equivalence covering both strict Key from the right and left, we can create a new strict FD: {columns of the left side of the join in the equivalence} -> {all columns after join}.
//
// <3.3> equivalence FD: let's see equivalence FD as double-directed strict FD from join equal conditions, and we only keep the
// rhs ~~> lhs.
// a b | c d e
// ------+----------------
// 1 1 | 1 NULL 1
// 1 2 | NULL NULL NULL
// 2 1 | NULL NULL NULL
// left join with (a,b) * (c,d,e) on (a=c and b=1). From the join equivalence condition can derive a new FD {ac} == {ac}.
// while since there are some supplied null value in the c column, we don't guarantee {ac} == {ac} yet, so do {a} -> {c}
// because two same determinant key {1} can point to different dependency {1} & {NULL}. But in return, FD like {c} -> {a}
// are degraded to the corresponding lax one.
//
// 4: the new formed FD {left primary key, right primary key} -> {all columns} are preserved in spite of the null-supplied rows.
// 5: There's no join key and no filters from the outer side. The join case is a cartesian product. In this case,
// the strict equivalence classes still exist.
// - If the right side has no row, we will supply null-extended rows, then the value of any column is NULL, and the equivalence class exists.
// - If the right side has rows, no row is filtered out after the filters since no row of the outer side is filtered out. Hence, the equivalence class remains.
//
func (s *FDSet) MakeOuterJoin(innerFDs, filterFDs *FDSet, outerCols, innerCols FastIntSet, opt *ArgOpts) {
// copy down the left PK and right PK before the s has changed for later usage.
leftPK, ok1 := s.FindPrimaryKey()
rightPK, ok2 := innerFDs.FindPrimaryKey()
copyLeftFDSet := &FDSet{}
copyLeftFDSet.AddFrom(s)
copyRightFDSet := &FDSet{}
copyRightFDSet.AddFrom(innerFDs)
for _, edge := range innerFDs.fdEdges {
// Rule #2.2, constant FD are removed from right side of left join.
if edge.isConstant() {
continue
}
// Rule #2.3, equivalence FD are removed from right side of left join.
if edge.equiv {
continue
}
// Rule #2.1, lax FD can be kept after the left join.
if !edge.strict {
s.addFunctionalDependency(edge.from, edge.to, edge.strict, edge.equiv)
continue
}
// Rule #2.1, strict FD can be kept when determinant contains not null column, otherwise, downgraded to the lax one.
//
// If the one of the column from the inner child's functional dependency's left side is not null, this FD can be remained.
// This is because that the outer join would generate null-extended rows. So if at least one row from the left side
// is not null. We can guarantee that the there's no same part between the original rows and the generated rows.
// So the null extended rows would not break the original functional dependency.
if edge.from.Intersects(innerFDs.NotNullCols) {
// One of determinant are not null column, strict FD are kept.
// According knowledge #2, we can't take use of right filter's not null attribute.
s.addFunctionalDependency(edge.from, edge.to, edge.strict, edge.equiv)
} else {
// Otherwise, the strict FD are downgraded to a lax one.
s.addFunctionalDependency(edge.from, edge.to, false, edge.equiv)
}
}
leftCombinedFDFrom := NewFastIntSet()
leftCombinedFDTo := NewFastIntSet()
for _, edge := range filterFDs.fdEdges {
// Rule #3.2, constant FD are removed from right side of left join.
if edge.isConstant() {
s.Rule333Equiv.Edges = append(s.Rule333Equiv.Edges, &fdEdge{
from: edge.from,
to: edge.to,
strict: edge.strict,
equiv: edge.equiv,
})
s.Rule333Equiv.InnerCols = innerCols
continue
}
// Rule #3.3, we only keep the lax FD from right side pointing the left side.
if edge.equiv {
equivColsRight := edge.from.Intersection(innerCols)
equivColsLeft := edge.from.Intersection(outerCols)
// equivalence: {superset} --> {superset}, either `from` or `to` side is ok here.
// Rule 3.3.1
if !opt.SkipFDRule331 {
if equivColsLeft.Len() > 0 && equivColsRight.Len() > 0 {
leftCombinedFDFrom.UnionWith(equivColsLeft)
leftCombinedFDTo.UnionWith(equivColsRight)
}
}
// Rule 3.3.2
rightAllCols := copyRightFDSet.AllCols()
leftAllCols := copyLeftFDSet.AllCols()
coveringStrictKeyRight := rightAllCols.SubsetOf(copyRightFDSet.ClosureOfStrict(equivColsRight))
coveringStrictKeyLeft := leftAllCols.SubsetOf(copyLeftFDSet.closureOfStrict(equivColsLeft))
if coveringStrictKeyLeft && coveringStrictKeyRight {
// find the minimum strict Key set, and add
s.addFunctionalDependency(copyLeftFDSet.ReduceCols(equivColsLeft), rightAllCols.Union(leftAllCols), true, false)
}
// Rule 3.3.3
// need to break down the superset of equivalence, adding each lax FD of them.
laxFDFrom := equivColsRight
laxFDTo := equivColsLeft
for i, ok := laxFDFrom.Next(0); ok; i, ok = laxFDFrom.Next(i + 1) {
for j, ok := laxFDTo.Next(0); ok; j, ok = laxFDTo.Next(j + 1) {
s.addFunctionalDependency(NewFastIntSet(i), NewFastIntSet(j), false, false)
}
}
s.Rule333Equiv.Edges = append(s.Rule333Equiv.Edges, &fdEdge{
from: laxFDFrom,
to: laxFDTo,
strict: true,
equiv: true,
})
s.Rule333Equiv.InnerCols = innerCols
}
// Rule #3.1, filters won't produce any strict/lax FDs.
}
// Rule #3.3.1 combinedFD case
if !opt.SkipFDRule331 {
s.addFunctionalDependency(leftCombinedFDFrom, leftCombinedFDTo, true, false)
}
// Rule #4, add new FD {left key + right key} -> {all columns} if it could.
if ok1 && ok2 {
s.addFunctionalDependency(leftPK.Union(*rightPK), outerCols.Union(innerCols), true, false)
}
// Rule #5, adding the strict equiv edges if there's no join key and no filters from outside.
if opt.OnlyInnerFilter {
if opt.InnerIsFalse {
s.AddConstants(innerCols)
} else {
for _, edge := range filterFDs.fdEdges {
// keep filterFD's constant and equivalence.
if edge.strict && (edge.equiv || edge.from.IsEmpty()) {
s.addFunctionalDependency(edge.from, edge.to, edge.strict, edge.equiv)
}
}
// keep all FDs from inner side.
for _, edge := range innerFDs.fdEdges {
s.addFunctionalDependency(edge.from, edge.to, edge.strict, edge.equiv)
}
}
}
// merge the not-null-cols/registered-map from both side together.
s.NotNullCols.UnionWith(innerFDs.NotNullCols)
s.NotNullCols.UnionWith(filterFDs.NotNullCols)
// inner cols can be nullable since then.
s.NotNullCols.DifferenceWith(innerCols)
if s.HashCodeToUniqueID == nil {
s.HashCodeToUniqueID = innerFDs.HashCodeToUniqueID
} else {
for k, v := range innerFDs.HashCodeToUniqueID {
if _, ok := s.HashCodeToUniqueID[k]; ok {
logutil.BgLogger().Warn("Error occurred when building the functional dependency")
}
s.HashCodeToUniqueID[k] = v
}
}
for i, ok := innerFDs.GroupByCols.Next(0); ok; i, ok = innerFDs.GroupByCols.Next(i + 1) {
s.GroupByCols.Insert(i)
}
s.HasAggBuilt = s.HasAggBuilt || innerFDs.HasAggBuilt
}
// MakeRestoreRule333 reset the status of how we deal with this rule.
func (s *FDSet) MakeRestoreRule333() {
for _, eg := range s.Rule333Equiv.Edges {
@ -565,20 +832,10 @@ func (s *FDSet) MakeRestoreRule333() {
// ArgOpts contains some arg used for FD maintenance.
type ArgOpts struct {
SkipFDRule331 bool
TypeFDRule331 TypeFilterFD331
OnlyInnerFilter bool
InnerIsFalse bool
}
// TypeFilterFD331 describes the type of the filter used in this rule.
type TypeFilterFD331 byte
// Here's the two specific type.
const (
SingleFD TypeFilterFD331 = 0
CombinedFD TypeFilterFD331 = 1
)
// FindPrimaryKey checks whether there's a key in the current set which implies key -> all cols.
func (s FDSet) FindPrimaryKey() (*FastIntSet, bool) {
allCols := s.AllCols()

View File

@ -0,0 +1,249 @@
// 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 funcdep_test
import (
"testing"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)
func TestOnlyFullGroupByOldCases(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@session.tidb_enable_new_only_full_group_by_check = 'on';")
// test case 1
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("drop view if exists v1")
tk.MustExec("CREATE TABLE t1 ( c1 INT, c2 INT, c4 DATE, c5 VARCHAR(1));")
tk.MustExec("CREATE TABLE t2 ( c1 INT, c2 INT, c3 INT, c5 VARCHAR(1));")
tk.MustExec("CREATE VIEW v1 AS SELECT alias1.c4 AS field1 FROM t1 AS alias1 INNER JOIN t1 AS alias2 ON 1 GROUP BY field1 ORDER BY alias1.c5;")
_, err := tk.Exec("SELECT * FROM v1;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #2 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.alias1.c5' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// test case 2
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("drop view if exists v1")
tk.MustExec("CREATE TABLE t1 (c1 INT, c2 INT, c4 DATE, c5 VARCHAR(1));")
tk.MustExec("CREATE TABLE t2 (c1 INT, c2 INT, c3 INT, c5 VARCHAR(1));")
tk.MustExec("CREATE definer='root'@'localhost' VIEW v1 AS SELECT alias1.c4 AS field1, alias1.c4 AS field2 FROM t1 AS alias1 INNER JOIN t1 AS alias2 ON (alias2.c1 = alias1.c2) WHERE ( NOT EXISTS ( SELECT SQ1_alias1.c5 AS SQ1_field1 FROM t2 AS SQ1_alias1 WHERE SQ1_alias1.c3 < alias1.c1 )) AND (alias1.c5 = alias1.c5 AND alias1.c5 = 'd' ) GROUP BY field1, field2 ORDER BY alias1.c5, field1, field2")
tk.MustQuery("SELECT * FROM v1;")
// test case 3
// need to resolve the name resolver problem first (view and base table's column can not refer each other)
// see some cases in 21
// test case 4
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("drop view if exists v2")
tk.MustExec("CREATE TABLE t1 ( col_varchar_10_utf8 VARCHAR(10) CHARACTER SET utf8, col_int_key INT, pk INT PRIMARY KEY);")
tk.MustExec("CREATE TABLE t2 ( col_varchar_10_utf8 VARCHAR(10) CHARACTER SET utf8 DEFAULT NULL, col_int_key INT DEFAULT NULL, pk INT PRIMARY KEY);")
tk.MustExec("CREATE ALGORITHM=MERGE definer='root'@'localhost' VIEW v2 AS SELECT t2.pk, COALESCE(t2.pk, 3) AS coa FROM t1 LEFT JOIN t2 ON 0;")
tk.MustQuery("SELECT v2.pk, v2.coa FROM t1 LEFT JOIN v2 AS v2 ON 0 GROUP BY v2.pk;")
// test case 5
tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE t ( a INT, c INT GENERATED ALWAYS AS (a+2), d INT GENERATED ALWAYS AS (c+2) );")
tk.MustQuery("SELECT c FROM t GROUP BY a;")
tk.MustQuery("SELECT d FROM t GROUP BY c;")
tk.MustQuery("SELECT d FROM t GROUP BY a;")
tk.MustQuery("SELECT 1+c FROM t GROUP BY a;")
tk.MustQuery("SELECT 1+d FROM t GROUP BY c;")
tk.MustQuery("SELECT 1+d FROM t GROUP BY a;")
tk.MustQuery("SELECT t1.d FROM t as t1, t as t2 WHERE t2.d=t1.c GROUP BY t2.a;")
_, err = tk.Exec("SELECT t1.d FROM t as t1, t as t2 WHERE t2.d>t1.c GROUP BY t2.a;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t1.d' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// test case 6
tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE t ( a INT, c INT GENERATED ALWAYS AS (a+2), d INT GENERATED ALWAYS AS (c+2) );")
_, err = tk.Exec("SELECT t1.d FROM t as t1, t as t2 WHERE t2.d>t1.c GROUP BY t2.a;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t1.d' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
_, err = tk.Exec("SELECT (SELECT t1.c FROM t as t1 GROUP BY -3) FROM t as t2;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t1.c' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
_, err = tk.Exec("SELECT DISTINCT t1.a FROM t as t1 ORDER BY t1.d LIMIT 1;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:3065]Expression #1 of ORDER BY clause is not in SELECT list, references column 'test.t.d' which is not in SELECT list; this is incompatible with DISTINCT")
_, err = tk.Exec("SELECT DISTINCT t1.a FROM t as t1 ORDER BY t1.d LIMIT 1;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:3065]Expression #1 of ORDER BY clause is not in SELECT list, references column 'test.t.d' which is not in SELECT list; this is incompatible with DISTINCT")
_, err = tk.Exec("SELECT (SELECT DISTINCT t1.a FROM t as t1 ORDER BY t1.d LIMIT 1) FROM t as t2;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:3065]Expression #1 of ORDER BY clause is not in SELECT list, references column 'test.t.d' which is not in SELECT list; this is incompatible with DISTINCT")
// test case 7
tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE t(a INT NULL, b INT NOT NULL, c INT, UNIQUE(a,b));")
tk.MustQuery("SELECT a,b,c FROM t WHERE a IS NOT NULL GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE NOT (a IS NULL) GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a > 3 GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a = 3 GROUP BY b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a BETWEEN 3 AND 6 GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a <> 3 GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a IN (3,4) GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a IN (SELECT b FROM t) GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a IS TRUE GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE (a <> 3) IS TRUE GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a IS FALSE GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE (a <> 3) IS FALSE GROUP BY a,b;")
tk.MustQuery("SELECT a,b,c FROM t WHERE a LIKE \"%abc%\" GROUP BY a,b;")
// todo: eval not-null refuse NOT wrapper.
// tk.MustQuery("SELECT a,b,c FROM t WHERE NOT(a IN (3,4)) GROUP BY a,b;")
// tk.MustQuery("SELECT a,b,c FROM t WHERE a NOT IN (3,4) GROUP BY a,b;")
// tk.MustQuery("SELECT a,b,c FROM t WHERE a NOT LIKE \"%abc%\" GROUP BY a,b;")
_, err = tk.Exec("SELECT a,b,c FROM t WHERE a<=>NULL GROUP BY b;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #3 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.c' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// is-not-true will let the null value pass, so evaluating it won't derive to a with not-null attribute.
_, err = tk.Exec("SELECT a,b,c FROM t WHERE a IS NOT TRUE GROUP BY a,b;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #3 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.c' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// test case 8
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("drop table if exists t3")
tk.MustExec("CREATE TABLE t1 (a INT, b INT);")
tk.MustExec("CREATE TABLE t2 (b INT);")
tk.MustExec("CREATE TABLE t3 (b INT NULL, c INT NULL, d INT NULL, e INT NULL, UNIQUE KEY (b,d,e));")
tk.MustQuery("SELECT * FROM t1, t2, t3 WHERE t2.b = t1.b AND t2.b = t3.b AND t3.d = 1 AND t3.e = 1 AND t3.d IS NOT NULL AND t1.a = 2 GROUP BY t1.b;")
// test case 9
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1(a int, b int not null, c int not null, d int, unique key(b,c), unique key(b,d));")
_, err = tk.Exec("select (select sin(a)) as z from t1 group by d,b;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'z' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// test case 10 & 11
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1(a int, b int not null, c int not null, d int, unique key(b,c), unique key(b,d));")
tk.MustExec("select t3.a from t1, t1 as t2, t1 as t3 where t3.b=t2.b and t3.c=t1.d and t2.b=t1.b and t2.c=t1.c group by t1.b,t1.c")
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t3")
tk.MustExec("create table t1(a int, b int not null, c int not null, d int, unique key(b,c), unique key(b,d));")
tk.MustExec("create table t3(pk int primary key, b int);")
tk.MustQuery("select t3.b from t1,t1 as t2,t3 where t3.pk=t2.d and t2.b=t1.b and t2.c=t1.a group by t1.b,t1.c;")
// test case 12
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t1(a int,b int not null,c int not null,d int, unique key(b,c), unique key(b,d));")
tk.MustExec("create table t2 like t1")
tk.MustQuery("select t1.a,t2.c from t1 left join t2 on t1.a=t2.c and cos(t2.c+t2.b)>0.5 and sin(t1.a+t2.d)<0.9 group by t1.a;")
tk.MustQuery("select t1.a,t2.d from t1 left join t2 on t1.a=t2.c and t1.d=t2.b and cos(t2.c+t2.b)>0.5 and sin(t1.a+t2.d)<0.9 group by t1.a,t1.d;")
// test case 17
tk.MustExec("drop table if exists customer1")
tk.MustExec("drop table if exists customer2")
tk.MustExec("drop view if exists customer")
tk.MustExec("create table customer1(pk int primary key, a int);")
tk.MustExec("create table customer2(pk int primary key, b int);")
tk.MustExec("CREATE algorithm=merge definer='root'@'localhost' VIEW customer as SELECT pk,a,b FROM customer1 JOIN customer2 USING (pk);")
tk.MustQuery("select customer.pk, customer.b from customer group by customer.pk;")
// classic cases
tk.MustQuery("select customer1.a, count(*) from customer1 left join customer2 on customer1.a=customer2.b where customer2.pk in (7,9) group by customer2.b;")
tk.MustQuery("select customer1.a, count(*) from customer1 left join customer2 on customer1.a=1 where customer2.pk in (7,9) group by customer2.b;")
tk.MustExec("drop view if exists customer")
// this left join can extend left pk to all cols.
tk.MustExec("CREATE algorithm=merge definer='root'@'localhost' VIEW customer as SELECT pk,a,b FROM customer1 LEFT JOIN customer2 USING (pk);")
tk.MustQuery("select customer.pk, customer.b from customer group by customer.pk;")
// test case 18
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t1(pk int primary key, a int);")
tk.MustExec("create table t2(pk int primary key, b int);")
tk.MustQuery("select t1.pk, t2.b from t1 join t2 on t1.pk=t2.pk group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t1 join t2 using(pk) group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t1 natural join t2 group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t1 left join t2 using(pk) group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t1 natural left join t2 group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t2 right join t1 using(pk) group by t1.pk;")
tk.MustQuery("select t1.pk, t2.b from t2 natural right join t1 group by t1.pk;")
// test case 20
tk.MustExec("drop table t1")
tk.MustExec("create table t1(pk int primary key, a int);")
tk.MustQuery("select t3.a from t1 left join (t1 as t2 left join t1 as t3 on 1) on 1 group by t3.pk;")
tk.MustQuery("select (select t1.a from t1 as t2 limit 1) from t1 group by pk;")
// test case 21
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1(a int, b int);")
// TODO: to be fixed.
//tk.MustExec("drop view if exists v1;")
//tk.MustExec("create view v1 as select a as a, 2*a as b, coalesce(a,3) as c from t1;")
//err = tk.ExecToErr("select v1.b from t1 left join v1 on 1 group by v1.a")
//require.NotNil(t, err)
//require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'z' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
tk.MustExec("create table t2(c int, d int);")
err = tk.ExecToErr("select t4.d from t1 left join (t2 as t3 join t2 as t4 on t4.d=3) on t1.a=10 group by \"\";")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t4.d' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
tk.MustExec("select t4.d from t1 join (t2 as t3 left join t2 as t4 on t4.d=3) on t1.a=10 group by \"\";")
//tk.MustExec("drop table t1")
//tk.MustExec("drop view v1")
//tk.MustExec("create table t1(a int not null, b int)")
//tk.MustExec("create view v1 as select a as a, 2*a as b, coalesce(a,3) as c from t1")
//tk.MustExec("select v1.b from t1 left join v1 on 1 group by v1.a")
// test issue #25196
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (i1 integer, c1 integer);")
tk.MustExec("insert into t1 values (2, 41), (1, 42), (3, 43), (0, null);")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t2 (i2 integer, c2 integer, f2 float);")
tk.MustExec("insert into t2 values (0, 43, null), (1, null, 0.1), (3, 42, 0.01), (2, 73, 0.12), (null, 41, -0.1), (null, null, null);")
err = tk.ExecToErr("SELECT * FROM t2 AS _tmp_1 JOIN (SELECT max(_tmp_3.f2) AS _tmp_4,min(_tmp_3.i2) AS _tmp_5 FROM t2 AS _tmp_3 WHERE _tmp_3.f2>=_tmp_3.c2 GROUP BY _tmp_3.c2 ORDER BY _tmp_3.i2) AS _tmp_2 WHERE _tmp_2._tmp_5=100;")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #3 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test._tmp_3.i2' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
// test issue #22301 and #33056
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (a int);")
tk.MustExec("create table t2 (a int, b int);")
tk.MustQuery("select t1.a from t1 join t2 on t2.a=t1.a group by t2.a having min(t2.b) > 0;")
tk.MustQuery("select t2.a, count(t2.b) from t1 join t2 using (a) where t1.a = 1;")
tk.MustQuery("select count(t2.b) from t1 join t2 using (a) order by t2.a;")
// test issue #30024
tk.MustExec("drop table if exists t1,t2;")
tk.MustExec("CREATE TABLE t1 (a INT, b INT, c INT DEFAULT 0);")
tk.MustExec("INSERT INTO t1 (a, b) VALUES (3,3), (2,2), (3,3), (2,2), (3,3), (4,4);")
tk.MustExec("CREATE TABLE t2 (a INT, b INT, c INT DEFAULT 0);")
tk.MustExec("INSERT INTO t2 (a, b) VALUES (3,3), (2,2), (3,3), (2,2), (3,3), (4,4);")
tk.MustQuery("SELECT t1.a FROM t1 GROUP BY t1.a HAVING t1.a IN (SELECT t2.a FROM t2 ORDER BY SUM(t1.b));")
// a normal case
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1(a int not null, b int not null, index(a))")
err = tk.ExecToErr("select b from t1 group by a")
require.NotNil(t, err)
require.Equal(t, err.Error(), "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t1.b' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by")
}