-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathaggregate_funcs.go
363 lines (333 loc) · 12.9 KB
/
aggregate_funcs.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
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package colexec
import (
"unsafe"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)
// isAggOptimized returns whether aggFn has an optimized implementation.
func isAggOptimized(aggFn execinfrapb.AggregatorSpec_Func) bool {
switch aggFn {
case execinfrapb.AggregatorSpec_ANY_NOT_NULL,
execinfrapb.AggregatorSpec_AVG,
execinfrapb.AggregatorSpec_SUM,
execinfrapb.AggregatorSpec_SUM_INT,
execinfrapb.AggregatorSpec_CONCAT_AGG,
execinfrapb.AggregatorSpec_COUNT_ROWS,
execinfrapb.AggregatorSpec_COUNT,
execinfrapb.AggregatorSpec_MIN,
execinfrapb.AggregatorSpec_MAX,
execinfrapb.AggregatorSpec_BOOL_AND,
execinfrapb.AggregatorSpec_BOOL_OR:
return true
default:
return false
}
}
// aggregateFunc is an aggregate function that performs computation on a batch
// when Compute(batch) is called and writes the output to the Vec passed in
// in Init. The aggregateFunc performs an aggregation per group and outputs the
// aggregation once the start of the new group is reached. If the end of the
// group is not reached before the batch is finished, the aggregateFunc will
// store a carry value that it will use next time Compute is called. Note that
// this carry value is stored at the output index. Therefore if any memory
// modification of the output vector is made, the caller *MUST* copy the value
// at the current index inclusive for a correct aggregation.
type aggregateFunc interface {
// Init sets the groups for the aggregation and the output vector. Each index
// in groups corresponds to a column value in the input batch. true represents
// the start of a new group. Note that the very first group in the whole
// input should *not* be marked as a start of a new group.
Init(groups []bool, vec coldata.Vec)
// Reset resets the aggregate function for another run. Primarily used for
// benchmarks.
Reset()
// CurrentOutputIndex returns the current index in the output vector that the
// aggregate function is writing to. All indices < the index returned are
// finished aggregations for previous groups. A negative index may be returned
// to signify an aggregate function that has not yet performed any
// computation.
CurrentOutputIndex() int
// SetOutputIndex sets the output index to write to. The value for the current
// index is carried over.
SetOutputIndex(idx int)
// Compute computes the aggregation on the input batch.
// Note: the implementations should be careful to account for their memory
// usage.
Compute(vecs []coldata.Vec, inputIdxs []uint32, inputLen int, sel []int)
// Flush flushes the result of aggregation on the last group. It should be
// called once after input batches have been Compute()'d. outputIdx is only
// used in case of hash aggregation - for ordered aggregation the aggregate
// function itself should maintain the output index to write to.
// Note: the implementations are free to not account for the memory used
// for the result of aggregation of the last group.
Flush(outputIdx int)
// HandleEmptyInputScalar populates the output for a case of an empty input
// when the aggregate function is in scalar context. The output must always
// be a single value (either null or zero, depending on the function).
HandleEmptyInputScalar()
}
type orderedAggregateFuncBase struct {
groups []bool
// curIdx tracks the current output index of this function.
curIdx int
// nulls is the nulls vector of the output vector of this function.
nulls *coldata.Nulls
}
func (o *orderedAggregateFuncBase) Init(groups []bool, vec coldata.Vec) {
o.groups = groups
o.nulls = vec.Nulls()
}
func (o *orderedAggregateFuncBase) Reset() {
o.curIdx = 0
o.nulls.UnsetNulls()
}
func (o *orderedAggregateFuncBase) CurrentOutputIndex() int {
return o.curIdx
}
func (o *orderedAggregateFuncBase) SetOutputIndex(idx int) {
o.curIdx = idx
}
func (o *orderedAggregateFuncBase) HandleEmptyInputScalar() {
// Most aggregate functions return a single NULL value on an empty input
// in the scalar context (the exceptions are COUNT aggregates which need
// to overwrite this method).
o.nulls.SetNull(0)
}
type hashAggregateFuncBase struct {
// nulls is the nulls vector of the output vector of this function.
nulls *coldata.Nulls
}
func (h *hashAggregateFuncBase) Init(_ []bool, vec coldata.Vec) {
h.nulls = vec.Nulls()
}
func (h *hashAggregateFuncBase) Reset() {
h.nulls.UnsetNulls()
}
func (h *hashAggregateFuncBase) CurrentOutputIndex() int {
colexecerror.InternalError("CurrentOutputIndex called with hash aggregation")
// This code is unreachable, but the compiler cannot infer that.
return 0
}
func (h *hashAggregateFuncBase) SetOutputIndex(int) {
colexecerror.InternalError("SetOutputIndex called with hash aggregation")
}
func (h *hashAggregateFuncBase) HandleEmptyInputScalar() {
colexecerror.InternalError("HandleEmptyInputScalar called with hash aggregation")
}
// aggregateFuncAlloc is an aggregate function allocator that pools allocations
// of the structs of the same statically-typed aggregate function.
type aggregateFuncAlloc interface {
// newAggFunc returns the aggregate function from the pool with all
// necessary fields initialized.
newAggFunc() aggregateFunc
}
// aggregateFuncsAlloc is a utility struct that pools allocations of multiple
// aggregate functions simultaneously (i.e. it supports a "schema of aggregate
// functions"). It will resolve the aggregate functions in its constructor to
// instantiate aggregateFuncAlloc objects and will use those to populate slices
// of new aggregation functions when requested.
type aggregateFuncsAlloc struct {
allocator *colmem.Allocator
// allocSize determines the number of objects allocated when the previous
// allocations have been used up.
allocSize int64
// returnFuncs is the pool for the slice to be returned in
// makeAggregateFuncs.
returnFuncs []aggregateFunc
// aggFuncAllocs are all necessary aggregate function allocators. Note that
// a separate aggregateFuncAlloc will be created for each aggFn from the
// schema (even if there are "duplicates" - exactly the same functions - in
// the function schema).
aggFuncAllocs []aggregateFuncAlloc
}
func newAggregateFuncsAlloc(
allocator *colmem.Allocator,
inputTypes []*types.T,
spec *execinfrapb.AggregatorSpec,
evalCtx *tree.EvalContext,
constructors []execinfrapb.AggregateConstructor,
constArguments []tree.Datums,
outputTypes []*types.T,
allocSize int64,
isHashAgg bool,
) (*aggregateFuncsAlloc, *vecToDatumConverter, Closers, error) {
funcAllocs := make([]aggregateFuncAlloc, len(spec.Aggregations))
var toClose Closers
var vecIdxsToConvert []int
for _, aggFn := range spec.Aggregations {
if !isAggOptimized(aggFn.Func) {
for _, vecIdx := range aggFn.ColIdx {
found := false
for i := range vecIdxsToConvert {
if vecIdxsToConvert[i] == int(vecIdx) {
found = true
break
}
}
if !found {
vecIdxsToConvert = append(vecIdxsToConvert, int(vecIdx))
}
}
}
}
inputArgsConverter := newVecToDatumConverter(len(inputTypes), vecIdxsToConvert)
for i, aggFn := range spec.Aggregations {
var err error
switch aggFn.Func {
case execinfrapb.AggregatorSpec_ANY_NOT_NULL:
if isHashAgg {
funcAllocs[i], err = newAnyNotNullHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i], err = newAnyNotNullOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_AVG:
if isHashAgg {
funcAllocs[i], err = newAvgHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i], err = newAvgOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_SUM:
if isHashAgg {
funcAllocs[i], err = newSumHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i], err = newSumOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_SUM_INT:
if isHashAgg {
funcAllocs[i], err = newSumIntHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i], err = newSumIntOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_CONCAT_AGG:
if isHashAgg {
funcAllocs[i] = newConcatHashAggAlloc(allocator, allocSize)
} else {
funcAllocs[i] = newConcatOrderedAggAlloc(allocator, allocSize)
}
case execinfrapb.AggregatorSpec_COUNT_ROWS:
if isHashAgg {
funcAllocs[i] = newCountRowsHashAggAlloc(allocator, allocSize)
} else {
funcAllocs[i] = newCountRowsOrderedAggAlloc(allocator, allocSize)
}
case execinfrapb.AggregatorSpec_COUNT:
if isHashAgg {
funcAllocs[i] = newCountHashAggAlloc(allocator, allocSize)
} else {
funcAllocs[i] = newCountOrderedAggAlloc(allocator, allocSize)
}
case execinfrapb.AggregatorSpec_MIN:
if isHashAgg {
funcAllocs[i] = newMinHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i] = newMinOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_MAX:
if isHashAgg {
funcAllocs[i] = newMaxHashAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
} else {
funcAllocs[i] = newMaxOrderedAggAlloc(allocator, inputTypes[aggFn.ColIdx[0]], allocSize)
}
case execinfrapb.AggregatorSpec_BOOL_AND:
if isHashAgg {
funcAllocs[i] = newBoolAndHashAggAlloc(allocator, allocSize)
} else {
funcAllocs[i] = newBoolAndOrderedAggAlloc(allocator, allocSize)
}
case execinfrapb.AggregatorSpec_BOOL_OR:
if isHashAgg {
funcAllocs[i] = newBoolOrHashAggAlloc(allocator, allocSize)
} else {
funcAllocs[i] = newBoolOrOrderedAggAlloc(allocator, allocSize)
}
// NOTE: if you're adding an implementation of a new aggregate
// function, make sure to account for the memory under that struct in
// its constructor.
default:
if isHashAgg {
funcAllocs[i] = newDefaultHashAggAlloc(
allocator, constructors[i], evalCtx, inputArgsConverter,
len(aggFn.ColIdx), constArguments[i], outputTypes[i], allocSize,
)
} else {
funcAllocs[i] = newDefaultOrderedAggAlloc(
allocator, constructors[i], evalCtx, inputArgsConverter,
len(aggFn.ColIdx), constArguments[i], outputTypes[i], allocSize,
)
}
toClose = append(toClose, funcAllocs[i].(Closer))
}
if err != nil {
return nil, nil, nil, err
}
}
return &aggregateFuncsAlloc{
allocator: allocator,
allocSize: allocSize,
aggFuncAllocs: funcAllocs,
}, inputArgsConverter, toClose, nil
}
// sizeOfAggregateFunc is the size of some aggregateFunc implementation.
// countHashAgg was chosen arbitrarily, but it's important that we use a
// pointer to the aggregate function struct.
const sizeOfAggregateFunc = int64(unsafe.Sizeof(&countHashAgg{}))
func (a *aggregateFuncsAlloc) makeAggregateFuncs() []aggregateFunc {
if len(a.returnFuncs) == 0 {
// We have exhausted the previously allocated pools of objects, so we
// need to allocate a new slice for a.returnFuncs, and we need it to be
// of 'allocSize x number of funcs in schema' length. Every
// aggFuncAlloc will allocate allocSize of objects on the newAggFunc
// call below.
a.allocator.AdjustMemoryUsage(sizeOfAggregateFunc * int64(len(a.aggFuncAllocs)) * a.allocSize)
a.returnFuncs = make([]aggregateFunc, len(a.aggFuncAllocs)*int(a.allocSize))
}
funcs := a.returnFuncs[:len(a.aggFuncAllocs)]
a.returnFuncs = a.returnFuncs[len(a.aggFuncAllocs):]
for i, alloc := range a.aggFuncAllocs {
funcs[i] = alloc.newAggFunc()
}
return funcs
}
type aggAllocBase struct {
allocator *colmem.Allocator
allocSize int64
}
// ProcessAggregations processes all aggregate functions specified in
// aggregations.
func ProcessAggregations(
evalCtx *tree.EvalContext,
semaCtx *tree.SemaContext,
aggregations []execinfrapb.AggregatorSpec_Aggregation,
inputTypes []*types.T,
) (
constructors []execinfrapb.AggregateConstructor,
constArguments []tree.Datums,
outputTypes []*types.T,
err error,
) {
constructors = make([]execinfrapb.AggregateConstructor, len(aggregations))
constArguments = make([]tree.Datums, len(aggregations))
outputTypes = make([]*types.T, len(aggregations))
for i, aggFn := range aggregations {
constructors[i], constArguments[i], outputTypes[i], err = execinfrapb.GetAggregateConstructor(
evalCtx, semaCtx, &aggFn, inputTypes,
)
if err != nil {
return
}
}
return
}