forked from cockroachdb/cockroach
/
aggregator.go
287 lines (257 loc) · 7.45 KB
/
aggregator.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
// Copyright 2016 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
//
// Author: Irfan Sharif (irfansharif@cockroachlabs.com)
package distsql
import (
"sync"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/pkg/errors"
"golang.org/x/net/context"
)
// aggregator is the processor core type that does "aggregation" in the SQL
// sense. It groups rows and computes an aggregate for each group. The group is
// configured using the group key and the aggregator can be configured with one
// or more of the following aggregation functions:
//
// SUM
// COUNT
// MIN
// MAX
// AVG
// DISTINCT
// COUNT DISTINCT
//
// aggregator's output schema is comprised of what is specified by the
// accompanying SELECT expressions.
type aggregator struct {
input RowSource
output RowReceiver
ctx context.Context
rows *RowBuffer
funcs []*aggregateFuncHolder
datumAlloc sqlbase.DatumAlloc
rowAlloc sqlbase.EncDatumRowAlloc
groupCols columns
inputCols columns
buckets map[string]struct{} // The set of bucket keys.
}
func newAggregator(
ctx *FlowCtx, spec *AggregatorSpec, input RowSource, output RowReceiver,
) (*aggregator, error) {
ag := &aggregator{
input: input,
output: output,
ctx: log.WithLogTag(ctx.Context, "Agg", nil),
rows: &RowBuffer{},
buckets: make(map[string]struct{}),
inputCols: make(columns, len(spec.Exprs)),
groupCols: make(columns, len(spec.GroupCols)),
}
ag.inputCols = make(columns, len(spec.Exprs))
for i, expr := range spec.Exprs {
ag.inputCols[i] = expr.ColIdx
}
copy(ag.groupCols, spec.GroupCols)
// Loop over the select expressions and extract any aggregate functions --
// non-aggregation functions are replaced with parser.NewIdentAggregate,
// (which just returns the last value added to them for a bucket) to provide
// grouped-by values for each bucket. ag.funcs is updated to contain all
// the functions which need to be fed values.
eh := &exprHelper{types: spec.Types}
eh.vars = parser.MakeIndexedVarHelper(eh, len(eh.types))
for _, expr := range spec.Exprs {
fn, err := ag.extractFunc(expr, eh)
if err != nil {
return nil, err
}
ag.funcs = append(ag.funcs, fn)
}
return ag, nil
}
// Run is part of the processor interface.
func (ag *aggregator) Run(wg *sync.WaitGroup) {
if wg != nil {
defer wg.Done()
}
if log.V(2) {
log.Infof(ag.ctx, "starting aggregation process")
defer log.Infof(ag.ctx, "exiting aggregator")
}
if err := ag.accumulateRows(); err != nil {
ag.output.Close(err)
return
}
if err := ag.computeAggregates(); err != nil {
ag.output.Close(err)
return
}
for {
row, err := ag.rows.NextRow()
if err != nil || row == nil {
ag.output.Close(err)
return
}
if log.V(3) {
log.Infof(ag.ctx, "pushing %s\n", row)
}
// Push the row to the output RowReceiver; stop if they don't need more
// rows.
if !ag.output.PushRow(row) {
if log.V(2) {
log.Infof(ag.ctx, "no more rows required")
}
ag.output.Close(nil)
return
}
}
}
func (ag *aggregator) accumulateRows() error {
var scratch []byte
for {
row, err := ag.input.NextRow()
if err != nil {
return err
}
if row == nil {
return nil
}
// The encoding computed here determines which bucket the non-grouping
// datums are accumulated to.
encoded, err := ag.encode(scratch, row)
if err != nil {
return err
}
ag.buckets[string(encoded)] = struct{}{}
// Feed the func holders for this bucket the non-grouping datums.
for i, colIdx := range ag.inputCols {
if err := row[colIdx].Decode(&ag.datumAlloc); err != nil {
return err
}
if err := ag.funcs[i].add(encoded, row[colIdx].Datum); err != nil {
return err
}
}
scratch = encoded[:0]
}
}
func (ag *aggregator) computeAggregates() error {
// Render the results.
tuple := make(parser.DTuple, 0, len(ag.funcs))
for bucket := range ag.buckets {
for _, f := range ag.funcs {
datum := f.get(bucket)
tuple = append(tuple, datum)
}
row := ag.rowAlloc.AllocRow(len(tuple))
err := sqlbase.DTupleToEncDatumRow(row, tuple)
if err != nil {
return err
}
if ok := ag.rows.PushRow(row); !ok {
return errors.Errorf("unable to add row %s", row)
}
tuple = tuple[:0]
}
return nil
}
type aggregateFuncHolder struct {
create func() parser.AggregateFunc
group *aggregator
buckets map[string]parser.AggregateFunc
seen map[string]struct{}
}
func (ag *aggregator) newAggregateFuncHolder(
create func() parser.AggregateFunc,
) *aggregateFuncHolder {
return &aggregateFuncHolder{
create: create,
group: ag,
buckets: make(map[string]parser.AggregateFunc),
}
}
func (a *aggregateFuncHolder) add(bucket []byte, d parser.Datum) error {
if a.seen != nil {
encoded, err := sqlbase.EncodeDatum(bucket, d)
if err != nil {
return err
}
if _, ok := a.seen[string(encoded)]; ok {
// skip
return nil
}
a.seen[string(encoded)] = struct{}{}
}
impl, ok := a.buckets[string(bucket)]
if !ok {
impl = a.create()
a.buckets[string(bucket)] = impl
}
impl.Add(d)
return nil
}
func (a *aggregateFuncHolder) get(bucket string) parser.Datum {
found, ok := a.buckets[bucket]
if !ok {
found = a.create()
}
return found.Result()
}
// encode returns the encoding for the grouping columns, this is then used as
// our group key to determine which bucket to add to.
func (ag *aggregator) encode(appendTo []byte, row sqlbase.EncDatumRow) (encoding []byte, err error) {
for _, colIdx := range ag.groupCols {
appendTo, err = row[colIdx].Encode(&ag.datumAlloc, sqlbase.DatumEncoding_VALUE, appendTo)
if err != nil {
return appendTo, err
}
}
return appendTo, nil
}
// extractFunc returns an aggregateFuncHolder for a given SelectExpr specifying
// an aggregation function.
func (ag *aggregator) extractFunc(
expr AggregatorSpec_Expr, eh *exprHelper,
) (*aggregateFuncHolder, error) {
if expr.Func == AggregatorSpec_IDENT {
fn := ag.newAggregateFuncHolder(parser.NewIdentAggregate)
return fn, nil
}
// In order to reuse the aggregate functions as defined in the parser
// package we are relying on the fact the each name defined in the Func enum
// within the AggregatorSpec matches a SQL function name known to the parser.
// See pkg/sql/parser/aggregate_builtins.go for the aggregate builtins we
// are repurposing.
p := &parser.FuncExpr{
Func: parser.ResolvableFunctionReference{
FunctionReference: parser.UnresolvedName{parser.Name(expr.Func.String())},
},
Exprs: []parser.Expr{eh.indexToExpr(int(expr.ColIdx))},
}
_, err := p.TypeCheck(nil, parser.NoTypePreference)
if err != nil {
return nil, err
}
if agg := p.GetAggregateConstructor(); agg != nil {
fn := ag.newAggregateFuncHolder(agg)
if expr.Distinct {
fn.seen = make(map[string]struct{})
}
return fn, nil
}
return nil, errors.Errorf("unable to get aggregate constructor for %s",
AggregatorSpec_Func_name[int32(expr.Func)])
}