-
Notifications
You must be signed in to change notification settings - Fork 240
/
GpuAggregateExec.scala
2209 lines (1986 loc) · 92.6 KB
/
GpuAggregateExec.scala
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 (c) 2019-2024, NVIDIA CORPORATION.
*
* 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 com.nvidia.spark.rapids
import scala.annotation.tailrec
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import ai.rapids.cudf
import ai.rapids.cudf.{NvtxColor, NvtxRange}
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.GpuAggregateIterator.{computeAggregateAndClose, computeAggregateWithoutPreprocessAndClose, concatenateBatches}
import com.nvidia.spark.rapids.GpuMetric._
import com.nvidia.spark.rapids.GpuOverrides.pluginSupportedOrderableSig
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetry, withRetryNoSplit}
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
import com.nvidia.spark.rapids.shims.{AggregationTagging, ShimUnaryExecNode}
import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, If, NamedExpression, SortOrder}
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, HashPartitioning, Partitioning, UnspecifiedDistribution}
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.catalyst.util.truncatedString
import org.apache.spark.sql.execution.{ExplainUtils, SortExec, SparkPlan}
import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter}
import org.apache.spark.sql.rapids.execution.{GpuBatchSubPartitioner, GpuShuffleMeta, TrampolineUtil}
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
object AggregateUtils extends Logging {
private val aggs = List("min", "max", "avg", "sum", "count", "first", "last")
/**
* Return true if the Attribute passed is one of aggregates in the aggs list.
* Use it with caution. We are comparing the name of a column looking for anything that matches
* with the values in aggs.
*/
def validateAggregate(attributes: AttributeSet): Boolean = {
attributes.toSeq.exists(attr => aggs.exists(agg => attr.name.contains(agg)))
}
/**
* Return true if there are multiple distinct functions along with non-distinct functions.
*/
def shouldFallbackMultiDistinct(aggExprs: Seq[AggregateExpression]): Boolean = {
// Check if there is an `If` within `First`. This is included in the plan for non-distinct
// functions only when multiple distincts along with non-distinct functions are present in the
// query. We fall back to CPU in this case when references of `If` are an aggregate. We cannot
// call `isDistinct` here on aggregateExpressions to get the total number of distinct functions.
// If there are multiple distincts, the plan is rewritten by `RewriteDistinctAggregates` where
// regular aggregations and every distinct aggregation is calculated in a separate group.
aggExprs.map(e => e.aggregateFunction).exists {
func => {
func match {
case First(If(_, _, _), _) if validateAggregate(func.references) => true
case _ => false
}
}
}
}
/**
* Computes a target input batch size based on the assumption that computation can consume up to
* 4X the configured batch size.
* @param confTargetSize user-configured maximum desired batch size
* @param inputTypes input batch schema
* @param outputTypes output batch schema
* @param isReductionOnly true if this is a reduction-only aggregation without grouping
* @return maximum target batch size to keep computation under the 4X configured batch limit
*/
def computeTargetBatchSize(
confTargetSize: Long,
inputTypes: Seq[DataType],
outputTypes: Seq[DataType],
isReductionOnly: Boolean): Long = {
def typesToSize(types: Seq[DataType]): Long =
types.map(GpuBatchUtils.estimateGpuMemory(_, nullable = false, rowCount = 1)).sum
val inputRowSize = typesToSize(inputTypes)
val outputRowSize = typesToSize(outputTypes)
// The cudf hash table implementation allocates four 32-bit integers per input row.
val hashTableRowSize = 4 * 4
// Using the memory management for joins as a reference, target 4X batch size as a budget.
var totalBudget = 4 * confTargetSize
// Compute the amount of memory being consumed per-row in the computation
var computationBytesPerRow = inputRowSize + hashTableRowSize
if (isReductionOnly) {
// Remove the lone output row size from the budget rather than track per-row in computation
totalBudget -= outputRowSize
} else {
// The worst-case memory consumption during a grouping aggregation is the case where the
// grouping does not combine any input rows, so just as many rows appear in the output.
computationBytesPerRow += outputRowSize
}
// Calculate the max rows that can be processed during computation within the budget
// Make sure it's not less than 1, otherwise some corner test cases may fail
val maxRows = Math.max(totalBudget / computationBytesPerRow, 1)
// Finally compute the input target batching size taking into account the cudf row limits
Math.min(inputRowSize * maxRows, Int.MaxValue)
}
/**
* Concatenate batches together and perform a merge aggregation on the result. The input batches
* will be closed as part of this operation.
*
* @param batches batches to concatenate and merge aggregate
* @return lazy spillable batch which has NOT been marked spillable
*/
def concatenateAndMerge(
batches: mutable.ArrayBuffer[SpillableColumnarBatch],
metrics: GpuHashAggregateMetrics,
concatAndMergeHelper: AggHelper): SpillableColumnarBatch = {
// TODO: concatenateAndMerge (and calling code) could output a sequence
// of batches for the partial aggregate case. This would be done in case
// a retry failed a certain number of times.
val concatBatch = withResource(batches) { _ =>
val concatSpillable = concatenateBatches(metrics, batches.toSeq)
withResource(concatSpillable) {
_.getColumnarBatch()
}
}
computeAggregateAndClose(metrics, concatBatch, concatAndMergeHelper)
}
/**
* Try to concat and merge neighbour input batches to reduce the number of output batches.
* For some cases where input is highly aggregate-able, we can merge multiple input batches
* into a single output batch. In such cases we can skip repartition at all.
*/
def streamAggregateNeighours(
aggregatedBatches: CloseableBufferedIterator[SpillableColumnarBatch],
metrics: GpuHashAggregateMetrics,
targetMergeBatchSize: Long,
configuredTargetBatchSize: Long,
helper: AggHelper
): Iterator[SpillableColumnarBatch] = {
new Iterator[SpillableColumnarBatch] {
override def hasNext: Boolean = aggregatedBatches.hasNext
override def next(): SpillableColumnarBatch = {
closeOnExcept(new ArrayBuffer[SpillableColumnarBatch]) { stagingBatches => {
var currentSize = 0L
while (aggregatedBatches.hasNext) {
val nextBatch = aggregatedBatches.head
if (currentSize + nextBatch.sizeInBytes > targetMergeBatchSize) {
if (stagingBatches.size == 1) {
return stagingBatches.head
} else if (stagingBatches.isEmpty) {
aggregatedBatches.next
return nextBatch
}
val merged = concatenateAndMerge(stagingBatches, metrics, helper)
stagingBatches.clear
currentSize = 0L
if (merged.sizeInBytes < configuredTargetBatchSize * 0.5) {
stagingBatches += merged
currentSize += merged.sizeInBytes
} else {
return merged
}
} else {
stagingBatches.append(nextBatch)
currentSize += nextBatch.sizeInBytes
aggregatedBatches.next
}
}
if (stagingBatches.size == 1) {
return stagingBatches.head
}
concatenateAndMerge(stagingBatches, metrics, helper)
}
}
}
}
}
/**
* Read the input batches and repartition them into buckets.
*/
def iterateAndRepartition(
aggregatedBatches: Iterator[SpillableColumnarBatch],
metrics: GpuHashAggregateMetrics,
targetMergeBatchSize: Long,
helper: AggHelper,
hashKeys: Seq[GpuExpression],
hashBucketNum: Int,
hashSeed: Int,
batchesByBucket: ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]
): Boolean = {
var repartitionHappened = false
def repartitionAndClose(batch: SpillableColumnarBatch): Unit = {
// OPTIMIZATION
if (!aggregatedBatches.hasNext && batchesByBucket.forall(_.size() == 0)) {
// If this is the only batch (after merging neighbours) to be repartitioned,
// we can just add it to the first bucket and skip repartitioning.
// This is a common case when total input size can fit into a single batch.
batchesByBucket.head.append(batch)
return
}
withResource(new NvtxWithMetrics("agg repartition",
NvtxColor.CYAN, metrics.repartitionTime)) { _ =>
withResource(new GpuBatchSubPartitioner(
Seq(batch).map(batch => {
withResource(batch) { _ =>
batch.getColumnarBatch()
}
}).iterator,
hashKeys, hashBucketNum, hashSeed, "aggRepartition")) {
partitioner => {
(0 until partitioner.partitionsCount).foreach { id =>
closeOnExcept(batchesByBucket) { _ => {
val newBatches = partitioner.releaseBatchesByPartition(id)
newBatches.foreach { newBatch =>
if (newBatch.numRows() > 0) {
batchesByBucket(id).append(newBatch)
} else {
newBatch.safeClose()
}
}
}
}
}
}
}
}
repartitionHappened = true
}
while (aggregatedBatches.hasNext) {
repartitionAndClose(aggregatedBatches.next)
}
// Deal with the over sized buckets
def needRepartitionAgain(bucket: AutoClosableArrayBuffer[SpillableColumnarBatch]) = {
bucket.map(_.sizeInBytes).sum > targetMergeBatchSize &&
bucket.size() != 1 &&
!bucket.forall(_.numRows() == 1) // this is for test
}
if (repartitionHappened && batchesByBucket.exists(needRepartitionAgain)) {
logDebug("Some of the repartition buckets are over sized, trying to split them")
val newBuckets = batchesByBucket.flatMap(bucket => {
if (needRepartitionAgain(bucket)) {
if (hashSeed + 7 > 200) {
log.warn("Too many times of repartition, may hit a bug? Size for each batch in " +
"current bucket: " + bucket.map(_.sizeInBytes).mkString(", ") + " rows: " +
bucket.map(_.numRows()).mkString(", ") + " targetMergeBatchSize: "
+ targetMergeBatchSize)
ArrayBuffer.apply(bucket)
} else {
val nextLayerBuckets =
ArrayBuffer.fill(hashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
// Recursively merge and repartition the over sized bucket
repartitionHappened =
iterateAndRepartition(
new CloseableBufferedIterator(bucket.iterator), metrics, targetMergeBatchSize,
helper, hashKeys, hashBucketNum, hashSeed + 7,
nextLayerBuckets) || repartitionHappened
nextLayerBuckets
}
} else {
ArrayBuffer.apply(bucket)
}
})
batchesByBucket.clear()
batchesByBucket.appendAll(newBuckets)
}
repartitionHappened
}
}
/** Utility class to hold all of the metrics related to hash aggregation */
case class GpuHashAggregateMetrics(
numOutputRows: GpuMetric,
numOutputBatches: GpuMetric,
numTasksRepartitioned: GpuMetric,
numTasksSkippedAgg: GpuMetric,
opTime: GpuMetric,
computeAggTime: GpuMetric,
concatTime: GpuMetric,
sortTime: GpuMetric,
repartitionTime: GpuMetric,
numAggOps: GpuMetric,
numPreSplits: GpuMetric,
singlePassTasks: GpuMetric,
heuristicTime: GpuMetric) {
}
/** Utility class to convey information on the aggregation modes being used */
case class AggregateModeInfo(
uniqueModes: Seq[AggregateMode],
hasPartialMode: Boolean,
hasPartialMergeMode: Boolean,
hasFinalMode: Boolean,
hasCompleteMode: Boolean)
object AggregateModeInfo {
def apply(uniqueModes: Seq[AggregateMode]): AggregateModeInfo = {
AggregateModeInfo(
uniqueModes = uniqueModes,
hasPartialMode = uniqueModes.contains(Partial),
hasPartialMergeMode = uniqueModes.contains(PartialMerge),
hasFinalMode = uniqueModes.contains(Final),
hasCompleteMode = uniqueModes.contains(Complete)
)
}
}
/**
* Internal class used in `computeAggregates` for the pre, agg, and post steps
*
* @param inputAttributes input attributes to identify the input columns from the input batches
* @param groupingExpressions expressions used for producing the grouping keys
* @param aggregateExpressions GPU aggregate expressions used to produce the aggregations
* @param forceMerge if true, we are merging two pre-aggregated batches, so we should use
* the merge steps for each aggregate function
* @param isSorted if the batch is sorted this is set to true and is passed to cuDF
* as an optimization hint
* @param conf A configuration used to control TieredProject operations in an
* aggregation.
*/
class AggHelper(
inputAttributes: Seq[Attribute],
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[GpuAggregateExpression],
forceMerge: Boolean,
conf: SQLConf,
isSorted: Boolean = false) extends Serializable {
private var doSortAgg = isSorted
def setSort(isSorted: Boolean): Unit = {
doSortAgg = isSorted
}
// `CudfAggregate` instances to apply, either update or merge aggregates
// package private for testing
private[rapids] val cudfAggregates = new mutable.ArrayBuffer[CudfAggregate]()
// integers for each column the aggregate is operating on
// package private for testing
private[rapids] val aggOrdinals = new mutable.ArrayBuffer[Int]
// grouping ordinals are the indices of the tables to aggregate that need to be
// the grouping key
// package private for testing
private[rapids] val groupingOrdinals: Array[Int] = groupingExpressions.indices.toArray
// the resulting data type from the cuDF aggregate (from
// the update or merge aggregate, be it reduction or group by)
private[rapids] val postStepDataTypes = new mutable.ArrayBuffer[DataType]()
private val groupingAttributes = groupingExpressions.map(_.toAttribute)
private val aggBufferAttributes = groupingAttributes ++
aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
// `GpuAggregateFunction` can add a pre and post step for update
// and merge aggregates.
private val preStep = new mutable.ArrayBuffer[Expression]()
private val postStep = new mutable.ArrayBuffer[Expression]()
private val postStepAttr = new mutable.ArrayBuffer[Attribute]()
// we add the grouping expression first, which should bind as pass-through
if (forceMerge) {
// a grouping expression can do actual computation, but we cannot do that computation again
// on a merge, nor would we want to if we could. So use the attributes instead of the
// original expression when we are forcing a merge.
preStep ++= groupingAttributes
} else {
preStep ++= groupingExpressions
}
postStep ++= groupingAttributes
postStepAttr ++= groupingAttributes
postStepDataTypes ++=
groupingExpressions.map(_.dataType)
private var ix = groupingAttributes.length
for (aggExp <- aggregateExpressions) {
val aggFn = aggExp.aggregateFunction
if ((aggExp.mode == Partial || aggExp.mode == Complete) && !forceMerge) {
val ordinals = (ix until ix + aggFn.updateAggregates.length)
aggOrdinals ++= ordinals
ix += ordinals.length
val updateAggs = aggFn.updateAggregates
postStepDataTypes ++= updateAggs.map(_.dataType)
cudfAggregates ++= updateAggs
preStep ++= aggFn.inputProjection
postStep ++= aggFn.postUpdate
postStepAttr ++= aggFn.postUpdateAttr
} else {
val ordinals = (ix until ix + aggFn.mergeAggregates.length)
aggOrdinals ++= ordinals
ix += ordinals.length
val mergeAggs = aggFn.mergeAggregates
postStepDataTypes ++= mergeAggs.map(_.dataType)
cudfAggregates ++= mergeAggs
preStep ++= aggFn.preMerge
postStep ++= aggFn.postMerge
postStepAttr ++= aggFn.postMergeAttr
}
}
// a bound expression that is applied before the cuDF aggregate
private val preStepAttributes = if (forceMerge) {
aggBufferAttributes
} else {
inputAttributes
}
val preStepBound = GpuBindReferences.bindGpuReferencesTiered(preStep.toList,
preStepAttributes.toList, conf)
// a bound expression that is applied after the cuDF aggregate
private val postStepBound = GpuBindReferences.bindGpuReferencesTiered(postStep.toList,
postStepAttr.toList, conf)
/**
* Apply the "pre" step: preMerge for merge, or pass-through in the update case
*
* @param toAggregateBatch - input (to the agg) batch from the child directly in the
* merge case, or from the `inputProjection` in the update case.
* @return a pre-processed batch that can be later cuDF aggregated
*/
def preProcess(
toAggregateBatch: ColumnarBatch,
metrics: GpuHashAggregateMetrics): SpillableColumnarBatch = {
val inputBatch = SpillableColumnarBatch(toAggregateBatch,
SpillPriorities.ACTIVE_ON_DECK_PRIORITY)
val projectedCb = withResource(new NvtxRange("pre-process", NvtxColor.DARK_GREEN)) { _ =>
preStepBound.projectAndCloseWithRetrySingleBatch(inputBatch)
}
SpillableColumnarBatch(
projectedCb,
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
def aggregate(preProcessed: ColumnarBatch, numAggs: GpuMetric): ColumnarBatch = {
val ret = if (groupingOrdinals.nonEmpty) {
performGroupByAggregation(preProcessed)
} else {
performReduction(preProcessed)
}
numAggs += 1
ret
}
def aggregateWithoutCombine(metrics: GpuHashAggregateMetrics,
preProcessed: Iterator[SpillableColumnarBatch]): Iterator[SpillableColumnarBatch] = {
val computeAggTime = metrics.computeAggTime
val opTime = metrics.opTime
val numAggs = metrics.numAggOps
preProcessed.flatMap { sb =>
withRetry(sb, splitSpillableInHalfByRows) { preProcessedAttempt =>
withResource(new NvtxWithMetrics("computeAggregate", NvtxColor.CYAN, computeAggTime,
opTime)) { _ =>
withResource(preProcessedAttempt.getColumnarBatch()) { cb =>
SpillableColumnarBatch(
aggregate(cb, numAggs),
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}
}
}
}
def aggregate(
metrics: GpuHashAggregateMetrics,
preProcessed: SpillableColumnarBatch): SpillableColumnarBatch = {
val numAggs = metrics.numAggOps
val aggregatedSeq =
withRetry(preProcessed, splitSpillableInHalfByRows) { preProcessedAttempt =>
withResource(preProcessedAttempt.getColumnarBatch()) { cb =>
SpillableColumnarBatch(
aggregate(cb, numAggs),
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}.toSeq
// We need to merge the aggregated batches into 1 before calling post process,
// if the aggregate code had to split on a retry
if (aggregatedSeq.size > 1) {
val concatted = concatenateBatches(metrics, aggregatedSeq)
withRetryNoSplit(concatted) { attempt =>
withResource(attempt.getColumnarBatch()) { cb =>
SpillableColumnarBatch(
aggregate(cb, numAggs),
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}
} else {
aggregatedSeq.head
}
}
/**
* Invoke reduction functions as defined in each `CudfAggreagte`
*
* @param preProcessed - a batch after the "pre" step
* @return
*/
def performReduction(preProcessed: ColumnarBatch): ColumnarBatch = {
withResource(new NvtxRange("reduce", NvtxColor.BLUE)) { _ =>
val cvs = mutable.ArrayBuffer[GpuColumnVector]()
cudfAggregates.zipWithIndex.foreach { case (cudfAgg, ix) =>
val aggFn = cudfAgg.reductionAggregate
val cols = GpuColumnVector.extractColumns(preProcessed)
val reductionCol = cols(aggOrdinals(ix))
withResource(aggFn(reductionCol.getBase)) { res =>
cvs += GpuColumnVector.from(
cudf.ColumnVector.fromScalar(res, 1), cudfAgg.dataType)
}
}
new ColumnarBatch(cvs.toArray, 1)
}
}
/**
* Used to produce a group-by aggregate
*
* @param preProcessed the batch after the "pre" step
* @return a Table that has been cuDF aggregated
*/
def performGroupByAggregation(preProcessed: ColumnarBatch): ColumnarBatch = {
withResource(new NvtxRange("groupby", NvtxColor.BLUE)) { _ =>
withResource(GpuColumnVector.from(preProcessed)) { preProcessedTbl =>
val groupOptions = cudf.GroupByOptions.builder()
.withIgnoreNullKeys(false)
.withKeysSorted(doSortAgg)
.build()
val cudfAggsOnColumn = cudfAggregates.zip(aggOrdinals).map {
case (cudfAgg, ord) => cudfAgg.groupByAggregate.onColumn(ord)
}
// perform the aggregate
val aggTbl = preProcessedTbl
.groupBy(groupOptions, groupingOrdinals: _*)
.aggregate(cudfAggsOnColumn.toSeq: _*)
withResource(aggTbl) { _ =>
GpuColumnVector.from(aggTbl, postStepDataTypes.toArray)
}
}
}
}
/**
* Used to produce the outbound batch from the aggregate that could be
* shuffled or could be passed through the evaluateExpression if we are in the final
* stage.
* It takes a cuDF aggregated batch and applies the "post" step:
* postUpdate for update, or postMerge for merge
*
* @param resultBatch - cuDF aggregated batch
* @return output batch from the aggregate
*/
def postProcess(
aggregatedSpillable: SpillableColumnarBatch,
metrics: GpuHashAggregateMetrics): SpillableColumnarBatch = {
val postProcessed =
withResource(new NvtxRange("post-process", NvtxColor.ORANGE)) { _ =>
postStepBound.projectAndCloseWithRetrySingleBatch(aggregatedSpillable)
}
SpillableColumnarBatch(
postProcessed,
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
def postProcess(input: Iterator[SpillableColumnarBatch],
metrics: GpuHashAggregateMetrics): Iterator[SpillableColumnarBatch] = {
val computeAggTime = metrics.computeAggTime
val opTime = metrics.opTime
input.map { aggregated =>
withResource(new NvtxWithMetrics("post-process", NvtxColor.ORANGE, computeAggTime,
opTime)) { _ =>
val postProcessed = postStepBound.projectAndCloseWithRetrySingleBatch(aggregated)
SpillableColumnarBatch(
postProcessed,
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}
}
}
object GpuAggregateIterator extends Logging {
/**
* @note abstracted away for a unit test..
* @param helper
* @param preProcessed
* @return
*/
def aggregate(
helper: AggHelper,
preProcessed: SpillableColumnarBatch,
metrics: GpuHashAggregateMetrics): SpillableColumnarBatch = {
helper.aggregate(metrics, preProcessed)
}
/**
* Compute the aggregations on the projected input columns, and close input batch.
*
* @note public for testing
* @param metrics metrics that will be updated during aggregation
* @param inputBatch input batch to aggregate
* @param helper an internal object that carries state required to execute the aggregate from
* different parts of the codebase.
* @return aggregated batch
*/
def computeAggregateAndClose(
metrics: GpuHashAggregateMetrics,
inputBatch: ColumnarBatch,
helper: AggHelper): SpillableColumnarBatch = {
val computeAggTime = metrics.computeAggTime
val opTime = metrics.opTime
withResource(new NvtxWithMetrics("computeAggregate", NvtxColor.CYAN, computeAggTime,
opTime)) { _ =>
// 1) a pre-processing step required before we go into the cuDF aggregate,
// in some cases casting and in others creating a struct (MERGE_M2 for instance,
// requires a struct)
// OOM retry happens within the projection in preProcess
val preProcessed = helper.preProcess(inputBatch, metrics)
// 2) perform the aggregation
// OOM retry means we could get a list of batches
val aggregatedSpillable = aggregate(helper, preProcessed, metrics)
// 3) a post-processing step required in some scenarios, casting or picking
// apart a struct
helper.postProcess(aggregatedSpillable, metrics)
}
}
def computeAggregateWithoutPreprocessAndClose(
metrics: GpuHashAggregateMetrics,
inputBatches: Iterator[ColumnarBatch],
helper: AggHelper): Iterator[SpillableColumnarBatch] = {
val computeAggTime = metrics.computeAggTime
val opTime = metrics.opTime
// 1) a pre-processing step required before we go into the cuDF aggregate, This has already
// been done and is skipped
val spillableInput = inputBatches.map { cb =>
withResource(new MetricRange(computeAggTime, opTime)) { _ =>
SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}
// 2) perform the aggregation
// OOM retry means we could get a list of batches
val aggregatedSpillable = helper.aggregateWithoutCombine(metrics, spillableInput)
// 3) a post-processing step required in some scenarios, casting or picking
// apart a struct
helper.postProcess(aggregatedSpillable, metrics)
}
/**
* Concatenates batches after extracting them from `SpllableColumnarBatch`
* @note the input batches are not closed as part of this operation
* @param metrics metrics that will be updated during aggregation
* @param toConcat spillable batches to concatenate
* @return concatenated batch result
*/
def concatenateBatches(
metrics: GpuHashAggregateMetrics,
toConcat: Seq[SpillableColumnarBatch]): SpillableColumnarBatch = {
if (toConcat.size == 1) {
toConcat.head
} else {
withRetryNoSplit(toConcat) { attempt =>
val concatTime = metrics.concatTime
val opTime = metrics.opTime
withResource(
new NvtxWithMetrics("concatenateBatches", NvtxColor.BLUE, concatTime,
opTime)) { _ =>
val batchesToConcat = attempt.safeMap(_.getColumnarBatch())
withResource(batchesToConcat) { _ =>
val numCols = batchesToConcat.head.numCols()
val dataTypes = (0 until numCols).map {
c => batchesToConcat.head.column(c).dataType
}.toArray
withResource(batchesToConcat.safeMap(GpuColumnVector.from)) { tbl =>
withResource(cudf.Table.concatenate(tbl: _*)) { concatenated =>
val cb = GpuColumnVector.from(concatenated, dataTypes)
SpillableColumnarBatch(cb,
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}
}
}
}
}
}
}
}
object GpuAggFirstPassIterator {
def apply(cbIter: Iterator[ColumnarBatch],
aggHelper: AggHelper,
metrics: GpuHashAggregateMetrics
): Iterator[SpillableColumnarBatch] = {
val preprocessProjectIter = cbIter.map { cb =>
val sb = SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)
aggHelper.preStepBound.projectAndCloseWithRetrySingleBatch(sb)
}
computeAggregateWithoutPreprocessAndClose(metrics, preprocessProjectIter, aggHelper)
}
}
// Partial mode:
// * boundFinalProjections: is a pass-through of the agg buffer
// * boundResultReferences: is a pass-through of the merged aggregate
//
// Final mode:
// * boundFinalProjections: on merged batches, finalize aggregates
// (GpuAverage => CudfSum/CudfCount)
// * boundResultReferences: project the result expressions Spark expects in the output.
//
// Complete mode:
// * boundFinalProjections: on merged batches, finalize aggregates
// (GpuAverage => CudfSum/CudfCount)
// * boundResultReferences: project the result expressions Spark expects in the output.
case class BoundExpressionsModeAggregates(
boundFinalProjections: Option[Seq[GpuExpression]],
boundResultReferences: Seq[Expression])
object GpuAggFinalPassIterator {
/**
* `setupReferences` binds input, final and result references for the aggregate.
* - input: used to obtain columns coming into the aggregate from the child
* - final: some aggregates like average use this to specify an expression to produce
* the final output of the aggregate. Average keeps sum and count throughout,
* and at the end it has to divide the two, to produce the single sum/count result.
* - result: used at the end to output to our parent
*/
def setupReferences(
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[GpuAggregateExpression],
aggregateAttributes: Seq[Attribute],
resultExpressions: Seq[NamedExpression],
modeInfo: AggregateModeInfo): BoundExpressionsModeAggregates = {
val groupingAttributes = groupingExpressions.map(_.toAttribute)
val aggBufferAttributes = groupingAttributes ++
aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
val boundFinalProjections = if (modeInfo.hasFinalMode || modeInfo.hasCompleteMode) {
val finalProjections = groupingAttributes ++
aggregateExpressions.map(_.aggregateFunction.evaluateExpression)
Some(GpuBindReferences.bindGpuReferences(finalProjections, aggBufferAttributes))
} else {
None
}
// allAttributes can be different things, depending on aggregation mode:
// - Partial mode: grouping key + cudf aggregates (e.g. no avg, instead sum::count
// - Final mode: grouping key + spark aggregates (e.g. avg)
val finalAttributes = groupingAttributes ++ aggregateAttributes
// boundResultReferences is used to project the aggregated input batch(es) for the result.
// - Partial mode: it's a pass through. We take whatever was aggregated and let it come
// out of the node as is.
// - Final or Complete mode: we use resultExpressions to pick out the correct columns that
// finalReferences has pre-processed for us
val boundResultReferences = if (modeInfo.hasPartialMode || modeInfo.hasPartialMergeMode) {
GpuBindReferences.bindGpuReferences(
resultExpressions,
resultExpressions.map(_.toAttribute))
} else if (modeInfo.hasFinalMode || modeInfo.hasCompleteMode) {
GpuBindReferences.bindGpuReferences(
resultExpressions,
finalAttributes)
} else {
GpuBindReferences.bindGpuReferences(
resultExpressions,
groupingAttributes)
}
BoundExpressionsModeAggregates(
boundFinalProjections,
boundResultReferences)
}
private[this] def reorderFinalBatch(finalBatch: SpillableColumnarBatch,
boundExpressions: BoundExpressionsModeAggregates,
metrics: GpuHashAggregateMetrics): ColumnarBatch = {
// Perform the last project to get the correct shape that Spark expects. Note this may
// add things like literals that were not part of the aggregate into the batch.
closeOnExcept(GpuProjectExec.projectAndCloseWithRetrySingleBatch(finalBatch,
boundExpressions.boundResultReferences)) { ret =>
metrics.numOutputRows += ret.numRows()
metrics.numOutputBatches += 1
ret
}
}
def makeIter(cbIter: Iterator[ColumnarBatch],
boundExpressions: BoundExpressionsModeAggregates,
metrics: GpuHashAggregateMetrics): Iterator[ColumnarBatch] = {
val aggTime = metrics.computeAggTime
val opTime = metrics.opTime
cbIter.map { batch =>
withResource(new NvtxWithMetrics("finalize agg", NvtxColor.DARK_GREEN, aggTime,
opTime)) { _ =>
val finalBatch = boundExpressions.boundFinalProjections.map { exprs =>
GpuProjectExec.projectAndCloseWithRetrySingleBatch(
SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_BATCHING_PRIORITY), exprs)
}.getOrElse(batch)
val finalSCB =
SpillableColumnarBatch(finalBatch, SpillPriorities.ACTIVE_BATCHING_PRIORITY)
reorderFinalBatch(finalSCB, boundExpressions, metrics)
}
}
}
def makeIterFromSpillable(sbIter: Iterator[SpillableColumnarBatch],
boundExpressions: BoundExpressionsModeAggregates,
metrics: GpuHashAggregateMetrics): Iterator[ColumnarBatch] = {
val aggTime = metrics.computeAggTime
val opTime = metrics.opTime
sbIter.map { sb =>
withResource(new NvtxWithMetrics("finalize agg", NvtxColor.DARK_GREEN, aggTime,
opTime)) { _ =>
val finalBatch = boundExpressions.boundFinalProjections.map { exprs =>
SpillableColumnarBatch(
GpuProjectExec.projectAndCloseWithRetrySingleBatch(sb, exprs),
SpillPriorities.ACTIVE_BATCHING_PRIORITY)
}.getOrElse(sb)
reorderFinalBatch(finalBatch, boundExpressions, metrics)
}
}
}
}
/**
* Iterator that takes another columnar batch iterator as input and emits new columnar batches that
* are aggregated based on the specified grouping and aggregation expressions. This iterator tries
* to perform a hash-based aggregation but is capable of falling back to a repartition-based
* aggregation which can operate on data that is either larger than can be represented by a cudf
* column or larger than can fit in GPU memory.
*
* In general, GpuMergeAggregateIterator works in this flow:
*
* (1) The iterator starts by pulling all batches from the input iterator, performing an initial
* projection and aggregation on each individual batch via `GpuAggFirstPassIterator`, we call it
* "First Pass Aggregate".
* (2) Then the batches after first pass agg is sent to "streamAggregateNeighours", where it tries
* to concat & merge the neighbour batches into fewer batches, then "iterateAndRepartition"
* repartition the batch into fixed size buckets. Recursive repartition will be applied on
* over-sized buckets until each bucket * is within the target size.
* We call this phase "Second Pass Aggregate".
* (3) At "Third Pass Aggregate", we take each bucket and perform a final aggregation on all batches
* in the bucket, check "RepartitionAggregateIterator" for details.
*
* @param firstPassIter iterator that has done a first aggregation pass over the input data.
* @param inputAttributes input attributes to identify the input columns from the input batches
* @param groupingExpressions expressions used for producing the grouping keys
* @param aggregateExpressions GPU aggregate expressions used to produce the aggregations
* @param aggregateAttributes attribute references to each aggregate expression
* @param resultExpressions output expression for the aggregation
* @param modeInfo identifies which aggregation modes are being used
* @param metrics metrics that will be updated during aggregation
* @param configuredTargetBatchSize user-specified value for the targeted input batch size
* @param allowNonFullyAggregatedOutput if allowed to skip third pass Agg
* @param skipAggPassReductionRatio skip if the ratio of rows after a pass is bigger than this value
* @param localInputRowsCount metric to track the number of input rows processed locally
*/
class GpuMergeAggregateIterator(
firstPassIter: CloseableBufferedIterator[SpillableColumnarBatch],
inputAttributes: Seq[Attribute],
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[GpuAggregateExpression],
aggregateAttributes: Seq[Attribute],
resultExpressions: Seq[NamedExpression],
modeInfo: AggregateModeInfo,
metrics: GpuHashAggregateMetrics,
configuredTargetBatchSize: Long,
conf: SQLConf,
allowNonFullyAggregatedOutput: Boolean,
skipAggPassReductionRatio: Double,
localInputRowsCount: LocalGpuMetric
)
extends Iterator[ColumnarBatch] with AutoCloseable with Logging {
private[this] val isReductionOnly = groupingExpressions.isEmpty
private[this] val targetMergeBatchSize = computeTargetMergeBatchSize(configuredTargetBatchSize)
private[this] val defaultHashBucketNum = 16
private[this] val defaultHashSeed = 107
private[this] var batchesByBucket =
ArrayBuffer.fill(defaultHashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
private[this] var firstBatchChecked = false
private[this] var bucketIter: Option[RepartitionAggregateIterator] = None
private[this] var realIter: Option[Iterator[ColumnarBatch]] = None
/** Whether a batch is pending for a reduction-only aggregation */
private[this] var hasReductionOnlyBatch: Boolean = isReductionOnly
// Don't install the callback if in a unit test
Option(TaskContext.get()).foreach { tc =>
onTaskCompletion(tc) {
close()
}
}
override def hasNext: Boolean = {
realIter.map(_.hasNext).getOrElse {
// reductions produce a result even if the input is empty
hasReductionOnlyBatch || firstPassIter.hasNext
}
}
override def next(): ColumnarBatch = {
realIter.map(_.next()).getOrElse {
// Handle reduction-only aggregation
if (isReductionOnly) {
val batches = ArrayBuffer.apply[SpillableColumnarBatch]()
while (firstPassIter.hasNext) {
batches += firstPassIter.next()
}
if (batches.isEmpty || batches.forall(_.numRows() == 0)) {
hasReductionOnlyBatch = false
return generateEmptyReductionBatch()
} else {
hasReductionOnlyBatch = false
val concat = AggregateUtils.concatenateAndMerge(batches, metrics, concatAndMergeHelper)
return withResource(concat) { cb =>
cb.getColumnarBatch()
}
}
}
// Handle the case of skipping second and third pass of aggregation
// This only work when spark.rapids.sql.agg.skipAggPassReductionRatio < 1
if (!firstBatchChecked && firstPassIter.hasNext
&& allowNonFullyAggregatedOutput) {
firstBatchChecked = true
val peek = firstPassIter.head
// It's only based on first batch of first pass agg, so it's an estimate
val firstPassReductionRatioEstimate = 1.0 * peek.numRows() / localInputRowsCount.value
if (firstPassReductionRatioEstimate > skipAggPassReductionRatio) {
logDebug("Skipping second and third pass aggregation due to " +
"too high reduction ratio in first pass: " +
s"$firstPassReductionRatioEstimate")
// if so, skip any aggregation, return the origin batch directly
realIter = Some(ConcatIterator.apply(firstPassIter, configuredTargetBatchSize))
metrics.numTasksSkippedAgg += 1
return realIter.get.next()
} else {
logInfo(s"The reduction ratio in first pass is not high enough to skip " +
s"second and third pass aggregation: peek.numRows: ${peek.numRows()}, " +
s"localInputRowsCount.value: ${localInputRowsCount.value}")
}
}
firstBatchChecked = true
val groupingAttributes = groupingExpressions.map(_.toAttribute)
val aggBufferAttributes = groupingAttributes ++