From 3ad53c1af95cc6ae3747d5c35eacc7bdb1169050 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 18 Jun 2020 15:21:14 -0500 Subject: [PATCH 1/4] Remove GpuAttributeReference and GpuSortOrder --- .../spark/rapids/api/ApiValidation.scala | 2 - docs/dev/README.md | 26 +- .../src/main/python/cache_test.py | 7 +- .../spark/rapids/GpuBoundAttribute.scala | 58 ++-- .../com/nvidia/spark/rapids/GpuCast.scala | 4 +- .../nvidia/spark/rapids/GpuExpandExec.scala | 25 +- .../nvidia/spark/rapids/GpuExpressions.scala | 13 +- .../nvidia/spark/rapids/GpuGenerateExec.scala | 11 +- .../com/nvidia/spark/rapids/GpuHashJoin.scala | 2 +- .../spark/rapids/GpuHashPartitioning.scala | 8 +- .../nvidia/spark/rapids/GpuOverrides.scala | 284 +++++++++--------- .../spark/rapids/GpuRangePartitioner.scala | 15 +- .../spark/rapids/GpuRangePartitioning.scala | 17 +- .../spark/rapids/GpuShuffleExchangeExec.scala | 13 +- .../rapids/GpuShuffledHashJoinExec.scala | 17 +- .../spark/rapids/GpuSinglePartitioning.scala | 2 +- .../com/nvidia/spark/rapids/GpuSortExec.scala | 69 +---- .../spark/rapids/GpuSortMergeJoinExec.scala | 10 +- .../spark/rapids/GpuTransitionOverrides.scala | 14 +- .../nvidia/spark/rapids/GpuWindowExec.scala | 9 +- .../spark/rapids/GpuWindowExpression.scala | 45 +-- .../com/nvidia/spark/rapids/RapidsMeta.scala | 40 ++- .../com/nvidia/spark/rapids/SortUtils.scala | 8 +- .../com/nvidia/spark/rapids/aggregate.scala | 84 +++--- .../spark/rapids/basicPhysicalOperators.scala | 16 +- .../spark/rapids/conditionalExpressions.scala | 34 +-- .../spark/rapids/constraintExpressions.scala | 9 +- .../spark/rapids/namedExpressions.scala | 141 +-------- .../nvidia/spark/rapids/nullExpressions.scala | 17 +- .../spark/sql/rapids/AggregateFunctions.scala | 120 ++++---- .../sql/rapids/GpuFileFormatDataWriter.scala | 4 +- .../sql/rapids/GpuFileFormatWriter.scala | 14 +- .../sql/rapids/NormalizeFloatingNumbers.scala | 4 +- .../apache/spark/sql/rapids/arithmetic.scala | 22 +- .../org/apache/spark/sql/rapids/bitwise.scala | 14 +- .../expressions/GpuRandomExpressions.scala | 4 +- .../sql/rapids/datetimeExpressions.scala | 72 ++--- .../execution/GpuBroadcastHashJoinExec.scala | 17 +- .../spark/sql/rapids/mathExpressions.scala | 59 ++-- .../spark/sql/rapids/stringFunctions.scala | 68 ++--- 40 files changed, 621 insertions(+), 777 deletions(-) diff --git a/api_validation/src/main/scala/com/nvidia/spark/rapids/api/ApiValidation.scala b/api_validation/src/main/scala/com/nvidia/spark/rapids/api/ApiValidation.scala index cb109e13a95..6bdc79e89b7 100644 --- a/api_validation/src/main/scala/com/nvidia/spark/rapids/api/ApiValidation.scala +++ b/api_validation/src/main/scala/com/nvidia/spark/rapids/api/ApiValidation.scala @@ -101,8 +101,6 @@ object ApiValidation extends Logging { "com.nvidia.spark.rapids.GpuExpression", "org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression" -> "org.apache.spark.sql.rapids.GpuAggregateExpression", - "org.apache.spark.sql.catalyst.expressions.AttributeReference" -> - "com.nvidia.spark.rapids.GpuAttributeReference", "org.apache.spark.sql.execution.command.DataWritingCommand" -> "com.nvidia.spark.rapids.GpuDataWritingCommand", "org.apache.spark.sql.execution.joins.BuildSide" -> diff --git a/docs/dev/README.md b/docs/dev/README.md index 99a55e8548b..2e78e25296e 100644 --- a/docs/dev/README.md +++ b/docs/dev/README.md @@ -172,20 +172,24 @@ On startup use: `--conf [conf key]=[conf value]`. For example: [...] ``` -### Input Expressions and Output Attributes -For the common case of nodes expecting GPU columnar data as input and +### Expressions +For nodes expecting GPU columnar data as input and producing GPU columnar data as output, the child node(s) passed to the case -class constructor should have the `GpuExpression` type. - -Note that any attribute references that appear in the node's `output` should -*not* be `GpuAttributeReference` instances but rather normal -`AttributeReference` instances. Using `GpuAttributeReference` instances in -the node's `output` can cause these attributes to find their way into -non-plugin nodes in the plan that are unaware of `GpuAttributeReference`, -causing an error when planning the query. +class constructor should have the `Expression` type. This is a little +odd because they should all be instances of `GpuExpression` except for +`AttributeReference` and `SortOrder`. This is needed because `AttributeReference` +is weaved into a lot of the magic that is build into Spark expressions. +`SortOrder` is similar as spark itself will insert `SortOrder` instances into +the plan automatically in many cases. These are both `Unevalable` expressions +so they should never be run columnar or otherwise. These `Expressions` should be +bound using `GpuBindReferences` which will make sure that all `AttributeReference` +instances are replaced with `GpuBoundReference` implementations and everything is +on the GPU. So after calling `GpuBindReferences.bindReferences` you should be able +to cast the result to `GpuExpression` unless you know you have a SortOrder in there, +which should be rare. ### The GPU Semaphore -Typically Spark runs a task per CPU core, but there are often many more CPU +Typically, Spark runs a task per CPU core, but there are often many more CPU cores than GPUs. This can lead to situations where Spark wants to run more concurrent tasks than can reasonably fit on a GPU. The plugin works around this problem with the `GpuSemaphore` object. This object acts as a traffic diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index 60e3a6bd8ae..5ecd9b3b792 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -18,16 +18,13 @@ from data_gen import * import pyspark.sql.functions as f -@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/187') def test_passing_gpuExpr_as_Expr(): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, string_gen) .select(f.col("a")).na.drop() .groupBy(f.col("a")) - .agg(f.count(f.col("a"))) - .orderBy("count(a)", ascending=False) + .agg(f.count(f.col("a")).alias("count_a")) + .orderBy(f.col("count_a").desc(), f.col("a")) .cache() .limit(50) ) - - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala index 602b1208480..15fb7dbd75a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala @@ -17,49 +17,61 @@ package com.nvidia.spark.rapids import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.AttributeSeq +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSeq, Expression, SortOrder} import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object GpuBindReferences extends Logging { + private[this] def postBindCheck[A <: Expression](base: A): Unit = { + base.foreach { expr => + // The condition is needed to have it match what transform + // looks at, otherwise we can check things that would nto be modified + if (expr.containsChild.nonEmpty) { + expr match { + case _: GpuExpression => + case _: SortOrder => + case other => + throw new IllegalArgumentException( + s"Bound an expression that shouldn't be here ${other.getClass}") + } + } + } + } + // Mostly copied from BoundAttribute.scala so we can do columnar processing - def bindReference[A <: GpuExpression]( + def bindReference[A <: Expression]( expression: A, input: AttributeSeq, allowFailures: Boolean = false): A = { - expression.transform { case a: GpuAttributeReference => - val ordinal = input.indexOf(a.exprId) - if (ordinal == -1) { - if (allowFailures) { - a + val ret = expression.transform { + case a: AttributeReference => + val ordinal = input.indexOf(a.exprId) + if (ordinal == -1) { + if (allowFailures) { + a + } else { + sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") + } } else { - sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") + GpuBoundReference(ordinal, a.dataType, input(ordinal).nullable) } - } else { - GpuBoundReference(ordinal, a.dataType, input(ordinal).nullable) - } }.asInstanceOf[A] + if (!allowFailures) { + postBindCheck(ret) + } + ret } /** - * bindReferences[GpuExpression]: a helper function to bind given expressions to - * an input schema where the expressions are GpuExpressions. + * A helper function to bind given expressions to an input schema where the expressions are + * to be processed on the GPU. */ - def bindReferences[A <: GpuExpression]( + def bindReferences[A <: Expression]( expressions: Seq[A], input: AttributeSeq): Seq[A] = { expressions.map(GpuBindReferences.bindReference(_, input)) } - - /** - * A version of `bindReferences` that takes `AttributeSeq` as its expressions - */ - def bindReferences(expressions: AttributeSeq, input: AttributeSeq): Seq[GpuExpression] = { - bindReferences(expressions.attrs.map(ref => GpuAttributeReference( - ref.name, ref.dataType, ref.nullable, ref.metadata)(ref.exprId, ref.qualifier)), - input) - } } case class GpuBoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 90386fd4885..13588f13ca5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -72,7 +72,7 @@ class CastExprMeta[INPUT <: CastBase]( } } - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuCast(child, toType, ansiEnabled, cast.timeZoneId) } @@ -179,7 +179,7 @@ object GpuCast { * Casts using the GPU */ case class GpuCast( - child: GpuExpression, + child: Expression, dataType: DataType, ansiMode: Boolean = false, timeZoneId: Option[String] = None) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index 0a79a381dc5..becc7f47001 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -37,21 +37,21 @@ class GpuExpandExecMeta( rule: ConfKeysAndIncompat) extends SparkPlanMeta[ExpandExec](expand, conf, parent, rule) { - private val gpuProjections: Seq[Seq[ExprMeta[_]]] = + private val gpuProjections: Seq[Seq[BaseExprMeta[_]]] = expand.projections.map(_.map(GpuOverrides.wrapExpr(_, conf, Some(this)))) - private val outputAttributes: Seq[ExprMeta[_]] = + private val outputAttributes: Seq[BaseExprMeta[_]] = expand.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = gpuProjections.flatten ++ outputAttributes + override val childExprs: Seq[BaseExprMeta[_]] = gpuProjections.flatten ++ outputAttributes /** * Convert what this wraps to a GPU enabled version. */ override def convertToGpu(): GpuExec = { val projections = gpuProjections.map(_.map(_.convertToGpu())) - val attributes = outputAttributes.map(_.convertToGpu()).asInstanceOf[Seq[GpuAttributeReference]] - GpuExpandExec(projections, attributes, childPlans.head.convertIfNeeded()) + GpuExpandExec(projections, expand.output.asInstanceOf[Seq[AttributeReference]], + childPlans.head.convertIfNeeded()) } } @@ -60,12 +60,12 @@ class GpuExpandExecMeta( * multiple output rows for an input row. * @param projections The group of expressions, all of the group expressions should * output the same schema specified bye the parameter `output` - * @param resultExpressions Attribute references to Output + * @param output Attribute references to Output * @param child Child operator */ case class GpuExpandExec( - projections: Seq[Seq[GpuExpression]], - resultExpressions: Seq[GpuAttributeReference], + projections: Seq[Seq[Expression]], + output: Seq[AttributeReference], child: SparkPlan) extends UnaryExecNode with GpuExec { @@ -81,16 +81,13 @@ case class GpuExpandExec( // as UNKNOWN partitioning override def outputPartitioning: Partitioning = UnknownPartitioning(0) - override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) - @transient override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val boundProjections: Seq[Seq[GpuExpression]] = + val boundProjections: Seq[Seq[Expression]] = projections.map(GpuBindReferences.bindReferences(_, child.output)) - child.executeColumnar().mapPartitions { it => new GpuExpandIterator(boundProjections, metrics, it) } @@ -103,7 +100,7 @@ case class GpuExpandExec( } class GpuExpandIterator( - boundProjections: Seq[Seq[GpuExpression]], + boundProjections: Seq[Seq[Expression]], metrics: Map[String, SQLMetric], it: Iterator[ColumnarBatch]) extends Iterator[ColumnarBatch] @@ -159,7 +156,7 @@ class GpuExpandIterator( val projectedColumns = boundProjections(projectionIndex).safeMap(fn = expr => { val rapidsType = GpuColumnVector.getRapidsType(expr.dataType) - val (cv, nullColumnReused) = expr.columnarEval(cb) match { + val (cv, nullColumnReused) = expr.asInstanceOf[GpuExpression].columnarEval(cb) match { case null => getOrCreateNullCV(rapidsType) case lit: GpuLiteral if lit.value == null => getOrCreateNullCV(rapidsType) case lit: GpuLiteral => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index 0666b1998c0..288373c0dbe 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -22,6 +22,7 @@ import ai.rapids.cudf.{BinaryOp, BinaryOperable, DType, Scalar, UnaryOp} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, BinaryOperator, ComplexTypeMergingExpression, Expression, String2TrimExpression, TernaryExpression, UnaryExpression, Unevaluable} +import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types.StringType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -44,7 +45,7 @@ object GpuExpressionsUtils { resultCvs } - def getTrimString(trimStr: Option[GpuExpression]): String = trimStr match { + def getTrimString(trimStr: Option[Expression]): String = trimStr match { case Some(GpuLiteral(data, StringType)) => if (data == null) { null @@ -79,7 +80,7 @@ trait GpuExpression extends Expression with Unevaluable with Arm { * we have to jump through some hoops to make this work. */ def disableCoalesceUntilInput(): Boolean = - children.exists{ + children.exists { case c: GpuExpression => c.disableCoalesceUntilInput() case _ => false // This path should never really happen } @@ -237,11 +238,11 @@ abstract class CudfBinaryOperator extends GpuBinaryOperator with CudfBinaryExpre trait GpuString2TrimExpression extends String2TrimExpression with GpuExpression { - override def srcStr: GpuExpression + override def srcStr: Expression - override def trimStr: Option[GpuExpression] + override def trimStr: Option[Expression] - override def children: Seq[GpuExpression] = srcStr +: trimStr.toSeq + override def children: Seq[Expression] = srcStr +: trimStr.toSeq def strippedColumnVector(value: GpuColumnVector, sclarValue: Scalar): GpuColumnVector @@ -254,7 +255,7 @@ trait GpuString2TrimExpression extends String2TrimExpression with GpuExpression override def columnarEval(batch: ColumnarBatch): Any = { val trim = GpuExpressionsUtils.getTrimString(trimStr) - val shouldBeColumn = srcStr.columnarEval(batch) + val shouldBeColumn = srcStr.asInstanceOf[GpuExpression].columnarEval(batch) try { // We know the first parameter is not a Literal, because trim(Literal, Literal) would already // have been optimized out diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala index 288f9a77413..0baf257b098 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala @@ -36,7 +36,7 @@ class GpuGenerateExecSparkPlanMeta( p: Option[RapidsMeta[_, _, _]], r: ConfKeysAndIncompat) extends SparkPlanMeta[GenerateExec](gen, conf, p, r) { - private def exprsFromArray(data: ArrayData, dataType: DataType): Seq[ExprMeta[Expression]] = { + private def exprsFromArray(data: ArrayData, dataType: DataType): Seq[BaseExprMeta[Expression]] = { (0 until data.numElements()).map { i => Literal(data.get(i, dataType), dataType).asInstanceOf[Expression] }.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -60,7 +60,7 @@ class GpuGenerateExecSparkPlanMeta( case _ => Seq.empty } - override val childExprs: Seq[ExprMeta[_]] = arrayExprs + override val childExprs: Seq[BaseExprMeta[_]] = arrayExprs override def tagPlanForGpu(): Unit = { // We can only run on the GPU if we are doing a posexplode of an array we are generating @@ -100,7 +100,7 @@ class GpuGenerateExecSparkPlanMeta( */ case class GpuGenerateExec( includePos: Boolean, - arrayProject: Seq[GpuExpression], + arrayProject: Seq[Expression], requiredChildOutput: Seq[Attribute], generatorOutput: Seq[Attribute], child: SparkPlan @@ -121,8 +121,9 @@ case class GpuGenerateExec( val totalTime = longMetric(TOTAL_TIME) val boundArrayProjectList = GpuBindReferences.bindReferences(arrayProject, child.output).toArray val numArrayColumns = boundArrayProjectList.length - val boundOthersProjectList = - GpuBindReferences.bindReferences(requiredChildOutput, child.output).toArray + val boundOthersProjectList: Array[GpuExpression] = + GpuBindReferences.bindReferences(requiredChildOutput, child.output) + .asInstanceOf[Seq[GpuExpression]].toArray val numOtherColumns = boundOthersProjectList.length val numExplodeColumns = if (includePos) 2 else 1 diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala index b1599e6524c..fea0aa676ea 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala @@ -71,7 +71,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { def doJoin(builtTable: Table, streamedBatch: ColumnarBatch, - condition: Option[GpuExpression], + condition: Option[Expression], numOutputRows: SQLMetric, numJoinOutputRows: SQLMetric, numOutputBatches: SQLMetric, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala index b88a87f4e1c..2e1f62a751f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Dist import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.sql.vectorized.ColumnarBatch -case class GpuHashPartitioning(expressions: Seq[GpuExpression], numPartitions: Int) +case class GpuHashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends GpuExpression with GpuPartitioning { override def children: Seq[Expression] = expressions @@ -47,8 +47,10 @@ case class GpuHashPartitioning(expressions: Seq[GpuExpression], numPartitions: I } } - def getGpuKeyColumns(batch: ColumnarBatch) : Array[GpuColumnVector] = - expressions.map(_.columnarEval(batch).asInstanceOf[GpuColumnVector]).toArray + def getGpuKeyColumns(batch: ColumnarBatch) : Array[GpuColumnVector] = { + expressions.map(_.asInstanceOf[GpuExpression].columnarEval(batch) + .asInstanceOf[GpuColumnVector]).toArray + } def getGpuDataColumns(batch: ColumnarBatch) : Array[GpuColumnVector] = GpuColumnVector.extractColumns(batch) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index fda1230cbec..d633cdc3945 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -20,8 +20,6 @@ import java.time.ZoneId import scala.reflect.ClassTag -import com.nvidia.spark.rapids.DateUtils.TimestampFormatConversionException - import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ @@ -170,11 +168,11 @@ class ExprRule[INPUT <: Expression]( INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], - ConfKeysAndIncompat) => ExprMeta[INPUT], + ConfKeysAndIncompat) => BaseExprMeta[INPUT], incompatDoc: Option[String], desc: String, tag: ClassTag[INPUT]) - extends ReplacementRule[INPUT, Expression, ExprMeta[INPUT]](doWrap, + extends ReplacementRule[INPUT, Expression, BaseExprMeta[INPUT]](doWrap, incompatDoc, desc, tag) { override val confKeyPart = "expression" @@ -395,7 +393,7 @@ object GpuOverrides { def expr[INPUT <: Expression]( desc: String, doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], ConfKeysAndIncompat) - => ExprMeta[INPUT]) + => BaseExprMeta[INPUT]) (implicit tag: ClassTag[INPUT]): ExprRule[INPUT] = { assert(desc != null) assert(doWrap != null) @@ -445,9 +443,9 @@ object GpuOverrides { def wrapExpr[INPUT <: Expression]( expr: INPUT, conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]]): ExprMeta[INPUT] = + parent: Option[RapidsMeta[_, _, _]]): BaseExprMeta[INPUT] = expressions.get(expr.getClass) - .map(r => r.wrap(expr, conf, parent, r).asInstanceOf[ExprMeta[INPUT]]) + .map(r => r.wrap(expr, conf, parent, r).asInstanceOf[BaseExprMeta[INPUT]]) .getOrElse(new RuleNotFoundExprMeta(expr, conf, parent)) val expressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( @@ -472,20 +470,19 @@ object GpuOverrides { expr[Signum]( "Returns -1.0, 0.0 or 1.0 as expr is negative, 0 or positive", (a, conf, p, r) => new UnaryExprMeta[Signum](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuSignum(child) + override def convertToGpu(child: Expression): GpuExpression = GpuSignum(child) }), expr[Alias]( "gives a column a name", (a, conf, p, r) => new UnaryExprMeta[Alias](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuAlias(child, a.name)(a.exprId, a.qualifier, a.explicitMetadata) }), expr[AttributeReference]( "references an input column", - (att, conf, p, r) => new ExprMeta[AttributeReference](att, conf, p, r) { - override def convertToGpu(): GpuExpression = - GpuAttributeReference(att.name, att.dataType, att.nullable, - att.metadata)(att.exprId, att.qualifier) + (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. override def print(append: StringBuilder, depth: Int, all: Boolean): Unit = {} @@ -500,12 +497,12 @@ object GpuOverrides { expr[ToDegrees]( "Converts radians to degrees", (a, conf, p, r) => new UnaryExprMeta[ToDegrees](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuToDegrees = GpuToDegrees(child) + override def convertToGpu(child: Expression): GpuToDegrees = GpuToDegrees(child) }), expr[ToRadians]( "Converts degrees to radians", (a, conf, p, r) => new UnaryExprMeta[ToRadians](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuToRadians = GpuToRadians(child) + 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 " + @@ -566,42 +563,42 @@ object GpuOverrides { expr[UnaryMinus]( "negate a numeric value", (a, conf, p, r) => new UnaryExprMeta[UnaryMinus](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuUnaryMinus(child) + override def convertToGpu(child: Expression): GpuExpression = GpuUnaryMinus(child) }), expr[UnaryPositive]( "a numeric value with a + in front of it", (a, conf, p, r) => new UnaryExprMeta[UnaryPositive](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuUnaryPositive(child) + override def convertToGpu(child: Expression): GpuExpression = GpuUnaryPositive(child) }), expr[Year]( "get the year from a date or timestamp", (a, conf, p, r) => new UnaryExprMeta[Year](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuYear(child) + override def convertToGpu(child: Expression): GpuExpression = GpuYear(child) }), expr[Month]( "get the month from a date or timestamp", (a, conf, p, r) => new UnaryExprMeta[Month](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuMonth(child) + override def convertToGpu(child: Expression): GpuExpression = GpuMonth(child) }), expr[DayOfMonth]( "get the day of the month from a date or timestamp", (a, conf, p, r) => new UnaryExprMeta[DayOfMonth](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuDayOfMonth(child) + override def convertToGpu(child: Expression): GpuExpression = GpuDayOfMonth(child) }), expr[Abs]( "absolute value", (a, conf, p, r) => new UnaryExprMeta[Abs](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAbs(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAbs(child) }), expr[Acos]( "inverse cosine", (a, conf, p, r) => new UnaryExprMeta[Acos](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAcos(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAcos(child) }), expr[Acosh]( "inverse hyperbolic cosine", (a, conf, p, r) => new UnaryExprMeta[Acosh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = if (conf.includeImprovedFloat) { GpuAcoshImproved(child) } else { @@ -611,12 +608,12 @@ object GpuOverrides { expr[Asin]( "inverse sine", (a, conf, p, r) => new UnaryExprMeta[Asin](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAsin(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAsin(child) }), expr[Asinh]( "inverse hyperbolic sine", (a, conf, p, r) => new UnaryExprMeta[Asinh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = if (conf.includeImprovedFloat) { GpuAsinhImproved(child) } else { @@ -626,59 +623,59 @@ object GpuOverrides { expr[Sqrt]( "square root", (a, conf, p, r) => new UnaryExprMeta[Sqrt](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuSqrt(child) + override def convertToGpu(child: Expression): GpuExpression = GpuSqrt(child) }), expr[Cbrt]( "cube root", (a, conf, p, r) => new UnaryExprMeta[Cbrt](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuCbrt(child) + override def convertToGpu(child: Expression): GpuExpression = GpuCbrt(child) }), expr[Floor]( "floor of a number", (a, conf, p, r) => new UnaryExprMeta[Floor](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuFloor(child) + override def convertToGpu(child: Expression): GpuExpression = GpuFloor(child) }), expr[Ceil]( "ceiling of a number", (a, conf, p, r) => new UnaryExprMeta[Ceil](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuCeil(child) + override def convertToGpu(child: Expression): GpuExpression = GpuCeil(child) }), expr[Not]( "boolean not operator", (a, conf, p, r) => new UnaryExprMeta[Not](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuNot(child) + override def convertToGpu(child: Expression): GpuExpression = GpuNot(child) }), expr[IsNull]( "checks if a value is null", (a, conf, p, r) => new UnaryExprMeta[IsNull](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuIsNull(child) + override def convertToGpu(child: Expression): GpuExpression = GpuIsNull(child) }), expr[IsNotNull]( "checks if a value is not null", (a, conf, p, r) => new UnaryExprMeta[IsNotNull](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuIsNotNull(child) + override def convertToGpu(child: Expression): GpuExpression = GpuIsNotNull(child) }), expr[IsNaN]( "checks if a value is NaN", (a, conf, p, r) => new UnaryExprMeta[IsNaN](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuIsNan(child) + override def convertToGpu(child: Expression): GpuExpression = GpuIsNan(child) }), expr[Rint]( "Rounds up a double value to the nearest double equal to an integer", (a, conf, p, r) => new UnaryExprMeta[Rint](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuRint(child) + override def convertToGpu(child: Expression): GpuExpression = GpuRint(child) }), expr[BitwiseNot]( "Returns the bitwise NOT of the operands", (a, conf, p, r) => new UnaryExprMeta[BitwiseNot](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = { + override def convertToGpu(child: Expression): GpuExpression = { GpuBitwiseNot(child) } }), expr[AtLeastNNonNulls]( "checks if number of non null/Nan values is greater than a given value", (a, conf, p, r) => new ExprMeta[AtLeastNNonNulls](a, conf, p, r) { - override val childExprs: Seq[ExprMeta[_]] = a.children + override val childExprs: Seq[BaseExprMeta[_]] = a.children .map(GpuOverrides.wrapExpr(_, conf, Some(this))) def convertToGpu(): GpuExpression = { GpuAtLeastNNonNulls(a.n, childExprs.map(_.convertToGpu())) @@ -687,14 +684,14 @@ object GpuOverrides { expr[DateAdd]( "Returns the date that is num_days after start_date", (a, conf, p, r) => new BinaryExprMeta[DateAdd](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuDateAdd(lhs, rhs) } ), expr[DateSub]( "Returns the date that is num_days before start_date", (a, conf, p, r) => new BinaryExprMeta[DateSub](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuDateSub(lhs, rhs) } ), @@ -715,53 +712,53 @@ object GpuOverrides { } } - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuTimeSub(lhs, rhs) } ), expr[NaNvl]( "evaluates to `left` iff left is not NaN, `right` otherwise.", (a, conf, p, r) => new BinaryExprMeta[NaNvl](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuNaNvl(lhs, rhs) } ), expr[ShiftLeft]( "Bitwise shift left (<<)", (a, conf, p, r) => new BinaryExprMeta[ShiftLeft](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuShiftLeft(lhs, rhs) }), expr[ShiftRight]( "Bitwise shift right (>>)", (a, conf, p, r) => new BinaryExprMeta[ShiftRight](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuShiftRight(lhs, rhs) }), expr[ShiftRightUnsigned]( "Bitwise unsigned shift right (>>>)", (a, conf, p, r) => new BinaryExprMeta[ShiftRightUnsigned](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuShiftRightUnsigned(lhs, rhs) }), expr[BitwiseAnd]( "Returns the bitwise AND of the operands", (a, conf, p, r) => new BinaryExprMeta[BitwiseAnd](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuBitwiseAnd(lhs, rhs) } ), expr[BitwiseOr]( "Returns the bitwise OR of the operands", (a, conf, p, r) => new BinaryExprMeta[BitwiseOr](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuBitwiseOr(lhs, rhs) } ), expr[BitwiseXor]( "Returns the bitwise XOR of the operands", (a, conf, p, r) => new BinaryExprMeta[BitwiseXor](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuBitwiseXor(lhs, rhs) } ), @@ -774,61 +771,61 @@ object GpuOverrides { expr[Atan]( "inverse tangent", (a, conf, p, r) => new UnaryExprMeta[Atan](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAtan(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAtan(child) }), expr[Atanh]( "inverse hyperbolic tangent", (a, conf, p, r) => new UnaryExprMeta[Atanh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAtanh(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAtanh(child) }), expr[Cos]( "cosine", (a, conf, p, r) => new UnaryExprMeta[Cos](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuCos(child) + override def convertToGpu(child: Expression): GpuExpression = GpuCos(child) }), expr[Exp]( "Euler's number e raised to a power", (a, conf, p, r) => new UnaryExprMeta[Exp](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuExp(child) + override def convertToGpu(child: Expression): GpuExpression = GpuExp(child) }), expr[Expm1]( "Euler's number e raised to a power minus 1", (a, conf, p, r) => new UnaryExprMeta[Expm1](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuExpm1(child) + override def convertToGpu(child: Expression): GpuExpression = GpuExpm1(child) }), expr[InitCap]("Returns str with the first letter of each word in uppercase. " + "All other letters are in lowercase", (a, conf, p, r) => new UnaryExprMeta[InitCap](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuInitCap(child) + override def convertToGpu(child: Expression): GpuExpression = GpuInitCap(child) }).incompat(CASE_MODIFICATION_INCOMPAT + " Spark also only sees the space character as " + "a word deliminator, but this uses more white space characters."), expr[Log]( "natural log", (a, conf, p, r) => new UnaryExprMeta[Log](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuLog(child) + override def convertToGpu(child: Expression): GpuExpression = GpuLog(child) }), expr[Log1p]( "natural log 1 + expr", (a, conf, p, r) => new UnaryExprMeta[Log1p](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuLog(GpuAdd(child, GpuLiteral(1d, DataTypes.DoubleType))) }), expr[Log2]( "log base 2", (a, conf, p, r) => new UnaryExprMeta[Log2](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuLogarithm(child, GpuLiteral(2d, DataTypes.DoubleType)) }), expr[Log10]( "log base 10", (a, conf, p, r) => new UnaryExprMeta[Log10](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuLogarithm(child, GpuLiteral(10d, DataTypes.DoubleType)) }), expr[Logarithm]( "log variable base", (a, conf, p, r) => new BinaryExprMeta[Logarithm](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { // the order of the parameters is transposed intentionally GpuLogarithm(rhs, lhs) } @@ -836,55 +833,55 @@ object GpuOverrides { expr[Sin]( "sine", (a, conf, p, r) => new UnaryExprMeta[Sin](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuSin(child) + override def convertToGpu(child: Expression): GpuExpression = GpuSin(child) }), expr[Sinh]( "hyperbolic sine", (a, conf, p, r) => new UnaryExprMeta[Sinh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuSinh(child) + override def convertToGpu(child: Expression): GpuExpression = GpuSinh(child) }), expr[Cosh]( "hyperbolic cosine", (a, conf, p, r) => new UnaryExprMeta[Cosh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuCosh(child) + override def convertToGpu(child: Expression): GpuExpression = GpuCosh(child) }), expr[Cot]( "Returns the cotangent", (a, conf, p, r) => new UnaryExprMeta[Cot](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuCot(child) + override def convertToGpu(child: Expression): GpuExpression = GpuCot(child) }), expr[Tanh]( "hyperbolic tangent", (a, conf, p, r) => new UnaryExprMeta[Tanh](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuTanh(child) + override def convertToGpu(child: Expression): GpuExpression = GpuTanh(child) }), expr[Tan]( "tangent", (a, conf, p, r) => new UnaryExprMeta[Tan](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuTan(child) + override def convertToGpu(child: Expression): GpuExpression = GpuTan(child) }), expr[NormalizeNaNAndZero]( "normalize nan and zero", (a, conf, p, r) => new UnaryExprMeta[NormalizeNaNAndZero](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuNormalizeNaNAndZero(child) }), expr[KnownFloatingPointNormalized]( "tag to prevent redundant normalization", (a, conf, p, r) => new UnaryExprMeta[KnownFloatingPointNormalized](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = + override def convertToGpu(child: Expression): GpuExpression = GpuKnownFloatingPointNormalized(child) }), expr[DateDiff]("datediff", (a, conf, p, r) => new BinaryExprMeta[DateDiff](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { GpuDateDiff(lhs, rhs) } }), expr[ToUnixTimestamp]( "Returns the UNIX timestamp of the given time", (a, conf, p, r) => new UnixTimeExprMeta[ToUnixTimestamp](a, conf, p, r){ - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { if (conf.isImprovedTimestampOpsEnabled) { // passing the already converted strf string for a little optimization GpuToUnixTimestampImproved(lhs, rhs, strfFormat) @@ -898,7 +895,7 @@ object GpuOverrides { expr[UnixTimestamp]( "Returns the UNIX timestamp of current or specified time", (a, conf, p, r) => new UnixTimeExprMeta[UnixTimestamp](a, conf, p, r){ - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { if (conf.isImprovedTimestampOpsEnabled) { // passing the already converted strf string for a little optimization GpuUnixTimestampImproved(lhs, rhs, strfFormat) @@ -918,9 +915,7 @@ object GpuOverrides { } } - override def convertToGpu(expr: GpuExpression): GpuExpression = { - GpuHour(expr) - } + override def convertToGpu(expr: Expression): GpuExpression = GpuHour(expr) }), expr[Minute]( "Returns the minute component of the string/timestamp.", @@ -931,9 +926,8 @@ object GpuOverrides { } } - override def convertToGpu(expr: GpuExpression): GpuExpression = { + override def convertToGpu(expr: Expression): GpuExpression = GpuMinute(expr) - } }), expr[Second]( "Returns the second component of the string/timestamp.", @@ -944,14 +938,13 @@ object GpuOverrides { } } - override def convertToGpu(expr: GpuExpression): GpuExpression = { + override def convertToGpu(expr: Expression): GpuExpression = GpuSecond(expr) - } }), expr[FromUnixTime]( "get the String from a unix timestamp", (a, conf, p, r) => new UnixTimeExprMeta[FromUnixTime](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = { + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { // passing the already converted strf string for a little optimization GpuFromUnixTime(lhs, rhs, strfFormat) } @@ -959,61 +952,61 @@ object GpuOverrides { expr[Pmod]( "pmod", (a, conf, p, r) => new BinaryExprMeta[Pmod](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuPmod(lhs, rhs) }), expr[Add]( "addition", (a, conf, p, r) => new BinaryExprMeta[Add](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuAdd(lhs, rhs) }), expr[Subtract]( "subtraction", (a, conf, p, r) => new BinaryExprMeta[Subtract](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuSubtract(lhs, rhs) }), expr[Multiply]( "multiplication", (a, conf, p, r) => new BinaryExprMeta[Multiply](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuMultiply(lhs, rhs) }), expr[And]( "logical and", (a, conf, p, r) => new BinaryExprMeta[And](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuAnd(lhs, rhs) }), expr[Or]( "logical or", (a, conf, p, r) => new BinaryExprMeta[Or](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuOr(lhs, rhs) }), expr[EqualNullSafe]( "check if the values are equal including nulls <=>", (a, conf, p, r) => new BinaryExprMeta[EqualNullSafe](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuEqualNullSafe(lhs, rhs) }), expr[EqualTo]( "check if the values are equal", (a, conf, p, r) => new BinaryExprMeta[EqualTo](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuEqualTo(lhs, rhs) }), expr[GreaterThan]( "> operator", (a, conf, p, r) => new BinaryExprMeta[GreaterThan](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuGreaterThan(lhs, rhs) }), expr[GreaterThanOrEqual]( ">= operator", (a, conf, p, r) => new BinaryExprMeta[GreaterThanOrEqual](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuGreaterThanOrEqual(lhs, rhs) }), expr[In]( @@ -1054,13 +1047,13 @@ object GpuOverrides { expr[LessThan]( "< operator", (a, conf, p, r) => new BinaryExprMeta[LessThan](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuLessThan(lhs, rhs) }), expr[LessThanOrEqual]( "<= operator", (a, conf, p, r) => new BinaryExprMeta[LessThanOrEqual](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuLessThanOrEqual(lhs, rhs) }), expr[CaseWhen]( @@ -1101,35 +1094,35 @@ object GpuOverrides { expr[Pow]( "lhs ^ rhs", (a, conf, p, r) => new BinaryExprMeta[Pow](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuPow(lhs, rhs) }), expr[Divide]( "division", (a, conf, p, r) => new BinaryExprMeta[Divide](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuDivide(lhs, rhs) }), expr[IntegralDivide]( "division with a integer result", (a, conf, p, r) => new BinaryExprMeta[IntegralDivide](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuIntegralDivide(lhs, rhs) }), expr[Remainder]( "remainder or modulo", (a, conf, p, r) => new BinaryExprMeta[Remainder](a, conf, p, r) { - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuRemainder(lhs, rhs) }), expr[AggregateExpression]( "aggregate expression", (a, conf, p, r) => new ExprMeta[AggregateExpression](a, conf, p, r) { - private val filter: Option[ExprMeta[_]] = + private val filter: Option[BaseExprMeta[_]] = a.filter.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val childrenExprMeta: Seq[ExprMeta[Expression]] = + private val childrenExprMeta: Seq[BaseExprMeta[Expression]] = a.children.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = if (filter.isDefined) { + override val childExprs: Seq[BaseExprMeta[_]] = if (filter.isDefined) { childrenExprMeta :+ filter.get } else { childrenExprMeta @@ -1146,14 +1139,18 @@ object GpuOverrides { resultMethod.invoke(a).asInstanceOf[Seq[ExprId]](0) } GpuAggregateExpression(childExprs(0).convertToGpu().asInstanceOf[GpuAggregateFunction], - a.mode, a.isDistinct, filter.map(_.convertToGpu()) ,resultId) + a.mode, a.isDistinct, filter.map(_.convertToGpu()), resultId) } }), expr[SortOrder]( "sort order", - (a, conf, p, r) => new UnaryExprMeta[SortOrder](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = - GpuSortOrder(child, a.direction, a.nullOrdering, a.sameOrderExpressions, a.child) +// (a, conf, p, r) => new UnaryExprMeta[SortOrder](a, conf, p, r) { +// override def convertToGpu(child: Expression): GpuExpression = +// GpuSortOrder(child, a.direction, a.nullOrdering, a.sameOrderExpressions, a.child) +// }), + (a, conf, p, r) => new BaseExprMeta[SortOrder](a, conf, p, r) { + override def convertToGpu(): Expression = + a.withNewChildren(childExprs.map(_.convertToGpu())) }), expr[Count]( "count aggregate operator", @@ -1177,7 +1174,7 @@ object GpuOverrides { s" ${RapidsConf.HAS_NANS} to false.") } } - override def convertToGpu(child: GpuExpression): GpuExpression = GpuMax(child) + override def convertToGpu(child: Expression): GpuExpression = GpuMax(child) }), expr[Min]( "min aggregate operator", @@ -1190,14 +1187,15 @@ object GpuOverrides { s" ${RapidsConf.HAS_NANS} to false.") } } - override def convertToGpu(child: GpuExpression): GpuExpression = GpuMin(child) + override def convertToGpu(child: Expression): GpuExpression = GpuMin(child) }), expr[First]( "first aggregate operator", (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { - val child: ExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: ExprMeta[_] = GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[ExprMeta[_]] = Seq(child, ignoreNulls) + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) override def convertToGpu(): GpuExpression = GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) @@ -1205,9 +1203,10 @@ object GpuOverrides { expr[Last]( "last aggregate operator", (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { - val child: ExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: ExprMeta[_] = GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[ExprMeta[_]] = Seq(child, ignoreNulls) + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) override def convertToGpu(): GpuExpression = GpuLast(child.convertToGpu(), ignoreNulls.convertToGpu()) @@ -1226,7 +1225,7 @@ object GpuOverrides { } } - override def convertToGpu(child: GpuExpression): GpuExpression = GpuSum(child) + override def convertToGpu(child: Expression): GpuExpression = GpuSum(child) }), expr[Average]( "average aggregate operator", @@ -1242,12 +1241,12 @@ object GpuOverrides { } } - override def convertToGpu(child: GpuExpression): GpuExpression = GpuAverage(child) + override def convertToGpu(child: Expression): GpuExpression = GpuAverage(child) }), expr[Rand]( "Generate a random column with i.i.d. uniformly distributed values in [0, 1)", (a, conf, p, r) => new UnaryExprMeta[Rand](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuRand(child) + override def convertToGpu(child: Expression): GpuExpression = GpuRand(child) }), expr[SparkPartitionID] ( "Returns the current partition id.", @@ -1282,13 +1281,13 @@ object GpuOverrides { expr[Upper]( "String uppercase operator", (a, conf, p, r) => new UnaryExprMeta[Upper](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuUpper(child) + override def convertToGpu(child: Expression): GpuExpression = GpuUpper(child) }) .incompat(CASE_MODIFICATION_INCOMPAT), expr[Lower]( "String lowercase operator", (a, conf, p, r) => new UnaryExprMeta[Lower](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuLower(child) + override def convertToGpu(child: Expression): GpuExpression = GpuLower(child) }) .incompat(CASE_MODIFICATION_INCOMPAT), expr[StringLocate]( @@ -1302,9 +1301,9 @@ object GpuOverrides { } } override def convertToGpu( - val0: GpuExpression, - val1: GpuExpression, - val2: GpuExpression): GpuExpression = + val0: Expression, + val1: Expression, + val2: Expression): GpuExpression = GpuStringLocate(val0, val1, val2) }), expr[Substring]( @@ -1318,9 +1317,9 @@ object GpuOverrides { } override def convertToGpu( - column: GpuExpression, - position: GpuExpression, - length: GpuExpression): GpuExpression = + column: Expression, + position: Expression, + length: Expression): GpuExpression = GpuSubstring(column, position, length) }), expr[SubstringIndex]( @@ -1337,17 +1336,17 @@ object GpuOverrides { } override def convertToGpu( - column: GpuExpression, - target: GpuExpression, - replace: GpuExpression): GpuExpression = + column: Expression, + target: Expression, + replace: Expression): GpuExpression = GpuStringReplace(column, target, replace) }), expr[StringTrim]( "StringTrim operator", (in, conf, p, r) => new String2TrimExpressionMeta[StringTrim](in, in.trimStr, conf, p, r) { override def convertToGpu( - column: GpuExpression, - target: Option[GpuExpression] = None): GpuExpression = + column: Expression, + target: Option[Expression] = None): GpuExpression = GpuStringTrim(column, target) }), expr[StringTrimLeft]( @@ -1355,8 +1354,8 @@ object GpuOverrides { (in, conf, p, r) => new String2TrimExpressionMeta[StringTrimLeft](in, in.trimStr, conf, p, r) { override def convertToGpu( - column: GpuExpression, - target: Option[GpuExpression] = None): GpuExpression = + column: Expression, + target: Option[Expression] = None): GpuExpression = GpuStringTrimLeft(column, target) }), expr[StringTrimRight]( @@ -1364,8 +1363,8 @@ object GpuOverrides { (in, conf, p, r) => new String2TrimExpressionMeta[StringTrimRight](in, in.trimStr, conf, p, r) { override def convertToGpu( - column: GpuExpression, - target: Option[GpuExpression] = None): GpuExpression = + column: Expression, + target: Option[Expression] = None): GpuExpression = GpuStringTrimRight(column, target) } ), @@ -1377,7 +1376,7 @@ object GpuOverrides { willNotWorkOnGpu("only literals are supported for startsWith") } } - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuStartsWith(lhs, rhs) }), expr[EndsWith]( @@ -1388,14 +1387,14 @@ object GpuOverrides { willNotWorkOnGpu("only literals are supported for endsWith") } } - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuEndsWith(lhs, rhs) }), expr[Concat]( "String Concatenate NO separator", (a, conf, p, r) => new ComplexTypeMergingExprMeta[Concat](a, conf, p, r) { override def tagExprForGpu(): Unit = {} - override def convertToGpu(child: Seq[GpuExpression]): GpuExpression = GpuConcat(child) + override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuConcat(child) }), expr[Contains]( "Contains", @@ -1406,7 +1405,7 @@ object GpuOverrides { " parameter") } } - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuContains(lhs, rhs) }), expr[Like]( @@ -1418,7 +1417,7 @@ object GpuOverrides { " parameter") } } - override def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression = + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuLike(lhs, rhs, a.escapeChar) }), expr[RegExpReplace]( @@ -1434,13 +1433,13 @@ object GpuOverrides { "are supported by RegExpReplace on the GPU") } } - override def convertToGpu(lhs: GpuExpression, regexp: GpuExpression, - rep: GpuExpression): GpuExpression = GpuStringReplace(lhs, regexp, rep) + override def convertToGpu(lhs: Expression, regexp: Expression, + rep: Expression): GpuExpression = GpuStringReplace(lhs, regexp, rep) }), expr[Length]( "String Character Length", (a, conf, p, r) => new UnaryExprMeta[Length](a, conf, p, r) { - override def convertToGpu(child: GpuExpression): GpuExpression = GpuLength(child) + override def convertToGpu(child: Expression): GpuExpression = GpuLength(child) }) ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap @@ -1521,7 +1520,7 @@ object GpuOverrides { part[HashPartitioning]( "Hash based partitioning", (hp, conf, p, r) => new PartMeta[HashPartitioning](hp, conf, p, r) { - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = hp.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override def convertToGpu(): GpuPartitioning = @@ -1529,12 +1528,19 @@ object GpuOverrides { }), part[RangePartitioning]( "Range Partitioning", (rp, conf, p, r) => new PartMeta[RangePartitioning](rp, conf, p, r) { - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = rp.ordering.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override def convertToGpu(): GpuPartitioning = { if (rp.numPartitions > 1) { - GpuRangePartitioning(childExprs.map(_.convertToGpu()) - .asInstanceOf[Seq[GpuSortOrder]], rp.numPartitions, new GpuRangePartitioner) + val gpuOrdering = childExprs.map(_.convertToGpu()).asInstanceOf[Seq[SortOrder]] + val tmp = gpuOrdering.flatMap { ord => + ord.child.references.map { field => + StructField(field.name, field.dataType) + } + } + val schema = new StructType(tmp.toArray) + + GpuRangePartitioning(gpuOrdering, rp.numPartitions, new GpuRangePartitioner, schema) } else { GpuSinglePartitioning(childExprs.map(_.convertToGpu())) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioner.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioner.scala index 8ecefdcba3f..c24f2060a88 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioner.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioner.scala @@ -24,7 +24,7 @@ import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences, BoundReference, SortOrder, UnsafeProjection} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -92,13 +92,12 @@ class GpuRangePartitioner extends Serializable { bounds.toArray } - def createRangeBounds(partitions: Int, gpuOrdering: Seq[GpuSortOrder], rdd: RDD[ColumnarBatch], + def createRangeBounds(partitions: Int, gpuOrdering: Seq[SortOrder], rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], samplePointsPerPartitionHint: Int): Unit = { val orderingAttributes = gpuOrdering.zipWithIndex.map { case (ord, i) => - val sortOrder = ord.toSortOrder - sortOrder.copy(child = BoundReference(i, sortOrder.dataType, sortOrder.nullable)) + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) } implicit val ordering: LazilyGeneratedOrdering = new LazilyGeneratedOrdering(orderingAttributes) val rowsRDD = rddForSampling(partitions, gpuOrdering, rdd, outputAttributes) @@ -155,11 +154,13 @@ class GpuRangePartitioner extends Serializable { this.rangeBounds = rangeBounds.asInstanceOf[Array[InternalRow]] } - def rddForSampling(partitions: Int, gpuOrdering: Seq[GpuSortOrder], rdd: RDD[ColumnarBatch], + def rddForSampling(partitions: Int, gpuOrdering: Seq[SortOrder], rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute]) : RDD[MutablePair[InternalRow, Null]] = { - val sortingExpressions = gpuOrdering.map(so => so.toSortOrder) - lazy val toUnsafe = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) + val sortingExpressions = gpuOrdering + lazy val toUnsafe = UnsafeProjection.create( + sortingExpressions.map(_.child), + outputAttributes) val rowsRDD = rdd.mapPartitions { batches => { new Iterator[InternalRow] { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioning.scala index 06c502ebb4b..6e1eddcf93f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRangePartitioning.scala @@ -22,6 +22,7 @@ import ai.rapids.cudf.{ColumnVector, Table} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, OrderedDistribution} import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -37,24 +38,16 @@ import org.apache.spark.sql.vectorized.ColumnarBatch */ case class GpuRangePartitioning( - gpuOrdering: Seq[GpuSortOrder], + gpuOrdering: Seq[SortOrder], numPartitions: Int, - part: GpuRangePartitioner) + part: GpuRangePartitioner, + schema: StructType) extends GpuExpression with GpuPartitioning { var rangeBounds: Array[InternalRow] = _ - var schema: StructType = new StructType() - gpuOrdering.foreach { ord => - val sortOrder = ord.toSortOrder - sortOrder.child.references.foreach(field => { - schema = schema.add(StructField(field.name, field.dataType)) - }) - } - - override def children: Seq[GpuExpression] = gpuOrdering + override def children: Seq[SortOrder] = gpuOrdering override def nullable: Boolean = false - override def dataType: DataType = IntegerType override def satisfies0(required: Distribution): Boolean = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleExchangeExec.scala index 0a35b5dd8b3..f548848ceaf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleExchangeExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} @@ -142,7 +142,7 @@ object GpuShuffleExchangeExec { * task when indeterminate tasks re-run. */ val newRdd = if (isRoundRobin && SQLConf.get.sortBeforeRepartition) { - val sorter = new GpuColumnarBatchSorter(Seq.empty[GpuSortOrder], + val sorter = new GpuColumnarBatchSorter(Seq.empty[SortOrder], null, false, false) rdd.mapPartitions { cbIter => val sortedIterator = sorter.sort(cbIter) @@ -228,16 +228,15 @@ object GpuShuffleExchangeExec { newPartitioning: Partitioning): GpuExpression with GpuPartitioning = { newPartitioning match { case h: GpuHashPartitioning => - GpuBindReferences.bindReferences(h :: Nil, outputAttributes).head + GpuBindReferences.bindReference(h, outputAttributes) case r: GpuRangePartitioning => r.part.createRangeBounds(r.numPartitions, r.gpuOrdering, rdd, outputAttributes, SQLConf.get.rangeExchangeSampleSizePerPartition) - val boundR = GpuBindReferences.bindReferences(r :: Nil, outputAttributes).head - boundR + GpuBindReferences.bindReference(r, outputAttributes) case s: GpuSinglePartitioning => - GpuBindReferences.bindReferences(s :: Nil, outputAttributes).head + GpuBindReferences.bindReference(s, outputAttributes) case rrp: GpuRoundRobinPartitioning => - GpuBindReferences.bindReferences(rrp :: Nil, outputAttributes).head + GpuBindReferences.bindReference(rrp, outputAttributes) case _ => sys.error(s"Exchange not implemented for $newPartitioning") } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala index 6de4a952223..1ec4488f851 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala @@ -21,6 +21,7 @@ import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} @@ -34,12 +35,14 @@ class GpuShuffledHashJoinMeta( parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) { - val leftKeys: Seq[ExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[ExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[ExprMeta[_]] = + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.condition) @@ -57,11 +60,11 @@ class GpuShuffledHashJoinMeta( } case class GpuShuffledHashJoinExec( - leftKeys: Seq[GpuExpression], - rightKeys: Seq[GpuExpression], + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, - condition: Option[GpuExpression], + condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSinglePartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSinglePartitioning.scala index 9aea5ebd10c..dd95b2986b7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSinglePartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSinglePartitioning.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.sql.vectorized.ColumnarBatch -case class GpuSinglePartitioning(expressions: Seq[GpuExpression]) +case class GpuSinglePartitioning(expressions: Seq[Expression]) extends GpuExpression with GpuPartitioning { /** * Returns the result of evaluating this expression on the entire `ColumnarBatch`. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala index 94026d658c6..1f05340f6a7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala @@ -23,12 +23,10 @@ import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, NullOrdering, NullsFirst, NullsLast, RowOrdering, SortDirection, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, NullsFirst, NullsLast, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{SortExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} import org.apache.spark.sql.vectorized.ColumnarBatch class GpuSortMeta( @@ -38,7 +36,7 @@ class GpuSortMeta( rule: ConfKeysAndIncompat) extends SparkPlanMeta[SortExec](sort, conf, parent, rule) { override def convertToGpu(): GpuExec = - GpuSortExec(childExprs.map(_.convertToGpu()).asInstanceOf[Seq[GpuSortOrder]], + GpuSortExec(childExprs.map(_.convertToGpu()).asInstanceOf[Seq[SortOrder]], sort.global, childPlans(0).convertIfNeeded()) @@ -50,15 +48,14 @@ class GpuSortMeta( } case class GpuSortExec( - sortOrder: Seq[GpuSortOrder], + sortOrder: Seq[SortOrder], global: Boolean, child: SparkPlan, coalesceGoal: CoalesceGoal = RequireSingleBatch, - testSpillFrequency: Int = 0 -) + testSpillFrequency: Int = 0) extends UnaryExecNode with GpuExec { - private val sparkSortOrder = sortOrder.map(_.toSortOrder) + private val sparkSortOrder = sortOrder override def childrenCoalesceGoal: Seq[CoalesceGoal] = Seq(coalesceGoal) @@ -101,7 +98,7 @@ case class GpuSortExec( } class GpuColumnarBatchSorter( - sortOrder: Seq[GpuSortOrder], + sortOrder: Seq[SortOrder], exec: GpuExec, singleBatchOnly: Boolean, shouldUpdateMetrics: Boolean = true) extends Serializable { @@ -255,57 +252,3 @@ class GpuColumnarBatchSorter( } } } - -/** - * GpuSortOrder where the child is a GpuExpression. - * - * As far as I can tell the sameOrderExpressions can stay as is. It's used to see if the - * ordering already matches for things like inserting shuffles and optimizing out redundant sorts - * and as long as the plugin isn't acting differently then the CPU that should just work. - * - * Keep the original child Expression around so that when we convert back to a SortOrder we - * can pass that in. If we don't do that then GpuExpressions will end up being used to - * check if the sort order satisfies the child order and things won't match up (specifically - * AttributeReference.semanticEquals won't match GpuAttributeReference. - * - */ -case class GpuSortOrder( - child: GpuExpression, - direction: SortDirection, - nullOrdering: NullOrdering, - sameOrderExpressions: Set[Expression], - private val origChild: Expression) - extends GpuUnevaluableUnaryExpression { - - /** Sort order is not foldable because we don't have an eval for it. */ - override def foldable: Boolean = false - - override def checkInputDataTypes(): TypeCheckResult = { - if (RowOrdering.isOrderable(dataType)) { - TypeCheckResult.TypeCheckSuccess - } else { - TypeCheckResult.TypeCheckFailure(s"cannot sort data type ${dataType.catalogString}") - } - } - - override def dataType: DataType = child.dataType - override def nullable: Boolean = child.nullable - - override def toString: String = s"$child ${direction.sql} ${nullOrdering.sql}" - override def sql: String = child.sql + " " + direction.sql + " " + nullOrdering.sql - - def isAscending: Boolean = direction == Ascending - - def toSortOrder: SortOrder = SortOrder(origChild, direction, nullOrdering, sameOrderExpressions) -} - -object GpuSortOrder { - def apply( - child: GpuExpression, - origChild: Expression, - direction: SortDirection, - sameOrderExpressions: Set[Expression] = Set.empty): GpuSortOrder = { - new GpuSortOrder(child, direction, direction.defaultNullOrdering, - sameOrderExpressions, origChild) - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala index c4935081ded..94d827cdf94 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala @@ -27,12 +27,14 @@ class GpuSortMergeJoinMeta( rule: ConfKeysAndIncompat) extends SparkPlanMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { - val leftKeys: Seq[ExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[ExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[ExprMeta[_]] = join.condition.map( + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition override def tagPlanForGpu(): Unit = { // Use conditions from Hash Join diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index b1c1c6e177d..c7bda99b9bc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, SortOrder} import org.apache.spark.sql.catalyst.expressions.objects.CreateExternalRow import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ @@ -188,17 +188,16 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { } } - private def getOptimizedSortOrder(plan: SparkPlan): Seq[GpuSortOrder] = { + private def getOptimizedSortOrder(plan: SparkPlan): Seq[SortOrder] = { plan.output.map { expr => val wrapped = GpuOverrides.wrapExpr(expr, conf, None) wrapped.tagForGpu() assert(wrapped.canThisBeReplaced) - GpuSortOrder( + SortOrder( wrapped.convertToGpu(), Ascending, Ascending.defaultNullOrdering, - Set.empty, - expr + Set.empty ) } } @@ -209,7 +208,10 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { } def assertIsOnTheGpu(exp: Expression, conf: RapidsConf): Unit = { - if (!exp.isInstanceOf[GpuExpression] && + // There are no GpuAttributeReference or GpuSortOrder + if (!exp.isInstanceOf[AttributeReference] && + !exp.isInstanceOf[SortOrder] && + !exp.isInstanceOf[GpuExpression] && !conf.testingAllowedNonGpu.contains(getBaseNameFromClass(exp.getClass.toString))) { throw new IllegalArgumentException(s"The expression $exp is not columnar ${exp.getClass}") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index 8bd414a668c..60111b059c3 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -21,7 +21,7 @@ import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -66,7 +66,7 @@ class GpuWindowExecMeta(windowExec: WindowExec, private val (inputWindowExpressions, resultColumnsOnly) = getWindowExpression - val windowExpressions: Seq[ExprMeta[NamedExpression]] = + val windowExpressions: Seq[BaseExprMeta[NamedExpression]] = inputWindowExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override def tagPlanForGpu(): Unit = { @@ -90,7 +90,7 @@ class GpuWindowExecMeta(windowExec: WindowExec, } case class GpuWindowExec( - windowExpressionAliases: Seq[GpuExpression], + windowExpressionAliases: Seq[Expression], child: SparkPlan, resultColumnsOnly: Boolean ) extends UnaryExecNode with GpuExec { @@ -131,8 +131,7 @@ case class GpuWindowExec( // Address bindings for all expressions evaluated by WindowExec. val boundProjectList = windowExpressionAliases.map( - alias => GpuBindReferences.bindReference(alias, child.output) - ) + alias => GpuBindReferences.bindReference(alias, child.output)) // Bind aggregation column. boundProjectList.map( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 8160764cb3e..9c11b0e358e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -84,7 +84,7 @@ class GpuWindowExpressionMeta( ) } -case class GpuWindowExpression(windowFunction: GpuExpression, windowSpec: GpuWindowSpecDefinition) +case class GpuWindowExpression(windowFunction: Expression, windowSpec: GpuWindowSpecDefinition) extends GpuExpression { override def children: Seq[Expression] = windowFunction :: windowSpec :: Nil @@ -99,11 +99,11 @@ case class GpuWindowExpression(windowFunction: GpuExpression, windowSpec: GpuWin override def sql: String = windowFunction.sql + " OVER " + windowSpec.sql - private var boundAggCol : GpuExpression = _ + private var boundAggCol : Expression = _ private val frameType : FrameType = windowSpec.frameSpecification.asInstanceOf[GpuSpecifiedWindowFrame].frameType - def setBoundAggCol(bound : GpuExpression) : Unit = { + def setBoundAggCol(bound : Expression) : Unit = { boundAggCol = bound } @@ -181,15 +181,16 @@ case class GpuWindowExpression(windowFunction: GpuExpression, windowSpec: GpuWin try { // Project required column batches. - groupingColsCB = GpuProjectExec.project(cb, windowSpec.partitionSpec) + groupingColsCB = GpuProjectExec.project(cb, windowSpec.partitionSpec) assert(windowSpec.orderSpec.size == 1, "Expected a single sort column.") - sortColsCB = GpuProjectExec.project(cb, windowSpec.orderSpec.map(_.child)) + sortColsCB = GpuProjectExec.project(cb, + windowSpec.orderSpec.map(_.child.asInstanceOf[GpuExpression])) aggregationColsCB = GpuProjectExec.project(cb, Seq(boundAggCol)) // Extract required columns columns. groupingCols = GpuColumnVector.extractColumns(groupingColsCB) - sortCols = GpuColumnVector.extractColumns(sortColsCB) + sortCols = GpuColumnVector.extractColumns(sortColsCB) aggregationCols = GpuColumnVector.extractColumns(aggregationColsCB) inputTable = new Table( ( groupingCols ++ sortCols ++ aggregationCols ).map(_.getBase) : _* ) @@ -234,7 +235,7 @@ case class GpuWindowExpression(windowFunction: GpuExpression, windowSpec: GpuWin object GpuWindowExpression { def getRowBasedWindowFrame(columnIndex : Int, - aggExpression : GpuExpression, + aggExpression : Expression, windowSpec : GpuSpecifiedWindowFrame) : WindowAggregate = { @@ -287,7 +288,7 @@ object GpuWindowExpression { def getRangeBasedWindowFrame(aggColumnIndex : Int, timeColumnIndex : Int, - aggExpression : GpuExpression, + aggExpression : Expression, windowSpec : GpuSpecifiedWindowFrame, timestampIsAscending : Boolean) : WindowAggregate = { @@ -341,7 +342,7 @@ object GpuWindowExpression { } } - def getBoundaryValue(boundary : GpuExpression) : Int = boundary match { + def getBoundaryValue(boundary : Expression) : Int = boundary match { case literal: GpuLiteral if literal.dataType.equals(IntegerType) => literal.value.asInstanceOf[Int] case literal: GpuLiteral if literal.dataType.equals(CalendarIntervalType) => @@ -360,11 +361,11 @@ class GpuWindowSpecDefinitionMeta( rule: ConfKeysAndIncompat) extends ExprMeta[WindowSpecDefinition](windowSpec, conf, parent, rule) { - val partitionSpec: Seq[ExprMeta[Expression]] = + val partitionSpec: Seq[BaseExprMeta[Expression]] = windowSpec.partitionSpec.map(wrapExpr(_, conf, Some(this))) - val orderSpec: Seq[ExprMeta[SortOrder]] = + val orderSpec: Seq[BaseExprMeta[SortOrder]] = windowSpec.orderSpec.map(wrapExpr(_, conf, Some(this))) - val windowFrame: ExprMeta[WindowFrame] = + val windowFrame: BaseExprMeta[WindowFrame] = wrapExpr(windowSpec.frameSpecification, conf, Some(this)) override val ignoreUnsetDataTypes: Boolean = true @@ -381,14 +382,14 @@ class GpuWindowSpecDefinitionMeta( override def convertToGpu(): GpuExpression = { GpuWindowSpecDefinition( partitionSpec.map(_.convertToGpu()), - orderSpec.map(_.convertToGpu().asInstanceOf[GpuSortOrder]), + orderSpec.map(_.convertToGpu().asInstanceOf[SortOrder]), windowFrame.convertToGpu().asInstanceOf[GpuWindowFrame]) } } case class GpuWindowSpecDefinition( - partitionSpec: Seq[GpuExpression], - orderSpec: Seq[GpuSortOrder], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], frameSpecification: GpuWindowFrame) extends GpuExpression with GpuUnevaluable { @@ -577,8 +578,8 @@ case object GpuUnspecifiedFrame extends GpuWindowFrame // Placeholder, to handle // This class closely follows what's done in SpecifiedWindowFrame. case class GpuSpecifiedWindowFrame( frameType: FrameType, - lower: GpuExpression, - upper: GpuExpression) + lower: Expression, + upper: Expression) extends GpuWindowFrame { override def children: Seq[Expression] = lower :: upper :: Nil @@ -712,11 +713,11 @@ case class GpuRowNumber() extends GpuAggregateWindowFunction { protected val zero: GpuLiteral = GpuLiteral(0, IntegerType) protected val one : GpuLiteral = GpuLiteral(1, IntegerType) - protected val rowNumber : GpuAttributeReference = - GpuAttributeReference("rowNumber", IntegerType)() - override def aggBufferAttributes: Seq[GpuAttributeReference] = rowNumber :: Nil + protected val rowNumber : AttributeReference = + AttributeReference("rowNumber", IntegerType)() + override def aggBufferAttributes: Seq[AttributeReference] = rowNumber :: Nil override val initialValues: Seq[GpuExpression] = zero :: Nil - override val updateExpressions: Seq[GpuExpression] = rowNumber :: one :: Nil - override val evaluateExpression : GpuExpression = rowNumber + override val updateExpressions: Seq[Expression] = rowNumber :: one :: Nil + override val evaluateExpression: Expression = rowNumber override val inputProjection: Seq[GpuExpression] = Nil } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 088b66b96bd..b02182af8ff 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -74,7 +74,7 @@ abstract class RapidsMeta[INPUT <: BASE, BASE, OUTPUT <: BASE]( /** * The wrapped expressions that should be examined */ - val childExprs: Seq[ExprMeta[_]] + val childExprs: Seq[BaseExprMeta[_]] /** * The wrapped scans that should be examined @@ -286,7 +286,7 @@ abstract class PartMeta[INPUT <: Partitioning](part: INPUT, extends RapidsMeta[INPUT, Partitioning, GpuPartitioning](part, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty - override val childExprs: Seq[ExprMeta[_]] = Seq.empty + override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty override val childScans: Seq[ScanMeta[_]] = Seq.empty override val childParts: Seq[PartMeta[_]] = Seq.empty override val childDataWriteCmds: Seq[DataWritingCommandMeta[_]] = Seq.empty @@ -328,7 +328,7 @@ abstract class ScanMeta[INPUT <: Scan](scan: INPUT, extends RapidsMeta[INPUT, Scan, Scan](scan, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty - override val childExprs: Seq[ExprMeta[_]] = Seq.empty + override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty override val childScans: Seq[ScanMeta[_]] = Seq.empty override val childParts: Seq[PartMeta[_]] = Seq.empty override val childDataWriteCmds: Seq[DataWritingCommandMeta[_]] = Seq.empty @@ -364,7 +364,7 @@ abstract class DataWritingCommandMeta[INPUT <: DataWritingCommand]( extends RapidsMeta[INPUT, DataWritingCommand, GpuDataWritingCommand](cmd, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty - override val childExprs: Seq[ExprMeta[_]] = Seq.empty + override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty override val childScans: Seq[ScanMeta[_]] = Seq.empty override val childParts: Seq[PartMeta[_]] = Seq.empty override val childDataWriteCmds: Seq[DataWritingCommandMeta[_]] = Seq.empty @@ -400,7 +400,7 @@ abstract class SparkPlanMeta[INPUT <: SparkPlan](plan: INPUT, override val childPlans: Seq[SparkPlanMeta[_]] = plan.children.map(GpuOverrides.wrapPlan(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = plan.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override val childScans: Seq[ScanMeta[_]] = Seq.empty override val childParts: Seq[PartMeta[_]] = Seq.empty @@ -556,15 +556,15 @@ final class RuleNotFoundSparkPlanMeta[INPUT <: SparkPlan]( /** * Base class for metadata around `Expression`. */ -abstract class ExprMeta[INPUT <: Expression]( +abstract class BaseExprMeta[INPUT <: Expression]( expr: INPUT, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends RapidsMeta[INPUT, Expression, GpuExpression](expr, conf, parent, rule) { + extends RapidsMeta[INPUT, Expression, Expression](expr, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = expr.children.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override val childScans: Seq[ScanMeta[_]] = Seq.empty override val childParts: Seq[PartMeta[_]] = Seq.empty @@ -601,6 +601,16 @@ abstract class ExprMeta[INPUT <: Expression]( def tagExprForGpu(): Unit = {} } +abstract class ExprMeta[INPUT <: Expression]( + expr: INPUT, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends BaseExprMeta[INPUT](expr, conf, parent, rule) { + + override def convertToGpu(): GpuExpression +} + /** * Base class for metadata around `UnaryExpression`. */ @@ -614,7 +624,7 @@ abstract class UnaryExprMeta[INPUT <: UnaryExpression]( override final def convertToGpu(): GpuExpression = convertToGpu(childExprs(0).convertToGpu()) - def convertToGpu(child: GpuExpression): GpuExpression + def convertToGpu(child: Expression): GpuExpression } /** @@ -630,7 +640,7 @@ abstract class AggExprMeta[INPUT <: AggregateFunction]( override final def convertToGpu(): GpuExpression = convertToGpu(childExprs(0).convertToGpu()) - def convertToGpu(child: GpuExpression): GpuExpression + def convertToGpu(child: Expression): GpuExpression } /** @@ -646,7 +656,7 @@ abstract class BinaryExprMeta[INPUT <: BinaryExpression]( override final def convertToGpu(): GpuExpression = convertToGpu(childExprs(0).convertToGpu(), childExprs(1).convertToGpu()) - def convertToGpu(lhs: GpuExpression, rhs: GpuExpression): GpuExpression + def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression } /** @@ -663,8 +673,8 @@ abstract class TernaryExprMeta[INPUT <: TernaryExpression]( convertToGpu(childExprs(0).convertToGpu(), childExprs(1).convertToGpu(), childExprs(2).convertToGpu()) - def convertToGpu(val0: GpuExpression, val1: GpuExpression, - val2: GpuExpression): GpuExpression + def convertToGpu(val0: Expression, val1: Expression, + val2: Expression): GpuExpression } abstract class String2TrimExpressionMeta[INPUT <: String2TrimExpression]( @@ -690,7 +700,7 @@ abstract class String2TrimExpressionMeta[INPUT <: String2TrimExpression]( convertToGpu(childExprs(0).convertToGpu(), trimParam) } - def convertToGpu(column: GpuExpression, target: Option[GpuExpression] = None): GpuExpression + def convertToGpu(column: Expression, target: Option[Expression] = None): GpuExpression } /** @@ -705,7 +715,7 @@ abstract class ComplexTypeMergingExprMeta[INPUT <: ComplexTypeMergingExpression] override final def convertToGpu(): GpuExpression = convertToGpu(childExprs.map(_.convertToGpu())) - def convertToGpu(childExprs: Seq[GpuExpression]): GpuExpression + def convertToGpu(childExprs: Seq[Expression]): GpuExpression } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala index a6504326ac3..5b05cd75385 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala @@ -20,7 +20,7 @@ import scala.collection.mutable.ArrayBuffer import com.nvidia.spark.rapids.GpuExpressionsUtils.evaluateBoundExpressions -import org.apache.spark.sql.catalyst.expressions.{NullsFirst, NullsLast} +import org.apache.spark.sql.catalyst.expressions.{NullsFirst, NullsLast, SortOrder} import org.apache.spark.sql.vectorized.ColumnarBatch object SortUtils { @@ -30,9 +30,9 @@ object SortUtils { * after the sort key columns. The sort key columns will be dropped after sorting. */ def getGpuColVectorsAndBindReferences(batch: ColumnarBatch, - boundInputReferences: Seq[GpuSortOrder]): Seq[GpuColumnVector] = { + boundInputReferences: Seq[SortOrder]): Seq[GpuColumnVector] = { val sortCvs = new ArrayBuffer[GpuColumnVector](boundInputReferences.length) - val childExprs = boundInputReferences.map(_.child) + val childExprs = boundInputReferences.map(_.child.asInstanceOf[GpuExpression]) sortCvs ++= evaluateBoundExpressions(batch, childExprs) val originalColumns = GpuColumnVector.extractColumns(batch) originalColumns.foreach(_.incRefCount()) @@ -42,7 +42,7 @@ object SortUtils { /* * Return true if nulls are needed first and ordering is ascending and vice versa */ - def areNullsSmallest(order: GpuSortOrder): Boolean = { + def areNullsSmallest(order: SortOrder): Boolean = { (order.isAscending && order.nullOrdering == NullsFirst) || (!order.isAscending && order.nullOrdering == NullsLast) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index c350e866d3b..a683ebe5e5a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -26,7 +26,7 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSeq, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSeq, AttributeSet, Expression, NamedExpression} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.util.truncatedString @@ -43,18 +43,18 @@ class GpuHashAggregateMeta( parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) extends SparkPlanMeta[HashAggregateExec](agg, conf, parent, rule) { - private val requiredChildDistributionExpressions: Option[Seq[ExprMeta[_]]] = + private val requiredChildDistributionExpressions: Option[Seq[BaseExprMeta[_]]] = agg.requiredChildDistributionExpressions.map(_.map(GpuOverrides.wrapExpr(_, conf, Some(this)))) - private val groupingExpressions: Seq[ExprMeta[_]] = + private val groupingExpressions: Seq[BaseExprMeta[_]] = agg.groupingExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val aggregateExpressions: Seq[ExprMeta[_]] = + private val aggregateExpressions: Seq[BaseExprMeta[_]] = agg.aggregateExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val aggregateAttributes: Seq[ExprMeta[_]] = + private val aggregateAttributes: Seq[BaseExprMeta[_]] = agg.aggregateAttributes.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val resultExpressions: Seq[ExprMeta[_]] = + private val resultExpressions: Seq[BaseExprMeta[_]] = agg.resultExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = requiredChildDistributionExpressions.getOrElse(Seq.empty) ++ groupingExpressions ++ aggregateExpressions ++ @@ -109,15 +109,16 @@ class GpuHashAggregateMeta( } } - override def convertToGpu(): GpuExec = + override def convertToGpu(): GpuExec = { GpuHashAggregateExec( requiredChildDistributionExpressions.map(_.map(_.convertToGpu())), groupingExpressions.map(_.convertToGpu()), aggregateExpressions.map(_.convertToGpu()).asInstanceOf[Seq[GpuAggregateExpression]], - aggregateAttributes.map(_.convertToGpu()).asInstanceOf[Seq[GpuAttributeReference]], + aggregateAttributes.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], agg.initialInputBufferOffset, resultExpressions.map(_.convertToGpu()).asInstanceOf[Seq[NamedExpression]], childPlans(0).convertIfNeeded()) + } } class GpuSortAggregateMeta( @@ -126,18 +127,18 @@ class GpuSortAggregateMeta( parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) extends SparkPlanMeta[SortAggregateExec](agg, conf, parent, rule) { - private val requiredChildDistributionExpressions: Option[Seq[ExprMeta[_]]] = + private val requiredChildDistributionExpressions: Option[Seq[BaseExprMeta[_]]] = agg.requiredChildDistributionExpressions.map(_.map(GpuOverrides.wrapExpr(_, conf, Some(this)))) - private val groupingExpressions: Seq[ExprMeta[_]] = + private val groupingExpressions: Seq[BaseExprMeta[_]] = agg.groupingExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val aggregateExpressions: Seq[ExprMeta[_]] = + private val aggregateExpressions: Seq[BaseExprMeta[_]] = agg.aggregateExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val aggregateAttributes: Seq[ExprMeta[_]] = + private val aggregateAttributes: Seq[BaseExprMeta[_]] = agg.aggregateAttributes.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private val resultExpressions: Seq[ExprMeta[_]] = + private val resultExpressions: Seq[BaseExprMeta[_]] = agg.resultExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = + override val childExprs: Seq[BaseExprMeta[_]] = requiredChildDistributionExpressions.getOrElse(Seq.empty) ++ groupingExpressions ++ aggregateExpressions ++ @@ -197,7 +198,7 @@ class GpuSortAggregateMeta( requiredChildDistributionExpressions.map(_.map(_.convertToGpu())), groupingExpressions.map(_.convertToGpu()), aggregateExpressions.map(_.convertToGpu()).asInstanceOf[Seq[GpuAggregateExpression]], - aggregateAttributes.map(_.convertToGpu()).asInstanceOf[Seq[GpuAttributeReference]], + aggregateAttributes.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], agg.initialInputBufferOffset, resultExpressions.map(_.convertToGpu()).asInstanceOf[Seq[NamedExpression]], childPlans(0).convertIfNeeded()) @@ -224,18 +225,18 @@ class GpuSortAggregateMeta( * node should project) * @param child incoming plan (where we get input columns from) */ -case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq[GpuExpression]], - groupingExpressions: Seq[GpuExpression], - aggregateExpressions: Seq[GpuAggregateExpression], - aggregateAttributes: Seq[GpuAttributeReference], - initialInputBufferOffset: Int, - //TODO: make resultExpressions a GpuNamedExpression - resultExpressions: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with GpuExec with Arm { - - case class BoundExpressionsModeAggregates(boundInputReferences: Seq[GpuExpression] , - boundFinalProjections: Option[scala.Seq[GpuExpression]], - boundResultReferences: scala.Seq[GpuExpression] , +case class GpuHashAggregateExec( + requiredChildDistributionExpressions: Option[Seq[Expression]], + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[GpuAggregateExpression], + aggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + child: SparkPlan) extends UnaryExecNode with GpuExec with Arm { + + case class BoundExpressionsModeAggregates(boundInputReferences: Seq[Expression] , + boundFinalProjections: Option[scala.Seq[Expression]], + boundResultReferences: scala.Seq[Expression] , aggModeCudfAggregates: scala.Seq[(AggregateMode, scala.Seq[CudfAggregate])]) // This handles GPU hash aggregation without spilling. // @@ -429,7 +430,8 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq // aggregatedCb is made up of ColumnVectors // and the final projections from the aggregates won't change that, // so we can assume they will be vectors after we eval - ref.columnarEval(aggregatedCb).asInstanceOf[GpuColumnVector] + ref.asInstanceOf[GpuExpression] + .columnarEval(aggregatedCb).asInstanceOf[GpuColumnVector] } aggregatedCb.close() aggregatedCb = null @@ -447,7 +449,7 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq // Perform the last project to get the correct shape that Spark expects. Note this will // add things like literals, that were not part of the aggregate into the batch. resultCvs = boundExpression.boundResultReferences.map { ref => - val result = ref.columnarEval(finalCb) + val result = ref.asInstanceOf[GpuExpression].columnarEval(finalCb) // Result references can be virtually anything, we need to coerce // them to be vectors since this is going into a ColumnarBatch result match { @@ -509,10 +511,9 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq } private def processIncomingBatch(batch: ColumnarBatch, - boundInputReferences: Seq[GpuExpression]): Seq[GpuColumnVector] = { - + boundInputReferences: Seq[Expression]): Seq[GpuColumnVector] = { boundInputReferences.safeMap { ref => - val in = ref.columnarEval(batch) + val in = ref.asInstanceOf[GpuExpression].columnarEval(batch) val childCv = in match { case cv: ColumnVector => cv.asInstanceOf[GpuColumnVector] case _ => @@ -583,7 +584,7 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq * expression in allExpressions */ def setupReferences(childAttr: AttributeSeq, - groupingExpressions: Seq[GpuExpression], + groupingExpressions: Seq[Expression], aggregateExpressions: Seq[GpuAggregateExpression]): BoundExpressionsModeAggregates = { val groupingAttributes = groupingExpressions.map(_.asInstanceOf[NamedExpression].toAttribute) @@ -640,13 +641,13 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq _.aggregateFunction.aggBufferAttributes) // Partial with no distinct or when modes are empty - val inputProjections: Seq[GpuExpression] = groupingExpressions ++ aggregateExpressions + val inputProjections: Seq[Expression] = groupingExpressions ++ aggregateExpressions .flatMap(_.aggregateFunction.inputProjection) var distinctAttributes = Seq[Attribute]() - var distinctExpressions = Seq[GpuExpression]() + var distinctExpressions = Seq[Expression]() var nonDistinctAttributes = Seq[Attribute]() - var nonDistinctExpressions = Seq[GpuExpression]() + var nonDistinctExpressions = Seq[Expression]() uniqueModes.foreach { case PartialMerge => nonDistinctAttributes = mergeAttributesNonDistinct @@ -688,7 +689,8 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq val boundInputReferences = if (uniqueModes.contains(PartialMerge)) { GpuBindReferences.bindReferences(inputBindExpressions, resultingBindAttributes) } else if (finalMode) { - GpuBindReferences.bindReferences(childAttr, childAttr) + GpuBindReferences.bindReferences(childAttr.attrs.asInstanceOf[Seq[Expression]], childAttr) + .asInstanceOf[Seq[GpuExpression]] } else { GpuBindReferences.bindReferences(inputProjections, childAttr) } @@ -716,18 +718,18 @@ case class GpuHashAggregateExec(requiredChildDistributionExpressions: Option[Seq } else if (finalMode || completeMode) { GpuBindReferences.bindReferences( resultExpressions.asInstanceOf[Seq[GpuExpression]], - finalAttributes.asInstanceOf[Seq[GpuAttributeReference]]) + finalAttributes.asInstanceOf[Seq[Attribute]]) } else { GpuBindReferences.bindReferences( resultExpressions.asInstanceOf[Seq[GpuExpression]], - groupingAttributes.asInstanceOf[Seq[GpuAttributeReference]]) + groupingAttributes.asInstanceOf[Seq[Attribute]]) } BoundExpressionsModeAggregates(boundInputReferences, boundFinalProjections, boundResultReferences, aggModeCudfAggregates) } def computeAggregate(toAggregateCvs: Seq[GpuColumnVector], - groupingExpressions: Seq[GpuExpression], + groupingExpressions: Seq[Expression], aggModeCudfAggregates : Seq[(AggregateMode, Seq[CudfAggregate])], merge : Boolean, computeAggTime: SQLMetric): ColumnarBatch = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index ccd9866c7e9..061e467b7f9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuProjectExec { - def projectAndClose[A <: GpuExpression](cb: ColumnarBatch, boundExprs: Seq[A], + def projectAndClose[A <: Expression](cb: ColumnarBatch, boundExprs: Seq[A], totalTime: SQLMetric): ColumnarBatch = { val nvtxRange = new NvtxWithMetrics("ProjectExec", NvtxColor.CYAN, totalTime) try { @@ -43,10 +43,10 @@ object GpuProjectExec { } } - def project[A <: GpuExpression](cb: ColumnarBatch, boundExprs: Seq[A]): ColumnarBatch = { + def project[A <: Expression](cb: ColumnarBatch, boundExprs: Seq[A]): ColumnarBatch = { val newColumns = boundExprs.safeMap { expr => { - val result = expr.columnarEval(cb) + val result = expr.asInstanceOf[GpuExpression].columnarEval(cb) result match { case cv: ColumnVector => cv case other => @@ -62,7 +62,7 @@ object GpuProjectExec { } } -case class GpuProjectExec(projectList: Seq[GpuExpression], child: SparkPlan) +case class GpuProjectExec(projectList: Seq[Expression], child: SparkPlan) extends UnaryExecNode with GpuExec { private val sparkProjectList = projectList.asInstanceOf[Seq[NamedExpression]] @@ -96,7 +96,7 @@ case class GpuProjectExec(projectList: Seq[GpuExpression], child: SparkPlan) object GpuFilter { def apply( batch: ColumnarBatch, - boundCondition: GpuExpression, + boundCondition: Expression, numOutputRows: SQLMetric, numOutputBatches: SQLMetric, filterTime: SQLMetric): ColumnarBatch = { @@ -106,7 +106,8 @@ object GpuFilter { var tbl: cudf.Table = null var filtered: cudf.Table = null val filteredBatch = try { - filterConditionCv = boundCondition.columnarEval(batch).asInstanceOf[GpuColumnVector] + filterConditionCv = boundCondition.asInstanceOf[GpuExpression] + .columnarEval(batch).asInstanceOf[GpuColumnVector] tbl = GpuColumnVector.from(batch) filtered = tbl.filter(filterConditionCv.getBase) GpuColumnVector.from(filtered) @@ -123,7 +124,7 @@ object GpuFilter { } } -case class GpuFilterExec(condition: GpuExpression, child: SparkPlan) +case class GpuFilterExec(condition: Expression, child: SparkPlan) extends UnaryExecNode with PredicateHelper with GpuExec { // Split out all the IsNotNulls from condition. @@ -169,7 +170,6 @@ case class GpuFilterExec(condition: GpuExpression, child: SparkPlan) val totalTime = longMetric(TOTAL_TIME) val boundCondition = GpuBindReferences.bindReference(condition, child.output) val rdd = child.executeColumnar() - rdd.map { batch => GpuFilter(batch, boundCondition, numOutputRows, numOutputBatches, totalTime) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala index dd6bd02f547..8ea035b04a8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.vectorized.ColumnarBatch abstract class GpuConditionalExpression extends ComplexTypeMergingExpression with GpuExpression { private def computePredicate( batch: ColumnarBatch, - predicateExpr: GpuExpression): GpuColumnVector = { - val predicate: Any = predicateExpr.columnarEval(batch) + predicateExpr: Expression): GpuColumnVector = { + val predicate: Any = predicateExpr.asInstanceOf[GpuExpression].columnarEval(batch) try { if (!predicate.isInstanceOf[GpuColumnVector]) { throw new IllegalStateException("Predicate result is not a column") @@ -52,12 +52,12 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit protected def computeIfElse( batch: ColumnarBatch, - predicateExpr: GpuExpression, - trueExpr: GpuExpression, + predicateExpr: Expression, + trueExpr: Expression, falseValues: GpuColumnVector): GpuColumnVector = { val predicate = computePredicate(batch, predicateExpr) try { - val trueResult: Any = trueExpr.columnarEval(batch) + val trueResult: Any = trueExpr.asInstanceOf[GpuExpression].columnarEval(batch) try { val result = trueResult match { case t: GpuColumnVector => predicate.getBase.ifElse(t.getBase, falseValues.getBase) @@ -84,12 +84,12 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit protected def computeIfElse( batch: ColumnarBatch, - predicateExpr: GpuExpression, - trueExpr: GpuExpression, + predicateExpr: Expression, + trueExpr: Expression, falseValue: Scalar): GpuColumnVector = { val predicate = computePredicate(batch, predicateExpr) try { - val trueResult: Any = trueExpr.columnarEval(batch) + val trueResult: Any = trueExpr.asInstanceOf[GpuExpression].columnarEval(batch) try { val result = trueResult match { case t: GpuColumnVector => predicate.getBase.ifElse(t.getBase, falseValue) @@ -116,10 +116,10 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit protected def computeIfElse( batch: ColumnarBatch, - predicateExpr: GpuExpression, - trueExpr: GpuExpression, - falseExpr: GpuExpression): GpuColumnVector = { - val falseResult: Any = falseExpr.columnarEval(batch) + predicateExpr: Expression, + trueExpr: Expression, + falseExpr: Expression): GpuColumnVector = { + val falseResult: Any = falseExpr.asInstanceOf[GpuExpression].columnarEval(batch) try { falseResult match { case f: GpuColumnVector => computeIfElse(batch, predicateExpr, trueExpr, f) @@ -142,9 +142,9 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit } case class GpuIf( - predicateExpr: GpuExpression, - trueExpr: GpuExpression, - falseExpr: GpuExpression) extends GpuConditionalExpression { + predicateExpr: Expression, + trueExpr: Expression, + falseExpr: Expression) extends GpuConditionalExpression { @transient override lazy val inputTypesForMerging: Seq[DataType] = { @@ -177,8 +177,8 @@ case class GpuIf( case class GpuCaseWhen( - branches: Seq[(GpuExpression, GpuExpression)], - elseValue: Option[GpuExpression] = None) extends GpuConditionalExpression with Serializable { + branches: Seq[(Expression, Expression)], + elseValue: Option[Expression] = None) extends GpuConditionalExpression with Serializable { override def children: Seq[Expression] = branches.flatMap(b => b._1 :: b._2 :: Nil) ++ elseValue diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala index 934cdd6e6f8..bbeadac9823 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala @@ -16,15 +16,16 @@ package com.nvidia.spark.rapids -import org.apache.spark.sql.catalyst.expressions.TaggingExpression +import org.apache.spark.sql.catalyst.expressions.{Expression, TaggingExpression} import org.apache.spark.sql.vectorized.ColumnarBatch /** * This is a TaggingExpression in spark, which gets matched in NormalizeFloatingNumbers (which is * a Rule). */ -// TODO: need coalesce as a feature request in cudf -case class GpuKnownFloatingPointNormalized(child: GpuExpression) extends TaggingExpression +case class GpuKnownFloatingPointNormalized(child: Expression) extends TaggingExpression with GpuExpression { - override def columnarEval(batch: ColumnarBatch): Any = child.columnarEval(batch) + override def columnarEval(batch: ColumnarBatch): Any = { + child.asInstanceOf[GpuExpression].columnarEval(batch) + } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala index a3793668399..535b33bf819 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala @@ -19,13 +19,13 @@ package com.nvidia.spark.rapids import java.util.Objects import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, ExprId, Generator, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, ExprId, Generator, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.types.{DataType, Metadata} import org.apache.spark.sql.vectorized.ColumnarBatch -case class GpuAlias(child: GpuExpression, name: String)( +case class GpuAlias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Seq[String] = Seq.empty, val explicitMetadata: Option[Metadata] = None) @@ -66,9 +66,8 @@ case class GpuAlias(child: GpuExpression, name: String)( override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix$delaySuffix" - override protected final def otherCopyArgs: Seq[AnyRef] = { + override protected final def otherCopyArgs: Seq[AnyRef] = exprId :: qualifier :: explicitMetadata :: Nil - } override def hashCode(): Int = { val state = Seq(name, exprId, child, qualifier, explicitMetadata) @@ -87,139 +86,9 @@ case class GpuAlias(child: GpuExpression, name: String)( s"${child.sql} AS $qualifierPrefix${quoteIdentifier(name)}" } - override def columnarEval(batch: ColumnarBatch): Any = child.columnarEval(batch) + override def columnarEval(batch: ColumnarBatch): Any = + child.asInstanceOf[GpuExpression].columnarEval(batch) override def doColumnar(input: GpuColumnVector): GpuColumnVector = throw new IllegalStateException("GpuAlias should never have doColumnar called") } - -object GpuAttributeReference { - def from(attr: Attribute): GpuAttributeReference = attr match { - case attr: GpuAttributeReference => attr - case attr: AttributeReference => - GpuAttributeReference( - attr.name, attr.dataType, attr.nullable, attr.metadata)(attr.exprId, attr.qualifier) - case attr => throw new IllegalStateException(s"Unexpected attribute $attr") - } -} - -case class GpuAttributeReference( - name: String, - dataType: DataType, - nullable: Boolean = true, - override val metadata: Metadata = Metadata.empty)( - override val exprId: ExprId = NamedExpression.newExprId, - override val qualifier: Seq[String] = Seq.empty[String]) - extends Attribute with GpuExpression { - - /** - * Returns true iff the expression id is the same for both attributes. - */ - def sameRef(other: GpuAttributeReference): Boolean = this.exprId == other.exprId - - override def equals(other: Any): Boolean = other match { - case ar: GpuAttributeReference => - name == ar.name && dataType == ar.dataType && nullable == ar.nullable && - metadata == ar.metadata && exprId == ar.exprId && qualifier == ar.qualifier - case _ => false - } - - override def semanticEquals(other: Expression): Boolean = other match { - case ar: GpuAttributeReference => sameRef(ar) - case _ => false - } - - override def semanticHash(): Int = { - this.exprId.hashCode() - } - - override def hashCode(): Int = { - // See http://stackoverflow.com/questions/113511/hash-code-implementation - var h = 17 - h = h * 37 + name.hashCode() - h = h * 37 + dataType.hashCode() - h = h * 37 + nullable.hashCode() - h = h * 37 + metadata.hashCode() - h = h * 37 + exprId.hashCode() - h = h * 37 + qualifier.hashCode() - h - } - - override def newInstance(): GpuAttributeReference = - GpuAttributeReference(name, dataType, nullable, metadata)(qualifier = qualifier) - - /** - * Returns a copy of this [[GpuAttributeReference]] with changed nullability. - */ - override def withNullability(newNullability: Boolean): GpuAttributeReference = { - if (nullable == newNullability) { - this - } else { - GpuAttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier) - } - } - - override def withName(newName: String): GpuAttributeReference = { - if (name == newName) { - this - } else { - GpuAttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier) - } - } - - /** - * Returns a copy of this [[GpuAttributeReference]] with new qualifier. - */ - override def withQualifier(newQualifier: Seq[String]): GpuAttributeReference = { - if (newQualifier == qualifier) { - this - } else { - GpuAttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier) - } - } - - override def withExprId(newExprId: ExprId): GpuAttributeReference = { - if (exprId == newExprId) { - this - } else { - GpuAttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier) - } - } - - override def withMetadata(newMetadata: Metadata): GpuAttributeReference = { - GpuAttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier) - } - - override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: Nil - } - - /** Used to signal the column used to calculate an eventTime watermark (e.g. a#1-T{delayMs}) */ - private def delaySuffix = if (metadata.contains(EventTimeWatermark.delayKey)) { - s"-T${metadata.getLong(EventTimeWatermark.delayKey)}ms" - } else { - "" - } - - override def toString: String = s"$name#${exprId.id}$typeSuffix$delaySuffix" - - // Since the expression id is not in the first constructor it is missing from the default - // tree string. - override def simpleString(maxFields: Int): String = { - s"$name#${exprId.id}: ${dataType.simpleString(maxFields)}" - } - - override def sql: String = { - val qualifierPrefix = if (qualifier.nonEmpty) qualifier.mkString(".") + "." else "" - s"$qualifierPrefix${quoteIdentifier(name)}" - } - - override def toAttribute: Attribute = - AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier) - - @transient - override lazy val references: AttributeSet = AttributeSet(toAttribute) - - override def columnarEval(batch: ColumnarBatch): Any = - throw new IllegalStateException("Attribute executed without being bound") -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala index c88513cb442..b8069f653d4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala @@ -45,7 +45,7 @@ object GpuNvl { } } -case class GpuCoalesce(children: Seq[GpuExpression]) extends GpuExpression with +case class GpuCoalesce(children: Seq[Expression]) extends GpuExpression with ComplexTypeMergingExpression { override def columnarEval(batch: ColumnarBatch): Any = { @@ -54,7 +54,7 @@ case class GpuCoalesce(children: Seq[GpuExpression]) extends GpuExpression with var runningScalar: Scalar = null try { children.reverse.foreach(expr => { - expr.columnarEval(batch) match { + expr.asInstanceOf[GpuExpression].columnarEval(batch) match { case data: GpuColumnVector => try { if (runningResult != null) { @@ -119,7 +119,7 @@ case class GpuCoalesce(children: Seq[GpuExpression]) extends GpuExpression with * UnaryOp */ -case class GpuIsNull(child: GpuExpression) extends GpuUnaryExpression with Predicate { +case class GpuIsNull(child: Expression) extends GpuUnaryExpression with Predicate { override def nullable: Boolean = false override def sql: String = s"(${child.sql} IS NULL)" @@ -128,7 +128,7 @@ case class GpuIsNull(child: GpuExpression) extends GpuUnaryExpression with Predi GpuColumnVector.from(input.getBase.isNull) } -case class GpuIsNotNull(child: GpuExpression) extends GpuUnaryExpression with Predicate { +case class GpuIsNotNull(child: Expression) extends GpuUnaryExpression with Predicate { override def nullable: Boolean = false override def sql: String = s"(${child.sql} IS NOT NULL)" @@ -137,7 +137,7 @@ case class GpuIsNotNull(child: GpuExpression) extends GpuUnaryExpression with Pr GpuColumnVector.from(input.getBase.isNotNull) } -case class GpuIsNan(child: GpuExpression) extends GpuUnaryExpression with Predicate { +case class GpuIsNan(child: Expression) extends GpuUnaryExpression with Predicate { override def nullable: Boolean = false override def sql: String = s"(${child.sql} IS NAN)" @@ -152,7 +152,7 @@ case class GpuIsNan(child: GpuExpression) extends GpuUnaryExpression with Predic */ case class GpuAtLeastNNonNulls( n: Int, - exprs: Seq[GpuExpression]) + exprs: Seq[Expression]) extends GpuExpression with Predicate { override def nullable: Boolean = false @@ -180,7 +180,8 @@ case class GpuAtLeastNNonNulls( var notNanVector: ColumnVector = null var nanAndNullVector: ColumnVector = null try { - cv = expr.columnarEval(batch).asInstanceOf[GpuColumnVector].getBase + cv = expr.asInstanceOf[GpuExpression] + .columnarEval(batch).asInstanceOf[GpuColumnVector].getBase notNullVector = cv.isNotNull if (cv.getType == DType.FLOAT32 || cv.getType == DType.FLOAT64) { notNanVector = cv.isNotNan @@ -247,7 +248,7 @@ case class GpuAtLeastNNonNulls( } } -case class GpuNaNvl(left: GpuExpression, right: GpuExpression) extends GpuBinaryExpression { +case class GpuNaNvl(left: Expression, right: Expression) extends GpuBinaryExpression { override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): GpuColumnVector = { var islhsNotNan: ColumnVector = null try { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 8374f15665b..1ae5d44b372 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -21,7 +21,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} -import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, ExprId, ImplicitCastInputTypes, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, ExprId, ImplicitCastInputTypes, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, Complete, Final, Partial, PartialMerge} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BooleanType, DataType, DoubleType, LongType, NumericType, StructType} @@ -29,7 +29,7 @@ import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, BooleanType, D trait GpuAggregateFunction extends GpuExpression { // using the child reference, define the shape of the vectors sent to // the update/merge expressions - val inputProjection: Seq[GpuExpression] + val inputProjection: Seq[Expression] /** An aggregate function is not foldable. */ final override def foldable: Boolean = false @@ -38,14 +38,14 @@ trait GpuAggregateFunction extends GpuExpression { def aggBufferSchema: StructType /** Attributes of fields in aggBufferSchema. */ - def aggBufferAttributes: Seq[GpuAttributeReference] + def aggBufferAttributes: Seq[AttributeReference] /** * Attributes of fields in input aggregation buffers (immutable aggregation buffers that are * merged with mutable aggregation buffers in the merge() function or merge expressions). * These attributes are created automatically by cloning the [[aggBufferAttributes]]. */ - def inputAggBufferAttributes: Seq[GpuAttributeReference] + def inputAggBufferAttributes: Seq[AttributeReference] /** * Result of the aggregate function when the input is empty. This is currently only used for the @@ -70,11 +70,11 @@ trait GpuAggregateFunction extends GpuExpression { case class GpuAggregateExpression(origAggregateFunction: GpuAggregateFunction, mode: AggregateMode, isDistinct: Boolean, - filter: Option[GpuExpression], + filter: Option[Expression], resultId: ExprId) extends GpuExpression with GpuUnevaluable { - case class WrappedAggFunction(aggregateFunction: GpuAggregateFunction, filter: GpuExpression) + case class WrappedAggFunction(aggregateFunction: GpuAggregateFunction, filter: Expression) extends GpuDeclarativeAggregate { override val inputProjection: Seq[GpuExpression] = { val caseWhenExpressions = aggregateFunction.inputProjection.map {ip => @@ -83,7 +83,7 @@ case class GpuAggregateExpression(origAggregateFunction: GpuAggregateFunction, caseWhenExpressions } /** Attributes of fields in aggBufferSchema. */ - override def aggBufferAttributes: Seq[GpuAttributeReference] = + override def aggBufferAttributes: Seq[AttributeReference] = aggregateFunction.aggBufferAttributes override def nullable: Boolean = aggregateFunction.nullable @@ -93,11 +93,11 @@ case class GpuAggregateExpression(origAggregateFunction: GpuAggregateFunction, override val initialValues: Seq[GpuExpression] = aggregateFunction.asInstanceOf[GpuDeclarativeAggregate].initialValues - override val updateExpressions: Seq[GpuExpression] = + override val updateExpressions: Seq[Expression] = aggregateFunction.asInstanceOf[GpuDeclarativeAggregate].updateExpressions override val mergeExpressions: Seq[GpuExpression] = aggregateFunction.asInstanceOf[GpuDeclarativeAggregate].mergeExpressions - override val evaluateExpression: GpuExpression = + override val evaluateExpression: Expression = aggregateFunction.asInstanceOf[GpuDeclarativeAggregate].evaluateExpression } @@ -117,7 +117,7 @@ case class GpuAggregateExpression(origAggregateFunction: GpuAggregateFunction, // For PartialMerge or Final mode, the input to the `aggregateFunction` is aggregate buffers, // and the actual children of `aggregateFunction` is not used, here we normalize the expr id. case PartialMerge | Final => aggregateFunction.transform { - case a: GpuAttributeReference => a.withExprId(ExprId(0)) + case a: AttributeReference => a.withExprId(ExprId(0)) } case Partial | Complete => aggregateFunction } @@ -154,10 +154,10 @@ case class GpuAggregateExpression(origAggregateFunction: GpuAggregateFunction, override def sql: String = aggregateFunction.sql(isDistinct) } -abstract case class CudfAggregate(ref: GpuExpression) extends GpuUnevaluable { +abstract case class CudfAggregate(ref: Expression) extends GpuUnevaluable { // we use this to get the ordinal of the bound reference, s.t. we can ask cudf to perform // the aggregate on that column - def getOrdinal(ref: GpuExpression): Int = ref.asInstanceOf[GpuBoundReference].ordinal + def getOrdinal(ref: Expression): Int = ref.asInstanceOf[GpuBoundReference].ordinal val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar val mergeReductionAggregate: cudf.ColumnVector => cudf.Scalar val updateAggregate: cudf.Aggregate @@ -168,7 +168,7 @@ abstract case class CudfAggregate(ref: GpuExpression) extends GpuUnevaluable { def children: Seq[Expression] = ref :: Nil } -class CudfCount(ref: GpuExpression) extends CudfAggregate(ref) { +class CudfCount(ref: Expression) extends CudfAggregate(ref) { // includeNulls set to false in count aggregate to exclude nulls while calculating count(column) val includeNulls = false override val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar = @@ -181,7 +181,7 @@ class CudfCount(ref: GpuExpression) extends CudfAggregate(ref) { override def toString(): String = "CudfCount" } -class CudfSum(ref: GpuExpression) extends CudfAggregate(ref) { +class CudfSum(ref: Expression) extends CudfAggregate(ref) { override val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => col.sum override val mergeReductionAggregate: cudf.ColumnVector => cudf.Scalar = @@ -191,7 +191,7 @@ class CudfSum(ref: GpuExpression) extends CudfAggregate(ref) { override def toString(): String = "CudfSum" } -class CudfMax(ref: GpuExpression) extends CudfAggregate(ref) { +class CudfMax(ref: Expression) extends CudfAggregate(ref) { override val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => col.max override val mergeReductionAggregate: cudf.ColumnVector => cudf.Scalar = @@ -201,7 +201,7 @@ class CudfMax(ref: GpuExpression) extends CudfAggregate(ref) { override def toString(): String = "CudfMax" } -class CudfMin(ref: GpuExpression) extends CudfAggregate(ref) { +class CudfMin(ref: Expression) extends CudfAggregate(ref) { override val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => col.min override val mergeReductionAggregate: cudf.ColumnVector => cudf.Scalar = @@ -211,7 +211,7 @@ class CudfMin(ref: GpuExpression) extends CudfAggregate(ref) { override def toString(): String = "CudfMin" } -abstract class CudfFirstLastBase(ref: GpuExpression) extends CudfAggregate(ref) { +abstract class CudfFirstLastBase(ref: Expression) extends CudfAggregate(ref) { override val updateReductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => throw new UnsupportedOperationException("first/last reduction not supported on GPU") @@ -220,7 +220,7 @@ abstract class CudfFirstLastBase(ref: GpuExpression) extends CudfAggregate(ref) throw new UnsupportedOperationException("first/last reduction not supported on GPU") } -class CudfFirstIncludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { +class CudfFirstIncludeNulls(ref: Expression) extends CudfFirstLastBase(ref) { val includeNulls = true override lazy val updateAggregate: cudf.Aggregate = cudf.Table.first(getOrdinal(ref), includeNulls) @@ -228,7 +228,7 @@ class CudfFirstIncludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { cudf.Table.first(getOrdinal(ref), includeNulls) } -class CudfFirstExcludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { +class CudfFirstExcludeNulls(ref: Expression) extends CudfFirstLastBase(ref) { val includeNulls = false override lazy val updateAggregate: cudf.Aggregate = cudf.Table.first(getOrdinal(ref), includeNulls) @@ -236,14 +236,14 @@ class CudfFirstExcludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { cudf.Table.first(getOrdinal(ref), includeNulls) } -class CudfLastIncludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { +class CudfLastIncludeNulls(ref: Expression) extends CudfFirstLastBase(ref) { val includeNulls = true override lazy val updateAggregate: cudf.Aggregate = cudf.Table.last(getOrdinal(ref), includeNulls) override lazy val mergeAggregate: cudf.Aggregate = cudf.Table.last(getOrdinal(ref), includeNulls) } -class CudfLastExcludeNulls(ref: GpuExpression) extends CudfFirstLastBase(ref) { +class CudfLastExcludeNulls(ref: Expression) extends CudfFirstLastBase(ref) { val includeNulls = false override lazy val updateAggregate: cudf.Aggregate = cudf.Table.last(getOrdinal(ref), includeNulls) @@ -258,31 +258,31 @@ abstract class GpuDeclarativeAggregate extends GpuAggregateFunction with GpuUnev val initialValues: Seq[GpuExpression] // update: first half of the aggregation (count = count) - val updateExpressions: Seq[GpuExpression] + val updateExpressions: Seq[Expression] // merge: second half of the aggregation (count = sum). Also use to merge multiple batches. val mergeExpressions: Seq[GpuExpression] // mostly likely a pass through (count => sum we merged above). // average has a more interesting expression to compute the division of sum/count - val evaluateExpression: GpuExpression + val evaluateExpression: Expression /** An expression-based aggregate's bufferSchema is derived from bufferAttributes. */ final override def aggBufferSchema: StructType = null //not used in GPU version - final lazy val inputAggBufferAttributes: Seq[GpuAttributeReference] = + final lazy val inputAggBufferAttributes: Seq[AttributeReference] = aggBufferAttributes.map(_.newInstance()) } -case class GpuMin(child: GpuExpression) extends GpuDeclarativeAggregate { - private lazy val cudfMin = GpuAttributeReference("cudf_min", child.dataType)() +case class GpuMin(child: Expression) extends GpuDeclarativeAggregate { + private lazy val cudfMin = AttributeReference("cudf_min", child.dataType)() - override lazy val inputProjection: Seq[GpuExpression] = Seq(child) + override lazy val inputProjection: Seq[Expression] = Seq(child) override lazy val updateExpressions: Seq[GpuExpression] = Seq(new CudfMin(cudfMin)) override lazy val mergeExpressions: Seq[GpuExpression] = Seq(new CudfMin(cudfMin)) - override lazy val evaluateExpression: GpuExpression = cudfMin + override lazy val evaluateExpression: Expression = cudfMin - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfMin :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfMin :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, child.dataType)) @@ -294,15 +294,15 @@ case class GpuMin(child: GpuExpression) extends GpuDeclarativeAggregate { TypeUtils.checkForOrderingExpr(child.dataType, "function gpu min") } -case class GpuMax(child: GpuExpression) extends GpuDeclarativeAggregate { - private lazy val cudfMax = GpuAttributeReference("cudf_max", child.dataType)() +case class GpuMax(child: Expression) extends GpuDeclarativeAggregate { + private lazy val cudfMax = AttributeReference("cudf_max", child.dataType)() - override lazy val inputProjection: Seq[GpuExpression] = Seq(child) + override lazy val inputProjection: Seq[Expression] = Seq(child) override lazy val updateExpressions: Seq[GpuExpression] = Seq(new CudfMax(cudfMax)) override lazy val mergeExpressions: Seq[GpuExpression] = Seq(new CudfMax(cudfMax)) - override lazy val evaluateExpression: GpuExpression = cudfMax + override lazy val evaluateExpression: Expression = cudfMax - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfMax :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfMax :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, child.dataType)) @@ -314,21 +314,21 @@ case class GpuMax(child: GpuExpression) extends GpuDeclarativeAggregate { TypeUtils.checkForOrderingExpr(child.dataType, "function gpu max") } -case class GpuSum(child: GpuExpression) +case class GpuSum(child: Expression) extends GpuDeclarativeAggregate with ImplicitCastInputTypes { private lazy val resultType = child.dataType match { case _: DoubleType => DoubleType case _ => LongType } - private lazy val cudfSum = GpuAttributeReference("cudf_sum", resultType)() + private lazy val cudfSum = AttributeReference("cudf_sum", resultType)() - override lazy val inputProjection: Seq[GpuExpression] = Seq(child) + override lazy val inputProjection: Seq[Expression] = Seq(child) override lazy val updateExpressions: Seq[GpuExpression] = Seq(new CudfSum(cudfSum)) override lazy val mergeExpressions: Seq[GpuExpression] = Seq(new CudfSum(cudfSum)) - override lazy val evaluateExpression: GpuExpression = cudfSum + override lazy val evaluateExpression: Expression = cudfSum - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfSum :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfSum :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, resultType)) @@ -341,16 +341,16 @@ case class GpuSum(child: GpuExpression) TypeUtils.checkForNumericExpr(child.dataType, "function gpu sum") } -case class GpuCount(children: Seq[GpuExpression]) extends GpuDeclarativeAggregate { +case class GpuCount(children: Seq[Expression]) extends GpuDeclarativeAggregate { // counts are Long - private lazy val cudfCount = GpuAttributeReference("cudf_count", LongType)() + private lazy val cudfCount = AttributeReference("cudf_count", LongType)() - override lazy val inputProjection: Seq[GpuExpression] = Seq(children.head) + override lazy val inputProjection: Seq[Expression] = Seq(children.head) override lazy val updateExpressions: Seq[GpuExpression] = Seq(new CudfCount(cudfCount)) override lazy val mergeExpressions: Seq[GpuExpression] = Seq(new CudfSum(cudfCount)) - override lazy val evaluateExpression: GpuExpression = cudfCount + override lazy val evaluateExpression: Expression = cudfCount - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfCount :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfCount :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(0L, LongType)) @@ -359,10 +359,10 @@ case class GpuCount(children: Seq[GpuExpression]) extends GpuDeclarativeAggregat override def dataType: DataType = LongType } -case class GpuAverage(child: GpuExpression) extends GpuDeclarativeAggregate { +case class GpuAverage(child: Expression) extends GpuDeclarativeAggregate { // averages are either Decimal or Double. We don't support decimal yet, so making this double. - private lazy val cudfSum = GpuAttributeReference("cudf_sum", DoubleType)() - private lazy val cudfCount = GpuAttributeReference("cudf_count", LongType)() + private lazy val cudfSum = AttributeReference("cudf_sum", DoubleType)() + private lazy val cudfCount = AttributeReference("cudf_count", LongType)() private def toDoubleLit(v: Any): GpuLiteral = { val litVal = v match { @@ -407,7 +407,7 @@ case class GpuAverage(child: GpuExpression) extends GpuDeclarativeAggregate { GpuLiteral(0.0, DoubleType), GpuLiteral(0L, LongType)) - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfSum :: cudfCount :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfSum :: cudfCount :: Nil // Copied from Average override def prettyName: String = "gpuavg" @@ -429,12 +429,12 @@ case class GpuAverage(child: GpuExpression) extends GpuDeclarativeAggregate { * to check if the value was set (if we don't ignore nulls, valueSet is true, that's what we do * here). */ -case class GpuFirst(child: GpuExpression, ignoreNullsExpr: GpuExpression) +case class GpuFirst(child: Expression, ignoreNullsExpr: Expression) extends GpuDeclarativeAggregate with ImplicitCastInputTypes { - private lazy val cudfFirst = GpuAttributeReference("cudf_first", child.dataType)() - private lazy val valueSet = GpuAttributeReference("valueSet", BooleanType)() + private lazy val cudfFirst = AttributeReference("cudf_first", child.dataType)() + private lazy val valueSet = AttributeReference("valueSet", BooleanType)() - override lazy val inputProjection: Seq[GpuExpression] = + override lazy val inputProjection: Seq[Expression] = Seq(child, GpuLiteral(ignoreNulls, BooleanType)) private lazy val commonExpressions: Seq[CudfAggregate] = if (ignoreNulls) { @@ -445,9 +445,9 @@ case class GpuFirst(child: GpuExpression, ignoreNullsExpr: GpuExpression) override lazy val updateExpressions: Seq[GpuExpression] = commonExpressions override lazy val mergeExpressions: Seq[GpuExpression] = commonExpressions - override lazy val evaluateExpression: GpuExpression = cudfFirst + override lazy val evaluateExpression: Expression = cudfFirst - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfFirst :: valueSet :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfFirst :: valueSet :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq( GpuLiteral(null, child.dataType), @@ -481,12 +481,12 @@ case class GpuFirst(child: GpuExpression, ignoreNullsExpr: GpuExpression) override def toString: String = s"gpufirst($child)${if (ignoreNulls) " ignore nulls"}" } -case class GpuLast(child: GpuExpression, ignoreNullsExpr: GpuExpression) +case class GpuLast(child: Expression, ignoreNullsExpr: Expression) extends GpuDeclarativeAggregate with ImplicitCastInputTypes { - private lazy val cudfLast = GpuAttributeReference("cudf_last", child.dataType)() - private lazy val valueSet = GpuAttributeReference("valueSet", BooleanType)() + private lazy val cudfLast = AttributeReference("cudf_last", child.dataType)() + private lazy val valueSet = AttributeReference("valueSet", BooleanType)() - override lazy val inputProjection: Seq[GpuExpression] = + override lazy val inputProjection: Seq[Expression] = Seq(child, GpuLiteral(!ignoreNulls, BooleanType)) private lazy val commonExpressions: Seq[CudfAggregate] = if (ignoreNulls) { @@ -497,9 +497,9 @@ case class GpuLast(child: GpuExpression, ignoreNullsExpr: GpuExpression) override lazy val updateExpressions: Seq[GpuExpression] = commonExpressions override lazy val mergeExpressions: Seq[GpuExpression] = commonExpressions - override lazy val evaluateExpression: GpuExpression = cudfLast + override lazy val evaluateExpression: Expression = cudfLast - override lazy val aggBufferAttributes: Seq[GpuAttributeReference] = cudfLast :: valueSet :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = cudfLast :: valueSet :: Nil override lazy val initialValues: Seq[GpuLiteral] = Seq( GpuLiteral(null, child.dataType), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index c3ed8d4788c..20fabf3c116 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -194,7 +194,7 @@ class GpuDynamicPartitionDataWriter( private lazy val getPartitionColumns: ColumnarBatch => Table = { val expressions = GpuBindReferences.bindReferences( description.partitionColumns, - description.allColumns) + description.allColumns).asInstanceOf[Seq[GpuExpression]] cb => { val batch = GpuProjectExec.project(cb, expressions) try { @@ -209,7 +209,7 @@ class GpuDynamicPartitionDataWriter( private lazy val getOutputColumns: ColumnarBatch => Table = { val expressions = GpuBindReferences.bindReferences( description.dataColumns, - description.allColumns) + description.allColumns).asInstanceOf[Seq[GpuExpression]] cb => { val batch = GpuProjectExec.project(cb, expressions) try { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 9feb10eb257..89231c91028 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -33,7 +33,7 @@ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Expression, SortOrder} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.{WriteJobStatsTracker, WriteTaskResult, WriteTaskStats} @@ -46,7 +46,7 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} object GpuFileFormatWriter extends Logging { /** A function that converts the empty string to null for partition values. */ - case class GpuEmpty2Null(child: GpuExpression) extends GpuUnaryExpression { + case class GpuEmpty2Null(child: Expression) extends GpuUnaryExpression { override def nullable: Boolean = true override def doColumnar(input: GpuColumnVector): GpuColumnVector = { @@ -118,12 +118,11 @@ object GpuFileFormatWriter extends Logging { val dataColumns = outputSpec.outputColumns.filterNot(partitionSet.contains) var needConvert = false - val projectList: Seq[GpuExpression] = plan.output.map { + val projectList: Seq[Expression] = plan.output.map { case p if partitionSet.contains(p) && p.dataType == StringType && p.nullable => - val gpuAttr = GpuAttributeReference.from(p) needConvert = true - GpuAlias(GpuEmpty2Null(gpuAttr), p.name)() - case attr => GpuAttributeReference.from(attr) + GpuAlias(GpuEmpty2Null(p), p.name)() + case other => other } val empty2NullPlan = if (needConvert) GpuProjectExec(projectList, plan) else plan @@ -199,8 +198,7 @@ object GpuFileFormatWriter extends Logging { // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. val orderingExpr = GpuBindReferences.bindReferences( requiredOrdering - .map(attr => (GpuAttributeReference.from(attr), attr)) - .map(both => GpuSortOrder(both._1, both._2, Ascending)), outputSpec.outputColumns) + .map(attr => SortOrder(attr, Ascending)), outputSpec.outputColumns) GpuSortExec( orderingExpr, global = false, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala index 2e75290f23f..c601e9d0d4c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuUnaryExpression} -import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression} import org.apache.spark.sql.types.{AbstractDataType, DataType, DoubleType, FloatType, TypeCollection} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -26,7 +26,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch // - input NaNs become Float.NaN, or Double.NaN // - that -0.0f and -0.0d becomes 0.0f, and 0.0d respectively // TODO: need coalesce as a feature request in cudf -case class GpuNormalizeNaNAndZero(child: GpuExpression) extends GpuUnaryExpression +case class GpuNormalizeNaNAndZero(child: Expression) extends GpuUnaryExpression with ExpectsInputTypes { override def dataType: DataType = child.dataType diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index a235331c15f..3a42982fce1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf._ import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, NullIntolerant} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, NullIntolerant} import org.apache.spark.sql.types._ -case class GpuUnaryMinus(child: GpuExpression) extends GpuUnaryExpression +case class GpuUnaryMinus(child: Expression) extends GpuUnaryExpression with ExpectsInputTypes with NullIntolerant { override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection.NumericAndInterval) @@ -42,7 +42,7 @@ case class GpuUnaryMinus(child: GpuExpression) extends GpuUnaryExpression } } -case class GpuUnaryPositive(child: GpuExpression) extends GpuUnaryExpression +case class GpuUnaryPositive(child: Expression) extends GpuUnaryExpression with ExpectsInputTypes with NullIntolerant { override def prettyName: String = "positive" @@ -55,7 +55,7 @@ case class GpuUnaryPositive(child: GpuExpression) extends GpuUnaryExpression override def doColumnar(input: GpuColumnVector) : GpuColumnVector = input } -case class GpuAbs(child: GpuExpression) extends CudfUnaryExpression +case class GpuAbs(child: Expression) extends CudfUnaryExpression with ExpectsInputTypes with NullIntolerant { override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) @@ -70,7 +70,7 @@ abstract class CudfBinaryArithmetic extends CudfBinaryOperator with NullIntolera override lazy val resolved: Boolean = childrenResolved && checkInputDataTypes().isSuccess } -case class GpuAdd(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuAdd(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = TypeCollection.NumericAndInterval override def symbol: String = "+" @@ -78,7 +78,7 @@ case class GpuAdd(left: GpuExpression, right: GpuExpression) extends CudfBinaryA override def binaryOp: BinaryOp = BinaryOp.ADD } -case class GpuSubtract(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuSubtract(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = TypeCollection.NumericAndInterval override def symbol: String = "-" @@ -86,7 +86,7 @@ case class GpuSubtract(left: GpuExpression, right: GpuExpression) extends CudfBi override def binaryOp: BinaryOp = BinaryOp.SUB } -case class GpuMultiply(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuMultiply(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = NumericType override def symbol: String = "*" @@ -186,7 +186,7 @@ trait GpuDivModLike extends CudfBinaryArithmetic { } // This is for doubles and floats... -case class GpuDivide(left: GpuExpression, right: GpuExpression) extends GpuDivModLike { +case class GpuDivide(left: Expression, right: Expression) extends GpuDivModLike { override def inputType: AbstractDataType = TypeCollection(DoubleType, DecimalType) override def symbol: String = "/" @@ -194,7 +194,7 @@ case class GpuDivide(left: GpuExpression, right: GpuExpression) extends GpuDivMo override def binaryOp: BinaryOp = BinaryOp.TRUE_DIV } -case class GpuIntegralDivide(left: GpuExpression, right: GpuExpression) extends GpuDivModLike { +case class GpuIntegralDivide(left: Expression, right: Expression) extends GpuDivModLike { override def inputType: AbstractDataType = TypeCollection(IntegralType, DecimalType) override def dataType: DataType = LongType @@ -207,7 +207,7 @@ case class GpuIntegralDivide(left: GpuExpression, right: GpuExpression) extends override def sqlOperator: String = "div" } -case class GpuRemainder(left: GpuExpression, right: GpuExpression) extends GpuDivModLike { +case class GpuRemainder(left: Expression, right: Expression) extends GpuDivModLike { override def inputType: AbstractDataType = NumericType override def symbol: String = "%" @@ -216,7 +216,7 @@ case class GpuRemainder(left: GpuExpression, right: GpuExpression) extends GpuDi } -case class GpuPmod(left: GpuExpression, right: GpuExpression) extends GpuDivModLike { +case class GpuPmod(left: Expression, right: Expression) extends GpuDivModLike { override def inputType: AbstractDataType = NumericType override def binaryOp: BinaryOp = BinaryOp.PMOD diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/bitwise.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/bitwise.scala index 226da171752..4808e31f98d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/bitwise.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/bitwise.scala @@ -80,7 +80,7 @@ trait GpuShiftBase extends GpuBinaryExpression with ImplicitCastInputTypes { } } -case class GpuShiftLeft(left: GpuExpression, right: GpuExpression) extends GpuShiftBase { +case class GpuShiftLeft(left: Expression, right: Expression) extends GpuShiftBase { override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType), IntegerType) @@ -89,7 +89,7 @@ case class GpuShiftLeft(left: GpuExpression, right: GpuExpression) extends GpuSh override def dataType: DataType = left.dataType } -case class GpuShiftRight(left: GpuExpression, right: GpuExpression) extends GpuShiftBase { +case class GpuShiftRight(left: Expression, right: Expression) extends GpuShiftBase { override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType), IntegerType) @@ -99,7 +99,7 @@ case class GpuShiftRight(left: GpuExpression, right: GpuExpression) extends GpuS } -case class GpuShiftRightUnsigned(left: GpuExpression, right: GpuExpression) extends GpuShiftBase { +case class GpuShiftRightUnsigned(left: Expression, right: Expression) extends GpuShiftBase { override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType), IntegerType) @@ -108,7 +108,7 @@ case class GpuShiftRightUnsigned(left: GpuExpression, right: GpuExpression) exte override def dataType: DataType = left.dataType } -case class GpuBitwiseAnd(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuBitwiseAnd(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = IntegralType override def symbol: String = "&" @@ -116,7 +116,7 @@ case class GpuBitwiseAnd(left: GpuExpression, right: GpuExpression) extends Cudf override def binaryOp: BinaryOp = BinaryOp.BITWISE_AND } -case class GpuBitwiseOr(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuBitwiseOr(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = IntegralType override def symbol: String = "|" @@ -124,7 +124,7 @@ case class GpuBitwiseOr(left: GpuExpression, right: GpuExpression) extends CudfB override def binaryOp: BinaryOp = BinaryOp.BITWISE_OR } -case class GpuBitwiseXor(left: GpuExpression, right: GpuExpression) extends CudfBinaryArithmetic { +case class GpuBitwiseXor(left: Expression, right: Expression) extends CudfBinaryArithmetic { override def inputType: AbstractDataType = IntegralType override def symbol: String = "^" @@ -132,7 +132,7 @@ case class GpuBitwiseXor(left: GpuExpression, right: GpuExpression) extends Cudf override def binaryOp: BinaryOp = BinaryOp.BITWISE_XOR } -case class GpuBitwiseNot(child: GpuExpression) extends GpuUnaryExpression with ExpectsInputTypes { +case class GpuBitwiseNot(child: Expression) extends GpuUnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) override def dataType: DataType = child.dataType diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index ee3294fc668..0ecec54bbaa 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -21,14 +21,14 @@ import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuLiteral} import org.apache.spark.TaskContext import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, ExpressionWithRandomSeed, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionWithRandomSeed, UnaryExpression} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ -case class GpuRand(child: GpuExpression) extends UnaryExpression with GpuExpression +case class GpuRand(child: Expression) extends UnaryExpression with GpuExpression with ExpectsInputTypes with ExpressionWithRandomSeed { def this() = this(GpuLiteral(Utils.random.nextLong(), LongType)) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index ed2a592dbc7..1c18289289b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -47,7 +47,7 @@ trait GpuTimeUnaryExpression extends GpuUnaryExpression with TimeZoneAwareExpres override lazy val resolved: Boolean = childrenResolved && checkInputDataTypes().isSuccess } -case class GpuMinute(child: GpuExpression, timeZoneId: Option[String] = None) +case class GpuMinute(child: Expression, timeZoneId: Option[String] = None) extends GpuTimeUnaryExpression { override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = @@ -58,7 +58,7 @@ case class GpuMinute(child: GpuExpression, timeZoneId: Option[String] = None) } } -case class GpuSecond(child: GpuExpression, timeZoneId: Option[String] = None) +case class GpuSecond(child: Expression, timeZoneId: Option[String] = None) extends GpuTimeUnaryExpression { override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = @@ -69,7 +69,7 @@ case class GpuSecond(child: GpuExpression, timeZoneId: Option[String] = None) } } -case class GpuHour(child: GpuExpression, timeZoneId: Option[String] = None) +case class GpuHour(child: Expression, timeZoneId: Option[String] = None) extends GpuTimeUnaryExpression { override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = @@ -86,15 +86,15 @@ case class GpuYear(child: Expression) extends GpuDateUnaryExpression { } case class GpuTimeSub( - start: GpuExpression, - interval: GpuExpression, + start: Expression, + interval: Expression, timeZoneId: Option[String] = None) extends BinaryExpression with GpuExpression with TimeZoneAwareExpression with ExpectsInputTypes { - def this(start: GpuExpression, interval: GpuExpression) = this(start, interval, None) + def this(start: Expression, interval: Expression) = this(start, interval, None) - override def left: GpuExpression = start - override def right: GpuExpression = interval + override def left: Expression = start + override def right: Expression = interval override def toString: String = s"$left - $right" override def sql: String = s"${left.sql} - ${right.sql}" @@ -112,8 +112,8 @@ case class GpuTimeSub( var lhs: Any = null var rhs: Any = null try { - lhs = left.columnarEval(batch) - rhs = right.columnarEval(batch) + lhs = left.asInstanceOf[GpuExpression].columnarEval(batch) + rhs = right.asInstanceOf[GpuExpression].columnarEval(batch) (lhs, rhs) match { case (l: GpuColumnVector, intvl: CalendarInterval) => @@ -305,8 +305,8 @@ abstract class GpuToTimestampImproved extends GpuToTimestamp { } } -case class GpuUnixTimestamp(strTs: GpuExpression, - format: GpuExpression, +case class GpuUnixTimestamp(strTs: Expression, + format: Expression, strf: String, timeZoneId: Option[String] = None) extends GpuToTimestamp { override def strfFormat = strf @@ -314,13 +314,13 @@ case class GpuUnixTimestamp(strTs: GpuExpression, copy(timeZoneId = Option(timeZoneId)) } - override def left: GpuExpression = strTs - override def right: GpuExpression = format + override def left: Expression = strTs + override def right: Expression = format } -case class GpuToUnixTimestamp(strTs: GpuExpression, - format: GpuExpression, +case class GpuToUnixTimestamp(strTs: Expression, + format: Expression, strf: String, timeZoneId: Option[String] = None) extends GpuToTimestamp { override def strfFormat = strf @@ -328,13 +328,13 @@ case class GpuToUnixTimestamp(strTs: GpuExpression, copy(timeZoneId = Option(timeZoneId)) } - override def left: GpuExpression = strTs - override def right: GpuExpression = format + override def left: Expression = strTs + override def right: Expression = format } -case class GpuUnixTimestampImproved(strTs: GpuExpression, - format: GpuExpression, +case class GpuUnixTimestampImproved(strTs: Expression, + format: Expression, strf: String, timeZoneId: Option[String] = None) extends GpuToTimestampImproved { override def strfFormat = strf @@ -342,13 +342,13 @@ case class GpuUnixTimestampImproved(strTs: GpuExpression, copy(timeZoneId = Option(timeZoneId)) } - override def left: GpuExpression = strTs - override def right: GpuExpression = format + override def left: Expression = strTs + override def right: Expression = format } -case class GpuToUnixTimestampImproved(strTs: GpuExpression, - format: GpuExpression, +case class GpuToUnixTimestampImproved(strTs: Expression, + format: Expression, strf: String, timeZoneId: Option[String] = None) extends GpuToTimestampImproved { override def strfFormat = strf @@ -356,14 +356,14 @@ case class GpuToUnixTimestampImproved(strTs: GpuExpression, copy(timeZoneId = Option(timeZoneId)) } - override def left: GpuExpression = strTs - override def right: GpuExpression = format + override def left: Expression = strTs + override def right: Expression = format } case class GpuFromUnixTime( - sec: GpuExpression, - format: GpuExpression, + sec: Expression, + format: Expression, strfFormat: String, timeZoneId: Option[String] = None) extends GpuBinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { @@ -397,11 +397,11 @@ case class GpuFromUnixTime( override def inputTypes: Seq[AbstractDataType] = Seq(LongType, StringType) - override def left: GpuExpression = sec + override def left: Expression = sec // we aren't using this "right" GpuExpression, as it was already converted in the GpuOverrides // while creating the expressions map and passed down here as strfFormat - override def right: GpuExpression = format + override def right: Expression = format override def dataType: DataType = StringType @@ -443,21 +443,21 @@ trait GpuDateMathBase extends GpuBinaryExpression with ExpectsInputTypes { } } -case class GpuDateSub(startDate: GpuExpression, days: GpuExpression) +case class GpuDateSub(startDate: Expression, days: Expression) extends GpuDateMathBase { - override def left: GpuExpression = startDate - override def right: GpuExpression = days + override def left: Expression = startDate + override def right: Expression = days override def prettyName: String = "date_sub" override def binaryOp: BinaryOp = BinaryOp.SUB } -case class GpuDateAdd(startDate: GpuExpression, days: GpuExpression) extends GpuDateMathBase { +case class GpuDateAdd(startDate: Expression, days: Expression) extends GpuDateMathBase { - override def left: GpuExpression = startDate - override def right: GpuExpression = days + override def left: Expression = startDate + override def right: Expression = days override def prettyName: String = "date_add" diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index dc12bbec5fc..7be40e84809 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -21,6 +21,7 @@ import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} @@ -36,12 +37,14 @@ class GpuBroadcastHashJoinMeta( rule: ConfKeysAndIncompat) extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { - val leftKeys: Seq[ExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[ExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[ExprMeta[_]] = + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[ExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.condition) @@ -81,11 +84,11 @@ class GpuBroadcastHashJoinMeta( } case class GpuBroadcastHashJoinExec( - leftKeys: Seq[GpuExpression], - rightKeys: Seq[GpuExpression], + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, - condition: Option[GpuExpression], + condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/mathExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/mathExpressions.scala index 0b0b1118b93..ca8c1ba914c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/mathExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/mathExpressions.scala @@ -19,11 +19,10 @@ package org.apache.spark.sql.rapids import java.io.Serializable import ai.rapids.cudf.{BinaryOp, ColumnVector, DType, Scalar, UnaryOp} -import com.nvidia.spark.rapids.{Arm, CudfBinaryExpression, CudfUnaryExpression, FloatUtils, GpuColumnVector, GpuExpression, GpuUnaryExpression} +import com.nvidia.spark.rapids.{Arm, CudfBinaryExpression, CudfUnaryExpression, FloatUtils, GpuColumnVector, GpuUnaryExpression} import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes} import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.ColumnarBatch abstract class CudfUnaryMathExpression(name: String) extends GpuUnaryMathExpression(name) with CudfUnaryExpression @@ -38,12 +37,12 @@ abstract class GpuUnaryMathExpression(name: String) extends GpuUnaryExpression override def prettyName: String = name } -case class GpuAcos(child: GpuExpression) extends CudfUnaryMathExpression("ACOS") { +case class GpuAcos(child: Expression) extends CudfUnaryMathExpression("ACOS") { override def unaryOp: UnaryOp = UnaryOp.ARCCOS override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuToDegrees(child: GpuExpression) extends GpuUnaryMathExpression("DEGREES") { +case class GpuToDegrees(child: Expression) extends GpuUnaryMathExpression("DEGREES") { override def doColumnar(input: GpuColumnVector): GpuColumnVector = { withResource(Scalar.fromDouble(180d / Math.PI)) { multiplier => @@ -52,7 +51,7 @@ case class GpuToDegrees(child: GpuExpression) extends GpuUnaryMathExpression("DE } } -case class GpuToRadians(child: GpuExpression) extends GpuUnaryMathExpression("RADIANS") { +case class GpuToRadians(child: Expression) extends GpuUnaryMathExpression("RADIANS") { override def doColumnar(input: GpuColumnVector): GpuColumnVector = { withResource(Scalar.fromDouble(Math.PI / 180d)) { multiplier => @@ -61,11 +60,11 @@ case class GpuToRadians(child: GpuExpression) extends GpuUnaryMathExpression("RA } } -case class GpuAcoshImproved(child: GpuExpression) extends CudfUnaryMathExpression("ACOSH") { +case class GpuAcoshImproved(child: Expression) extends CudfUnaryMathExpression("ACOSH") { override def unaryOp: UnaryOp = UnaryOp.ARCCOSH } -case class GpuAcoshCompat(child: GpuExpression) extends GpuUnaryMathExpression("ACOSH") { +case class GpuAcoshCompat(child: Expression) extends GpuUnaryMathExpression("ACOSH") { override def outputTypeOverride: DType = DType.FLOAT64 override def doColumnar(input: GpuColumnVector): GpuColumnVector = { @@ -89,16 +88,16 @@ case class GpuAcoshCompat(child: GpuExpression) extends GpuUnaryMathExpression(" } } -case class GpuAsin(child: GpuExpression) extends CudfUnaryMathExpression("ASIN") { +case class GpuAsin(child: Expression) extends CudfUnaryMathExpression("ASIN") { override def unaryOp: UnaryOp = UnaryOp.ARCSIN override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuAsinhImproved(child: GpuExpression) extends CudfUnaryMathExpression("ASINH") { +case class GpuAsinhImproved(child: Expression) extends CudfUnaryMathExpression("ASINH") { override def unaryOp: UnaryOp = UnaryOp.ARCSINH } -case class GpuAsinhCompat(child: GpuExpression) extends GpuUnaryMathExpression("ASINH") { +case class GpuAsinhCompat(child: Expression) extends GpuUnaryMathExpression("ASINH") { override def outputTypeOverride: DType = DType.FLOAT64 def computeBasic(input: ColumnVector): ColumnVector = @@ -133,17 +132,17 @@ case class GpuAsinhCompat(child: GpuExpression) extends GpuUnaryMathExpression(" } } -case class GpuAtan(child: GpuExpression) extends CudfUnaryMathExpression("ATAN") { +case class GpuAtan(child: Expression) extends CudfUnaryMathExpression("ATAN") { override def unaryOp: UnaryOp = UnaryOp.ARCTAN override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuAtanh(child: GpuExpression) extends CudfUnaryMathExpression("ATANH") { +case class GpuAtanh(child: Expression) extends CudfUnaryMathExpression("ATANH") { override def unaryOp: UnaryOp = UnaryOp.ARCTANH override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuCeil(child: GpuExpression) extends CudfUnaryMathExpression("CEIL") { +case class GpuCeil(child: Expression) extends CudfUnaryMathExpression("CEIL") { override def dataType: DataType = child.dataType match { case dt @ DecimalType.Fixed(_, 0) => dt case DecimalType.Fixed(precision, scale) => @@ -169,17 +168,17 @@ case class GpuCeil(child: GpuExpression) extends CudfUnaryMathExpression("CEIL") } } -case class GpuCos(child: GpuExpression) extends CudfUnaryMathExpression("COS") { +case class GpuCos(child: Expression) extends CudfUnaryMathExpression("COS") { override def unaryOp: UnaryOp = UnaryOp.COS override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuExp(child: GpuExpression) extends CudfUnaryMathExpression("EXP") { +case class GpuExp(child: Expression) extends CudfUnaryMathExpression("EXP") { override def unaryOp: UnaryOp = UnaryOp.EXP override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuExpm1(child: GpuExpression) extends CudfUnaryMathExpression("EXPM1") { +case class GpuExpm1(child: Expression) extends CudfUnaryMathExpression("EXPM1") { override def unaryOp: UnaryOp = UnaryOp.EXP override def outputTypeOverride: DType = DType.FLOAT64 @@ -198,7 +197,7 @@ case class GpuExpm1(child: GpuExpression) extends CudfUnaryMathExpression("EXPM1 } } -case class GpuFloor(child: GpuExpression) extends CudfUnaryMathExpression("FLOOR") { +case class GpuFloor(child: Expression) extends CudfUnaryMathExpression("FLOOR") { override def dataType: DataType = child.dataType match { case dt @ DecimalType.Fixed(_, 0) => dt case DecimalType.Fixed(precision, scale) => @@ -225,7 +224,7 @@ case class GpuFloor(child: GpuExpression) extends CudfUnaryMathExpression("FLOOR } } -case class GpuLog(child: GpuExpression) extends CudfUnaryMathExpression("LOG") { +case class GpuLog(child: Expression) extends CudfUnaryMathExpression("LOG") { override def unaryOp: UnaryOp = UnaryOp.LOG override def outputTypeOverride: DType = DType.FLOAT64 override def doColumnar(input: GpuColumnVector): GpuColumnVector = { @@ -264,7 +263,7 @@ object GpuLogarithm extends Arm { } } -case class GpuLogarithm(left: GpuExpression, right: GpuExpression) +case class GpuLogarithm(left: Expression, right: Expression) extends CudfBinaryMathExpression("LOG_BASE") { override def binaryOp: BinaryOp = BinaryOp.LOG_BASE @@ -289,12 +288,12 @@ case class GpuLogarithm(left: GpuExpression, right: GpuExpression) } } -case class GpuSin(child: GpuExpression) extends CudfUnaryMathExpression("SIN") { +case class GpuSin(child: Expression) extends CudfUnaryMathExpression("SIN") { override def unaryOp: UnaryOp = UnaryOp.SIN override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuSignum(child: GpuExpression) extends GpuUnaryMathExpression("SIGNUM") { +case class GpuSignum(child: Expression) extends GpuUnaryMathExpression("SIGNUM") { override def doColumnar(input: GpuColumnVector): GpuColumnVector = { val num = Scalar.fromDouble(0) @@ -318,37 +317,37 @@ case class GpuSignum(child: GpuExpression) extends GpuUnaryMathExpression("SIGNU override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuTanh(child: GpuExpression) extends CudfUnaryMathExpression("TANH") { +case class GpuTanh(child: Expression) extends CudfUnaryMathExpression("TANH") { override def unaryOp: UnaryOp = UnaryOp.TANH override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuCosh(child: GpuExpression) extends CudfUnaryMathExpression("COSH") { +case class GpuCosh(child: Expression) extends CudfUnaryMathExpression("COSH") { override def unaryOp: UnaryOp = UnaryOp.COSH override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuSinh(child: GpuExpression) extends CudfUnaryMathExpression("SINH") { +case class GpuSinh(child: Expression) extends CudfUnaryMathExpression("SINH") { override def unaryOp: UnaryOp = UnaryOp.SINH override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuSqrt(child: GpuExpression) extends CudfUnaryMathExpression("SQRT") { +case class GpuSqrt(child: Expression) extends CudfUnaryMathExpression("SQRT") { override def unaryOp: UnaryOp = UnaryOp.SQRT override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuCbrt(child: GpuExpression) extends CudfUnaryMathExpression("CBRT") { +case class GpuCbrt(child: Expression) extends CudfUnaryMathExpression("CBRT") { override def unaryOp: UnaryOp = UnaryOp.CBRT override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuTan(child: GpuExpression) extends CudfUnaryMathExpression("TAN") { +case class GpuTan(child: Expression) extends CudfUnaryMathExpression("TAN") { override def unaryOp: UnaryOp = UnaryOp.TAN override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuCot(child: GpuExpression) extends GpuUnaryMathExpression("COT") { +case class GpuCot(child: Expression) extends GpuUnaryMathExpression("COT") { override def doColumnar(input: GpuColumnVector): GpuColumnVector = { withResource(Scalar.fromInt(1)) { one => @@ -367,13 +366,13 @@ abstract class CudfBinaryMathExpression(name: String) extends CudfBinaryExpressi override def dataType: DataType = DoubleType } -case class GpuPow(left: GpuExpression, right: GpuExpression) +case class GpuPow(left: Expression, right: Expression) extends CudfBinaryMathExpression("POWER") { override def binaryOp: BinaryOp = BinaryOp.POW override def outputTypeOverride: DType = DType.FLOAT64 } -case class GpuRint(child: GpuExpression) extends CudfUnaryMathExpression("ROUND") { +case class GpuRint(child: Expression) extends CudfUnaryMathExpression("ROUND") { override def unaryOp: UnaryOp = UnaryOp.RINT override def outputTypeOverride: DType = DType.FLOAT64 } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 82f2f1d732f..4ec42a7643c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -158,7 +158,7 @@ case class GpuStringLocate(substr: Expression, col: Expression, start: Expressio throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") } -case class GpuStartsWith(left: GpuExpression, right: GpuExpression) +case class GpuStartsWith(left: Expression, right: Expression) extends GpuBinaryExpression with Predicate with ImplicitCastInputTypes with NullIntolerant { override def inputTypes: Seq[DataType] = Seq(StringType) @@ -193,7 +193,7 @@ case class GpuStartsWith(left: GpuExpression, right: GpuExpression) "Really should not be here, cannot have a scalar as left side operand in StartsWith") } -case class GpuEndsWith(left: GpuExpression, right: GpuExpression) +case class GpuEndsWith(left: Expression, right: Expression) extends GpuBinaryExpression with Predicate with ImplicitCastInputTypes with NullIntolerant { override def inputTypes: Seq[DataType] = Seq(StringType) @@ -228,17 +228,17 @@ case class GpuEndsWith(left: GpuExpression, right: GpuExpression) "Really should not be here, cannot have a scalar as left side operand in EndsWith") } -case class GpuStringTrim(column: GpuExpression, trimParameters: Option[GpuExpression] = None) +case class GpuStringTrim(column: Expression, trimParameters: Option[Expression] = None) extends GpuString2TrimExpression with ImplicitCastInputTypes { - override def srcStr: GpuExpression = column + override def srcStr: Expression = column - override def trimStr:Option[GpuExpression] = trimParameters + override def trimStr:Option[Expression] = trimParameters - def this(trimParameters: GpuExpression, column: GpuExpression) = + def this(trimParameters: Expression, column: Expression) = this(column, Option(trimParameters)) - def this(column: GpuExpression) = this(column, None) + def this(column: Expression) = this(column, None) override protected def direction: String = "BOTH" @@ -247,17 +247,17 @@ case class GpuStringTrim(column: GpuExpression, trimParameters: Option[GpuExpres } } -case class GpuStringTrimLeft(column: GpuExpression, trimParameters: Option[GpuExpression] = None) +case class GpuStringTrimLeft(column: Expression, trimParameters: Option[Expression] = None) extends GpuString2TrimExpression with ImplicitCastInputTypes { - override def srcStr: GpuExpression = column + override def srcStr: Expression = column - override def trimStr:Option[GpuExpression] = trimParameters + override def trimStr:Option[Expression] = trimParameters - def this(trimParameters: GpuExpression, column: GpuExpression) = + def this(trimParameters: Expression, column: Expression) = this(column, Option(trimParameters)) - def this(column: GpuExpression) = this(column, None) + def this(column: Expression) = this(column, None) override protected def direction: String = "LEADING" @@ -266,17 +266,17 @@ case class GpuStringTrimLeft(column: GpuExpression, trimParameters: Option[GpuEx } } -case class GpuStringTrimRight(column: GpuExpression, trimParameters: Option[GpuExpression] = None) +case class GpuStringTrimRight(column: Expression, trimParameters: Option[Expression] = None) extends GpuString2TrimExpression with ImplicitCastInputTypes { - override def srcStr: GpuExpression = column + override def srcStr: Expression = column - override def trimStr:Option[GpuExpression] = trimParameters + override def trimStr:Option[Expression] = trimParameters - def this(trimParameters: GpuExpression, column: GpuExpression) = + def this(trimParameters: Expression, column: Expression) = this(column, Option(trimParameters)) - def this(column: GpuExpression) = this(column, None) + def this(column: Expression) = this(column, None) override protected def direction: String = "TRAILING" @@ -285,7 +285,7 @@ case class GpuStringTrimRight(column: GpuExpression, trimParameters: Option[GpuE } } -case class GpuConcat(children: Seq[GpuExpression]) extends GpuComplexTypeMergingExpression { +case class GpuConcat(children: Seq[Expression]) extends GpuComplexTypeMergingExpression { override def dataType = StringType override def nullable: Boolean = children.exists(_.nullable) @@ -297,7 +297,7 @@ case class GpuConcat(children: Seq[GpuExpression]) extends GpuComplexTypeMerging val columns: ArrayBuffer[ColumnVector] = new ArrayBuffer[ColumnVector]() try { nullStrScalar = GpuScalar.from(null, StringType) - children.foreach(childEvals += _.columnarEval(batch)) + children.foreach(childEvals += _.asInstanceOf[GpuExpression].columnarEval(batch)) childEvals.foreach { case vector: GpuColumnVector => columns += vector.getBase @@ -327,14 +327,14 @@ case class GpuConcat(children: Seq[GpuExpression]) extends GpuComplexTypeMerging } } -case class GpuContains(left: GpuExpression, right: GpuExpression) extends GpuBinaryExpression +case class GpuContains(left: Expression, right: Expression) extends GpuBinaryExpression with Predicate with ImplicitCastInputTypes with NullIntolerant { override def inputTypes: Seq[DataType] = Seq(StringType) override def sql: String = { val inputSQL = left.sql - val listSQL = right.sql.toString + val listSQL = right.sql s"($inputSQL CONTAINS ($listSQL))" } @@ -432,7 +432,7 @@ case class GpuSubstring(str: Expression, pos: Expression, len: Expression) throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") } -case class GpuInitCap(child: GpuExpression) extends GpuUnaryExpression with ImplicitCastInputTypes { +case class GpuInitCap(child: Expression) extends GpuUnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[DataType] = Seq(StringType) override def dataType: DataType = StringType override protected def doColumnar(input: GpuColumnVector): GpuColumnVector = { @@ -441,9 +441,9 @@ case class GpuInitCap(child: GpuExpression) extends GpuUnaryExpression with Impl } case class GpuStringReplace( - srcExpr: GpuExpression, - searchExpr: GpuExpression, - replaceExpr: GpuExpression) + srcExpr: Expression, + searchExpr: Expression, + replaceExpr: Expression) extends GpuTernaryExpression with ImplicitCastInputTypes { override def dataType: DataType = srcExpr.dataType @@ -452,7 +452,7 @@ case class GpuStringReplace( override def children: Seq[Expression] = Seq(srcExpr, searchExpr, replaceExpr) - def this(srcExpr: GpuExpression, searchExpr: GpuExpression) = { + def this(srcExpr: Expression, searchExpr: Expression) = { this(srcExpr, searchExpr, GpuLiteral("", StringType)) } @@ -528,10 +528,10 @@ object CudfRegexp { } } -case class GpuLike(left: GpuExpression, right: GpuExpression, escapeChar: Char) +case class GpuLike(left: Expression, right: Expression, escapeChar: Char) extends GpuBinaryExpression with ImplicitCastInputTypes with NullIntolerant { - def this(left: GpuExpression, right: GpuExpression) = this(left, right, '\\') + def this(left: Expression, right: Expression) = this(left, right, '\\') override def toString: String = escapeChar match { case '\\' => s"$left gpulike $right" @@ -622,9 +622,9 @@ class SubstringIndexMeta( } override def convertToGpu( - column: GpuExpression, - delim: GpuExpression, - count: GpuExpression): GpuExpression = GpuSubstringIndex(column, this.regexp, delim, count) + column: Expression, + delim: Expression, + count: Expression): GpuExpression = GpuSubstringIndex(column, this.regexp, delim, count) } object GpuSubstringIndex { @@ -661,10 +661,10 @@ object GpuSubstringIndex { } } -case class GpuSubstringIndex(strExpr: GpuExpression, +case class GpuSubstringIndex(strExpr: Expression, regexp: String, - ignoredDelimExpr: GpuExpression, - ignoredCountExpr: GpuExpression) + ignoredDelimExpr: Expression, + ignoredCountExpr: Expression) extends GpuTernaryExpression with ImplicitCastInputTypes { override def dataType: DataType = StringType From 18d74968ed3a4fa76bc6d0b1ace508b313f36142 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 22 Jun 2020 09:58:19 -0500 Subject: [PATCH 2/4] Simple Review Comments --- docs/dev/README.md | 6 +++--- .../scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala | 4 ++-- .../main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala | 4 ++-- .../main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 5 +---- .../com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala | 2 +- .../com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala | 2 +- .../src/main/scala/com/nvidia/spark/rapids/SortUtils.scala | 2 +- .../com/nvidia/spark/rapids/basicPhysicalOperators.scala | 2 +- .../com/nvidia/spark/rapids/constraintExpressions.scala | 2 +- .../scala/com/nvidia/spark/rapids/nullExpressions.scala | 2 +- .../org/apache/spark/sql/rapids/AggregateFunctions.scala | 2 +- .../org/apache/spark/sql/rapids/GpuFileFormatWriter.scala | 2 +- .../apache/spark/sql/rapids/NormalizeFloatingNumbers.scala | 2 +- .../scala/org/apache/spark/sql/rapids/stringFunctions.scala | 2 +- 14 files changed, 18 insertions(+), 21 deletions(-) diff --git a/docs/dev/README.md b/docs/dev/README.md index 2e78e25296e..6671f37715a 100644 --- a/docs/dev/README.md +++ b/docs/dev/README.md @@ -178,9 +178,9 @@ producing GPU columnar data as output, the child node(s) passed to the case class constructor should have the `Expression` type. This is a little odd because they should all be instances of `GpuExpression` except for `AttributeReference` and `SortOrder`. This is needed because `AttributeReference` -is weaved into a lot of the magic that is build into Spark expressions. -`SortOrder` is similar as spark itself will insert `SortOrder` instances into -the plan automatically in many cases. These are both `Unevalable` expressions +is weaved into a lot of the magic that is built into Spark expressions. +`SortOrder` is similar as Spark itself will insert `SortOrder` instances into +the plan automatically in many cases. These are both `Unevaluable` expressions so they should never be run columnar or otherwise. These `Expressions` should be bound using `GpuBindReferences` which will make sure that all `AttributeReference` instances are replaced with `GpuBoundReference` implementations and everything is diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala index 15fb7dbd75a..a4f2bc9e9e9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,7 +26,7 @@ object GpuBindReferences extends Logging { private[this] def postBindCheck[A <: Expression](base: A): Unit = { base.foreach { expr => // The condition is needed to have it match what transform - // looks at, otherwise we can check things that would nto be modified + // looks at, otherwise we can check things that would not be modified. if (expr.containsChild.nonEmpty) { expr match { case _: GpuExpression => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index becc7f47001..af10f5ef53c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -50,7 +50,7 @@ class GpuExpandExecMeta( */ override def convertToGpu(): GpuExec = { val projections = gpuProjections.map(_.map(_.convertToGpu())) - GpuExpandExec(projections, expand.output.asInstanceOf[Seq[AttributeReference]], + GpuExpandExec(projections, expand.output, childPlans.head.convertIfNeeded()) } } @@ -65,7 +65,7 @@ class GpuExpandExecMeta( */ case class GpuExpandExec( projections: Seq[Seq[Expression]], - output: Seq[AttributeReference], + output: Seq[Attribute], child: SparkPlan) extends UnaryExecNode with GpuExec { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index d633cdc3945..bb546d36c55 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1144,11 +1144,8 @@ object GpuOverrides { }), expr[SortOrder]( "sort order", -// (a, conf, p, r) => new UnaryExprMeta[SortOrder](a, conf, p, r) { -// override def convertToGpu(child: Expression): GpuExpression = -// GpuSortOrder(child, a.direction, a.nullOrdering, a.sameOrderExpressions, a.child) -// }), (a, conf, p, r) => new BaseExprMeta[SortOrder](a, conf, p, r) { + // One of the few expressions that are not replaced with a GPU version override def convertToGpu(): Expression = a.withNewChildren(childExprs.map(_.convertToGpu())) }), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala index 1ec4488f851..5be8d1aae8d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala index 94d827cdf94..3b2b4473d42 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala index 5b05cd75385..84b1f946633 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SortUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index 061e467b7f9..6c3fcd514e9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala index bbeadac9823..b0cf233546a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala index b8069f653d4..514906666a6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 1ae5d44b372..422bd450e50 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 89231c91028..66121c13f5c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala index c601e9d0d4c..363a6a8add5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/NormalizeFloatingNumbers.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 4ec42a7643c..eb7b28ee923 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From 60baaffc9bf719a305bc25bd39bfd9e04127b7b5 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 22 Jun 2020 10:46:49 -0500 Subject: [PATCH 3/4] Added in implicit conversion --- .../com/nvidia/spark/rapids/GpuExpandExec.scala | 2 +- .../com/nvidia/spark/rapids/GpuExpressions.scala | 15 +++++++-------- .../nvidia/spark/rapids/GpuHashPartitioning.scala | 2 +- .../scala/com/nvidia/spark/rapids/aggregate.scala | 7 +++---- .../spark/rapids/basicPhysicalOperators.scala | 5 ++--- .../spark/rapids/conditionalExpressions.scala | 9 +++++---- .../spark/rapids/constraintExpressions.scala | 4 +++- .../scala/com/nvidia/spark/rapids/implicits.scala | 7 +++++++ .../nvidia/spark/rapids/namedExpressions.scala | 4 +++- .../com/nvidia/spark/rapids/nullExpressions.scala | 5 ++--- .../spark/sql/rapids/datetimeExpressions.scala | 7 ++++--- .../apache/spark/sql/rapids/stringFunctions.scala | 2 +- 12 files changed, 39 insertions(+), 30 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index af10f5ef53c..7b48d31fd6c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -156,7 +156,7 @@ class GpuExpandIterator( val projectedColumns = boundProjections(projectionIndex).safeMap(fn = expr => { val rapidsType = GpuColumnVector.getRapidsType(expr.dataType) - val (cv, nullColumnReused) = expr.asInstanceOf[GpuExpression].columnarEval(cb) match { + val (cv, nullColumnReused) = expr.columnarEval(cb) match { case null => getOrCreateNullCV(rapidsType) case lit: GpuLiteral if lit.value == null => getOrCreateNullCV(rapidsType) case lit: GpuLiteral => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index 288373c0dbe..188adb74c94 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -22,7 +22,6 @@ import ai.rapids.cudf.{BinaryOp, BinaryOperable, DType, Scalar, UnaryOp} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, BinaryOperator, ComplexTypeMergingExpression, Expression, String2TrimExpression, TernaryExpression, UnaryExpression, Unevaluable} -import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types.StringType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -119,7 +118,7 @@ abstract class GpuUnaryExpression extends UnaryExpression with GpuExpression { def outputTypeOverride: DType = null override def columnarEval(batch: ColumnarBatch): Any = { - val input = child.asInstanceOf[GpuExpression].columnarEval(batch) + val input = child.columnarEval(batch) try { input match { case vec: GpuColumnVector => @@ -166,8 +165,8 @@ trait GpuBinaryExpression extends BinaryExpression with GpuExpression { var lhs: Any = null var rhs: Any = null try { - lhs = left.asInstanceOf[GpuExpression].columnarEval(batch) - rhs = right.asInstanceOf[GpuExpression].columnarEval(batch) + lhs = left.columnarEval(batch) + rhs = right.columnarEval(batch) (lhs, rhs) match { case (l: GpuColumnVector, r: GpuColumnVector) => doColumnar(l, r) @@ -255,7 +254,7 @@ trait GpuString2TrimExpression extends String2TrimExpression with GpuExpression override def columnarEval(batch: ColumnarBatch): Any = { val trim = GpuExpressionsUtils.getTrimString(trimStr) - val shouldBeColumn = srcStr.asInstanceOf[GpuExpression].columnarEval(batch) + val shouldBeColumn = srcStr.columnarEval(batch) try { // We know the first parameter is not a Literal, because trim(Literal, Literal) would already // have been optimized out @@ -295,9 +294,9 @@ trait GpuTernaryExpression extends TernaryExpression with GpuExpression { var val1: Any = null var val2: Any = null try { - val0 = children(0).asInstanceOf[GpuExpression].columnarEval(batch) - val1 = children(1).asInstanceOf[GpuExpression].columnarEval(batch) - val2 = children(2).asInstanceOf[GpuExpression].columnarEval(batch) + val0 = children(0).columnarEval(batch) + val1 = children(1).columnarEval(batch) + val2 = children(2).columnarEval(batch) (val0, val1, val2) match { case (v0: GpuColumnVector, v1: GpuColumnVector, v2: GpuColumnVector) => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala index 2e1f62a751f..f2398f7f436 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioning.scala @@ -48,7 +48,7 @@ case class GpuHashPartitioning(expressions: Seq[Expression], numPartitions: Int) } def getGpuKeyColumns(batch: ColumnarBatch) : Array[GpuColumnVector] = { - expressions.map(_.asInstanceOf[GpuExpression].columnarEval(batch) + expressions.map(_.columnarEval(batch) .asInstanceOf[GpuColumnVector]).toArray } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index a683ebe5e5a..46557d4acbd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -430,8 +430,7 @@ case class GpuHashAggregateExec( // aggregatedCb is made up of ColumnVectors // and the final projections from the aggregates won't change that, // so we can assume they will be vectors after we eval - ref.asInstanceOf[GpuExpression] - .columnarEval(aggregatedCb).asInstanceOf[GpuColumnVector] + ref.columnarEval(aggregatedCb).asInstanceOf[GpuColumnVector] } aggregatedCb.close() aggregatedCb = null @@ -449,7 +448,7 @@ case class GpuHashAggregateExec( // Perform the last project to get the correct shape that Spark expects. Note this will // add things like literals, that were not part of the aggregate into the batch. resultCvs = boundExpression.boundResultReferences.map { ref => - val result = ref.asInstanceOf[GpuExpression].columnarEval(finalCb) + val result = ref.columnarEval(finalCb) // Result references can be virtually anything, we need to coerce // them to be vectors since this is going into a ColumnarBatch result match { @@ -513,7 +512,7 @@ case class GpuHashAggregateExec( private def processIncomingBatch(batch: ColumnarBatch, boundInputReferences: Seq[Expression]): Seq[GpuColumnVector] = { boundInputReferences.safeMap { ref => - val in = ref.asInstanceOf[GpuExpression].columnarEval(batch) + val in = ref.columnarEval(batch) val childCv = in match { case cv: ColumnVector => cv.asInstanceOf[GpuColumnVector] case _ => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index 6c3fcd514e9..dd719adcf60 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -46,7 +46,7 @@ object GpuProjectExec { def project[A <: Expression](cb: ColumnarBatch, boundExprs: Seq[A]): ColumnarBatch = { val newColumns = boundExprs.safeMap { expr => { - val result = expr.asInstanceOf[GpuExpression].columnarEval(cb) + val result = expr.columnarEval(cb) result match { case cv: ColumnVector => cv case other => @@ -106,8 +106,7 @@ object GpuFilter { var tbl: cudf.Table = null var filtered: cudf.Table = null val filteredBatch = try { - filterConditionCv = boundCondition.asInstanceOf[GpuExpression] - .columnarEval(batch).asInstanceOf[GpuColumnVector] + filterConditionCv = boundCondition.columnarEval(batch).asInstanceOf[GpuColumnVector] tbl = GpuColumnVector.from(batch) filtered = tbl.filter(filterConditionCv.getBase) GpuColumnVector.from(filtered) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala index 8ea035b04a8..8c3ac88fa0c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/conditionalExpressions.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.Scalar +import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.{ComplexTypeMergingExpression, Expression} @@ -27,7 +28,7 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit private def computePredicate( batch: ColumnarBatch, predicateExpr: Expression): GpuColumnVector = { - val predicate: Any = predicateExpr.asInstanceOf[GpuExpression].columnarEval(batch) + val predicate: Any = predicateExpr.columnarEval(batch) try { if (!predicate.isInstanceOf[GpuColumnVector]) { throw new IllegalStateException("Predicate result is not a column") @@ -57,7 +58,7 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit falseValues: GpuColumnVector): GpuColumnVector = { val predicate = computePredicate(batch, predicateExpr) try { - val trueResult: Any = trueExpr.asInstanceOf[GpuExpression].columnarEval(batch) + val trueResult: Any = trueExpr.columnarEval(batch) try { val result = trueResult match { case t: GpuColumnVector => predicate.getBase.ifElse(t.getBase, falseValues.getBase) @@ -89,7 +90,7 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit falseValue: Scalar): GpuColumnVector = { val predicate = computePredicate(batch, predicateExpr) try { - val trueResult: Any = trueExpr.asInstanceOf[GpuExpression].columnarEval(batch) + val trueResult: Any = trueExpr.columnarEval(batch) try { val result = trueResult match { case t: GpuColumnVector => predicate.getBase.ifElse(t.getBase, falseValue) @@ -119,7 +120,7 @@ abstract class GpuConditionalExpression extends ComplexTypeMergingExpression wit predicateExpr: Expression, trueExpr: Expression, falseExpr: Expression): GpuColumnVector = { - val falseResult: Any = falseExpr.asInstanceOf[GpuExpression].columnarEval(batch) + val falseResult: Any = falseExpr.columnarEval(batch) try { falseResult match { case f: GpuColumnVector => computeIfElse(batch, predicateExpr, trueExpr, f) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala index b0cf233546a..eb8dd4bebd8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/constraintExpressions.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids +import com.nvidia.spark.rapids.RapidsPluginImplicits._ + import org.apache.spark.sql.catalyst.expressions.{Expression, TaggingExpression} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -26,6 +28,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuKnownFloatingPointNormalized(child: Expression) extends TaggingExpression with GpuExpression { override def columnarEval(batch: ColumnarBatch): Any = { - child.asInstanceOf[GpuExpression].columnarEval(batch) + child.columnarEval(batch) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/implicits.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/implicits.scala index fbd8fe0c57f..799b3c5be32 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/implicits.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/implicits.scala @@ -20,6 +20,7 @@ import scala.collection.{mutable, SeqLike} import scala.collection.generic.CanBuildFrom import scala.reflect.ClassTag +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -29,6 +30,12 @@ import org.apache.spark.sql.vectorized.ColumnarBatch object RapidsPluginImplicits { import scala.language.implicitConversions + implicit class ReallyAGpuExpression[A <: Expression](exp: Expression) { + def columnarEval(batch: ColumnarBatch): Any = { + exp.asInstanceOf[GpuExpression].columnarEval(batch) + } + } + implicit class AutoCloseableColumn[A <: AutoCloseable](autoCloseable: AutoCloseable) { /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala index 535b33bf819..1452674f98a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/namedExpressions.scala @@ -18,6 +18,8 @@ package com.nvidia.spark.rapids import java.util.Objects +import com.nvidia.spark.rapids.RapidsPluginImplicits._ + import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, ExprId, Generator, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark @@ -87,7 +89,7 @@ case class GpuAlias(child: Expression, name: String)( } override def columnarEval(batch: ColumnarBatch): Any = - child.asInstanceOf[GpuExpression].columnarEval(batch) + child.columnarEval(batch) override def doColumnar(input: GpuColumnVector): GpuColumnVector = throw new IllegalStateException("GpuAlias should never have doColumnar called") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala index 514906666a6..7ba2da5d9ec 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala @@ -54,7 +54,7 @@ case class GpuCoalesce(children: Seq[Expression]) extends GpuExpression with var runningScalar: Scalar = null try { children.reverse.foreach(expr => { - expr.asInstanceOf[GpuExpression].columnarEval(batch) match { + expr.columnarEval(batch) match { case data: GpuColumnVector => try { if (runningResult != null) { @@ -180,8 +180,7 @@ case class GpuAtLeastNNonNulls( var notNanVector: ColumnVector = null var nanAndNullVector: ColumnVector = null try { - cv = expr.asInstanceOf[GpuExpression] - .columnarEval(batch).asInstanceOf[GpuColumnVector].getBase + cv = expr.columnarEval(batch).asInstanceOf[GpuColumnVector].getBase notNullVector = cv.isNotNull if (cv.getType == DType.FLOAT32 || cv.getType == DType.FLOAT64) { notNanVector = cv.isNotNan diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 1c18289289b..84cb05dbc05 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -22,8 +22,9 @@ import ai.rapids.cudf.{BinaryOp, ColumnVector, DType, Scalar} import com.nvidia.spark.rapids.{BinaryExprMeta, ConfKeysAndIncompat, DateUtils, GpuBinaryExpression, GpuColumnVector, GpuExpression, GpuOverrides, GpuScalar, GpuUnaryExpression, RapidsConf, RapidsMeta} import com.nvidia.spark.rapids.DateUtils.TimestampFormatConversionException import com.nvidia.spark.rapids.GpuOverrides.extractStringLit +import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression, UnixTime} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.CalendarInterval @@ -112,8 +113,8 @@ case class GpuTimeSub( var lhs: Any = null var rhs: Any = null try { - lhs = left.asInstanceOf[GpuExpression].columnarEval(batch) - rhs = right.asInstanceOf[GpuExpression].columnarEval(batch) + lhs = left.columnarEval(batch) + rhs = right.columnarEval(batch) (lhs, rhs) match { case (l: GpuColumnVector, intvl: CalendarInterval) => diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index eb7b28ee923..4db5d92a851 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -297,7 +297,7 @@ case class GpuConcat(children: Seq[Expression]) extends GpuComplexTypeMergingExp val columns: ArrayBuffer[ColumnVector] = new ArrayBuffer[ColumnVector]() try { nullStrScalar = GpuScalar.from(null, StringType) - children.foreach(childEvals += _.asInstanceOf[GpuExpression].columnarEval(batch)) + children.foreach(childEvals += _.columnarEval(batch)) childEvals.foreach { case vector: GpuColumnVector => columns += vector.getBase From 8db31c013daac2a8c6b1cb44afc52ff99bfde4f4 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 22 Jun 2020 12:43:19 -0500 Subject: [PATCH 4/4] Cleanup bindings --- .../spark/rapids/GpuBoundAttribute.scala | 42 ++++++++++++------- .../nvidia/spark/rapids/GpuExpandExec.scala | 6 +-- .../nvidia/spark/rapids/GpuGenerateExec.scala | 6 +-- .../com/nvidia/spark/rapids/GpuHashJoin.scala | 6 +-- .../com/nvidia/spark/rapids/aggregate.scala | 36 ++++++++-------- .../spark/rapids/basicPhysicalOperators.scala | 2 +- .../sql/rapids/GpuFileFormatDataWriter.scala | 8 ++-- 7 files changed, 57 insertions(+), 49 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala index a4f2bc9e9e9..b696dd4313d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala @@ -40,38 +40,50 @@ object GpuBindReferences extends Logging { } // Mostly copied from BoundAttribute.scala so we can do columnar processing - def bindReference[A <: Expression]( + private[this] def bindRefInternal[A <: Expression, R <: Expression]( expression: A, - input: AttributeSeq, - allowFailures: Boolean = false): A = { + input: AttributeSeq): R = { val ret = expression.transform { case a: AttributeReference => val ordinal = input.indexOf(a.exprId) if (ordinal == -1) { - if (allowFailures) { - a - } else { - sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") - } + sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") } else { GpuBoundReference(ordinal, a.dataType, input(ordinal).nullable) } - }.asInstanceOf[A] - if (!allowFailures) { - postBindCheck(ret) - } + }.asInstanceOf[R] + postBindCheck(ret) ret } + def bindGpuReference[A <: Expression]( + expression: A, + input: AttributeSeq): GpuExpression = + bindRefInternal(expression, input) + /** * A helper function to bind given expressions to an input schema where the expressions are - * to be processed on the GPU. + * to be processed on the GPU, and the result type indicates this. + */ + def bindGpuReferences[A <: Expression]( + expressions: Seq[A], + input: AttributeSeq): Seq[GpuExpression] = + expressions.map(GpuBindReferences.bindGpuReference(_, input)) + + def bindReference[A <: Expression]( + expression: A, + input: AttributeSeq): A = + bindRefInternal(expression, input) + + /** + * A helper function to bind given expressions to an input schema where the expressions are + * to be processed on the GPU. Most of the time `bindGpuReferences` should be used, unless + * you know that the return type is `SortOrder` or is a comment trait like `Attribute`. */ def bindReferences[A <: Expression]( expressions: Seq[A], - input: AttributeSeq): Seq[A] = { + input: AttributeSeq): Seq[A] = expressions.map(GpuBindReferences.bindReference(_, input)) - } } case class GpuBoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index 7b48d31fd6c..a695c6c784f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -86,8 +86,8 @@ case class GpuExpandExec( AttributeSet(projections.flatten.flatMap(_.references)) override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val boundProjections: Seq[Seq[Expression]] = - projections.map(GpuBindReferences.bindReferences(_, child.output)) + val boundProjections: Seq[Seq[GpuExpression]] = + projections.map(GpuBindReferences.bindGpuReferences(_, child.output)) child.executeColumnar().mapPartitions { it => new GpuExpandIterator(boundProjections, metrics, it) } @@ -100,7 +100,7 @@ case class GpuExpandExec( } class GpuExpandIterator( - boundProjections: Seq[Seq[Expression]], + boundProjections: Seq[Seq[GpuExpression]], metrics: Map[String, SQLMetric], it: Iterator[ColumnarBatch]) extends Iterator[ColumnarBatch] diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala index 0baf257b098..2917dac191f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala @@ -119,11 +119,11 @@ case class GpuGenerateExec( val numOutputRows = longMetric(NUM_OUTPUT_ROWS) val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) val totalTime = longMetric(TOTAL_TIME) - val boundArrayProjectList = GpuBindReferences.bindReferences(arrayProject, child.output).toArray + val boundArrayProjectList = + GpuBindReferences.bindGpuReferences(arrayProject, child.output).toArray val numArrayColumns = boundArrayProjectList.length val boundOthersProjectList: Array[GpuExpression] = - GpuBindReferences.bindReferences(requiredChildOutput, child.output) - .asInstanceOf[Seq[GpuExpression]].toArray + GpuBindReferences.bindGpuReferences(requiredChildOutput, child.output).toArray val numOtherColumns = boundOthersProjectList.length val numExplodeColumns = if (includePos) 2 else 1 diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala index fea0aa676ea..e6290d1e280 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala @@ -42,10 +42,8 @@ trait GpuHashJoin extends GpuExec with HashJoin { protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindReferences(leftKeys.asInstanceOf[Seq[GpuExpression]], - left.output) - val rkeys = GpuBindReferences.bindReferences(rightKeys.asInstanceOf[Seq[GpuExpression]], - right.output) + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) buildSide match { case BuildLeft => (lkeys, rkeys) case BuildRight => (rkeys, lkeys) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 46557d4acbd..d36e29f167f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -69,7 +69,6 @@ class GpuHashAggregateMeta( willNotWorkOnGpu("First/Last reductions are not supported on GPU") } } - val groupingExpressionTypes = agg.groupingExpressions.map(_.dataType) if (agg.resultExpressions.isEmpty) { willNotWorkOnGpu("result expressions is empty") } @@ -234,8 +233,8 @@ case class GpuHashAggregateExec( resultExpressions: Seq[NamedExpression], child: SparkPlan) extends UnaryExecNode with GpuExec with Arm { - case class BoundExpressionsModeAggregates(boundInputReferences: Seq[Expression] , - boundFinalProjections: Option[scala.Seq[Expression]], + case class BoundExpressionsModeAggregates(boundInputReferences: Seq[GpuExpression] , + boundFinalProjections: Option[scala.Seq[GpuExpression]], boundResultReferences: scala.Seq[Expression] , aggModeCudfAggregates: scala.Seq[(AggregateMode, scala.Seq[CudfAggregate])]) // This handles GPU hash aggregation without spilling. @@ -503,7 +502,7 @@ case class GpuHashAggregateExec( } childCvs.safeClose() concatCvs.safeClose() - (Seq(batch, aggregatedInputCb, aggregatedCb, finalCb)) + Seq(batch, aggregatedInputCb, aggregatedCb, finalCb) .safeClose() } }} @@ -607,10 +606,10 @@ case class GpuHashAggregateExec( val aggModeCudfAggregates = aggregateExpressions.zipWithIndex.map { case (expr, modeIndex) => val cudfAggregates = if (expr.mode == Partial || expr.mode == Complete) { - GpuBindReferences.bindReferences(updateExpressionsSeq(modeIndex), aggBufferAttributes) + GpuBindReferences.bindGpuReferences(updateExpressionsSeq(modeIndex), aggBufferAttributes) .asInstanceOf[Seq[CudfAggregate]] } else { - GpuBindReferences.bindReferences(mergeExpressionsSeq(modeIndex), aggBufferAttributes) + GpuBindReferences.bindGpuReferences(mergeExpressionsSeq(modeIndex), aggBufferAttributes) .asInstanceOf[Seq[CudfAggregate]] } (expr.mode, cudfAggregates) @@ -686,16 +685,15 @@ case class GpuHashAggregateExec( // for Partial and non distinct merge expressions for PartialMerge. // - Final mode: we pick the columns in the order as handed to us. val boundInputReferences = if (uniqueModes.contains(PartialMerge)) { - GpuBindReferences.bindReferences(inputBindExpressions, resultingBindAttributes) + GpuBindReferences.bindGpuReferences(inputBindExpressions, resultingBindAttributes) } else if (finalMode) { - GpuBindReferences.bindReferences(childAttr.attrs.asInstanceOf[Seq[Expression]], childAttr) - .asInstanceOf[Seq[GpuExpression]] + GpuBindReferences.bindGpuReferences(childAttr.attrs.asInstanceOf[Seq[Expression]], childAttr) } else { - GpuBindReferences.bindReferences(inputProjections, childAttr) + GpuBindReferences.bindGpuReferences(inputProjections, childAttr) } val boundFinalProjections = if (finalMode || completeMode) { - Some(GpuBindReferences.bindReferences(finalProjections, aggBufferAttributes)) + Some(GpuBindReferences.bindGpuReferences(finalProjections, aggBufferAttributes)) } else { None } @@ -711,17 +709,17 @@ case class GpuHashAggregateExec( // - Final or Complete mode: we use resultExpressions to pick out the correct columns that // finalReferences has pre-processed for us val boundResultReferences = if (partialMode) { - GpuBindReferences.bindReferences( - resultExpressions.asInstanceOf[Seq[GpuExpression]], + GpuBindReferences.bindGpuReferences( + resultExpressions, resultExpressions.map(_.toAttribute)) } else if (finalMode || completeMode) { - GpuBindReferences.bindReferences( - resultExpressions.asInstanceOf[Seq[GpuExpression]], - finalAttributes.asInstanceOf[Seq[Attribute]]) + GpuBindReferences.bindGpuReferences( + resultExpressions, + finalAttributes) } else { - GpuBindReferences.bindReferences( - resultExpressions.asInstanceOf[Seq[GpuExpression]], - groupingAttributes.asInstanceOf[Seq[Attribute]]) + GpuBindReferences.bindGpuReferences( + resultExpressions, + groupingAttributes) } BoundExpressionsModeAggregates(boundInputReferences, boundFinalProjections, boundResultReferences, aggModeCudfAggregates) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index dd719adcf60..f7d3bf9e846 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -80,7 +80,7 @@ case class GpuProjectExec(projectList: Seq[Expression], child: SparkPlan) val numOutputRows = longMetric(NUM_OUTPUT_ROWS) val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) val totalTime = longMetric(TOTAL_TIME) - val boundProjectList = GpuBindReferences.bindReferences(projectList, child.output) + val boundProjectList = GpuBindReferences.bindGpuReferences(projectList, child.output) val rdd = child.executeColumnar() rdd.map { cb => numOutputBatches += 1 diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index 20fabf3c116..418c2ce2f13 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -192,9 +192,9 @@ class GpuDynamicPartitionDataWriter( /** Extracts the partition values out of an input batch. */ private lazy val getPartitionColumns: ColumnarBatch => Table = { - val expressions = GpuBindReferences.bindReferences( + val expressions = GpuBindReferences.bindGpuReferences( description.partitionColumns, - description.allColumns).asInstanceOf[Seq[GpuExpression]] + description.allColumns) cb => { val batch = GpuProjectExec.project(cb, expressions) try { @@ -207,9 +207,9 @@ class GpuDynamicPartitionDataWriter( /** Extracts the output values of an input batch. */ private lazy val getOutputColumns: ColumnarBatch => Table = { - val expressions = GpuBindReferences.bindReferences( + val expressions = GpuBindReferences.bindGpuReferences( description.dataColumns, - description.allColumns).asInstanceOf[Seq[GpuExpression]] + description.allColumns) cb => { val batch = GpuProjectExec.project(cb, expressions) try {