+ * Array is represented as `list column` in cudf, which has child column and offset column.
+ * When `legacyCastToString = true`, given an input with 3 rows: + * `[ [1, 2, null, 3], [], null]`
+ * Step 1: cast all not-null elements in array to string type: + * `[ ["1", "2", null, "3"], [], null]`
+ * Step 2: add space char in the front of all not-null elements: + * `[ [" 1", " 2", null, " 3"], [], null]`
+ * step 3: cast `null` elements to their string representation : + * `[ [" 1", " 2", "", " 3"], [], null]`(here we use "" to represent null)
+ * step 4: concatenate list elements, seperated by `","`: + * `[" 1, 2,, 3", null, null]`
+ * step 5: remove the first char, if it is an `' '`: + * `["1, 2,, 3", null, null]`
+ * step 6: replace nulls with empty string: + * `["1, 2,, 3", "", ""]`
+ * step 7: add brackets: + * `["[1, 2,, 3]", "[]", "[]"]`
+ * step 8: add `null` masks using original input: + * `["[1, 2,, 3]", "[]", null]`
+ * + * when `legacyCastToString = false`, step 2, 5 are skipped + */ + private def castArrayToString(input: ColumnView, + elementType: DataType, + ansiMode: Boolean, + legacyCastToString: Boolean, + stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + + val (leftStr, rightStr) = ("[", "]") + val emptyStr = "" + val spaceStr = " " + val nullStr = if (legacyCastToString) emptyStr else "null" + val sepStr = "," + (if (legacyCastToString) emptyStr else spaceStr) + val numRows = input.getRowCount.toInt + + withResource( + Seq(leftStr, rightStr, emptyStr, spaceStr, nullStr, sepStr).safeMap(Scalar.fromString) + ){ case Seq(left, right, empty, space, nullRep, sep) => + + /* -------------------------------- helper functions -----------------------*/ + + /** + * cast all not-null elements in a child column to string type
+ * add `' '` to all elements when `legacyCastToString = true` + * @param child child column of an array column + * @return a string type child column + */ + def castChildToStr(child: ColumnView): ColumnView = { + withResource( + doCast(child, elementType, StringType, ansiMode, + legacyCastToString, stringToDateAnsiModeEnabled) + ) { strChildWithNull => + withResource(strChildWithNull.replaceNulls(nullRep)) { strChild => + if (legacyCastToString) {// add a space string to each non-null element + withResource(ColumnVector.fromScalar(space, child.getRowCount.toInt)) { spaceVec => + withResource( + ColumnVector.stringConcatenate(Array(spaceVec, strChild)) + ) { addSpace => + withResource(child.isNotNull) {_.ifElse(addSpace, strChild)} + } + } + } + else { strChild.incRefCount } + } + } + } + + /** + * If the first char of a string is ' ', remove it (only for legacyCastToString = true) + * @param strVec a string type column vector + */ + def removeFirstSpace(strVec: ColumnVector): ColumnVector = { + if (legacyCastToString){ + withResource(strVec.substring(0,1)) { fstChar => + withResource(strVec.substring(1)) { remain => + withResource(fstChar.equalTo(space)) {_.ifElse(remain, strVec)} + } + } + } + else {strVec.incRefCount} + } + + /** + * Add brackets to each string. Ex: ["1, 2, 3", "4, 5"] => ["[1, 2, 3]", "[4, 5]"] + * @param strVec a string vector + */ + def addBrackets(strVec: ColumnVector): ColumnVector = { + withResource( + Seq(left, right).safeMap(s => ColumnVector.fromScalar(s, numRows)) + ) { case Seq(leftColumn, rightColumn) => + ColumnVector.stringConcatenate(empty, nullRep, Array(leftColumn, strVec, rightColumn)) + } + } + /* -------------------------------- helper functions -----------------------*/ + + + // cast child column to string type + withResource(input.getChildColumnView(0)) { childView => + withResource(castChildToStr(childView)){ stringChild => + withResource(input.replaceListChild(stringChild)) {strArr => + // concatenate each row. cast from list column to string column + withResource(ColumnVector.fromScalar(sep, numRows)){ sepCol => + withResource( + strArr.stringConcatenateListElements(sepCol) + ) { strColContainsNull => + withResource(strColContainsNull.replaceNulls(empty)){strCol => + withResource(removeFirstSpace(strCol)){withoutBrackets => + withResource(addBrackets(withoutBrackets))( + _.mergeAndSetValidity(BinaryOp.BITWISE_AND, input) + ) + } + } + } + } + } + } + } + } + } + + + private def castStructToString( input: ColumnView, inputSchema: Array[StructField], @@ -831,13 +952,11 @@ object GpuCast extends Arm { input: ColumnVector, ansiEnabled: Boolean, dType: DType): ColumnVector = { - // 1. convert the different infinities to "Inf"/"-Inf" which is the only variation cudf - // understands - // 2. identify the nans - // 3. identify the floats. "nan", "null" and letters are not considered floats - // 4. if ansi is enabled we want to throw an exception if the string is neither float nor nan - // 5. convert everything that's not floats to null - // 6. set the indices where we originally had nans to Float.NaN + // 1. identify the nans + // 2. identify the floats. "null" and letters are not considered floats + // 3. if ansi is enabled we want to throw an exception if the string is neither float nor nan + // 4. convert everything that's not floats to null + // 5. set the indices where we originally had nans to Float.NaN // // NOTE Limitation: "1.7976931348623159E308" and "-1.7976931348623159E308" are not considered // Inf even though Spark does @@ -1226,24 +1345,30 @@ object GpuCast extends Arm { } } + def fixDecimalBounds(input: ColumnView, + outOfBounds: ColumnView, + ansiMode: Boolean): ColumnVector = { + if (ansiMode) { + withResource(outOfBounds.any()) { isAny => + if (isAny.isValid && isAny.getBoolean) { + throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) + } + } + input.copyToColumnVector() + } else { + withResource(Scalar.fromNull(input.getType)) { nullVal => + outOfBounds.ifElse(nullVal, input) + } + } + } + def checkNFixDecimalBounds( input: ColumnView, to: DecimalType, ansiMode: Boolean): ColumnVector = { assert(input.getType.isDecimalType) withResource(DecimalUtil.outOfBounds(input, to)) { outOfBounds => - if (ansiMode) { - withResource(outOfBounds.any()) { isAny => - if (isAny.isValid && isAny.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) - } - } - input.copyToColumnVector() - } else { - withResource(Scalar.fromNull(input.getType)) { nullVal => - outOfBounds.ifElse(nullVal, input) - } - } + fixDecimalBounds(input, outOfBounds, ansiMode) } } 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 4574f5ba14e..2e4730cc45c 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 @@ -163,7 +163,7 @@ class GpuExpandIterator( val (cv, nullColumnReused) = expr.columnarEval(cb) match { case null => getOrCreateNullCV(sparkType) case other => - (GpuExpressionsUtils.resolveColumnVector(other, cb.numRows, sparkType), false) + (GpuExpressionsUtils.resolveColumnVector(other, cb.numRows), false) } if (!nullColumnReused) { uniqueDeviceColumns += cv 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 c2a171f7b46..7732f76bbc4 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 @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.shims.v2.{ShimBinaryExpression, ShimExpression, S import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.StringType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -64,15 +64,14 @@ object GpuExpressionsUtils extends Arm { * should cover all the cases for GPU pipelines. * * @param any the input value. It will be closed if it is a closeable after the call done. - * @param numRows the expected row number of the output column, used when 'any' is a Scalar. - * @param dType the data type of the output column, used when 'any' is a Scalar. + * @param numRows the expected row number of the output column, used when 'any' is a GpuScalar. * @return a `GpuColumnVector` if it succeeds. Users should close the column vector to avoid * memory leak. */ - def resolveColumnVector(any: Any, numRows: Int, dType: DataType): GpuColumnVector = { + def resolveColumnVector(any: Any, numRows: Int): GpuColumnVector = { withResourceIfAllowed(any) { case c: GpuColumnVector => c.incRefCount() - case s: GpuScalar => GpuColumnVector.from(s, numRows, dType) + case s: GpuScalar => GpuColumnVector.from(s, numRows, s.dataType) case other => throw new IllegalArgumentException(s"Cannot resolve a ColumnVector from the value:" + s" $other. Please convert it to a GpuScalar or a GpuColumnVector before returning.") @@ -91,7 +90,7 @@ object GpuExpressionsUtils extends Arm { * memory leak. */ def columnarEvalToColumn(expr: Expression, batch: ColumnarBatch): GpuColumnVector = - resolveColumnVector(expr.columnarEval(batch), batch.numRows, expr.dataType) + resolveColumnVector(expr.columnarEval(batch), batch.numRows) /** * Extract the GpuLiteral diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala index 098f0a69396..fa1052e9fcc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMapUtils.scala @@ -127,7 +127,7 @@ object GpuMapUtils extends Arm { } // Copied from Spark org.apache.spark.sql.errors.QueryExecutionErrors - private[this] def duplicateMapKeyFoundError: Throwable = { + def duplicateMapKeyFoundError: Throwable = { new RuntimeException(s"Duplicate map key was found, please check the input " + "data. If you want to remove the duplicated keys, you can set " + s"${SQLConf.MAP_KEY_DEDUP_POLICY.key} to ${SQLConf.MapKeyDedupPolicy.LAST_WIN} so that " + 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 d65501ceb43..5834a707a96 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 @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import ai.rapids.cudf.DType import com.nvidia.spark.rapids.RapidsConf.{SUPPRESS_PLANNING_FAILURE, TEST_CONF} -import com.nvidia.spark.rapids.shims.v2.{GpuSpecifiedWindowFrameMeta, GpuWindowExpressionMeta, OffsetWindowFunctionMeta} +import com.nvidia.spark.rapids.shims.v2.{AQEUtils, GpuSpecifiedWindowFrameMeta, GpuWindowExpressionMeta, OffsetWindowFunctionMeta} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SparkSession} @@ -56,7 +56,6 @@ import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.hive.rapids.GpuHiveOverrides import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids._ -import org.apache.spark.sql.rapids.aggregate.GpuSum import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand import org.apache.spark.sql.rapids.execution._ import org.apache.spark.sql.rapids.execution.python._ @@ -548,6 +547,30 @@ object GpuOverrides extends Logging { } } + /** + * Searches the plan for ReusedExchangeExec instances containing a GPU shuffle where the + * output types between the two plan nodes do not match. In such a case the ReusedExchangeExec + * will be updated to match the GPU shuffle output types. + */ + def fixupReusedExchangeExecs(plan: SparkPlan): SparkPlan = { + def outputTypesMatch(a: Seq[Attribute], b: Seq[Attribute]): Boolean = + a.corresponds(b)((x, y) => x.dataType == y.dataType) + plan.transformUp { + case sqse: ShuffleQueryStageExec => + sqse.plan match { + case ReusedExchangeExec(output, gsee: GpuShuffleExchangeExecBase) if ( + !outputTypesMatch(output, gsee.output)) => + val newOutput = sqse.plan.output.zip(gsee.output).map { case (c, g) => + assert(c.isInstanceOf[AttributeReference] && g.isInstanceOf[AttributeReference], + s"Expected AttributeReference but found $c and $g") + AttributeReference(c.name, g.dataType, c.nullable, c.metadata)(c.exprId, c.qualifier) + } + AQEUtils.newReuseInstance(sqse, newOutput) + case _ => sqse + } + } + } + @scala.annotation.tailrec def extractLit(exp: Expression): Option[Literal] = exp match { case l: Literal => Some(l) @@ -2264,21 +2287,6 @@ object GpuOverrides extends Logging { override def tagAggForGpu(): Unit = { val inputDataType = a.child.dataType checkAndTagFloatAgg(inputDataType, conf, this) - - a.dataType match { - case _: DecimalType => - val unboundPrecision = a.child.dataType.asInstanceOf[DecimalType].precision + 10 - if (unboundPrecision > DType.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("overflow checking on sum would need " + - s"a precision of $unboundPrecision to properly detect overflows") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"sum(${a.child.dataType}) produces ${a.dataType}") - } - } - case _ => // NOOP - } } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = @@ -2286,14 +2294,7 @@ object GpuOverrides extends Logging { }), expr[First]( "first aggregate operator", { - val checks = ExprChecks.aggNotWindow( - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all, - Seq(ParamCheck("input", - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all)) - ).asInstanceOf[ExprChecksImpl] - // TODO: support GpuFirst on nested types for reduction - // https://github.com/NVIDIA/spark-rapids/issues/3221 - val nestedChecks = ContextChecks( + ExprChecks.aggNotWindow( (TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all, @@ -2302,7 +2303,6 @@ object GpuOverrides extends Logging { TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all)) ) - ExprChecksImpl(checks.contexts ++ Map(GroupByAggExprContext -> nestedChecks)) }, (a, conf, p, r) => new AggExprMeta[First](a, conf, p, r) { override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = @@ -2313,14 +2313,7 @@ object GpuOverrides extends Logging { }), expr[Last]( "last aggregate operator", { - val checks = ExprChecks.aggNotWindow( - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all, - Seq(ParamCheck("input", - TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL, TypeSig.all)) - ).asInstanceOf[ExprChecksImpl] - // TODO: support GpuLast on nested types for reduction - // https://github.com/NVIDIA/spark-rapids/issues/3221 - val nestedChecks = ContextChecks( + ExprChecks.aggNotWindow( (TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all, @@ -2329,7 +2322,6 @@ object GpuOverrides extends Logging { TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all)) ) - ExprChecksImpl(checks.contexts ++ Map(GroupByAggExprContext -> nestedChecks)) }, (a, conf, p, r) => new AggExprMeta[Last](a, conf, p, r) { override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = @@ -3038,6 +3030,17 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new GpuRLikeMeta(a, conf, p, r)).disabledByDefault( "the implementation is not 100% compatible. " + "See the compatibility guide for more information."), + expr[RegExpExtract]( + "RegExpExtract", + ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING, + Seq(ParamCheck("str", TypeSig.STRING, TypeSig.STRING), + ParamCheck("regexp", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), + ParamCheck("idx", TypeSig.lit(TypeEnum.INT), + TypeSig.lit(TypeEnum.INT)))), + (a, conf, p, r) => new GpuRegExpExtractMeta(a, conf, p, r)) + .disabledByDefault( + "the implementation is not 100% compatible. " + + "See the compatibility guide for more information."), expr[Length]( "String character length or binary byte length", ExprChecks.unaryProject(TypeSig.INT, TypeSig.INT, @@ -3678,7 +3681,7 @@ object GpuOverrides extends Logging { exec[SampleExec]( "The backend for the sample operator", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_64).nested(), TypeSig.all), + TypeSig.ARRAY + TypeSig.DECIMAL_128_FULL).nested(), TypeSig.all), (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) ), ShimLoader.getSparkShims.aqeShuffleReaderExec, @@ -3910,7 +3913,11 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { val updatedPlan = if (plan.conf.adaptiveExecutionEnabled) { // AQE can cause Spark to inject undesired CPU shuffles into the plan because GPU and CPU // distribution expressions are not semantically equal. - GpuOverrides.removeExtraneousShuffles(plan, conf) + val newPlan = GpuOverrides.removeExtraneousShuffles(plan, conf) + + // AQE can cause ReusedExchangeExec instance to cache the wrong aggregation buffer type + // compared to the desired buffer type from a reused GPU shuffle. + GpuOverrides.fixupReusedExchangeExecs(newPlan) } else { plan } 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 09041d8a9ef..e93bd20b223 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,14 +16,8 @@ package com.nvidia.spark.rapids -import java.lang.reflect.Method - import scala.annotation.tailrec -import com.nvidia.spark.rapids.shims.v2.ShimUnaryExecNode - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, SortOrder} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ @@ -34,8 +28,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.rapids.{GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv} -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastToCpuExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToCpuExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} /** * Rules that run after the row to columnar and columnar to row transitions have been inserted. @@ -82,48 +75,64 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { def optimizeAdaptiveTransitions( plan: SparkPlan, parent: Option[SparkPlan]): SparkPlan = plan match { + + case bb @ GpuBringBackToHost(child) if parent.isEmpty => + // This is hacky but we need to remove the GpuBringBackToHost from the final + // query stage, if there is one. It gets inserted by + // GpuTransitionOverrides.insertColumnarFromGpu around columnar adaptive + // plans when we are writing to columnar formats on the GPU. It would be nice to avoid + // inserting it in the first place but we just don't have enough context + // at the time GpuTransitionOverrides is applying rules. + optimizeAdaptiveTransitions(child, Some(bb)) + // HostColumnarToGpu(RowToColumnarExec(..)) => GpuRowToColumnarExec(..) case HostColumnarToGpu(r2c: RowToColumnarExec, goal) => val child = optimizeAdaptiveTransitions(r2c.child, Some(r2c)) - val preProcessing = child.getTagValue(GpuOverrides.preRowToColProjection) - .getOrElse(Seq.empty) - val transition = GpuRowToColumnarExec(child, goal, preProcessing) - r2c.child match { - case _: AdaptiveSparkPlanExec => - // When the input is an adaptive plan we do not get to see the GPU version until - // the plan is executed and sometimes the plan will have a GpuColumnarToRowExec as the - // final operator and we can bypass this to keep the data columnar by inserting - // the [[AvoidAdaptiveTransitionToRow]] operator here - AvoidAdaptiveTransitionToRow(transition) + child match { + case a: AdaptiveSparkPlanExec => + // we hit this case when we have an adaptive plan wrapped in a write + // to columnar file format on the GPU + val columnarAdaptivePlan = ShimLoader.getSparkShims.columnarAdaptivePlan(a, goal) + optimizeAdaptiveTransitions(columnarAdaptivePlan, None) case _ => - transition + val preProcessing = child.getTagValue(GpuOverrides.preRowToColProjection) + .getOrElse(Seq.empty) + GpuRowToColumnarExec(child, goal, preProcessing) } - case ColumnarToRowExec(GpuBringBackToHost( - GpuShuffleCoalesceExec(e: GpuShuffleExchangeExecBase, _))) if parent.isEmpty => - // We typically want the final operator in the plan (the operator that has no parent) to be - // wrapped in `ColumnarToRowExec(GpuBringBackToHost(ShuffleCoalesceExec(_)))` operators to - // bring the data back onto the host and be translated to rows so that it can be returned - // from the Spark API. However, in the case of AQE, each exchange operator is treated as an - // individual query with no parent and we need to remove these operators in this case - // because we need to return an operator that implements `BroadcastExchangeLike` or - // `ShuffleExchangeLike`. The coalesce step gets added back into the plan later on, in a - // future query stage that reads the output from this query stage. This is handled in the - // case clauses below. - e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) - - case ColumnarToRowExec(GpuBringBackToHost( - GpuCoalesceBatches(e: GpuShuffleExchangeExecBase, _))) if parent.isEmpty => - // We typically want the final operator in the plan (the operator that has no parent) to be - // wrapped in `ColumnarToRowExec(GpuBringBackToHost(GpuCoalesceBatches(_)))` operators to - // bring the data back onto the host and be translated to rows so that it can be returned - // from the Spark API. However, in the case of AQE, each exchange operator is treated as an - // individual query with no parent and we need to remove these operators in this case - // because we need to return an operator that implements `BroadcastExchangeLike` or - // `ShuffleExchangeLike`. The coalesce step gets added back into the plan later on, in a - // future query stage that reads the output from this query stage. This is handled in the - // case clauses below. - e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + // adaptive plan final query stage with columnar output + case r2c @ RowToColumnarExec(child) if parent.isEmpty => + val optimizedChild = optimizeAdaptiveTransitions(child, Some(r2c)) + val preProcessing = optimizedChild.getTagValue(GpuOverrides.preRowToColProjection) + .getOrElse(Seq.empty) + GpuRowToColumnarExec(optimizedChild, TargetSize(rapidsConf.gpuTargetBatchSizeBytes), + preProcessing) + + case ColumnarToRowExec(bb: GpuBringBackToHost) => + // We typically want the final operator in the plan (the operator that has no parent) to be + // wrapped in `ColumnarToRowExec(GpuBringBackToHost(_))` operators to + // bring the data back onto the host and be translated to rows so that it can be returned + // from the Spark API. However, in the case of AQE, each exchange operator is treated as an + // individual query with no parent and we need to remove these operators in this case + // because we need to return an operator that implements `BroadcastExchangeLike` or + // `ShuffleExchangeLike`. + bb.child match { + case GpuShuffleCoalesceExec(e: GpuShuffleExchangeExecBase, _) if parent.isEmpty => + // The coalesce step gets added back into the plan later on, in a + // future query stage that reads the output from this query stage. This + // is handled in the case clauses below. + e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + case GpuCoalesceBatches(e: GpuShuffleExchangeExecBase, _) if parent.isEmpty => + // The coalesce step gets added back into the plan later on, in a + // future query stage that reads the output from this query stage. This + // is handled in the case clauses below. + e.withNewChildren(e.children.map(c => optimizeAdaptiveTransitions(c, Some(e)))) + case _ => optimizeAdaptiveTransitions(bb.child, Some(bb)) match { + case e: GpuBroadcastExchangeExecBase => e + case e: GpuShuffleExchangeExecBase => e + case other => getColumnarToRowExec(other) + } + } case s: ShuffleQueryStageExec => // When reading a materialized shuffle query stage in AQE mode, we need to insert an @@ -170,13 +179,6 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { case ColumnarToRowExec(e: ShuffleQueryStageExec) => getColumnarToRowExec(optimizeAdaptiveTransitions(e, Some(plan))) - case ColumnarToRowExec(bb: GpuBringBackToHost) => - optimizeAdaptiveTransitions(bb.child, Some(bb)) match { - case e: GpuBroadcastExchangeExec => e - case e: GpuShuffleExchangeExecBase => e - case other => getColumnarToRowExec(other) - } - // inserts postColumnarToRowTransition into newly-created GpuColumnarToRowExec case p if p.getTagValue(GpuOverrides.postColToRowProjection).nonEmpty => val c2r = p.children.map(optimizeAdaptiveTransitions(_, Some(p))).head @@ -613,61 +615,4 @@ object GpuTransitionOverrides { case _: InputFileBlockLength => true case e => e.children.exists(checkHasInputFileExpressions) } -} - -/** - * This operator will attempt to optimize the case when we are writing the results of - * an adaptive query to disk so that we remove the redundant transitions from columnar - * to row within AdaptiveSparkPlanExec followed by a row to columnar transition. - * - * Specifically, this is the plan we see in this case: - * - * {{{ - * GpuRowToColumnar(AdaptiveSparkPlanExec(GpuColumnarToRow(child)) - * }}} - * - * We perform this optimization at runtime rather than during planning, because when the adaptive - * plan is being planned and executed, we don't know whether it is being called from an operation - * that wants rows (such as CollectTailExec) or from an operation that wants columns (such as - * GpuDataWritingCommandExec). - * - * Spark does not provide a mechanism for executing an adaptive plan and retrieving columnar - * results and the internal methods that we need to call are private, so we use reflection to - * call them. - * - * @param child The plan to execute - */ -case class AvoidAdaptiveTransitionToRow(child: SparkPlan) extends ShimUnaryExecNode with GpuExec { - - override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException(s"Row-based execution should not occur for $this") - - override def output: Seq[Attribute] = child.output - - override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child match { - case GpuRowToColumnarExec(a: AdaptiveSparkPlanExec, _, _) => - val getFinalPhysicalPlan = getPrivateMethod("getFinalPhysicalPlan") - val plan = getFinalPhysicalPlan.invoke(a) - val rdd = plan match { - case t: GpuColumnarToRowExec => - t.child.executeColumnar() - case _ => - child.executeColumnar() - } - - // final UI update - val finalPlanUpdate = getPrivateMethod("finalPlanUpdate") - finalPlanUpdate.invoke(a) - - rdd - - case _ => - child.executeColumnar() - } - - private def getPrivateMethod(name: String): Method = { - val m = classOf[AdaptiveSparkPlanExec].getDeclaredMethod(name) - m.setAccessible(true) - m - } -} +} \ No newline at end of file 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 aeda8d8b394..3fef0aac5ed 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 @@ -322,11 +322,13 @@ object GpuWindowExec extends Arm { // First pass replace any operations that should be totally replaced. val replacePass = expr.transformDown { case GpuWindowExpression( - GpuAggregateExpression(rep: GpuReplaceWindowFunction, _, _, _, _), spec) => + GpuAggregateExpression(rep: GpuReplaceWindowFunction, _, _, _, _), spec) + if rep.shouldReplaceWindow(spec) => // We don't actually care about the GpuAggregateExpression because it is ignored // by our GPU window operations anyways. rep.windowReplacement(spec) - case GpuWindowExpression(rep: GpuReplaceWindowFunction, spec) => + case GpuWindowExpression(rep: GpuReplaceWindowFunction, spec) + if rep.shouldReplaceWindow(spec) => rep.windowReplacement(spec) } // Second pass looks for GpuWindowFunctions and GpuWindowSpecDefinitions to build up @@ -372,6 +374,8 @@ object GpuWindowExec extends Arm { def isRunningWindow(spec: GpuWindowSpecDefinition): Boolean = spec match { case GpuWindowSpecDefinition(_, _, GpuSpecifiedWindowFrame(RowFrame, GpuSpecialFrameBoundary(UnboundedPreceding), GpuSpecialFrameBoundary(CurrentRow))) => true + case GpuWindowSpecDefinition(_, _, GpuSpecifiedWindowFrame(RowFrame, + GpuSpecialFrameBoundary(UnboundedPreceding), GpuLiteral(value, _))) if value == 0 => true case _ => false } } 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 ada4000e9c4..547a6cc0d64 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 @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging 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._ -import org.apache.spark.sql.rapids.{GpuAggregateExpression, GpuCreateNamedStruct} +import org.apache.spark.sql.rapids.{GpuAdd, GpuAggregateExpression, GpuCreateNamedStruct} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -630,6 +630,12 @@ trait GpuReplaceWindowFunction extends GpuWindowFunction { * in the future. */ def windowReplacement(spec: GpuWindowSpecDefinition): Expression + + /** + * Return true if windowReplacement should be called to replace this GpuWindowFunction with + * something else. + */ + def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = true } /** @@ -881,14 +887,17 @@ class BatchedRunningWindowBinaryFixer(val binOp: BinaryOp, val name: String) class SumBinaryFixer(toType: DataType, isAnsi: Boolean) extends BatchedRunningWindowFixer with Arm with Logging { private val name = "sum" - private val binOp = BinaryOp.ADD private var previousResult: Option[Scalar] = None + private var previousOverflow: Option[Scalar] = None - def updateState(finalOutputColumn: cudf.ColumnVector): Unit = { + def updateState(finalOutputColumn: cudf.ColumnVector, + wasOverflow: Option[cudf.ColumnVector]): Unit = { + val lastIndex = finalOutputColumn.getRowCount.toInt - 1 logDebug(s"$name: updateState from $previousResult to...") previousResult.foreach(_.close) - previousResult = - Some(finalOutputColumn.getScalarElement(finalOutputColumn.getRowCount.toInt - 1)) + previousResult = Some(finalOutputColumn.getScalarElement(lastIndex)) + previousOverflow.foreach(_.close()) + previousOverflow = wasOverflow.map(_.getScalarElement(lastIndex)) logDebug(s"$name: ... $previousResult") } @@ -911,8 +920,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) throw new IllegalArgumentException(s"Making a zero scalar for $other is not supported") } - override def fixUp(samePartitionMask: Either[cudf.ColumnVector, Boolean], - sameOrderMask: Option[Either[cudf.ColumnVector, Boolean]], + private[this] def fixUpNonDecimal(samePartitionMask: Either[cudf.ColumnVector, Boolean], windowedColumnOutput: cudf.ColumnView): cudf.ColumnVector = { logDebug(s"$name: fix up $previousResult $samePartitionMask") val ret = (previousResult, samePartitionMask) match { @@ -928,7 +936,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } withResource(nullsReplaced) { nullsReplaced => - nullsReplaced.binaryOp(binOp, prev, prev.getType) + nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType) } } else { // prev is NULL but NULL + something == NULL which we don't want @@ -948,7 +956,7 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } withResource(nullsReplaced) { nullsReplaced => - withResource(nullsReplaced.binaryOp(binOp, prev, prev.getType)) { updated => + withResource(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { updated => mask.ifElse(updated, windowedColumnOutput) } } @@ -958,15 +966,126 @@ class SumBinaryFixer(toType: DataType, isAnsi: Boolean) } } closeOnExcept(ret) { ret => - updateState(ret) + updateState(ret, None) + ret + } + } + + private[this] def fixUpDecimal(samePartitionMask: Either[cudf.ColumnVector, Boolean], + windowedColumnOutput: cudf.ColumnView, + dt: DecimalType): cudf.ColumnVector = { + logDebug(s"$name: fix up $previousResult $samePartitionMask") + val (ret, decimalOverflowOnAdd) = (previousResult, previousOverflow, samePartitionMask) match { + case (None, None, _) => + // The mask is all false so do nothing + withResource(Scalar.fromBool(false)) { falseVal => + closeOnExcept(ColumnVector.fromScalar(falseVal, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + case (Some(prev), Some(previousOver), scala.util.Right(mask)) => + if (mask) { + if (!prev.isValid) { + // So in the window operation we can have a null if all of the input values before it + // were also null or if we overflowed the result and inserted in a null. + // + // If we overflowed, then all of the output for this group should be null, but the + // overflow check code can handle inserting that, so just inc the ref count and return + // the overflow column. + // + // If we didn't overflow, and the input is null then + // prev is NULL but NULL + something == NULL which we don't want, so also + // just increment the reference count and go on. + closeOnExcept(ColumnVector.fromScalar(previousOver, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } else { + // The previous didn't overflow, so now we need to do the add and check for overflow. + val nullsReplaced = withResource(windowedColumnOutput.isNull) { nulls => + withResource(makeZeroScalar(windowedColumnOutput.getType)) { zero => + nulls.ifElse(zero, windowedColumnOutput) + } + } + withResource(nullsReplaced) { nullsReplaced => + closeOnExcept(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { added => + (added, GpuAdd.didDecimalOverflow(nullsReplaced, prev, added)) + } + } + } + } else { + // The mask is all false so do nothing + withResource(Scalar.fromBool(false)) { falseVal => + closeOnExcept(ColumnVector.fromScalar(falseVal, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + } + case (Some(prev), Some(previousOver), scala.util.Left(mask)) => + if (prev.isValid) { + // The previous didn't overflow, so now we need to do the add and check for overflow. + val nullsReplaced = withResource(windowedColumnOutput.isNull) { nulls => + withResource(nulls.and(mask)) { shouldReplace => + withResource(makeZeroScalar(windowedColumnOutput.getType)) { zero => + shouldReplace.ifElse(zero, windowedColumnOutput) + } + } + } + withResource(nullsReplaced) { nullsReplaced => + withResource(nullsReplaced.binaryOp(BinaryOp.ADD, prev, prev.getType)) { added => + closeOnExcept(mask.ifElse(added, windowedColumnOutput)) { updated => + withResource(Scalar.fromBool(false)) { falseVal => + withResource(GpuAdd.didDecimalOverflow(nullsReplaced, prev, added)) { over => + (updated, mask.ifElse(over, falseVal)) + } + } + } + } + } + } else { + // So in the window operation we can have a null if all of the input values before it + // were also null or if we overflowed the result and inserted in a null. + // + // If we overflowed, then all of the output for this group should be null, but the + // overflow check code can handle inserting that, so just inc the ref count and return + // the overflow column. + // + // If we didn't overflow, and the input is null then + // prev is NULL but NULL + something == NULL which we don't want, so also + // just increment the reference count and go on. + closeOnExcept(ColumnVector.fromScalar(previousOver, + windowedColumnOutput.getRowCount.toInt)) { over => + (incRef(windowedColumnOutput), over) + } + } + case _ => + throw new IllegalStateException("INTERNAL ERROR: Should never have a situation where " + + "prev and previousOver do not match.") } + withResource(ret) { ret => + withResource(decimalOverflowOnAdd) { decimalOverflowOnAdd => + withResource(DecimalUtil.outOfBounds(ret, dt)) { valOutOfBounds => + withResource(valOutOfBounds.or(decimalOverflowOnAdd)) { outOfBounds => + closeOnExcept(GpuCast.fixDecimalBounds(ret, outOfBounds, isAnsi)) { replaced => + updateState(replaced, Some(outOfBounds)) + replaced + } + } + } + } + } + } + + override def fixUp(samePartitionMask: Either[cudf.ColumnVector, Boolean], + sameOrderMask: Option[Either[cudf.ColumnVector, Boolean]], + windowedColumnOutput: cudf.ColumnView): cudf.ColumnVector = { toType match { case dt: DecimalType => - withResource(ret) { ret => - GpuCast.checkNFixDecimalBounds(ret, dt, isAnsi) - } + fixUpDecimal(samePartitionMask, windowedColumnOutput, dt) case _ => - ret + fixUpNonDecimal(samePartitionMask, windowedColumnOutput) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 12d9c07965f..6783f9b5ddf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -352,7 +352,7 @@ object RapidsConf { .doc("The amount of GPU memory that should remain unallocated by RMM and left for " + "system use such as memory needed for kernels and kernel launches.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(ByteUnit.MiB.toBytes(256)) + .createWithDefault(ByteUnit.MiB.toBytes(640)) val HOST_SPILL_STORAGE_SIZE = conf("spark.rapids.memory.host.spillStorageSize") .doc("Amount of off-heap host memory to use for buffering spilled GPU data " + @@ -642,14 +642,6 @@ object RapidsConf { .booleanConf .createWithDefault(false) - val ENABLE_CREATE_MAP = conf("spark.rapids.sql.createMap.enabled") - .doc("The GPU-enabled version of the `CreateMap` expression (`map` SQL function) does not " + - "detect duplicate keys in all cases and does not guarantee which key wins if there are " + - "duplicates. When this config is set to true, `CreateMap` will be enabled to run on the " + - "GPU even when there might be duplicate keys.") - .booleanConf - .createWithDefault(false) - val ENABLE_INNER_JOIN = conf("spark.rapids.sql.join.inner.enabled") .doc("When set to true inner joins are enabled on the GPU") .booleanConf @@ -1327,6 +1319,12 @@ object RapidsConf { .booleanConf .createWithDefault(value = false) + val ENABLE_FAST_SAMPLE = conf("spark.rapids.sql.fast.sample") + .doc("Option to turn on fast sample. If enable it is inconsistent with CPU sample " + + "because of GPU sample algorithm is inconsistent with CPU.") + .booleanConf + .createWithDefault(value = false) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -1589,8 +1587,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isProjectAstEnabled: Boolean = get(ENABLE_PROJECT_AST) - lazy val isCreateMapEnabled: Boolean = get(ENABLE_CREATE_MAP) - lazy val isParquetEnabled: Boolean = get(ENABLE_PARQUET) lazy val isParquetInt96WriteEnabled: Boolean = get(ENABLE_PARQUET_INT96_WRITE) @@ -1743,6 +1739,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isCpuBasedUDFEnabled: Boolean = get(ENABLE_CPU_BASED_UDF) + lazy val isFastSampleEnabled: Boolean = get(ENABLE_FAST_SAMPLE) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because 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 37a72474e13..ff5d159c190 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 @@ -46,7 +46,7 @@ trait DataFromReplacementRule { * A version of DataFromReplacementRule that is used when no replacement rule can be found. */ final class NoRuleDataFromReplacementRule extends DataFromReplacementRule { - override val operationName: String = "NOT_FOUND" + override val operationName: String = "" override def confKey = "NOT_FOUND" @@ -463,7 +463,7 @@ final class RuleNotFoundPartMeta[INPUT <: Partitioning]( extends PartMeta[INPUT](part, conf, parent, new NoRuleDataFromReplacementRule) { override def tagPartForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU enabled version of partitioning ${part.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${part.getClass}") } override def convertToGpu(): GpuPartitioning = @@ -498,7 +498,7 @@ final class RuleNotFoundScanMeta[INPUT <: Scan]( extends ScanMeta[INPUT](scan, conf, parent, new NoRuleDataFromReplacementRule) { override def tagSelfForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU enabled version of scan ${scan.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${scan.getClass}") } override def convertToGpu(): Scan = @@ -534,7 +534,7 @@ final class RuleNotFoundDataWritingCommandMeta[INPUT <: DataWritingCommand]( extends DataWritingCommandMeta[INPUT](cmd, conf, parent, new NoRuleDataFromReplacementRule) { override def tagSelfForGpu(): Unit = { - willNotWorkOnGpu(s"no GPU accelerated version of command ${cmd.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${cmd.getClass}") } override def convertToGpu(): GpuDataWritingCommand = @@ -795,7 +795,7 @@ final class RuleNotFoundSparkPlanMeta[INPUT <: SparkPlan]( extends SparkPlanMeta[INPUT](plan, conf, parent, new NoRuleDataFromReplacementRule) { override def tagPlanForGpu(): Unit = - willNotWorkOnGpu(s"no GPU enabled version of operator ${plan.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${plan.getClass}") override def convertToGpu(): GpuExec = throw new IllegalStateException("Cannot be converted to GPU") @@ -1096,6 +1096,16 @@ abstract class UnaryExprMeta[INPUT <: UnaryExpression]( convertToGpu(childExprs.head.convertToGpu()) def convertToGpu(child: Expression): GpuExpression + + /** + * `ConstantFolding` executes early in the logical plan process, which + * simplifies many things before we get to the physical plan. If you enable + * AQE, some optimizations can cause new expressions to show up that would have been + * folded in by the logical plan optimizer (like `cast(null as bigint)` which just + * becomes Literal(null, Long) after `ConstantFolding`), so enabling this here + * allows us to handle these when they are generated by an AQE rule. + */ + override val isFoldableNonLitAllowed: Boolean = true } /** Base metadata class for unary expressions that support conversion to AST as well */ @@ -1307,7 +1317,7 @@ final class RuleNotFoundExprMeta[INPUT <: Expression]( extends ExprMeta[INPUT](expr, conf, parent, new NoRuleDataFromReplacementRule) { override def tagExprForGpu(): Unit = - willNotWorkOnGpu(s"no GPU enabled version of expression ${expr.getClass} could be found") + willNotWorkOnGpu(s"GPU does not currently support the operator ${expr.getClass}") override def convertToGpu(): GpuExpression = throw new IllegalStateException("Cannot be converted to GPU") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index c60e8db3524..43ac51cb56d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -150,9 +150,11 @@ class RegexParser(pattern: String) { case '[' => // treat as a literal character and add to the character class characterClass.append(ch) - case ']' if pos > start + 1 => + case ']' if (!characterClass.negated && pos > start + 1) || + (characterClass.negated && pos > start + 2) => // "[]" is not a valid character class // "[]a]" is a valid character class containing the characters "]" and "a" + // "[^]a]" is a valid negated character class containing the characters "]" and "a" characterClassComplete = true case '^' if pos == start + 1 => // Negates the character class, causing it to match a single character not listed in @@ -167,11 +169,9 @@ class RegexParser(pattern: String) { throw new RegexUnsupportedException( s"unexpected EOF while parsing escaped character", Some(pos)) case Some(ch) => - ch match { - case '\\' | '^' | '-' | ']' | '+' => - // escaped metacharacter within character class - characterClass.appendEscaped(consumeExpected(ch)) - } + // typically an escaped metacharacter ('\\', '^', '-', ']', '+') + // within the character class, but could be any escaped character + characterClass.appendEscaped(consumeExpected(ch)) } case '\u0000' => throw new RegexUnsupportedException( @@ -444,7 +444,17 @@ class CudfRegexTranspiler(replace: Boolean) { // this is a bit extreme and it would be good to replace with finer-grained // rules throw new RegexUnsupportedException("regexp_replace on GPU does not support ^ or $") - + case '$' => + RegexSequence(ListBuffer( + RegexRepetition( + RegexCharacterClass(negated = false, + characters = ListBuffer(RegexChar('\r'))), + SimpleQuantifier('?')), + RegexRepetition( + RegexCharacterClass(negated = false, + characters = ListBuffer(RegexChar('\n'))), + SimpleQuantifier('?')), + RegexChar('$'))) case _ => regex } @@ -482,7 +492,48 @@ class CudfRegexTranspiler(replace: Boolean) { } val components: Seq[RegexCharacterClassComponent] = characters .map(x => rewrite(x).asInstanceOf[RegexCharacterClassComponent]) - RegexCharacterClass(negated, ListBuffer(components: _*)) + + if (negated) { + // There are differences between cuDF and Java handling of newlines + // for negative character matches. The expression `[^a]` will match + // `\r` and `\n` in Java but not in cuDF, so we replace `[^a]` with + // `(?:[\r\n]|[^a])`. We also have to take into account whether any + // newline characters are included in the character range. + // + // Examples: + // + // `[^a]` => `(?:[\r\n]|[^a])` + // `[^a\r]` => `(?:[\n]|[^a])` + // `[^a\n]` => `(?:[\r]|[^a])` + // `[^a\r\n]` => `[^a]` + // `[^\r\n]` => `[^\r\n]` + + val linefeedCharsInPattern = components.flatMap { + case RegexChar(ch) if ch == '\n' || ch == '\r' => Seq(ch) + case RegexEscaped(ch) if ch == 'n' => Seq('\n') + case RegexEscaped(ch) if ch == 'r' => Seq('\r') + case _ => Seq.empty + } + + val onlyLinefeedChars = components.length == linefeedCharsInPattern.length + + val negatedNewlines = Seq('\r', '\n').diff(linefeedCharsInPattern.distinct) + + if (onlyLinefeedChars && linefeedCharsInPattern.length == 2) { + // special case for `[^\r\n]` and `[^\\r\\n]` + RegexCharacterClass(negated = true, ListBuffer(components: _*)) + } else if (negatedNewlines.isEmpty) { + RegexCharacterClass(negated = true, ListBuffer(components: _*)) + } else { + RegexGroup(capture = false, + RegexChoice( + RegexCharacterClass(negated = false, + characters = ListBuffer(negatedNewlines.map(RegexChar): _*)), + RegexCharacterClass(negated = true, ListBuffer(components: _*)))) + } + } else { + RegexCharacterClass(negated, ListBuffer(components: _*)) + } case RegexSequence(parts) => if (parts.isEmpty) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index f7d851c4451..b8372e7392c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -253,8 +253,25 @@ trait SparkShims { def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] + def isEmptyRelation(relation: Any): Boolean + def broadcastModeTransform(mode: BroadcastMode, toArray: Array[InternalRow]): Any + /** + * This call can produce an `EmptyHashedRelation` or an empty array, + * allowing the AQE rule `EliminateJoinToEmptyRelation` in Spark 3.1.x + * to optimize certain joins. + * + * In Spark 3.2.0, the optimization is still performed (under `AQEPropagateEmptyRelation`), + * but the AQE optimizer is looking at the metrics for the query stage to determine + * if numRows == 0, and if so it can eliminate certain joins. + * + * The call is implemented only for Spark 3.1.x+. It is disabled in + * Databricks because it requires a task context to perform the + * `BroadcastMode.transform` call, but we'd like to call this from the driver. + */ + def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] + def isAqePlan(p: SparkPlan): Boolean def isExchangeOp(plan: SparkPlanMeta[_]): Boolean @@ -298,4 +315,13 @@ trait SparkShims { def timestampFormatInRead(csvOpts: CSVOptions): Option[String] def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] + + /** + * Determine if the Spark version allows the supportsColumnar flag to be overridden + * in AdaptiveSparkPlanExec. This feature was introduced in Spark 3.2 as part of + * SPARK-35881. + */ + def supportsColumnarAdaptivePlans: Boolean + + def columnarAdaptivePlan(a: AdaptiveSparkPlanExec, goal: CoalesceSizeGoal): SparkPlan } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index b9b5741e497..e04d442d448 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -19,8 +19,6 @@ package com.nvidia.spark.rapids import java.io.{File, FileOutputStream} import java.time.ZoneId -import scala.collection.mutable - import ai.rapids.cudf.DType import com.nvidia.spark.rapids.shims.v2.TypeSigUtil @@ -1163,27 +1161,6 @@ object CreateMapCheck extends ExprChecks { val context = exprMeta.context if (context != ProjectExprContext) { meta.willNotWorkOnGpu(s"this is not supported in the $context context") - } else { - // if there are more than two key-value pairs then there is the possibility of duplicate keys - if (meta.childExprs.length > 2) { - // check for duplicate keys if the keys are literal values - val keyExprs = meta.childExprs.indices.filter(_ % 2 == 0).map(meta.childExprs) - val litKeys = keyExprs.map(e => GpuOverrides.extractLit(e.wrapped.asInstanceOf[Expression])) - if (litKeys.forall(_.isDefined)) { - val keys = litKeys.map(_.get.value) - val uniqueKeys = new mutable.HashSet[Any]() - for (key <- keys) { - if (!uniqueKeys.add(key)) { - meta.willNotWorkOnGpu("CreateMap with duplicate literal keys is not supported") - } - } - } else if (!meta.conf.isCreateMapEnabled) { - meta.willNotWorkOnGpu("CreateMap is not enabled by default when there are " + - "multiple key-value pairs and where the keys are not literal values because handling " + - "of duplicate keys is not compatible with Spark. " + - s"Set ${RapidsConf.ENABLE_CREATE_MAP}=true to enable it anyway.") - } - } } } @@ -1293,7 +1270,7 @@ class CastChecks extends ExprChecks { val calendarChecks: TypeSig = none val sparkCalendarSig: TypeSig = CALENDAR + STRING - val arrayChecks: TypeSig = ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + + val arrayChecks: TypeSig = STRING + ARRAY.nested(commonCudfTypes + DECIMAL_128_FULL + NULL + ARRAY + BINARY + STRUCT + MAP) + psNote(TypeEnum.ARRAY, "The array's child type must also support being cast to " + "the desired child type") 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 448ddfe6be6..a90b3309163 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 @@ -188,6 +188,7 @@ class GpuHashAggregateIterator( metrics: GpuHashAggregateMetrics, configuredTargetBatchSize: Long) extends Iterator[ColumnarBatch] with Arm with AutoCloseable with Logging { + // Partial mode: // 1. boundInputReferences: picks column from raw input // 2. boundFinalProjections: is a pass-through of the agg buffer @@ -205,7 +206,6 @@ class GpuHashAggregateIterator( // (GpuAverage => CudfSum/CudfCount) // 3. boundResultReferences: project the result expressions Spark expects in the output. private case class BoundExpressionsModeAggregates( - boundInputReferences: Seq[GpuExpression], boundFinalProjections: Option[Seq[GpuExpression]], boundResultReferences: Seq[Expression]) @@ -277,13 +277,10 @@ class GpuHashAggregateIterator( /** Aggregate all input batches and place the results in the aggregatedBatches queue. */ private def aggregateInputBatches(): Unit = { - val aggHelper = new AggHelper(merge = false) + val aggHelper = new AggHelper(forceMerge = false) while (cbIter.hasNext) { - val (childBatch, isLastInputBatch) = withResource(cbIter.next()) { inputBatch => - val isLast = GpuColumnVector.isTaggedAsFinalBatch(inputBatch) - (processIncomingBatch(inputBatch), isLast) - } - withResource(childBatch) { _ => + withResource(cbIter.next()) { childBatch => + val isLastInputBatch = GpuColumnVector.isTaggedAsFinalBatch(childBatch) withResource(computeAggregate(childBatch, aggHelper)) { aggBatch => val batch = LazySpillableColumnarBatch(aggBatch, metrics.spillCallback, "aggbatch") // Avoid making batch spillable for the common case of the last and only batch @@ -385,7 +382,7 @@ class GpuHashAggregateIterator( wasBatchMerged } - private lazy val concatAndMergeHelper = new AggHelper(merge = true) + private lazy val concatAndMergeHelper = new AggHelper(forceMerge = true) /** * Concatenate batches together and perform a merge aggregation on the result. The input batches @@ -426,8 +423,8 @@ class GpuHashAggregateIterator( } val shims = ShimLoader.getSparkShims - val ordering = groupingExpressions.map(shims.sortOrder(_, Ascending, NullsFirst)) val groupingAttributes = groupingExpressions.map(_.toAttribute) + val ordering = groupingAttributes.map(shims.sortOrder(_, Ascending, NullsFirst)) val aggBufferAttributes = groupingAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) val sorter = new GpuSorter(ordering, aggBufferAttributes) @@ -544,27 +541,6 @@ class GpuHashAggregateIterator( } } - /** Perform the initial projection on the input batch and extract the result columns */ - private def processIncomingBatch(batch: ColumnarBatch): ColumnarBatch = { - val aggTime = metrics.computeAggTime - val opTime = metrics.opTime - withResource(new NvtxWithMetrics("prep agg batch", NvtxColor.CYAN, aggTime, - opTime)) { _ => - val cols = boundExpressions.boundInputReferences.safeMap { ref => - val childCv = GpuExpressionsUtils.columnarEvalToColumn(ref, batch) - if (DataType.equalsStructurally(childCv.dataType, ref.dataType, ignoreNullability = true)) { - childCv - } else { - withResource(childCv) { childCv => - val rapidsType = GpuColumnVector.getNonNestedRapidsType(ref.dataType) - GpuColumnVector.from(childCv.getBase.castTo(rapidsType), ref.dataType) - } - } - } - new ColumnarBatch(cols.toArray, batch.numRows()) - } - } - /** * Concatenates batches after extracting them from `LazySpillableColumnarBatch` * @note the input batches are not closed as part of this operation @@ -603,30 +579,6 @@ class GpuHashAggregateIterator( val aggBufferAttributes = groupingAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) - // Adapted from `AggregationIterator.initializeAggregateFunctions` in Spark: - // - we use the "imperative aggregate" way as it used bound expressions due to - // lack of support of codegen (like our case) - // - for partial/complete: we bind to the inputProjection as specified by each - // `GpuAggregateFunction` to the `inputAttributes` (see how those are defined) - // - for partial merge/final: it is the pass through case, we are getting as input - // the "agg buffer", and we are using `inputAggBufferAttributes` to match the Spark - // function. We still bind to `inputAttributes`, as those would be setup for pass-through - // in the partial merge/final cases. - val aggBound = aggregateExpressions.flatMap { agg => - agg.mode match { - case Partial | Complete => - agg.aggregateFunction.inputProjection - case PartialMerge | Final => - agg.aggregateFunction.inputAggBufferAttributes - case mode => - throw new NotImplementedError(s"can't translate ${mode}") - } - } - - val boundInputReferences = GpuBindReferences.bindGpuReferences( - groupingExpressions ++ aggBound, - inputAttributes) - val boundFinalProjections = if (modeInfo.hasFinalMode || modeInfo.hasCompleteMode) { val finalProjections = groupingExpressions ++ aggregateExpressions.map(_.aggregateFunction.evaluateExpression) @@ -659,7 +611,6 @@ class GpuHashAggregateIterator( groupingAttributes) } BoundExpressionsModeAggregates( - boundInputReferences, boundFinalProjections, boundResultReferences) } @@ -667,12 +618,12 @@ class GpuHashAggregateIterator( /** * Internal class used in `computeAggregates` for the pre, agg, and post steps * - * @param merge - if true, we are merging two pre-aggregated batches, so we should use + * @param forceMerge - if true, we are merging two pre-aggregated batches, so we should use * the merge steps for each aggregate function * @param isSorted - if the batch is sorted this is set to true and is passed to cuDF * as an optimization hint */ - class AggHelper(merge: Boolean, isSorted: Boolean = false) { + class AggHelper(forceMerge: Boolean, isSorted: Boolean = false) { // `CudfAggregate` instances to apply, either update or merge aggregates private val cudfAggregates = new mutable.ArrayBuffer[CudfAggregate]() @@ -693,11 +644,16 @@ class GpuHashAggregateIterator( private val postStep = new mutable.ArrayBuffer[Expression]() private val postStepAttr = new mutable.ArrayBuffer[Attribute]() - // we add the grouping expression first, which bind as pass-through - preStep ++= GpuBindReferences.bindGpuReferences( - groupingAttributes, groupingAttributes) - postStep ++= GpuBindReferences.bindGpuReferences( - groupingAttributes, groupingAttributes) + // we add the grouping expression first, which should bind as pass-through + if (forceMerge) { + // a grouping expression can do actual computation, but we cannot do that computation again + // on a merge, nor would we want to if we could. So use the attributes instead of the + // original expression when we are forcing a merge. + preStep ++= groupingAttributes + } else { + preStep ++= groupingExpressions + } + postStep ++= groupingAttributes postStepAttr ++= groupingAttributes postStepDataTypes ++= groupingExpressions.map(_.dataType) @@ -705,14 +661,14 @@ class GpuHashAggregateIterator( private var ix = groupingAttributes.length for (aggExp <- aggregateExpressions) { val aggFn = aggExp.aggregateFunction - if ((aggExp.mode == Partial || aggExp.mode == Complete) && !merge) { + if ((aggExp.mode == Partial || aggExp.mode == Complete) && !forceMerge) { val ordinals = (ix until ix + aggFn.updateAggregates.length) aggOrdinals ++= ordinals ix += ordinals.length val updateAggs = aggFn.updateAggregates postStepDataTypes ++= updateAggs.map(_.dataType) cudfAggregates ++= updateAggs - preStep ++= aggFn.aggBufferAttributes + preStep ++= aggFn.inputProjection postStep ++= aggFn.postUpdate postStepAttr ++= aggFn.postUpdateAttr } else { @@ -729,8 +685,11 @@ class GpuHashAggregateIterator( } // a bound expression that is applied before the cuDF aggregate - private val preStepBound = - GpuBindReferences.bindGpuReferences(preStep, aggBufferAttributes) + private val preStepBound = if (forceMerge) { + GpuBindReferences.bindGpuReferences(preStep.toList, aggBufferAttributes.toList) + } else { + GpuBindReferences.bindGpuReferences(preStep, inputAttributes) + } // a bound expression that is applied after the cuDF aggregate private val postStepBound = @@ -1388,11 +1347,11 @@ case class GpuHashAggregateExec( private val inputAggBufferAttributes: Seq[Attribute] = { aggregateExpressions - // there're exactly four cases needs `inputAggBufferAttributes` from child according to the - // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, - // Partial -> PartialMerge, PartialMerge -> PartialMerge. - .filter(a => a.mode == Final || a.mode == PartialMerge) - .flatMap(_.aggregateFunction.inputAggBufferAttributes) + // there're exactly four cases needs `inputAggBufferAttributes` from child according to the + // agg planning in `AggUtils`: Partial -> Final, PartialMerge -> Final, + // Partial -> PartialMerge, PartialMerge -> PartialMerge. + .filter(a => a.mode == Final || a.mode == PartialMerge) + .flatMap(_.aggregateFunction.aggBufferAttributes) } private lazy val uniqueModes: Seq[AggregateMode] = aggregateExpressions.map(_.mode).distinct @@ -1519,7 +1478,7 @@ case class GpuHashAggregateExec( */ override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ - aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) 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 a882a19dde1..73b719a8bec 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 @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import scala.annotation.tailrec +import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf import ai.rapids.cudf._ @@ -365,19 +366,32 @@ case class GpuFilterExec( } } -class GpuSampleExecMeta(sample: SampleExec, conf: RapidsConf, p: Option[RapidsMeta[_, _, _]], +class GpuSampleExecMeta( + sample: SampleExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], r: DataFromReplacementRule) extends SparkPlanMeta[SampleExec](sample, conf, p, r) - with Logging { + with Logging { override def convertToGpu(): GpuExec = { val gpuChild = childPlans.head.convertIfNeeded() - GpuSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, - sample.seed, gpuChild) + if (conf.isFastSampleEnabled) { + // Use GPU sample JNI, this is faster, but the output is not the same as CPU produces + GpuFastSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, + sample.seed, gpuChild) + } else { + // The output is the same as CPU produces + // First generates row indexes by CPU sampler, then use GPU to gathers + GpuSampleExec(sample.lowerBound, sample.upperBound, sample.withReplacement, + sample.seed, gpuChild) + } } } -case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement: Boolean, - seed: Long, child: SparkPlan) - extends ShimUnaryExecNode with GpuExec { +case class GpuSampleExec( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, child: SparkPlan) extends ShimUnaryExecNode with GpuExec { override lazy val additionalMetrics: Map[String, GpuMetric] = Map( OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME)) @@ -404,7 +418,9 @@ case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) val opTime = gpuLongMetric(OP_TIME) + val rdd = child.executeColumnar() + // CPU consistent, first generates sample row indexes by CPU, then gathers by GPU if (withReplacement) { new GpuPartitionwiseSampledRDD( rdd, @@ -415,46 +431,106 @@ case class GpuSampleExec(lowerBound: Double, upperBound: Double, withReplacement } else { rdd.mapPartitionsWithIndex( (index, iterator) => { - // use CPU sampler generate filter + // use CPU sampler generate row indexes val sampler = new BernoulliCellSampler(lowerBound, upperBound) sampler.setSeed(seed + index) - iterator.map[ColumnarBatch] { batch => - numOutputBatches += 1 - withResource(batch) { b => // will generate new columnar column, close this - val numRows = b.numRows() - val filter = withResource(HostColumnVector.builder(DType.BOOL8, numRows)) { - builder => - (0 until numRows).foreach { _ => - val n = sampler.sample() - if (n > 0) { - builder.append(1.toByte) - numOutputRows += 1 - } else { - builder.append(0.toByte) - } + iterator.map[ColumnarBatch] { columnarBatch => + // collect sampled row idx + // samples idx in batch one by one, so it's same as CPU execution + withResource(new NvtxWithMetrics("Sample Exec", NvtxColor.YELLOW, opTime)) { _ => + withResource(columnarBatch) { cb => + // generate sampled row indexes by CPU + val sampledRows = new ArrayBuffer[Int] + var rowIndex = 0 + while (rowIndex < cb.numRows()) { + if (sampler.sample() > 0) { + sampledRows += rowIndex } - builder.buildAndPutOnDevice() + rowIndex += 1 + } + numOutputBatches += 1 + numOutputRows += sampledRows.length + // gather by row indexes + GatherUtils.gather(cb, sampledRows) } + } + } + } + , preservesPartitioning = true + ) + } + } +} - // use GPU filer rows - val colTypes = GpuColumnVector.extractTypes(b) - withResource(filter) { filter => - withResource(GpuColumnVector.from(b)) { tbl => - withResource(tbl.filter(filter)) { filteredData => - if (filteredData.getRowCount == 0) { - GpuColumnVector.emptyBatchFromTypes(colTypes) - } else { - GpuColumnVector.from(filteredData, colTypes) - } +case class GpuFastSampleExec( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, + child: SparkPlan) extends ShimUnaryExecNode with GpuExec { + + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME)) + + override def output: Seq[Attribute] = { + child.output + } + + // add one coalesce exec to avoid empty batch and small batch, + // because sample will shrink the batch + override val coalesceAfter: Boolean = true + + // Note GPU sample does not preserve the ordering + override def outputOrdering: Seq[SortOrder] = Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException(s"Row-based execution should not occur for $this") + + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val opTime = gpuLongMetric(OP_TIME) + val rdd = child.executeColumnar() + + // CPU inconsistent, uses GPU sample JNI + rdd.mapPartitionsWithIndex( + (index, iterator) => { + iterator.map[ColumnarBatch] { columnarBatch => + withResource(new NvtxWithMetrics("Fast Sample Exec", NvtxColor.YELLOW, opTime)) { _ => + withResource(columnarBatch) { cb => + numOutputBatches += 1 + val numSampleRows = (cb.numRows() * (upperBound - lowerBound)).toLong + + val colTypes = GpuColumnVector.extractTypes(cb) + if (numSampleRows == 0L) { + GpuColumnVector.emptyBatchFromTypes(colTypes) + } else if (cb.numCols() == 0) { + // for count agg, num of cols is 0 + val c = GpuColumnVector.emptyBatchFromTypes(colTypes) + c.setNumRows(numSampleRows.toInt) + c + } else { + withResource(GpuColumnVector.from(cb)) { table => + // GPU sample + withResource(table.sample(numSampleRows, withReplacement, seed + index)) { + sampled => + val cb = GpuColumnVector.from(sampled, colTypes) + numOutputRows += cb.numRows() + cb } } } } } } - ,preservesPartitioning = true - ) - } + } + , preservesPartitioning = true + ) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala index 6384461e0d1..770fcb028fe 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.rapids import com.nvidia.spark.RapidsUDF -import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RepeatingParamCheck, ShimLoader, TypeSig} +import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RapidsConf, RepeatingParamCheck, ShimLoader, TypeSig} import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig import org.apache.spark.sql.catalyst.expressions.Expression @@ -45,59 +45,86 @@ object GpuHiveOverrides { Seq( GpuOverrides.expr[HiveSimpleUDF]( - "Hive UDF, support requires the UDF to implement a RAPIDS accelerated interface", + "Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to" + + " get better performance", ExprChecks.projectOnly( udfTypeSig, TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))), (a, conf, p, r) => new ExprMeta[HiveSimpleUDF](a, conf, p, r) { + private val opRapidsFunc = a.function match { + case rapidsUDF: RapidsUDF => Some(rapidsUDF) + case _ => None + } + override def tagExprForGpu(): Unit = { - a.function match { - case _: RapidsUDF => - case _ => - willNotWorkOnGpu(s"Hive UDF ${a.name} implemented by " + - s"${a.funcWrapper.functionClassName} does not provide a GPU implementation") + if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) { + willNotWorkOnGpu(s"Hive SimpleUDF ${a.name} implemented by " + + s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " + + s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`") } } override def convertToGpu(): GpuExpression = { - // To avoid adding a Hive dependency just to check if the UDF function is deterministic, - // we use the original HiveSimpleUDF `deterministic` method as a proxy. - GpuHiveSimpleUDF( - a.name, - a.funcWrapper, - childExprs.map(_.convertToGpu()), - a.dataType, - a.deterministic) + opRapidsFunc.map { _ => + // We use the original HiveGenericUDF `deterministic` method as a proxy + // for simplicity. + GpuHiveSimpleUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu()), + a.dataType, + a.deterministic) + }.getOrElse { + // This `require` is just for double check. + require(conf.isCpuBasedUDFEnabled) + GpuRowBasedHiveSimpleUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu())) + } } }), GpuOverrides.expr[HiveGenericUDF]( - "Hive Generic UDF, support requires the UDF to implement a " + - "RAPIDS accelerated interface", + "Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to" + + " get better performance", ExprChecks.projectOnly( udfTypeSig, TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))), (a, conf, p, r) => new ExprMeta[HiveGenericUDF](a, conf, p, r) { + private val opRapidsFunc = a.function match { + case rapidsUDF: RapidsUDF => Some(rapidsUDF) + case _ => None + } + override def tagExprForGpu(): Unit = { - a.function match { - case _: RapidsUDF => - case _ => - willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " + - s"${a.funcWrapper.functionClassName} does not provide a GPU implementation") + if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) { + willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " + + s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " + + s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`") } } override def convertToGpu(): GpuExpression = { - // To avoid adding a Hive dependency just to check if the UDF function is deterministic, - // we use the original HiveGenericUDF `deterministic` method as a proxy. - GpuHiveGenericUDF( - a.name, - a.funcWrapper, - childExprs.map(_.convertToGpu()), - a.dataType, - a.deterministic, - a.foldable) + opRapidsFunc.map { _ => + // We use the original HiveGenericUDF `deterministic` method as a proxy + // for simplicity. + GpuHiveGenericUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu()), + a.dataType, + a.deterministic, + a.foldable) + }.getOrElse { + // This `require` is just for double check. + require(conf.isCpuBasedUDFEnabled) + GpuRowBasedHiveGenericUDF( + a.name, + a.funcWrapper, + childExprs.map(_.convertToGpu())) + } } }) ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala new file mode 100644 index 00000000000..85cb9c09c9b --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/rowBasedHiveUDFs.scala @@ -0,0 +1,190 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.rapids + +import scala.collection.JavaConverters._ + +import com.nvidia.spark.rapids.{GpuExpression, GpuLiteral, GpuRowBasedUserDefinedFunction, GpuScalar} +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF} +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, SpecializedGetters} +import org.apache.spark.sql.hive.{DeferredObjectAdapter, HiveInspectors} +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.types.DataType + +/** Common implementation across row-based Hive UDFs */ +trait GpuRowBasedHiveUDFBase extends GpuRowBasedUserDefinedFunction with HiveInspectors { + val funcWrapper: HiveFunctionWrapper + + @transient + val function: AnyRef + + override val udfDeterministic: Boolean = { + val udfType = function.getClass.getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() && !udfType.stateful() + } + + override final val checkNull: Boolean = false + + override def nullable: Boolean = true + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def prettyName: String = name + + /** Create an object inspector from a GPU expression. */ + protected def gpuToInspector(expr: Expression): ObjectInspector = expr match { + case GpuLiteral(value, dataType) => + // Convert GpuLiterals to CPU Literals as possible as we can for primitive types, to + // leverage the Spark `toInspector(Expression)` method as much as possible. + // Because the `toInspector(Expression)` method will take care of the CPU Literal + // especially, converting it to a ConstantObjectInspector when it is primitive type. A + // `ConstantObjectInspector` can accelerate the row data reading by caching the actual + // value and skipping the null check which becomes unnecessary. + value match { + case scalar: ai.rapids.cudf.Scalar => + if (scalar.getType.isNestedType) { + // Nested type, so create an inspector from the data type. + toInspector(dataType) + } else { + try { + toInspector(Literal.create(GpuScalar.extract(scalar), dataType)) + } catch { + // Unsupported type for extraction, so use the data type way instead. + case _: UnsupportedOperationException => toInspector(dataType) + } + } + case _ => toInspector(Literal.create(value, dataType)) + } + case ge: GpuExpression if ge.foldable => + // Create an inspector from the data type instead, to avoid evaluation on the driver side, + // which will be triggered inside the `toInspector(Expression)` method for a foldable + // expression. Because GPU expressions should not be evaluated on the driver side. + toInspector(ge.dataType) + case _ => + // For other expressions, it is safe to call `toInspector(Expression)`, which will call into + // `toInspector(DataType)` directly for now. + toInspector(expr) + } + + @transient + protected lazy val childRowAccessors: Array[SpecializedGetters => Any] = + children.zipWithIndex.map { case (child, i) => + val accessor = InternalRow.getAccessor(child.dataType, child.nullable) + row: SpecializedGetters => accessor(row, i) + }.toArray + + @transient + protected lazy val argumentInspectors = children.map(gpuToInspector) +} + +/** Row-based version of Spark's `HiveSimpleUDF` running in a GPU operation */ +case class GpuRowBasedHiveSimpleUDF( + name: String, + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends GpuRowBasedHiveUDFBase { + + @scala.annotation.nowarn("msg=class UDF in package exec is deprecated") + @transient + override lazy val function: UDF = funcWrapper.createFunction[UDF]() + + @transient + private lazy val wrappers = children.map(x => wrapperFor(gpuToInspector(x), x.dataType)).toArray + + @transient + private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) + + @transient + private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray + + @transient + private lazy val method = + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo).asJava) + + // Create parameter converters + @transient + private lazy val conversionHelper = new ConversionHelper(method, argumentInspectors.toArray) + + @transient + private lazy val unwrapper = unwrapperFor( + ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType, ObjectInspectorOptions.JAVA)) + + override protected def evaluateRow(childrenRow: InternalRow): Any = { + val inputs = wrap(childRowAccessors.map(_(childrenRow)), wrappers, cached, inputDataTypes) + val ret = FunctionRegistry.invoke( + method, + function, + conversionHelper.convertIfNecessary(inputs : _*): _*) + unwrapper(ret) + } + + override lazy val dataType: DataType = javaTypeToDataType(method.getGenericReturnType) + + override def foldable: Boolean = udfDeterministic && children.forall(_.foldable) + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" +} + +/** Row-based version of Spark's `HiveGenericUDF` running in a GPU operation */ +case class GpuRowBasedHiveGenericUDF( + name: String, + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends GpuRowBasedHiveUDFBase { + + @transient + override lazy val function: GenericUDF = funcWrapper.createFunction[GenericUDF]() + + @transient + private lazy val returnInspector = + function.initializeAndFoldConstants(argumentInspectors.toArray) + + @transient + private lazy val deferredObjects = argumentInspectors.zip(children).map { + case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType) + }.toArray + + @transient + private lazy val unwrapper = unwrapperFor(returnInspector) + + override protected def evaluateRow(childrenRow: InternalRow): Any = { + returnInspector // Make sure initialized. + + var i = 0 + val length = children.length + while (i < length) { + val idx = i + deferredObjects(i).set(() => childRowAccessors(idx)(childrenRow)) + i += 1 + } + unwrapper(function.evaluate(deferredObjects.asInstanceOf[Array[DeferredObject]])) + } + + override lazy val dataType: DataType = inspectorToDataType(returnInspector) + + override def foldable: Boolean = + udfDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] +} 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 c9fc473e221..55953cc1b7c 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 @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeS import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.util.{ArrayData, TypeUtils} -import org.apache.spark.sql.rapids.aggregate.GpuSum +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.aggregate.GpuSumDefaults import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -176,14 +177,12 @@ trait GpuAggregateFunction extends GpuExpression */ val evaluateExpression: Expression - /** Attributes of fields in aggBufferSchema. */ - def aggBufferAttributes: Seq[AttributeReference] - /** - * Result of the aggregate function when the input is empty. This is currently only used for the - * proper rewriting of distinct aggregate functions. + * This is the contract with the outside world. It describes what the output of postUpdate should + * look like, and what the input to preMerge looks like. It also describes what the output of + * postMerge must look like. */ - def defaultResult: Option[GpuLiteral] = None + def aggBufferAttributes: Seq[AttributeReference] def sql(isDistinct: Boolean): String = { val distinct = if (isDistinct) "DISTINCT " else "" @@ -196,19 +195,8 @@ trait GpuAggregateFunction extends GpuExpression prettyName + flatArguments.mkString(start, ", ", ")") } - /** - * 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]]. - */ - final lazy val inputAggBufferAttributes: Seq[AttributeReference] = - aggBufferAttributes.map(_.newInstance()) - /** An aggregate function is not foldable. */ final override def foldable: Boolean = false - - /** The schema of the aggregation buffer. */ - def aggBufferSchema: StructType = null //not used in GPU version } case class WrappedAggFunction(aggregateFunction: GpuAggregateFunction, filter: Expression) @@ -575,6 +563,75 @@ case class GpuMax(child: Expression) extends GpuAggregateFunction } } +/** + * All decimal processing in Spark has overflow detection as a part of it. Either it replaces + * the value with a null in non-ANSI mode, or it throws an exception in ANSI mode. Spark will also + * do the processing for larger values as `Decimal` values which are based on `BigDecimal` and have + * unbounded precision. So in most cases it is impossible to overflow/underflow so much that an + * incorrect value is returned. Spark will just use more and more memory to hold the value and + * then check for overflow at some point when the result needs to be turned back into a 128-bit + * value. + * + * We cannot do the same thing. Instead we take three strategies to detect overflow. + * + * 1. For decimal values with a precision of 8 or under we follow Spark and do the SUM + * on the unscaled value as a long, and then bit-cast the result back to a Decimal value. + * this means that we can SUM `174,467,442,481` maximum or minimum decimal values with a + * precision of 8 before overflow can no longer be detected. It is much higher for decimal + * values with a smaller precision. + * 2. For decimal values with a precision from 9 to 20 inclusive we sum them as 128-bit values. + * this is very similar to what we do in the first strategy. The main differences are that we + * use a 128-bit value when doing the sum, and we check for overflow after processing each batch. + * In the case of group-by and reduction that happens after the update stage and also after each + * merge stage. This gives us enough room that we can always detect overflow when summing a + * single batch. Even on a merge where we could be doing the aggregation on a batch that has + * all max output values in it. + * 3. For values from 21 to 28 inclusive we have enough room to not check for overflow on teh update + * aggregation, but for the merge aggregation we need to do some extra checks. This is done by + * taking the digits above 28 and sum them separately. We then check to see if they would have + * overflowed the original limits. This lets us detect overflow in cases where the original + * value would have wrapped around. The reason this works is because we have a hard limit on the + * maximum number of values in a single batch being processed. `Int.MaxValue`, or about 2.2 + * billion values. So we use a precision on the higher values that is large enough to handle + * 2.2 billion values and still detect overflow. This equates to a precision of about 10 more + * than is needed to hold the higher digits. This effectively gives us unlimited overflow + * detection. + * 4. For anything larger than precision 28 we do the same overflow detection for strategy 3, but + * also do it on the update aggregation. This lets us fully detect overflows in any stage of + * an aggregation. + * + * Note that for Window operations either there is no merge stage or it only has a single value + * being merged into a batch instead of an entire batch being merged together. This lets us handle + * the overflow detection with what is built into GpuAdd. + */ +object GpuDecimalSumOverflow { + /** + * The increase in precision for the output of a SUM from the input. This is hard coded by + * Spark so we just have it here. This means that for most types without being limited to + * a precision of 38 you get 10-billion+ values before an overflow would even be possible. + */ + val sumPrecisionIncrease: Int = 10 + + /** + * Generally we want a guarantee that is at least 10x larger than the original overflow. + */ + val extraGuaranteePrecision: Int = 1 + + /** + * The precision above which we need extra overflow checks while doing an update. This is because + * anything above this precision could in theory overflow beyond detection within a single input + * batch. + */ + val updateCutoffPrecision: Int = 28 + + /** + * This is the precision above which we need to do extra checks for overflow when merging + * results. This is because anything above this precision could in theory overflow a decimal128 + * value beyond detection in a batch of already updated and checked values. + */ + val mergeCutoffPrecision: Int = 20 +} + /** * This is equivalent to what Spark does after a sum to check for overflow * ` @@ -627,15 +684,122 @@ case class GpuCheckOverflowAfterSum( override def children: Seq[Expression] = Seq(data, isEmpty) } -trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes - with GpuBatchedRunningWindowWithFixer - with GpuAggregateWindowFunction - with GpuRunningWindowFunction { +/** + * This extracts the highest digits from a Decimal value as a part of doing a SUM. + */ +case class GpuDecimalSumHighDigits( + input: Expression, + originalInputType: DecimalType) extends GpuExpression with ShimExpression { + + override def nullable: Boolean = input.nullable + + override def toString: String = s"GpuDecimalSumHighDigits($input)" + + override def sql: String = input.sql + + override val dataType: DecimalType = DecimalType(originalInputType.precision + + GpuDecimalSumOverflow.sumPrecisionIncrease + GpuDecimalSumOverflow.extraGuaranteePrecision - + GpuDecimalSumOverflow.updateCutoffPrecision, 0) + // Marking these as lazy because they are not serializable + private lazy val outputDType = GpuColumnVector.getNonNestedRapidsType(dataType) + private lazy val intermediateDType = + DType.create(DType.DTypeEnum.DECIMAL128, outputDType.getScale) + + private lazy val divisionFactor: Decimal = + Decimal(math.pow(10, GpuDecimalSumOverflow.updateCutoffPrecision)) + private val divisionType = DecimalType(38, 0) + + override def columnarEval(batch: ColumnarBatch): Any = { + withResource(GpuProjectExec.projectSingle(batch, input)) { inputCol => + val inputBase = inputCol.getBase + // We don't have direct access to 128 bit ints so we use a decimal with a scale of 0 + // as a stand in. + val bitCastInputType = DType.create(DType.DTypeEnum.DECIMAL128, 0) + val divided = withResource(inputBase.bitCastTo(bitCastInputType)) { bitCastInput => + withResource(GpuScalar.from(divisionFactor, divisionType)) { divisor => + bitCastInput.div(divisor, intermediateDType) + } + } + val ret = withResource(divided) { divided => + if (divided.getType.equals(outputDType)) { + divided.incRefCount() + } else { + divided.castTo(outputDType) + } + } + GpuColumnVector.from(ret, dataType) + } + } + + override def children: Seq[Expression] = Seq(input) +} + +/** + * Return a boolean if this decimal overflowed or not + */ +case class GpuDecimalDidOverflow( + data: Expression, + rangeType: DecimalType, + nullOnOverflow: Boolean) extends GpuExpression with ShimExpression { - val child: Expression - val resultType: DataType - val failOnErrorOverride: Boolean - val extraDecimalOverflowChecks: Boolean + override def nullable: Boolean = true + + override def toString: String = + s"GpuDecimalDidOverflow($data, $rangeType, $nullOnOverflow)" + + override def sql: String = data.sql + + override def dataType: DataType = BooleanType + + override def columnarEval(batch: ColumnarBatch): Any = { + withResource(GpuProjectExec.projectSingle(batch, data)) { dataCol => + val dataBase = dataCol.getBase + withResource(DecimalUtil.outOfBounds(dataBase, rangeType)) { outOfBounds => + if (!nullOnOverflow) { + withResource(outOfBounds.any()) { isAny => + if (isAny.isValid && isAny.getBoolean) { + throw new ArithmeticException("Overflow as a part of SUM") + } + } + } else { + GpuColumnVector.from(outOfBounds.incRefCount(), dataType) + } + } + } + } + + override def children: Seq[Expression] = Seq(data) +} + +case class GpuSum(child: Expression, + resultType: DataType, + failOnErrorOverride: Boolean = SQLConf.get.ansiEnabled, + forceWindowSumToNotBeReplaced: Boolean = false) + extends GpuAggregateFunction with ImplicitCastInputTypes + with GpuReplaceWindowFunction + with GpuBatchedRunningWindowWithFixer + with GpuAggregateWindowFunction + with GpuRunningWindowFunction { + + private lazy val childIsDecimal: Boolean = + child.dataType.isInstanceOf[DecimalType] + + private lazy val childDecimalType: DecimalType = + child.dataType.asInstanceOf[DecimalType] + + private lazy val needsDec128MergeOverflowChecks: Boolean = + childIsDecimal && childDecimalType.precision > GpuDecimalSumOverflow.mergeCutoffPrecision + + private lazy val needsDec128UpdateOverflowChecks: Boolean = + childIsDecimal && + childDecimalType.precision > GpuDecimalSumOverflow.updateCutoffPrecision + + // For some operations we need to SUm the higher digits in addition to the regular value so + // we can detect overflow. This is the type of the higher digits SUM value. + private lazy val higherDigitsCheckType: DecimalType = { + val t = resultType.asInstanceOf[DecimalType] + DecimalType(t.precision - GpuDecimalSumOverflow.updateCutoffPrecision, 0) + } private lazy val zeroDec = { val dt = resultType.asInstanceOf[DecimalType] @@ -643,74 +807,198 @@ trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes } override lazy val initialValues: Seq[GpuLiteral] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => + case _: DecimalType if GpuSumDefaults.hasIsEmptyField => Seq(zeroDec, GpuLiteral(true, BooleanType)) case _ => Seq(GpuLiteral(null, resultType)) } + private lazy val updateHigherOrderBits = { + val input = if (child.dataType != resultType) { + GpuCast(child, resultType) + } else { + child + } + GpuDecimalSumHighDigits(input, childDecimalType) + } + // we need to cast to `resultType` here, since Spark is not widening types // as done before Spark 3.2.0. See CudfSum for more info. override lazy val inputProjection: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - // Spark tracks null columns through a second column isEmpty for decimal. - Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), GpuIsNull(child)) + case _: DecimalType => + // Decimal is complicated... + if (GpuSumDefaults.hasIsEmptyField) { + // Spark tracks null columns through a second column isEmpty for decimal. So null values + // are replaced with 0, and a separate boolean column for isNull is added + if (needsDec128UpdateOverflowChecks) { + // If we want extra checks for overflow, then we also want to include it here + Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), + GpuIsNull(child), + updateHigherOrderBits) + } else { + Seq(GpuIf(GpuIsNull(child), zeroDec, GpuCast(child, resultType)), GpuIsNull(child)) + } + } else { + if (needsDec128UpdateOverflowChecks) { + // If we want extra checks for overflow, then we also want to include it here + Seq(GpuCast(child, resultType), updateHigherOrderBits) + } else { + Seq(GpuCast(child, resultType)) + } + } case _ => Seq(GpuCast(child, resultType)) } private lazy val updateSum = new CudfSum(resultType) private lazy val updateIsEmpty = new CudfMin(BooleanType) + private lazy val updateOverflow = new CudfSum(updateHigherOrderBits.dataType) override lazy val updateAggregates: Seq[CudfAggregate] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(updateSum, updateIsEmpty) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128UpdateOverflowChecks) { + Seq(updateSum, updateIsEmpty, updateOverflow) + } else { + Seq(updateSum, updateIsEmpty) + } + } else { + if (needsDec128UpdateOverflowChecks) { + Seq(updateSum, updateOverflow) + } else { + Seq(updateSum) + } + } case _ => Seq(updateSum) } + private[this] def extendedPostUpdateDecOverflowCheck(dt: DecimalType) = + GpuCheckOverflow( + GpuIf( + GpuDecimalDidOverflow(updateOverflow.attr, + higherDigitsCheckType, + !failOnErrorOverride), + GpuLiteral(null, dt), + updateSum.attr), + dt, !failOnErrorOverride) + override lazy val postUpdate: Seq[Expression] = resultType match { - case dt: DecimalType if extraDecimalOverflowChecks => - Seq(GpuCheckOverflow(updateSum.attr, dt, !failOnErrorOverride), updateIsEmpty.attr) + case dt: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128UpdateOverflowChecks) { + Seq(extendedPostUpdateDecOverflowCheck(dt), updateIsEmpty.attr) + } else { + Seq(GpuCheckOverflow(updateSum.attr, dt, !failOnErrorOverride), updateIsEmpty.attr) + } + } else { + if (needsDec128UpdateOverflowChecks) { + Seq(extendedPostUpdateDecOverflowCheck(dt)) + } else { + postUpdateAttr + } + } case _ => postUpdateAttr } // output of GpuSum private lazy val sum = AttributeReference("sum", resultType)() + // Used for Decimal overflow detection private lazy val isEmpty = AttributeReference("isEmpty", BooleanType)() override lazy val aggBufferAttributes: Seq[AttributeReference] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => + case _: DecimalType if GpuSumDefaults.hasIsEmptyField => sum :: isEmpty :: Nil case _ => sum :: Nil } + private lazy val mergeHigherOrderBits = GpuDecimalSumHighDigits(sum, childDecimalType) + override lazy val preMerge: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(sum, isEmpty, GpuIsNull(sum)) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq(sum, isEmpty, GpuIsNull(sum), mergeHigherOrderBits) + } else { + Seq(sum, isEmpty, GpuIsNull(sum)) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq(sum, mergeHigherOrderBits) + } else { + aggBufferAttributes + } + } case _ => aggBufferAttributes } private lazy val mergeSum = new CudfSum(resultType) private lazy val mergeIsEmpty = new CudfMin(BooleanType) private lazy val mergeIsOverflow = new CudfMax(BooleanType) + private lazy val mergeOverflow = new CudfSum(mergeHigherOrderBits.dataType) // To be able to do decimal overflow detection, we need a CudfSum that does **not** ignore nulls. // Cudf does not have such an aggregation, so for merge we have to work around that similar to // what happens with isEmpty override lazy val mergeAggregates: Seq[CudfAggregate] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(mergeSum, mergeIsEmpty, mergeIsOverflow) + case _: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq(mergeSum, mergeIsEmpty, mergeIsOverflow, mergeOverflow) + } else { + Seq(mergeSum, mergeIsEmpty, mergeIsOverflow) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq(mergeSum, mergeOverflow) + } else { + Seq(mergeSum) + } + } case _ => Seq(mergeSum) } override lazy val postMerge: Seq[Expression] = resultType match { - case _: DecimalType if extraDecimalOverflowChecks => - Seq(GpuIf(mergeIsOverflow.attr, GpuLiteral.create(null, resultType), mergeSum.attr), - mergeIsEmpty.attr) + case dt: DecimalType => + if (GpuSumDefaults.hasIsEmptyField) { + if (needsDec128MergeOverflowChecks) { + Seq( + GpuCheckOverflow( + GpuIf( + GpuOr( + GpuDecimalDidOverflow(mergeOverflow.attr, higherDigitsCheckType, + !failOnErrorOverride), + mergeIsOverflow.attr), + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride), + mergeIsEmpty.attr) + } else { + Seq( + GpuCheckOverflow(GpuIf(mergeIsOverflow.attr, + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride), + mergeIsEmpty.attr) + } + } else { + if (needsDec128MergeOverflowChecks) { + Seq( + GpuCheckOverflow( + GpuIf( + GpuDecimalDidOverflow(mergeOverflow.attr, higherDigitsCheckType, + !failOnErrorOverride), + GpuLiteral.create(null, resultType), + mergeSum.attr), + dt, !failOnErrorOverride)) + } else { + postMergeAttr + } + } + case _ => postMergeAttr } override lazy val evaluateExpression: Expression = resultType match { case dt: DecimalType => - if (extraDecimalOverflowChecks) { + if (GpuSumDefaults.hasIsEmptyField) { GpuCheckOverflowAfterSum(sum, isEmpty, dt, !failOnErrorOverride) } else { GpuCheckOverflow(sum, dt, !failOnErrorOverride) @@ -726,6 +1014,32 @@ trait GpuSumBase extends GpuAggregateFunction with ImplicitCastInputTypes override def checkInputDataTypes(): TypeCheckResult = TypeUtils.checkForNumericExpr(child.dataType, "function gpu sum") + // Replacement Window Function + override def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = { + // We only will replace this if we think an update will fail. In the cases where we can + // handle a window function larger than a single batch, we already have merge overflow + // detection enabled. + !forceWindowSumToNotBeReplaced && needsDec128UpdateOverflowChecks + } + + override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { + // We need extra overflow checks for some larger decimal type. To do these checks we + // extract the higher digits and SUM them separately to see if they would overflow. + // If they do we know that the regular SUM also overflowed. If not we know that we can rely on + // the existing overflow code to detect it. + val regularSum = GpuWindowExpression( + GpuSum(child, resultType, failOnErrorOverride = failOnErrorOverride, + forceWindowSumToNotBeReplaced = true), + spec) + val highOrderDigitsSum = GpuWindowExpression( + GpuSum( + GpuDecimalSumHighDigits(GpuCast(child, resultType), childDecimalType), + higherDigitsCheckType, + failOnErrorOverride = failOnErrorOverride), + spec) + GpuIf(GpuIsNull(highOrderDigitsSum), GpuLiteral(null, resultType), regularSum) + } + // GENERAL WINDOW FUNCTION // Spark 3.2.0+ stopped casting the input data to the output type before the sum operation // This fixes that. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala index 24d543ac169..24003addfef 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuPoissonSampler.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.rapids import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{DeviceMemoryBuffer, DType, GatherMap, HostMemoryBuffer, NvtxColor} -import com.nvidia.spark.rapids.{Arm, GpuColumnVector, GpuMetric, NvtxWithMetrics} +import ai.rapids.cudf.NvtxColor +import com.nvidia.spark.rapids.{Arm, GatherUtils, GpuMetric, NvtxWithMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.random.PoissonSampler @@ -37,52 +37,35 @@ class GpuPoissonSampler(fraction: Double, useGapSamplingIfPossible: Boolean, } else { batchIterator.map { columnarBatch => withResource(new NvtxWithMetrics("Sample Exec", NvtxColor.YELLOW, opTime)) { _ => - numOutputBatches += 1 withResource(columnarBatch) { cb => // collect sampled row idx // samples idx in batch one by one, so it's same with CPU version val sampledRows = sample(cb.numRows()) - val intBytes = DType.INT32.getSizeInBytes() - val totalBytes = sampledRows.length * intBytes - withResource(HostMemoryBuffer.allocate(totalBytes)) { hostBuffer => - // copy row idx to host buffer - for (idx <- 0 until sampledRows.length) { - hostBuffer.setInt(idx * intBytes, sampledRows(idx)) - } - - // generate gather map and send to GPU to gather - withResource(DeviceMemoryBuffer.allocate(totalBytes)) { deviceBuffer => - deviceBuffer.copyFromHostBuffer(0, hostBuffer, 0, totalBytes) - withResource(new GatherMap(deviceBuffer).toColumnView(0, sampledRows.length)) { - gatherCv => - val colTypes = GpuColumnVector.extractTypes(cb) - withResource(GpuColumnVector.from(cb)) { table => - withResource(table.gather(gatherCv)) { gatheredTable => - GpuColumnVector.from(gatheredTable, colTypes) - } - } - } - } - } + numOutputBatches += 1 + numOutputRows += sampledRows.length + GatherUtils.gather(cb, sampledRows) } } } } } - // collect the sampled row idx + // collect the sampled row indexes, Note one row can be sampled multiple times private def sample(numRows: Int): ArrayBuffer[Int] = { val buf = new ArrayBuffer[Int] - for (rowIdx <- 0 until numRows) { + var rowIdx = 0 + while (rowIdx < numRows) { // invoke PoissonSampler sample val rowCount = super.sample() if (rowCount > 0) { - numOutputRows += rowCount - for (_ <- 0 until rowCount) { + var i = 0 + while (i < rowCount) { buf += rowIdx + i = i + 1 } } + rowIdx += 1 } buf } 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 a349c875965..86de80c5703 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 @@ -144,17 +144,8 @@ abstract class CudfBinaryArithmetic extends CudfBinaryOperator with NullIntolera override def dataType: DataType = left.dataType } -case class GpuAdd( - left: Expression, - right: Expression, - failOnError: Boolean) extends CudfBinaryArithmetic { - override def inputType: AbstractDataType = TypeCollection.NumericAndInterval - - override def symbol: String = "+" - - override def binaryOp: BinaryOp = BinaryOp.ADD - - private[this] def basicOpOverflowCheck( +object GpuAdd extends Arm { + def basicOpOverflowCheck( lhs: BinaryOperable, rhs: BinaryOperable, ret: ColumnVector): Unit = { @@ -179,7 +170,7 @@ case class GpuAdd( } } - private[this] def decimalOpOverflowCheck( + def didDecimalOverflow( lhs: BinaryOperable, rhs: BinaryOperable, ret: ColumnVector): ColumnVector = { @@ -189,7 +180,7 @@ case class GpuAdd( // the result val numRows = ret.getRowCount.toInt val zero = BigDecimal(0) - val overflow = withResource(DecimalUtil.lessThan(rhs, zero, numRows)) { rhsLz => + withResource(DecimalUtil.lessThan(rhs, zero, numRows)) { rhsLz => val argsSignSame = withResource(DecimalUtil.lessThan(lhs, zero, numRows)) { lhsLz => lhsLz.equalTo(rhsLz) } @@ -203,7 +194,14 @@ case class GpuAdd( } } } - withResource(overflow) { overflow => + } + + def decimalOpOverflowCheck( + lhs: BinaryOperable, + rhs: BinaryOperable, + ret: ColumnVector, + failOnError: Boolean): ColumnVector = { + withResource(didDecimalOverflow(lhs, rhs, ret)) { overflow => if (failOnError) { withResource(overflow.any()) { any => if (any.isValid && any.getBoolean) { @@ -212,23 +210,34 @@ case class GpuAdd( } ret.incRefCount() } else { - withResource(GpuScalar.from(null, dataType)) { nullVal => + withResource(Scalar.fromNull(ret.getType)) { nullVal => overflow.ifElse(nullVal, ret) } } } } +} + +case class GpuAdd( + left: Expression, + right: Expression, + failOnError: Boolean) extends CudfBinaryArithmetic { + override def inputType: AbstractDataType = TypeCollection.NumericAndInterval + + override def symbol: String = "+" + + override def binaryOp: BinaryOp = BinaryOp.ADD override def doColumnar(lhs: BinaryOperable, rhs: BinaryOperable): ColumnVector = { val ret = super.doColumnar(lhs, rhs) withResource(ret) { ret => // No shims are needed, because it actually supports ANSI mode from Spark v3.0.1. if (failOnError && GpuAnsi.needBasicOpOverflowCheck(dataType)) { - basicOpOverflowCheck(lhs, rhs, ret) + GpuAdd.basicOpOverflowCheck(lhs, rhs, ret) } if (dataType.isInstanceOf[DecimalType]) { - decimalOpOverflowCheck(lhs, rhs, ret) + GpuAdd.decimalOpOverflowCheck(lhs, rhs, ret, failOnError) } else { ret.incRefCount() } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala index 3062aeccf51..61ac28cb9b1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf.{ColumnVector, DType} -import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuExpressionsUtils} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuExpressionsUtils, GpuMapUtils} import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq import com.nvidia.spark.rapids.shims.v2.ShimExpression @@ -80,7 +80,10 @@ case class GpuCreateArray(children: Seq[Expression], useStringTypeWhenEmpty: Boo } } -case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boolean) +case class GpuCreateMap( + children: Seq[Expression], + useStringTypeWhenEmpty: Boolean, + isExceptionDedupePolicy: Boolean) extends GpuExpression with ShimExpression { private val valueIndices: Seq[Int] = children.indices.filter(_ % 2 != 0) @@ -106,7 +109,22 @@ case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boole val structs = Range(0, columns.length, 2) .safeMap(i => ColumnVector.makeStruct(columns(i), columns(i + 1))) withResource(structs) { _ => - GpuColumnVector.from(ColumnVector.makeList(numRows, DType.STRUCT, structs: _*), dataType) + withResource(ColumnVector.makeList(numRows, DType.STRUCT, structs: _*)) { listOfStruct => + withResource(listOfStruct.dropListDuplicatesWithKeysValues()) { deduped => + if (isExceptionDedupePolicy) { + // compare child data row count before and after + // removing duplicates to determine if there were duplicates + withResource(deduped.getChildColumnView(0)) { a => + withResource(listOfStruct.getChildColumnView(0)) { b => + if (a.getRowCount != b.getRowCount) { + throw GpuMapUtils.duplicateMapKeyFoundError + } + } + } + } + GpuColumnVector.from(deduped.incRefCount(), dataType) + } + } } } } @@ -128,7 +146,9 @@ case class GpuCreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boole object GpuCreateMap { def apply(children: Seq[Expression]): GpuCreateMap = { new GpuCreateMap(children, - SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE)) + SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE), + SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY) == + SQLConf.MapKeyDedupPolicy.EXCEPTION.toString) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index ceede1fd0cc..e697d41ac34 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -303,28 +303,36 @@ abstract class GpuBroadcastExchangeExecBase( // Setup a job group here so later it may get cancelled by groupId if necessary. sparkContext.setJobGroup(_runId.toString, s"broadcast exchange (runId ${_runId})", interruptOnCancel = true) - val batch = withResource(new NvtxWithMetrics("broadcast collect", NvtxColor.GREEN, - collectTime)) { _ => - val data = child.executeColumnar().map(cb => try { - new SerializeBatchDeserializeHostBuffer(cb) - } finally { - cb.close() - }) - val d = data.collect() - new SerializeConcatHostBuffersDeserializeBatch(d, output) - } - - val numRows = batch.numRows - checkRowLimit(numRows) - numOutputBatches += 1 - numOutputRows += numRows - + var dataSize = 0L + val broadcastResult = + withResource(new NvtxWithMetrics("broadcast collect", NvtxColor.GREEN, + collectTime)) { _ => + val childRdd = child.executeColumnar() + val data = childRdd.map(cb => try { + new SerializeBatchDeserializeHostBuffer(cb) + } finally { + cb.close() + }) + + val d = data.collect() + val emptyRelation: Option[Any] = if (d.isEmpty) { + ShimLoader.getSparkShims.tryTransformIfEmptyRelation(mode) + } else { + None + } + + emptyRelation.getOrElse({ + val batch = new SerializeConcatHostBuffersDeserializeBatch(d, output) + val numRows = batch.numRows + checkRowLimit(numRows) + numOutputBatches += 1 + numOutputRows += numRows + dataSize += batch.dataSize + batch + }) + } withResource(new NvtxWithMetrics("broadcast build", NvtxColor.DARK_GREEN, buildTime)) { _ => - // we only support hashjoin so this is a noop - // val relation = mode.transform(input, Some(numRows)) - val dataSize = batch.dataSize - gpuLongMetric("dataSize") += dataSize if (dataSize >= MAX_BROADCAST_TABLE_BYTES) { throw new SparkException( @@ -335,7 +343,7 @@ abstract class GpuBroadcastExchangeExecBase( val broadcasted = withResource(new NvtxWithMetrics("broadcast", NvtxColor.CYAN, broadcastTime)) { _ => // Broadcast the relation - sparkContext.broadcast(batch.asInstanceOf[Any]) + sparkContext.broadcast(broadcastResult) } SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala new file mode 100644 index 00000000000..47b9c1cada2 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala @@ -0,0 +1,76 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{GpuColumnVector, ShimLoader} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +object GpuBroadcastHelper { + /** + * Given a broadcast relation get a ColumnarBatch that can be used on the GPU. + * + * The broadcast relation may or may not contain any data, so we special case + * the empty relation case (hash or identity depending on the type of join). + * + * If a broadcast result is unexpected we throw, but at this moment other + * cases are not known, so this is a defensive measure. + * + * @param broadcastRelation - the broadcast as produced by a broadcast exchange + * @param broadcastSchema - the broadcast schema + * @return a `ColumnarBatch` or throw if the broadcast can't be handled + */ + def getBroadcastBatch(broadcastRelation: Broadcast[Any], + broadcastSchema: StructType): ColumnarBatch = { + broadcastRelation.value match { + case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => + val builtBatch = broadcastBatch.batch + GpuColumnVector.incRefCounts(builtBatch) + builtBatch + case v if ShimLoader.getSparkShims.isEmptyRelation(v) => + GpuColumnVector.emptyBatch(broadcastSchema) + case t => + throw new IllegalStateException(s"Invalid broadcast batch received $t") + } + } + + /** + * Given a broadcast relation get the number of rows that the received batch + * contains + * + * The broadcast relation may or may not contain any data, so we special case + * the empty relation case (hash or identity depending on the type of join). + * + * If a broadcast result is unexpected we throw, but at this moment other + * cases are not known, so this is a defensive measure. + * + * @param broadcastRelation - the broadcast as produced by a broadcast exchange + * @return number of rows for a batch received, or 0 if it's an empty relation + */ + def getBroadcastBatchNumRows(broadcastRelation: Broadcast[Any]): Int = { + broadcastRelation.value match { + case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => + broadcastBatch.batch.numRows() + case v if ShimLoader.getSparkShims.isEmptyRelation(v) => 0 + case t => + throw new IllegalStateException(s"Invalid broadcast batch received $t") + } + } +} + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala index dac5429f07f..e13c69479c1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala @@ -435,23 +435,23 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private[this] def makeBuiltBatch( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], buildTime: GpuMetric, buildDataSize: GpuMetric): ColumnarBatch = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => - val ret = broadcastRelation.value.batch - buildDataSize += GpuColumnVector.getTotalDeviceMemoryUsed(ret) - GpuColumnVector.incRefCounts(ret) + val builtBatch = GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, broadcast.schema) + buildDataSize += GpuColumnVector.getTotalDeviceMemoryUsed(builtBatch) + builtBatch } } private[this] def computeBuildRowCount( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], buildTime: GpuMetric, buildDataSize: GpuMetric): Int = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => buildDataSize += 0 - broadcastRelation.value.batch.numRows() + GpuBroadcastHelper.getBroadcastBatchNumRows(broadcastRelation) } } @@ -468,7 +468,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } val broadcastRelation = - broadcastExchange.executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + broadcastExchange.executeColumnarBroadcast[Any]() val joinCondition = boundCondition.orElse { // For outer joins use a true condition if there are any columns in the build side @@ -489,7 +489,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private def leftExistenceJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], exists: Boolean, buildTime: GpuMetric, buildDataSize: GpuMetric): RDD[ColumnarBatch] = { @@ -504,9 +504,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } } - private def doUnconditionalJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch] - ): RDD[ColumnarBatch] = { + private def doUnconditionalJoin(broadcastRelation: Broadcast[Any]): RDD[ColumnarBatch] = { if (output.isEmpty) { doUnconditionalJoinRowCount(broadcastRelation) } else { @@ -565,9 +563,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } /** Special-case handling of an unconditional join that just needs to output a row count. */ - private def doUnconditionalJoinRowCount( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch] - ): RDD[ColumnarBatch] = { + private def doUnconditionalJoinRowCount(broadcastRelation: Broadcast[Any]): RDD[ColumnarBatch] = { if (joinType == LeftAnti) { // degenerate case, no rows are returned. left.executeColumnar().mapPartitions { _ => @@ -604,13 +600,13 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } private def doConditionalJoin( - broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], + broadcastRelation: Broadcast[Any], boundCondition: Option[GpuExpression], numFirstTableColumns: Int): RDD[ColumnarBatch] = { val buildTime = gpuLongMetric(BUILD_TIME) val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) + val spillCallback = GpuMetric.makeSpillCallback(allMetrics) lazy val builtBatch = makeBuiltBatch(broadcastRelation, buildTime, buildDataSize) - val streamAttributes = streamed.output val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) @@ -619,7 +615,6 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val nestedLoopJoinType = joinType val buildSide = getGpuBuildSide - val spillCallback = GpuMetric.makeSpillCallback(allMetrics) streamed.executeColumnar().mapPartitions { streamedIter => val lazyStream = streamedIter.map { cb => withResource(cb) { cb => diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala index 8c3af1b23b3..3a51b5d89a1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRddConverter.scala @@ -16,6 +16,7 @@ package org.apache.spark.sql.rapids.execution +import ai.rapids.cudf.HostColumnVector.ColumnBuilder import ai.rapids.cudf.Table import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder @@ -25,8 +26,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.{MapPartitionsRDD, RDD} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.execution.SQLExecutionRDD -import org.apache.spark.sql.rapids.execution.GpuExternalRowToColumnConverter.{FixedWidthTypeConverter, VariableWidthTypeConverter} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -39,15 +40,10 @@ private class GpuExternalRowToColumnConverter(schema: StructType) extends Serial f => GpuExternalRowToColumnConverter.getConverterForType(f.dataType, f.nullable) } - final def convert(row: Row, builders: GpuColumnarBatchBuilder): Long = { - var bytes: Long = 0 + final def convert(row: Row, builders: GpuColumnarBatchBuilder): Double = { + var bytes: Double = 0 for (idx <- 0 until row.length) { - converters(idx) match { - case tc: FixedWidthTypeConverter => - tc.append(row, idx, builders.builder(idx)) - case tc: VariableWidthTypeConverter => - bytes += tc.append(row, idx, builders.builder(idx)) - } + bytes += converters(idx).append(row, idx, builders.builder(idx)) } bytes } @@ -55,18 +51,31 @@ private class GpuExternalRowToColumnConverter(schema: StructType) extends Serial private object GpuExternalRowToColumnConverter { - private trait TypeConverter extends Serializable - - private abstract class FixedWidthTypeConverter extends TypeConverter { - /** Append row value to the column builder */ - def append(row: Row, column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit - } + // Sizes estimates for different things + /* + * size of an offset entry. In general we have 1 more offset entry than rows, so + * we might be off by one entry per column. + */ + private[this] val OFFSET = Integer.BYTES + private[this] val VALIDITY = 0.125 // 1/8th of a byte (1 bit) + private[this] val VALIDITY_N_OFFSET = OFFSET + VALIDITY - private abstract class VariableWidthTypeConverter extends TypeConverter { + private abstract class TypeConverter extends Serializable { /** Append row value to the column builder and return the number of data bytes written */ - def append(row: Row, column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Long + def append(row: Row, column: Int, builder: ColumnBuilder): Double + + /** + * This is here for structs. When you append a null to a struct the size is not known + * ahead of time. Also because structs push nulls down to the children this size should + * assume a validity even if the schema says it cannot be null. + */ + def getNullSize: Double } + private def getConverterFor(field: StructField): TypeConverter = + getConverterForType(field.dataType, field.nullable) + + private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = { (dataType, nullable) match { case (BooleanType, true) => BooleanConverter @@ -89,181 +98,329 @@ private object GpuExternalRowToColumnConverter { case (TimestampType, false) => NotNullLongConverter case (StringType, true) => StringConverter case (StringType, false) => NotNullStringConverter - // NOT SUPPORTED YET case CalendarIntervalType => CalendarConverter - // NOT SUPPORTED YET case at: ArrayType => new ArrayConverter( - // getConverterForType(at.elementType)) - // NOT SUPPORTED YET case st: StructType => new StructConverter(st.fields.map( - // (f) => getConverterForType(f.dataType))) - // NOT SUPPORTED YET case dt: DecimalType => new DecimalConverter(dt) - // NOT SUPPORTED YET case mt: MapType => new MapConverter(getConverterForType(mt.keyType), - // getConverterForType(mt.valueType)) + case (BinaryType, true) => BinaryConverter + case (BinaryType, false) => NotNullBinaryConverter + // NOT SUPPORTED YET + // case CalendarIntervalType => CalendarConverter + case (at: ArrayType, true) => + ArrayConverter(getConverterForType(at.elementType, at.containsNull)) + case (at: ArrayType, false) => + NotNullArrayConverter(getConverterForType(at.elementType, at.containsNull)) + case (st: StructType, true) => + StructConverter(st.fields.map(getConverterFor)) + case (st: StructType, false) => + NotNullStructConverter(st.fields.map(getConverterFor)) + case (dt: DecimalType, true) => + new DecimalConverter(dt.precision, dt.scale) + case (dt: DecimalType, false) => + new NotNullDecimalConverter(dt.precision, dt.scale) + case (MapType(k, v, vcn), true) => + MapConverter(getConverterForType(k, nullable = false), + getConverterForType(v, vcn)) + case (MapType(k, v, vcn), false) => + NotNullMapConverter(getConverterForType(k, nullable = false), + getConverterForType(v, vcn)) + case (NullType, true) => + NullConverter case (unknown, _) => throw new UnsupportedOperationException( s"Type $unknown not supported") } } + private object NullConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + builder.appendNull() + 1 + VALIDITY + } - private object BooleanConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + override def getNullSize: Double = 1 + VALIDITY + } + + private object BooleanConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullBooleanConverter.append(row, column, builder) } + 1 + VALIDITY + } + + override def getNullSize: Double = 1 + VALIDITY } - private object NotNullBooleanConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullBooleanConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(if (row.getBoolean(column)) 1.toByte else 0.toByte) + 1 + } + + override def getNullSize: Double = 1 + VALIDITY } - private object ByteConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object ByteConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullByteConverter.append(row, column, builder) } + 1 + VALIDITY + } + + override def getNullSize: Double = 1 + VALIDITY } - private object NotNullByteConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullByteConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getByte(column)) + 1 + } + + override def getNullSize: Double = 1 + VALIDITY } - private object ShortConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object ShortConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullShortConverter.append(row, column, builder) } + 2 + VALIDITY + } + + override def getNullSize: Double = 2 + VALIDITY } - private object NotNullShortConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullShortConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getShort(column)) + 2 + } + + override def getNullSize: Double = 2 + VALIDITY } - private object IntConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object IntConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullIntConverter.append(row, column, builder) } + 4 + VALIDITY + } + + override def getNullSize: Double = 4 + VALIDITY } - private object NotNullIntConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullIntConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getInt(column)) + 4 + } + + override def getNullSize: Double = 4 + VALIDITY } - private object FloatConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object FloatConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullFloatConverter.append(row, column, builder) } + 4 + VALIDITY + } + + override def getNullSize: Double = 4 + VALIDITY } - private object NotNullFloatConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullFloatConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getFloat(column)) + 4 + } + + override def getNullSize: Double = 4 + VALIDITY } - private object LongConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object LongConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullLongConverter.append(row, column, builder) } + 8 + VALIDITY + } + + override def getNullSize: Double = 8 + VALIDITY } - private object NotNullLongConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullLongConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getLong(column)) + 8 + } + + override def getNullSize: Double = 8 + VALIDITY } - private object DoubleConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object DoubleConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { if (row.isNullAt(column)) { builder.appendNull() } else { NotNullDoubleConverter.append(row, column, builder) } + 8 + VALIDITY + } + + override def getNullSize: Double = 8 + VALIDITY } - private object NotNullDoubleConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object NotNullDoubleConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { builder.append(row.getDouble(column)) + 8 + } + + override def getNullSize: Double = 8 + VALIDITY } - private object StringConverter extends FixedWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Unit = + private object StringConverter extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = if (row.isNullAt(column)) { builder.appendNull() + VALIDITY_N_OFFSET } else { - NotNullStringConverter.append(row, column, builder) + NotNullStringConverter.append(row, column, builder) + VALIDITY } + + override def getNullSize: Double = VALIDITY_N_OFFSET } - private object NotNullStringConverter extends VariableWidthTypeConverter { - override def append( - row: Row, - column: Int, - builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Long = { + private object NotNullStringConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { val bytes = row.getString(column).getBytes builder.appendUTF8String(bytes) - bytes.length + bytes.length + OFFSET + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } +// + private object BinaryConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + NotNullBinaryConverter.append(row, column, builder) + VALIDITY + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private object NotNullBinaryConverter extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + val child = builder.getChild(0) + val bytes = row.asInstanceOf[GenericRow].getSeq[Byte](column) + bytes.foreach(child.append) + builder.endList() + bytes.length + OFFSET + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private[this] def mapConvert( + keyConverter: TypeConverter, + valueConverter: TypeConverter, + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val m = row.getMap[Any, Any](column) + val numElements = m.size + val srcKeys = m.keys.toArray + val srcValues = m.values.toArray + val structBuilder = builder.getChild(0) + val keyBuilder = structBuilder.getChild(0) + val valueBuilder = structBuilder.getChild(1) + for (i <- 0 until numElements) { + ret += keyConverter.append(Row(srcKeys: _*), i, keyBuilder) + ret += valueConverter.append(Row(srcValues: _*), i, valueBuilder) + structBuilder.endStruct() } + builder.endList() + ret + OFFSET + } + + private case class MapConverter( + keyConverter: TypeConverter, + valueConverter: TypeConverter) extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + mapConvert(keyConverter, valueConverter, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private case class NotNullMapConverter( + keyConverter: TypeConverter, + valueConverter: TypeConverter) extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = + mapConvert(keyConverter, valueConverter, row, column, builder) + + override def getNullSize: Double = VALIDITY_N_OFFSET } + // // private object CalendarConverter extends FixedWidthTypeConverter { // override def append( @@ -280,90 +437,125 @@ private object GpuExternalRowToColumnConverter { // } // } // } - // - // private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val values = row.getArray(column) - // val numElements = values.numElements() - // cv.appendArray(numElements) - // val arrData = cv.arrayData() - // for (i <- 0 until numElements) { - // childConverter.append(values, i, arrData) - // } - // } - // } - // } - // - // private case class StructConverter(childConverters: Array[TypeConverter]) - // extends TypeConverter { - // override def append(row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // cv.appendStruct(false) - // val data = row.getStruct(column, childConverters.length) - // for (i <- 0 until childConverters.length) { - // childConverters(i).append(data, i, cv.getChild(i)) - // } - // } - // } - // } - // - // private case class DecimalConverter(dt: DecimalType) extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val d = row.getDecimal(column, dt.precision, dt.scale) - // if (dt.precision <= Decimal.MAX_INT_DIGITS) { - // cv.appendInt(d.toUnscaledLong.toInt) - // } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) { - // cv.appendLong(d.toUnscaledLong) - // } else { - // val integer = d.toJavaBigDecimal.unscaledValue - // val bytes = integer.toByteArray - // cv.appendByteArray(bytes, 0, bytes.length) - // } - // } - // } - // } - // - // private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter) - // extends TypeConverter { - // override def append( - // row: SpecializedGetters, - // column: Int, - // builder: ai.rapids.cudf.HostColumnVector.Builder): Unit = { - // if (row.isNullAt(column)) { - // builder.appendNull() - // } else { - // val m = row.getMap(column) - // val keys = cv.getChild(0) - // val values = cv.getChild(1) - // val numElements = m.numElements() - // cv.appendArray(numElements) - // - // val srcKeys = m.keyArray() - // val srcValues = m.valueArray() - // - // for (i <- 0 until numElements) { - // keyConverter.append(srcKeys, i, keys) - // valueConverter.append(srcValues, i, values) - // } - // } - // } - // } + + private[this] def arrayConvert( + childConverter: TypeConverter, + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val values = row.getSeq(column) + val numElements = values.size + val child = builder.getChild(0) + for (i <- 0 until numElements) { + ret += childConverter.append(Row(values: _*), i, child) + } + builder.endList() + ret + OFFSET + } + + private case class ArrayConverter(childConverter: TypeConverter) + extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + VALIDITY_N_OFFSET + } else { + arrayConvert(childConverter, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private case class NotNullArrayConverter(childConverter: TypeConverter) + extends TypeConverter { + override def append(row: Row, + column: Int, builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + arrayConvert(childConverter, row, column, builder) + } + + override def getNullSize: Double = VALIDITY_N_OFFSET + } + + private[this] def structConvert( + childConverters: Array[TypeConverter], + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder) : Double = { + var ret = 0.0 + val struct = row.getStruct(column) + for (i <- childConverters.indices) { + ret += childConverters(i).append(struct, i, builder.getChild(i)) + } + builder.endStruct() + ret + } + + private case class StructConverter( + childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + childConverters.map(_.getNullSize).sum + VALIDITY + // each child has to insert a null too, which is dependent on the child + } else { + structConvert(childConverters, row, column, builder) + VALIDITY + } + } + + override def getNullSize: Double = childConverters.map(_.getNullSize).sum + VALIDITY + } + + private case class NotNullStructConverter( + childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + structConvert(childConverters, row, column, builder) + } + + override def getNullSize: Double = childConverters.map(_.getNullSize).sum + VALIDITY + } + + private class DecimalConverter( + precision: Int, scale: Int) extends NotNullDecimalConverter(precision, scale) { + private val appendedSize = DecimalUtil.createCudfDecimal(precision, scale).getSizeInBytes + + VALIDITY + + override def append( + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + if (row.isNullAt(column)) { + builder.appendNull() + } else { + super.append(row, column, builder) + } + appendedSize + } + } + + private class NotNullDecimalConverter(precision: Int, scale: Int) extends TypeConverter { + private val appendedSize = DecimalUtil.createCudfDecimal(precision, scale).getSizeInBytes + + VALIDITY + + override def append( + row: Row, + column: Int, + builder: ai.rapids.cudf.HostColumnVector.ColumnBuilder): Double = { + val bigDecimal = row.getDecimal(column) + builder.append(bigDecimal) + appendedSize + } + + override def getNullSize: Double = { + appendedSize + VALIDITY + } + } } private class ExternalRowToColumnarIterator( @@ -372,11 +564,6 @@ private class ExternalRowToColumnarIterator( localGoal: CoalesceSizeGoal, converters: GpuExternalRowToColumnConverter) extends Iterator[ColumnarBatch] { - private val dataTypes: Array[DataType] = localSchema.fields.map(_.dataType) - private val variableWidthColumnCount = dataTypes.count(dt => !GpuBatchUtils.isFixedWidth(dt)) - private val fixedWidthDataSizePerRow = dataTypes.filter(GpuBatchUtils.isFixedWidth) - .map(_.defaultSize).sum - private val nullableColumns = localSchema.fields.count(_.nullable) private val targetSizeBytes = localGoal.targetSizeBytes private var targetRows = 0 private var totalOutputBytes: Long = 0 @@ -407,16 +594,11 @@ private class ExternalRowToColumnarIterator( val builders = new GpuColumnarBatchBuilder(localSchema, targetRows) try { var rowCount = 0 - var byteCount: Long = variableWidthColumnCount * 4 // offset bytes + // Double because validity can be < 1 byte, and this is just an estimate anyways + var byteCount: Double = 0 while (rowCount < targetRows && byteCount < targetSizeBytes && rowIter.hasNext) { val row = rowIter.next() - val variableWidthDataBytes = converters.convert(row, builders) - byteCount += fixedWidthDataSizePerRow // fixed-width data bytes - byteCount += variableWidthDataBytes // variable-width data bytes - byteCount += variableWidthColumnCount * GpuBatchUtils.OFFSET_BYTES // offset bytes - if (nullableColumns > 0 && rowCount % GpuBatchUtils.VALIDITY_BUFFER_BOUNDARY_ROWS == 0) { - byteCount += GpuBatchUtils.VALIDITY_BUFFER_BOUNDARY_BYTES * nullableColumns - } + byteCount += converters.convert(row, builders) rowCount += 1 } @@ -472,10 +654,13 @@ object InternalColumnarRddConverter extends Logging { convert(df) } - def convert(df: DataFrame): RDD[Table] = { + // Extract RDD[ColumnarBatch] directly + def extractRDDColumnarBatch(df: DataFrame): (Option[RDD[ColumnarBatch]], RDD[Row]) = { val schema = df.schema - if (!GpuOverrides.areAllSupportedTypes(schema.map(_.dataType) :_*)) { - val unsupported = schema.map(_.dataType).filter(!GpuOverrides.isSupportedType(_)).toSet + val unsupported = schema.map(_.dataType).filter( dt => !GpuOverrides.isSupportedType(dt, + allowMaps = true, allowStringMaps = true, allowNull = true, allowStruct = true, allowArray + = true, allowBinary = true, allowDecimal = true, allowNesting = true)).toSet + if (unsupported.nonEmpty) { throw new IllegalArgumentException(s"Cannot convert $df to GPU columnar $unsupported are " + s"not currently supported data types for columnar.") } @@ -525,7 +710,12 @@ object InternalColumnarRddConverter extends Logging { logDebug(s"Cannot extract columnar RDD directly. " + s"(First MapPartitionsRDD not found $rdd)") } + (batch, input) + } + def convert(df: DataFrame): RDD[Table] = { + val schema = df.schema + val (batch, input) = extractRDDColumnarBatch(df) val b = batch.getOrElse({ // We have to fall back to doing a slow transition. val converters = new GpuExternalRowToColumnConverter(schema) 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 6a14f74b561..692be261f6a 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 @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.v2.ShimExpression -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ImplicitCastInputTypes, Literal, NullIntolerant, Predicate, RLike, StringSplit, SubstringIndex} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ImplicitCastInputTypes, Literal, NullIntolerant, Predicate, RegExpExtract, RLike, StringSplit, SubstringIndex} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String @@ -801,24 +801,9 @@ case class GpuRLike(left: Expression, right: Expression, pattern: String) override def dataType: DataType = BooleanType } -case class GpuRegExpReplace( - srcExpr: Expression, - searchExpr: Expression, - replaceExpr: Expression, - cudfRegexPattern: String) - extends GpuTernaryExpression with ImplicitCastInputTypes { +abstract class GpuRegExpTernaryBase extends GpuTernaryExpression { - override def dataType: DataType = srcExpr.dataType - - override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) - - override def first: Expression = srcExpr - override def second: Expression = searchExpr - override def third: Expression = replaceExpr - - def this(srcExpr: Expression, searchExpr: Expression, cudfRegexPattern: String) = { - this(srcExpr, searchExpr, GpuLiteral("", StringType), cudfRegexPattern) - } + override def dataType: DataType = StringType override def doColumnar( strExpr: GpuColumnVector, @@ -852,23 +837,171 @@ case class GpuRegExpReplace( override def doColumnar( strExpr: GpuColumnVector, - searchExpr: GpuScalar, - replaceExpr: GpuScalar): ColumnVector = { - strExpr.getBase.replaceRegex(cudfRegexPattern, replaceExpr.getBase) - } + searchExpr: GpuColumnVector, + replaceExpr: GpuScalar): ColumnVector = + throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") override def doColumnar(numRows: Int, val0: GpuScalar, val1: GpuScalar, val2: GpuScalar): ColumnVector = { - withResource(GpuColumnVector.from(val0, numRows, srcExpr.dataType)) { val0Col => + withResource(GpuColumnVector.from(val0, numRows, first.dataType)) { val0Col => doColumnar(val0Col, val1, val2) } } +} + +case class GpuRegExpReplace( + srcExpr: Expression, + searchExpr: Expression, + replaceExpr: Expression, + cudfRegexPattern: String) + extends GpuRegExpTernaryBase with ImplicitCastInputTypes { + + override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) + + override def first: Expression = srcExpr + override def second: Expression = searchExpr + override def third: Expression = replaceExpr + + def this(srcExpr: Expression, searchExpr: Expression, cudfRegexPattern: String) = { + this(srcExpr, searchExpr, GpuLiteral("", StringType), cudfRegexPattern) + } + override def doColumnar( strExpr: GpuColumnVector, - searchExpr: GpuColumnVector, - replaceExpr: GpuScalar): ColumnVector = - throw new UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") + searchExpr: GpuScalar, + replaceExpr: GpuScalar): ColumnVector = { + strExpr.getBase.replaceRegex(cudfRegexPattern, replaceExpr.getBase) + } + +} + +class GpuRegExpExtractMeta( + expr: RegExpExtract, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends TernaryExprMeta[RegExpExtract](expr, conf, parent, rule) { + + private var pattern: Option[String] = None + private var numGroups = 0 + + override def tagExprForGpu(): Unit = { + + def countGroups(regexp: RegexAST): Int = { + regexp match { + case RegexGroup(_, term) => 1 + countGroups(term) + case other => other.children().map(countGroups).sum + } + } + + expr.regexp match { + case Literal(str: UTF8String, DataTypes.StringType) if str != null => + try { + val javaRegexpPattern = str.toString + // verify that we support this regex and can transpile it to cuDF format + val cudfRegexPattern = new CudfRegexTranspiler(replace = false) + .transpile(javaRegexpPattern) + pattern = Some(cudfRegexPattern) + numGroups = countGroups(new RegexParser(javaRegexpPattern).parse()) + } catch { + case e: RegexUnsupportedException => + willNotWorkOnGpu(e.getMessage) + } + case _ => + willNotWorkOnGpu(s"only non-null literal strings are supported on GPU") + } + + expr.idx match { + case Literal(value, DataTypes.IntegerType) => + val idx = value.asInstanceOf[Int] + if (idx < 0) { + willNotWorkOnGpu("the specified group index cannot be less than zero") + } + if (idx > numGroups) { + willNotWorkOnGpu( + s"regex group count is $numGroups, but the specified group index is $idx") + } + case _ => + willNotWorkOnGpu("GPU only supports literal index") + } + } + + override def convertToGpu( + str: Expression, + regexp: Expression, + idx: Expression): GpuExpression = { + val cudfPattern = pattern.getOrElse( + throw new IllegalStateException("Expression has not been tagged with cuDF regex pattern")) + GpuRegExpExtract(str, regexp, idx, cudfPattern) + } +} + +case class GpuRegExpExtract( + subject: Expression, + regexp: Expression, + idx: Expression, + cudfRegexPattern: String) + extends GpuRegExpTernaryBase with ImplicitCastInputTypes with NullIntolerant { + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def first: Expression = subject + override def second: Expression = regexp + override def third: Expression = idx + + override def prettyName: String = "regexp_extract" + + override def doColumnar( + str: GpuColumnVector, + regexp: GpuScalar, + idx: GpuScalar): ColumnVector = { + + val groupIndex = idx.getValue.asInstanceOf[Int] + + // There are some differences in behavior between cuDF and Java so we have + // to handle those cases here. + // + // Given the pattern `^([a-z]*)([0-9]*)([a-z]*)$` the following table + // shows the value that would be extracted for group index 2 given a range + // of inputs. The behavior is mostly consistent except for the case where + // the input is non-null and does not match the pattern. + // + // | Input | Java | cuDF | + // |--------|-------|-------| + // | '' | '' | '' | + // | NULL | NULL | NULL | + // | 'a1a' | '1' | '1' | + // | '1a1' | '' | NULL | + + if (groupIndex == 0) { + withResource(GpuScalar.from("", DataTypes.StringType)) { emptyString => + withResource(GpuScalar.from(null, DataTypes.StringType)) { nullString => + withResource(str.getBase.matchesRe(cudfRegexPattern)) { matches => + withResource(str.getBase.isNull) { isNull => + withResource(matches.ifElse(str.getBase, emptyString)) { + isNull.ifElse(nullString, _) + } + } + } + } + } + } else { + withResource(GpuScalar.from("", DataTypes.StringType)) { emptyString => + withResource(GpuScalar.from(null, DataTypes.StringType)) { nullString => + withResource(str.getBase.extractRe(cudfRegexPattern)) { extract => + withResource(str.getBase.matchesRe(cudfRegexPattern)) { matches => + withResource(str.getBase.isNull) { isNull => + withResource(matches.ifElse(extract.getColumn(groupIndex - 1), emptyString)) { + isNull.ifElse(nullString, _) + } + } + } + } + } + } + } + } + } class SubstringIndexMeta( diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala index b920b201a5d..2710cdcda7a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions.{col, when} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.rapids.execution.GpuCustomShuffleReaderExec import org.apache.spark.sql.types.{ArrayType, DataTypes, DecimalType, IntegerType, StringType, StructField, StructType} @@ -329,14 +330,21 @@ class AdaptiveQueryExecSuite _.isInstanceOf[AdaptiveSparkPlanExec]) .get.asInstanceOf[AdaptiveSparkPlanExec] - val transition = adaptiveSparkPlanExec + if (ShimLoader.getSparkShims.supportsColumnarAdaptivePlans) { + // we avoid the transition entirely with Spark 3.2+ due to the changes in SPARK-35881 to + // support columnar adaptive plans + assert(adaptiveSparkPlanExec .executedPlan - .asInstanceOf[GpuColumnarToRowExec] - - // although the plan contains a GpuColumnarToRowExec, we bypass it in - // AvoidAdaptiveTransitionToRow so the metrics should reflect that - assert(transition.metrics("numOutputRows").value === 0) + .isInstanceOf[GpuFileSourceScanExec]) + } else { + val transition = adaptiveSparkPlanExec + .executedPlan + .asInstanceOf[GpuColumnarToRowExec] + // although the plan contains a GpuColumnarToRowExec, we bypass it in + // AvoidAdaptiveTransitionToRow so the metrics should reflect that + assert(transition.metrics("numOutputRows").value === 0) + } }, conf) } @@ -378,6 +386,31 @@ class AdaptiveQueryExecSuite }, conf) } + // repro case for https://github.com/NVIDIA/spark-rapids/issues/4351 + test("Write parquet from AQE shuffle with limit") { + logError("Write parquet from AQE shuffle with limit") + + val conf = new SparkConf() + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + + withGpuSparkSession(spark => { + import spark.implicits._ + + val path = new File(TEST_FILES_ROOT, "AvoidTransitionInput.parquet").getAbsolutePath + (0 until 100).toDF("a") + .write + .mode(SaveMode.Overwrite) + .parquet(path) + + val outputPath = new File(TEST_FILES_ROOT, "AvoidTransitionOutput.parquet").getAbsolutePath + spark.read.parquet(path) + .limit(100) + .write.mode(SaveMode.Overwrite) + .parquet(outputPath) + }, conf) + } + + test("Exchange reuse") { logError("Exchange reuse") assumeSpark301orLater diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index ad373a2423f..d127b216888 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -932,10 +932,11 @@ class CastOpSuite extends GpuExpressionTestSuite { test("CAST string to float - sanitize step") { val testPairs = Seq( - ("\tinf", "Inf"), - ("\t+InFinITy", "Inf"), - ("\tInFinITy", "Inf"), - ("\t-InFinITy", "-Inf"), + ("\tinf", "inf"), + ("\riNf", "iNf"), + ("\t+InFinITy", "+InFinITy"), + ("\tInFinITy", "InFinITy"), + ("\t-InFinITy", "-InFinITy"), ("\t61f", "61"), (".8E4f", ".8E4") ) @@ -1392,6 +1393,12 @@ object CastOpSuite { def validTimestamps(session: SparkSession): DataFrame = { import session.sqlContext.implicits._ val timestampStrings = Seq( + "8669-07-22T04:45:57.73", + "6233-08-04T19:30:55.701", + "8220-02-25T10:01:15.106", + "9754-01-21T16:53:02.137", + "7649-11-16T15:56:04.996", + "7027-04-09T15:08:52.627", "1920-12-31T11:59:59.999", "1969-12-31T23:59:59.999", "1969-12-31T23:59:59.999999", diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala index 3c8bfda7403..1a8a6579f68 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala @@ -22,9 +22,10 @@ import scala.util.Random import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder import org.scalatest.FunSuite +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.types.{DataTypes, Decimal, DecimalType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericRow} +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String class GpuBatchUtilsSuite extends FunSuite { @@ -197,58 +198,139 @@ object GpuBatchUtilsSuite { rows.toArray } + def createExternalRows(schema: StructType, rowCount: Int): Array[Row] = { + val externalRows = new mutable.ArrayBuffer[Row](rowCount) + val r = new Random(0) + for (i <- 0 until rowCount) { + externalRows.append(new GenericRow(createExternalRowValues(i, r, schema.fields))) + } + externalRows.toArray + } + + private def createValueForType(i: Int, r: Random, dt: DataType, nullable: Boolean): Any = { + dt match { + case DataTypes.BooleanType => maybeNull(nullable, i, r.nextBoolean()) + case DataTypes.ByteType => maybeNull(nullable, i, r.nextInt().toByte) + case DataTypes.ShortType => maybeNull(nullable, i, r.nextInt().toShort) + case DataTypes.IntegerType => maybeNull(nullable, i, r.nextInt()) + case DataTypes.LongType => maybeNull(nullable, i, r.nextLong()) + case DataTypes.FloatType => maybeNull(nullable, i, r.nextFloat()) + case DataTypes.DoubleType => maybeNull(nullable, i, r.nextDouble()) + // Spark use Int to store a Date internally, so use nextInt to avoid + // 1). create Date object 2). convert Date to EpochDays int value + case DataTypes.DateType => maybeNull(nullable, i, r.nextInt()) + // Spark use Long to store a Timestamp internally, so use nextLong to avoid + // 1). create Timestamp object 2). convert Timestamp to microsecond long value + case DataTypes.TimestampType => maybeNull(nullable, i, r.nextLong()) + case dataType: DecimalType => + val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) + val unScaledValue = r.nextLong() % upperBound + maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + case dataType@DataTypes.StringType => + if (nullable) { + // since we want a deterministic test that compares the estimate with actual + // usage we need to make sure the average length of strings is `dataType.defaultSize` + if (i % 2 == 0) { + null + } else { + createUTF8String(dataType.defaultSize * 2) + } + } else { + createUTF8String(dataType.defaultSize) + } + case dataType@DataTypes.BinaryType => + if (nullable) { + // since we want a deterministic test that compares the estimate with actual usage we + // need to make sure the average length of binary values is `dataType.defaultSize` + if (i % 2 == 0) { + null + } else { + r.nextString(dataType.defaultSize * 2).getBytes + } + } else { + r.nextString(dataType.defaultSize).getBytes + } + case ArrayType(elementType, containsNull) => + if (nullable && i % 2 == 0) { + null + } else { + val arrayValues = new mutable.ArrayBuffer[Any]() + for (_ <- 0 to r.nextInt(10)) { + arrayValues.append(createValueForType(i, r, elementType, containsNull)) + } + arrayValues.toArray.toSeq + } + case MapType(keyType, valueType, valueContainsNull) => + if (nullable && i % 2 == 0) { + null + } else { + // TODO: add other types + val map = mutable.Map[String, String]() + for ( j <- 0 until 10) { + if (valueContainsNull && j % 2 == 0) { + map += (createUTF8String(10).toString -> null) + } else { + map += (createUTF8String(10).toString -> createUTF8String(10).toString) + } + } + map + } + case StructType(fields) => + new GenericRow(fields.map(f => createValueForType(i, r, f.dataType, nullable))) + case unknown => throw new UnsupportedOperationException( + s"Type $unknown not supported") + } + } + + private def createRowValues(i: Int, r: Random, fields: Array[StructField]) = { + val values: Array[Any] = fields.map(field => { + createValueForType(i, r, field.dataType, field.nullable) + }) + values + } + + private def createExternalRowValues(i: Int, r: Random, fields: Array[StructField]): Array[Any] = { val values: Array[Any] = fields.map(field => { field.dataType match { - case DataTypes.BooleanType => maybeNull(field, i, r.nextBoolean()) - case DataTypes.ByteType => maybeNull(field, i, r.nextInt().toByte) - case DataTypes.ShortType => maybeNull(field, i, r.nextInt().toShort) - case DataTypes.IntegerType => maybeNull(field, i, r.nextInt()) - case DataTypes.LongType => maybeNull(field, i, r.nextLong()) - case DataTypes.FloatType => maybeNull(field, i, r.nextFloat()) - case DataTypes.DoubleType => maybeNull(field, i, r.nextDouble()) - case dataType: DecimalType => - val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) - val unScaledValue = r.nextLong() % upperBound - maybeNull(field, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) - case dataType@DataTypes.StringType => - if (field.nullable) { - // since we want a deterministic test that compares the estimate with actual - // usage we need to make sure the average length of strings is `dataType.defaultSize` - if (i % 2 == 0) { - null - } else { - createString(dataType.defaultSize * 2) - } + // Since it's using the createUTF8String method for InternalRow case, need to convert to + // String for Row case. + case StringType => + val utf8StringOrNull = createValueForType(i, r, field.dataType, field.nullable) + if (utf8StringOrNull != null) { + utf8StringOrNull.asInstanceOf[UTF8String].toString } else { - createString(dataType.defaultSize) + utf8StringOrNull } - case dataType@DataTypes.BinaryType => - if (field.nullable) { - // since we want a deterministic test that compares the estimate with actual usage we - // need to make sure the average length of binary values is `dataType.defaultSize` - if (i % 2 == 0) { - null - } else { - r.nextString(dataType.defaultSize * 2).getBytes - } + case BinaryType => + val b = createValueForType(i, r, field.dataType, field.nullable) + if (b != null) { + b.asInstanceOf[Array[Byte]].toSeq + } else { + b + } + case DecimalType() => + val d = createValueForType(i, r, field.dataType, field.nullable) + if (d != null) { + d.asInstanceOf[Decimal].toJavaBigDecimal } else { - r.nextString(dataType.defaultSize).getBytes + d } + case _ => createValueForType(i, r, field.dataType, field.nullable) } }) values } - private def maybeNull(field: StructField, i: Int, value: Any): Any = { - if (field.nullable && i % 2 == 0) { + private def maybeNull(nullable: Boolean, i: Int, value: Any): Any = { + if (nullable && i % 2 == 0) { null } else { value } } - private def createString(size: Int): UTF8String = { + private def createUTF8String(size: Int): UTF8String = { // avoid multi byte characters to keep the test simple val str = (0 until size).map(_ => 'a').mkString UTF8String.fromString(str) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala index aac297eea0a..4836c8f15cb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala @@ -68,12 +68,25 @@ class RegularExpressionParserSuite extends FunSuite { RegexCharacterRange('A', 'Z')))))) } + test("character class complex example") { + assert(parse("[^]+d]+") === RegexSequence(ListBuffer( + RegexRepetition( + RegexCharacterClass(negated = true, + ListBuffer(RegexChar(']'), RegexChar('+'), RegexChar('d'))), + SimpleQuantifier('+'))))) + } + test("character classes containing ']'") { // "[]a]" is a valid character class containing ']' and 'a' assert(parse("[]a]") === RegexSequence(ListBuffer( RegexCharacterClass(negated = false, ListBuffer(RegexChar(']'), RegexChar('a')))))) + // "[^]a]" is a valid negated character class containing ']' and 'a' + assert(parse("[^]a]") === + RegexSequence(ListBuffer( + RegexCharacterClass(negated = true, + ListBuffer(RegexChar(']'), RegexChar('a')))))) // "[a]]" is a valid character class "[a]" followed by character ']' assert(parse("[a]]") === RegexSequence(ListBuffer( diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala similarity index 76% rename from tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala rename to tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala index 6ba3deed5c6..db3a1791317 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/StringFallbackSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionSuite.scala @@ -16,10 +16,13 @@ package com.nvidia.spark.rapids import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, SparkSession} -class StringFallbackSuite extends SparkQueryCompareTestSuite { +class RegularExpressionSuite extends SparkQueryCompareTestSuite { - private val conf = new SparkConf().set("spark.rapids.sql.expression.RegExpReplace", "true") + private val conf = new SparkConf() + .set("spark.rapids.sql.expression.RegExpReplace", "true") + .set("spark.rapids.sql.expression.RegExpExtract", "true") testGpuFallback( "String regexp_replace replace str columnar fall back", @@ -90,4 +93,31 @@ class StringFallbackSuite extends SparkQueryCompareTestSuite { nullableStringsFromCsv, conf = conf) { frame => frame.selectExpr("regexp_replace(strings,'\\(foo\\)','D')") } + + testSparkResultsAreEqual("String regexp_extract regex 1", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract(strings, '^([a-z]*)([0-9]*)([a-z]*)$', 1)") + } + + testSparkResultsAreEqual("String regexp_extract regex 2", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract(strings, '^([a-z]*)([0-9]*)([a-z]*)$', 2)") + } + + testSparkResultsAreEqual("String regexp_extract literal input", + extractStrings, conf = conf) { + frame => frame.selectExpr("regexp_extract('abc123def', '^([a-z]*)([0-9]*)([a-z]*)$', 2)") + } + + private def extractStrings(session: SparkSession): DataFrame = { + import session.sqlContext.implicits._ + Seq[(String)]( + (""), + (null), + ("abc123def"), + ("abc\r\n12\r3\ndef"), + ("123abc456") + ).toDF("strings") + } + } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index 9bd859f6562..73295d02ecb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -15,7 +15,7 @@ */ package com.nvidia.spark.rapids -import java.util.regex.Pattern +import java.util.regex.{Matcher, Pattern} import scala.collection.mutable.ListBuffer import scala.util.{Random, Try} @@ -132,13 +132,16 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { assertUnsupported(pattern, "nothing to repeat")) } - ignore("known issue - multiline difference between CPU and GPU") { - // see https://github.com/rapidsai/cudf/issues/9620 + test("end of line anchor with strings ending in valid newline") { val pattern = "2$" - // this matches "2" but not "2\n" on the GPU assertCpuGpuMatchesRegexpFind(Seq(pattern), Seq("2", "2\n", "2\r", "2\r\n")) } + test("end of line anchor with strings ending in invalid newline") { + val pattern = "2$" + assertCpuGpuMatchesRegexpFind(Seq(pattern), Seq("2\n\r")) + } + test("dot matches CR on GPU but not on CPU") { // see https://github.com/rapidsai/cudf/issues/9619 val pattern = "1." @@ -162,7 +165,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { test("transpile character class unescaped range symbol") { val patterns = Seq("a[-b]", "a[+-]", "a[-+]", "a[-]", "a[^-]") - val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", raw"a[^\-]") + val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", "a(?:[\r\n]|[^\\-])") val transpiler = new CudfRegexTranspiler(replace=false) val transpiled = patterns.map(transpiler.transpile) assert(transpiled === expected) @@ -187,8 +190,10 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { ")" + "$" // end of line - // input and output should be identical - doTranspileTest(VALID_FLOAT_REGEX, VALID_FLOAT_REGEX) + // input and output should be identical except for '$' being replaced with '[\r]?[\n]?$' + doTranspileTest(VALID_FLOAT_REGEX, + VALID_FLOAT_REGEX.replaceAll("\\$", + Matcher.quoteReplacement("[\r]?[\n]?$"))) } test("transpile complex regex 2") { @@ -197,9 +202,11 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { "(.[1-9]*(?:0)?[1-9]+)?(.0*[1-9]+)?(?:.0*)?$" // input and output should be identical except for `.` being replaced with `[^\r\n]` + // and '$' being replaced with '[\r]?[\n]?$' doTranspileTest(TIMESTAMP_TRUNCATE_REGEX, - TIMESTAMP_TRUNCATE_REGEX.replaceAll("\\.", "[^\r\n]")) - + TIMESTAMP_TRUNCATE_REGEX + .replaceAll("\\.", "[^\r\n]") + .replaceAll("\\$", Matcher.quoteReplacement("[\r]?[\n]?$"))) } test("compare CPU and GPU: character range including unescaped + and -") { @@ -241,6 +248,13 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { assertCpuGpuMatchesRegexpReplace(patterns, inputs) } + test("compare CPU and GPU: regexp replace negated character class") { + val inputs = Seq("a", "b", "a\nb", "a\r\nb\n\rc\rd") + val patterns = Seq("[^z]", "[^\r]", "[^\n]", "[^\r]", "[^\r\n]", + "[^a\n]", "[^b\r]", "[^bc\r\n]", "[^\\r\\n]") + assertCpuGpuMatchesRegexpReplace(patterns, inputs) + } + test("compare CPU and GPU: regexp replace fuzz test with limited chars") { // testing with this limited set of characters finds issues much // faster than using the full ASCII set @@ -257,7 +271,6 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { // LF has been excluded due to known issues val chars = (0x00 to 0x7F) .map(_.toChar) - .filterNot(_ == '\n') doFuzzTest(Some(chars.mkString), replace = true) } @@ -272,8 +285,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { options = FuzzerOptions(validChars, maxStringLen = 12)) val data = Range(0, 1000) - // remove trailing newlines as workaround for https://github.com/rapidsai/cudf/issues/9620 - .map(_ => removeTrailingNewlines(r.nextString())) + .map(_ => r.nextString()) // generate patterns that are valid on both CPU and GPU val patterns = ListBuffer[String]() @@ -291,14 +303,6 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { } } - private def removeTrailingNewlines(input: String): String = { - var s = input - while (s.endsWith("\r") || s.endsWith("\n")) { - s = s.substring(0, s.length - 1) - } - s - } - private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]) = { for (javaPattern <- javaPatterns) { val cpu = cpuContains(javaPattern, input) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index f1c4ca1a7d2..d881624c05d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -165,7 +165,7 @@ object RapidsShuffleTestHelper extends MockitoSugar with Arm { } def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { - val rows: Seq[Integer] = (0 until numRows.toInt).map(new Integer(_)) + val rows: Seq[Integer] = (0 until numRows.toInt).map(Int.box) withResource(ColumnVector.fromBoxedInts(rows:_*)) { cvBase => cvBase.incRefCount() val gpuCv = GpuColumnVector.from(cvBase, IntegerType) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala new file mode 100644 index 00000000000..89d3ccb743a --- /dev/null +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala @@ -0,0 +1,288 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.execution + +import scala.collection.mutable + +import com.nvidia.spark.rapids.{ColumnarToRowIterator, GpuBatchUtilsSuite, NoopMetric, SparkQueryCompareTestSuite, TestResourceFinder} +import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.util.MapData +import org.apache.spark.sql.types._ + +class InternalColumnarRDDConverterSuite extends SparkQueryCompareTestSuite { + + def compareMapAndMapDate[K,V](map: collection.Map[K, V], mapData: MapData) = { + assert(map.size == mapData.numElements()) + val outputMap = mutable.Map[Any, Any]() + // Only String now, TODO: support other data types in Map + mapData.foreach(StringType, StringType, f = (k, v) => outputMap += (k.toString -> v.toString)) + val diff = outputMap.toSet diff map.toSet + assert(diff.toMap.isEmpty) + } + + test("transform binary data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Binary", BinaryType), + StructField("BinaryNotNull", BinaryType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getSeq[Byte](0) sameElements output.getBinary(0)) + } + assert(input.getSeq[Byte](1) sameElements output.getBinary(1)) + } + } + } + } + + test("transform boolean, byte, short, int, float, long, double, date, timestamp data" + + " back and forth between Row and Columnar") { + val schema = StructType(Seq( + StructField("Boolean", BooleanType), + StructField("BinaryNotNull", BooleanType, nullable = false), + StructField("Byte", ByteType), + StructField("ByteNotNull",ByteType, nullable = false), + StructField("Short", ShortType), + StructField("ShortNotNull", ShortType, nullable = false), + StructField("Int", IntegerType), + StructField("IntNotNull", IntegerType, nullable = false), + StructField("Float", FloatType), + StructField("FloatNotNull", FloatType, nullable = false), + StructField("Long", LongType), + StructField("LongNotNull", LongType, nullable = false), + StructField("Double", DoubleType), + StructField("DoubleNotNull", DoubleType, nullable = false), + StructField("Date", DateType), + StructField("DateNotNull", DateType, nullable = false), + StructField("Timestamp", TimestampType), + StructField("TimestampNotNull", TimestampType, nullable = false), + StructField("Decimal", DecimalType(20,10)), + StructField("DecimalNotNull", DecimalType(20,10), nullable = false) + )) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getBoolean(0) == output.getBoolean(0)) + } + assert(input.getBoolean(1) == output.getBoolean(1)) + + for ((f, i) <- schema.fields.zipWithIndex) { + if (f.nullable && input.isNullAt(i)) { + assert(output.isNullAt(i)) + } else { + if (f.dataType.isInstanceOf[DecimalType]) { + val l = input.get(i) + val r = output.get(i, f.dataType) + assert(input.get(i) == output.get(i, f.dataType) + .asInstanceOf[Decimal].toJavaBigDecimal) + } else { + assert(input.get(i) == output.get(i, f.dataType)) + } + } + } + } + } + } + } + + test("transform string data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("String", StringType), + StructField("StringNotNull", StringType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getString(0) == output.getString(0)) + } + assert(input.getString(1) == output.getString(1)) + } + } + } + } + + + test("transform byte data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Byte", ByteType), + StructField("ByteNotNull", ByteType, nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getByte(0) == output.getByte(0)) + } + assert(input.getByte(1) == output.getByte(1)) + } + } + } + } + + test("transform array data back and forth between Row and Columnar") { + val schema = StructType(Seq(StructField("Array", ArrayType.apply(DoubleType)), + StructField("ArrayNotNull", ArrayType.apply(DoubleType, false), nullable = false))) + val numRows = 300 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + assert(input.getSeq(0) sameElements output.getArray(0).toDoubleArray()) + } + assert(input.getSeq(1) sameElements output.getArray(1).toDoubleArray()) + } + } + } + } + + test("transform map data back and forth between Row and Columnar") { + val schema = StructType(Seq( + StructField("Map", DataTypes.createMapType(StringType, StringType)), + StructField("MapNotNull", DataTypes.createMapType(StringType, StringType), + nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + compareMapAndMapDate(input.getMap(0), output.getMap(0)) + } + compareMapAndMapDate(input.getMap(1), output.getMap(1)) + } + } + } + } + + test("transform struct data back and forth between Row and Columnar") { + val structFieldArray = Array( + StructField("struct_int", IntegerType), + StructField("struct_double", DoubleType), + StructField("struct_array", DataTypes.createArrayType(DoubleType)) + ) + val schema = StructType(Seq( + StructField("Struct", DataTypes.createStructType(structFieldArray)), + StructField("StructNotNull", DataTypes.createStructType(structFieldArray), + nullable = false))) + val numRows = 100 + val rows = GpuBatchUtilsSuite.createExternalRows(schema, numRows) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val extR2CConverter = new GpuExternalRowToColumnConverter(schema) + rows.foreach(extR2CConverter.convert(_, batchBuilder)) + closeOnExcept(batchBuilder.build(numRows)) { columnarBatch => + val c2rIterator = new ColumnarToRowIterator(Iterator(columnarBatch), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + rows.foreach { input => + val output = c2rIterator.next() + if (input.isNullAt(0)) { + assert(output.isNullAt(0)) + } else { + val inputStructRow = input.getStruct(0) + val outputStructRow = output.getStruct(0, 3) + if (inputStructRow.isNullAt(0)) { + assert(outputStructRow.isNullAt(0)) + } else { + assert(inputStructRow.getInt(0) == outputStructRow.getInt(0)) + } + if (inputStructRow.isNullAt(1)) { + assert(outputStructRow.isNullAt(1)) + } else { + assert(inputStructRow.getDouble(1) == outputStructRow.getDouble(1)) + } + if (inputStructRow.isNullAt(2)) { + assert(outputStructRow.isNullAt(2)) + } else { + assert(inputStructRow.getSeq(2) sameElements outputStructRow.getArray(2) + .toDoubleArray()) + } + } + } + } + } + } + + test("InternalColumnarRddConverter should extractRDDTable RDD[ColumnarBatch]") { + withGpuSparkSession(spark => { + val path = TestResourceFinder.getResourcePath("disorder-read-schema.parquet") + val df = spark.read.parquet(path) + val (optionRddColumnBatch, _) = InternalColumnarRddConverter.extractRDDColumnarBatch(df) + + assert(optionRddColumnBatch.isDefined, "Can't extract RDD[ColumnarBatch]") + + }, new SparkConf().set("spark.rapids.sql.test.allowedNonGpu", "DeserializeToObjectExec")) + } + +} + diff --git a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java new file mode 100644 index 00000000000..b026936eb7e --- /dev/null +++ b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveGenericUDF.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.udf.hive; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.Text; + +/** An empty Hive GenericUDF returning the input directly for row-based UDF test only */ +public class EmptyHiveGenericUDF extends GenericUDF { + private transient PrimitiveObjectInspectorConverter.TextConverter converter; + private final Text textResult = new Text(); + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentException("One argument is supported, but found: " + arguments.length); + } + if (!(arguments[0] instanceof PrimitiveObjectInspector)) { + throw new UDFArgumentException("Unsupported argument type: " + arguments[0].getTypeName()); + } + PrimitiveObjectInspector poi = (PrimitiveObjectInspector) arguments[0]; + converter = new PrimitiveObjectInspectorConverter.TextConverter(poi); + return PrimitiveObjectInspectorFactory.writableStringObjectInspector; + } + + @Override + public Object evaluate(DeferredObject[] deferredObjects) throws HiveException { + Text text = converter.convert(deferredObjects[0].get()); + textResult.set(text == null ? "" : text.toString()); + return textResult; + } + + @Override + public String getDisplayString(String[] strings) { + return getStandardDisplayString("empty", strings); + } +} diff --git a/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java new file mode 100644 index 00000000000..f2c5a51a49f --- /dev/null +++ b/udf-examples/src/main/java/com/nvidia/spark/rapids/udf/hive/EmptyHiveSimpleUDF.java @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2021, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.udf.hive; + +import org.apache.hadoop.hive.ql.exec.UDF; + +/** An empty Hive simple UDF returning the first input directly for row-based UDF test only. */ +public class EmptyHiveSimpleUDF extends UDF { + public String evaluate(String in, String in2) { + return in; + } +}