Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
55 commits
Select commit Hold shift + click to select a range
ade64fc
planner: refactor the interface of the LogicalPlan
winoros Dec 7, 2021
e21c106
remove Children method
winoros Dec 7, 2021
413822e
add basic fdSet and test (#1)
AilinKid Dec 14, 2021
327491a
add basic adding constant and equivalence logic (#2)
AilinKid Dec 20, 2021
21dd9b3
extractFD for group/datasource/selection/projection and ported some t…
AilinKid Jan 5, 2022
8a55d15
introduce expr/constant into fdSet with uniqueID and refine some op's…
AilinKid Feb 9, 2022
0d603f9
Revert "remove Children method"
winoros Feb 16, 2022
aa0b72c
Revert "planner: refactor the interface of the LogicalPlan"
winoros Feb 16, 2022
716584a
Merge branch 'master' into functional-dependency
winoros Feb 16, 2022
d4ef9af
planner: maintain functional dependency for joins (#5)
winoros Mar 1, 2022
6d9b0ef
refactor join logic (#8)
AilinKid Mar 7, 2022
7726e19
Merge branch 'master' into functional-dependency
winoros Mar 7, 2022
d297b73
Merge branch 'master' into functional-dependency
winoros Mar 13, 2022
2a8e86f
planner: open the new only_full_group_by check (#7)
winoros Mar 14, 2022
540d854
rename to funcdep to make ci happy
winoros Mar 14, 2022
438aa74
Merge branch 'master' into functional-dependency
winoros Mar 14, 2022
1527279
fix issue 25196 using the alias name for error msg (#9)
AilinKid Mar 14, 2022
f78b56f
use require instead of assert
winoros Mar 14, 2022
2ef6d7b
Fix 22301 and 33056 (#10)
AilinKid Mar 14, 2022
f2d0b3d
Merge branch 'master' into functional-dependency
winoros Mar 15, 2022
ed4e051
planner: add the functional dependency for Datasource, proj, select, agg
winoros Mar 15, 2022
748ba62
fix lint
winoros Mar 15, 2022
6b2043f
undo some changes
winoros Mar 15, 2022
ba60938
fix tests
winoros Mar 15, 2022
c06fc01
to https://github.com/winoros/tidb/pull/12
winoros Mar 15, 2022
39f3c78
add license header and fix check and fix import
winoros Mar 15, 2022
89b7b99
reslove the correlated agg func's correlated col when it in the sub-q…
AilinKid Mar 15, 2022
b357c0e
address comments
winoros Mar 15, 2022
d554d9b
address comments
winoros Mar 16, 2022
038d875
planner: fix that normal index cannot build fd
winoros Mar 16, 2022
548ff66
Merge pull request #13 from winoros/fix-normal-case
AilinKid Mar 16, 2022
0e80bab
remove the laxUni = false case
winoros Mar 16, 2022
23f8abc
only use public index to build
winoros Mar 16, 2022
b2216e1
use isNullRejective instead
winoros Mar 16, 2022
d63eac9
Merge branch 'master' into fd-not-outer-join-and-open
winoros Mar 16, 2022
34151b6
Merge branch 'master' into functional-dependency
winoros Mar 16, 2022
8fd7cc4
Merge remote-tracking branch 'yiding/functional-dependency' into func…
winoros Mar 16, 2022
2bf3aaa
Merge branch 'fd-not-outer-join-and-open' into xxx
winoros Mar 16, 2022
e7a1d5f
Merge branch 'master' into fd-last
winoros Mar 29, 2022
98e9a68
remove doc.go for now
winoros Mar 29, 2022
ee035db
remove the usage of `panic`
winoros Mar 29, 2022
ace0e63
add license
winoros Mar 29, 2022
3ef220e
Merge branch 'master' into fd-last
winoros Apr 6, 2022
a9c74da
fix a bug about correlated column
winoros Apr 6, 2022
db30207
address comments
winoros Apr 6, 2022
01f4406
address comments
winoros Apr 18, 2022
b6e5ce5
Update planner/funcdep/fd_graph.go
winoros Apr 18, 2022
3cf1de4
address comments
winoros Apr 18, 2022
d24726f
address comments
winoros Apr 19, 2022
a4d341d
Merge branch 'master' into fd-last
winoros Apr 19, 2022
e95341f
Merge remote-tracking branch 'yiding/fd-last' into fd-last
winoros Apr 19, 2022
ddd9bdf
revert some small changes
winoros Apr 20, 2022
9b2878f
Merge branch 'master' into fd-last
AilinKid Apr 20, 2022
29795b0
format import
winoros Apr 20, 2022
02fd5c0
Merge branch 'fd-last' of https://github.com/winoros/tidb into fd-last
winoros Apr 20, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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