-
Notifications
You must be signed in to change notification settings - Fork 3.7k
/
aggregator.go
987 lines (888 loc) · 28.9 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
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
// Copyright 2016 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 rowexec
import (
"context"
"fmt"
"unsafe"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/stringarena"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/opentracing/opentracing-go"
)
type aggregateFuncs []tree.AggregateFunc
func (af aggregateFuncs) close(ctx context.Context) {
for _, f := range af {
f.Close(ctx)
}
}
// aggregatorBase is the foundation of 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 aggregation functions, as defined in the
// AggregatorSpec_Func enum.
//
// aggregatorBase's output schema is comprised of what is specified by the
// accompanying SELECT expressions.
type aggregatorBase struct {
execinfra.ProcessorBase
// runningState represents the state of the aggregator. This is in addition to
// ProcessorBase.State - the runningState is only relevant when
// ProcessorBase.State == StateRunning.
runningState aggregatorState
input execinfra.RowSource
inputDone bool
inputTypes []types.T
funcs []*aggregateFuncHolder
outputTypes []types.T
datumAlloc sqlbase.DatumAlloc
rowAlloc sqlbase.EncDatumRowAlloc
bucketsAcc mon.BoundAccount
aggFuncsAcc mon.BoundAccount
// isScalar can only be set if there are no groupCols, and it means that we
// will generate a result row even if there are no input rows. Used for
// queries like SELECT MAX(n) FROM t.
isScalar bool
groupCols []uint32
orderedGroupCols []uint32
aggregations []execinfrapb.AggregatorSpec_Aggregation
lastOrdGroupCols sqlbase.EncDatumRow
arena stringarena.Arena
row sqlbase.EncDatumRow
scratch []byte
cancelChecker *sqlbase.CancelChecker
}
// init initializes the aggregatorBase.
//
// trailingMetaCallback is passed as part of ProcStateOpts; the inputs to drain
// are in aggregatorBase.
func (ag *aggregatorBase) init(
self execinfra.RowSource,
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.AggregatorSpec,
input execinfra.RowSource,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
trailingMetaCallback func(context.Context) []execinfrapb.ProducerMetadata,
) error {
ctx := flowCtx.EvalCtx.Ctx()
memMonitor := execinfra.NewMonitor(ctx, flowCtx.EvalCtx.Mon, "aggregator-mem")
if sp := opentracing.SpanFromContext(ctx); sp != nil && tracing.IsRecording(sp) {
input = newInputStatCollector(input)
ag.FinishTrace = ag.outputStatsToTrace
}
ag.input = input
ag.isScalar = spec.IsScalar()
ag.groupCols = spec.GroupCols
ag.orderedGroupCols = spec.OrderedGroupCols
ag.aggregations = spec.Aggregations
ag.funcs = make([]*aggregateFuncHolder, len(spec.Aggregations))
ag.outputTypes = make([]types.T, len(spec.Aggregations))
ag.row = make(sqlbase.EncDatumRow, len(spec.Aggregations))
ag.bucketsAcc = memMonitor.MakeBoundAccount()
ag.arena = stringarena.Make(&ag.bucketsAcc)
ag.aggFuncsAcc = memMonitor.MakeBoundAccount()
// 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.
ag.inputTypes = input.OutputTypes()
for i, aggInfo := range spec.Aggregations {
if aggInfo.FilterColIdx != nil {
col := *aggInfo.FilterColIdx
if col >= uint32(len(ag.inputTypes)) {
return errors.Errorf("FilterColIdx out of range (%d)", col)
}
t := ag.inputTypes[col].Family()
if t != types.BoolFamily && t != types.UnknownFamily {
return errors.Errorf(
"filter column %d must be of boolean type, not %s", *aggInfo.FilterColIdx, t,
)
}
}
argTypes := make([]types.T, len(aggInfo.ColIdx)+len(aggInfo.Arguments))
for j, c := range aggInfo.ColIdx {
if c >= uint32(len(ag.inputTypes)) {
return errors.Errorf("ColIdx out of range (%d)", aggInfo.ColIdx)
}
argTypes[j] = ag.inputTypes[c]
}
arguments := make(tree.Datums, len(aggInfo.Arguments))
for j, argument := range aggInfo.Arguments {
h := execinfra.ExprHelper{}
// Pass nil types and row - there are no variables in these expressions.
if err := h.Init(argument, nil /* types */, flowCtx.EvalCtx); err != nil {
return errors.Wrapf(err, "%s", argument)
}
d, err := h.Eval(nil /* row */)
if err != nil {
return errors.Wrapf(err, "%s", argument)
}
argTypes[len(aggInfo.ColIdx)+j] = *d.ResolvedType()
if err != nil {
return errors.Wrapf(err, "%s", argument)
}
arguments[j] = d
}
aggConstructor, retType, err := execinfrapb.GetAggregateInfo(aggInfo.Func, argTypes...)
if err != nil {
return err
}
ag.funcs[i] = ag.newAggregateFuncHolder(aggConstructor, arguments)
if aggInfo.Distinct {
ag.funcs[i].seen = make(map[string]struct{})
}
ag.outputTypes[i] = *retType
}
return ag.ProcessorBase.Init(
self, post, ag.outputTypes, flowCtx, processorID, output, memMonitor,
execinfra.ProcStateOpts{
InputsToDrain: []execinfra.RowSource{ag.input},
TrailingMetaCallback: trailingMetaCallback,
},
)
}
var _ execinfrapb.DistSQLSpanStats = &AggregatorStats{}
const aggregatorTagPrefix = "aggregator."
// Stats implements the SpanStats interface.
func (as *AggregatorStats) Stats() map[string]string {
inputStatsMap := as.InputStats.Stats(aggregatorTagPrefix)
inputStatsMap[aggregatorTagPrefix+MaxMemoryTagSuffix] = humanizeutil.IBytes(as.MaxAllocatedMem)
return inputStatsMap
}
// StatsForQueryPlan implements the DistSQLSpanStats interface.
func (as *AggregatorStats) StatsForQueryPlan() []string {
stats := as.InputStats.StatsForQueryPlan("" /* prefix */)
if as.MaxAllocatedMem != 0 {
stats = append(stats,
fmt.Sprintf("%s: %s", MaxMemoryQueryPlanSuffix, humanizeutil.IBytes(as.MaxAllocatedMem)))
}
return stats
}
func (ag *aggregatorBase) outputStatsToTrace() {
is, ok := getInputStats(ag.FlowCtx, ag.input)
if !ok {
return
}
if sp := opentracing.SpanFromContext(ag.Ctx); sp != nil {
tracing.SetSpanStats(
sp,
&AggregatorStats{
InputStats: is,
MaxAllocatedMem: ag.MemMonitor.MaximumBytes(),
},
)
}
}
// ChildCount is part of the execinfra.OpNode interface.
func (ag *aggregatorBase) ChildCount(verbose bool) int {
if _, ok := ag.input.(execinfra.OpNode); ok {
return 1
}
return 0
}
// Child is part of the execinfra.OpNode interface.
func (ag *aggregatorBase) Child(nth int, verbose bool) execinfra.OpNode {
if nth == 0 {
if n, ok := ag.input.(execinfra.OpNode); ok {
return n
}
panic("input to aggregatorBase is not an execinfra.OpNode")
}
panic(fmt.Sprintf("invalid index %d", nth))
}
const (
// hashAggregatorBucketsInitialLen is a guess on how many "items" the
// 'buckets' map of hashAggregator has the capacity for initially.
hashAggregatorBucketsInitialLen = 8
// hashAggregatorSizeOfBucketsItem is a guess on how much space (in bytes)
// each item added to 'buckets' map of hashAggregator takes up in the map
// (i.e. it is memory internal to the map, orthogonal to "key-value" pair
// that we're adding to the map).
hashAggregatorSizeOfBucketsItem = 64
)
// hashAggregator is a specialization of aggregatorBase that must keep track of
// multiple grouping buckets at a time.
type hashAggregator struct {
aggregatorBase
// buckets is used during the accumulation phase to track the bucket keys
// that have been seen. After accumulation, the keys are extracted into
// bucketsIter for iteration.
buckets map[string]aggregateFuncs
bucketsIter []string
// bucketsLenGrowThreshold is the threshold which, when reached by the
// number of items in 'buckets', will trigger the update to memory
// accounting. It will start out at hashAggregatorBucketsInitialLen and
// then will be doubling in size.
bucketsLenGrowThreshold int
// alreadyAccountedFor tracks the number of items in 'buckets' memory for
// which we have already accounted for.
alreadyAccountedFor int
}
// orderedAggregator is a specialization of aggregatorBase that only needs to
// keep track of a single grouping bucket at a time.
type orderedAggregator struct {
aggregatorBase
// bucket is used during the accumulation phase to aggregate results.
bucket aggregateFuncs
}
var _ execinfra.Processor = &hashAggregator{}
var _ execinfra.RowSource = &hashAggregator{}
var _ execinfra.OpNode = &hashAggregator{}
const hashAggregatorProcName = "hash aggregator"
var _ execinfra.Processor = &orderedAggregator{}
var _ execinfra.RowSource = &orderedAggregator{}
var _ execinfra.OpNode = &orderedAggregator{}
const orderedAggregatorProcName = "ordered aggregator"
// aggregatorState represents the state of the processor.
type aggregatorState int
const (
aggStateUnknown aggregatorState = iota
// aggAccumulating means that rows are being read from the input and used to
// compute intermediary aggregation results.
aggAccumulating
// aggEmittingRows means that accumulation has finished and rows are being
// sent to the output.
aggEmittingRows
)
func newAggregator(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.AggregatorSpec,
input execinfra.RowSource,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
) (execinfra.Processor, error) {
if spec.IsRowCount() {
return newCountAggregator(flowCtx, processorID, input, post, output)
}
if len(spec.OrderedGroupCols) == len(spec.GroupCols) {
return newOrderedAggregator(flowCtx, processorID, spec, input, post, output)
}
ag := &hashAggregator{
buckets: make(map[string]aggregateFuncs),
bucketsLenGrowThreshold: hashAggregatorBucketsInitialLen,
}
if err := ag.init(
ag,
flowCtx,
processorID,
spec,
input,
post,
output,
func(context.Context) []execinfrapb.ProducerMetadata {
ag.close()
return nil
},
); err != nil {
return nil, err
}
// A new tree.EvalCtx was created during initializing aggregatorBase above
// and will be used only by this aggregator, so it is ok to update EvalCtx
// directly.
ag.EvalCtx.SingleDatumAggMemAccount = &ag.aggFuncsAcc
return ag, nil
}
func newOrderedAggregator(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.AggregatorSpec,
input execinfra.RowSource,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
) (*orderedAggregator, error) {
ag := &orderedAggregator{}
if err := ag.init(
ag,
flowCtx,
processorID,
spec,
input,
post,
output,
func(context.Context) []execinfrapb.ProducerMetadata {
ag.close()
return nil
},
); err != nil {
return nil, err
}
// A new tree.EvalCtx was created during initializing aggregatorBase above
// and will be used only by this aggregator, so it is ok to update EvalCtx
// directly.
ag.EvalCtx.SingleDatumAggMemAccount = &ag.aggFuncsAcc
return ag, nil
}
// Start is part of the RowSource interface.
func (ag *hashAggregator) Start(ctx context.Context) context.Context {
return ag.start(ctx, hashAggregatorProcName)
}
// Start is part of the RowSource interface.
func (ag *orderedAggregator) Start(ctx context.Context) context.Context {
return ag.start(ctx, orderedAggregatorProcName)
}
func (ag *aggregatorBase) start(ctx context.Context, procName string) context.Context {
ag.input.Start(ctx)
ctx = ag.StartInternal(ctx, procName)
ag.cancelChecker = sqlbase.NewCancelChecker(ctx)
ag.runningState = aggAccumulating
return ctx
}
func (ag *hashAggregator) close() {
if ag.InternalClose() {
log.VEventf(ag.Ctx, 2, "exiting aggregator")
// If we have started emitting rows, bucketsIter will represent which
// buckets are still open, since buckets are closed once their results are
// emitted.
if ag.bucketsIter == nil {
for _, bucket := range ag.buckets {
bucket.close(ag.Ctx)
}
} else {
for _, bucket := range ag.bucketsIter {
ag.buckets[bucket].close(ag.Ctx)
}
}
// Make sure to release any remaining memory under 'buckets'.
ag.buckets = nil
// Note that we should be closing accounts only after closing all the
// buckets since the latter might be releasing some precisely tracked
// memory, and if we were to close the accounts first, there would be
// no memory to release for the buckets.
ag.bucketsAcc.Close(ag.Ctx)
ag.aggFuncsAcc.Close(ag.Ctx)
ag.MemMonitor.Stop(ag.Ctx)
}
}
func (ag *orderedAggregator) close() {
if ag.InternalClose() {
log.VEventf(ag.Ctx, 2, "exiting aggregator")
if ag.bucket != nil {
ag.bucket.close(ag.Ctx)
}
// Note that we should be closing accounts only after closing the
// bucket since the latter might be releasing some precisely tracked
// memory, and if we were to close the accounts first, there would be
// no memory to release for the bucket.
ag.bucketsAcc.Close(ag.Ctx)
ag.aggFuncsAcc.Close(ag.Ctx)
ag.MemMonitor.Stop(ag.Ctx)
}
}
// matchLastOrdGroupCols takes a row and matches it with the row stored by
// lastOrdGroupCols. It returns true if the two rows are equal on the grouping
// columns, and false otherwise.
func (ag *aggregatorBase) matchLastOrdGroupCols(row sqlbase.EncDatumRow) (bool, error) {
for _, colIdx := range ag.orderedGroupCols {
res, err := ag.lastOrdGroupCols[colIdx].Compare(
&ag.inputTypes[colIdx], &ag.datumAlloc, ag.EvalCtx, &row[colIdx],
)
if res != 0 || err != nil {
return false, err
}
}
return true, nil
}
// accumulateRows continually reads rows from the input and accumulates them
// into intermediary aggregate results. If it encounters metadata, the metadata
// is immediately returned. Subsequent calls of this function will resume row
// accumulation.
func (ag *hashAggregator) accumulateRows() (
aggregatorState,
sqlbase.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
for {
row, meta := ag.input.Next()
if meta != nil {
if meta.Err != nil {
ag.MoveToDraining(nil /* err */)
return aggStateUnknown, nil, meta
}
return aggAccumulating, nil, meta
}
if row == nil {
log.VEvent(ag.Ctx, 1, "accumulation complete")
ag.inputDone = true
break
}
if ag.lastOrdGroupCols == nil {
ag.lastOrdGroupCols = ag.rowAlloc.CopyRow(row)
} else {
matched, err := ag.matchLastOrdGroupCols(row)
if err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
if !matched {
copy(ag.lastOrdGroupCols, row)
break
}
}
if err := ag.accumulateRow(row); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
}
// Queries like `SELECT MAX(n) FROM t` expect a row of NULLs if nothing was
// aggregated.
if len(ag.buckets) < 1 && len(ag.groupCols) == 0 {
bucket, err := ag.createAggregateFuncs()
if err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
ag.buckets[""] = bucket
}
// Note that, for simplicity, we're ignoring the overhead of the slice of
// strings.
if err := ag.bucketsAcc.Grow(ag.Ctx, int64(len(ag.buckets))*sizeOfString); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
ag.bucketsIter = make([]string, 0, len(ag.buckets))
for bucket := range ag.buckets {
ag.bucketsIter = append(ag.bucketsIter, bucket)
}
// Transition to aggEmittingRows, and let it generate the next row/meta.
return aggEmittingRows, nil, nil
}
// accumulateRows continually reads rows from the input and accumulates them
// into intermediary aggregate results. If it encounters metadata, the metadata
// is immediately returned. Subsequent calls of this function will resume row
// accumulation.
func (ag *orderedAggregator) accumulateRows() (
aggregatorState,
sqlbase.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
for {
row, meta := ag.input.Next()
if meta != nil {
if meta.Err != nil {
ag.MoveToDraining(nil /* err */)
return aggStateUnknown, nil, meta
}
return aggAccumulating, nil, meta
}
if row == nil {
log.VEvent(ag.Ctx, 1, "accumulation complete")
ag.inputDone = true
break
}
if ag.lastOrdGroupCols == nil {
ag.lastOrdGroupCols = ag.rowAlloc.CopyRow(row)
} else {
matched, err := ag.matchLastOrdGroupCols(row)
if err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
if !matched {
copy(ag.lastOrdGroupCols, row)
break
}
}
if err := ag.accumulateRow(row); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
}
// Queries like `SELECT MAX(n) FROM t` expect a row of NULLs if nothing was
// aggregated.
if ag.bucket == nil && ag.isScalar {
var err error
ag.bucket, err = ag.createAggregateFuncs()
if err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
}
// Transition to aggEmittingRows, and let it generate the next row/meta.
return aggEmittingRows, nil, nil
}
func (ag *aggregatorBase) getAggResults(
bucket aggregateFuncs,
) (aggregatorState, sqlbase.EncDatumRow, *execinfrapb.ProducerMetadata) {
for i, b := range bucket {
result, err := b.Result()
if err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
if result == nil {
// We can't encode nil into an EncDatum, so we represent it with DNull.
result = tree.DNull
}
ag.row[i] = sqlbase.DatumToEncDatum(&ag.outputTypes[i], result)
}
bucket.close(ag.Ctx)
if outRow := ag.ProcessRowHelper(ag.row); outRow != nil {
return aggEmittingRows, outRow, nil
}
// We might have switched to draining, we might not have. In case we
// haven't, aggEmittingRows is accurate. If we have, it will be ignored by
// the caller.
return aggEmittingRows, nil, nil
}
// emitRow constructs an output row from an accumulated bucket and returns it.
//
// emitRow() might move to stateDraining. It might also not return a row if the
// ProcOutputHelper filtered the current row out.
func (ag *hashAggregator) emitRow() (
aggregatorState,
sqlbase.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
if len(ag.bucketsIter) == 0 {
// We've exhausted all of the aggregation buckets.
if ag.inputDone {
// The input has been fully consumed. Transition to draining so that we
// emit any metadata that we've produced.
ag.MoveToDraining(nil /* err */)
return aggStateUnknown, nil, nil
}
// We've only consumed part of the input where the rows are equal over
// the columns specified by ag.orderedGroupCols, so we need to continue
// accumulating the remaining rows.
if err := ag.arena.UnsafeReset(ag.Ctx); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
// Before we create a new 'buckets' map below, we need to "release" the
// already accounted for memory of the current map.
ag.bucketsAcc.Shrink(ag.Ctx, int64(ag.alreadyAccountedFor)*hashAggregatorSizeOfBucketsItem)
// Note that, for simplicity, we're ignoring the overhead of the slice of
// strings.
ag.bucketsAcc.Shrink(ag.Ctx, int64(len(ag.buckets))*sizeOfString)
ag.bucketsIter = nil
ag.buckets = make(map[string]aggregateFuncs)
ag.bucketsLenGrowThreshold = hashAggregatorBucketsInitialLen
ag.alreadyAccountedFor = 0
for _, f := range ag.funcs {
if f.seen != nil {
f.seen = make(map[string]struct{})
}
}
if err := ag.accumulateRow(ag.lastOrdGroupCols); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
return aggAccumulating, nil, nil
}
bucket := ag.bucketsIter[0]
ag.bucketsIter = ag.bucketsIter[1:]
// Once we get the results from the bucket, we can delete it from the map.
// This will allow us to return the memory to the system before the hash
// aggregator is fully done (which matters when we have many buckets).
state, row, meta := ag.getAggResults(ag.buckets[bucket])
delete(ag.buckets, bucket)
return state, row, meta
}
// emitRow constructs an output row from an accumulated bucket and returns it.
//
// emitRow() might move to stateDraining. It might also not return a row if the
// ProcOutputHelper filtered a the current row out.
func (ag *orderedAggregator) emitRow() (
aggregatorState,
sqlbase.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
if ag.bucket == nil {
// We've exhausted all of the aggregation buckets.
if ag.inputDone {
// The input has been fully consumed. Transition to draining so that we
// emit any metadata that we've produced.
ag.MoveToDraining(nil /* err */)
return aggStateUnknown, nil, nil
}
// We've only consumed part of the input where the rows are equal over
// the columns specified by ag.orderedGroupCols, so we need to continue
// accumulating the remaining rows.
if err := ag.arena.UnsafeReset(ag.Ctx); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
for _, f := range ag.funcs {
if f.seen != nil {
f.seen = make(map[string]struct{})
}
}
if err := ag.accumulateRow(ag.lastOrdGroupCols); err != nil {
ag.MoveToDraining(err)
return aggStateUnknown, nil, nil
}
return aggAccumulating, nil, nil
}
bucket := ag.bucket
ag.bucket = nil
return ag.getAggResults(bucket)
}
// Next is part of the RowSource interface.
func (ag *hashAggregator) Next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadata) {
for ag.State == execinfra.StateRunning {
var row sqlbase.EncDatumRow
var meta *execinfrapb.ProducerMetadata
switch ag.runningState {
case aggAccumulating:
ag.runningState, row, meta = ag.accumulateRows()
case aggEmittingRows:
ag.runningState, row, meta = ag.emitRow()
default:
log.Fatalf(ag.Ctx, "unsupported state: %d", ag.runningState)
}
if row == nil && meta == nil {
continue
}
return row, meta
}
return nil, ag.DrainHelper()
}
// Next is part of the RowSource interface.
func (ag *orderedAggregator) Next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadata) {
for ag.State == execinfra.StateRunning {
var row sqlbase.EncDatumRow
var meta *execinfrapb.ProducerMetadata
switch ag.runningState {
case aggAccumulating:
ag.runningState, row, meta = ag.accumulateRows()
case aggEmittingRows:
ag.runningState, row, meta = ag.emitRow()
default:
log.Fatalf(ag.Ctx, "unsupported state: %d", ag.runningState)
}
if row == nil && meta == nil {
continue
}
return row, meta
}
return nil, ag.DrainHelper()
}
// ConsumerClosed is part of the RowSource interface.
func (ag *hashAggregator) ConsumerClosed() {
// The consumer is done, Next() will not be called again.
ag.close()
}
// ConsumerClosed is part of the RowSource interface.
func (ag *orderedAggregator) ConsumerClosed() {
// The consumer is done, Next() will not be called again.
ag.close()
}
func (ag *aggregatorBase) accumulateRowIntoBucket(
row sqlbase.EncDatumRow, groupKey []byte, bucket aggregateFuncs,
) error {
var err error
// Feed the func holders for this bucket the non-grouping datums.
for i, a := range ag.aggregations {
if a.FilterColIdx != nil {
col := *a.FilterColIdx
if err := row[col].EnsureDecoded(&ag.inputTypes[col], &ag.datumAlloc); err != nil {
return err
}
if row[*a.FilterColIdx].Datum != tree.DBoolTrue {
// This row doesn't contribute to this aggregation.
continue
}
}
// Extract the corresponding arguments from the row to feed into the
// aggregate function.
// Most functions require at most one argument thus we separate
// the first argument and allocation of (if applicable) a variadic
// collection of arguments thereafter.
var firstArg tree.Datum
var otherArgs tree.Datums
if len(a.ColIdx) > 1 {
otherArgs = make(tree.Datums, len(a.ColIdx)-1)
}
isFirstArg := true
for j, c := range a.ColIdx {
if err := row[c].EnsureDecoded(&ag.inputTypes[c], &ag.datumAlloc); err != nil {
return err
}
if isFirstArg {
firstArg = row[c].Datum
isFirstArg = false
continue
}
otherArgs[j-1] = row[c].Datum
}
canAdd := true
if a.Distinct {
canAdd, err = ag.funcs[i].isDistinct(
ag.Ctx,
&ag.datumAlloc,
groupKey,
firstArg,
otherArgs,
)
if err != nil {
return err
}
}
if !canAdd {
continue
}
if err := bucket[i].Add(ag.Ctx, firstArg, otherArgs...); err != nil {
return err
}
}
return nil
}
// accumulateRow accumulates a single row, returning an error if accumulation
// failed for any reason.
func (ag *hashAggregator) accumulateRow(row sqlbase.EncDatumRow) error {
if err := ag.cancelChecker.Check(); err != nil {
return err
}
// The encoding computed here determines which bucket the non-grouping
// datums are accumulated to.
encoded, err := ag.encode(ag.scratch, row)
if err != nil {
return err
}
ag.scratch = encoded[:0]
bucket, ok := ag.buckets[string(encoded)]
if !ok {
s, err := ag.arena.AllocBytes(ag.Ctx, encoded)
if err != nil {
return err
}
bucket, err = ag.createAggregateFuncs()
if err != nil {
return err
}
ag.buckets[s] = bucket
if len(ag.buckets) == ag.bucketsLenGrowThreshold {
toAccountFor := ag.bucketsLenGrowThreshold - ag.alreadyAccountedFor
if err := ag.bucketsAcc.Grow(ag.Ctx, int64(toAccountFor)*hashAggregatorSizeOfBucketsItem); err != nil {
return err
}
ag.alreadyAccountedFor = ag.bucketsLenGrowThreshold
ag.bucketsLenGrowThreshold *= 2
}
}
return ag.accumulateRowIntoBucket(row, encoded, bucket)
}
// accumulateRow accumulates a single row, returning an error if accumulation
// failed for any reason.
func (ag *orderedAggregator) accumulateRow(row sqlbase.EncDatumRow) error {
if err := ag.cancelChecker.Check(); err != nil {
return err
}
if ag.bucket == nil {
var err error
ag.bucket, err = ag.createAggregateFuncs()
if err != nil {
return err
}
}
return ag.accumulateRowIntoBucket(row, nil /* groupKey */, ag.bucket)
}
type aggregateFuncHolder struct {
create func(*tree.EvalContext, tree.Datums) tree.AggregateFunc
// arguments is the list of constant (non-aggregated) arguments to the
// aggregate, for instance, the separator in string_agg.
arguments tree.Datums
group *aggregatorBase
seen map[string]struct{}
arena *stringarena.Arena
}
const (
sizeOfString = int64(unsafe.Sizeof(""))
sizeOfAggregateFuncs = int64(unsafe.Sizeof(aggregateFuncs{}))
sizeOfAggregateFunc = int64(unsafe.Sizeof(tree.AggregateFunc(nil)))
)
func (ag *aggregatorBase) newAggregateFuncHolder(
create func(*tree.EvalContext, tree.Datums) tree.AggregateFunc, arguments tree.Datums,
) *aggregateFuncHolder {
return &aggregateFuncHolder{
create: create,
group: ag,
arena: &ag.arena,
arguments: arguments,
}
}
// isDistinct returns whether this aggregateFuncHolder has not already seen the
// encoding of grouping columns and argument columns. It should be used *only*
// when we have DISTINCT aggregation so that we can aggregate only the "first"
// row in the group.
func (a *aggregateFuncHolder) isDistinct(
ctx context.Context,
alloc *sqlbase.DatumAlloc,
prefix []byte,
firstArg tree.Datum,
otherArgs tree.Datums,
) (bool, error) {
// Allocate one EncDatum that will be reused when encoding every argument.
ed := sqlbase.EncDatum{Datum: firstArg}
encoded, err := ed.Fingerprint(firstArg.ResolvedType(), alloc, prefix)
if err != nil {
return false, err
}
if otherArgs != nil {
for _, arg := range otherArgs {
ed.Datum = arg
encoded, err = ed.Fingerprint(arg.ResolvedType(), alloc, encoded)
if err != nil {
return false, err
}
}
}
if _, ok := a.seen[string(encoded)]; ok {
// We have already seen a row with such combination of grouping and
// argument columns.
return false, nil
}
s, err := a.arena.AllocBytes(ctx, encoded)
if err != nil {
return false, err
}
a.seen[s] = struct{}{}
return true, nil
}
// 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 *aggregatorBase) encode(
appendTo []byte, row sqlbase.EncDatumRow,
) (encoding []byte, err error) {
for _, colIdx := range ag.groupCols {
appendTo, err = row[colIdx].Fingerprint(
&ag.inputTypes[colIdx], &ag.datumAlloc, appendTo)
if err != nil {
return appendTo, err
}
}
return appendTo, nil
}
func (ag *aggregatorBase) createAggregateFuncs() (aggregateFuncs, error) {
if err := ag.bucketsAcc.Grow(ag.Ctx, sizeOfAggregateFuncs+sizeOfAggregateFunc*int64(len(ag.funcs))); err != nil {
return nil, err
}
bucket := make(aggregateFuncs, len(ag.funcs))
for i, f := range ag.funcs {
agg := f.create(ag.EvalCtx, f.arguments)
if err := ag.bucketsAcc.Grow(ag.Ctx, agg.Size()); err != nil {
return nil, err
}
bucket[i] = agg
}
return bucket, nil
}