Skip to content

Commit

Permalink
planner: fix compatibility bug for window function (pingcap#11488) (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
lzmhhh123 authored and zz-jason committed Jul 31, 2019
1 parent 39b4cf0 commit 16a81b6
Show file tree
Hide file tree
Showing 2 changed files with 104 additions and 38 deletions.
114 changes: 76 additions & 38 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2986,14 +2986,7 @@ func (b *PlanBuilder) buildWindowFunctionFrameBound(spec *ast.WindowSpec, orderB
if bound.Type == ast.CurrentRow {
return bound, nil
}
// Rows type does not support interval range.
if boundClause.Unit != nil {
return nil, ErrWindowRowsIntervalUse.GenWithStackByArgs(getWindowName(spec.Name.O))
}
numRows, isNull, isExpectedType := getUintFromNode(b.ctx, boundClause.Expr)
if isNull || !isExpectedType {
return nil, ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
numRows, _, _ := getUintFromNode(b.ctx, boundClause.Expr)
bound.Num = numRows
return bound, nil
}
Expand All @@ -3009,23 +3002,7 @@ func (b *PlanBuilder) buildWindowFunctionFrameBound(spec *ast.WindowSpec, orderB
return bound, nil
}

if len(orderByItems) != 1 {
return nil, ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
col := orderByItems[0].Col
isNumeric, isTemporal := types.IsTypeNumeric(col.RetType.Tp), types.IsTypeTemporal(col.RetType.Tp)
if !isNumeric && !isTemporal {
return nil, ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
// Interval bounds only support order by temporal types.
if boundClause.Unit != nil && isNumeric {
return nil, ErrWindowRangeFrameNumericType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
// Non-interval bound only support order by numeric types.
if boundClause.Unit == nil && !isNumeric {
return nil, ErrWindowRangeFrameTemporalType.GenWithStackByArgs(getWindowName(spec.Name.O))
}

// TODO: We also need to raise error for non-deterministic expressions, like rand().
val, err := evalAstExpr(b.ctx, boundClause.Expr)
if err != nil {
Expand Down Expand Up @@ -3110,25 +3087,13 @@ func (b *PlanBuilder) buildWindowFunctionFrame(spec *ast.WindowSpec, orderByItem
if frameClause == nil {
return nil, nil
}
if frameClause.Type == ast.Groups {
return nil, ErrNotSupportedYet.GenWithStackByArgs("GROUPS")
}
frame := &WindowFrame{Type: frameClause.Type}
start := frameClause.Extent.Start
if start.Type == ast.Following && start.UnBounded {
return nil, ErrWindowFrameStartIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
var err error
frame.Start, err = b.buildWindowFunctionFrameBound(spec, orderByItems, &start)
frame.Start, err = b.buildWindowFunctionFrameBound(spec, orderByItems, &frameClause.Extent.Start)
if err != nil {
return nil, err
}

end := frameClause.Extent.End
if end.Type == ast.Preceding && end.UnBounded {
return nil, ErrWindowFrameEndIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
frame.End, err = b.buildWindowFunctionFrameBound(spec, orderByItems, &end)
frame.End, err = b.buildWindowFunctionFrameBound(spec, orderByItems, &frameClause.Extent.End)
return frame, err
}

Expand Down Expand Up @@ -3203,6 +3168,10 @@ func (b *PlanBuilder) buildWindowFunctions(p LogicalPlan, groupedFuncs map[*ast.
if err != nil {
return nil, nil, err
}
err = b.checkOriginWindowSpecs(funcs, orderBy)
if err != nil {
return nil, nil, err
}
frame, err := b.buildWindowFunctionFrame(spec, orderBy)
if err != nil {
return nil, nil, err
Expand Down Expand Up @@ -3243,6 +3212,74 @@ func (b *PlanBuilder) buildWindowFunctions(p LogicalPlan, groupedFuncs map[*ast.
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.Item) error {
for _, f := range funcs {
spec := f.Spec
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 {
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.Item) error {
if bound.Type == ast.CurrentRow || bound.UnBounded {
return nil
}

frameType := spec.Frame.Type
if frameType == ast.Rows {
if bound.Unit != nil {
return ErrWindowRowsIntervalUse.GenWithStackByArgs(getWindowName(spec.Name.O))
}
_, isNull, isExpectedType := getUintFromNode(b.ctx, bound.Expr)
if isNull || !isExpectedType {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
return nil
}

if len(orderByItems) != 1 {
return ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
orderItemType := orderByItems[0].Col.RetType.Tp
isNumeric, isTemporal := types.IsTypeNumeric(orderItemType), types.IsTypeTemporal(orderItemType)
if !isNumeric && !isTemporal {
return ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if bound.Unit != nil && !isTemporal {
return ErrWindowRangeFrameNumericType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if bound.Unit == nil && !isNumeric {
return ErrWindowRangeFrameTemporalType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
return nil
}

func extractWindowFuncs(fields []*ast.SelectField) []*ast.WindowFuncExpr {
extractor := &WindowFuncExtractor{}
for _, f := range fields {
Expand Down Expand Up @@ -3309,6 +3346,7 @@ func (b *PlanBuilder) groupWindowFuncs(windowFuncs []*ast.WindowFuncExpr) (map[*
if !ok {
return nil, ErrWindowNoSuchWindow.GenWithStackByArgs(windowFunc.Spec.Name.O)
}
windowFunc.Spec = *spec
newSpec, updated := b.handleDefaultFrame(spec, windowFunc.F)
if !updated {
groupedWindow[spec] = append(groupedWindow[spec], windowFunc)
Expand Down
28 changes: 28 additions & 0 deletions planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2332,6 +2332,34 @@ func (s *testPlanSuite) TestWindowFunction(c *C) {
sql: "select sum(a) over (partition by a order by b), sum(b) over (order by a, b, c), sum(c) over(partition by a order by c), sum(d) over() from t",
result: "TableReader(Table(t))->Sort->Window(sum(cast(test.t.c)) over(partition by test.t.a order by test.t.c asc range between unbounded preceding and current row))->Sort->Window(sum(cast(test.t.b)) over(order by test.t.a asc, test.t.b asc, test.t.c asc range between unbounded preceding and current row))->Window(sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc range between unbounded preceding and current row))->Window(sum(cast(test.t.d)) over())->Projection",
},
// Test issue 11010.
{
sql: "select dense_rank() over w1, a, b from t window w1 as (partition by t.b order by t.a desc, t.b desc range between current row and 1 following)",
result: "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type",
},
{
sql: "select dense_rank() over w1, a, b from t window w1 as (partition by t.b order by t.a desc, t.b desc range between current row and unbounded following)",
result: "TableReader(Table(t))->Sort->Window(dense_rank() over(partition by test.t.b order by test.t.a desc, test.t.b desc))->Projection",
},
{
sql: "select dense_rank() over w1, a, b from t window w1 as (partition by t.b order by t.a desc, t.b desc range between 1 preceding and 1 following)",
result: "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type",
},
// Test issue 11001.
{
sql: "SELECT PERCENT_RANK() OVER w1 AS 'percent_rank', fieldA, fieldB FROM ( SELECT a AS fieldA, b AS fieldB FROM t ) t1 WINDOW w1 AS ( ROWS BETWEEN 0 FOLLOWING AND UNBOUNDED PRECEDING)",
result: "[planner:3585]Window 'w1': frame end cannot be UNBOUNDED PRECEDING.",
},
// Test issue 11002.
{
sql: "SELECT PERCENT_RANK() OVER w1 AS 'percent_rank', fieldA, fieldB FROM ( SELECT a AS fieldA, b AS fieldB FROM t ) as t1 WINDOW w1 AS ( ROWS BETWEEN UNBOUNDED FOLLOWING AND UNBOUNDED FOLLOWING)",
result: "[planner:3584]Window 'w1': frame start cannot be UNBOUNDED FOLLOWING.",
},
// Test issue 11011.
{
sql: "select dense_rank() over w1, a, b from t window w1 as (partition by t.b order by t.a asc range between 1250951168 following AND 1250951168 preceding)",
result: "[planner:3586]Window 'w1': frame start or end is negative, NULL or of non-integral type",
},
}

s.Parser.EnableWindowFunc(true)
Expand Down

0 comments on commit 16a81b6

Please sign in to comment.