diff --git a/pkg/sql/opt/optbuilder/groupby.go b/pkg/sql/opt/optbuilder/groupby.go index 6d79c1bfb71c..c1573c85a650 100644 --- a/pkg/sql/opt/optbuilder/groupby.go +++ b/pkg/sql/opt/optbuilder/groupby.go @@ -304,14 +304,14 @@ func (b *Builder) constructGroupBy( // buildGroupingColumns builds the grouping columns and adds them to the // groupby scopes that will be used to build the aggregation expression. // Returns the slice of grouping columns. -func (b *Builder) buildGroupingColumns(sel *tree.SelectClause, fromScope *scope) { +func (b *Builder) buildGroupingColumns(sel *tree.SelectClause, projectionsScope, fromScope *scope) { if fromScope.groupby == nil { fromScope.initGrouping() } g := fromScope.groupby // The "from" columns are visible to any grouping expressions. - b.buildGroupingList(sel.GroupBy, sel.Exprs, fromScope) + b.buildGroupingList(sel.GroupBy, sel.Exprs, projectionsScope, fromScope) // Copy the grouping columns to the aggOutScope. g.aggOutScope.appendColumns(g.groupingCols()) @@ -458,7 +458,7 @@ func (b *Builder) buildHaving(having tree.TypedExpr, fromScope *scope) opt.Scala // indicates that the grouping is on the second select expression, k. // fromScope The scope for the input to the aggregation (the FROM clause). func (b *Builder) buildGroupingList( - groupBy tree.GroupBy, selects tree.SelectExprs, fromScope *scope, + groupBy tree.GroupBy, selects tree.SelectExprs, projectionsScope *scope, fromScope *scope, ) { g := fromScope.groupby g.groupStrs = make(groupByStrSet, len(groupBy)) @@ -476,7 +476,7 @@ func (b *Builder) buildGroupingList( // a grouping error until the grouping columns are fully built. g.buildingGroupingCols = true for _, e := range groupBy { - b.buildGrouping(e, selects, fromScope, g.aggInScope) + b.buildGrouping(e, selects, projectionsScope, fromScope, g.aggInScope) } g.buildingGroupingCols = false } @@ -486,26 +486,97 @@ func (b *Builder) buildGroupingList( // groupStrs and to the aggInScope. // // -// groupBy The given GROUP BY expression. -// selects The select expressions are needed in case the GROUP BY expression -// is an index into to the select list. -// fromScope The scope for the input to the aggregation (the FROM clause). -// aggInScope The scope that will contain the grouping expressions as well as -// the aggregate function arguments. +// groupBy The given GROUP BY expression. +// selects The select expressions are needed in case the GROUP BY +// expression is an index into to the select list. +// projectionsScope The scope that contains the columns for the SELECT targets +// (used when GROUP BY refers to a target by alias). +// fromScope The scope for the input to the aggregation (the FROM +// clause). +// aggInScope The scope that will contain the grouping expressions as well +// as the aggregate function arguments. func (b *Builder) buildGrouping( - groupBy tree.Expr, selects tree.SelectExprs, fromScope, aggInScope *scope, + groupBy tree.Expr, selects tree.SelectExprs, projectionsScope, fromScope, aggInScope *scope, ) { // Unwrap parenthesized expressions like "((a))" to "a". groupBy = tree.StripParens(groupBy) - - // Check whether the GROUP BY clause refers to a column in the SELECT list - // by index, e.g. `SELECT a, SUM(b) FROM y GROUP BY 1`. - col := colIndex(len(selects), groupBy, "GROUP BY") alias := "" - if col != -1 { - groupBy = selects[col].Expr - alias = string(selects[col].As) - } + + // Comment below pasted from PostgreSQL (findTargetListEntrySQL92 in + // src/backend/parser/parse_clause.c). + // + // Handle two special cases as mandated by the SQL92 spec: + // + // 1. Bare ColumnName (no qualifier or subscripts) + // For a bare identifier, we search for a matching column name + // in the existing target list. Multiple matches are an error + // unless they refer to identical values; for example, + // we allow SELECT a, a FROM table ORDER BY a + // but not SELECT a AS b, b FROM table ORDER BY b + // If no match is found, we fall through and treat the identifier + // as an expression. + // For GROUP BY, it is incorrect to match the grouping item against + // targetlist entries: according to SQL92, an identifier in GROUP BY + // is a reference to a column name exposed by FROM, not to a target + // list column. However, many implementations (including pre-7.0 + // PostgreSQL) accept this anyway. So for GROUP BY, we look first + // to see if the identifier matches any FROM column name, and only + // try for a targetlist name if it doesn't. This ensures that we + // adhere to the spec in the case where the name could be both. + // DISTINCT ON isn't in the standard, so we can do what we like there; + // we choose to make it work like ORDER BY, on the rather flimsy + // grounds that ordinary DISTINCT works on targetlist entries. + // + // 2. IntegerConstant + // This means to use the n'th item in the existing target list. + // Note that it would make no sense to order/group/distinct by an + // actual constant, so this does not create a conflict with SQL99. + // GROUP BY column-number is not allowed by SQL92, but since + // the standard has no other behavior defined for this syntax, + // we may as well accept this common extension. + + // This function sets groupBy and alias in these special cases. + func() { + // Check whether the GROUP BY clause refers to a column in the SELECT list + // by index, e.g. `SELECT a, SUM(b) FROM y GROUP BY 1` (case 2 above). + if col := colIndex(len(selects), groupBy, "GROUP BY"); col != -1 { + groupBy, alias = selects[col].Expr, string(selects[col].As) + return + } + + if name, ok := groupBy.(*tree.UnresolvedName); ok { + if name.NumParts != 1 || name.Star { + return + } + // Case 1 above. + targetName := name.Parts[0] + + // We must prefer a match against a FROM-clause column (but ignore upper + // scopes); in this case we let the general case below handle the reference. + for i := range fromScope.cols { + if string(fromScope.cols[i].name) == targetName { + return + } + } + // See if it matches exactly one of the target lists. + var match *scopeColumn + for i := range projectionsScope.cols { + if col := &projectionsScope.cols[i]; string(col.name) == targetName { + if match != nil { + // Multiple matches are only allowed if they refer to identical + // expressions. + if match.getExprStr() != col.getExprStr() { + panic(pgerror.Newf(pgcode.AmbiguousColumn, "GROUP BY %q is ambiguous", targetName)) + } + } + match = col + } + } + if match != nil { + groupBy, alias = match.expr, targetName + } + } + }() // We need to save and restore the previous value of the field in semaCtx // in case we are recursively called within a subquery context. diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 34177686cc7f..d38ec5c2283a 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -828,7 +828,7 @@ func (b *Builder) buildSelectClause( // Grouping columns must be built before building the projection list so // we can check that any column references that appear in the SELECT list // outside of aggregate functions are present in the grouping list. - b.buildGroupingColumns(sel, fromScope) + b.buildGroupingColumns(sel, projectionsScope, fromScope) having = b.buildHaving(havingExpr, fromScope) } diff --git a/pkg/sql/opt/optbuilder/testdata/aggregate b/pkg/sql/opt/optbuilder/testdata/aggregate index 1bd8ad7499c6..6ca4ff06a372 100644 --- a/pkg/sql/opt/optbuilder/testdata/aggregate +++ b/pkg/sql/opt/optbuilder/testdata/aggregate @@ -3727,3 +3727,144 @@ project └── eq [type=bool] ├── variable: k [type=int] └── variable: a [type=int] + +# Tests with aliases (see #28059). +build +SELECT x + 1 AS z FROM abxy GROUP BY z +---- +group-by + ├── columns: z:5(int) + ├── grouping columns: z:5(int) + └── project + ├── columns: z:5(int) + ├── scan abxy + │ └── columns: a:1(int!null) b:2(int!null) x:3(int) y:4(int) + └── projections + └── plus [type=int] + ├── variable: x [type=int] + └── const: 1 [type=int] + +# The FROM column has precedence, we should be grouping by abxy.x, not by x%10. +build +SELECT (x % 10) AS x FROM abxy GROUP BY x +---- +project + ├── columns: x:5(int) + ├── group-by + │ ├── columns: abxy.x:3(int) + │ ├── grouping columns: abxy.x:3(int) + │ └── project + │ ├── columns: abxy.x:3(int) + │ └── scan abxy + │ └── columns: a:1(int!null) b:2(int!null) abxy.x:3(int) y:4(int) + └── projections + └── mod [type=int] + ├── variable: abxy.x [type=int] + └── const: 10 [type=int] + +# But aliases have precedence over columns from higher scopes. Here we are +# grouping by v, not by the outer x. +build +SELECT x, (SELECT v AS x FROM kv GROUP BY x) FROM abxy +---- +project + ├── columns: x:3(int) x:9(int) + ├── scan abxy + │ └── columns: a:1(int!null) b:2(int!null) abxy.x:3(int) y:4(int) + └── projections + └── subquery [type=int] + └── max1-row + ├── columns: v:6(int) + └── group-by + ├── columns: v:6(int) + ├── grouping columns: v:6(int) + └── project + ├── columns: v:6(int) + └── scan kv + └── columns: k:5(int!null) v:6(int) w:7(int) s:8(string) + +build +SELECT sum(x) AS u FROM abxy GROUP BY u +---- +error (42803): sum(): aggregate functions are not allowed in GROUP BY + +# Implicit aliases should work too. +build +SELECT x + 1 FROM abxy GROUP BY "?column?" +---- +group-by + ├── columns: "?column?":5(int) + ├── grouping columns: "?column?":5(int) + └── project + ├── columns: "?column?":5(int) + ├── scan abxy + │ └── columns: a:1(int!null) b:2(int!null) x:3(int) y:4(int) + └── projections + └── plus [type=int] + ├── variable: x [type=int] + └── const: 1 [type=int] + +build +SELECT sum(x) FROM abxy GROUP BY sum +---- +error (42803): sum(): aggregate functions are not allowed in GROUP BY + +# Ambiguous aliases should error out. +build +SELECT (x + 1) AS u, (y + 1) AS u FROM abxy GROUP BY u +---- +error (42702): GROUP BY "u" is ambiguous + +# In this case we would have had an outer column if it wasn't for the aliases; +# this should error out just the same. +build +SELECT x, (SELECT v AS x, w AS x FROM kv GROUP BY x) FROM abxy +---- +error (42702): GROUP BY "x" is ambiguous + +# Duplicate expressions with the same alias are not ambiguous. +build +SELECT (x + 1) AS u, (x + 1) AS u FROM abxy GROUP BY u +---- +group-by + ├── columns: u:5(int) u:5(int) + ├── grouping columns: u:5(int) + └── project + ├── columns: u:5(int) + ├── scan abxy + │ └── columns: a:1(int!null) b:2(int!null) x:3(int) y:4(int) + └── projections + └── plus [type=int] + ├── variable: x [type=int] + └── const: 1 [type=int] + +build +SELECT (x + 1) AS u, (x + 1) AS u, (y + 1) AS u FROM abxy GROUP BY u +---- +error (42702): GROUP BY "u" is ambiguous + +# In this case, the FROM column has precedence. +build +SELECT sum(x + 1) AS x, sum(y + 1) AS x FROM abxy GROUP BY x +---- +project + ├── columns: x:6(decimal) x:8(decimal) + └── group-by + ├── columns: x:3(int) sum:6(decimal) sum:8(decimal) + ├── grouping columns: x:3(int) + ├── project + │ ├── columns: column5:5(int) column7:7(int) x:3(int) + │ ├── scan abxy + │ │ └── columns: a:1(int!null) b:2(int!null) x:3(int) y:4(int) + │ └── projections + │ ├── plus [type=int] + │ │ ├── variable: x [type=int] + │ │ └── const: 1 [type=int] + │ └── plus [type=int] + │ ├── variable: y [type=int] + │ └── const: 1 [type=int] + └── aggregations + ├── sum [type=decimal] + │ └── variable: column5 [type=int] + └── sum [type=decimal] + └── variable: column7 [type=int]