-
Notifications
You must be signed in to change notification settings - Fork 242
/
Copy pathGpuOverrides.scala
4558 lines (4375 loc) · 209 KB
/
GpuOverrides.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-2023, 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 java.time.ZoneId
import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag
import scala.util.control.NonFatal
import ai.rapids.cudf.DType
import com.nvidia.spark.rapids.RapidsConf.{SUPPRESS_PLANNING_FAILURE, TEST_CONF}
import com.nvidia.spark.rapids.shims._
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.apache.spark.rapids.shims.GpuShuffleExchangeExec
import org.apache.spark.sql.{DataFrame, SparkSession}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.rapids.TimeStamp
import org.apache.spark.sql.catalyst.json.rapids.GpuJsonScan
import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.catalyst.util.ArrayData
import org.apache.spark.sql.connector.read.Scan
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, QueryStageExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec}
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.command.{DataWritingCommand, DataWritingCommandExec, ExecutedCommandExec, RunnableCommand}
import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCommand, SaveIntoDataSourceCommand}
import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
import org.apache.spark.sql.execution.datasources.json.JsonFileFormat
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.text.TextFileFormat
import org.apache.spark.sql.execution.datasources.v2._
import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan
import org.apache.spark.sql.execution.datasources.v2.json.JsonScan
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ENSURE_REQUIREMENTS, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.execution.python._
import org.apache.spark.sql.execution.window.WindowExec
import org.apache.spark.sql.hive.rapids.GpuHiveOverrides
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids._
import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand
import org.apache.spark.sql.rapids.execution._
import org.apache.spark.sql.rapids.execution.python._
import org.apache.spark.sql.rapids.execution.python.GpuFlatMapGroupsInPandasExecMeta
import org.apache.spark.sql.rapids.shims.GpuTimeAdd
import org.apache.spark.sql.rapids.zorder.ZOrderRules
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
/**
* Base class for all ReplacementRules
* @param doWrap wraps a part of the plan in a [[RapidsMeta]] for further processing.
* @param desc a description of what this part of the plan does.
* @param tag metadata used to determine what INPUT is at runtime.
* @tparam INPUT the exact type of the class we are wrapping.
* @tparam BASE the generic base class for this type of stage, i.e. SparkPlan, Expression, etc.
* @tparam WRAP_TYPE base class that should be returned by doWrap.
*/
abstract class ReplacementRule[INPUT <: BASE, BASE, WRAP_TYPE <: RapidsMeta[INPUT, BASE, _]](
protected var doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => WRAP_TYPE,
protected var desc: String,
protected val checks: Option[TypeChecks[_]],
final val tag: ClassTag[INPUT]) extends DataFromReplacementRule {
private var _incompatDoc: Option[String] = None
private var _disabledDoc: Option[String] = None
private var _visible: Boolean = true
def isVisible: Boolean = _visible
def description: String = desc
override def incompatDoc: Option[String] = _incompatDoc
override def disabledMsg: Option[String] = _disabledDoc
override def getChecks: Option[TypeChecks[_]] = checks
/**
* Mark this expression as incompatible with the original Spark version
* @param str a description of how it is incompatible.
* @return this for chaining.
*/
final def incompat(str: String) : this.type = {
_incompatDoc = Some(str)
this
}
/**
* Mark this expression as disabled by default.
* @param str a description of why it is disabled by default.
* @return this for chaining.
*/
final def disabledByDefault(str: String) : this.type = {
_disabledDoc = Some(str)
this
}
final def invisible(): this.type = {
_visible = false
this
}
/**
* Provide a function that will wrap a spark type in a [[RapidsMeta]] instance that is used for
* conversion to a GPU version.
* @param func the function
* @return this for chaining.
*/
final def wrap(func: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => WRAP_TYPE): this.type = {
doWrap = func
this
}
/**
* Set a description of what the operation does.
* @param str the description.
* @return this for chaining
*/
final def desc(str: String): this.type = {
this.desc = str
this
}
private var confKeyCache: String = null
protected val confKeyPart: String
override def confKey: String = {
if (confKeyCache == null) {
confKeyCache = "spark.rapids.sql." + confKeyPart + "." + tag.runtimeClass.getSimpleName
}
confKeyCache
}
def notes(): Option[String] = if (incompatDoc.isDefined) {
Some(s"This is not 100% compatible with the Spark version because ${incompatDoc.get}")
} else if (disabledMsg.isDefined) {
Some(s"This is disabled by default because ${disabledMsg.get}")
} else {
None
}
def confHelp(asTable: Boolean = false, sparkSQLFunctions: Option[String] = None): Unit = {
if (_visible) {
val notesMsg = notes()
if (asTable) {
import ConfHelper.makeConfAnchor
print(s"${makeConfAnchor(confKey)}")
if (sparkSQLFunctions.isDefined) {
print(s"|${sparkSQLFunctions.get}")
}
val incompatOps = RapidsConf.INCOMPATIBLE_OPS.asInstanceOf[ConfEntryWithDefault[Boolean]]
val expressionEnabled = disabledMsg.isEmpty &&
(incompatDoc.isEmpty || incompatOps.defaultValue)
print(s"|$desc|$expressionEnabled|")
if (notesMsg.isDefined) {
print(s"${notesMsg.get}")
} else {
print("None")
}
println("|")
} else {
println(s"$confKey:")
println(s"\tEnable (true) or disable (false) the $tag $operationName.")
if (sparkSQLFunctions.isDefined) {
println(s"\tsql function: ${sparkSQLFunctions.get}")
}
println(s"\t$desc")
if (notesMsg.isDefined) {
println(s"\t${notesMsg.get}")
}
println(s"\tdefault: ${notesMsg.isEmpty}")
println()
}
}
}
final def wrap(
op: BASE,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
r: DataFromReplacementRule): WRAP_TYPE = {
doWrap(op.asInstanceOf[INPUT], conf, parent, r)
}
def getClassFor: Class[_] = tag.runtimeClass
}
/**
* Holds everything that is needed to replace an Expression with a GPU enabled version.
*/
class ExprRule[INPUT <: Expression](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => BaseExprMeta[INPUT],
desc: String,
checks: Option[ExprChecks],
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, Expression, BaseExprMeta[INPUT]](
doWrap, desc, checks, tag) {
override val confKeyPart = "expression"
override val operationName = "Expression"
}
/**
* Holds everything that is needed to replace a `Scan` with a GPU enabled version.
*/
class ScanRule[INPUT <: Scan](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => ScanMeta[INPUT],
desc: String,
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, Scan, ScanMeta[INPUT]](
doWrap, desc, None, tag) {
override val confKeyPart: String = "input"
override val operationName: String = "Input"
}
/**
* Holds everything that is needed to replace a `Partitioning` with a GPU enabled version.
*/
class PartRule[INPUT <: Partitioning](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => PartMeta[INPUT],
desc: String,
checks: Option[PartChecks],
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, Partitioning, PartMeta[INPUT]](
doWrap, desc, checks, tag) {
override val confKeyPart: String = "partitioning"
override val operationName: String = "Partitioning"
}
/**
* Holds everything that is needed to replace a `SparkPlan` with a GPU enabled version.
*/
class ExecRule[INPUT <: SparkPlan](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => SparkPlanMeta[INPUT],
desc: String,
checks: Option[ExecChecks],
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, SparkPlan, SparkPlanMeta[INPUT]](
doWrap, desc, checks, tag) {
// TODO finish this...
override val confKeyPart: String = "exec"
override val operationName: String = "Exec"
}
/**
* Holds everything that is needed to replace a `DataWritingCommand` with a
* GPU enabled version.
*/
class DataWritingCommandRule[INPUT <: DataWritingCommand](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => DataWritingCommandMeta[INPUT],
desc: String,
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, DataWritingCommand, DataWritingCommandMeta[INPUT]](
doWrap, desc, None, tag) {
override val confKeyPart: String = "output"
override val operationName: String = "Output"
}
final class InsertIntoHadoopFsRelationCommandMeta(
cmd: InsertIntoHadoopFsRelationCommand,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
extends DataWritingCommandMeta[InsertIntoHadoopFsRelationCommand](cmd, conf, parent, rule) {
private var fileFormat: Option[ColumnarFileFormat] = None
override def tagSelfForGpu(): Unit = {
if (cmd.bucketSpec.isDefined) {
willNotWorkOnGpu("bucketing is not supported")
}
val spark = SparkSession.active
val formatCls = cmd.fileFormat.getClass
fileFormat = if (formatCls == classOf[CSVFileFormat]) {
willNotWorkOnGpu("CSV output is not supported")
None
} else if (formatCls == classOf[JsonFileFormat]) {
willNotWorkOnGpu("JSON output is not supported")
None
} else if (GpuOrcFileFormat.isSparkOrcFormat(formatCls)) {
GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema)
} else if (formatCls == classOf[ParquetFileFormat]) {
GpuParquetFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema)
} else if (formatCls == classOf[TextFileFormat]) {
willNotWorkOnGpu("text output is not supported")
None
} else {
willNotWorkOnGpu(s"unknown file format: ${formatCls.getCanonicalName}")
None
}
}
override def convertToGpu(): GpuDataWritingCommand = {
val format = fileFormat.getOrElse(
throw new IllegalStateException("fileFormat missing, tagSelfForGpu not called?"))
GpuInsertIntoHadoopFsRelationCommand(
cmd.outputPath,
cmd.staticPartitions,
cmd.ifPartitionNotExists,
cmd.partitionColumns,
cmd.bucketSpec,
format,
cmd.options,
cmd.query,
cmd.mode,
cmd.catalogTable,
cmd.fileIndex,
cmd.outputColumnNames,
conf.stableSort,
conf.concurrentWriterPartitionFlushSize)
}
}
/**
* Holds everything that is needed to replace a `RunnableCommand` with a
* GPU enabled version.
*/
class RunnableCommandRule[INPUT <: RunnableCommand](
doWrap: (
INPUT,
RapidsConf,
Option[RapidsMeta[_, _, _]],
DataFromReplacementRule) => RunnableCommandMeta[INPUT],
desc: String,
tag: ClassTag[INPUT])
extends ReplacementRule[INPUT, RunnableCommand, RunnableCommandMeta[INPUT]](
doWrap, desc, None, tag) {
override val confKeyPart: String = "command"
override val operationName: String = "Command"
}
/**
* Listener trait so that tests can confirm that the expected optimizations are being applied
*/
trait GpuOverridesListener {
def optimizedPlan(
plan: SparkPlanMeta[SparkPlan],
sparkPlan: SparkPlan,
costOptimizations: Seq[Optimization]): Unit
}
sealed trait FileFormatType
object CsvFormatType extends FileFormatType {
override def toString = "CSV"
}
object HiveDelimitedTextFormatType extends FileFormatType {
override def toString = "HiveText"
}
object ParquetFormatType extends FileFormatType {
override def toString = "Parquet"
}
object OrcFormatType extends FileFormatType {
override def toString = "ORC"
}
object JsonFormatType extends FileFormatType {
override def toString = "JSON"
}
object AvroFormatType extends FileFormatType {
override def toString = "Avro"
}
object IcebergFormatType extends FileFormatType {
override def toString = "Iceberg"
}
object DeltaFormatType extends FileFormatType {
override def toString = "Delta"
}
sealed trait FileFormatOp
object ReadFileOp extends FileFormatOp {
override def toString = "read"
}
object WriteFileOp extends FileFormatOp {
override def toString = "write"
}
object GpuOverrides extends Logging {
val FLOAT_DIFFERS_GROUP_INCOMPAT =
"when enabling these, there may be extra groups produced for floating point grouping " +
"keys (e.g. -0.0, and 0.0)"
val CASE_MODIFICATION_INCOMPAT =
"the Unicode version used by cuDF and the JVM may differ, resulting in some " +
"corner-case characters not changing case correctly."
val UTC_TIMEZONE_ID = ZoneId.of("UTC").normalized()
// Based on https://docs.oracle.com/javase/8/docs/api/java/util/regex/Pattern.html
private[this] lazy val regexList: Seq[String] = Seq("\\", "\u0000", "\\x", "\t", "\n", "\r",
"\f", "\\a", "\\e", "\\cx", "[", "]", "^", "&", ".", "*", "\\d", "\\D", "\\h", "\\H", "\\s",
"\\S", "\\v", "\\V", "\\w", "\\w", "\\p", "$", "\\b", "\\B", "\\A", "\\G", "\\Z", "\\z", "\\R",
"?", "|", "(", ")", "{", "}", "\\k", "\\Q", "\\E", ":", "!", "<=", ">")
/**
* Provides a way to log an info message about how long an operation took in milliseconds.
*/
def logDuration[T](shouldLog: Boolean, msg: Double => String)(block: => T): T = {
val start = System.nanoTime()
val ret = block
val end = System.nanoTime()
if (shouldLog) {
val timeTaken = (end - start).toDouble / java.util.concurrent.TimeUnit.MILLISECONDS.toNanos(1)
logInfo(msg(timeTaken))
}
ret
}
val gpuCommonTypes = TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128
val pluginSupportedOrderableSig: TypeSig = (gpuCommonTypes + TypeSig.STRUCT).nested()
private[this] def isStructType(dataType: DataType) = dataType match {
case StructType(_) => true
case _ => false
}
private[this] def isArrayOfStructType(dataType: DataType) = dataType match {
case ArrayType(elementType, _) =>
elementType match {
case StructType(_) => true
case _ => false
}
case _ => false
}
// this listener mechanism is global and is intended for use by unit tests only
private lazy val listeners: ListBuffer[GpuOverridesListener] =
new ListBuffer[GpuOverridesListener]()
def addListener(listener: GpuOverridesListener): Unit = {
listeners += listener
}
def removeListener(listener: GpuOverridesListener): Unit = {
listeners -= listener
}
def removeAllListeners(): Unit = {
listeners.clear()
}
private def convertPartToGpuIfPossible(part: Partitioning, conf: RapidsConf): Partitioning = {
part match {
case _: GpuPartitioning => part
case _ =>
val wrapped = wrapPart(part, conf, None)
wrapped.tagForGpu()
if (wrapped.canThisBeReplaced) {
wrapped.convertToGpu()
} else {
part
}
}
}
/**
* Removes unnecessary CPU shuffles that Spark can add to the plan when it does not realize
* a GPU partitioning satisfies a CPU distribution because CPU and GPU expressions are not
* semantically equal.
*/
def removeExtraneousShuffles(plan: SparkPlan, conf: RapidsConf): SparkPlan = {
plan.transformUp {
case cpuShuffle: ShuffleExchangeExec =>
cpuShuffle.child match {
case sqse: ShuffleQueryStageExec =>
GpuTransitionOverrides.getNonQueryStagePlan(sqse) match {
case gpuShuffle: GpuShuffleExchangeExecBase =>
val converted = convertPartToGpuIfPossible(cpuShuffle.outputPartitioning, conf)
if (converted == gpuShuffle.outputPartitioning) {
sqse
} else {
cpuShuffle
}
case _ => cpuShuffle
}
case _ => cpuShuffle
}
}
}
/**
* On some Spark platforms, AQE planning can result in old CPU exchanges being placed in the
* plan even after they have been replaced previously. This looks for subquery reuses of CPU
* exchanges that can be replaced with recently planned GPU exchanges that match the original
* CPU plan
*/
def fixupCpuReusedExchanges(plan: SparkPlan): SparkPlan = {
plan.transformUp {
case bqse: BroadcastQueryStageExec =>
bqse.plan match {
case ReusedExchangeExec(output, b: BroadcastExchangeExec) =>
val cpuCanonical = b.canonicalized.asInstanceOf[BroadcastExchangeExec]
val gpuExchange = ExchangeMappingCache.findGpuExchangeReplacement(cpuCanonical)
gpuExchange.map { g =>
SparkShimImpl.newBroadcastQueryStageExec(bqse, ReusedExchangeExec(output, g))
}.getOrElse(bqse)
case _ => bqse
}
}
}
/**
* Searches the plan for ReusedExchangeExec instances containing a GPU shuffle where the
* output types between the two plan nodes do not match. In such a case the ReusedExchangeExec
* will be updated to match the GPU shuffle output types.
*/
def fixupReusedExchangeOutputs(plan: SparkPlan): SparkPlan = {
def outputTypesMatch(a: Seq[Attribute], b: Seq[Attribute]): Boolean =
a.corresponds(b)((x, y) => x.dataType == y.dataType)
plan.transformUp {
case sqse: ShuffleQueryStageExec =>
sqse.plan match {
case ReusedExchangeExec(output, gsee: GpuShuffleExchangeExecBase) if (
!outputTypesMatch(output, gsee.output)) =>
val newOutput = sqse.plan.output.zip(gsee.output).map { case (c, g) =>
assert(c.isInstanceOf[AttributeReference] && g.isInstanceOf[AttributeReference],
s"Expected AttributeReference but found $c and $g")
AttributeReference(c.name, g.dataType, c.nullable, c.metadata)(c.exprId, c.qualifier)
}
AQEUtils.newReuseInstance(sqse, newOutput)
case _ => sqse
}
}
}
@scala.annotation.tailrec
def extractLit(exp: Expression): Option[Literal] = exp match {
case l: Literal => Some(l)
case a: Alias => extractLit(a.child)
case _ => None
}
def isOfType(l: Option[Literal], t: DataType): Boolean = l.exists(_.dataType == t)
def isStringLit(exp: Expression): Boolean =
isOfType(extractLit(exp), StringType)
def extractStringLit(exp: Expression): Option[String] = extractLit(exp) match {
case Some(Literal(v: UTF8String, StringType)) =>
val s = if (v == null) null else v.toString
Some(s)
case _ => None
}
def isLit(exp: Expression): Boolean = extractLit(exp).isDefined
def isNullLit(lit: Literal): Boolean = {
lit.value == null
}
def isSupportedStringReplacePattern(strLit: String): Boolean = {
// check for regex special characters, except for \u0000 which we can support
!regexList.filterNot(_ == "\u0000").exists(pattern => strLit.contains(pattern))
}
def isSupportedStringReplacePattern(exp: Expression): Boolean = {
extractLit(exp) match {
case Some(Literal(null, _)) => false
case Some(Literal(value: UTF8String, DataTypes.StringType)) =>
val strLit = value.toString
if (strLit.isEmpty) {
false
} else {
// check for regex special characters, except for \u0000 which we can support
isSupportedStringReplacePattern(strLit)
}
case _ => false
}
}
def areAllSupportedTypes(types: DataType*): Boolean = types.forall(isSupportedType(_))
/**
* Is this particular type supported or not.
* @param dataType the type to check
* @param allowNull should NullType be allowed
* @param allowDecimal should DecimalType be allowed
* @param allowBinary should BinaryType be allowed
* @param allowCalendarInterval should CalendarIntervalType be allowed
* @param allowArray should ArrayType be allowed
* @param allowStruct should StructType be allowed
* @param allowStringMaps should a Map[String, String] specifically be allowed
* @param allowMaps should MapType be allowed generically
* @param allowNesting should nested types like array struct and map allow nested types
* within them, or only primitive types.
* @return true if it is allowed else false
*/
def isSupportedType(dataType: DataType,
allowNull: Boolean = false,
allowDecimal: Boolean = false,
allowBinary: Boolean = false,
allowCalendarInterval: Boolean = false,
allowArray: Boolean = false,
allowStruct: Boolean = false,
allowStringMaps: Boolean = false,
allowMaps: Boolean = false,
allowNesting: Boolean = false): Boolean = {
def checkNested(dataType: DataType): Boolean = {
isSupportedType(dataType,
allowNull = allowNull,
allowDecimal = allowDecimal,
allowBinary = allowBinary && allowNesting,
allowCalendarInterval = allowCalendarInterval && allowNesting,
allowArray = allowArray && allowNesting,
allowStruct = allowStruct && allowNesting,
allowStringMaps = allowStringMaps && allowNesting,
allowMaps = allowMaps && allowNesting,
allowNesting = allowNesting)
}
dataType match {
case BooleanType => true
case ByteType => true
case ShortType => true
case IntegerType => true
case LongType => true
case FloatType => true
case DoubleType => true
case DateType => true
case TimestampType =>
TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) &&
TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone)
case StringType => true
case dt: DecimalType if allowDecimal => dt.precision <= DType.DECIMAL64_MAX_PRECISION
case NullType => allowNull
case BinaryType => allowBinary
case CalendarIntervalType => allowCalendarInterval
case ArrayType(elementType, _) if allowArray => checkNested(elementType)
case MapType(StringType, StringType, _) if allowStringMaps => true
case MapType(keyType, valueType, _) if allowMaps =>
checkNested(keyType) && checkNested(valueType)
case StructType(fields) if allowStruct =>
fields.map(_.dataType).forall(checkNested)
case _ => false
}
}
/**
* Checks to see if any expressions are a String Literal
*/
def isAnyStringLit(expressions: Seq[Expression]): Boolean =
expressions.exists(isStringLit)
def isOrContainsFloatingPoint(dataType: DataType): Boolean =
TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == FloatType || dt == DoubleType)
/** Tries to predict whether an adaptive plan will end up with data on the GPU or not. */
def probablyGpuPlan(adaptivePlan: AdaptiveSparkPlanExec, conf: RapidsConf): Boolean = {
def findRootProcessingNode(plan: SparkPlan): SparkPlan = plan match {
case p: AdaptiveSparkPlanExec => findRootProcessingNode(p.executedPlan)
case p: QueryStageExec => findRootProcessingNode(p.plan)
case p: ReusedSubqueryExec => findRootProcessingNode(p.child)
case p: ReusedExchangeExec => findRootProcessingNode(p.child)
case p => p
}
val aqeSubPlan = findRootProcessingNode(adaptivePlan.executedPlan)
aqeSubPlan match {
case _: GpuExec =>
// plan is already on the GPU
true
case p =>
// see if the root processing node of the current subplan will translate to the GPU
val meta = GpuOverrides.wrapAndTagPlan(p, conf)
meta.canThisBeReplaced
}
}
def checkAndTagFloatAgg(dataType: DataType, conf: RapidsConf, meta: RapidsMeta[_,_,_]): Unit = {
if (!conf.isFloatAggEnabled && isOrContainsFloatingPoint(dataType)) {
meta.willNotWorkOnGpu("the GPU will aggregate floating point values in" +
" parallel and the result is not always identical each time. This can cause" +
" some Spark queries to produce an incorrect answer if the value is computed" +
" more than once as part of the same query. To enable this anyways set" +
s" ${RapidsConf.ENABLE_FLOAT_AGG} to true.")
}
}
/**
* Helper function specific to ANSI mode for the aggregate functions that should
* fallback, since we don't have the same overflow checks that Spark provides in
* the CPU
* @param checkType Something other than `None` triggers logic to detect whether
* the agg should fallback in ANSI mode. Otherwise (None), it's
* an automatic fallback.
* @param meta agg expression meta
*/
def checkAndTagAnsiAgg(checkType: Option[DataType], meta: AggExprMeta[_]): Unit = {
val failOnError = SQLConf.get.ansiEnabled
if (failOnError) {
if (checkType.isDefined) {
val typeToCheck = checkType.get
val failedType = typeToCheck match {
case _: DecimalType | LongType | IntegerType | ShortType | ByteType => true
case _ => false
}
if (failedType) {
meta.willNotWorkOnGpu(
s"ANSI mode not supported for ${meta.expr} with $typeToCheck result type")
}
} else {
// Average falls into this category, where it produces Doubles, but
// internally it uses Double and Long, and Long could overflow (technically)
// and failOnError given that it is based on catalyst Add.
meta.willNotWorkOnGpu(
s"ANSI mode not supported for ${meta.expr}")
}
}
}
def expr[INPUT <: Expression](
desc: String,
pluginChecks: ExprChecks,
doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule)
=> BaseExprMeta[INPUT])
(implicit tag: ClassTag[INPUT]): ExprRule[INPUT] = {
assert(desc != null)
assert(doWrap != null)
new ExprRule[INPUT](doWrap, desc, Some(pluginChecks), tag)
}
def scan[INPUT <: Scan](
desc: String,
doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule)
=> ScanMeta[INPUT])
(implicit tag: ClassTag[INPUT]): ScanRule[INPUT] = {
assert(desc != null)
assert(doWrap != null)
new ScanRule[INPUT](doWrap, desc, tag)
}
def part[INPUT <: Partitioning](
desc: String,
checks: PartChecks,
doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule)
=> PartMeta[INPUT])
(implicit tag: ClassTag[INPUT]): PartRule[INPUT] = {
assert(desc != null)
assert(doWrap != null)
new PartRule[INPUT](doWrap, desc, Some(checks), tag)
}
/**
* Create an exec rule that should never be replaced, because it is something that should always
* run on the CPU, or should just be ignored totally for what ever reason.
*/
def neverReplaceExec[INPUT <: SparkPlan](desc: String)
(implicit tag: ClassTag[INPUT]): ExecRule[INPUT] = {
assert(desc != null)
def doWrap(
exec: INPUT,
conf: RapidsConf,
p: Option[RapidsMeta[_, _, _]],
cc: DataFromReplacementRule) =
new DoNotReplaceOrWarnSparkPlanMeta[INPUT](exec, conf, p)
new ExecRule[INPUT](doWrap, desc, None, tag).invisible()
}
def exec[INPUT <: SparkPlan](
desc: String,
pluginChecks: ExecChecks,
doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule)
=> SparkPlanMeta[INPUT])
(implicit tag: ClassTag[INPUT]): ExecRule[INPUT] = {
assert(desc != null)
assert(doWrap != null)
new ExecRule[INPUT](doWrap, desc, Some(pluginChecks), tag)
}
def dataWriteCmd[INPUT <: DataWritingCommand](
desc: String,
doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule)
=> DataWritingCommandMeta[INPUT])
(implicit tag: ClassTag[INPUT]): DataWritingCommandRule[INPUT] = {
assert(desc != null)
assert(doWrap != null)
new DataWritingCommandRule[INPUT](doWrap, desc, tag)
}
def wrapExpr[INPUT <: Expression](
expr: INPUT,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]]): BaseExprMeta[INPUT] =
expressions.get(expr.getClass)
.map(r => r.wrap(expr, conf, parent, r).asInstanceOf[BaseExprMeta[INPUT]])
.getOrElse(new RuleNotFoundExprMeta(expr, conf, parent))
lazy val fileFormats: Map[FileFormatType, Map[FileFormatOp, FileFormatChecks]] = Map(
(CsvFormatType, FileFormatChecks(
cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 +
GpuTypeShims.additionalCsvSupportedTypes,
cudfWrite = TypeSig.none,
sparkSig = TypeSig.cpuAtomics)),
(HiveDelimitedTextFormatType, FileFormatChecks(
// Keep the supported types in sync with GpuHiveTextFileUtils.isSupportedType.
cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128,
cudfWrite = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128,
sparkSig = TypeSig.all)),
(DeltaFormatType, FileFormatChecks(
cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT +
TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY +
GpuTypeShims.additionalParquetSupportedTypes).nested(),
cudfWrite = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT +
TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY +
GpuTypeShims.additionalParquetSupportedTypes).nested(),
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested())),
(ParquetFormatType, FileFormatChecks(
cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT +
TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY +
GpuTypeShims.additionalParquetSupportedTypes).nested(),
cudfWrite = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT +
TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY +
GpuTypeShims.additionalParquetSupportedTypes).nested(),
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested())),
(OrcFormatType, FileFormatChecks(
cudfRead = (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.DECIMAL_128 +
TypeSig.STRUCT + TypeSig.MAP).nested(),
cudfWrite = (TypeSig.commonCudfTypes + TypeSig.ARRAY +
// Note Map is not put into nested, now CUDF only support single level map
TypeSig.STRUCT + TypeSig.DECIMAL_128).nested() + TypeSig.MAP,
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT).nested())),
(JsonFormatType, FileFormatChecks(
cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128,
cudfWrite = TypeSig.none,
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT).nested())),
(AvroFormatType, FileFormatChecks(
cudfRead = TypeSig.BOOLEAN + TypeSig.BYTE + TypeSig.SHORT + TypeSig.INT + TypeSig.LONG +
TypeSig.FLOAT + TypeSig.DOUBLE + TypeSig.STRING,
cudfWrite = TypeSig.none,
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT).nested())),
(IcebergFormatType, FileFormatChecks(
cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.BINARY +
TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(),
cudfWrite = TypeSig.none,
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.BINARY + TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested())))
val commonExpressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq(
expr[Literal](
"Holds a static value from the query",
ExprChecks.projectAndAst(
TypeSig.astTypes,
(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.CALENDAR
+ TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT)
.nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 +
TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT),
TypeSig.all),
(lit, conf, p, r) => new LiteralExprMeta(lit, conf, p, r)),
expr[Signum](
"Returns -1.0, 0.0 or 1.0 as expr is negative, 0 or positive",
ExprChecks.mathUnary,
(a, conf, p, r) => new UnaryExprMeta[Signum](a, conf, p, r) {
override def convertToGpu(child: Expression): GpuExpression = GpuSignum(child)
}),
expr[Alias](
"Gives a column a name",
ExprChecks.unaryProjectAndAstInputMatchesOutput(
TypeSig.astTypes + GpuTypeShims.additionalCommonOperatorSupportedTypes,
(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT
+ TypeSig.DECIMAL_128 + TypeSig.BINARY
+ GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(),
TypeSig.all),
(a, conf, p, r) => new UnaryAstExprMeta[Alias](a, conf, p, r) {
override def convertToGpu(child: Expression): GpuExpression =
GpuAlias(child, a.name)(a.exprId, a.qualifier, a.explicitMetadata)
}),
expr[AttributeReference](
"References an input column",
ExprChecks.projectAndAst(
TypeSig.astTypes + GpuTypeShims.additionalArithmeticSupportedTypes,
(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY +
TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY +
GpuTypeShims.additionalArithmeticSupportedTypes).nested(),
TypeSig.all),
(att, conf, p, r) => new BaseExprMeta[AttributeReference](att, conf, p, r) {
// This is the only NOOP operator. It goes away when things are bound
override def convertToGpu(): Expression = att
// There are so many of these that we don't need to print them out, unless it
// will not work on the GPU
override def print(append: StringBuilder, depth: Int, all: Boolean): Unit = {
if (!this.canThisBeReplaced || cannotRunOnGpuBecauseOfSparkPlan) {
super.print(append, depth, all)
}
}
}),
expr[ToDegrees](
"Converts radians to degrees",
ExprChecks.mathUnary,
(a, conf, p, r) => new UnaryExprMeta[ToDegrees](a, conf, p, r) {
override def convertToGpu(child: Expression): GpuToDegrees = GpuToDegrees(child)
}),
expr[ToRadians](
"Converts degrees to radians",
ExprChecks.mathUnary,
(a, conf, p, r) => new UnaryExprMeta[ToRadians](a, conf, p, r) {
override def convertToGpu(child: Expression): GpuToRadians = GpuToRadians(child)
}),
expr[WindowExpression](
"Calculates a return value for every input row of a table based on a group (or " +
"\"window\") of rows",
ExprChecks.windowOnly(
TypeSig.all,
TypeSig.all,
Seq(ParamCheck("windowFunction", TypeSig.all, TypeSig.all),
ParamCheck("windowSpec",
TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_64,
TypeSig.numericAndInterval))),
(windowExpression, conf, p, r) => new GpuWindowExpressionMeta(windowExpression, conf, p, r)),
expr[SpecifiedWindowFrame](
"Specification of the width of the group (or \"frame\") of input rows " +
"around which a window function is evaluated",
ExprChecks.projectOnly(
TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral,
TypeSig.numericAndInterval,
Seq(
ParamCheck("lower",
TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128 +
TypeSig.FLOAT + TypeSig.DOUBLE,
TypeSig.numericAndInterval),
ParamCheck("upper",
TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128 +
TypeSig.FLOAT + TypeSig.DOUBLE,
TypeSig.numericAndInterval))),
(windowFrame, conf, p, r) => new GpuSpecifiedWindowFrameMeta(windowFrame, conf, p, r) ),
expr[WindowSpecDefinition](
"Specification of a window function, indicating the partitioning-expression, the row " +
"ordering, and the width of the window",
WindowSpecCheck,
(windowSpec, conf, p, r) => new GpuWindowSpecDefinitionMeta(windowSpec, conf, p, r)),
expr[CurrentRow.type](
"Special boundary for a window frame, indicating stopping at the current row",
ExprChecks.projectOnly(TypeSig.NULL, TypeSig.NULL),
(currentRow, conf, p, r) => new ExprMeta[CurrentRow.type](currentRow, conf, p, r) {
override def convertToGpu(): GpuExpression = GpuSpecialFrameBoundary(currentRow)
}),
expr[UnboundedPreceding.type](
"Special boundary for a window frame, indicating all rows preceding the current row",
ExprChecks.projectOnly(TypeSig.NULL, TypeSig.NULL),
(unboundedPreceding, conf, p, r) =>
new ExprMeta[UnboundedPreceding.type](unboundedPreceding, conf, p, r) {
override def convertToGpu(): GpuExpression = GpuSpecialFrameBoundary(unboundedPreceding)
}),