-
Notifications
You must be signed in to change notification settings - Fork 5.8k
/
aggregate.go
1921 lines (1784 loc) · 58.1 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
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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,
// 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.
package executor
import (
"bytes"
"context"
"fmt"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/executor/aggfuncs"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/set"
"github.com/twmb/murmur3"
"go.uber.org/zap"
)
type aggPartialResultMapper map[string][]aggfuncs.PartialResult
// baseHashAggWorker stores the common attributes of HashAggFinalWorker and HashAggPartialWorker.
// nolint:structcheck
type baseHashAggWorker struct {
ctx sessionctx.Context
finishCh <-chan struct{}
aggFuncs []aggfuncs.AggFunc
maxChunkSize int
stats *AggWorkerStat
memTracker *memory.Tracker
BInMap int // indicate there are 2^BInMap buckets in Golang Map.
}
func newBaseHashAggWorker(ctx sessionctx.Context, finishCh <-chan struct{}, aggFuncs []aggfuncs.AggFunc,
maxChunkSize int, memTrack *memory.Tracker) baseHashAggWorker {
baseWorker := baseHashAggWorker{
ctx: ctx,
finishCh: finishCh,
aggFuncs: aggFuncs,
maxChunkSize: maxChunkSize,
memTracker: memTrack,
BInMap: 0,
}
return baseWorker
}
// HashAggPartialWorker indicates the partial workers of parallel hash agg execution,
// the number of the worker can be set by `tidb_hashagg_partial_concurrency`.
type HashAggPartialWorker struct {
baseHashAggWorker
inputCh chan *chunk.Chunk
outputChs []chan *HashAggIntermData
globalOutputCh chan *AfFinalResult
giveBackCh chan<- *HashAggInput
partialResultsMap aggPartialResultMapper
groupByItems []expression.Expression
groupKey [][]byte
// chk stores the input data from child,
// and is reused by childExec and partial worker.
chk *chunk.Chunk
}
// HashAggFinalWorker indicates the final workers of parallel hash agg execution,
// the number of the worker can be set by `tidb_hashagg_final_concurrency`.
type HashAggFinalWorker struct {
baseHashAggWorker
rowBuffer []types.Datum
mutableRow chunk.MutRow
partialResultMap aggPartialResultMapper
groupSet set.StringSetWithMemoryUsage
inputCh chan *HashAggIntermData
outputCh chan *AfFinalResult
finalResultHolderCh chan *chunk.Chunk
groupKeys [][]byte
}
// AfFinalResult indicates aggregation functions final result.
type AfFinalResult struct {
chk *chunk.Chunk
err error
giveBackCh chan *chunk.Chunk
}
// 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 PartialAggFuncs.
// The parallel execution flow is as the following graph shows:
//
// +-------------+
// | Main Thread |
// +------+------+
// ^
// |
// +
// +-+- +-+
// | | ...... | | finalOutputCh
// +++- +-+
// ^
// |
// +---------------+
// | |
// +--------------+ +--------------+
// | final worker | ...... | final worker |
// +------------+-+ +-+------------+
// ^ ^
// | |
// +-+ +-+ ...... +-+
// | | | | | |
// ... ... ... partialOutputChs
// | | | | | |
// +++ +++ +++
// ^ ^ ^
// +-+ | | |
// | | +--------o----+ |
// inputCh +-+ | +-----------------+---+
// | | | |
// ... +---+------------+ +----+-----------+
// | | | partial worker | ...... | partial worker |
// +++ +--------------+-+ +-+--------------+
// | ^ ^
// | | |
// +----v---------+ +++ +-+ +++
// | data fetcher | +------> | | | | ...... | | partialInputChs
// +--------------+ +-+ +-+ +-+
type HashAggExec struct {
baseExecutor
sc *stmtctx.StatementContext
PartialAggFuncs []aggfuncs.AggFunc
FinalAggFuncs []aggfuncs.AggFunc
partialResultMap aggPartialResultMapper
bInMap int64 // indicate there are 2^bInMap buckets in partialResultMap
groupSet set.StringSetWithMemoryUsage
groupKeys []string
cursor4GroupKey int
GroupByItems []expression.Expression
groupKeyBuffer [][]byte
finishCh chan struct{}
finalOutputCh chan *AfFinalResult
partialOutputChs []chan *HashAggIntermData
inputCh chan *HashAggInput
partialInputChs []chan *chunk.Chunk
partialWorkers []HashAggPartialWorker
finalWorkers []HashAggFinalWorker
defaultVal *chunk.Chunk
childResult *chunk.Chunk
// isChildReturnEmpty indicates whether the child executor only returns an empty input.
isChildReturnEmpty bool
// After we support parallel execution for aggregation functions with distinct,
// we can remove this attribute.
isUnparallelExec bool
parallelExecInitialized bool
prepared bool
executed bool
memTracker *memory.Tracker // track memory usage.
diskTracker *disk.Tracker
stats *HashAggRuntimeStats
// listInDisk is the chunks to store row values for spilled data.
// The HashAggExec may be set to `spill mode` multiple times, and all spilled data will be appended to ListInDisk.
listInDisk *chunk.ListInDisk
// numOfSpilledChks indicates the number of all the spilled chunks.
numOfSpilledChks int
// offsetOfSpilledChks indicates the offset of the chunk be read from the disk.
// In each round of processing, we need to re-fetch all the chunks spilled in the last one.
offsetOfSpilledChks int
// inSpillMode indicates whether HashAgg is in `spill mode`.
// When HashAgg is in `spill mode`, the size of `partialResultMap` is no longer growing and all the data fetched
// from the child executor is spilled to the disk.
inSpillMode uint32
// tmpChkForSpill is the temp chunk for spilling.
tmpChkForSpill *chunk.Chunk
// spillAction save the Action for spilling.
spillAction *AggSpillDiskAction
// isChildDrained indicates whether the all data from child has been taken out.
isChildDrained bool
}
// HashAggInput indicates the input of hash agg exec.
type HashAggInput struct {
chk *chunk.Chunk
// giveBackCh is bound with specific partial worker,
// it's used to reuse the `chk`,
// and tell the data-fetcher which partial worker it should send data to.
giveBackCh chan<- *chunk.Chunk
}
// HashAggIntermData indicates the intermediate data of aggregation execution.
type HashAggIntermData struct {
groupKeys []string
cursor int
partialResultMap aggPartialResultMapper
}
// getPartialResultBatch fetches a batch of partial results from HashAggIntermData.
func (d *HashAggIntermData) getPartialResultBatch(sc *stmtctx.StatementContext, prs [][]aggfuncs.PartialResult, aggFuncs []aggfuncs.AggFunc, maxChunkSize int) (_ [][]aggfuncs.PartialResult, groupKeys []string, reachEnd bool) {
keyStart := d.cursor
for ; d.cursor < len(d.groupKeys) && len(prs) < maxChunkSize; d.cursor++ {
prs = append(prs, d.partialResultMap[d.groupKeys[d.cursor]])
}
if d.cursor == len(d.groupKeys) {
reachEnd = true
}
return prs, d.groupKeys[keyStart:d.cursor], reachEnd
}
// Close implements the Executor Close interface.
func (e *HashAggExec) Close() error {
if e.isUnparallelExec {
var firstErr error
e.childResult = nil
e.groupSet, _ = set.NewStringSetWithMemoryUsage()
e.partialResultMap = nil
if e.memTracker != nil {
e.memTracker.ReplaceBytesUsed(0)
}
if e.listInDisk != nil {
firstErr = e.listInDisk.Close()
}
e.spillAction, e.tmpChkForSpill = nil, nil
if err := e.baseExecutor.Close(); firstErr == nil {
firstErr = err
}
return firstErr
}
if e.parallelExecInitialized {
// `Close` may be called after `Open` without calling `Next` in test.
if !e.prepared {
close(e.inputCh)
for _, ch := range e.partialOutputChs {
close(ch)
}
for _, ch := range e.partialInputChs {
close(ch)
}
close(e.finalOutputCh)
}
close(e.finishCh)
for _, ch := range e.partialOutputChs {
for range ch {
}
}
for _, ch := range e.partialInputChs {
for range ch {
}
}
for range e.finalOutputCh {
}
e.executed = false
if e.memTracker != nil {
e.memTracker.ReplaceBytesUsed(0)
}
}
return e.baseExecutor.Close()
}
// Open implements the Executor Open interface.
func (e *HashAggExec) Open(ctx context.Context) error {
failpoint.Inject("mockHashAggExecBaseExecutorOpenReturnedError", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.New("mock HashAggExec.baseExecutor.Open returned error"))
}
})
if err := e.baseExecutor.Open(ctx); err != nil {
return err
}
// If panic here, the children executor should be closed because they are open.
defer closeBaseExecutor(&e.baseExecutor)
e.prepared = false
e.memTracker = memory.NewTracker(e.id, -1)
if e.ctx.GetSessionVars().TrackAggregateMemoryUsage {
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
}
if e.isUnparallelExec {
e.initForUnparallelExec()
return nil
}
e.initForParallelExec(e.ctx)
return nil
}
func (e *HashAggExec) initForUnparallelExec() {
var setSize int64
e.groupSet, setSize = set.NewStringSetWithMemoryUsage()
e.partialResultMap = make(aggPartialResultMapper)
e.bInMap = 0
failpoint.Inject("ConsumeRandomPanic", nil)
e.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice*(1<<e.bInMap) + setSize)
e.groupKeyBuffer = make([][]byte, 0, 8)
e.childResult = newFirstChunk(e.children[0])
e.memTracker.Consume(e.childResult.MemoryUsage())
e.offsetOfSpilledChks, e.numOfSpilledChks = 0, 0
e.executed, e.isChildDrained = false, false
e.listInDisk = chunk.NewListInDisk(retTypes(e.children[0]))
e.tmpChkForSpill = newFirstChunk(e.children[0])
if e.ctx.GetSessionVars().TrackAggregateMemoryUsage && config.GetGlobalConfig().OOMUseTmpStorage {
e.diskTracker = disk.NewTracker(e.id, -1)
e.diskTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.DiskTracker)
e.listInDisk.GetDiskTracker().AttachTo(e.diskTracker)
e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewActionForSoftLimit(e.ActionSpill())
}
}
func closeBaseExecutor(b *baseExecutor) {
if r := recover(); r != nil {
// Release the resource, but throw the panic again and let the top level handle it.
terror.Log(b.Close())
logutil.BgLogger().Warn("panic in Open(), close base executor and throw exception again")
panic(r)
}
}
func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) {
sessionVars := e.ctx.GetSessionVars()
finalConcurrency := sessionVars.HashAggFinalConcurrency()
partialConcurrency := sessionVars.HashAggPartialConcurrency()
e.isChildReturnEmpty = true
e.finalOutputCh = make(chan *AfFinalResult, finalConcurrency+partialConcurrency+1)
e.inputCh = make(chan *HashAggInput, partialConcurrency)
e.finishCh = make(chan struct{}, 1)
e.partialInputChs = make([]chan *chunk.Chunk, partialConcurrency)
for i := range e.partialInputChs {
e.partialInputChs[i] = make(chan *chunk.Chunk, 1)
}
e.partialOutputChs = make([]chan *HashAggIntermData, finalConcurrency)
for i := range e.partialOutputChs {
e.partialOutputChs[i] = make(chan *HashAggIntermData, partialConcurrency)
}
e.partialWorkers = make([]HashAggPartialWorker, partialConcurrency)
e.finalWorkers = make([]HashAggFinalWorker, finalConcurrency)
e.initRuntimeStats()
// Init partial workers.
for i := 0; i < partialConcurrency; i++ {
w := HashAggPartialWorker{
baseHashAggWorker: newBaseHashAggWorker(e.ctx, e.finishCh, e.PartialAggFuncs, e.maxChunkSize, e.memTracker),
inputCh: e.partialInputChs[i],
outputChs: e.partialOutputChs,
giveBackCh: e.inputCh,
globalOutputCh: e.finalOutputCh,
partialResultsMap: make(aggPartialResultMapper),
groupByItems: e.GroupByItems,
chk: newFirstChunk(e.children[0]),
groupKey: make([][]byte, 0, 8),
}
// There is a bucket in the empty partialResultsMap.
failpoint.Inject("ConsumeRandomPanic", nil)
e.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice * (1 << w.BInMap))
if e.stats != nil {
w.stats = &AggWorkerStat{}
e.stats.PartialStats = append(e.stats.PartialStats, w.stats)
}
e.memTracker.Consume(w.chk.MemoryUsage())
e.partialWorkers[i] = w
input := &HashAggInput{
chk: newFirstChunk(e.children[0]),
giveBackCh: w.inputCh,
}
e.memTracker.Consume(input.chk.MemoryUsage())
e.inputCh <- input
}
// Init final workers.
for i := 0; i < finalConcurrency; i++ {
groupSet, setSize := set.NewStringSetWithMemoryUsage()
w := HashAggFinalWorker{
baseHashAggWorker: newBaseHashAggWorker(e.ctx, e.finishCh, e.FinalAggFuncs, e.maxChunkSize, e.memTracker),
partialResultMap: make(aggPartialResultMapper),
groupSet: groupSet,
inputCh: e.partialOutputChs[i],
outputCh: e.finalOutputCh,
finalResultHolderCh: make(chan *chunk.Chunk, 1),
rowBuffer: make([]types.Datum, 0, e.Schema().Len()),
mutableRow: chunk.MutRowFromTypes(retTypes(e)),
groupKeys: make([][]byte, 0, 8),
}
// There is a bucket in the empty partialResultsMap.
e.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice*(1<<w.BInMap) + setSize)
if e.stats != nil {
w.stats = &AggWorkerStat{}
e.stats.FinalStats = append(e.stats.FinalStats, w.stats)
}
e.finalWorkers[i] = w
e.finalWorkers[i].finalResultHolderCh <- newFirstChunk(e)
}
e.parallelExecInitialized = true
}
func (w *HashAggPartialWorker) getChildInput() bool {
select {
case <-w.finishCh:
return false
case chk, ok := <-w.inputCh:
if !ok {
return false
}
w.chk.SwapColumns(chk)
w.giveBackCh <- &HashAggInput{
chk: chk,
giveBackCh: w.inputCh,
}
}
return true
}
func recoveryHashAgg(output chan *AfFinalResult, r interface{}) {
err := errors.Errorf("%v", r)
output <- &AfFinalResult{err: errors.Errorf("%v", r)}
logutil.BgLogger().Error("parallel hash aggregation panicked", zap.Error(err), zap.Stack("stack"))
}
func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) {
start := time.Now()
needShuffle, sc := false, ctx.GetSessionVars().StmtCtx
defer func() {
if r := recover(); r != nil {
recoveryHashAgg(w.globalOutputCh, r)
}
if needShuffle {
w.shuffleIntermData(sc, finalConcurrency)
}
w.memTracker.Consume(-w.chk.MemoryUsage())
if w.stats != nil {
w.stats.WorkerTime += int64(time.Since(start))
}
waitGroup.Done()
}()
for {
waitStart := time.Now()
ok := w.getChildInput()
if w.stats != nil {
w.stats.WaitTime += int64(time.Since(waitStart))
}
if !ok {
return
}
execStart := time.Now()
if err := w.updatePartialResult(ctx, sc, w.chk, len(w.partialResultsMap)); err != nil {
w.globalOutputCh <- &AfFinalResult{err: err}
return
}
if w.stats != nil {
w.stats.ExecTime += int64(time.Since(execStart))
w.stats.TaskNum += 1
}
// The intermData can be promised to be not empty if reaching here,
// so we set needShuffle to be true.
needShuffle = true
}
}
func getGroupKeyMemUsage(groupKey [][]byte) int64 {
mem := int64(0)
for _, key := range groupKey {
mem += int64(cap(key))
}
mem += aggfuncs.DefSliceSize * int64(cap(groupKey))
return mem
}
func (w *HashAggPartialWorker) updatePartialResult(ctx sessionctx.Context, sc *stmtctx.StatementContext, chk *chunk.Chunk, finalConcurrency int) (err error) {
memSize := getGroupKeyMemUsage(w.groupKey)
w.groupKey, err = getGroupKey(w.ctx, chk, w.groupKey, w.groupByItems)
failpoint.Inject("ConsumeRandomPanic", nil)
w.memTracker.Consume(getGroupKeyMemUsage(w.groupKey) - memSize)
if err != nil {
return err
}
partialResults := w.getPartialResult(sc, w.groupKey, w.partialResultsMap)
numRows := chk.NumRows()
rows := make([]chunk.Row, 1)
allMemDelta := int64(0)
for i := 0; i < numRows; i++ {
for j, af := range w.aggFuncs {
rows[0] = chk.GetRow(i)
memDelta, err := af.UpdatePartialResult(ctx, rows, partialResults[i][j])
if err != nil {
return err
}
allMemDelta += memDelta
}
}
w.memTracker.Consume(allMemDelta)
return nil
}
// shuffleIntermData shuffles the intermediate data of partial workers to corresponded final workers.
// We only support parallel execution for single-machine, so process of encode and decode can be skipped.
func (w *HashAggPartialWorker) shuffleIntermData(sc *stmtctx.StatementContext, finalConcurrency int) {
groupKeysSlice := make([][]string, finalConcurrency)
for groupKey := range w.partialResultsMap {
finalWorkerIdx := int(murmur3.Sum32([]byte(groupKey))) % finalConcurrency
if groupKeysSlice[finalWorkerIdx] == nil {
groupKeysSlice[finalWorkerIdx] = make([]string, 0, len(w.partialResultsMap)/finalConcurrency)
}
groupKeysSlice[finalWorkerIdx] = append(groupKeysSlice[finalWorkerIdx], groupKey)
}
for i := range groupKeysSlice {
if groupKeysSlice[i] == nil {
continue
}
w.outputChs[i] <- &HashAggIntermData{
groupKeys: groupKeysSlice[i],
partialResultMap: w.partialResultsMap,
}
}
}
// getGroupKey evaluates the group items and args of aggregate functions.
func getGroupKey(ctx sessionctx.Context, input *chunk.Chunk, groupKey [][]byte, groupByItems []expression.Expression) ([][]byte, error) {
numRows := input.NumRows()
avlGroupKeyLen := mathutil.Min(len(groupKey), numRows)
for i := 0; i < avlGroupKeyLen; i++ {
groupKey[i] = groupKey[i][:0]
}
for i := avlGroupKeyLen; i < numRows; i++ {
groupKey = append(groupKey, make([]byte, 0, 10*len(groupByItems)))
}
for _, item := range groupByItems {
tp := item.GetType()
buf, err := expression.GetColumn(tp.EvalType(), numRows)
if err != nil {
return nil, err
}
if err := expression.EvalExpr(ctx, item, tp.EvalType(), input, buf); err != nil {
expression.PutColumn(buf)
return nil, err
}
// This check is used to avoid error during the execution of `EncodeDecimal`.
if item.GetType().GetType() == mysql.TypeNewDecimal {
newTp := *tp
newTp.SetFlen(0)
tp = &newTp
}
groupKey, err = codec.HashGroupKey(ctx.GetSessionVars().StmtCtx, input.NumRows(), buf, groupKey, tp)
if err != nil {
expression.PutColumn(buf)
return nil, err
}
expression.PutColumn(buf)
}
return groupKey, nil
}
func (w *baseHashAggWorker) getPartialResult(sc *stmtctx.StatementContext, groupKey [][]byte, mapper aggPartialResultMapper) [][]aggfuncs.PartialResult {
n := len(groupKey)
partialResults := make([][]aggfuncs.PartialResult, n)
allMemDelta := int64(0)
for i := 0; i < n; i++ {
var ok bool
if partialResults[i], ok = mapper[string(groupKey[i])]; ok {
continue
}
for _, af := range w.aggFuncs {
partialResult, memDelta := af.AllocPartialResult()
partialResults[i] = append(partialResults[i], partialResult)
allMemDelta += memDelta + 8 // the memory usage of PartialResult
}
mapper[string(groupKey[i])] = partialResults[i]
allMemDelta += int64(len(groupKey[i]))
// Map will expand when count > bucketNum * loadFactor. The memory usage will double.
if len(mapper) > (1<<w.BInMap)*hack.LoadFactorNum/hack.LoadFactorDen {
w.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice * (1 << w.BInMap))
w.BInMap++
}
}
failpoint.Inject("ConsumeRandomPanic", nil)
w.memTracker.Consume(allMemDelta)
return partialResults
}
func (w *HashAggFinalWorker) getPartialInput() (input *HashAggIntermData, ok bool) {
select {
case <-w.finishCh:
return nil, false
case input, ok = <-w.inputCh:
if !ok {
return nil, false
}
}
return
}
func (w *HashAggFinalWorker) consumeIntermData(sctx sessionctx.Context) (err error) {
var (
input *HashAggIntermData
ok bool
intermDataBuffer [][]aggfuncs.PartialResult
groupKeys []string
sc = sctx.GetSessionVars().StmtCtx
)
for {
waitStart := time.Now()
input, ok = w.getPartialInput()
if w.stats != nil {
w.stats.WaitTime += int64(time.Since(waitStart))
}
if !ok {
return nil
}
execStart := time.Now()
if intermDataBuffer == nil {
intermDataBuffer = make([][]aggfuncs.PartialResult, 0, w.maxChunkSize)
}
// Consume input in batches, size of every batch is less than w.maxChunkSize.
for reachEnd := false; !reachEnd; {
intermDataBuffer, groupKeys, reachEnd = input.getPartialResultBatch(sc, intermDataBuffer[:0], w.aggFuncs, w.maxChunkSize)
groupKeysLen := len(groupKeys)
memSize := getGroupKeyMemUsage(w.groupKeys)
w.groupKeys = w.groupKeys[:0]
for i := 0; i < groupKeysLen; i++ {
w.groupKeys = append(w.groupKeys, []byte(groupKeys[i]))
}
failpoint.Inject("ConsumeRandomPanic", nil)
w.memTracker.Consume(getGroupKeyMemUsage(w.groupKeys) - memSize)
finalPartialResults := w.getPartialResult(sc, w.groupKeys, w.partialResultMap)
allMemDelta := int64(0)
for i, groupKey := range groupKeys {
if !w.groupSet.Exist(groupKey) {
allMemDelta += w.groupSet.Insert(groupKey)
}
prs := intermDataBuffer[i]
for j, af := range w.aggFuncs {
memDelta, err := af.MergePartialResult(sctx, prs[j], finalPartialResults[i][j])
if err != nil {
return err
}
allMemDelta += memDelta
}
}
w.memTracker.Consume(allMemDelta)
}
if w.stats != nil {
w.stats.ExecTime += int64(time.Since(execStart))
w.stats.TaskNum += 1
}
}
}
func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) {
waitStart := time.Now()
result, finished := w.receiveFinalResultHolder()
if w.stats != nil {
w.stats.WaitTime += int64(time.Since(waitStart))
}
if finished {
return
}
execStart := time.Now()
memSize := getGroupKeyMemUsage(w.groupKeys)
w.groupKeys = w.groupKeys[:0]
for groupKey := range w.groupSet.StringSet {
w.groupKeys = append(w.groupKeys, []byte(groupKey))
}
failpoint.Inject("ConsumeRandomPanic", nil)
w.memTracker.Consume(getGroupKeyMemUsage(w.groupKeys) - memSize)
partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, w.groupKeys, w.partialResultMap)
for i := 0; i < len(w.groupSet.StringSet); i++ {
for j, af := range w.aggFuncs {
if err := af.AppendFinalResult2Chunk(sctx, partialResults[i][j], result); err != nil {
logutil.BgLogger().Error("HashAggFinalWorker failed to append final result to Chunk", zap.Error(err))
}
}
if len(w.aggFuncs) == 0 {
result.SetNumVirtualRows(result.NumRows() + 1)
}
if result.IsFull() {
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
result, finished = w.receiveFinalResultHolder()
if finished {
return
}
}
}
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
if w.stats != nil {
w.stats.ExecTime += int64(time.Since(execStart))
}
}
func (w *HashAggFinalWorker) receiveFinalResultHolder() (*chunk.Chunk, bool) {
select {
case <-w.finishCh:
return nil, true
case result, ok := <-w.finalResultHolderCh:
return result, !ok
}
}
func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup) {
start := time.Now()
defer func() {
if r := recover(); r != nil {
recoveryHashAgg(w.outputCh, r)
}
if w.stats != nil {
w.stats.WorkerTime += int64(time.Since(start))
}
waitGroup.Done()
}()
if err := w.consumeIntermData(ctx); err != nil {
w.outputCh <- &AfFinalResult{err: err}
}
w.getFinalResult(ctx)
}
// Next implements the Executor Next interface.
func (e *HashAggExec) Next(ctx context.Context, req *chunk.Chunk) error {
req.Reset()
if e.isUnparallelExec {
return e.unparallelExec(ctx, req)
}
return e.parallelExec(ctx, req)
}
func (e *HashAggExec) fetchChildData(ctx context.Context, waitGroup *sync.WaitGroup) {
var (
input *HashAggInput
chk *chunk.Chunk
ok bool
err error
)
defer func() {
if r := recover(); r != nil {
recoveryHashAgg(e.finalOutputCh, r)
}
for i := range e.partialInputChs {
close(e.partialInputChs[i])
}
waitGroup.Done()
}()
for {
select {
case <-e.finishCh:
return
case input, ok = <-e.inputCh:
if !ok {
return
}
chk = input.chk
}
mSize := chk.MemoryUsage()
err = Next(ctx, e.children[0], chk)
if err != nil {
e.finalOutputCh <- &AfFinalResult{err: err}
e.memTracker.Consume(-mSize)
return
}
if chk.NumRows() == 0 {
e.memTracker.Consume(-mSize)
return
}
failpoint.Inject("ConsumeRandomPanic", nil)
e.memTracker.Consume(chk.MemoryUsage() - mSize)
input.giveBackCh <- chk
}
}
func (e *HashAggExec) waitPartialWorkerAndCloseOutputChs(waitGroup *sync.WaitGroup) {
waitGroup.Wait()
close(e.inputCh)
for input := range e.inputCh {
e.memTracker.Consume(-input.chk.MemoryUsage())
}
for _, ch := range e.partialOutputChs {
close(ch)
}
}
func (e *HashAggExec) waitAllWorkersAndCloseFinalOutputCh(waitGroups ...*sync.WaitGroup) {
for _, waitGroup := range waitGroups {
waitGroup.Wait()
}
close(e.finalOutputCh)
}
func (e *HashAggExec) prepare4ParallelExec(ctx context.Context) {
fetchChildWorkerWaitGroup := &sync.WaitGroup{}
fetchChildWorkerWaitGroup.Add(1)
go e.fetchChildData(ctx, fetchChildWorkerWaitGroup)
partialWorkerWaitGroup := &sync.WaitGroup{}
partialWorkerWaitGroup.Add(len(e.partialWorkers))
partialStart := time.Now()
for i := range e.partialWorkers {
go e.partialWorkers[i].run(e.ctx, partialWorkerWaitGroup, len(e.finalWorkers))
}
go func() {
e.waitPartialWorkerAndCloseOutputChs(partialWorkerWaitGroup)
if e.stats != nil {
atomic.AddInt64(&e.stats.PartialWallTime, int64(time.Since(partialStart)))
}
}()
finalWorkerWaitGroup := &sync.WaitGroup{}
finalWorkerWaitGroup.Add(len(e.finalWorkers))
finalStart := time.Now()
for i := range e.finalWorkers {
go e.finalWorkers[i].run(e.ctx, finalWorkerWaitGroup)
}
go func() {
finalWorkerWaitGroup.Wait()
if e.stats != nil {
atomic.AddInt64(&e.stats.FinalWallTime, int64(time.Since(finalStart)))
}
}()
// All workers may send error message to e.finalOutputCh when they panic.
// And e.finalOutputCh should be closed after all goroutines gone.
go e.waitAllWorkersAndCloseFinalOutputCh(fetchChildWorkerWaitGroup, partialWorkerWaitGroup, finalWorkerWaitGroup)
}
// HashAggExec employs one input reader, M partial workers and N final workers to execute parallelly.
// The parallel execution flow is:
// 1. input reader reads data from child executor and send them to partial workers.
// 2. partial worker receives the input data, updates the partial results, and shuffle the partial results to the final workers.
// 3. final worker receives partial results from all the partial workers, evaluates the final results and sends the final results to the main thread.
func (e *HashAggExec) parallelExec(ctx context.Context, chk *chunk.Chunk) error {
if !e.prepared {
e.prepare4ParallelExec(ctx)
e.prepared = true
}
failpoint.Inject("parallelHashAggError", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.New("HashAggExec.parallelExec error"))
}
})
if e.executed {
return nil
}
for {
result, ok := <-e.finalOutputCh
if !ok {
e.executed = true
if e.isChildReturnEmpty && e.defaultVal != nil {
chk.Append(e.defaultVal, 0, 1)
}
return nil
}
if result.err != nil {
return result.err
}
chk.SwapColumns(result.chk)
result.chk.Reset()
result.giveBackCh <- result.chk
if chk.NumRows() > 0 {
e.isChildReturnEmpty = false
return nil
}
}
}
// unparallelExec executes hash aggregation algorithm in single thread.
func (e *HashAggExec) unparallelExec(ctx context.Context, chk *chunk.Chunk) error {
chk.Reset()
for {
if e.prepared {
// Since we return e.maxChunkSize rows every time, so we should not traverse
// `groupSet` because of its randomness.
for ; e.cursor4GroupKey < len(e.groupKeys); e.cursor4GroupKey++ {
partialResults := e.getPartialResults(e.groupKeys[e.cursor4GroupKey])
if len(e.PartialAggFuncs) == 0 {
chk.SetNumVirtualRows(chk.NumRows() + 1)
}
for i, af := range e.PartialAggFuncs {
if err := af.AppendFinalResult2Chunk(e.ctx, partialResults[i], chk); err != nil {
return err
}
}
if chk.IsFull() {
e.cursor4GroupKey++
return nil
}
}
e.resetSpillMode()
}
if e.executed {
return nil
}
if err := e.execute(ctx); err != nil {
return err
}
if (len(e.groupSet.StringSet) == 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.memTracker.Consume(e.groupSet.Insert(""))
e.groupKeys = append(e.groupKeys, "")
}
e.prepared = true
}
}
func (e *HashAggExec) resetSpillMode() {
e.cursor4GroupKey, e.groupKeys = 0, e.groupKeys[:0]
var setSize int64
e.groupSet, setSize = set.NewStringSetWithMemoryUsage()
e.partialResultMap = make(aggPartialResultMapper)
e.bInMap = 0
e.prepared = false
e.executed = e.numOfSpilledChks == e.listInDisk.NumChunks() // No data is spilling again, all data have been processed.
e.numOfSpilledChks = e.listInDisk.NumChunks()
e.memTracker.ReplaceBytesUsed(setSize)
atomic.StoreUint32(&e.inSpillMode, 0)
}
// execute fetches Chunks from src and update each aggregate function for each row in Chunk.
func (e *HashAggExec) execute(ctx context.Context) (err error) {
defer func() {
if e.tmpChkForSpill.NumRows() > 0 && err == nil {
err = e.listInDisk.Add(e.tmpChkForSpill)
e.tmpChkForSpill.Reset()
}
}()
for {
mSize := e.childResult.MemoryUsage()
if err := e.getNextChunk(ctx); err != nil {
return err
}
failpoint.Inject("ConsumeRandomPanic", nil)
e.memTracker.Consume(e.childResult.MemoryUsage() - mSize)
if err != nil {
return err
}
failpoint.Inject("unparallelHashAggError", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.New("HashAggExec.unparallelExec error"))
}
})
// no more data.
if e.childResult.NumRows() == 0 {
return nil
}
e.groupKeyBuffer, err = getGroupKey(e.ctx, e.childResult, e.groupKeyBuffer, e.GroupByItems)
if err != nil {
return err
}
allMemDelta := int64(0)
sel := make([]int, 0, e.childResult.NumRows())
for j := 0; j < e.childResult.NumRows(); j++ {
groupKey := string(e.groupKeyBuffer[j]) // do memory copy here, because e.groupKeyBuffer may be reused.
if !e.groupSet.Exist(groupKey) {
if atomic.LoadUint32(&e.inSpillMode) == 1 && e.groupSet.Count() > 0 {
sel = append(sel, j)
continue
}
allMemDelta += e.groupSet.Insert(groupKey)