Skip to content

Commit

Permalink
planner: using the funcdep to check the only_full_group_by (#33567)
Browse files Browse the repository at this point in the history
ref #29766
  • Loading branch information
winoros authored Apr 20, 2022
1 parent ebffff0 commit 571d97b
Show file tree
Hide file tree
Showing 6 changed files with 840 additions and 63 deletions.
19 changes: 12 additions & 7 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))")
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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")
Expand All @@ -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")
Expand Down
146 changes: 145 additions & 1 deletion planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -2022,6 +2127,7 @@ func (a *havingWindowAndOrderbyExprResolver) resolveFromPlan(v *ast.ColumnNameEx
}
}
if err != nil || idx < 0 {
// nowhere to be found.
return -1, err
}
}
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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{}
Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit 571d97b

Please sign in to comment.