-
Notifications
You must be signed in to change notification settings - Fork 232
/
Spark31XShims.scala
567 lines (512 loc) · 26.5 KB
/
Spark31XShims.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
/*
* Copyright (c) 2021-2022, 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.shims.v2
import java.nio.ByteBuffer
import com.nvidia.spark.InMemoryTableScanMeta
import com.nvidia.spark.rapids._
import org.apache.arrow.memory.ReferenceManager
import org.apache.arrow.vector.ValueVector
import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.rapids.shims.v2.GpuShuffleExchangeExec
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.Average
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning}
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec}
import org.apache.spark.sql.execution.joins.EmptyHashedRelation
import org.apache.spark.sql.execution.python._
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.rapids._
import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase
import org.apache.spark.sql.rapids.execution.python._
import org.apache.spark.sql.rapids.execution.python.shims.v2._
import org.apache.spark.sql.rapids.shims.v2.{GpuColumnarToRowTransitionExec, HadoopFSUtilsShim}
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
// 31x nondb shims, used by 311cdh and 31x
abstract class Spark31XShims extends Spark301until320Shims with Logging {
override def int96ParquetRebaseRead(conf: SQLConf): String =
conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ)
override def int96ParquetRebaseWrite(conf: SQLConf): String =
conf.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE)
override def int96ParquetRebaseReadKey: String =
SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key
override def int96ParquetRebaseWriteKey: String =
SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key
override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = {
Some(broadcastModeTransform(mode, Array.empty)).filter(isEmptyRelation)
}
override def isEmptyRelation(relation: Any): Boolean = relation match {
case EmptyHashedRelation => true
case arr: Array[InternalRow] if arr.isEmpty => true
case _ => false
}
override def hasSeparateINT96RebaseConf: Boolean = true
override def getScalaUDFAsExpression(
function: AnyRef,
dataType: DataType,
children: Seq[Expression],
inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil,
outputEncoder: Option[ExpressionEncoder[_]] = None,
udfName: Option[String] = None,
nullable: Boolean = true,
udfDeterministic: Boolean = true): Expression = {
ScalaUDF(function, dataType, children, inputEncoders, outputEncoder, udfName, nullable,
udfDeterministic)
}
override def getMapSizesByExecutorId(
shuffleId: Int,
startMapIndex: Int,
endMapIndex: Int,
startPartition: Int,
endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = {
SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId,
startMapIndex, endMapIndex, startPartition, endPartition)
}
override def getFileSourceMaxMetadataValueLength(sqlConf: SQLConf): Int =
sqlConf.maxMetadataStringLength
override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq(
GpuOverrides.expr[Cast](
"Convert a column of one type of data into another type",
new CastChecks(),
(cast, conf, p, r) => new CastExprMeta[Cast](cast,
SparkSession.active.sessionState.conf.ansiEnabled, conf, p, r,
doFloatToIntCheck = true, stringToAnsiDate = false)),
GpuOverrides.expr[AnsiCast](
"Convert a column of one type of data into another type",
new CastChecks {
import TypeSig._
// nullChecks are the same
override val booleanChecks: TypeSig = integral + fp + BOOLEAN + STRING + DECIMAL_128
override val sparkBooleanSig: TypeSig = cpuNumeric + BOOLEAN + STRING
override val integralChecks: TypeSig = gpuNumeric + BOOLEAN + STRING
override val sparkIntegralSig: TypeSig = cpuNumeric + BOOLEAN + STRING
override val fpChecks: TypeSig = (gpuNumeric + BOOLEAN + STRING)
.withPsNote(TypeEnum.STRING, fpToStringPsNote)
override val sparkFpSig: TypeSig = cpuNumeric + BOOLEAN + STRING
override val dateChecks: TypeSig = TIMESTAMP + DATE + STRING
override val sparkDateSig: TypeSig = TIMESTAMP + DATE + STRING
override val timestampChecks: TypeSig = TIMESTAMP + DATE + STRING
override val sparkTimestampSig: TypeSig = TIMESTAMP + DATE + STRING
// stringChecks are the same
// binaryChecks are the same
override val decimalChecks: TypeSig = gpuNumeric + STRING
override val sparkDecimalSig: TypeSig = cpuNumeric + BOOLEAN + STRING
// calendarChecks are the same
override val arrayChecks: TypeSig =
ARRAY.nested(commonCudfTypes + DECIMAL_128 + NULL + ARRAY + BINARY + STRUCT) +
psNote(TypeEnum.ARRAY, "The array's child type must also support being cast to " +
"the desired child type")
override val sparkArraySig: TypeSig = ARRAY.nested(all)
override val mapChecks: TypeSig =
MAP.nested(commonCudfTypes + DECIMAL_128 + NULL + ARRAY + BINARY + STRUCT + MAP) +
psNote(TypeEnum.MAP, "the map's key and value must also support being cast to the " +
"desired child types")
override val sparkMapSig: TypeSig = MAP.nested(all)
override val structChecks: TypeSig =
STRUCT.nested(commonCudfTypes + DECIMAL_128 + NULL + ARRAY + BINARY + STRUCT) +
psNote(TypeEnum.STRUCT, "the struct's children must also support being cast to the " +
"desired child type(s)")
override val sparkStructSig: TypeSig = STRUCT.nested(all)
override val udtChecks: TypeSig = none
override val sparkUdtSig: TypeSig = UDT
},
(cast, conf, p, r) => new CastExprMeta[AnsiCast](cast, ansiEnabled = true, conf = conf,
parent = p, rule = r, doFloatToIntCheck = true, stringToAnsiDate = false)),
GpuOverrides.expr[Average](
"Average aggregate operator",
ExprChecks.fullAgg(
TypeSig.DOUBLE + TypeSig.DECIMAL_128,
TypeSig.DOUBLE + TypeSig.DECIMAL_128,
Seq(ParamCheck("input",
TypeSig.integral + TypeSig.fp + TypeSig.DECIMAL_128,
TypeSig.cpuNumeric))),
(a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) {
override def tagAggForGpu(): Unit = {
// For Decimal Average the SUM adds a precision of 10 to avoid overflowing
// then it divides by the count with an output scale that is 4 more than the input
// scale. With how our divide works to match Spark, this means that we will need a
// precision of 5 more. So 38 - 10 - 5 = 23
val dataType = a.child.dataType
dataType match {
case dt: DecimalType =>
if (dt.precision > 23) {
if (conf.needDecimalGuarantees) {
willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " +
s"a precision large than 23. The current precision is ${dt.precision}")
} else {
logWarning("Decimal overflow guarantees disabled for " +
s"Average(${a.child.dataType}) produces ${dt} with an " +
s"intermediate precision of ${dt.precision + 15}")
}
}
case _ => // NOOP
}
GpuOverrides.checkAndTagFloatAgg(dataType, conf, this)
}
override def convertToGpu(childExprs: Seq[Expression]): GpuExpression =
GpuAverage(childExprs.head)
// Average is not supported in ANSI mode right now, no matter the type
override val ansiTypeToCheck: Option[DataType] = None
}),
GpuOverrides.expr[Abs](
"Absolute value",
ExprChecks.unaryProjectAndAstInputMatchesOutput(
TypeSig.implicitCastsAstTypes, TypeSig.gpuNumeric,
TypeSig.cpuNumeric),
(a, conf, p, r) => new UnaryAstExprMeta[Abs](a, conf, p, r) {
// ANSI support for ABS was added in 3.2.0 SPARK-33275
override def convertToGpu(child: Expression): GpuExpression = GpuAbs(child, false)
}),
GpuOverrides.expr[RegExpReplace](
"RegExpReplace support for string literal input patterns",
ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING,
Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING),
ParamCheck("regex", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING),
ParamCheck("rep", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING),
ParamCheck("pos", TypeSig.lit(TypeEnum.INT)
.withPsNote(TypeEnum.INT, "only a value of 1 is supported"),
TypeSig.lit(TypeEnum.INT)))),
(a, conf, p, r) => new GpuRegExpReplaceMeta(a, conf, p, r)).disabledByDefault(
"the implementation is not 100% compatible. " +
"See the compatibility guide for more information."),
// Spark 3.1.1-specific LEAD expression, using custom OffsetWindowFunctionMeta.
GpuOverrides.expr[Lead](
"Window function that returns N entries ahead of this one",
ExprChecks.windowOnly(
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL +
TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all,
Seq(
ParamCheck("input",
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 +
TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all),
ParamCheck("offset", TypeSig.INT, TypeSig.INT),
ParamCheck("default",
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL +
TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all)
)
),
(lead, conf, p, r) => new OffsetWindowFunctionMeta[Lead](lead, conf, p, r) {
override def convertToGpu(): GpuExpression =
GpuLead(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu())
}),
// Spark 3.1.1-specific LAG expression, using custom OffsetWindowFunctionMeta.
GpuOverrides.expr[Lag](
"Window function that returns N entries behind this one",
ExprChecks.windowOnly(
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL +
TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all,
Seq(
ParamCheck("input",
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 +
TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all),
ParamCheck("offset", TypeSig.INT, TypeSig.INT),
ParamCheck("default",
(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL +
TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all)
)
),
(lag, conf, p, r) => new OffsetWindowFunctionMeta[Lag](lag, conf, p, r) {
override def convertToGpu(): GpuExpression = {
GpuLag(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu())
}
}),
GpuOverrides.expr[GetArrayItem](
"Gets the field at `ordinal` in the Array",
ExprChecks.binaryProject(
(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL +
TypeSig.DECIMAL_128 + TypeSig.MAP).nested(),
TypeSig.all,
("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY +
TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP),
TypeSig.ARRAY.nested(TypeSig.all)),
("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT)),
(in, conf, p, r) => new GpuGetArrayItemMeta(in, conf, p, r){
override def convertToGpu(arr: Expression, ordinal: Expression): GpuExpression =
GpuGetArrayItem(arr, ordinal, SQLConf.get.ansiEnabled)
}),
GpuOverrides.expr[GetMapValue](
"Gets Value from a Map based on a key",
ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all,
("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)),
("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all)),
(in, conf, p, r) => new GpuGetMapValueMeta(in, conf, p, r){
override def convertToGpu(map: Expression, key: Expression): GpuExpression =
GpuGetMapValue(map, key, SQLConf.get.ansiEnabled)
}),
GpuOverrides.expr[ElementAt](
"Returns element of array at given(1-based) index in value if column is array. " +
"Returns value for the given key in value if column is map.",
ExprChecks.binaryProject(
(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL +
TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), TypeSig.all,
("array/map", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY +
TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP) +
TypeSig.MAP.nested(TypeSig.STRING)
.withPsNote(TypeEnum.MAP ,"If it's map, only string is supported."),
TypeSig.ARRAY.nested(TypeSig.all) + TypeSig.MAP.nested(TypeSig.all)),
("index/key", (TypeSig.lit(TypeEnum.INT) + TypeSig.lit(TypeEnum.STRING))
.withPsNote(TypeEnum.INT, "ints are only supported as array indexes, " +
"not as maps keys")
.withPsNote(TypeEnum.STRING, "strings are only supported as map keys, " +
"not array indexes"),
TypeSig.all)),
(in, conf, p, r) => new BinaryExprMeta[ElementAt](in, conf, p, r) {
override def tagExprForGpu(): Unit = {
// To distinguish the supported nested type between Array and Map
val checks = in.left.dataType match {
case _: MapType =>
// Match exactly with the checks for GetMapValue
ExprChecks.binaryProject(TypeSig.STRING, TypeSig.all,
("map", TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.all)),
("key", TypeSig.lit(TypeEnum.STRING), TypeSig.all))
case _: ArrayType =>
// Match exactly with the checks for GetArrayItem
ExprChecks.binaryProject(
(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL +
TypeSig.DECIMAL_128 + TypeSig.MAP).nested(),
TypeSig.all,
("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY +
TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP),
TypeSig.ARRAY.nested(TypeSig.all)),
("ordinal", TypeSig.lit(TypeEnum.INT), TypeSig.INT))
case _ => throw new IllegalStateException("Only Array or Map is supported as input.")
}
checks.tag(this)
}
override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = {
GpuElementAt(lhs, rhs, SQLConf.get.ansiEnabled)
}
}),
GpuScalaUDFMeta.exprMeta
).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap
override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = {
Seq(
GpuOverrides.exec[WindowInPandasExec](
"The backend for Window Aggregation Pandas UDF, Accelerates the data transfer between" +
" the Java process and the Python process. It also supports scheduling GPU resources" +
" for the Python process when enabled. For now it only supports row based window frame.",
ExecChecks(
(TypeSig.commonCudfTypes + TypeSig.ARRAY).nested(TypeSig.commonCudfTypes),
TypeSig.all),
(winPy, conf, p, r) => new GpuWindowInPandasExecMetaBase(winPy, conf, p, r) {
override val windowExpressions: Seq[BaseExprMeta[NamedExpression]] =
winPy.windowExpression.map(GpuOverrides.wrapExpr(_, conf, Some(this)))
override def convertToGpu(): GpuExec = {
GpuWindowInPandasExec(
windowExpressions.map(_.convertToGpu()),
partitionSpec.map(_.convertToGpu()),
// leave ordering expression on the CPU, it's not used for GPU computation
winPy.orderSpec,
childPlans.head.convertIfNeeded()
)(winPy.partitionSpec)
}
}).disabledByDefault("it only supports row based frame for now"),
GpuOverrides.exec[FileSourceScanExec](
"Reading data from files, often from Hive tables",
ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP +
TypeSig.ARRAY + TypeSig.DECIMAL_128).nested(), TypeSig.all),
(fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) {
// Replaces SubqueryBroadcastExec inside dynamic pruning filters with GPU counterpart
// if possible. Instead regarding filters as childExprs of current Meta, we create
// a new meta for SubqueryBroadcastExec. The reason is that the GPU replacement of
// FileSourceScan is independent from the replacement of the partitionFilters. It is
// possible that the FileSourceScan is on the CPU, while the dynamic partitionFilters
// are on the GPU. And vice versa.
private lazy val partitionFilters = {
val convertBroadcast = (bc: SubqueryBroadcastExec) => {
val meta = GpuOverrides.wrapAndTagPlan(bc, conf)
meta.tagForExplain()
meta.convertIfNeeded().asInstanceOf[BaseSubqueryExec]
}
wrapped.partitionFilters.map { filter =>
filter.transformDown {
case dpe @ DynamicPruningExpression(inSub: InSubqueryExec) =>
inSub.plan match {
case bc: SubqueryBroadcastExec =>
dpe.copy(inSub.copy(plan = convertBroadcast(bc)))
case reuse @ ReusedSubqueryExec(bc: SubqueryBroadcastExec) =>
dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc))))
case _ =>
dpe
}
}
}
}
// partition filters and data filters are not run on the GPU
override val childExprs: Seq[ExprMeta[_]] = Seq.empty
override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this)
override def convertToCpu(): SparkPlan = {
wrapped.copy(partitionFilters = partitionFilters)
}
override def convertToGpu(): GpuExec = {
val sparkSession = wrapped.relation.sparkSession
val options = wrapped.relation.options
val location = replaceWithAlluxioPathIfNeeded(
conf,
wrapped.relation,
partitionFilters,
wrapped.dataFilters)
val newRelation = HadoopFsRelation(
location,
wrapped.relation.partitionSchema,
wrapped.relation.dataSchema,
wrapped.relation.bucketSpec,
GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat),
options)(sparkSession)
GpuFileSourceScanExec(
newRelation,
wrapped.output,
wrapped.requiredSchema,
partitionFilters,
wrapped.optionalBucketSet,
wrapped.optionalNumCoalescedBuckets,
wrapped.dataFilters,
wrapped.tableIdentifier,
wrapped.disableBucketedScan)(conf)
}
}),
GpuOverrides.exec[InMemoryTableScanExec](
"Implementation of InMemoryTableScanExec to use GPU accelerated Caching",
ExecChecks((TypeSig.commonCudfTypes + TypeSig.DECIMAL_64 + TypeSig.STRUCT
+ TypeSig.ARRAY + TypeSig.MAP).nested(), TypeSig.all),
(scan, conf, p, r) => new InMemoryTableScanMeta(scan, conf, p, r)),
GpuOverrides.exec[ArrowEvalPythonExec](
"The backend of the Scalar Pandas UDFs. Accelerates the data transfer between the" +
" Java process and the Python process. It also supports scheduling GPU resources" +
" for the Python process when enabled",
ExecChecks(
(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all),
(e, conf, p, r) =>
new SparkPlanMeta[ArrowEvalPythonExec](e, conf, p, r) {
val udfs: Seq[BaseExprMeta[PythonUDF]] =
e.udfs.map(GpuOverrides.wrapExpr(_, conf, Some(this)))
val resultAttrs: Seq[BaseExprMeta[Attribute]] =
e.resultAttrs.map(GpuOverrides.wrapExpr(_, conf, Some(this)))
override val childExprs: Seq[BaseExprMeta[_]] = udfs ++ resultAttrs
override def replaceMessage: String = "partially run on GPU"
override def noReplacementPossibleMessage(reasons: String): String =
s"cannot run even partially on the GPU because $reasons"
override def convertToGpu(): GpuExec =
GpuArrowEvalPythonExec(udfs.map(_.convertToGpu()).asInstanceOf[Seq[GpuPythonUDF]],
resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]],
childPlans.head.convertIfNeeded(),
e.evalType)
}),
GpuOverrides.exec[MapInPandasExec](
"The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" +
" Java process and the Python process. It also supports scheduling GPU resources" +
" for the Python process when enabled.",
ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(),
TypeSig.all),
(mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)),
GpuOverrides.exec[FlatMapGroupsInPandasExec](
"The backend for Flat Map Groups Pandas UDF, Accelerates the data transfer between the" +
" Java process and the Python process. It also supports scheduling GPU resources" +
" for the Python process when enabled.",
ExecChecks(TypeSig.commonCudfTypes, TypeSig.all),
(flatPy, conf, p, r) => new GpuFlatMapGroupsInPandasExecMeta(flatPy, conf, p, r)),
GpuOverrides.exec[AggregateInPandasExec](
"The backend for an Aggregation Pandas UDF, this accelerates the data transfer between" +
" the Java process and the Python process. It also supports scheduling GPU resources" +
" for the Python process when enabled.",
ExecChecks(TypeSig.commonCudfTypes, TypeSig.all),
(aggPy, conf, p, r) => new GpuAggregateInPandasExecMeta(aggPy, conf, p, r))
).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap
}
override def getGpuColumnarToRowTransition(plan: SparkPlan,
exportColumnRdd: Boolean): GpuColumnarToRowExecParent = {
val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
val serClass = ShimLoader.loadClass(serName)
if (serClass == classOf[com.nvidia.spark.ParquetCachedBatchSerializer]) {
GpuColumnarToRowTransitionExec(plan, exportColumnRdd)
} else {
GpuColumnarToRowExec(plan, exportColumnRdd)
}
}
override def getGpuShuffleExchangeExec(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
cpuOutputPartitioning: Partitioning,
cpuShuffle: Option[ShuffleExchangeExec]): GpuShuffleExchangeExecBase = {
val shuffleOrigin = cpuShuffle.map(_.shuffleOrigin).getOrElse(ENSURE_REQUIREMENTS)
GpuShuffleExchangeExec(gpuOutputPartitioning, child, shuffleOrigin)(cpuOutputPartitioning)
}
override def getGpuShuffleExchangeExec(
queryStage: ShuffleQueryStageExec): GpuShuffleExchangeExecBase = {
queryStage.shuffle.asInstanceOf[GpuShuffleExchangeExecBase]
}
override def sortOrder(
child: Expression,
direction: SortDirection,
nullOrdering: NullOrdering): SortOrder = SortOrder(child, direction, nullOrdering, Seq.empty)
override def copySortOrderWithNewChild(s: SortOrder, child: Expression) = {
s.copy(child = child)
}
override def shouldIgnorePath(path: String): Boolean = {
HadoopFSUtilsShim.shouldIgnorePath(path)
}
override def getLegacyComplexTypeToString(): Boolean = {
SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING)
}
// Arrow version changed between Spark versions
override def getArrowDataBuf(vec: ValueVector): (ByteBuffer, ReferenceManager) = {
val arrowBuf = vec.getDataBuffer()
(arrowBuf.nioBuffer(), arrowBuf.getReferenceManager)
}
/** matches SPARK-33008 fix in 3.1.1 */
override def shouldFailDivByZero(): Boolean = SQLConf.get.ansiEnabled
/** dropped by SPARK-34234 */
override def attachTreeIfSupported[TreeType <: TreeNode[_], A](
tree: TreeType,
msg: String)(
f: => A
): A = {
attachTree(tree, msg)(f)
}
override def shouldFallbackOnAnsiTimestamp(): Boolean = SQLConf.get.ansiEnabled
override def getAdaptiveInputPlan(adaptivePlan: AdaptiveSparkPlanExec): SparkPlan = {
adaptivePlan.inputPlan
}
override def getLegacyStatisticalAggregate(): Boolean =
SQLConf.get.legacyStatisticalAggregate
override def hasCastFloatTimestampUpcast: Boolean = false
override def isCastingStringToNegDecimalScaleSupported: Boolean = false
override def supportsColumnarAdaptivePlans: Boolean = false
override def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan = {
// When the input is an adaptive plan we do not get to see the GPU version until
// the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the
// final operator and we can bypass this to keep the data columnar by inserting
// the [[AvoidAdaptiveTransitionToRow]] operator here
AvoidAdaptiveTransitionToRow(GpuRowToColumnarExec(a, goal))
}
}