Skip to content

Commit

Permalink
sql: Add efficient min, max, sum, avg when used as window functions.
Browse files Browse the repository at this point in the history
Adds linear-time implementations of min, max, sum, and avg
(using sliding window approach) instead of naive quadratic
version.

Addresses: cockroachdb#26464.

Bonus: min and max are an order of magnitude faster than PG
(when window frame doesn't include the whole partition).

Release note (performance improvement): min, max, sum, avg
now take linear time when used for aggregation as window
functions.
  • Loading branch information
yuzefovich committed Jul 10, 2018
1 parent 95ed31b commit 77ce2b7
Show file tree
Hide file tree
Showing 7 changed files with 679 additions and 26 deletions.
46 changes: 45 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/window
Original file line number Diff line number Diff line change
Expand Up @@ -1708,7 +1708,6 @@ Tablet iPad 700.00 NULL
Tablet Kindle Fire 150.00 NULL
Tablet Samsung 200.00 NULL


query TRRR
SELECT product_name, price, min(price) OVER (PARTITION BY group_name ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS min_over_three, max(price) OVER (PARTITION BY group_name ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS max_over_partition FROM products ORDER BY group_id;
----
Expand All @@ -1723,3 +1722,48 @@ Dell 800.00 700.00 1200.00
iPad 700.00 150.00 700.00
Kindle Fire 150.00 150.00 700.00
Samsung 200.00 150.00 700.00

query TTRT
SELECT group_name, product_name, price, min(price) OVER (PARTITION BY group_name ROWS CURRENT ROW) AS min_over_single_row FROM products ORDER BY group_id;
----
Smartphone Microsoft Lumia 200.00 200.00
Smartphone HTC One 400.00 400.00
Smartphone Nexus 500.00 500.00
Smartphone iPhone 900.00 900.00
Laptop HP Elite 1200.00 1200.00
Laptop Lenovo Thinkpad 700.00 700.00
Laptop Sony VAIO 700.00 700.00
Laptop Dell 800.00 800.00
Tablet iPad 700.00 700.00
Tablet Kindle Fire 150.00 150.00
Tablet Samsung 200.00 200.00

query TTRR
SELECT group_name, product_name, price, avg(price) OVER (PARTITION BY group_name ROWS BETWEEN 1 FOLLOWING AND UNBOUNDED FOLLOWING) AS running_avg FROM products ORDER BY group_id;
----
Smartphone Microsoft Lumia 200.00 600.00
Smartphone HTC One 400.00 700.00
Smartphone Nexus 500.00 900.00
Smartphone iPhone 900.00 NULL
Laptop HP Elite 1200.00 733.33333333333333333
Laptop Lenovo Thinkpad 700.00 750.00
Laptop Sony VAIO 700.00 800.00
Laptop Dell 800.00 NULL
Tablet iPad 700.00 175.00
Tablet Kindle Fire 150.00 200.00
Tablet Samsung 200.00 NULL

query TRRRRR
SELECT product_name, price, min(price) OVER (PARTITION BY group_name ROWS UNBOUNDED PRECEDING), max(price) OVER (PARTITION BY group_name ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), sum(price) OVER (PARTITION BY group_name ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING), avg(price) OVER (PARTITION BY group_name ROWS CURRENT ROW) FROM products ORDER BY group_id;
----
Microsoft Lumia 200.00 200.00 400.00 2000.00 200.00
HTC One 400.00 200.00 500.00 2000.00 400.00
Nexus 500.00 200.00 900.00 1800.00 500.00
iPhone 900.00 200.00 900.00 1400.00 900.00
HP Elite 1200.00 1200.00 1200.00 3400.00 1200.00
Lenovo Thinkpad 700.00 700.00 1200.00 3400.00 700.00
Sony VAIO 700.00 700.00 1200.00 2200.00 700.00
Dell 800.00 700.00 1200.00 1500.00 800.00
iPad 700.00 700.00 700.00 1050.00 700.00
Kindle Fire 150.00 150.00 700.00 1050.00 150.00
Samsung 200.00 150.00 700.00 350.00 200.00
35 changes: 33 additions & 2 deletions pkg/sql/sem/builtins/aggregate_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,9 @@ var aggregates = map[string]builtinDefinition{
ReturnType: tree.FixedReturnType(types.Int),
AggregateFunc: newCountRowsAggregate,
WindowFunc: func(params []types.T, evalCtx *tree.EvalContext) tree.WindowFunc {
return newFramableAggregateWindow(newCountRowsAggregate(params, evalCtx))
return newFramableAggregateWindow(newCountRowsAggregate(params, evalCtx), func(evalCtx *tree.EvalContext) tree.AggregateFunc {
return newCountRowsAggregate(params, evalCtx)
})
},
Info: "Calculates the number of rows.",
},
Expand Down Expand Up @@ -318,7 +320,36 @@ func makeAggOverloadWithReturnType(
ReturnType: retType,
AggregateFunc: f,
WindowFunc: func(params []types.T, evalCtx *tree.EvalContext) tree.WindowFunc {
return newFramableAggregateWindow(f(params, evalCtx))
aggWindowFunc := f(params, evalCtx)
switch w := aggWindowFunc.(type) {
case *MinAggregate:
min := &slidingWindowFunc{}
min.sw = makeSlidingWindow(evalCtx, func(evalCtx *tree.EvalContext, a, b tree.Datum) int {
return -a.Compare(evalCtx, b)
})
return min
case *MaxAggregate:
max := &slidingWindowFunc{}
max.sw = makeSlidingWindow(evalCtx, func(evalCtx *tree.EvalContext, a, b tree.Datum) int {
return a.Compare(evalCtx, b)
})
return max
case *intSumAggregate:
return &slidingWindowSumFunc{agg: aggWindowFunc}
case *decimalSumAggregate:
return &slidingWindowSumFunc{agg: aggWindowFunc}
case *floatSumAggregate:
return &slidingWindowSumFunc{agg: aggWindowFunc}
case *intervalSumAggregate:
return &slidingWindowSumFunc{agg: aggWindowFunc}
case *avgAggregate:
// w.agg is a sum aggregate.
return &avgWindowFunc{sum: slidingWindowSumFunc{agg: w.agg}}
}

return newFramableAggregateWindow(aggWindowFunc, func(evalCtx *tree.EvalContext) tree.AggregateFunc {
return f(params, evalCtx)
})
},
Info: info,
}
Expand Down
33 changes: 16 additions & 17 deletions pkg/sql/sem/builtins/window_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,15 +178,26 @@ func (w *aggregateWindowFunc) Close(ctx context.Context, evalCtx *tree.EvalConte
w.agg.Close(ctx)
}

// ShouldReset sets shouldReset to true if w is framableAggregateWindowFunc.
func ShouldReset(w tree.WindowFunc) {
if f, ok := w.(*framableAggregateWindowFunc); ok {
f.shouldReset = true
}
}

// framableAggregateWindowFunc is a wrapper around aggregateWindowFunc that allows
// to reset the aggregate by creating a new instance via a provided constructor.
// shouldReset indicates whether the resetting behavior is desired.
type framableAggregateWindowFunc struct {
agg *aggregateWindowFunc
aggConstructor func(*tree.EvalContext) tree.AggregateFunc
shouldReset bool
}

func newFramableAggregateWindow(agg tree.AggregateFunc) tree.WindowFunc {
return &framableAggregateWindowFunc{agg: &aggregateWindowFunc{agg: agg}}
func newFramableAggregateWindow(
agg tree.AggregateFunc, aggConstructor func(*tree.EvalContext) tree.AggregateFunc,
) tree.WindowFunc {
return &framableAggregateWindowFunc{agg: &aggregateWindowFunc{agg: agg}, aggConstructor: aggConstructor}
}

func (w *framableAggregateWindowFunc) Compute(
Expand All @@ -195,12 +206,12 @@ func (w *framableAggregateWindowFunc) Compute(
if !wfr.FirstInPeerGroup() {
return w.agg.peerRes, nil
}
if w.aggConstructor == nil {
// No constructor is given, so we use default approach.
if !w.shouldReset {
// We should not reset, so we will use the same aggregateWindowFunc.
return w.agg.Compute(ctx, evalCtx, wfr)
}

// When aggConstructor is provided, we want to dispose of the old aggregate function
// We should reset the aggregate, so we dispose of the old aggregate function
// and construct a new one for the computation.
w.agg.Close(ctx, evalCtx)
*w.agg = aggregateWindowFunc{w.aggConstructor(evalCtx), tree.DNull}
Expand Down Expand Up @@ -231,18 +242,6 @@ func (w *framableAggregateWindowFunc) Close(ctx context.Context, evalCtx *tree.E
w.agg.Close(ctx, evalCtx)
}

// AddAggregateConstructorToFramableAggregate adds provided constructor to framableAggregateWindowFunc
// so that aggregates can be 'reset' when computing values over a window frame.
func AddAggregateConstructorToFramableAggregate(
windowFunc tree.WindowFunc, aggConstructor func(*tree.EvalContext) tree.AggregateFunc,
) {
// We only want to add aggConstructor to framableAggregateWindowFunc's since
// all non-aggregates builtins specific to window functions support framing "natively".
if framableAgg, ok := windowFunc.(*framableAggregateWindowFunc); ok {
framableAgg.aggConstructor = aggConstructor
}
}

// rowNumberWindow computes the number of the current row within its partition,
// counting from 1.
type rowNumberWindow struct{}
Expand Down
Loading

0 comments on commit 77ce2b7

Please sign in to comment.