forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
aggregate.go
441 lines (411 loc) · 11.9 KB
/
aggregate.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
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
// Copyright 2016 PingCAP, Inc.
//
// 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,
// See the License for the specific language governing permissions and
// limitations under the License.
package executor
import (
"github.com/juju/errors"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/mvmap"
"golang.org/x/net/context"
)
type aggCtxsMapper map[string][]*aggregation.AggEvaluateContext
// HashAggExec deals with all the aggregate functions.
// It is built from the Aggregate Plan. When Next() is called, it reads all the data from Src
// and updates all the items in AggFuncs.
type HashAggExec struct {
baseExecutor
executed bool
sc *stmtctx.StatementContext
AggFuncs []aggregation.Aggregation
aggCtxsMap aggCtxsMapper
groupMap *mvmap.MVMap
groupIterator *mvmap.Iterator
mutableRow chunk.MutRow
rowBuffer []types.Datum
GroupByItems []expression.Expression
groupKey []byte
groupVals [][]byte
}
// Close implements the Executor Close interface.
func (e *HashAggExec) Close() error {
if err := e.baseExecutor.Close(); err != nil {
return errors.Trace(err)
}
e.groupMap = nil
e.groupIterator = nil
e.aggCtxsMap = nil
return nil
}
// Open implements the Executor Open interface.
func (e *HashAggExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return errors.Trace(err)
}
e.executed = false
e.groupMap = mvmap.NewMVMap()
e.groupIterator = e.groupMap.NewIterator()
e.aggCtxsMap = make(aggCtxsMapper, 0)
e.mutableRow = chunk.MutRowFromTypes(e.retTypes())
e.rowBuffer = make([]types.Datum, 0, e.Schema().Len())
e.groupKey = make([]byte, 0, 8)
e.groupVals = make([][]byte, 0, 8)
return nil
}
// NextChunk implements the Executor NextChunk interface.
func (e *HashAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error {
// In this stage we consider all data from src as a single group.
if !e.executed {
err := e.execute(ctx)
if err != nil {
return errors.Trace(err)
}
if (e.groupMap.Len() == 0) && len(e.GroupByItems) == 0 {
// If no groupby and no data, we should add an empty group.
// For example:
// "select count(c) from t;" should return one row [0]
// "select count(c) from t group by c1;" should return empty result set.
e.groupMap.Put([]byte{}, []byte{})
}
e.executed = true
}
chk.Reset()
for {
groupKey, _ := e.groupIterator.Next()
if groupKey == nil {
return nil
}
aggCtxs := e.getContexts(groupKey)
e.rowBuffer = e.rowBuffer[:0]
for i, af := range e.AggFuncs {
e.rowBuffer = append(e.rowBuffer, af.GetResult(aggCtxs[i]))
}
e.mutableRow.SetDatums(e.rowBuffer...)
chk.AppendRow(e.mutableRow.ToRow())
if chk.NumRows() == e.maxChunkSize {
return nil
}
}
}
// innerNextChunk fetches Chunks from src and update each aggregate function for each row in Chunk.
func (e *HashAggExec) execute(ctx context.Context) (err error) {
inputIter := chunk.NewIterator4Chunk(e.childrenResults[0])
for {
err := e.children[0].NextChunk(ctx, e.childrenResults[0])
if err != nil {
return errors.Trace(err)
}
// no more data.
if e.childrenResults[0].NumRows() == 0 {
return nil
}
for row := inputIter.Begin(); row != inputIter.End(); row = inputIter.Next() {
groupKey, err := e.getGroupKey(row)
if err != nil {
return errors.Trace(err)
}
if len(e.groupMap.Get(groupKey, e.groupVals[:0])) == 0 {
e.groupMap.Put(groupKey, []byte{})
}
aggCtxs := e.getContexts(groupKey)
for i, af := range e.AggFuncs {
err = af.Update(aggCtxs[i], e.sc, row)
if err != nil {
return errors.Trace(err)
}
}
}
}
}
// Next implements the Executor Next interface.
func (e *HashAggExec) Next(ctx context.Context) (Row, error) {
// In this stage we consider all data from src as a single group.
if !e.executed {
for {
hasMore, err := e.innerNext(ctx)
if err != nil {
return nil, errors.Trace(err)
}
if !hasMore {
break
}
}
if (e.groupMap.Len() == 0) && len(e.GroupByItems) == 0 {
// If no groupby and no data, we should add an empty group.
// For example:
// "select count(c) from t;" should return one row [0]
// "select count(c) from t group by c1;" should return empty result set.
e.groupMap.Put([]byte{}, []byte{})
}
e.executed = true
}
groupKey, _ := e.groupIterator.Next()
if groupKey == nil {
return nil, nil
}
retRow := make([]types.Datum, 0, len(e.AggFuncs))
aggCtxs := e.getContexts(groupKey)
for i, af := range e.AggFuncs {
retRow = append(retRow, af.GetResult(aggCtxs[i]))
}
return retRow, nil
}
func (e *HashAggExec) getGroupKey(row types.Row) ([]byte, error) {
vals := make([]types.Datum, 0, len(e.GroupByItems))
for _, item := range e.GroupByItems {
v, err := item.Eval(row)
if item.GetType().Tp == mysql.TypeNewDecimal {
v.SetLength(0)
}
if err != nil {
return nil, errors.Trace(err)
}
vals = append(vals, v)
}
var err error
e.groupKey, err = codec.EncodeValue(e.sc, e.groupKey[:0], vals...)
if err != nil {
return nil, errors.Trace(err)
}
return e.groupKey, nil
}
// innerNext fetches a single row from src and update each aggregate function.
// If the first return value is false, it means there is no more data from src.
func (e *HashAggExec) innerNext(ctx context.Context) (ret bool, err error) {
srcRow, err := e.children[0].Next(ctx)
if err != nil {
return false, errors.Trace(err)
}
if srcRow == nil {
return false, nil
}
e.executed = true
groupKey, err := e.getGroupKey(srcRow)
if err != nil {
return false, errors.Trace(err)
}
if len(e.groupMap.Get(groupKey, e.groupVals[:0])) == 0 {
e.groupMap.Put(groupKey, []byte{})
}
aggCtxs := e.getContexts(groupKey)
for i, af := range e.AggFuncs {
err = af.Update(aggCtxs[i], e.sc, srcRow)
if err != nil {
return false, errors.Trace(err)
}
}
return true, nil
}
func (e *HashAggExec) getContexts(groupKey []byte) []*aggregation.AggEvaluateContext {
groupKeyString := string(groupKey)
aggCtxs, ok := e.aggCtxsMap[groupKeyString]
if !ok {
aggCtxs = make([]*aggregation.AggEvaluateContext, 0, len(e.AggFuncs))
for _, af := range e.AggFuncs {
aggCtxs = append(aggCtxs, af.CreateContext(e.ctx.GetSessionVars().StmtCtx))
}
e.aggCtxsMap[groupKeyString] = aggCtxs
}
return aggCtxs
}
// StreamAggExec deals with all the aggregate functions.
// It assumes all the input data is sorted by group by key.
// When Next() is called, it will return a result for the same group.
type StreamAggExec struct {
baseExecutor
executed bool
hasData bool
StmtCtx *stmtctx.StatementContext
AggFuncs []aggregation.Aggregation
aggCtxs []*aggregation.AggEvaluateContext
GroupByItems []expression.Expression
curGroupKey []types.Datum
tmpGroupKey []types.Datum
// for chunk execution.
inputIter *chunk.Iterator4Chunk
inputRow chunk.Row
mutableRow chunk.MutRow
rowBuffer []types.Datum
}
// Open implements the Executor Open interface.
func (e *StreamAggExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return errors.Trace(err)
}
e.executed = false
e.hasData = false
e.inputIter = chunk.NewIterator4Chunk(e.childrenResults[0])
e.inputRow = e.inputIter.End()
e.mutableRow = chunk.MutRowFromTypes(e.retTypes())
e.rowBuffer = make([]types.Datum, 0, e.Schema().Len())
e.aggCtxs = make([]*aggregation.AggEvaluateContext, 0, len(e.AggFuncs))
for _, agg := range e.AggFuncs {
e.aggCtxs = append(e.aggCtxs, agg.CreateContext(e.ctx.GetSessionVars().StmtCtx))
}
return nil
}
// Next implements the Executor Next interface.
func (e *StreamAggExec) Next(ctx context.Context) (Row, error) {
if e.executed {
return nil, nil
}
retRow := make([]types.Datum, 0, len(e.AggFuncs))
for {
row, err := e.children[0].Next(ctx)
if err != nil {
return nil, errors.Trace(err)
}
var newGroup bool
if row == nil {
newGroup = true
e.executed = true
} else {
e.hasData = true
newGroup, err = e.meetNewGroup(row)
if err != nil {
return nil, errors.Trace(err)
}
}
if newGroup {
for i, af := range e.AggFuncs {
retRow = append(retRow, af.GetResult(e.aggCtxs[i]))
// Clear stream results after grabbing them.
e.aggCtxs[i] = af.CreateContext(e.ctx.GetSessionVars().StmtCtx)
}
}
if e.executed {
break
}
for i, af := range e.AggFuncs {
err = af.Update(e.aggCtxs[i], e.StmtCtx, row)
if err != nil {
return nil, errors.Trace(err)
}
}
if newGroup {
break
}
}
if !e.hasData && len(e.GroupByItems) > 0 {
return nil, nil
}
return retRow, nil
}
// NextChunk implements the Executor NextChunk interface.
func (e *StreamAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error {
chk.Reset()
for !e.executed && chk.NumRows() < e.maxChunkSize {
err := e.consumeOneGroup(ctx, chk)
if err != nil {
e.executed = true
return errors.Trace(err)
}
}
return nil
}
func (e *StreamAggExec) consumeOneGroup(ctx context.Context, chk *chunk.Chunk) error {
for !e.executed {
if err := e.fetchChildIfNecessary(ctx, chk); err != nil {
return errors.Trace(err)
}
for ; e.inputRow != e.inputIter.End(); e.inputRow = e.inputIter.Next() {
meetNewGroup, err := e.meetNewGroup(e.inputRow)
if err != nil {
return errors.Trace(err)
}
if meetNewGroup {
e.appendResult2Chunk(chk)
}
for i, af := range e.AggFuncs {
err := af.Update(e.aggCtxs[i], e.StmtCtx, e.inputRow)
if err != nil {
return errors.Trace(err)
}
}
if meetNewGroup {
e.inputRow = e.inputIter.Next()
return nil
}
}
}
return nil
}
func (e *StreamAggExec) fetchChildIfNecessary(ctx context.Context, chk *chunk.Chunk) error {
if e.inputRow != e.inputIter.End() {
return nil
}
err := e.children[0].NextChunk(ctx, e.childrenResults[0])
if err != nil {
return errors.Trace(err)
}
// No more data.
if e.childrenResults[0].NumRows() == 0 {
if e.hasData || len(e.GroupByItems) == 0 {
e.appendResult2Chunk(chk)
}
e.executed = true
return nil
}
// Reach here, "e.childrenResults[0].NumRows() > 0" is guaranteed.
e.inputRow = e.inputIter.Begin()
e.hasData = true
return nil
}
// appendResult2Chunk appends result of all the aggregation functions to the
// result chunk, and realloc the evaluation context for each aggregation.
func (e *StreamAggExec) appendResult2Chunk(chk *chunk.Chunk) {
e.rowBuffer = e.rowBuffer[:0]
for i, af := range e.AggFuncs {
e.rowBuffer = append(e.rowBuffer, af.GetResult(e.aggCtxs[i]))
e.aggCtxs[i] = af.CreateContext(e.ctx.GetSessionVars().StmtCtx)
}
e.mutableRow.SetDatums(e.rowBuffer...)
chk.AppendRow(e.mutableRow.ToRow())
}
// meetNewGroup returns a value that represents if the new group is different from last group.
func (e *StreamAggExec) meetNewGroup(row types.Row) (bool, error) {
if len(e.GroupByItems) == 0 {
return false, nil
}
e.tmpGroupKey = e.tmpGroupKey[:0]
matched, firstGroup := true, false
if len(e.curGroupKey) == 0 {
matched, firstGroup = false, true
}
for i, item := range e.GroupByItems {
v, err := item.Eval(row)
if err != nil {
return false, errors.Trace(err)
}
if matched {
c, err := v.CompareDatum(e.StmtCtx, &e.curGroupKey[i])
if err != nil {
return false, errors.Trace(err)
}
matched = c == 0
}
e.tmpGroupKey = append(e.tmpGroupKey, v)
}
if matched {
return false, nil
}
e.curGroupKey = e.curGroupKey[:0]
for _, v := range e.tmpGroupKey {
e.curGroupKey = append(e.curGroupKey, *((&v).Copy()))
}
return !firstGroup, nil
}