Skip to content

Commit

Permalink
planner: report error for invalid window specs which are not used (pi…
Browse files Browse the repository at this point in the history
  • Loading branch information
dyzsr authored Dec 23, 2020
1 parent 61e068a commit f2a88ff
Show file tree
Hide file tree
Showing 3 changed files with 109 additions and 36 deletions.
15 changes: 15 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2049,6 +2049,21 @@ func (s *testIntegrationSuite) TestOrderByNotInSelectDistinct(c *C) {
tk.MustQuery("select distinct v1 as z from ttest order by v1+z").Check(testkit.Rows("1", "4"))
}

func (s *testIntegrationSuite) TestInvalidNamedWindowSpec(c *C) {
// #12356
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("DROP TABLE IF EXISTS temptest")
tk.MustExec("create table temptest (val int, val1 int)")
tk.MustQuery("SELECT val FROM temptest WINDOW w AS (ORDER BY val RANGE 1 PRECEDING)").Check(testkit.Rows())
tk.MustGetErrMsg("SELECT val FROM temptest WINDOW w AS (ORDER BY val, val1 RANGE 1 PRECEDING)",
"[planner:3587]Window 'w' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type")
tk.MustGetErrMsg("select val1, avg(val1) as a from temptest group by val1 window w as (order by a)",
"[planner:1054]Unknown column 'a' in 'window order by'")
tk.MustGetErrMsg("select val1, avg(val1) as a from temptest group by val1 window w as (partition by a)",
"[planner:1054]Unknown column 'a' in 'window partition by'")
}

func (s *testIntegrationSuite) TestCorrelatedAggregate(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
126 changes: 90 additions & 36 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1870,6 +1870,17 @@ type havingWindowAndOrderbyExprResolver struct {
outerSchemas []*expression.Schema
outerNames [][]*types.FieldName
curClause clauseCode
prevClause []clauseCode
}

func (a *havingWindowAndOrderbyExprResolver) pushCurClause(newClause clauseCode) {
a.prevClause = append(a.prevClause, a.curClause)
a.curClause = newClause
}

func (a *havingWindowAndOrderbyExprResolver) popCurClause() {
a.curClause = a.prevClause[len(a.prevClause)-1]
a.prevClause = a.prevClause[:len(a.prevClause)-1]
}

// Enter implements Visitor interface.
Expand All @@ -1886,6 +1897,12 @@ func (a *havingWindowAndOrderbyExprResolver) Enter(n ast.Node) (node ast.Node, s
// Enter a new context, skip it.
// For example: select sum(c) + c + exists(select c from t) from t;
return n, true
case *ast.PartitionByClause:
a.pushCurClause(partitionByClause)
case *ast.OrderByClause:
if a.inWindowSpec {
a.pushCurClause(windowOrderByClause)
}
default:
a.inExpr = true
}
Expand Down Expand Up @@ -1966,6 +1983,12 @@ func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, o
}
case *ast.WindowSpec:
a.inWindowSpec = false
case *ast.PartitionByClause:
a.popCurClause()
case *ast.OrderByClause:
if a.inWindowSpec {
a.popCurClause()
}
case *ast.ColumnNameExpr:
resolveFieldsFirst := true
if a.inAggFunc || a.inWindowFunc || a.inWindowSpec || (a.curClause == orderByClause && a.inExpr) || a.curClause == fieldList {
Expand Down Expand Up @@ -2018,7 +2041,8 @@ func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, o
var err error
index, err = a.resolveFromPlan(v, a.p)
_ = err
if index == -1 && a.curClause != fieldList {
if index == -1 && a.curClause != fieldList &&
a.curClause != windowOrderByClause && a.curClause != partitionByClause {
index, a.err = resolveFromSelectFields(v, a.selectFields, false)
if index != -1 && a.curClause == havingClause && ast.HasWindowFlag(a.selectFields[index].Expr) {
a.err = ErrWindowInvalidWindowFuncAliasUse.GenWithStackByArgs(v.Name.Name.O)
Expand Down Expand Up @@ -3392,7 +3416,7 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L
}

var windowMapper map[*ast.WindowFuncExpr]int
if hasWindowFuncField {
if hasWindowFuncField || sel.WindowSpecs != nil {
windowFuncs := extractWindowFuncs(sel.Fields.Fields)
// we need to check the func args first before we check the window spec
err := b.checkWindowFuncArgs(ctx, p, windowFuncs, windowAggMap)
Expand All @@ -3407,10 +3431,14 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L
if err != nil {
return nil, err
}
// Now we build the window function fields.
p, projExprs, oldLen, err = b.buildProjection(ctx, p, sel.Fields.Fields, windowAggMap, windowMapper, true, false)
if err != nil {
return nil, err
// `hasWindowFuncField == false` means there's only unused named window specs without window functions.
// In such case plan `p` is not changed, so we don't have to build another projection.
if hasWindowFuncField {
// Now we build the window function fields.
p, projExprs, oldLen, err = b.buildProjection(ctx, p, sel.Fields.Fields, windowAggMap, windowMapper, true, false)
if err != nil {
return nil, err
}
}
}

Expand Down Expand Up @@ -5087,7 +5115,16 @@ func (b *PlanBuilder) buildWindowFunctions(ctx context.Context, p LogicalPlan, g
if err != nil {
return nil, nil, err
}
err = b.checkOriginWindowSpecs(funcs, orderBy)
if len(funcs) == 0 {
// len(funcs) == 0 indicates this an unused named window spec,
// so we just check for its validity and don't have to build plan for it.
err := b.checkOriginWindowSpec(spec, orderBy)
if err != nil {
return nil, nil, err
}
continue
}
err = b.checkOriginWindowFuncs(funcs, orderBy)
if err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -5132,9 +5169,9 @@ func (b *PlanBuilder) buildWindowFunctions(ctx context.Context, p LogicalPlan, g
return p, windowMap, nil
}

// checkOriginWindowSpecs checks the validation for origin window specifications for a group of functions.
// Because of the grouped specification is different from it, we should especially check them before build window frame.
func (b *PlanBuilder) checkOriginWindowSpecs(funcs []*ast.WindowFuncExpr, orderByItems []property.SortItem) error {
// checkOriginWindowFuncs checks the validity for original window specifications for a group of functions.
// Because the grouped specification is different from them, we should especially check them before build window frame.
func (b *PlanBuilder) checkOriginWindowFuncs(funcs []*ast.WindowFuncExpr, orderByItems []property.SortItem) error {
for _, f := range funcs {
if f.IgnoreNull {
return ErrNotSupportedYet.GenWithStackByArgs("IGNORE NULLS")
Expand All @@ -5149,38 +5186,46 @@ func (b *PlanBuilder) checkOriginWindowSpecs(funcs []*ast.WindowFuncExpr, orderB
if f.Spec.Name.L != "" {
spec = b.windowSpecs[f.Spec.Name.L]
}
if spec.Frame == nil {
continue
}
if spec.Frame.Type == ast.Groups {
return ErrNotSupportedYet.GenWithStackByArgs("GROUPS")
}
start, end := spec.Frame.Extent.Start, spec.Frame.Extent.End
if start.Type == ast.Following && start.UnBounded {
return ErrWindowFrameStartIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if end.Type == ast.Preceding && end.UnBounded {
return ErrWindowFrameEndIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if start.Type == ast.Following && (end.Type == ast.Preceding || end.Type == ast.CurrentRow) {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if (start.Type == ast.Following || start.Type == ast.CurrentRow) && end.Type == ast.Preceding {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}

err := b.checkOriginWindowFrameBound(&start, spec, orderByItems)
if err != nil {
return err
}
err = b.checkOriginWindowFrameBound(&end, spec, orderByItems)
if err != nil {
if err := b.checkOriginWindowSpec(spec, orderByItems); err != nil {
return err
}
}
return nil
}

// checkOriginWindowSpec checks the validity for given window specification.
func (b *PlanBuilder) checkOriginWindowSpec(spec *ast.WindowSpec, orderByItems []property.SortItem) error {
if spec.Frame == nil {
return nil
}
if spec.Frame.Type == ast.Groups {
return ErrNotSupportedYet.GenWithStackByArgs("GROUPS")
}
start, end := spec.Frame.Extent.Start, spec.Frame.Extent.End
if start.Type == ast.Following && start.UnBounded {
return ErrWindowFrameStartIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if end.Type == ast.Preceding && end.UnBounded {
return ErrWindowFrameEndIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if start.Type == ast.Following && (end.Type == ast.Preceding || end.Type == ast.CurrentRow) {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if (start.Type == ast.Following || start.Type == ast.CurrentRow) && end.Type == ast.Preceding {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}

err := b.checkOriginWindowFrameBound(&start, spec, orderByItems)
if err != nil {
return err
}
err = b.checkOriginWindowFrameBound(&end, spec, orderByItems)
if err != nil {
return err
}
return nil
}

func (b *PlanBuilder) checkOriginWindowFrameBound(bound *ast.FrameBound, spec *ast.WindowSpec, orderByItems []property.SortItem) error {
if bound.Type == ast.CurrentRow || bound.UnBounded {
return nil
Expand Down Expand Up @@ -5300,6 +5345,15 @@ func (b *PlanBuilder) groupWindowFuncs(windowFuncs []*ast.WindowFuncExpr) (map[*
groupedWindow[updatedSpec] = append(groupedWindow[updatedSpec], windowFunc)
}
}
// Unused window specs should also be checked in b.buildWindowFunctions,
// so we add them to `groupedWindow` with empty window functions.
for _, spec := range b.windowSpecs {
if _, ok := groupedWindow[spec]; !ok {
if _, ok = updatedSpecMap[spec.Name.L]; !ok {
groupedWindow[spec] = nil
}
}
}
return groupedWindow, nil
}

Expand Down
4 changes: 4 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,6 +373,8 @@ const (
showStatement
globalOrderByClause
expressionClause
windowOrderByClause
partitionByClause
)

var clauseMsg = map[clauseCode]string{
Expand All @@ -386,6 +388,8 @@ var clauseMsg = map[clauseCode]string{
showStatement: "show statement",
globalOrderByClause: "global ORDER clause",
expressionClause: "expression",
windowOrderByClause: "window order by",
partitionByClause: "window partition by",
}

type capFlagType = uint64
Expand Down

0 comments on commit f2a88ff

Please sign in to comment.