-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathhash_aggregator.go
558 lines (519 loc) · 22 KB
/
hash_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
// Copyright 2019 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 (
"context"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/colconv"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)
// hashAggregatorState represents the state of the hash aggregator operator.
type hashAggregatorState int
const (
// hashAggregatorBuffering is the state in which the hashAggregator reads
// the batches from the input and buffers them up. Once the number of
// buffered tuples reaches maxBuffered or the input has been fully exhausted,
// the hashAggregator transitions to hashAggregatorAggregating state.
hashAggregatorBuffering hashAggregatorState = iota
// hashAggregatorAggregating is the state in which the hashAggregator is
// performing the aggregation on the buffered tuples. If the input has been
// fully exhausted and the buffer is empty, the hashAggregator transitions
// to hashAggregatorOutputting state.
hashAggregatorAggregating
// hashAggregatorOutputting is the state in which the hashAggregator is
// writing its aggregation results to the output buffer.
hashAggregatorOutputting
// hashAggregatorDone is the state in which the hashAggregator has finished
// writing to the output buffer.
hashAggregatorDone
)
// hashAggregator is an operator that performs aggregation based on the
// specified grouping columns. This operator performs aggregation in online
// fashion. It reads from the input one batch at a time, groups all tuples into
// the equality chains, probes heads of those chains against already existing
// buckets and creates new buckets for new groups. After the input is
// exhausted, the operator begins to write the result into an output buffer.
// The output row ordering of this operator is arbitrary.
// Note that throughout this file "buckets" and "groups" mean the same thing
// and are used interchangeably.
type hashAggregator struct {
// Note that we don't use colexecop.OneInputInitCloserHelper here instead of
// the three options below because we need a custom behavior for Init() and
// Close().
colexecop.OneInputNode
colexecop.InitHelper
colexecop.CloserHelper
hashTableAllocator *colmem.Allocator
accountingHelper colmem.SetAccountingHelper
spec *execinfrapb.AggregatorSpec
aggHelper aggregatorHelper
inputTypes []*types.T
outputTypes []*types.T
inputArgsConverter *colconv.VecToDatumConverter
// maxBuffered determines the maximum number of tuples that are buffered up
// for aggregation at once.
maxBuffered int
bufferingState struct {
// tuples contains the tuples that we have buffered up for aggregation.
// Its length will not exceed maxBuffered.
tuples *colexecutils.AppendOnlyBufferedBatch
// pendingBatch stores the last read batch from the input that hasn't
// been fully processed yet.
pendingBatch coldata.Batch
// unprocessedIdx is the index of the first tuple in pendingBatch that
// hasn't been processed yet.
unprocessedIdx int
}
// numPreviouslyCreatedBuckets tracks the maximum number of buckets that
// have been created throughout the lifetime of this hashAggregator. This
// matters if the hashAggregator is reset - we reuse the same buckets on the
// next run.
// If non-zero, all buckets available to use are in
// buckets[len(buckets):numPreviouslyCreatedBuckets] range. Note that
// cap(buckets) might be higher than this number, but all buckets past
// numPreviouslyCreatedBuckets haven't been instantiated properly, so
// cap(buckets) should be ignored.
numPreviouslyCreatedBuckets int
// buckets contains all aggregation groups that we have so far. There is
// 1-to-1 mapping between buckets[i] and ht.Vals[i].
buckets []*aggBucket
// ht stores tuples that are "heads" of the corresponding aggregation
// groups ("head" here means the tuple that was first seen from the group).
ht *colexechash.HashTable
// state stores the current state of hashAggregator.
state hashAggregatorState
scratch struct {
// eqChains stores the chains of tuples from the current batch that are
// equal on the grouping columns (meaning that all tuples from the
// batch will be included into one of these chains). These chains must
// be set to zero length once the batch has been processed so that the
// memory could be reused.
eqChains [][]int
// intSlice and anotherIntSlice are simply scratch int slices that are
// reused for several purposes by the hashAggregator.
intSlice []int
anotherIntSlice []int
}
// inputTrackingState tracks all the input tuples which is needed in order
// to fallback to the external hash aggregator.
inputTrackingState struct {
tuples *colexecutils.SpillingQueue
zeroBatchEnqueued bool
}
// curOutputBucketIdx tracks the index in buckets to be flushed next when
// populating the output.
curOutputBucketIdx int
maxOutputBatchMemSize int64
// maxCapacity if non-zero indicates the target capacity of the output
// batch. It is set when, after setting a row, we realize that the output
// batch has exceeded the memory limit.
maxCapacity int
output coldata.Batch
aggFnsAlloc *colexecagg.AggregateFuncsAlloc
hashAlloc aggBucketAlloc
datumAlloc rowenc.DatumAlloc
toClose colexecop.Closers
}
var _ colexecop.ResettableOperator = &hashAggregator{}
var _ colexecop.BufferingInMemoryOperator = &hashAggregator{}
var _ colexecop.ClosableOperator = &hashAggregator{}
// hashAggregatorAllocSize determines the allocation size used by the hash
// aggregator's allocators. This number was chosen after running benchmarks of
// 'sum' aggregation on ints and decimals with varying group sizes (powers of 2
// from 1 to 4096).
const hashAggregatorAllocSize = 128
// NewHashAggregator creates a hash aggregator on the given grouping columns.
// The input specifications to this function are the same as that of the
// NewOrderedAggregator function.
// newSpillingQueueArgs - when non-nil - specifies the arguments to
// instantiate a SpillingQueue with which will be used to keep all of the
// input tuples in case the in-memory hash aggregator needs to fallback to
// the disk-backed operator. Pass in nil in order to not track all input
// tuples.
func NewHashAggregator(
args *colexecagg.NewAggregatorArgs,
newSpillingQueueArgs *colexecutils.NewSpillingQueueArgs,
outputUnlimitedAllocator *colmem.Allocator,
maxOutputBatchMemSize int64,
) (colexecop.ResettableOperator, error) {
aggFnsAlloc, inputArgsConverter, toClose, err := colexecagg.NewAggregateFuncsAlloc(
args, args.Spec.Aggregations, hashAggregatorAllocSize, colexecagg.HashAggKind,
)
// We want this number to be coldata.MaxBatchSize, but then we would lose
// some test coverage due to disabling of the randomization of the batch
// size, so we, instead, use 4 x coldata.BatchSize() (which ends up being
// coldata.MaxBatchSize in non-test environment).
maxBuffered := 4 * coldata.BatchSize()
if maxBuffered > coldata.MaxBatchSize {
// When randomizing coldata.BatchSize() in tests we might exceed
// coldata.MaxBatchSize, so we need to shrink it.
maxBuffered = coldata.MaxBatchSize
}
hashAgg := &hashAggregator{
OneInputNode: colexecop.NewOneInputNode(args.Input),
hashTableAllocator: args.Allocator,
spec: args.Spec,
state: hashAggregatorBuffering,
inputTypes: args.InputTypes,
outputTypes: args.OutputTypes,
inputArgsConverter: inputArgsConverter,
maxBuffered: maxBuffered,
toClose: toClose,
maxOutputBatchMemSize: maxOutputBatchMemSize,
aggFnsAlloc: aggFnsAlloc,
hashAlloc: aggBucketAlloc{allocator: args.Allocator},
}
hashAgg.accountingHelper.Init(outputUnlimitedAllocator, args.OutputTypes, nil /* notNeededVecIdxs */)
hashAgg.bufferingState.tuples = colexecutils.NewAppendOnlyBufferedBatch(args.Allocator, args.InputTypes, nil /* colsToStore */)
hashAgg.datumAlloc.AllocSize = hashAggregatorAllocSize
hashAgg.aggHelper = newAggregatorHelper(args, &hashAgg.datumAlloc, true /* isHashAgg */, hashAgg.maxBuffered)
if newSpillingQueueArgs != nil {
hashAgg.inputTrackingState.tuples = colexecutils.NewSpillingQueue(newSpillingQueueArgs)
}
return hashAgg, err
}
func (op *hashAggregator) Init(ctx context.Context) {
if !op.InitHelper.Init(ctx) {
return
}
op.Input.Init(op.Ctx)
// These numbers were chosen after running the micro-benchmarks and relevant
// TPCH queries using tpchvec/bench.
const hashTableLoadFactor = 0.1
const hashTableNumBuckets = 256
op.ht = colexechash.NewHashTable(
op.Ctx,
op.hashTableAllocator,
hashTableLoadFactor,
hashTableNumBuckets,
op.inputTypes,
op.spec.GroupCols,
true, /* allowNullEquality */
colexechash.HashTableDistinctBuildMode,
colexechash.HashTableDefaultProbeMode,
)
}
func (op *hashAggregator) Next() coldata.Batch {
for {
switch op.state {
case hashAggregatorBuffering:
if op.bufferingState.pendingBatch != nil && op.bufferingState.unprocessedIdx < op.bufferingState.pendingBatch.Length() {
op.bufferingState.tuples.AppendTuples(
op.bufferingState.pendingBatch, op.bufferingState.unprocessedIdx, op.bufferingState.pendingBatch.Length(),
)
}
op.bufferingState.pendingBatch, op.bufferingState.unprocessedIdx = op.Input.Next(), 0
n := op.bufferingState.pendingBatch.Length()
if op.inputTrackingState.tuples != nil {
op.inputTrackingState.tuples.Enqueue(op.Ctx, op.bufferingState.pendingBatch)
op.inputTrackingState.zeroBatchEnqueued = n == 0
}
if n == 0 {
// This is the last input batch.
if op.bufferingState.tuples.Length() == 0 {
// There are currently no buffered tuples to perform the
// aggregation on.
if len(op.buckets) == 0 {
// We don't have any buckets which means that there were
// no input tuples whatsoever, so we can transition to
// finished state right away.
op.state = hashAggregatorDone
} else {
// There are some buckets, so we proceed to the
// outputting state.
op.state = hashAggregatorOutputting
}
} else {
// There are some buffered tuples on which we need to run
// the aggregation.
op.state = hashAggregatorAggregating
}
continue
}
toBuffer := n
if op.bufferingState.tuples.Length()+toBuffer > op.maxBuffered {
toBuffer = op.maxBuffered - op.bufferingState.tuples.Length()
}
if toBuffer > 0 {
op.bufferingState.tuples.AppendTuples(op.bufferingState.pendingBatch, 0 /* startIdx */, toBuffer)
op.bufferingState.unprocessedIdx = toBuffer
}
if op.bufferingState.tuples.Length() == op.maxBuffered {
op.state = hashAggregatorAggregating
continue
}
case hashAggregatorAggregating:
op.inputArgsConverter.ConvertBatch(op.bufferingState.tuples)
op.onlineAgg(op.bufferingState.tuples)
if op.bufferingState.pendingBatch.Length() == 0 {
if len(op.buckets) == 0 {
op.state = hashAggregatorDone
} else {
op.state = hashAggregatorOutputting
}
continue
}
op.bufferingState.tuples.ResetInternalBatch()
op.state = hashAggregatorBuffering
case hashAggregatorOutputting:
// Note that ResetMaybeReallocate truncates the requested capacity
// at coldata.BatchSize(), so we can just try asking for
// len(op.buckets) capacity.
op.output, _ = op.accountingHelper.ResetMaybeReallocate(
op.outputTypes, op.output, len(op.buckets), op.maxOutputBatchMemSize,
)
curOutputIdx := 0
for curOutputIdx < op.output.Capacity() &&
op.curOutputBucketIdx < len(op.buckets) &&
(op.maxCapacity == 0 || curOutputIdx < op.maxCapacity) {
bucket := op.buckets[op.curOutputBucketIdx]
for fnIdx, fn := range bucket.fns {
fn.SetOutput(op.output.ColVec(fnIdx))
fn.Flush(curOutputIdx)
}
op.accountingHelper.AccountForSet(curOutputIdx)
curOutputIdx++
op.curOutputBucketIdx++
if op.maxCapacity == 0 && op.accountingHelper.Allocator.Used() >= op.maxOutputBatchMemSize {
op.maxCapacity = curOutputIdx
}
}
if op.curOutputBucketIdx >= len(op.buckets) {
op.state = hashAggregatorDone
}
op.output.SetLength(curOutputIdx)
return op.output
case hashAggregatorDone:
return coldata.ZeroBatch
default:
colexecerror.InternalError(errors.AssertionFailedf("hash aggregator in unhandled state"))
// This code is unreachable, but the compiler cannot infer that.
return nil
}
}
}
func (op *hashAggregator) setupScratchSlices(numBuffered int) {
if len(op.scratch.eqChains) < numBuffered {
op.scratch.eqChains = make([][]int, numBuffered)
op.scratch.intSlice = make([]int, numBuffered)
op.scratch.anotherIntSlice = make([]int, numBuffered)
}
}
// onlineAgg groups all tuples in b into equality chains, then probes the
// heads of those chains against already existing groups, aggregates matched
// chains into the corresponding buckets and creates new buckets for new
// aggregation groups.
//
// Let's go through an example of how this function works: our input stream
// contains the following tuples:
// {-3}, {-3}, {-2}, {-1}, {-4}, {-1}, {-1}, {-4}.
// (Note that negative values are chosen in order to visually distinguish them
// from the IDs that we'll be working with below.)
// We will use coldata.BatchSize() == 4 and let's assume that we will use a
// simple hash function h(i) = i % 2 with two buckets in the hash table.
//
// I. we get a batch [-3, -3, -2, -1].
// 1. a) compute hash buckets: ProbeScratch.next = [reserved, 1, 1, 0, 1]
// b) build 'next' chains between hash buckets:
// ProbeScratch.first = [3, 1] (length of first == # of hash buckets)
// ProbeScratch.next = [reserved, 2, 4, 0, 0]
// (Note that we have a hash collision in the bucket with hash 1.)
// c) find "equality" buckets (populate HeadID):
// ProbeScratch.HeadID = [1, 1, 3, 4]
// (This means that tuples at position 0 and 1 are the same, and the
// tuple at position HeadID-1 is the head of the equality chain.)
// 2. divide all tuples into the equality chains based on HeadID:
// eqChains[0] = [0, 1]
// eqChains[1] = [2]
// eqChains[2] = [3]
// The special "heads of equality chains" selection vector is [0, 2, 3].
// 3. we don't have any existing buckets yet, so this step is a noop.
// 4. each of the three equality chains contains tuples from a separate
// aggregation group, so we perform aggregation on each of them in turn.
// After we do so, we will have three buckets and the hash table will contain
// three tuples (with buckets and tuples corresponding to each other):
// buckets = [<bucket for -3>, <bucket for -2>, <bucket for -1>]
// ht.Vals = [-3, -2, -1].
// We have fully processed the first batch.
//
// II. we get a batch [-4, -1, -1, -4].
// 1. a) compute hash buckets: ProbeScratch.next = [reserved, 0, 1, 1, 0]
// b) build 'next' chains between hash buckets:
// ProbeScratch.first = [1, 2]
// ProbeScratch.next = [reserved, 4, 3, 0, 0]
// c) find "equality" buckets:
// ProbeScratch.HeadID = [1, 2, 2, 1]
// 2. divide all tuples into the equality chains based on HeadID:
// eqChains[0] = [0, 3]
// eqChains[1] = [1, 2]
// The special "heads of equality chains" selection vector is [0, 1].
// 3. probe that special "heads" selection vector against the tuples already
// present in the hash table:
// ProbeScratch.HeadID = [0, 3]
// Value 0 indicates that the first equality chain doesn't have an
// existing bucket, but the second chain does and the ID of its bucket is
// HeadID-1 = 2. We aggregate the second equality chain into that bucket.
// 4. the first equality chain contains tuples from a new aggregation group,
// so we create a new bucket for it and perform the aggregation.
// After we do so, we will have four buckets and the hash table will contain
// four tuples:
// buckets = [<bucket for -3>, <bucket for -2>, <bucket for -1>, <bucket for -4>]
// ht.Vals = [-3, -2, -1, -4].
// We have fully processed the second batch.
//
// We have processed the input fully, so we're ready to emit the output.
//
// NOTE: b *must* be non-zero length batch.
func (op *hashAggregator) onlineAgg(b coldata.Batch) {
op.setupScratchSlices(b.Length())
inputVecs := b.ColVecs()
// Step 1: find "equality" buckets: we compute the hash buckets for all
// tuples, build 'next' chains between them, and then find equality buckets
// for the tuples.
op.ht.ComputeHashAndBuildChains(b)
op.ht.FindBuckets(
b, op.ht.Keys, op.ht.ProbeScratch.First, op.ht.ProbeScratch.Next, op.ht.CheckProbeForDistinct,
)
// Step 2: now that we have op.ht.ProbeScratch.HeadID populated we can
// populate the equality chains.
eqChainsCount, eqChainsHeadsSel := op.populateEqChains(b)
b.SetLength(eqChainsCount)
// Make a copy of the selection vector that contains heads of the
// corresponding equality chains because the underlying memory will be
// modified below.
eqChainsHeads := op.scratch.intSlice[:eqChainsCount]
copy(eqChainsHeads, eqChainsHeadsSel)
// Step 3: if we have any existing buckets, we need to probe the heads of
// the equality chains (which the selection vector on b currently contains)
// against the heads of the existing groups.
if len(op.buckets) > 0 {
op.ht.FindBuckets(
b, op.ht.Keys, op.ht.BuildScratch.First, op.ht.BuildScratch.Next, op.ht.CheckBuildForAggregation,
)
for eqChainsSlot, HeadID := range op.ht.ProbeScratch.HeadID[:eqChainsCount] {
if HeadID != 0 {
// Tuples in this equality chain belong to an already existing
// group.
eqChain := op.scratch.eqChains[eqChainsSlot]
bucket := op.buckets[HeadID-1]
op.aggHelper.performAggregation(
op.Ctx, inputVecs, len(eqChain), eqChain, bucket, nil, /* groups */
)
// We have fully processed this equality chain, so we need to
// reset its length.
op.scratch.eqChains[eqChainsSlot] = op.scratch.eqChains[eqChainsSlot][:0]
}
}
}
// Step 4: now we go over all equality chains and check whether there are
// any that haven't been processed yet (they will be of non-zero length).
// If we find any, we'll create a new bucket for each.
newGroupsHeadsSel := op.scratch.anotherIntSlice[:0]
newGroupCount := 0
for eqChainSlot, eqChain := range op.scratch.eqChains[:eqChainsCount] {
if len(eqChain) > 0 {
// Tuples in this equality chain belong to a new aggregation group,
// so we'll use a new bucket and make sure that the head of this
// equality chain is appended to the hash table in the
// corresponding position.
var bucket *aggBucket
if nextBucketIdx := len(op.buckets); op.numPreviouslyCreatedBuckets > nextBucketIdx {
// We still have a bucket created on the previous run of the
// hash aggregator. Increase the length of op.buckets, using
// previously-allocated capacity, and then reset the bucket for
// reuse.
op.buckets = op.buckets[:nextBucketIdx+1]
bucket = op.buckets[nextBucketIdx]
bucket.reset()
} else {
// Need to allocate a new bucket.
bucket = op.hashAlloc.newAggBucket()
op.buckets = append(op.buckets, bucket)
// We know that all selected tuples belong to the same single
// group, so we can pass 'nil' for the 'groups' argument.
bucket.init(
op.aggFnsAlloc.MakeAggregateFuncs(), op.aggHelper.makeSeenMaps(), nil, /* groups */
)
}
op.aggHelper.performAggregation(
op.Ctx, inputVecs, len(eqChain), eqChain, bucket, nil, /* groups */
)
newGroupsHeadsSel = append(newGroupsHeadsSel, eqChainsHeads[eqChainSlot])
// We need to compact the hash buffer according to the new groups
// head tuples selection vector we're building.
op.ht.ProbeScratch.HashBuffer[newGroupCount] = op.ht.ProbeScratch.HashBuffer[eqChainSlot]
newGroupCount++
op.scratch.eqChains[eqChainSlot] = op.scratch.eqChains[eqChainSlot][:0]
}
}
if newGroupCount > 0 {
// We have created new buckets, so we need to append the heads of those
// buckets to the hash table.
copy(b.Selection(), newGroupsHeadsSel)
b.SetLength(newGroupCount)
op.ht.AppendAllDistinct(b)
}
}
func (op *hashAggregator) ExportBuffered(input colexecop.Operator) coldata.Batch {
if !op.inputTrackingState.zeroBatchEnqueued {
// Per the contract of the spilling queue, we need to append a
// zero-length batch.
op.inputTrackingState.tuples.Enqueue(op.Ctx, coldata.ZeroBatch)
op.inputTrackingState.zeroBatchEnqueued = true
}
batch, err := op.inputTrackingState.tuples.Dequeue(op.Ctx)
if err != nil {
colexecerror.InternalError(err)
}
return batch
}
func (op *hashAggregator) Reset(ctx context.Context) {
if r, ok := op.Input.(colexecop.Resetter); ok {
r.Reset(ctx)
}
op.bufferingState.tuples.ResetInternalBatch()
op.bufferingState.pendingBatch = nil
op.bufferingState.unprocessedIdx = 0
if op.numPreviouslyCreatedBuckets < len(op.buckets) {
op.numPreviouslyCreatedBuckets = len(op.buckets)
}
// Set up buckets for reuse.
op.buckets = op.buckets[:0]
op.ht.Reset(ctx)
if op.inputTrackingState.tuples != nil {
op.inputTrackingState.tuples.Reset(ctx)
op.inputTrackingState.zeroBatchEnqueued = false
}
op.curOutputBucketIdx = 0
op.state = hashAggregatorBuffering
}
func (op *hashAggregator) Close() error {
if !op.CloserHelper.Close() {
return nil
}
op.accountingHelper.Release()
var retErr error
if op.inputTrackingState.tuples != nil {
retErr = op.inputTrackingState.tuples.Close(op.EnsureCtx())
}
if err := op.toClose.Close(); err != nil {
retErr = err
}
return retErr
}