From 688c757c7534fb6651b23e6c2dfa10231418c6e0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 14 Jul 2020 11:03:43 -0500 Subject: [PATCH 01/47] Working 30 and 31 --- dist/pom.xml | 10 + pom.xml | 2 + spark30/pom.xml | 114 ++++++++++ .../scala/com/nvidia/spark/Spark30Shims.scala | 133 +++++++++++ .../rapids/GpuBroadcastHashJoinExec30.scala | 77 +++++++ .../GpuBroadcastNestedLoopJoinExec30.scala | 82 +++++++ .../com/nvidia/spark/rapids/GpuHashJoin.scala | 89 ++------ .../rapids/GpuShuffledHashJoinExec30.scala | 83 +++++++ .../spark/rapids/GpuSortMergeJoinExec30.scala | 94 ++++++++ .../GpuBroadcastHashJoinExecBase30.scala | 193 ++++++++++------ .../shims/GpuShuffledHashJoinExecBase30.scala | 147 ++++++++---- spark31/pom.xml | 114 ++++++++++ .../spark/GpuBroadcastHashJoinExec31.scala | 76 +++++++ .../GpuBroadcastNestedLoopJoinExec31.scala | 82 +++++++ .../com/nvidia/spark/rapids/GpuHashJoin.scala | 209 +++++++++++++++++ .../rapids/GpuShuffledHashJoinExec31.scala | 93 ++++++++ .../spark/rapids/GpuSortMergeJoinExec31.scala | 41 +++- .../GpuBroadcastHashJoinExecBase31.scala | 213 ++++++++++++++++++ .../shims/GpuShuffledHashJoinExecBase31.scala | 200 ++++++++++++++++ .../spark/rapids/shims/Spark31Shims.scala | 129 +++++++++++ .../nvidia/spark/rapids/GpuOverrides.scala | 16 +- .../spark/rapids/GpuTransitionOverrides.scala | 5 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 12 + .../com/nvidia/spark/rapids/ShimLoader.scala | 142 ++++++++++++ .../com/nvidia/spark/rapids/SparkShims.scala | 48 ++++ .../sql/rapids/GpuCartesianProductExec.scala | 8 +- .../sql/rapids/datetimeExpressions.scala | 5 +- .../GpuBroadcastNestedLoopJoinExec.scala | 62 ++--- .../spark/rapids/HashSortOptimizeSuite.scala | 8 +- 29 files changed, 2260 insertions(+), 227 deletions(-) create mode 100644 spark30/pom.xml create mode 100644 spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala create mode 100644 spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala create mode 100644 spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala rename {sql-plugin => spark30}/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala (76%) create mode 100644 spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala create mode 100644 spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala rename sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala => spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala (59%) rename sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala => spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala (59%) create mode 100644 spark31/pom.xml create mode 100644 spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala rename sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala => spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala (64%) create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala create mode 100644 spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala diff --git a/dist/pom.xml b/dist/pom.xml index 438ebbbcf8d..89cbea68223 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -41,6 +41,16 @@ rapids-4-spark-shuffle_${scala.binary.version} ${project.version} + + com.nvidia + rapids-4-spark-shims-spark30_${scala.binary.version} + ${project.version} + + + com.nvidia + rapids-4-spark-shims-spark31_${scala.binary.version} + ${project.version} + diff --git a/pom.xml b/pom.xml index 630e0a47ce9..f50960ab940 100644 --- a/pom.xml +++ b/pom.xml @@ -76,6 +76,8 @@ sql-plugin tests integration_tests + spark30 + spark31 api_validation diff --git a/spark30/pom.xml b/spark30/pom.xml new file mode 100644 index 00000000000..5ee3bcbb7eb --- /dev/null +++ b/spark30/pom.xml @@ -0,0 +1,114 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-parent + 0.2.0-SNAPSHOT + ../pom.xml + + com.nvidia + rapids-4-spark-shims-spark30_2.12 + RAPIDS Accelerator for Apache Spark SQL Plugin + The RAPIDS SQL plugin for Apache Spark Shim + 0.2.0-SNAPSHOT + + + + databricks + + + + + 3.0.0 + + + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + + + ai.rapids + cudf + ${cuda.version} + + + com.google.flatbuffers + flatbuffers-java + compile + + + org.scala-lang + scala-library + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.orc + orc-core + ${orc.classifier} + + + org.slf4j + slf4j-api + + + + + org.apache.orc + orc-mapreduce + ${orc.classifier} + + + com.google.code.findbugs + jsr305 + + + + + org.apache.hive + hive-storage-api + + + org.slf4j + slf4j-api + + + + + com.google.protobuf + protobuf-java + + + + + + + net.alchim31.maven + scala-maven-plugin + + + + diff --git a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala new file mode 100644 index 00000000000..f88f3f9c59f --- /dev/null +++ b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala @@ -0,0 +1,133 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import java.time.ZoneId + +import com.nvidia.spark.rapids._ +import org.apache.spark.sql.rapids._ + +//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + + + +class Spark30Shims extends SparkShims with Logging { + + def isGpuHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuHashJoin30 => true + case p => false + } + } + + def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { + Seq( + GpuOverrides.exec[SortMergeJoinExec]( + "Sort merge join, replacing with shuffled hash join", + (join, conf, p, r) => new GpuSortMergeJoinMeta30(join, conf, p, r)), + GpuOverrides.exec[BroadcastHashJoinExec]( + "Implementation of join using broadcast data", + (join, conf, p, r) => new GpuBroadcastHashJoinMeta30(join, conf, p, r)), + GpuOverrides.exec[ShuffledHashJoinExec]( + "Implementation of join using hashed shuffled data", + (join, conf, p, r) => new GpuShuffledHashJoinMeta30(join, conf, p, r)), + ) + } + + def getExprs: Seq[ExprRule[_ <: Expression]] = { + Seq( + GpuOverrides.expr[TimeSub]( + "Subtracts interval from timestamp", + (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + logWarning("in TimeSub") + a.interval match { + case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => + if (intvl.months != 0) { + willNotWorkOnGpu("interval months isn't supported") + } + case _ => + willNotWorkOnGpu("only literals are supported for intervals") + } + if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { + willNotWorkOnGpu("Only UTC zone id is supported") + } + } + + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { + logWarning("in TimeSub convert") + GpuTimeSub(lhs, rhs) + } + } + ), + ) + } + + + def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } + + def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("bnlje Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("bnlje Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } + def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("bnlje Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("bnlje Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } +} + diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala new file mode 100644 index 00000000000..5e52c3e7127 --- /dev/null +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala @@ -0,0 +1,77 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuBroadcastHashJoinExec30( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends GpuBroadcastHashJoinExecBase30 with Logging { + + + logWarning("Tom in hadh join exec build side is: " + buildSide) + + def getBuildSide: GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuBroadcastHashJoinExec30 extends Logging { + + def createInstance( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + join: BroadcastHashJoinExec, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan): GpuBroadcastHashJoinExec30 = { + + GpuBroadcastHashJoinExec30(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) + } + +} diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala new file mode 100644 index 00000000000..3c9e0b10d49 --- /dev/null +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ +import org.apache.spark.sql.rapids.execution._ + +//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuBroadcastNestedLoopJoinExec30( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { + + + logWarning("Tom in broadcast nested loop join exec build side is: ") + + def getBuildSide: GpuBuildSide = { + join.buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuBroadcastNestedLoopJoinExec30 extends Logging { + + def createInstance( + left: SparkPlan, + right: SparkPlan, + joinType: JoinType, + join: BroadcastNestedLoopJoinExec, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase= { + + /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { + logWarning("Tom in shuffled hash join") + join.asInstanceOf[ShuffledHashJoinExec].buildSide + } else { + logWarning("Tom in not shuffled hash join") + BuildRight + } */ + + val res = GpuBroadcastNestedLoopJoinExec30(left, right, join, joinType, condition) + res + } + +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala similarity index 76% rename from sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala rename to spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala index 94b42dc4d2d..79f97323cdc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala @@ -18,13 +18,15 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{NvtxColor, Table} import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, HashJoin} +import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -object GpuHashJoin { +object GpuHashJoin30 { def tagJoin( meta: RapidsMeta[_, _, _], joinType: JoinType, @@ -48,7 +50,9 @@ object GpuHashJoin { } } -trait GpuHashJoin extends GpuExec with HashJoin { + +trait GpuHashJoin30 extends GpuExec with HashJoin with Logging { + override def output: Seq[Attribute] = { joinType match { @@ -69,16 +73,18 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) - } - } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] + + protected val gpuBuildKeys: Seq[GpuExpression] + protected val gpuStreamedKeys: Seq[GpuExpression] /** * Place the columns in left and the columns in right into a single ColumnarBatch @@ -143,14 +149,14 @@ trait GpuHashJoin extends GpuExec with HashJoin { if (stream.hasNext) { val cb = stream.next() val startTime = System.nanoTime() - nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } else if (first) { // We have to at least try one in some cases val startTime = System.nanoTime() val cb = GpuColumnVector.emptyBatch(streamedPlan.output.asJava) - nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } @@ -170,58 +176,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - private[this] def doJoin(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - buildSide match { - case BuildLeft => doJoinLeftRight(builtTable, streamedTable) - case BuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } - - private[this] def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { + def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { val joinedTable = joinType match { case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) .leftJoin(rightTable.onColumns(joinKeyIndices: _*)) diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala new file mode 100644 index 00000000000..f69a00975c9 --- /dev/null +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala @@ -0,0 +1,83 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuShuffledHashJoinExec30( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends GpuShuffledHashJoinExecBase30 with Logging { + + + logWarning("Tom in hadh join exec build side is: " + buildSide) + + def getBuildSide: GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuShuffledHashJoinExec30 extends Logging { + + def createInstance( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + join: SparkPlan, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan): GpuShuffledHashJoinExec30 = { + + val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { + logWarning("Tom in shuffled hash join") + join.asInstanceOf[ShuffledHashJoinExec].buildSide + } else { + logWarning("Tom in not shuffled hash join") + BuildRight + } + GpuShuffledHashJoinExec30(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + } + +} diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala new file mode 100644 index 00000000000..4206ed561d4 --- /dev/null +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala @@ -0,0 +1,94 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SortExec, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +class GpuSortMergeJoinMeta30( + join: SortMergeJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + + // Use conditions from Hash Join + GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + if (!conf.enableReplaceSortMergeJoin) { + willNotWorkOnGpu(s"Not replacing sort merge join with hash join, " + + s"see ${RapidsConf.ENABLE_REPLACE_SORTMERGEJOIN.key}") + } + + // make sure this is last check - if this is SortMergeJoin, the children can be Sorts and we + // want to validate they can run on GPU and remove them before replacing this with a + // ShuffleHashJoin + if (canThisBeReplaced) { + childPlans.foreach { plan => + if (plan.wrapped.isInstanceOf[SortExec]) { + if (!plan.canThisBeReplaced) { + willNotWorkOnGpu(s"can't replace sortMergeJoin because one of the SortExec's before " + + s"can't be replaced.") + } else { + plan.shouldBeRemoved("removing SortExec as part replacing sortMergeJoin with " + + s"shuffleHashJoin") + } + } + } + } + } + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec30( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + BuildRight, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } + +} + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala similarity index 59% rename from sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala rename to spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala index ac444d16471..ae4d2f85794 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,11 +14,15 @@ * limitations under the License. */ -package org.apache.spark.sql.rapids.execution +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ +import org.apache.spark.sql.rapids.execution._ +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression @@ -29,69 +33,24 @@ import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging -class GpuBroadcastHashJoinMeta( - join: BroadcastHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = - join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - - override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) +abstract class GpuBroadcastHashJoinExecBase30 extends BinaryExecNode with GpuHashJoin30 with Logging { + + def getBuildSide: GpuBuildSide - val buildSide = join.buildSide match { - case BuildLeft => childPlans(0) - case BuildRight => childPlans(1) - } - - if (!buildSide.canThisBeReplaced) { - willNotWorkOnGpu("the broadcast for this join must be on the GPU too") - } - - if (!canThisBeReplaced) { - buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + getBuildSide match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) } } - override def convertToGpu(): GpuExec = { - val left = childPlans(0).convertIfNeeded() - val right = childPlans(1).convertIfNeeded() - // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = join.buildSide match { - case BuildLeft => left - case BuildRight => right - } - if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { - throw new IllegalStateException("the broadcast must be on the GPU too") - } - GpuBroadcastHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, join.buildSide, - condition.map(_.convertToGpu()), - left, right) - } -} - -case class GpuBroadcastHashJoinExec( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - buildSide: BuildSide, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), @@ -99,14 +58,13 @@ case class GpuBroadcastHashJoinExec( override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) - buildSide match { - case BuildLeft => + getBuildSide match { + case GpuBuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil - case BuildRight => + case GpuBuildRight => UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } - def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { case gpu: GpuBroadcastExchangeExec => gpu case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] @@ -143,4 +101,113 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + getBuildSide match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + +} + + +class GpuBroadcastHashJoinMeta30( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + ShimLoader.getSparkShims.getBuildSide(join) + } + + override def tagPlanForGpu(): Unit = { + GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => childPlans(0) + case GpuBuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => left + case GpuBuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec30( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) + } + } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala similarity index 59% rename from sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala rename to spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala index 7ae310bd40f..d73592ac045 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,8 +14,12 @@ * limitations under the License. */ -package com.nvidia.spark.rapids +package com.nvidia.spark.rapids.shims +import ai.rapids.cudf.{NvtxColor, Table} + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuHashJoinBaseMeta import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext @@ -25,49 +29,27 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide, ShuffledHashJoinExec} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging -class GpuShuffledHashJoinMeta( - join: ShuffledHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) { - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = - join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition +abstract class GpuShuffledHashJoinExecBase30 extends BinaryExecNode with GpuHashJoin30 with Logging { + + def getBuildSide: GpuBuildSide - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + getBuildSide match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) + } } - override def convertToGpu(): GpuExec = - GpuShuffledHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, - join.buildSide, - condition.map(_.convertToGpu()), - childPlans(0).convertIfNeeded(), - childPlans(1).convertIfNeeded()) -} - -case class GpuShuffledHashJoinExec( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - buildSide: BuildSide, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), @@ -83,9 +65,9 @@ case class GpuShuffledHashJoinExec( "GpuShuffledHashJoin does not support the execute() code path.") } - override def childrenCoalesceGoal: Seq[CoalesceGoal] = buildSide match { - case BuildLeft => Seq(RequireSingleBatch, null) - case BuildRight => Seq(null, RequireSingleBatch) + override def childrenCoalesceGoal: Seq[CoalesceGoal] = getBuildSide match { + case GpuBuildLeft => Seq(RequireSingleBatch, null) + case GpuBuildRight => Seq(null, RequireSingleBatch) } override def doExecuteColumnar() : RDD[ColumnarBatch] = { @@ -132,4 +114,87 @@ case class GpuShuffledHashJoinExec( } } } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + getBuildSide match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + +} + +class GpuShuffledHashJoinMeta30( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + } + + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec30( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } } + diff --git a/spark31/pom.xml b/spark31/pom.xml new file mode 100644 index 00000000000..313f45a1ed2 --- /dev/null +++ b/spark31/pom.xml @@ -0,0 +1,114 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-parent + 0.2.0-SNAPSHOT + ../pom.xml + + com.nvidia + rapids-4-spark-shims-spark31_2.12 + RAPIDS Accelerator for Apache Spark SQL Plugin + The RAPIDS SQL plugin for Apache Spark Shim + 0.2.0-SNAPSHOT + + + + databricks + + + + + 3.1.0-SNAPSHOT + + + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + + + ai.rapids + cudf + ${cuda.version} + + + com.google.flatbuffers + flatbuffers-java + compile + + + org.scala-lang + scala-library + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.orc + orc-core + ${orc.classifier} + + + org.slf4j + slf4j-api + + + + + org.apache.orc + orc-mapreduce + ${orc.classifier} + + + com.google.code.findbugs + jsr305 + + + + + org.apache.hive + hive-storage-api + + + org.slf4j + slf4j-api + + + + + com.google.protobuf + protobuf-java + + + + + + + net.alchim31.maven + scala-maven-plugin + + + + diff --git a/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala new file mode 100644 index 00000000000..87913ed0d69 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala @@ -0,0 +1,76 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuBroadcastHashJoinExec31( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends GpuBroadcastHashJoinExecBase31 with Logging { + + + logWarning("Tom in hadh join exec build side is: " + buildSide) + + def getBuildSide: GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuBroadcastHashJoinExec31 extends Logging { + + def createInstance( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + join: BroadcastHashJoinExec, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan): GpuBroadcastHashJoinExec31 = { + + GpuBroadcastHashJoinExec31(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) + } + +} diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala new file mode 100644 index 00000000000..df6c1c82065 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ +import org.apache.spark.sql.rapids.execution._ + +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuBroadcastNestedLoopJoinExec31( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { + + + logWarning("Tom in broadcast nested loop join exec build side is: ") + + def getBuildSide: GpuBuildSide = { + join.buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuBroadcastNestedLoopJoinExec31 extends Logging { + + def createInstance( + left: SparkPlan, + right: SparkPlan, + joinType: JoinType, + join: BroadcastNestedLoopJoinExec, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { + + /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { + logWarning("Tom in shuffled hash join") + join.asInstanceOf[ShuffledHashJoinExec].buildSide + } else { + logWarning("Tom in not shuffled hash join") + BuildRight + } */ + + val res = GpuBroadcastNestedLoopJoinExec31(left, right, join, joinType, condition) + res + } + + def createTom(): Unit = {} +} diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala new file mode 100644 index 00000000000..085540fd673 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala @@ -0,0 +1,209 @@ +/* + * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids + +import ai.rapids.cudf.{NvtxColor, Table} + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.execution.joins.HashJoin +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec + +object GpuHashJoin31 { + def tagJoin( + meta: RapidsMeta[_, _, _], + joinType: JoinType, + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + condition: Option[Expression]): Unit = joinType match { + case _: InnerLike => + case FullOuter => + if (leftKeys.exists(_.nullable) || rightKeys.exists(_.nullable)) { + // https://github.com/rapidsai/cudf/issues/5563 + meta.willNotWorkOnGpu("Full outer join does not work on nullable keys") + } + if (condition.isDefined) { + meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") + } + case RightOuter | LeftOuter | LeftSemi | LeftAnti => + if (condition.isDefined) { + meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") + } + case _ => meta.willNotWorkOnGpu(s"$joinType currently is not supported") + } +} + + +trait GpuHashJoin31 extends GpuExec with HashJoin with Logging { + + + override def output: Seq[Attribute] = { + joinType match { + case _: InnerLike => + left.output ++ right.output + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case j: ExistenceJoin => + left.output :+ j.exists + case LeftExistence(_) => + left.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case x => + throw new IllegalArgumentException(s"GpuHashJoin should not take $x as the JoinType") + } + } + + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] + + protected val gpuBuildKeys: Seq[GpuExpression] + protected val gpuStreamedKeys: Seq[GpuExpression] + + /** + * Place the columns in left and the columns in right into a single ColumnarBatch + */ + def combine(left: ColumnarBatch, right: ColumnarBatch): ColumnarBatch = { + val l = GpuColumnVector.extractColumns(left) + val r = GpuColumnVector.extractColumns(right) + val c = l ++ r + new ColumnarBatch(c.asInstanceOf[Array[ColumnVector]], left.numRows()) + } + + // TODO eventually dedupe the keys + lazy val joinKeyIndices: Range = gpuBuildKeys.indices + + val localBuildOutput: Seq[Attribute] = buildPlan.output + // The first columns are the ones we joined on and need to remove + lazy val joinIndices: Seq[Int] = joinType match { + case RightOuter => + // The left table and right table are switched in the output + // because we don't support a right join, only left + val numRight = right.output.length + val numLeft = left.output.length + val joinLength = joinKeyIndices.length + def remap(index: Int): Int = { + if (index < numLeft) { + // part of the left table, but is on the right side of the tmp output + index + joinLength + numRight + } else { + // part of the right table, but is on the left side of the tmp output + index + joinLength - numLeft + } + } + output.indices.map (remap) + case _ => + val joinLength = joinKeyIndices.length + output.indices.map (v => v + joinLength) + } + + def doJoin(builtTable: Table, + stream: Iterator[ColumnarBatch], + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + joinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric, + totalTime: SQLMetric): Iterator[ColumnarBatch] = { + new Iterator[ColumnarBatch] { + import scala.collection.JavaConverters._ + var nextCb: Option[ColumnarBatch] = None + var first: Boolean = true + + TaskContext.get().addTaskCompletionListener[Unit](_ => closeCb()) + + def closeCb(): Unit = { + nextCb.map(_.close()) + nextCb = None + } + + override def hasNext: Boolean = { + while (nextCb.isEmpty && (first || stream.hasNext)) { + if (stream.hasNext) { + val cb = stream.next() + val startTime = System.nanoTime() + nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + numOutputBatches, joinTime, filterTime) + totalTime += (System.nanoTime() - startTime) + } else if (first) { + // We have to at least try one in some cases + val startTime = System.nanoTime() + val cb = GpuColumnVector.emptyBatch(streamedPlan.output.asJava) + nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + numOutputBatches, joinTime, filterTime) + totalTime += (System.nanoTime() - startTime) + } + first = false + } + nextCb.isDefined + } + + override def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException() + } + val ret = nextCb.get + nextCb = None + ret + } + } + } + + def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { + val joinedTable = joinType match { + case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) + .leftJoin(rightTable.onColumns(joinKeyIndices: _*)) + case RightOuter => rightTable.onColumns(joinKeyIndices: _*) + .leftJoin(leftTable.onColumns(joinKeyIndices: _*)) + case _: InnerLike => + leftTable.onColumns(joinKeyIndices: _*).innerJoin(rightTable.onColumns(joinKeyIndices: _*)) + case LeftSemi => + leftTable.onColumns(joinKeyIndices: _*) + .leftSemiJoin(rightTable.onColumns(joinKeyIndices: _*)) + case LeftAnti => + leftTable.onColumns(joinKeyIndices: _*) + .leftAntiJoin(rightTable.onColumns(joinKeyIndices: _*)) + case FullOuter => + leftTable.onColumns(joinKeyIndices: _*) + .fullJoin(rightTable.onColumns(joinKeyIndices: _*)) + case _ => throw new NotImplementedError(s"Joint Type ${joinType.getClass} is not currently" + + s" supported") + } + try { + val result = joinIndices.map(joinIndex => + GpuColumnVector.from(joinedTable.getColumn(joinIndex).incRefCount())) + .toArray[ColumnVector] + + new ColumnarBatch(result, joinedTable.getRowCount.toInt) + } finally { + joinedTable.close() + } + } +} diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala new file mode 100644 index 00000000000..0922116f97f --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala @@ -0,0 +1,93 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + + +case class GpuShuffledHashJoinExec31 ( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + buildSide: BuildSide, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends GpuShuffledHashJoinExecBase31 with Logging { + + + + /* + val buildSide: org.apache.spark.sql.execution.joins.BuildSide = { + logInfo("Tom gpu build side is: " + gpuBuildSide) + val res = gpuBuildSide match { + case GpuBuildRight => org.apache.spark.sql.execution.joins.BuildRight + case GpuBuildLeft => org.apache.spark.sql.execution.joins.BuildLeft + } + logInfo("Tom build side is: " + res) + res + } + */ + + def getBuildSide: GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +object GpuShuffledHashJoinExec31 extends Logging { + + def createInstance( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + join: SparkPlan, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan): GpuShuffledHashJoinExec31 = { + + val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { + logWarning("Tom in shuffled hash join") + join.asInstanceOf[ShuffledHashJoinExec].buildSide + } else { + logWarning("Tom in not shuffled hash join") + BuildRight + } + GpuShuffledHashJoinExec31(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + } + +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala similarity index 64% rename from sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala rename to spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala index 29ba63d625e..05e80d4f8c6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,18 +14,35 @@ * limitations under the License. */ -package com.nvidia.spark.rapids +package com.nvidia.spark.rapids.shims +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SortExec, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SortExec -import org.apache.spark.sql.execution.joins.{BuildRight, SortMergeJoinExec} -class GpuSortMergeJoinMeta( + + +class GpuSortMergeJoinMeta31( join: SortMergeJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends SparkPlanMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { + extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -34,11 +51,10 @@ class GpuSortMergeJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map( GpuOverrides.wrapExpr(_, conf, Some(this))) - override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition - override def tagPlanForGpu(): Unit = { + // Use conditions from Hash Join - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) if (!conf.enableReplaceSortMergeJoin) { willNotWorkOnGpu(s"Not replacing sort merge join with hash join, " + @@ -62,15 +78,16 @@ class GpuSortMergeJoinMeta( } } } - override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec( + GpuShuffledHashJoinExec31( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, - BuildRight, // just hardcode one side + BuildRight, condition.map(_.convertToGpu()), childPlans(0).convertIfNeeded(), childPlans(1).convertIfNeeded()) } + } + diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala new file mode 100644 index 00000000000..bb26bffdd14 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala @@ -0,0 +1,213 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{NvtxColor, Table} + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ +import org.apache.spark.sql.rapids.execution._ + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + +abstract class GpuBroadcastHashJoinExecBase31 extends BinaryExecNode with GpuHashJoin31 with Logging { + + def getBuildSide: GpuBuildSide + + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + getBuildSide match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) + } + } + + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = { + val mode = HashedRelationBroadcastMode(buildKeys) + getBuildSide match { + case GpuBuildLeft => + BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil + case GpuBuildRight => + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil + } + } + def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { + case gpu: GpuBroadcastExchangeExec => gpu + case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] + } + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val broadcastRelation = broadcastExchange + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + lazy val builtTable = { + // TODO clean up intermediate results... + val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) + val combined = combine(keys, broadcastRelation.value.batch) + val ret = GpuColumnVector.from(combined) + // Don't warn for a leak, because we cannot control when we are done with this + (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) + ret + } + + val rdd = streamedPlan.executeColumnar() + rdd.mapPartitions(it => + doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, + numOutputBatches, joinTime, filterTime, totalTime)) + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + getBuildSide match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + +} + + +class GpuBroadcastHashJoinMeta31( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + ShimLoader.getSparkShims.getBuildSide(join) + } + + override def tagPlanForGpu(): Unit = { + GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => childPlans(0) + case GpuBuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => left + case GpuBuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec31( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) + } + +} diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala new file mode 100644 index 00000000000..7342f913053 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala @@ -0,0 +1,200 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{NvtxColor, Table} + +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuHashJoinBaseMeta +import com.nvidia.spark.rapids.GpuMetricNames._ + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + + +abstract class GpuShuffledHashJoinExecBase31 extends BinaryExecNode with GpuHashJoin31 with Logging { + + def getBuildSide: GpuBuildSide + + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + getBuildSide match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) + } + } + + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), + "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException( + "GpuShuffledHashJoin does not support the execute() code path.") + } + + override def childrenCoalesceGoal: Seq[CoalesceGoal] = getBuildSide match { + case GpuBuildLeft => Seq(RequireSingleBatch, null) + case GpuBuildRight => Seq(null, RequireSingleBatch) + } + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val buildDataSize = longMetric("buildDataSize") + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val buildTime = longMetric("buildTime") + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { + (streamIter, buildIter) => { + var combinedSize = 0 + val startTime = System.nanoTime() + val buildBatch = + ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) + val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) + val builtTable = try { + // Combine does not inc any reference counting + val combined = combine(keys, buildBatch) + combinedSize = + GpuColumnVector.extractColumns(combined) + .map(_.getBase.getDeviceMemorySize).sum.toInt + GpuColumnVector.from(combined) + } finally { + keys.close() + buildBatch.close() + } + + val delta = System.nanoTime() - startTime + buildTime += delta + totalTime += delta + buildDataSize += combinedSize + val context = TaskContext.get() + context.addTaskCompletionListener[Unit](_ => builtTable.close()) + + doJoin(builtTable, streamIter, boundCondition, + numOutputRows, joinOutputRows, numOutputBatches, + joinTime, filterTime, totalTime) + } + } + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + getBuildSide match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + +} + + +class GpuShuffledHashJoinMeta31( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + } + + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec31( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } +} diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala new file mode 100644 index 00000000000..6165f516306 --- /dev/null +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala @@ -0,0 +1,129 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import java.time.ZoneId + +import com.nvidia.spark.rapids._ +import org.apache.spark.sql.rapids._ + +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.sql.types._ + + +class Spark31Shims extends SparkShims with Logging { + + def isGpuHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuHashJoin31 => true + case p => false + } + } + + def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { + Seq( + GpuOverrides.exec[SortMergeJoinExec]( + "Sort merge join, replacing with shuffled hash join", + (join, conf, p, r) => new GpuSortMergeJoinMeta31(join, conf, p, r)), + GpuOverrides.exec[BroadcastHashJoinExec]( + "Implementation of join using broadcast data", + (join, conf, p, r) => new GpuBroadcastHashJoinMeta31(join, conf, p, r)), + GpuOverrides.exec[ShuffledHashJoinExec]( + "Implementation of join using hashed shuffled data", + (join, conf, p, r) => new GpuShuffledHashJoinMeta31(join, conf, p, r)), + ) + } + + def getExprs: Seq[ExprRule[_ <: Expression]] = { + Seq( + + GpuOverrides.expr[TimeAdd]( + "Subtracts interval from timestamp", + (a, conf, p, r) => new BinaryExprMeta[TimeAdd](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.interval match { + case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => + if (intvl.months != 0) { + willNotWorkOnGpu("interval months isn't supported") + } + case _ => + willNotWorkOnGpu("only literals are supported for intervals") + } + if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { + willNotWorkOnGpu("Only UTC zone id is supported") + } + } + + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuTimeSub(lhs, rhs) + } + ) + ) + } + + def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } + + def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("bnlje Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("bnlje Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } + def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + val buildSide = join.buildSide + buildSide match { + case e: buildSide.type if e.toString.contains("BuildRight") => { + logInfo("Tom buildright " + e) + GpuBuildRight + } + case l: buildSide.type if l.toString.contains("BuildLeft") => { + logInfo("Tom buildleft "+ l) + GpuBuildLeft + } + case _ => throw new Exception("unknown buildSide Type") + } + } + +} + 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 17d8c450e19..747a2ce7a61 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 @@ -46,7 +46,7 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNes import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.rapids._ import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand -import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMeta, GpuBroadcastMeta, GpuBroadcastNestedLoopJoinMeta} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastMeta, GpuBroadcastNestedLoopJoinMeta} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -458,7 +458,7 @@ object GpuOverrides { .map(r => r.wrap(expr, conf, parent, r).asInstanceOf[BaseExprMeta[INPUT]]) .getOrElse(new RuleNotFoundExprMeta(expr, conf, parent)) - val expressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + val expressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = (Seq( expr[Literal]( "holds a static value from the query", (lit, conf, p, r) => new ExprMeta[Literal](lit, conf, p, r) { @@ -715,6 +715,7 @@ object GpuOverrides { GpuDateSub(lhs, rhs) } ), + /* expr[TimeSub]( "Subtracts interval from timestamp", (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { @@ -736,6 +737,7 @@ object GpuOverrides { GpuTimeSub(lhs, rhs) } ), + */ expr[NaNvl]( "evaluates to `left` iff left is not NaN, `right` otherwise.", (a, conf, p, r) => new BinaryExprMeta[NaNvl](a, conf, p, r) { @@ -1476,7 +1478,7 @@ object GpuOverrides { (a, conf, p, r) => new UnaryExprMeta[Length](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = GpuLength(child) }) - ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + ) ++ ShimLoader.getSparkShims.getExprs).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap def wrapScan[INPUT <: Scan]( scan: INPUT, @@ -1619,7 +1621,7 @@ object GpuOverrides { .map(r => r.wrap(plan, conf, parent, r).asInstanceOf[SparkPlanMeta[INPUT]]) .getOrElse(new RuleNotFoundSparkPlanMeta(plan, conf, parent)) - val execs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( + val execs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = (Seq( exec[GenerateExec] ( "The backend for operations that generate more output rows than input rows like explode.", (gen, conf, p, r) => new GpuGenerateExecSparkPlanMeta(gen, conf, p, r)), @@ -1717,12 +1719,14 @@ object GpuOverrides { exec[BroadcastExchangeExec]( "The backend for broadcast exchange of data", (exchange, conf, p, r) => new GpuBroadcastMeta(exchange, conf, p, r)), + /* exec[BroadcastHashJoinExec]( "Implementation of join using broadcast data", (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), exec[ShuffledHashJoinExec]( "Implementation of join using hashed shuffled data", (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), + */ exec[BroadcastNestedLoopJoinExec]( "Implementation of join using brute force", (join, conf, p, r) => new GpuBroadcastNestedLoopJoinMeta(join, conf, p, r)) @@ -1743,9 +1747,11 @@ object GpuOverrides { conf.gpuTargetBatchSizeBytes) }) .disabledByDefault("large joins can cause out of memory errors"), + /* exec[SortMergeJoinExec]( "Sort merge join, replacing with shuffled hash join", (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), + */ exec[HashAggregateExec]( "The backend for hash based aggregations", (agg, conf, p, r) => new GpuHashAggregateMeta(agg, conf, p, r)), @@ -1763,7 +1769,7 @@ object GpuOverrides { (windowOp, conf, p, r) => new GpuWindowExecMeta(windowOp, conf, p, r) ) - ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + ) ++ ShimLoader.getSparkShims.getExecs).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap } case class GpuOverrides() extends Rule[SparkPlan] with Logging { 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 c29499671e9..5e21ef69dc9 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 @@ -174,7 +174,10 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { // intermediate nodes that have a specified sort order. This helps with the size of // Parquet and Orc files plan match { - case _: GpuHashJoin | _: GpuHashAggregateExec => + case s if ShimLoader.getSparkShims.isGpuHashJoin(s) => + val sortOrder = getOptimizedSortOrder(plan) + GpuSortExec(sortOrder, false, plan, TargetSize(conf.gpuTargetBatchSizeBytes)) + case _: GpuHashAggregateExec => val sortOrder = getOptimizedSortOrder(plan) GpuSortExec(sortOrder, false, plan, TargetSize(conf.gpuTargetBatchSizeBytes)) case p => 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 834ec51ed19..d95d8e439e9 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 @@ -47,6 +47,18 @@ final class NoRuleConfKeysAndIncompat extends ConfKeysAndIncompat { override def confKey = "NOT_FOUND" } +abstract class GpuHashJoinBaseMeta[INPUT <: SparkPlan]( + plan: INPUT, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends SparkPlanMeta[INPUT](plan, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] + val rightKeys: Seq[BaseExprMeta[_]] + val condition: Option[BaseExprMeta[_]] +} + /** * Holds metadata about a stage in the physical plan that is separate from the plan itself. * This is helpful in deciding when to replace part of the plan with a GPU enabled version. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala new file mode 100644 index 00000000000..3319145b86f --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -0,0 +1,142 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import scala.collection.immutable.HashMap +import scala.collection.JavaConverters._ + +import com.nvidia.spark.rapids._ +import org.apache.spark.sql.rapids.execution._ + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.internal.Logging + +import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} +import org.apache.spark.internal.Logging + +object ShimLoader extends Logging { + + val SPARK30DATABRICKSSVERSIONNAME = "3.0.0-databricks" + val SPARK30VERSIONNAME = "3.0.0" + val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" + + private var sparkShims: SparkShims = null + private var gpuBroadcastNestedJoinShims: GpuBroadcastNestedLoopJoinExecBase = null + + /** + * The names of the classes for shimming Spark for each major version. + */ + private val SPARK_SHIM_CLASSES = HashMap( + SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark30Shims", + SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark300DatabricksShims", + SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark31Shims", + ) + + /** + * Factory method to get an instance of HadoopShims based on the + * version of Hadoop on the classpath. + */ + def getSparkShims: SparkShims = { + if (sparkShims == null) { + sparkShims = loadShims(SPARK_SHIM_CLASSES, classOf[SparkShims]) + } + sparkShims + } + + private val BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES = HashMap( + SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec30", + SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec300Databricks", + SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec31", + ) + + def getGpuBroadcastNestedLoopJoinShims( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { + if (sparkShims == null) { + gpuBroadcastNestedJoinShims = loadShimsNestedBroadcastJoin(BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES, classOf[GpuBroadcastNestedLoopJoinExecBase], + left, right, join, joinType, condition) + } + gpuBroadcastNestedJoinShims + } + + private def loadShims[T](classMap: Map[String, String], xface: Class[T]): T = { + val vers = getVersion(); + val className = classMap.get(vers) + if (className.isEmpty) { + throw new Exception(s"No shim layer for $vers") + } + createShim(className.get, xface) + } + + private def createShim[T](className: String, xface: Class[T]): T = try { + val clazz = Class.forName(className) + val res = clazz.newInstance().asInstanceOf[T] + res + } catch { + case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) + } + + private def loadShimsNestedBroadcastJoin[T](classMap: Map[String, String], xface: Class[T], + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): T = { + val vers = getVersion(); + val className = classMap.get(vers) + if (className.isEmpty) { + throw new Exception(s"No shim layer for $vers") + } + createShimNestedBroadcastJoin(className.get, xface, left, right, join, joinType, condition) + } + + private def createShimNestedBroadcastJoin[T](className: String, xface: Class[T], + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): T = try { + val clazz = Class.forName(className) + val resultMethod = clazz.getDeclaredMethod("createInstance", classOf[org.apache.spark.sql.execution.SparkPlan],classOf[org.apache.spark.sql.execution.SparkPlan],classOf[org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec], classOf[org.apache.spark.sql.catalyst.plans.JoinType], classOf[scala.Option[org.apache.spark.sql.catalyst.expressions.Expression]]) + val res = resultMethod.invoke(clazz, left, right, join, joinType, condition).asInstanceOf[T] + res + } catch { + case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) + } + + def getVersion(): String = { + // hack for databricks, try to find something more reliable? + if (SPARK_BUILD_USER.equals("Databricks")) { + SPARK_VERSION + "-databricks" + } else { + SPARK_VERSION + } + } + +} 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 new file mode 100644 index 00000000000..c58e849887e --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch + +sealed abstract class GpuBuildSide + +case object GpuBuildRight extends GpuBuildSide + +case object GpuBuildLeft extends GpuBuildSide + + +trait SparkShims { + + def isGpuHashJoin(plan: SparkPlan): Boolean + def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide + def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide + def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide + def getExprs: Seq[ExprRule[_ <: Expression]] + def getExecs: Seq[ExecRule[_ <: SparkPlan]] + +} + + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala index 4c8c5405d0c..1143e35b920 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.rapids import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import ai.rapids.cudf.{JCudfSerialization, NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.{Arm, GpuBindReferences, GpuColumnarBatchSerializer, GpuColumnVector, GpuExec, GpuExpression, GpuSemaphore} +import com.nvidia.spark.rapids.{Arm, GpuBindReferences, GpuBuildLeft, GpuColumnarBatchSerializer, GpuColumnVector, GpuExec, GpuExpression, GpuSemaphore} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.{Dependency, NarrowDependency, Partition, SparkContext, TaskContext} @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils, SparkPlan} import org.apache.spark.sql.execution.joins.BuildLeft import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExec +import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.util.{CompletionIterator, Utils} @@ -146,10 +146,10 @@ class GpuCartesianRDD( // Ideally instead of looping through and recomputing rdd2 for // each batch in rdd1 we would instead cache rdd2 in a way that // it could spill to disk so we can avoid re-computation - val ret = GpuBroadcastNestedLoopJoinExec.innerLikeJoin( + val ret = GpuBroadcastNestedLoopJoinExecBase.innerLikeJoin( rdd2.iterator(currSplit.s2, context).map(i => i.getBatch), table, - BuildLeft, + GpuBuildLeft, boundCondition, joinTime, joinOutputRows, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 51a17d7715f..ebed7913020 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInput import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.internal.Logging trait GpuDateUnaryExpression extends GpuUnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -121,10 +122,12 @@ case class GpuTimeSub( start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with GpuExpression with TimeZoneAwareExpression with ExpectsInputTypes { + extends BinaryExpression with GpuExpression with TimeZoneAwareExpression with ExpectsInputTypes with Logging { def this(start: Expression, interval: Expression) = this(start, interval, None) + logWarning("in gpu time sub") + override def left: Expression = start override def right: Expression = interval diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index c120444b4b9..a52747074f0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, Table} -import com.nvidia.spark.rapids.{Arm, BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuColumnVector, GpuExec, GpuExpression, GpuFilter, GpuOverrides, NvtxWithMetrics, RapidsConf, RapidsMeta, SparkPlanMeta} +import com.nvidia.spark.rapids.{Arm, BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBuildRight, GpuBuildLeft, GpuBuildSide, GpuColumnVector, GpuExec, GpuExpression, GpuFilter, GpuOverrides, NvtxWithMetrics, RapidsConf, RapidsMeta, ShimLoader, SparkPlanMeta} import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import org.apache.spark.rdd.RDD @@ -50,9 +50,10 @@ class GpuBroadcastNestedLoopJoinMeta( case _ => willNotWorkOnGpu(s"$join.joinType currently is not supported") } - val buildSide = join.buildSide match { - case BuildLeft => childPlans.head - case BuildRight => childPlans(1) + val gpuBuildSide = ShimLoader.getSparkShims.getBuildSide(join) + val buildSide = gpuBuildSide match { + case GpuBuildLeft => childPlans.head + case GpuBuildRight => childPlans(1) } if (!buildSide.canThisBeReplaced) { @@ -69,26 +70,26 @@ class GpuBroadcastNestedLoopJoinMeta( val left = childPlans.head.convertIfNeeded() val right = childPlans(1).convertIfNeeded() // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = join.buildSide match { - case BuildLeft => left - case BuildRight => right - } + val gpuBuildSide = ShimLoader.getSparkShims.getBuildSide(join) + val buildSide = gpuBuildSide match { + case GpuBuildLeft => left + case GpuBuildRight => right + } if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") } - GpuBroadcastNestedLoopJoinExec( - left, right, join.buildSide, + ShimLoader.getGpuBroadcastNestedLoopJoinShims( + left, right, join, join.joinType, - condition.map(_.convertToGpu()), - conf.gpuTargetBatchSizeBytes) + condition.map(_.convertToGpu())) } } -object GpuBroadcastNestedLoopJoinExec extends Arm { +object GpuBroadcastNestedLoopJoinExecBase extends Arm { def innerLikeJoin( streamedIter: Iterator[ColumnarBatch], builtTable: Table, - buildSide: BuildSide, + buildSide: GpuBuildSide, boundCondition: Option[GpuExpression], joinTime: SQLMetric, joinOutputRows: SQLMetric, @@ -105,8 +106,8 @@ object GpuBroadcastNestedLoopJoinExec extends Arm { withResource(new NvtxWithMetrics("join", NvtxColor.ORANGE, joinTime)) { _ => val joinedTable = withResource(streamTable) { tab => buildSide match { - case BuildLeft => builtTable.crossJoin(tab) - case BuildRight => tab.crossJoin(builtTable) + case GpuBuildLeft => builtTable.crossJoin(tab) + case GpuBuildRight => tab.crossJoin(builtTable) } } withResource(joinedTable) { jt => @@ -127,13 +128,14 @@ object GpuBroadcastNestedLoopJoinExec extends Arm { } } -case class GpuBroadcastNestedLoopJoinExec( +abstract class GpuBroadcastNestedLoopJoinExecBase( left: SparkPlan, right: SparkPlan, - buildSide: BuildSide, + join: BroadcastNestedLoopJoinExec, joinType: JoinType, - condition: Option[Expression], - targetSize: Long) extends BinaryExecNode with GpuExec { + condition: Option[Expression]) extends BinaryExecNode with GpuExec { + + def getBuildSide: GpuBuildSide override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException("This should only be called from columnar") @@ -146,9 +148,9 @@ case class GpuBroadcastNestedLoopJoinExec( "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) /** BuildRight means the right relation <=> the broadcast relation. */ - private val (streamed, broadcast) = buildSide match { - case BuildRight => (left, right) - case BuildLeft => (right, left) + private val (streamed, broadcast) = getBuildSide match { + case GpuBuildRight => (left, right) + case GpuBuildLeft => (right, left) } def broadcastExchange: GpuBroadcastExchangeExec = broadcast match { @@ -156,10 +158,10 @@ case class GpuBroadcastNestedLoopJoinExec( case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] } - override def requiredChildDistribution: Seq[Distribution] = buildSide match { - case BuildLeft => + override def requiredChildDistribution: Seq[Distribution] = getBuildSide match { + case GpuBuildLeft => BroadcastDistribution(IdentityBroadcastMode) :: UnspecifiedDistribution :: Nil - case BuildRight => + case GpuBuildRight => UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } @@ -214,12 +216,12 @@ case class GpuBroadcastNestedLoopJoinExec( streamed.executeColumnar().mapPartitions { streamedIter => joinType match { - case _: InnerLike => GpuBroadcastNestedLoopJoinExec.innerLikeJoin(streamedIter, - builtTable, buildSide, boundCondition, + case _: InnerLike => GpuBroadcastNestedLoopJoinExecBase.innerLikeJoin(streamedIter, + builtTable, getBuildSide, boundCondition, joinTime, joinOutputRows, numOutputRows, numOutputBatches, filterTime, totalTime) - case _ => throw new IllegalArgumentException(s"$joinType + $buildSide is not supported" + + case _ => throw new IllegalArgumentException(s"$joinType + $getBuildSide is not supported" + s" and should be run on the CPU") } } } -} \ No newline at end of file +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala index 43a0be08c04..1de997eea68 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.execution.{SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec -import org.apache.spark.sql.rapids.execution.GpuBroadcastHashJoinExec +// import org.apache.spark.sql.rapids.execution.GpuBroadcastHashJoinExecBase /** Test plan modifications to add optimizing sorts after hash joins in the plan */ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { @@ -65,10 +65,11 @@ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { assertResult(joinNode) { sortChild.children.head } } + /* test("sort inserted after broadcast hash join") { val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(_.isInstanceOf[GpuBroadcastHashJoinExec]) + val joinNode = plan.find(_.isInstanceOf[GpuBroadcastHashJoinExecBase]) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) } @@ -77,10 +78,11 @@ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { spark.conf.set("spark.sql.autoBroadcastJoinThreshold", 0) val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(_.isInstanceOf[GpuShuffledHashJoinExec]) + val joinNode = plan.find(_.isInstanceOf[GpuShuffledHashJoinExecBase]) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) } + */ test("config to disable") { spark.conf.set(RapidsConf.ENABLE_HASH_OPTIMIZE_SORT.key, "false") From e9b3268ec0cb2ac37f4365030e6a52fa92b33d67 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 14 Jul 2020 13:17:35 -0500 Subject: [PATCH 02/47] Build with both spark3.0 and spark 3.1 --- pom.xml | 3 +++ spark30/pom.xml | 2 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 8 +++---- .../sql/rapids/GpuCartesianProductExec.scala | 1 - .../sql/rapids/GpuFileSourceScanExec.scala | 7 ++++++ .../GpuBroadcastNestedLoopJoinExec.scala | 2 +- tests/pom.xml | 24 +++++++++++++++++++ 7 files changed, 40 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index f50960ab940..2509f3606b0 100644 --- a/pom.xml +++ b/pom.xml @@ -130,6 +130,9 @@ true + + spark31 + diff --git a/spark30/pom.xml b/spark30/pom.xml index 5ee3bcbb7eb..ae962cd0063 100644 --- a/spark30/pom.xml +++ b/spark30/pom.xml @@ -37,7 +37,7 @@ - + 3.0.0 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 d95d8e439e9..8ef94bc851f 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.types.{CalendarIntervalType, DataType, DataTypes, StringType} trait ConfKeysAndIncompat { @@ -432,9 +432,9 @@ abstract class SparkPlanMeta[INPUT <: SparkPlan](plan: INPUT, private def findShuffleExchanges(): Seq[SparkPlanMeta[ShuffleExchangeExec]] = wrapped match { case _: ShuffleExchangeExec => this.asInstanceOf[SparkPlanMeta[ShuffleExchangeExec]] :: Nil - case bkj: BroadcastHashJoinExec => bkj.buildSide match { - case BuildLeft => childPlans(1).findShuffleExchanges() - case BuildRight => childPlans(0).findShuffleExchanges() + case bkj: BroadcastHashJoinExec => ShimLoader.getSparkShims.getBuildSide(bkj) match { + case GpuBuildLeft => childPlans(1).findShuffleExchanges() + case GpuBuildRight => childPlans(0).findShuffleExchanges() } case _ => childPlans.flatMap(_.findShuffleExchanges()) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala index 1143e35b920..4aefdde6d7d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala @@ -28,7 +28,6 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils, SparkPlan} -import org.apache.spark.sql.execution.joins.BuildLeft import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index e2c8c976851..ca4c044e94a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -60,9 +60,16 @@ case class GpuFileSourceScanExec( // that is the logicalRelation. We don't know what its used for exactly but haven't // run into any issues in testing using the one we create here. @transient val logicalRelation = LogicalRelation(relation) + try { constructor.newInstance(relation, output, requiredSchema, partitionFilters, optionalBucketSet, dataFilters, tableIdentifier, logicalRelation).asInstanceOf[FileSourceScanExec] + } catch { + case il: IllegalArgumentException => + // TODO - workaround until https://github.com/NVIDIA/spark-rapids/issues/354 + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, None, dataFilters, tableIdentifier).asInstanceOf[FileSourceScanExec] + } } else { constructor.newInstance(relation, output, requiredSchema, partitionFilters, optionalBucketSet, dataFilters, tableIdentifier).asInstanceOf[FileSourceScanExec] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index a52747074f0..379f568998a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{Cross, ExistenceJoin, FullOuter, Inn import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, IdentityBroadcastMode, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/tests/pom.xml b/tests/pom.xml index 62e56f603ec..dc35d1942cd 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -30,6 +30,30 @@ RAPIDS plugin for Apache Spark integration tests 0.2.0-SNAPSHOT + + + spark31 + + 3.1.0-SNAPSHOT + 1.7.30 + + + + org.slf4j + jul-to-slf4j + ${slf4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + + + + + + org.scala-lang From ab09032547e4ae821ee1f6751a6630121b683d10 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 14 Jul 2020 16:34:55 -0500 Subject: [PATCH 03/47] minor fixes --- dist/pom.xml | 24 +++++++++++++++++++ sql-plugin/pom.xml | 24 ------------------- .../nvidia/spark/rapids/GpuOverrides.scala | 6 +++-- .../com/nvidia/spark/rapids/ShimLoader.scala | 2 +- .../sql/rapids/datetimeExpressions.scala | 5 +--- .../GpuBroadcastNestedLoopJoinExec.scala | 2 +- tests/pom.xml | 1 - 7 files changed, 31 insertions(+), 33 deletions(-) diff --git a/dist/pom.xml b/dist/pom.xml index 89cbea68223..28c4d1d68db 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -114,6 +114,30 @@ + + net.alchim31.maven + scala-maven-plugin + + + update_config + verify + + run + + + + + + + update_rapids_config + com.nvidia.spark.rapids.RapidsConf + + ${project.basedir}/../docs/configs.md + + + + + diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index f7465d8efc8..0af3a6cb9a0 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -141,30 +141,6 @@ - - net.alchim31.maven - scala-maven-plugin - - - update_config - verify - - run - - - - - - - update_rapids_config - com.nvidia.spark.rapids.RapidsConf - - ${project.basedir}/../docs/configs.md - - - - - org.scalastyle scalastyle-maven-plugin 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 747a2ce7a61..760bf6f19a6 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 @@ -1478,7 +1478,8 @@ object GpuOverrides { (a, conf, p, r) => new UnaryExprMeta[Length](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = GpuLength(child) }) - ) ++ ShimLoader.getSparkShims.getExprs).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + ) ++ ShimLoader.getSparkShims.getExprs) + .map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap def wrapScan[INPUT <: Scan]( scan: INPUT, @@ -1769,7 +1770,8 @@ object GpuOverrides { (windowOp, conf, p, r) => new GpuWindowExecMeta(windowOp, conf, p, r) ) - ) ++ ShimLoader.getSparkShims.getExecs).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + ) ++ ShimLoader.getSparkShims.getExecs) + .map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap } case class GpuOverrides() extends Rule[SparkPlan] with Logging { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 3319145b86f..f01568088a0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -138,5 +138,5 @@ object ShimLoader extends Logging { SPARK_VERSION } } - } + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index ebed7913020..51a17d7715f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInput import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.internal.Logging trait GpuDateUnaryExpression extends GpuUnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) @@ -122,12 +121,10 @@ case class GpuTimeSub( start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with GpuExpression with TimeZoneAwareExpression with ExpectsInputTypes with Logging { + extends BinaryExpression with GpuExpression with TimeZoneAwareExpression with ExpectsInputTypes { def this(start: Expression, interval: Expression) = this(start, interval, None) - logWarning("in gpu time sub") - override def left: Expression = start override def right: Expression = interval diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 379f568998a..6720b70d628 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, Table} -import com.nvidia.spark.rapids.{Arm, BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBuildRight, GpuBuildLeft, GpuBuildSide, GpuColumnVector, GpuExec, GpuExpression, GpuFilter, GpuOverrides, NvtxWithMetrics, RapidsConf, RapidsMeta, ShimLoader, SparkPlanMeta} +import com.nvidia.spark.rapids.{Arm, BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBuildLeft, GpuBuildRight, GpuBuildSide, GpuColumnVector, GpuExec, GpuExpression, GpuFilter, GpuOverrides, NvtxWithMetrics, RapidsConf, RapidsMeta, ShimLoader, SparkPlanMeta} import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import org.apache.spark.rdd.RDD diff --git a/tests/pom.xml b/tests/pom.xml index dc35d1942cd..89a0239dc5a 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -53,7 +53,6 @@ - org.scala-lang From e511110208d2797986db0827e0a877e205c62d07 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 14 Jul 2020 16:59:22 -0500 Subject: [PATCH 04/47] Formatting --- .../com/nvidia/spark/rapids/ShimLoader.scala | 49 +++++++++---------- 1 file changed, 22 insertions(+), 27 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index f01568088a0..b87dc0046a5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -17,27 +17,15 @@ package com.nvidia.spark.rapids import scala.collection.immutable.HashMap -import scala.collection.JavaConverters._ -import com.nvidia.spark.rapids._ -import org.apache.spark.sql.rapids.execution._ - -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - +import org.apache.spark.sql.rapids.execution._ import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} -import org.apache.spark.internal.Logging -object ShimLoader extends Logging { +object ShimLoader { val SPARK30DATABRICKSSVERSIONNAME = "3.0.0-databricks" val SPARK30VERSIONNAME = "3.0.0" @@ -78,15 +66,15 @@ object ShimLoader extends Logging { join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { - if (sparkShims == null) { - gpuBroadcastNestedJoinShims = loadShimsNestedBroadcastJoin(BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES, classOf[GpuBroadcastNestedLoopJoinExecBase], - left, right, join, joinType, condition) + if (gpuBroadcastNestedJoinShims == null) { + gpuBroadcastNestedJoinShims = loadShimsNestedBroadcastJoin(BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES, + classOf[GpuBroadcastNestedLoopJoinExecBase], left, right, join, joinType, condition) } gpuBroadcastNestedJoinShims } private def loadShims[T](classMap: Map[String, String], xface: Class[T]): T = { - val vers = getVersion(); + val vers = getVersion() val className = classMap.get(vers) if (className.isEmpty) { throw new Exception(s"No shim layer for $vers") @@ -96,19 +84,20 @@ object ShimLoader extends Logging { private def createShim[T](className: String, xface: Class[T]): T = try { val clazz = Class.forName(className) - val res = clazz.newInstance().asInstanceOf[T] - res + clazz.newInstance().asInstanceOf[T] } catch { case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) } - private def loadShimsNestedBroadcastJoin[T](classMap: Map[String, String], xface: Class[T], + private def loadShimsNestedBroadcastJoin[T]( + classMap: Map[String, String], + xface: Class[T], left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]): T = { - val vers = getVersion(); + val vers = getVersion() val className = classMap.get(vers) if (className.isEmpty) { throw new Exception(s"No shim layer for $vers") @@ -116,16 +105,22 @@ object ShimLoader extends Logging { createShimNestedBroadcastJoin(className.get, xface, left, right, join, joinType, condition) } - private def createShimNestedBroadcastJoin[T](className: String, xface: Class[T], + private def createShimNestedBroadcastJoin[T]( + className: String, + xface: Class[T], left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]): T = try { val clazz = Class.forName(className) - val resultMethod = clazz.getDeclaredMethod("createInstance", classOf[org.apache.spark.sql.execution.SparkPlan],classOf[org.apache.spark.sql.execution.SparkPlan],classOf[org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec], classOf[org.apache.spark.sql.catalyst.plans.JoinType], classOf[scala.Option[org.apache.spark.sql.catalyst.expressions.Expression]]) - val res = resultMethod.invoke(clazz, left, right, join, joinType, condition).asInstanceOf[T] - res + val resultMethod = clazz.getDeclaredMethod("createInstance", + classOf[org.apache.spark.sql.execution.SparkPlan], + classOf[org.apache.spark.sql.execution.SparkPlan], + classOf[org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec], + classOf[org.apache.spark.sql.catalyst.plans.JoinType], + classOf[scala.Option[org.apache.spark.sql.catalyst.expressions.Expression]]) + resultMethod.invoke(clazz, left, right, join, joinType, condition).asInstanceOf[T] } catch { case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) } From 0d9768771a514fd13067243bc9d79e8eff15b44f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 16 Jul 2020 11:24:04 -0500 Subject: [PATCH 05/47] put back building configs --- pom.xml | 10 ++++++++ sql-plugin/pom.xml | 24 +++++++++++++++++++ .../com/nvidia/spark/rapids/ShimLoader.scala | 10 ++++---- 3 files changed, 39 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 2509f3606b0..cd010307250 100644 --- a/pom.xml +++ b/pom.xml @@ -550,5 +550,15 @@ true + + apache-snapshots-repo + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 0af3a6cb9a0..f7465d8efc8 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -141,6 +141,30 @@ + + net.alchim31.maven + scala-maven-plugin + + + update_config + verify + + run + + + + + + + update_rapids_config + com.nvidia.spark.rapids.RapidsConf + + ${project.basedir}/../docs/configs.md + + + + + org.scalastyle scalastyle-maven-plugin diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index b87dc0046a5..f94f2703386 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -44,8 +44,8 @@ object ShimLoader { ) /** - * Factory method to get an instance of HadoopShims based on the - * version of Hadoop on the classpath. + * Factory method to get an instance of SparkShims based on the + * version of Spark on the classpath. */ def getSparkShims: SparkShims = { if (sparkShims == null) { @@ -74,7 +74,7 @@ object ShimLoader { } private def loadShims[T](classMap: Map[String, String], xface: Class[T]): T = { - val vers = getVersion() + val vers = getVersion val className = classMap.get(vers) if (className.isEmpty) { throw new Exception(s"No shim layer for $vers") @@ -97,7 +97,7 @@ object ShimLoader { join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]): T = { - val vers = getVersion() + val vers = getVersion val className = classMap.get(vers) if (className.isEmpty) { throw new Exception(s"No shim layer for $vers") @@ -125,7 +125,7 @@ object ShimLoader { case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) } - def getVersion(): String = { + def getVersion: String = { // hack for databricks, try to find something more reliable? if (SPARK_BUILD_USER.equals("Databricks")) { SPARK_VERSION + "-databricks" From 773edfd1c845fa7fc15a1a8ef94f2fe984ea5d8e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 16 Jul 2020 22:33:15 +0000 Subject: [PATCH 06/47] iMove GpuFileSourceScanExec to spark specific dirs --- .../scala/com/nvidia/spark/Spark30Shims.scala | 27 +++ .../sql/rapids/GpuFileSourceScanExec30.scala | 4 +- .../spark/rapids/shims/Spark31Shims.scala | 27 +++ .../sql/rapids/GpuFileSourceScanExec31.scala | 178 ++++++++++++++++++ 4 files changed, 234 insertions(+), 2 deletions(-) rename sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala => spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala (99%) create mode 100644 spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala index f88f3f9c59f..7fd5df4d7a1 100644 --- a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala +++ b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala @@ -44,6 +44,33 @@ class Spark30Shims extends SparkShims with Logging { def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( + + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { + // partition filters and data filters are not run on the GPU + override val childExprs: Seq[ExprMeta[_]] = Seq.empty + + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec30.tagSupport(this) + + override def convertToGpu(): GpuExec = { + val newRelation = HadoopFsRelation( + wrapped.relation.location, + wrapped.relation.partitionSchema, + wrapped.relation.dataSchema, + wrapped.relation.bucketSpec, + GpuFileSourceScanExec30.convertFileFormat(wrapped.relation.fileFormat), + wrapped.relation.options)(wrapped.relation.sparkSession) + GpuFileSourceScanExec30( + newRelation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.dataFilters, + wrapped.tableIdentifier) + } + }), GpuOverrides.exec[SortMergeJoinExec]( "Sort merge join, replacing with shuffled hash join", (join, conf, p, r) => new GpuSortMergeJoinMeta30(join, conf, p, r)), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala similarity index 99% rename from sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala rename to spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala index ca4c044e94a..fffca5b82c6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet -case class GpuFileSourceScanExec( +case class GpuFileSourceScanExec30( @transient relation: HadoopFsRelation, output: Seq[Attribute], requiredSchema: StructType, @@ -155,7 +155,7 @@ case class GpuFileSourceScanExec( } } -object GpuFileSourceScanExec { +object GpuFileSourceScanExec30 { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { meta.wrapped.relation.fileFormat match { case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala index 6165f516306..8cfc1350e1e 100644 --- a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala @@ -42,6 +42,33 @@ class Spark31Shims extends SparkShims with Logging { def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( + + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { + // partition filters and data filters are not run on the GPU + override val childExprs: Seq[ExprMeta[_]] = Seq.empty + + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec31.tagSupport(this) + + override def convertToGpu(): GpuExec = { + val newRelation = HadoopFsRelation( + wrapped.relation.location, + wrapped.relation.partitionSchema, + wrapped.relation.dataSchema, + wrapped.relation.bucketSpec, + GpuFileSourceScanExec31.convertFileFormat(wrapped.relation.fileFormat), + wrapped.relation.options)(wrapped.relation.sparkSession) + GpuFileSourceScanExec31( + newRelation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.dataFilters, + wrapped.tableIdentifier) + } + }), GpuOverrides.exec[SortMergeJoinExec]( "Sort merge join, replacing with shuffled hash join", (join, conf, p, r) => new GpuSortMergeJoinMeta31(join, conf, p, r)), diff --git a/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala b/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala new file mode 100644 index 00000000000..358e65c2fdc --- /dev/null +++ b/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala @@ -0,0 +1,178 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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 + +import java.util.concurrent.TimeUnit.NANOSECONDS + +import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{DataSourceScanExec, ExplainUtils, FileSourceScanExec} +import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.collection.BitSet + +case class GpuFileSourceScanExec31( + @transient relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + optionalBucketSet: Option[BitSet], + dataFilters: Seq[Expression], + override val tableIdentifier: Option[TableIdentifier]) + extends DataSourceScanExec with GpuExec { + + override val nodeName: String = { + s"GpuScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" + } + + private[this] val wrapped: FileSourceScanExec = { + val tclass = classOf[org.apache.spark.sql.execution.FileSourceScanExec] + val constructors = tclass.getConstructors() + if (constructors.size > 1) { + throw new IllegalStateException(s"Only expected 1 constructor for FileSourceScanExec") + } + val constructor = constructors(0) + val instance = if (constructor.getParameterCount() == 8) { + // Some distributions of Spark modified FileSourceScanExec to take an additional parameter + // that is the logicalRelation. We don't know what its used for exactly but haven't + // run into any issues in testing using the one we create here. + @transient val logicalRelation = LogicalRelation(relation) + try { + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, dataFilters, tableIdentifier, + logicalRelation).asInstanceOf[FileSourceScanExec] + } catch { + case il: IllegalArgumentException => + // TODO - workaround until https://github.com/NVIDIA/spark-rapids/issues/354 + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, None, dataFilters, tableIdentifier).asInstanceOf[FileSourceScanExec] + } + } else { + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, dataFilters, tableIdentifier).asInstanceOf[FileSourceScanExec] + } + instance + } + + override lazy val outputPartitioning: Partitioning = wrapped.outputPartitioning + + override lazy val outputOrdering: Seq[SortOrder] = wrapped.outputOrdering + + override lazy val metadata: Map[String, String] = wrapped.metadata + + override lazy val metrics: Map[String, SQLMetric] = wrapped.metrics + + override def verboseStringWithOperatorId(): String = { + val metadataStr = metadata.toSeq.sorted.filterNot { + case (_, value) if (value.isEmpty || value.equals("[]")) => true + case (key, _) if (key.equals("DataFilters") || key.equals("Format")) => true + case (_, _) => false + }.map { + case (key, _) if (key.equals("Location")) => + val location = wrapped.relation.location + val numPaths = location.rootPaths.length + val abbreviatedLoaction = if (numPaths <= 1) { + location.rootPaths.mkString("[", ", ", "]") + } else { + "[" + location.rootPaths.head + s", ... ${numPaths - 1} entries]" + } + s"$key: ${location.getClass.getSimpleName} ${redact(abbreviatedLoaction)}" + case (key, value) => s"$key: ${redact(value)}" + } + + s""" + |$formattedNodeName + |${ExplainUtils.generateFieldString("Output", output)} + |${metadataStr.mkString("\n")} + |""".stripMargin + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + wrapped.inputRDD :: Nil + } + + override protected def doExecute(): RDD[InternalRow] = + throw new IllegalStateException(s"Row-based execution should not occur for $this") + + override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + val scanTime = longMetric("scanTime") + wrapped.inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches => + new Iterator[ColumnarBatch] { + + override def hasNext: Boolean = { + // The `FileScanRDD` returns an iterator which scans the file during the `hasNext` call. + val startNs = System.nanoTime() + val res = batches.hasNext + scanTime += NANOSECONDS.toMillis(System.nanoTime() - startNs) + res + } + + override def next(): ColumnarBatch = { + val batch = batches.next() + numOutputRows += batch.numRows() + batch + } + } + } + } + + override val nodeNamePrefix: String = "Gpu" + wrapped.nodeNamePrefix + + override def doCanonicalize(): GpuFileSourceScanExec = { + val canonical = wrapped.doCanonicalize() + GpuFileSourceScanExec( + canonical.relation, + canonical.output, + canonical.requiredSchema, + canonical.partitionFilters, + canonical.optionalBucketSet, + canonical.dataFilters, + canonical.tableIdentifier) + } +} + +object GpuFileSourceScanExec31 { + def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + meta.wrapped.relation.fileFormat match { + case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) + case _: OrcFileFormat => GpuReadOrcFileFormat.tagSupport(meta) + case _: ParquetFileFormat => GpuReadParquetFileFormat.tagSupport(meta) + case f => + meta.willNotWorkOnGpu(s"unsupported file format: ${f.getClass.getCanonicalName}") + } + } + + def convertFileFormat(format: FileFormat): FileFormat = { + format match { + case _: CSVFileFormat => new GpuReadCSVFileFormat + case _: OrcFileFormat => new GpuReadOrcFileFormat + case _: ParquetFileFormat => new GpuReadParquetFileFormat + case f => + throw new IllegalArgumentException(s"${f.getClass.getCanonicalName} is not supported") + } + } +} From 8b6ec0a6b49797830d194dfff2bb6282dad0a746 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 16 Jul 2020 17:34:36 -0500 Subject: [PATCH 07/47] Fix order of params --- .../nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala | 2 +- .../nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala index 3c9e0b10d49..9dc519ad1ec 100644 --- a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala @@ -63,8 +63,8 @@ object GpuBroadcastNestedLoopJoinExec30 extends Logging { def createInstance( left: SparkPlan, right: SparkPlan, - joinType: JoinType, join: BroadcastNestedLoopJoinExec, + joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase= { /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala index df6c1c82065..244a3251e27 100644 --- a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala @@ -62,8 +62,8 @@ object GpuBroadcastNestedLoopJoinExec31 extends Logging { def createInstance( left: SparkPlan, right: SparkPlan, - joinType: JoinType, join: BroadcastNestedLoopJoinExec, + joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { From ad9f2e06f44d8f3ea17dbc49e6aec426ae227eaa Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 17 Jul 2020 08:00:03 -0500 Subject: [PATCH 08/47] remove logging --- .../scala/com/nvidia/spark/Spark30Shims.scala | 7 +---- .../rapids/GpuBroadcastHashJoinExec30.scala | 2 -- .../GpuBroadcastNestedLoopJoinExec30.scala | 13 +--------- .../rapids/GpuShuffledHashJoinExec30.scala | 4 --- .../sql/rapids/GpuFileSourceScanExec30.scala | 6 ++--- .../spark/GpuBroadcastHashJoinExec31.scala | 2 -- .../GpuBroadcastNestedLoopJoinExec31.scala | 14 +--------- .../rapids/GpuShuffledHashJoinExec31.scala | 16 ------------ .../spark/rapids/shims/Spark31Shims.scala | 7 +---- .../sql/rapids/GpuFileSourceScanExec31.scala | 6 ++--- .../nvidia/spark/rapids/GpuOverrides.scala | 26 ------------------- .../spark/rapids/GpuTransitionOverrides.scala | 4 +-- tests/pom.xml | 14 +++++----- 13 files changed, 19 insertions(+), 102 deletions(-) diff --git a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala index 7fd5df4d7a1..fdd126a89e4 100644 --- a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala +++ b/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.rapids._ import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ @@ -117,11 +118,9 @@ class Spark30Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") @@ -132,11 +131,9 @@ class Spark30Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("bnlje Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("bnlje Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") @@ -146,11 +143,9 @@ class Spark30Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("bnlje Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("bnlje Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala index 5e52c3e7127..0be24a2895b 100644 --- a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala @@ -49,8 +49,6 @@ case class GpuBroadcastHashJoinExec30( right: SparkPlan) extends GpuBroadcastHashJoinExecBase30 with Logging { - logWarning("Tom in hadh join exec build side is: " + buildSide) - def getBuildSide: GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala index 9dc519ad1ec..d76942763b1 100644 --- a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala @@ -47,8 +47,6 @@ case class GpuBroadcastNestedLoopJoinExec30( condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { - logWarning("Tom in broadcast nested loop join exec build side is: ") - def getBuildSide: GpuBuildSide = { join.buildSide match { case BuildRight => GpuBuildRight @@ -67,16 +65,7 @@ object GpuBroadcastNestedLoopJoinExec30 extends Logging { joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase= { - /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - logWarning("Tom in shuffled hash join") - join.asInstanceOf[ShuffledHashJoinExec].buildSide - } else { - logWarning("Tom in not shuffled hash join") - BuildRight - } */ - - val res = GpuBroadcastNestedLoopJoinExec30(left, right, join, joinType, condition) - res + GpuBroadcastNestedLoopJoinExec30(left, right, join, joinType, condition) } } diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala index f69a00975c9..0e7f9c911e5 100644 --- a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala +++ b/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala @@ -48,8 +48,6 @@ case class GpuShuffledHashJoinExec30( right: SparkPlan) extends GpuShuffledHashJoinExecBase30 with Logging { - logWarning("Tom in hadh join exec build side is: " + buildSide) - def getBuildSide: GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight @@ -71,10 +69,8 @@ object GpuShuffledHashJoinExec30 extends Logging { right: SparkPlan): GpuShuffledHashJoinExec30 = { val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - logWarning("Tom in shuffled hash join") join.asInstanceOf[ShuffledHashJoinExec].buildSide } else { - logWarning("Tom in not shuffled hash join") BuildRight } GpuShuffledHashJoinExec30(leftKeys, rightKeys, joinType, buildSide, condition, left, right) diff --git a/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala b/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala index fffca5b82c6..f8e142e8c4f 100644 --- a/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala +++ b/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala @@ -42,7 +42,7 @@ case class GpuFileSourceScanExec30( optionalBucketSet: Option[BitSet], dataFilters: Seq[Expression], override val tableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec with GpuExec { + extends DataSourceScanExec with GpuFileSourceScanExecBase with GpuExec { override val nodeName: String = { s"GpuScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" @@ -142,9 +142,9 @@ case class GpuFileSourceScanExec30( override val nodeNamePrefix: String = "Gpu" + wrapped.nodeNamePrefix - override def doCanonicalize(): GpuFileSourceScanExec = { + override def doCanonicalize(): GpuFileSourceScanExec30 = { val canonical = wrapped.doCanonicalize() - GpuFileSourceScanExec( + GpuFileSourceScanExec30( canonical.relation, canonical.output, canonical.requiredSchema, diff --git a/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala index 87913ed0d69..18969c838d2 100644 --- a/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala +++ b/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala @@ -48,8 +48,6 @@ case class GpuBroadcastHashJoinExec31( right: SparkPlan) extends GpuBroadcastHashJoinExecBase31 with Logging { - logWarning("Tom in hadh join exec build side is: " + buildSide) - def getBuildSide: GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala index 244a3251e27..eb31ea584a1 100644 --- a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala @@ -46,8 +46,6 @@ case class GpuBroadcastNestedLoopJoinExec31( condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { - logWarning("Tom in broadcast nested loop join exec build side is: ") - def getBuildSide: GpuBuildSide = { join.buildSide match { case BuildRight => GpuBuildRight @@ -66,17 +64,7 @@ object GpuBroadcastNestedLoopJoinExec31 extends Logging { joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { - /* val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - logWarning("Tom in shuffled hash join") - join.asInstanceOf[ShuffledHashJoinExec].buildSide - } else { - logWarning("Tom in not shuffled hash join") - BuildRight - } */ - - val res = GpuBroadcastNestedLoopJoinExec31(left, right, join, joinType, condition) - res + GpuBroadcastNestedLoopJoinExec31(left, right, join, joinType, condition) } - def createTom(): Unit = {} } diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala index 0922116f97f..697641854e5 100644 --- a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala @@ -46,20 +46,6 @@ case class GpuShuffledHashJoinExec31 ( left: SparkPlan, right: SparkPlan) extends GpuShuffledHashJoinExecBase31 with Logging { - - - /* - val buildSide: org.apache.spark.sql.execution.joins.BuildSide = { - logInfo("Tom gpu build side is: " + gpuBuildSide) - val res = gpuBuildSide match { - case GpuBuildRight => org.apache.spark.sql.execution.joins.BuildRight - case GpuBuildLeft => org.apache.spark.sql.execution.joins.BuildLeft - } - logInfo("Tom build side is: " + res) - res - } - */ - def getBuildSide: GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight @@ -81,10 +67,8 @@ object GpuShuffledHashJoinExec31 extends Logging { right: SparkPlan): GpuShuffledHashJoinExec31 = { val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - logWarning("Tom in shuffled hash join") join.asInstanceOf[ShuffledHashJoinExec].buildSide } else { - logWarning("Tom in not shuffled hash join") BuildRight } GpuShuffledHashJoinExec31(leftKeys, rightKeys, joinType, buildSide, condition, left, right) diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala index 8cfc1350e1e..8616ee903ea 100644 --- a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala +++ b/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.rapids._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ @@ -112,11 +113,9 @@ class Spark31Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") @@ -127,11 +126,9 @@ class Spark31Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("bnlje Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("bnlje Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") @@ -141,11 +138,9 @@ class Spark31Shims extends SparkShims with Logging { val buildSide = join.buildSide buildSide match { case e: buildSide.type if e.toString.contains("BuildRight") => { - logInfo("Tom buildright " + e) GpuBuildRight } case l: buildSide.type if l.toString.contains("BuildLeft") => { - logInfo("Tom buildleft "+ l) GpuBuildLeft } case _ => throw new Exception("unknown buildSide Type") diff --git a/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala b/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala index 358e65c2fdc..1c5af89232b 100644 --- a/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala +++ b/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala @@ -42,7 +42,7 @@ case class GpuFileSourceScanExec31( optionalBucketSet: Option[BitSet], dataFilters: Seq[Expression], override val tableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec with GpuExec { + extends DataSourceScanExec with GpuFileSourceScanExecBase with GpuExec { override val nodeName: String = { s"GpuScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" @@ -142,9 +142,9 @@ case class GpuFileSourceScanExec31( override val nodeNamePrefix: String = "Gpu" + wrapped.nodeNamePrefix - override def doCanonicalize(): GpuFileSourceScanExec = { + override def doCanonicalize(): GpuFileSourceScanExec31 = { val canonical = wrapped.doCanonicalize() - GpuFileSourceScanExec( + GpuFileSourceScanExec31( canonical.relation, canonical.output, canonical.requiredSchema, 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 760bf6f19a6..e44725129b1 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 @@ -1659,32 +1659,6 @@ object GpuOverrides { GpuDataWritingCommandExec(childDataWriteCmds.head.convertToGpu(), childPlans.head.convertIfNeeded()) }), - exec[FileSourceScanExec]( - "Reading data from files, often from Hive tables", - (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { - // partition filters and data filters are not run on the GPU - override val childExprs: Seq[ExprMeta[_]] = Seq.empty - - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) - - override def convertToGpu(): GpuExec = { - val newRelation = HadoopFsRelation( - wrapped.relation.location, - wrapped.relation.partitionSchema, - wrapped.relation.dataSchema, - wrapped.relation.bucketSpec, - GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), - wrapped.relation.options)(wrapped.relation.sparkSession) - GpuFileSourceScanExec( - newRelation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.dataFilters, - wrapped.tableIdentifier) - } - }), exec[LocalLimitExec]( "Per-partition limiting of results", (localLimitExec, conf, p, r) => 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 5e21ef69dc9..a1e3ef405c5 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} -import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase /** * Rules that run after the row to columnar and columnar to row transitions have been inserted. @@ -252,7 +252,7 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { val planOutput = plan.output.toSet // avoid checking expressions of GpuFileSourceScanExec since all expressions are // processed by driver and not run on GPU. - if (!plan.isInstanceOf[GpuFileSourceScanExec]) { + if (!plan.isInstanceOf[GpuFileSourceScanExecBase]) { plan.expressions.filter(_ match { case a: Attribute => !planOutput.contains(a) case _ => true diff --git a/tests/pom.xml b/tests/pom.xml index 89a0239dc5a..aa5987a60ef 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -33,11 +33,16 @@ spark31 + + + + + - 3.1.0-SNAPSHOT + 3.0.0 1.7.30 - + org.slf4j jul-to-slf4j @@ -49,11 +54,6 @@ ${slf4j.version} - - - - - org.scala-lang scala-library From c48e9185bebaa69e7c046027737f8d2c356d65fd Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 17 Jul 2020 14:36:15 -0500 Subject: [PATCH 09/47] move spark30 and spark31 into shims modules --- dist/pom.xml | 7 +------ pom.xml | 3 +-- {spark30 => shims/spark30}/pom.xml | 2 +- .../src/main/scala/com/nvidia/spark/Spark30Shims.scala | 0 .../nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala | 0 .../spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala | 0 .../main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala | 0 .../nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala | 0 .../com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala | 0 .../rapids/shims/GpuBroadcastHashJoinExecBase30.scala | 0 .../spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala | 0 .../apache/spark/sql/rapids/GpuFileSourceScanExec30.scala | 0 {spark31 => shims/spark31}/pom.xml | 2 +- .../com/nvidia/spark/GpuBroadcastHashJoinExec31.scala | 0 .../spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala | 0 .../main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala | 0 .../nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala | 0 .../com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala | 0 .../rapids/shims/GpuBroadcastHashJoinExecBase31.scala | 0 .../spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala | 0 .../scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala | 0 .../apache/spark/sql/rapids/GpuFileSourceScanExec31.scala | 0 22 files changed, 4 insertions(+), 10 deletions(-) rename {spark30 => shims/spark30}/pom.xml (98%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/Spark30Shims.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala (100%) rename {spark30 => shims/spark30}/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala (100%) rename {spark31 => shims/spark31}/pom.xml (98%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala (100%) rename {spark31 => shims/spark31}/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala (100%) diff --git a/dist/pom.xml b/dist/pom.xml index 28c4d1d68db..46f57d49cdd 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -43,12 +43,7 @@ com.nvidia - rapids-4-spark-shims-spark30_${scala.binary.version} - ${project.version} - - - com.nvidia - rapids-4-spark-shims-spark31_${scala.binary.version} + rapids-4-spark-shims_${scala.binary.version} ${project.version} diff --git a/pom.xml b/pom.xml index cd010307250..d2fabe10475 100644 --- a/pom.xml +++ b/pom.xml @@ -76,8 +76,7 @@ sql-plugin tests integration_tests - spark30 - spark31 + shims api_validation diff --git a/spark30/pom.xml b/shims/spark30/pom.xml similarity index 98% rename from spark30/pom.xml rename to shims/spark30/pom.xml index ae962cd0063..cad9301f312 100644 --- a/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -23,7 +23,7 @@ com.nvidia rapids-4-spark-parent 0.2.0-SNAPSHOT - ../pom.xml + ../../pom.xml com.nvidia rapids-4-spark-shims-spark30_2.12 diff --git a/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala diff --git a/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala similarity index 100% rename from spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala diff --git a/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala similarity index 100% rename from spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala rename to shims/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala diff --git a/spark31/pom.xml b/shims/spark31/pom.xml similarity index 98% rename from spark31/pom.xml rename to shims/spark31/pom.xml index 313f45a1ed2..58d82f21997 100644 --- a/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -23,7 +23,7 @@ com.nvidia rapids-4-spark-parent 0.2.0-SNAPSHOT - ../pom.xml + ../../pom.xml com.nvidia rapids-4-spark-shims-spark31_2.12 diff --git a/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala diff --git a/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala similarity index 100% rename from spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala diff --git a/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala similarity index 100% rename from spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala rename to shims/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala From c98b38e67d71ded34a8c83740dcd815177a08ef2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 17 Jul 2020 14:37:17 -0500 Subject: [PATCH 10/47] Add missing files --- shims/aggregator/pom.xml | 55 +++++++++++++++++++ shims/pom.xml | 49 +++++++++++++++++ .../rapids/GpuFileSourceScanExecBase.scala | 38 +++++++++++++ 3 files changed, 142 insertions(+) create mode 100644 shims/aggregator/pom.xml create mode 100644 shims/pom.xml create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala diff --git a/shims/aggregator/pom.xml b/shims/aggregator/pom.xml new file mode 100644 index 00000000000..aa9dc84a780 --- /dev/null +++ b/shims/aggregator/pom.xml @@ -0,0 +1,55 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-parent + 0.2.0-SNAPSHOT + ../../pom.xml + + com.nvidia + rapids-4-spark-shims_2.12 + jar + RAPIDS Accelerator for Apache Spark SQL Plugin Shim Aggregator + The RAPIDS SQL plugin for Apache Spark Shim Aggregator + 0.2.0-SNAPSHOT + + + + databricks + + + + + + com.nvidia + rapids-4-spark-shims-spark31_${scala.binary.version} + ${project.version} + compile + + + com.nvidia + rapids-4-spark-shims-spark30_${scala.binary.version} + ${project.version} + compile + + + diff --git a/shims/pom.xml b/shims/pom.xml new file mode 100644 index 00000000000..5d18156cba0 --- /dev/null +++ b/shims/pom.xml @@ -0,0 +1,49 @@ + + + + 4.0.0 + + + com.nvidia + rapids-4-spark-parent + 0.2.0-SNAPSHOT + ../pom.xml + + com.nvidia + rapids-4-spark-shims_aggregator_2.12 + pom + RAPIDS Accelerator for Apache Spark SQL Plugin Shims + The RAPIDS SQL plugin for Apache Spark Shims + 0.2.0-SNAPSHOT + + + + databricks + + databricks + + + + + + spark30 + spark31 + aggregator + + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala new file mode 100644 index 00000000000..26e82195221 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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 + +import java.util.concurrent.TimeUnit.NANOSECONDS + +import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{DataSourceScanExec, ExplainUtils, FileSourceScanExec} +import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.collection.BitSet + +trait GpuFileSourceScanExecBase { +} From 918cbd46a9c05d41efd25f574a85911799f52ae2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Jul 2020 10:06:25 -0500 Subject: [PATCH 11/47] Move packages and use serviceloader --- .../com.nvidia.spark.rapids.SparkShimLoader | 1 + .../spark30/GpuBroadcastHashJoinExec.scala} | 12 +++---- .../GpuBroadcastHashJoinExecBase.scala} | 10 +++--- .../GpuBroadcastNestedLoopJoinExec.scala} | 8 ++--- .../{ => shims/spark30}/GpuHashJoin.scala | 7 ++-- .../spark30/GpuShuffledHashJoinExec.scala} | 12 +++---- .../GpuShuffledHashJoinExecBase.scala} | 10 +++--- .../spark30/GpuSortMergeJoinExec.scala} | 8 ++--- .../shims/spark30/Spark30ShimLoader.scala | 35 +++++++++++++++++++ .../shims/spark30}/Spark30Shims.scala | 19 +++++----- .../spark30/GpuFileSourceScanExec.scala} | 11 +++--- .../com.nvidia.spark.rapids.SparkShimLoader | 1 + .../spark31/GpuBroadcastHashJoinExec.scala} | 12 +++---- .../GpuBroadcastHashJoinExecBase.scala} | 10 +++--- .../{ => shims/spark31}/GpuHashJoin.scala | 7 ++-- .../spark31/GpuShuffledHashJoinExec.scala} | 12 +++---- .../GpuShuffledHashJoinExecBase.scala} | 10 +++--- .../spark31/GpuSortMergeJoinExec.scala} | 8 ++--- .../shims/spark31/Spark31ShimLoader.scala | 34 ++++++++++++++++++ .../shims/{ => spark31}/Spark31Shims.scala | 19 +++++----- .../spark31/GpuFileSourceScanExec.scala} | 11 +++--- .../com/nvidia/spark/rapids/ShimLoader.scala | 26 +++++++++++--- .../nvidia/spark/rapids/SparkShimLoader.scala | 22 ++++++++++++ .../sql/rapids/datetimeExpressions.scala | 1 + tests/pom.xml | 34 +++++++++--------- 25 files changed, 228 insertions(+), 112 deletions(-) create mode 100644 shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/{GpuBroadcastHashJoinExec30.scala => shims/spark30/GpuBroadcastHashJoinExec.scala} (86%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/{GpuBroadcastHashJoinExecBase30.scala => spark30/GpuBroadcastHashJoinExecBase.scala} (96%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/{GpuBroadcastNestedLoopJoinExec30.scala => shims/spark30/GpuBroadcastNestedLoopJoinExec.scala} (91%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/{ => shims/spark30}/GpuHashJoin.scala (97%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/{GpuShuffledHashJoinExec30.scala => shims/spark30/GpuShuffledHashJoinExec.scala} (87%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/{GpuShuffledHashJoinExecBase30.scala => spark30/GpuShuffledHashJoinExecBase.scala} (95%) rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/{GpuSortMergeJoinExec30.scala => shims/spark30/GpuSortMergeJoinExec.scala} (94%) create mode 100644 shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala rename shims/spark30/src/main/scala/com/nvidia/spark/{ => rapids/shims/spark30}/Spark30Shims.scala (90%) rename shims/{spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala => spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala} (96%) create mode 100644 shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader rename shims/spark31/src/main/scala/com/nvidia/spark/{GpuBroadcastHashJoinExec31.scala => rapids/shims/spark31/GpuBroadcastHashJoinExec.scala} (86%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/{GpuBroadcastHashJoinExecBase31.scala => spark31/GpuBroadcastHashJoinExecBase.scala} (96%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/{ => shims/spark31}/GpuHashJoin.scala (97%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/{GpuShuffledHashJoinExec31.scala => shims/spark31/GpuShuffledHashJoinExec.scala} (86%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/{GpuShuffledHashJoinExecBase31.scala => spark31/GpuShuffledHashJoinExecBase.scala} (95%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/{GpuSortMergeJoinExec31.scala => shims/spark31/GpuSortMergeJoinExec.scala} (94%) create mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/{ => spark31}/Spark31Shims.scala (90%) rename shims/{spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala => spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala} (96%) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala diff --git a/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader b/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader new file mode 100644 index 00000000000..0009a24af47 --- /dev/null +++ b/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader @@ -0,0 +1 @@ +com.nvidia.spark.rapids.shims.spark30.Spark30ShimLoader diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala similarity index 86% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 0be24a2895b..52b08df0789 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -39,14 +39,14 @@ import org.apache.spark.internal.Logging -case class GpuBroadcastHashJoinExec30( +case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuBroadcastHashJoinExecBase30 with Logging { + right: SparkPlan) extends GpuBroadcastHashJoinExecBase with Logging { def getBuildSide: GpuBuildSide = { @@ -58,7 +58,7 @@ case class GpuBroadcastHashJoinExec30( } } -object GpuBroadcastHashJoinExec30 extends Logging { +object GpuBroadcastHashJoinExec extends Logging { def createInstance( leftKeys: Seq[Expression], @@ -67,9 +67,9 @@ object GpuBroadcastHashJoinExec30 extends Logging { join: BroadcastHashJoinExec, condition: Option[Expression], left: SparkPlan, - right: SparkPlan): GpuBroadcastHashJoinExec30 = { + right: SparkPlan): GpuBroadcastHashJoinExec = { - GpuBroadcastHashJoinExec30(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) + GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala similarity index 96% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala index ae4d2f85794..b1bd9f096f6 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import ai.rapids.cudf.{NvtxColor, Table} @@ -36,7 +36,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging -abstract class GpuBroadcastHashJoinExecBase30 extends BinaryExecNode with GpuHashJoin30 with Logging { +abstract class GpuBroadcastHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { def getBuildSide: GpuBuildSide @@ -156,7 +156,7 @@ abstract class GpuBroadcastHashJoinExecBase30 extends BinaryExecNode with GpuHas } -class GpuBroadcastHashJoinMeta30( +class GpuBroadcastHashJoinMeta( join: BroadcastHashJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -175,7 +175,7 @@ class GpuBroadcastHashJoinMeta30( } override def tagPlanForGpu(): Unit = { - GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) val buildSide = getBuildSide(join) match { case GpuBuildLeft => childPlans(0) @@ -202,7 +202,7 @@ class GpuBroadcastHashJoinMeta30( if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") } - GpuBroadcastHashJoinExec30( + GpuBroadcastHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, join.buildSide, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala similarity index 91% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala index d76942763b1..6df1a5c02e3 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging -case class GpuBroadcastNestedLoopJoinExec30( +case class GpuBroadcastNestedLoopJoinExec( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, @@ -56,7 +56,7 @@ case class GpuBroadcastNestedLoopJoinExec30( } } -object GpuBroadcastNestedLoopJoinExec30 extends Logging { +object GpuBroadcastNestedLoopJoinExec extends Logging { def createInstance( left: SparkPlan, @@ -65,7 +65,7 @@ object GpuBroadcastNestedLoopJoinExec30 extends Logging { joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase= { - GpuBroadcastNestedLoopJoinExec30(left, right, join, joinType, condition) + GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala similarity index 97% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 79f97323cdc..577a6db140e 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -13,9 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.nvidia.spark.rapids +package com.nvidia.spark.rapids.shims.spark30 import ai.rapids.cudf.{NvtxColor, Table} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -26,7 +27,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -object GpuHashJoin30 { +object GpuHashJoin { def tagJoin( meta: RapidsMeta[_, _, _], joinType: JoinType, @@ -51,7 +52,7 @@ object GpuHashJoin30 { } -trait GpuHashJoin30 extends GpuExec with HashJoin with Logging { +trait GpuHashJoin extends GpuExec with HashJoin with Logging { override def output: Seq[Attribute] = { diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala similarity index 87% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index 0e7f9c911e5..dcce05ee7e7 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -38,14 +38,14 @@ import org.apache.spark.internal.Logging -case class GpuShuffledHashJoinExec30( +case class GpuShuffledHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuShuffledHashJoinExecBase30 with Logging { + right: SparkPlan) extends GpuShuffledHashJoinExecBase with Logging { def getBuildSide: GpuBuildSide = { @@ -57,7 +57,7 @@ case class GpuShuffledHashJoinExec30( } } -object GpuShuffledHashJoinExec30 extends Logging { +object GpuShuffledHashJoinExec extends Logging { def createInstance( leftKeys: Seq[Expression], @@ -66,14 +66,14 @@ object GpuShuffledHashJoinExec30 extends Logging { join: SparkPlan, condition: Option[Expression], left: SparkPlan, - right: SparkPlan): GpuShuffledHashJoinExec30 = { + right: SparkPlan): GpuShuffledHashJoinExec = { val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { join.asInstanceOf[ShuffledHashJoinExec].buildSide } else { BuildRight } - GpuShuffledHashJoinExec30(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + GpuShuffledHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala similarity index 95% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala index d73592ac045..a3e929b5df3 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import ai.rapids.cudf.{NvtxColor, Table} @@ -35,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging -abstract class GpuShuffledHashJoinExecBase30 extends BinaryExecNode with GpuHashJoin30 with Logging { +abstract class GpuShuffledHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { def getBuildSide: GpuBuildSide @@ -168,7 +168,7 @@ abstract class GpuShuffledHashJoinExecBase30 extends BinaryExecNode with GpuHash } -class GpuShuffledHashJoinMeta30( +class GpuShuffledHashJoinMeta( join: ShuffledHashJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -183,11 +183,11 @@ class GpuShuffledHashJoinMeta30( GpuOverrides.wrapExpr(_, conf, Some(this))) override def tagPlanForGpu(): Unit = { - GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec30( + GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala similarity index 94% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index 4206ed561d4..b1aefae869a 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec30.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -38,7 +38,7 @@ import org.apache.spark.internal.Logging -class GpuSortMergeJoinMeta30( +class GpuSortMergeJoinMeta( join: SortMergeJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -55,7 +55,7 @@ class GpuSortMergeJoinMeta30( override def tagPlanForGpu(): Unit = { // Use conditions from Hash Join - GpuHashJoin30.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) if (!conf.enableReplaceSortMergeJoin) { willNotWorkOnGpu(s"Not replacing sort merge join with hash join, " + @@ -80,7 +80,7 @@ class GpuSortMergeJoinMeta30( } } override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec30( + GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala new file mode 100644 index 00000000000..9b302abd6ec --- /dev/null +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark30 + +import com.nvidia.spark.rapids._ + + +class Spark30ShimLoader extends SparkShimLoader { + + val SPARK30VERSIONNAME = "3.0.0" + + def matchesVersion(version: String): Boolean = { + version == SPARK30VERSIONNAME + } + + def buildShim: SparkShims = { + new Spark30Shims() + } + +} + diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala similarity index 90% rename from shims/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index fdd126a89e4..775420e11c6 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -14,12 +14,13 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark30 import java.time.ZoneId import com.nvidia.spark.rapids._ -import org.apache.spark.sql.rapids._ +import org.apache.spark.sql.rapids.GpuTimeSub +import org.apache.spark.sql.rapids.shims.spark30._ //import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} @@ -38,7 +39,7 @@ class Spark30Shims extends SparkShims with Logging { def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { - case _: GpuHashJoin30 => true + case _: GpuHashJoin => true case p => false } } @@ -52,7 +53,7 @@ class Spark30Shims extends SparkShims with Logging { // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec30.tagSupport(this) + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) override def convertToGpu(): GpuExec = { val newRelation = HadoopFsRelation( @@ -60,9 +61,9 @@ class Spark30Shims extends SparkShims with Logging { wrapped.relation.partitionSchema, wrapped.relation.dataSchema, wrapped.relation.bucketSpec, - GpuFileSourceScanExec30.convertFileFormat(wrapped.relation.fileFormat), + GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), wrapped.relation.options)(wrapped.relation.sparkSession) - GpuFileSourceScanExec30( + GpuFileSourceScanExec( newRelation, wrapped.output, wrapped.requiredSchema, @@ -74,13 +75,13 @@ class Spark30Shims extends SparkShims with Logging { }), GpuOverrides.exec[SortMergeJoinExec]( "Sort merge join, replacing with shuffled hash join", - (join, conf, p, r) => new GpuSortMergeJoinMeta30(join, conf, p, r)), + (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), GpuOverrides.exec[BroadcastHashJoinExec]( "Implementation of join using broadcast data", - (join, conf, p, r) => new GpuBroadcastHashJoinMeta30(join, conf, p, r)), + (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), GpuOverrides.exec[ShuffledHashJoinExec]( "Implementation of join using hashed shuffled data", - (join, conf, p, r) => new GpuShuffledHashJoinMeta30(join, conf, p, r)), + (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), ) } diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala similarity index 96% rename from shims/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala rename to shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala index 1c5af89232b..bdfe0766b2c 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec31.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala @@ -14,11 +14,12 @@ * limitations under the License. */ -package org.apache.spark.sql.rapids +package org.apache.spark.sql.rapids.shims.spark30 import java.util.concurrent.TimeUnit.NANOSECONDS import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} +import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} @@ -34,7 +35,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet -case class GpuFileSourceScanExec31( +case class GpuFileSourceScanExec( @transient relation: HadoopFsRelation, output: Seq[Attribute], requiredSchema: StructType, @@ -142,9 +143,9 @@ case class GpuFileSourceScanExec31( override val nodeNamePrefix: String = "Gpu" + wrapped.nodeNamePrefix - override def doCanonicalize(): GpuFileSourceScanExec31 = { + override def doCanonicalize(): GpuFileSourceScanExec = { val canonical = wrapped.doCanonicalize() - GpuFileSourceScanExec31( + GpuFileSourceScanExec( canonical.relation, canonical.output, canonical.requiredSchema, @@ -155,7 +156,7 @@ case class GpuFileSourceScanExec31( } } -object GpuFileSourceScanExec31 { +object GpuFileSourceScanExec { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { meta.wrapped.relation.fileFormat match { case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) diff --git a/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader b/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader new file mode 100644 index 00000000000..a131979d004 --- /dev/null +++ b/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader @@ -0,0 +1 @@ +com.nvidia.spark.rapids.shims.spark31.Spark31ShimLoader diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala similarity index 86% rename from shims/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 18969c838d2..43625ea512e 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/GpuBroadcastHashJoinExec31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -38,14 +38,14 @@ import org.apache.spark.internal.Logging -case class GpuBroadcastHashJoinExec31( +case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuBroadcastHashJoinExecBase31 with Logging { + right: SparkPlan) extends GpuBroadcastHashJoinExecBase with Logging { def getBuildSide: GpuBuildSide = { @@ -57,7 +57,7 @@ case class GpuBroadcastHashJoinExec31( } } -object GpuBroadcastHashJoinExec31 extends Logging { +object GpuBroadcastHashJoinExec extends Logging { def createInstance( leftKeys: Seq[Expression], @@ -66,9 +66,9 @@ object GpuBroadcastHashJoinExec31 extends Logging { join: BroadcastHashJoinExec, condition: Option[Expression], left: SparkPlan, - right: SparkPlan): GpuBroadcastHashJoinExec31 = { + right: SparkPlan): GpuBroadcastHashJoinExec = { - GpuBroadcastHashJoinExec31(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) + GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala similarity index 96% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala index bb26bffdd14..4d1319f6c12 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuBroadcastHashJoinExecBase31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} @@ -36,7 +36,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging -abstract class GpuBroadcastHashJoinExecBase31 extends BinaryExecNode with GpuHashJoin31 with Logging { +abstract class GpuBroadcastHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { def getBuildSide: GpuBuildSide @@ -156,7 +156,7 @@ abstract class GpuBroadcastHashJoinExecBase31 extends BinaryExecNode with GpuHas } -class GpuBroadcastHashJoinMeta31( +class GpuBroadcastHashJoinMeta( join: BroadcastHashJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -175,7 +175,7 @@ class GpuBroadcastHashJoinMeta31( } override def tagPlanForGpu(): Unit = { - GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) val buildSide = getBuildSide(join) match { case GpuBuildLeft => childPlans(0) @@ -202,7 +202,7 @@ class GpuBroadcastHashJoinMeta31( if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") } - GpuBroadcastHashJoinExec31( + GpuBroadcastHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, join.buildSide, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala similarity index 97% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index 085540fd673..7d6027faffc 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -13,9 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.nvidia.spark.rapids +package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -26,7 +27,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -object GpuHashJoin31 { +object GpuHashJoin { def tagJoin( meta: RapidsMeta[_, _, _], joinType: JoinType, @@ -51,7 +52,7 @@ object GpuHashJoin31 { } -trait GpuHashJoin31 extends GpuExec with HashJoin with Logging { +trait GpuHashJoin extends GpuExec with HashJoin with Logging { override def output: Seq[Attribute] = { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala similarity index 86% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index 697641854e5..ad5c6b4c004 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -37,14 +37,14 @@ import org.apache.spark.internal.Logging -case class GpuShuffledHashJoinExec31 ( +case class GpuShuffledHashJoinExec ( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuShuffledHashJoinExecBase31 with Logging { + right: SparkPlan) extends GpuShuffledHashJoinExecBase with Logging { def getBuildSide: GpuBuildSide = { buildSide match { @@ -55,7 +55,7 @@ case class GpuShuffledHashJoinExec31 ( } } -object GpuShuffledHashJoinExec31 extends Logging { +object GpuShuffledHashJoinExec extends Logging { def createInstance( leftKeys: Seq[Expression], @@ -64,14 +64,14 @@ object GpuShuffledHashJoinExec31 extends Logging { join: SparkPlan, condition: Option[Expression], left: SparkPlan, - right: SparkPlan): GpuShuffledHashJoinExec31 = { + right: SparkPlan): GpuShuffledHashJoinExec = { val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { join.asInstanceOf[ShuffledHashJoinExec].buildSide } else { BuildRight } - GpuShuffledHashJoinExec31(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + GpuShuffledHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala similarity index 95% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala index 7342f913053..0a070ddba0f 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/GpuShuffledHashJoinExecBase31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} @@ -35,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging -abstract class GpuShuffledHashJoinExecBase31 extends BinaryExecNode with GpuHashJoin31 with Logging { +abstract class GpuShuffledHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { def getBuildSide: GpuBuildSide @@ -169,7 +169,7 @@ abstract class GpuShuffledHashJoinExecBase31 extends BinaryExecNode with GpuHash } -class GpuShuffledHashJoinMeta31( +class GpuShuffledHashJoinMeta( join: ShuffledHashJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -184,11 +184,11 @@ class GpuShuffledHashJoinMeta31( GpuOverrides.wrapExpr(_, conf, Some(this))) override def tagPlanForGpu(): Unit = { - GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec31( + GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala similarity index 94% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala index 05e80d4f8c6..fffef3e5e74 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuSortMergeJoinExec31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ @@ -37,7 +37,7 @@ import org.apache.spark.internal.Logging -class GpuSortMergeJoinMeta31( +class GpuSortMergeJoinMeta( join: SortMergeJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], @@ -54,7 +54,7 @@ class GpuSortMergeJoinMeta31( override def tagPlanForGpu(): Unit = { // Use conditions from Hash Join - GpuHashJoin31.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) if (!conf.enableReplaceSortMergeJoin) { willNotWorkOnGpu(s"Not replacing sort merge join with hash join, " + @@ -79,7 +79,7 @@ class GpuSortMergeJoinMeta31( } } override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec31( + GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala new file mode 100644 index 00000000000..1c048ffcca8 --- /dev/null +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark31 + +import com.nvidia.spark.rapids._ + +class Spark31ShimLoader extends SparkShimLoader { + + val SPARK31VERSIONNAME = "3.1.0" + + def matchesVersion(version: String): Boolean = { + version == SPARK31VERSIONNAME + } + + def buildShim: SparkShims = { + new Spark31Shims() + } + +} + diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala similarity index 90% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 8616ee903ea..1a2f02b9193 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -14,12 +14,13 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import java.time.ZoneId import com.nvidia.spark.rapids._ -import org.apache.spark.sql.rapids._ +import org.apache.spark.sql.rapids.GpuTimeSub +import org.apache.spark.sql.rapids.shims.spark31._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging @@ -36,7 +37,7 @@ class Spark31Shims extends SparkShims with Logging { def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { - case _: GpuHashJoin31 => true + case _: GpuHashJoin => true case p => false } } @@ -50,7 +51,7 @@ class Spark31Shims extends SparkShims with Logging { // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec31.tagSupport(this) + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) override def convertToGpu(): GpuExec = { val newRelation = HadoopFsRelation( @@ -58,9 +59,9 @@ class Spark31Shims extends SparkShims with Logging { wrapped.relation.partitionSchema, wrapped.relation.dataSchema, wrapped.relation.bucketSpec, - GpuFileSourceScanExec31.convertFileFormat(wrapped.relation.fileFormat), + GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), wrapped.relation.options)(wrapped.relation.sparkSession) - GpuFileSourceScanExec31( + GpuFileSourceScanExec( newRelation, wrapped.output, wrapped.requiredSchema, @@ -72,13 +73,13 @@ class Spark31Shims extends SparkShims with Logging { }), GpuOverrides.exec[SortMergeJoinExec]( "Sort merge join, replacing with shuffled hash join", - (join, conf, p, r) => new GpuSortMergeJoinMeta31(join, conf, p, r)), + (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), GpuOverrides.exec[BroadcastHashJoinExec]( "Implementation of join using broadcast data", - (join, conf, p, r) => new GpuBroadcastHashJoinMeta31(join, conf, p, r)), + (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), GpuOverrides.exec[ShuffledHashJoinExec]( "Implementation of join using hashed shuffled data", - (join, conf, p, r) => new GpuShuffledHashJoinMeta31(join, conf, p, r)), + (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), ) } diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala similarity index 96% rename from shims/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala rename to shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala index f8e142e8c4f..53ba2fff21f 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec30.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala @@ -14,11 +14,12 @@ * limitations under the License. */ -package org.apache.spark.sql.rapids +package org.apache.spark.sql.rapids.shims.spark31 import java.util.concurrent.TimeUnit.NANOSECONDS import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} +import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} @@ -34,7 +35,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet -case class GpuFileSourceScanExec30( +case class GpuFileSourceScanExec( @transient relation: HadoopFsRelation, output: Seq[Attribute], requiredSchema: StructType, @@ -142,9 +143,9 @@ case class GpuFileSourceScanExec30( override val nodeNamePrefix: String = "Gpu" + wrapped.nodeNamePrefix - override def doCanonicalize(): GpuFileSourceScanExec30 = { + override def doCanonicalize(): GpuFileSourceScanExec = { val canonical = wrapped.doCanonicalize() - GpuFileSourceScanExec30( + GpuFileSourceScanExec( canonical.relation, canonical.output, canonical.requiredSchema, @@ -155,7 +156,7 @@ case class GpuFileSourceScanExec30( } } -object GpuFileSourceScanExec30 { +object GpuFileSourceScanExec { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { meta.wrapped.relation.fileFormat match { case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index f94f2703386..7fdfc56e595 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids +import java.util.ServiceLoader +import scala.collection.JavaConverters._ import scala.collection.immutable.HashMap import org.apache.spark.sql.catalyst.expressions.Expression @@ -27,11 +29,24 @@ import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} object ShimLoader { + // This is no ideal but pass the version in here because otherwise loader that match the + // same version (3.0.0 Apache and 3.0.0 Databricks) would need to know how to differentiate. + val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimLoader]) + .asScala.filter(_.matchesVersion(getVersion)) + if (sparkShimLoaders.size > 1) { + throw new IllegalArgumentException(s"Multiple Spark Shim Loaders found: $sparkShimLoaders") + } + val loader = sparkShimLoaders.headOption match { + case Some(loader) => loader + case None => throw new IllegalArgumentException("Could not find Spark Shim Loader") + } + private val sparkShims: SparkShims = loader.buildShim + + val SPARK30DATABRICKSSVERSIONNAME = "3.0.0-databricks" val SPARK30VERSIONNAME = "3.0.0" val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" - private var sparkShims: SparkShims = null private var gpuBroadcastNestedJoinShims: GpuBroadcastNestedLoopJoinExecBase = null /** @@ -49,15 +64,16 @@ object ShimLoader { */ def getSparkShims: SparkShims = { if (sparkShims == null) { - sparkShims = loadShims(SPARK_SHIM_CLASSES, classOf[SparkShims]) + + // sparkShims = loadShims(SPARK_SHIM_CLASSES, classOf[SparkShims]) } sparkShims } private val BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES = HashMap( - SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec30", - SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec300Databricks", - SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec31", + SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30.GpuBroadcastNestedLoopJoinExec", + SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30databricks.GpuBroadcastNestedLoopJoinExec", + SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.spark31.GpuBroadcastNestedLoopJoinExec", ) def getGpuBroadcastNestedLoopJoinShims( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala new file mode 100644 index 00000000000..41a873456f1 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala @@ -0,0 +1,22 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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 + +trait SparkShimLoader { + def matchesVersion(version:String): Boolean + def buildShim: SparkShims +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 51a17d7715f..7b587a600d0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -1,4 +1,5 @@ /* + * rg.apache.spark.sql.rapids * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/tests/pom.xml b/tests/pom.xml index aa5987a60ef..f7638744614 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -30,30 +30,30 @@ RAPIDS plugin for Apache Spark integration tests 0.2.0-SNAPSHOT + + 1.7.30 + spark31 - - + + 3.1.0 + - - 3.0.0 - 1.7.30 - - - org.slf4j - jul-to-slf4j - ${slf4j.version} - - - org.slf4j - jcl-over-slf4j - ${slf4j.version} - - + + org.slf4j + jul-to-slf4j + ${slf4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + org.scala-lang scala-library From bcd5eb013fb40ed370f9b6b18bb9b57a14980746 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Jul 2020 11:20:20 -0500 Subject: [PATCH 12/47] Move GpuFirst to shim --- .../spark/rapids/shims/spark30/GpuFirst.scala | 50 ++++++++++++++ .../rapids/shims/spark30/Spark30Shims.scala | 15 ++++- .../spark/rapids/shims/spark31/GpuFirst.scala | 40 +++++++++++ .../shims/spark31/Spark31ShimLoader.scala | 2 +- .../rapids/shims/spark31/Spark31Shims.scala | 66 +++++++++++-------- .../nvidia/spark/rapids/GpuOverrides.scala | 11 ---- .../spark/sql/rapids/AggregateFunctions.scala | 23 ++----- tests/pom.xml | 3 +- 8 files changed, 148 insertions(+), 62 deletions(-) create mode 100644 shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala create mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala new file mode 100644 index 00000000000..b0f4893f25a --- /dev/null +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala @@ -0,0 +1,50 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark30 + +import org.apache.spark.sql.rapids.GpuFirstBase +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +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.{Expression, Literal} + + +case class GpuFirst(child: Expression, ignoreNullsExpr: Expression) extends GpuFirstBase(child) { + override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + + override val ignoreNulls: Boolean = ignoreNullsExpr match { + case l: Literal => l.value.asInstanceOf[Boolean] + case l: GpuLiteral => l.value.asInstanceOf[Boolean] + case _ => throw new IllegalArgumentException( + s"$this should only receive literals for ignoreNulls expression") + } + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else if (!ignoreNullsExpr.foldable) { + TypeCheckFailure(s"The second argument of GpuFirst must be a boolean literal, but " + + s"got: ${ignoreNullsExpr.sql}") + } else { + TypeCheckSuccess + } + } +} + diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 775420e11c6..3d9d664fdd2 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark30._ -//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.expressions.aggregate.First import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -33,8 +33,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} - - class Spark30Shims extends SparkShims with Logging { def isGpuHashJoin(plan: SparkPlan): Boolean = { @@ -111,6 +109,17 @@ class Spark30Shims extends SparkShims with Logging { } } ), + GpuOverrides.expr[First]( + "first aggregate operator", + (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) + + override def convertToGpu(): GpuExpression = + GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) + }), ) } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala new file mode 100644 index 00000000000..342a3e24c05 --- /dev/null +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark31 + +import org.apache.spark.sql.rapids.GpuFirstBase +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +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.Expression + + +case class GpuFirst(child: Expression, ignoreNulls: Boolean) extends GpuFirstBase(child) { + override def children: Seq[Expression] = child :: Nil + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else { + TypeCheckSuccess + } + } +} + diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala index 1c048ffcca8..f63cd2b9359 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala @@ -20,7 +20,7 @@ import com.nvidia.spark.rapids._ class Spark31ShimLoader extends SparkShimLoader { - val SPARK31VERSIONNAME = "3.1.0" + val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" def matchesVersion(version: String): Boolean = { version == SPARK31VERSIONNAME diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 1a2f02b9193..89a2cfef9b1 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -22,6 +22,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark31._ +import org.apache.spark.sql.catalyst.expressions.aggregate.First import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -42,6 +43,44 @@ class Spark31Shims extends SparkShims with Logging { } } + def getExprs: Seq[ExprRule[_ <: Expression]] = { + Seq( + + GpuOverrides.expr[TimeAdd]( + "Subtracts interval from timestamp", + (a, conf, p, r) => new BinaryExprMeta[TimeAdd](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.interval match { + case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => + if (intvl.months != 0) { + willNotWorkOnGpu("interval months isn't supported") + } + case _ => + willNotWorkOnGpu("only literals are supported for intervals") + } + if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { + willNotWorkOnGpu("Only UTC zone id is supported") + } + } + + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuTimeSub(lhs, rhs) + } + ), + GpuOverrides.expr[First]( + "first aggregate operator", + (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { + // val childt: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + + // override val childExprs: Seq[BaseExprMeta[_]] = Seq(childt) + + override def convertToGpu(): GpuExpression = + GpuFirst(childExprs(0).convertToGpu(), a.ignoreNulls) + }), + ) + } + + def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( @@ -83,33 +122,6 @@ class Spark31Shims extends SparkShims with Logging { ) } - def getExprs: Seq[ExprRule[_ <: Expression]] = { - Seq( - - GpuOverrides.expr[TimeAdd]( - "Subtracts interval from timestamp", - (a, conf, p, r) => new BinaryExprMeta[TimeAdd](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.interval match { - case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => - if (intvl.months != 0) { - willNotWorkOnGpu("interval months isn't supported") - } - case _ => - willNotWorkOnGpu("only literals are supported for intervals") - } - if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { - willNotWorkOnGpu("Only UTC zone id is supported") - } - } - - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - GpuTimeSub(lhs, rhs) - } - ) - ) - } - def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide = { val buildSide = join.buildSide buildSide match { 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 e44725129b1..4dea9ba97f1 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 @@ -1226,17 +1226,6 @@ object GpuOverrides { } override def convertToGpu(child: Expression): GpuExpression = GpuMin(child) }), - expr[First]( - "first aggregate operator", - (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { - val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: BaseExprMeta[_] = - GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) - - override def convertToGpu(): GpuExpression = - GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) - }), expr[Last]( "last aggregate operator", (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { 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 422bd450e50..b208d57d7c7 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 @@ -429,8 +429,11 @@ case class GpuAverage(child: Expression) extends GpuDeclarativeAggregate { * to check if the value was set (if we don't ignore nulls, valueSet is true, that's what we do * here). */ -case class GpuFirst(child: Expression, ignoreNullsExpr: Expression) +abstract class GpuFirstBase(child: Expression) extends GpuDeclarativeAggregate with ImplicitCastInputTypes { + + val ignoreNulls: Boolean + private lazy val cudfFirst = AttributeReference("cudf_first", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() @@ -458,26 +461,8 @@ case class GpuFirst(child: Expression, ignoreNullsExpr: Expression) override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, BooleanType) override def nullable: Boolean = true override def dataType: DataType = child.dataType - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil // First is not a deterministic function. override lazy val deterministic: Boolean = false - private def ignoreNulls: Boolean = ignoreNullsExpr match { - case l: Literal => l.value.asInstanceOf[Boolean] - case l: GpuLiteral => l.value.asInstanceOf[Boolean] - case _ => throw new IllegalArgumentException( - s"$this should only receive literals for ignoreNulls expression") - } - override def checkInputDataTypes(): TypeCheckResult = { - val defaultCheck = super.checkInputDataTypes() - if (defaultCheck.isFailure) { - defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure(s"The second argument of GpuFirst must be a boolean literal, but " + - s"got: ${ignoreNullsExpr.sql}") - } else { - TypeCheckSuccess - } - } override def toString: String = s"gpufirst($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/tests/pom.xml b/tests/pom.xml index f7638744614..3a4c85c7aed 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -32,12 +32,12 @@ 1.7.30 + 3.1.0-SNAPSHOT spark31 - 3.1.0 @@ -61,6 +61,7 @@ org.apache.spark spark-sql_${scala.binary.version} + ${spark.test.version} org.scalatest From edf01fa653b703f1b1dcf810cdb14ba751dbbdbc Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Jul 2020 12:40:33 -0500 Subject: [PATCH 13/47] First and Last moved --- .../shims/spark30/GpuSortMergeJoinExec.scala | 3 --- .../rapids/shims/spark30/Spark30Shims.scala | 13 ++++++++- .../spark/rapids/shims/spark31/GpuFirst.scala | 5 ++++ .../rapids/shims/spark31/Spark31Shims.scala | 12 +++++---- .../nvidia/spark/rapids/GpuOverrides.scala | 11 -------- .../spark/sql/rapids/AggregateFunctions.scala | 27 +++++-------------- tests/pom.xml | 3 ++- 7 files changed, 32 insertions(+), 42 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index b1aefae869a..3f0c6f864f1 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -19,7 +19,6 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames._ -//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec @@ -36,8 +35,6 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging - - class GpuSortMergeJoinMeta( join: SortMergeJoinExec, conf: RapidsConf, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 3d9d664fdd2..4d09b997854 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark30._ -import org.apache.spark.sql.catalyst.expressions.aggregate.First +import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -120,6 +120,17 @@ class Spark30Shims extends SparkShims with Logging { override def convertToGpu(): GpuExpression = GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) }), + GpuOverrides.expr[Last]( + "last aggregate operator", + (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) + + override def convertToGpu(): GpuExpression = + GpuLast(child.convertToGpu(), ignoreNulls.convertToGpu()) + }), ) } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala index 342a3e24c05..2b69d7ff5e0 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala @@ -23,9 +23,14 @@ import com.nvidia.spark.rapids.GpuMetricNames._ 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.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.FirstLast case class GpuFirst(child: Expression, ignoreNulls: Boolean) extends GpuFirstBase(child) { + def this(child: Expression) = this(child, false) + def this(child: Expression, ignoreNullsExpr: Expression) = { + this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "last")) + } override def children: Seq[Expression] = child :: Nil override def checkInputDataTypes(): TypeCheckResult = { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 89a2cfef9b1..2636dfb9418 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark31._ -import org.apache.spark.sql.catalyst.expressions.aggregate.First +import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -70,13 +70,15 @@ class Spark31Shims extends SparkShims with Logging { GpuOverrides.expr[First]( "first aggregate operator", (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { - // val childt: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - - // override val childExprs: Seq[BaseExprMeta[_]] = Seq(childt) - override def convertToGpu(): GpuExpression = GpuFirst(childExprs(0).convertToGpu(), a.ignoreNulls) }), + GpuOverrides.expr[Last]( + "last aggregate operator", + (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { + override def convertToGpu(): GpuExpression = + GpuLast(childExprs(0).convertToGpu(), a.ignoreNulls) + }), ) } 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 4dea9ba97f1..8f87aa1a988 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 @@ -1226,17 +1226,6 @@ object GpuOverrides { } override def convertToGpu(child: Expression): GpuExpression = GpuMin(child) }), - expr[Last]( - "last aggregate operator", - (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { - val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: BaseExprMeta[_] = - GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) - - override def convertToGpu(): GpuExpression = - GpuLast(child.convertToGpu(), ignoreNulls.convertToGpu()) - }), expr[Sum]( "sum aggregate operator", (a, conf, p, r) => new AggExprMeta[Sum](a, conf, p, r) { 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 b208d57d7c7..0676d12f22f 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 @@ -430,7 +430,7 @@ case class GpuAverage(child: Expression) extends GpuDeclarativeAggregate { * here). */ abstract class GpuFirstBase(child: Expression) - extends GpuDeclarativeAggregate with ImplicitCastInputTypes { + extends GpuDeclarativeAggregate with ImplicitCastInputTypes with Serializable { val ignoreNulls: Boolean @@ -466,8 +466,11 @@ abstract class GpuFirstBase(child: Expression) override def toString: String = s"gpufirst($child)${if (ignoreNulls) " ignore nulls"}" } -case class GpuLast(child: Expression, ignoreNullsExpr: Expression) - extends GpuDeclarativeAggregate with ImplicitCastInputTypes { +abstract class GpuLastBase(child: Expression) + extends GpuDeclarativeAggregate with ImplicitCastInputTypes with Serializable { + + val ignoreNulls: Boolean + private lazy val cudfLast = AttributeReference("cudf_last", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() @@ -494,25 +497,7 @@ case class GpuLast(child: Expression, ignoreNullsExpr: Expression) override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, BooleanType) override def nullable: Boolean = true override def dataType: DataType = child.dataType - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil // Last is not a deterministic function. override lazy val deterministic: Boolean = false - private def ignoreNulls: Boolean = ignoreNullsExpr match { - case l: Literal => l.value.asInstanceOf[Boolean] - case l: GpuLiteral => l.value.asInstanceOf[Boolean] - case _ => throw new IllegalArgumentException( - s"$this should only receive literals for ignoreNulls expression") - } - override def checkInputDataTypes(): TypeCheckResult = { - val defaultCheck = super.checkInputDataTypes() - if (defaultCheck.isFailure) { - defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure(s"The second argument of GpuLast must be a boolean literal, but " + - s"got: ${ignoreNullsExpr.sql}") - } else { - TypeCheckSuccess - } - } override def toString: String = s"gpulast($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/tests/pom.xml b/tests/pom.xml index 3a4c85c7aed..c9da73c8f06 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -32,12 +32,13 @@ 1.7.30 - 3.1.0-SNAPSHOT + 3.0.0 spark31 + 3.1.0-SNAPSHOT From af27493c6176bcc57e3205069452fe5ac4db4a0d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Jul 2020 15:02:31 -0500 Subject: [PATCH 14/47] Allow multiple serviceloaders in dist jar --- dist/pom.xml | 3 ++ .../spark/rapids/shims/spark30/GpuLast.scala | 50 +++++++++++++++++++ .../spark/rapids/shims/spark31/GpuLast.scala | 40 +++++++++++++++ .../com/nvidia/spark/rapids/ShimLoader.scala | 9 +++- 4 files changed, 100 insertions(+), 2 deletions(-) create mode 100644 shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala create mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala diff --git a/dist/pom.xml b/dist/pom.xml index 46f57d49cdd..6eea5b7ecec 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -54,6 +54,9 @@ org.apache.maven.plugins maven-shade-plugin + + + false true diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala new file mode 100644 index 00000000000..8f194ec92ba --- /dev/null +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala @@ -0,0 +1,50 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark30 + +import org.apache.spark.sql.rapids.GpuLastBase +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +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.{Expression, Literal} + + +case class GpuLast(child: Expression, ignoreNullsExpr: Expression) extends GpuLastBase(child) { + override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + + override val ignoreNulls: Boolean = ignoreNullsExpr match { + case l: Literal => l.value.asInstanceOf[Boolean] + case l: GpuLiteral => l.value.asInstanceOf[Boolean] + case _ => throw new IllegalArgumentException( + s"$this should only receive literals for ignoreNulls expression") + } + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else if (!ignoreNullsExpr.foldable) { + TypeCheckFailure(s"The second argument of GpuLast must be a boolean literal, but " + + s"got: ${ignoreNullsExpr.sql}") + } else { + TypeCheckSuccess + } + } +} + diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala new file mode 100644 index 00000000000..4f4461727d9 --- /dev/null +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark31 + +import org.apache.spark.sql.rapids.GpuLastBase +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames._ + +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.Expression + + +case class GpuLast(child: Expression, ignoreNulls: Boolean) extends GpuLastBase(child) { + override def children: Seq[Expression] = child :: Nil + + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else { + TypeCheckSuccess + } + } +} + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 7fdfc56e595..c639661a83b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -20,6 +20,7 @@ import java.util.ServiceLoader import scala.collection.JavaConverters._ import scala.collection.immutable.HashMap +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.execution.SparkPlan @@ -27,15 +28,19 @@ import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution._ import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} -object ShimLoader { +object ShimLoader extends Logging { + + private val sparkVersion = getVersion + logInfo(s"Loading shim for version: $sparkVersion") // This is no ideal but pass the version in here because otherwise loader that match the // same version (3.0.0 Apache and 3.0.0 Databricks) would need to know how to differentiate. val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimLoader]) - .asScala.filter(_.matchesVersion(getVersion)) + .asScala.filter(_.matchesVersion(sparkVersion)) if (sparkShimLoaders.size > 1) { throw new IllegalArgumentException(s"Multiple Spark Shim Loaders found: $sparkShimLoaders") } + logWarning(s"Found shims: $sparkShimLoaders") val loader = sparkShimLoaders.headOption match { case Some(loader) => loader case None => throw new IllegalArgumentException("Could not find Spark Shim Loader") From 06c403d1fdde777c7fd90fbc716c97f04027d786 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Jul 2020 16:49:03 -0500 Subject: [PATCH 15/47] Cleanup --- shims/aggregator/pom.xml | 6 --- shims/pom.xml | 9 ---- shims/spark30/pom.xml | 10 +--- .../com.nvidia.spark.rapids.SparkShimLoader | 1 - ...idia.spark.rapids.SparkShimServiceProvider | 1 + .../spark30/GpuBroadcastHashJoinExec.scala | 20 +------- .../GpuBroadcastNestedLoopJoinExec.scala | 20 ++------ .../spark/rapids/shims/spark30/GpuFirst.scala | 1 - .../rapids/shims/spark30/GpuHashJoin.scala | 3 +- .../spark/rapids/shims/spark30/GpuLast.scala | 1 - .../spark30/GpuShuffledHashJoinExec.scala | 18 +------ .../spark30/GpuShuffledHashJoinExecBase.scala | 3 +- .../shims/spark30/GpuSortMergeJoinExec.scala | 15 +----- ...scala => Spark30ShimServiceProvider.scala} | 9 ++-- .../rapids/shims/spark30/Spark30Shims.scala | 14 +++++- shims/spark31/pom.xml | 10 +--- .../com.nvidia.spark.rapids.SparkShimLoader | 1 - ...idia.spark.rapids.SparkShimServiceProvider | 1 + .../spark31/GpuBroadcastHashJoinExec.scala | 20 ++------ .../GpuBroadcastNestedLoopJoinExec.scala} | 33 ++++--------- .../spark/rapids/shims/spark31/GpuFirst.scala | 2 - .../rapids/shims/spark31/GpuHashJoin.scala | 7 ++- .../spark/rapids/shims/spark31/GpuLast.scala | 2 - .../spark31/GpuShuffledHashJoinExec.scala | 18 ++----- .../shims/spark31/GpuSortMergeJoinExec.scala | 18 +------ ...scala => Spark31ShimServiceProvider.scala} | 8 ++-- .../rapids/shims/spark31/Spark31Shims.scala | 14 +++++- .../rapids}/GpuShuffledHashJoinExecBase.scala | 5 +- .../com/nvidia/spark/rapids/ShimLoader.scala | 47 ++----------------- ...r.scala => SparkShimServiceProvider.scala} | 5 +- .../com/nvidia/spark/rapids/SparkShims.scala | 17 ++++--- .../GpuBroadcastNestedLoopJoinExec.scala | 2 +- 32 files changed, 89 insertions(+), 252 deletions(-) delete mode 100644 shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader create mode 100644 shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider rename shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/{Spark30ShimLoader.scala => Spark30ShimServiceProvider.scala} (92%) delete mode 100644 shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader create mode 100644 shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/{GpuBroadcastNestedLoopJoinExec31.scala => shims/spark31/GpuBroadcastNestedLoopJoinExec.scala} (61%) rename shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/{Spark31ShimLoader.scala => Spark31ShimServiceProvider.scala} (92%) rename {shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31 => sql-plugin/src/main/scala/com/nvidia/spark/rapids}/GpuShuffledHashJoinExecBase.scala (97%) rename sql-plugin/src/main/scala/com/nvidia/spark/rapids/{SparkShimLoader.scala => SparkShimServiceProvider.scala} (89%) diff --git a/shims/aggregator/pom.xml b/shims/aggregator/pom.xml index aa9dc84a780..f9576968f74 100644 --- a/shims/aggregator/pom.xml +++ b/shims/aggregator/pom.xml @@ -32,12 +32,6 @@ The RAPIDS SQL plugin for Apache Spark Shim Aggregator 0.2.0-SNAPSHOT - - - databricks - - - com.nvidia diff --git a/shims/pom.xml b/shims/pom.xml index 5d18156cba0..436d4a31b5f 100644 --- a/shims/pom.xml +++ b/shims/pom.xml @@ -32,15 +32,6 @@ The RAPIDS SQL plugin for Apache Spark Shims 0.2.0-SNAPSHOT - - - databricks - - databricks - - - - spark30 spark31 diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index cad9301f312..92138418f15 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -27,16 +27,10 @@ com.nvidia rapids-4-spark-shims-spark30_2.12 - RAPIDS Accelerator for Apache Spark SQL Plugin - The RAPIDS SQL plugin for Apache Spark Shim + RAPIDS Accelerator for Apache Spark SQL Plugin Spark 3.0 Shim + The RAPIDS SQL plugin for Apache Spark 3.0 Shim 0.2.0-SNAPSHOT - - - databricks - - - 3.0.0 diff --git a/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader b/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader deleted file mode 100644 index 0009a24af47..00000000000 --- a/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader +++ /dev/null @@ -1 +0,0 @@ -com.nvidia.spark.rapids.shims.spark30.Spark30ShimLoader diff --git a/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider b/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider new file mode 100644 index 00000000000..a7727cca514 --- /dev/null +++ b/shims/spark30/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider @@ -0,0 +1 @@ +com.nvidia.spark.rapids.shims.spark30.Spark30ShimServiceProvider diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 52b08df0789..07165691742 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -17,27 +17,12 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], @@ -71,5 +56,4 @@ object GpuBroadcastHashJoinExec extends Logging { GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) } - } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala index 6df1a5c02e3..84e96ecb3d6 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala @@ -17,26 +17,13 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.sql.rapids.execution._ -//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BuildLeft, BuildRight} case class GpuBroadcastNestedLoopJoinExec( @@ -44,7 +31,8 @@ case class GpuBroadcastNestedLoopJoinExec( right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, - condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { + condition: Option[Expression]) + extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { def getBuildSide: GpuBuildSide = { diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala index b0f4893f25a..e9f5a629227 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala @@ -18,7 +18,6 @@ package com.nvidia.spark.rapids.shims.spark30 import org.apache.spark.sql.rapids.GpuFirstBase import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 577a6db140e..7a6451e6029 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -15,7 +15,7 @@ */ package com.nvidia.spark.rapids.shims.spark30 -import ai.rapids.cudf.{NvtxColor, Table} +import ai.rapids.cudf.Table import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec object GpuHashJoin { def tagJoin( diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala index 8f194ec92ba..50558835bdd 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala @@ -18,7 +18,6 @@ package com.nvidia.spark.rapids.shims.spark30 import org.apache.spark.sql.rapids.GpuLastBase import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index dcce05ee7e7..0e622179bfa 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -17,26 +17,12 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -//import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide, ShuffledHashJoinExec} case class GpuShuffledHashJoinExec( leftKeys: Seq[Expression], diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala index a3e929b5df3..e61d3d86b63 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala @@ -26,9 +26,8 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.BinaryExecNode import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index 3f0c6f864f1..eb5f7d4a6b2 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -17,23 +17,10 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SortExec, SparkPlan} +import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging class GpuSortMergeJoinMeta( join: SortMergeJoinExec, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala similarity index 92% rename from shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala index 9b302abd6ec..ca9685aff50 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimLoader.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala @@ -18,8 +18,7 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ - -class Spark30ShimLoader extends SparkShimLoader { +class Spark30ShimServiceProvider extends SparkShimServiceProvider { val SPARK30VERSIONNAME = "3.0.0" @@ -29,7 +28,5 @@ class Spark30ShimLoader extends SparkShimLoader { def buildShim: SparkShims = { new Spark30Shims() - } - -} - + } +} \ No newline at end of file diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 4d09b997854..4dc47828ba8 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -21,20 +21,30 @@ import java.time.ZoneId import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark30._ - import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class Spark30Shims extends SparkShims with Logging { + def getGpuBroadcastNestedLoopJoinShims( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { + GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) + } + def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuHashJoin => true diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 58d82f21997..110ca1ff83c 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -27,16 +27,10 @@ com.nvidia rapids-4-spark-shims-spark31_2.12 - RAPIDS Accelerator for Apache Spark SQL Plugin - The RAPIDS SQL plugin for Apache Spark Shim + RAPIDS Accelerator for Apache Spark SQL Plugin Spark 3.1 Shim + The RAPIDS SQL plugin for Apache Spark 3.1 Shim 0.2.0-SNAPSHOT - - - databricks - - - 3.1.0-SNAPSHOT diff --git a/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader b/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader deleted file mode 100644 index a131979d004..00000000000 --- a/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimLoader +++ /dev/null @@ -1 +0,0 @@ -com.nvidia.spark.rapids.shims.spark31.Spark31ShimLoader diff --git a/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider b/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider new file mode 100644 index 00000000000..0c854281b7d --- /dev/null +++ b/shims/spark31/src/main/resources/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider @@ -0,0 +1 @@ +com.nvidia.spark.rapids.shims.spark31.Spark31ShimServiceProvider diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 43625ea512e..7d6c523d9f4 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -17,26 +17,13 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], @@ -70,5 +57,4 @@ object GpuBroadcastHashJoinExec extends Logging { GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) } - } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala similarity index 61% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala index eb31ea584a1..e8f27dec384 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastNestedLoopJoinExec31.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala @@ -14,37 +14,25 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims +package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.rapids.execution._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec +import org.apache.spark.sql.rapids.execution._ -case class GpuBroadcastNestedLoopJoinExec31( +case class GpuBroadcastNestedLoopJoinExec( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, - condition: Option[Expression]) extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { - + condition: Option[Expression]) + extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { def getBuildSide: GpuBuildSide = { join.buildSide match { @@ -55,7 +43,7 @@ case class GpuBroadcastNestedLoopJoinExec31( } } -object GpuBroadcastNestedLoopJoinExec31 extends Logging { +object GpuBroadcastNestedLoopJoinExec extends Logging { def createInstance( left: SparkPlan, @@ -64,7 +52,6 @@ object GpuBroadcastNestedLoopJoinExec31 extends Logging { joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { - GpuBroadcastNestedLoopJoinExec31(left, right, join, joinType, condition) + GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) } - } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala index 2b69d7ff5e0..891b327497e 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala @@ -17,8 +17,6 @@ package com.nvidia.spark.rapids.shims.spark31 import org.apache.spark.sql.rapids.GpuFirstBase -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index 7d6027faffc..18fea577f56 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -15,17 +15,16 @@ */ package com.nvidia.spark.rapids.shims.spark31 -import ai.rapids.cudf.{NvtxColor, Table} +import ai.rapids.cudf.Table import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} object GpuHashJoin { def tagJoin( diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala index 4f4461727d9..3271d5ee8fb 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala @@ -17,8 +17,6 @@ package com.nvidia.spark.rapids.shims.spark31 import org.apache.spark.sql.rapids.GpuLastBase -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index ad5c6b4c004..7e345d5508c 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -17,25 +17,13 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec case class GpuShuffledHashJoinExec ( leftKeys: Seq[Expression], diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala index fffef3e5e74..075af30aad2 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala @@ -17,25 +17,11 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec -import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SortExec, SparkPlan} +import org.apache.spark.sql.catalyst.optimizer.BuildRight +import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - class GpuSortMergeJoinMeta( join: SortMergeJoinExec, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala similarity index 92% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala index f63cd2b9359..114aee1e69a 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimLoader.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -class Spark31ShimLoader extends SparkShimLoader { +class Spark31ShimServiceProvider extends SparkShimServiceProvider { val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" @@ -28,7 +28,5 @@ class Spark31ShimLoader extends SparkShimLoader { def buildShim: SparkShims = { new Spark31Shims() - } - -} - + } +} \ No newline at end of file diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 2636dfb9418..07586305094 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -19,9 +19,9 @@ package com.nvidia.spark.rapids.shims.spark31 import java.time.ZoneId import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec31 import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark31._ - import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging @@ -30,12 +30,24 @@ import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.sql.types._ class Spark31Shims extends SparkShims with Logging { + def getGpuBroadcastNestedLoopJoinShims( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { + GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) + } + + def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuHashJoin => true diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala similarity index 97% rename from shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala rename to sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala index 0a070ddba0f..e9fd81d7745 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.shims.spark31 +package com.nvidia.spark.rapids import ai.rapids.cudf.{NvtxColor, Table} @@ -26,9 +26,8 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.BinaryExecNode import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index c639661a83b..3b28fe256bd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -17,64 +17,43 @@ package com.nvidia.spark.rapids import java.util.ServiceLoader + import scala.collection.JavaConverters._ import scala.collection.immutable.HashMap +import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution._ -import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} object ShimLoader extends Logging { private val sparkVersion = getVersion logInfo(s"Loading shim for version: $sparkVersion") - // This is no ideal but pass the version in here because otherwise loader that match the + // This is not ideal, but pass the version in here because otherwise loader that match the // same version (3.0.0 Apache and 3.0.0 Databricks) would need to know how to differentiate. - val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimLoader]) + val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimServiceProvider]) .asScala.filter(_.matchesVersion(sparkVersion)) if (sparkShimLoaders.size > 1) { throw new IllegalArgumentException(s"Multiple Spark Shim Loaders found: $sparkShimLoaders") } - logWarning(s"Found shims: $sparkShimLoaders") + logInfo(s"Found shims: $sparkShimLoaders") val loader = sparkShimLoaders.headOption match { case Some(loader) => loader case None => throw new IllegalArgumentException("Could not find Spark Shim Loader") } private val sparkShims: SparkShims = loader.buildShim - val SPARK30DATABRICKSSVERSIONNAME = "3.0.0-databricks" val SPARK30VERSIONNAME = "3.0.0" val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" private var gpuBroadcastNestedJoinShims: GpuBroadcastNestedLoopJoinExecBase = null - /** - * The names of the classes for shimming Spark for each major version. - */ - private val SPARK_SHIM_CLASSES = HashMap( - SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark30Shims", - SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark300DatabricksShims", - SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.Spark31Shims", - ) - - /** - * Factory method to get an instance of SparkShims based on the - * version of Spark on the classpath. - */ - def getSparkShims: SparkShims = { - if (sparkShims == null) { - - // sparkShims = loadShims(SPARK_SHIM_CLASSES, classOf[SparkShims]) - } - sparkShims - } - private val BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES = HashMap( SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30.GpuBroadcastNestedLoopJoinExec", SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30databricks.GpuBroadcastNestedLoopJoinExec", @@ -94,22 +73,6 @@ object ShimLoader extends Logging { gpuBroadcastNestedJoinShims } - private def loadShims[T](classMap: Map[String, String], xface: Class[T]): T = { - val vers = getVersion - val className = classMap.get(vers) - if (className.isEmpty) { - throw new Exception(s"No shim layer for $vers") - } - createShim(className.get, xface) - } - - private def createShim[T](className: String, xface: Class[T]): T = try { - val clazz = Class.forName(className) - clazz.newInstance().asInstanceOf[T] - } catch { - case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) - } - private def loadShimsNestedBroadcastJoin[T]( classMap: Map[String, String], xface: Class[T], diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimServiceProvider.scala similarity index 89% rename from sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala rename to sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimServiceProvider.scala index 41a873456f1..e1429a1f706 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShimServiceProvider.scala @@ -16,7 +16,10 @@ package com.nvidia.spark.rapids -trait SparkShimLoader { +/** + * A Spark version shim layer interface. + */ +trait SparkShimServiceProvider { def matchesVersion(version:String): Boolean def buildShim: SparkShims } 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 c58e849887e..c8851ecf7a9 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 @@ -16,16 +16,11 @@ package com.nvidia.spark.rapids -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase sealed abstract class GpuBuildSide @@ -33,7 +28,6 @@ case object GpuBuildRight extends GpuBuildSide case object GpuBuildLeft extends GpuBuildSide - trait SparkShims { def isGpuHashJoin(plan: SparkPlan): Boolean @@ -42,7 +36,12 @@ trait SparkShims { def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] def getExecs: Seq[ExecRule[_ <: SparkPlan]] - + def getGpuBroadcastNestedLoopJoinShims( + left: SparkPlan, + right: SparkPlan, + join: BroadcastNestedLoopJoinExec, + joinType: JoinType, + condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 6720b70d628..d6baaee383d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -78,7 +78,7 @@ class GpuBroadcastNestedLoopJoinMeta( if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") } - ShimLoader.getGpuBroadcastNestedLoopJoinShims( + ShimLoader.getSparkShims.getGpuBroadcastNestedLoopJoinShims( left, right, join, join.joinType, condition.map(_.convertToGpu())) From 5d45aeb11f4b052de38d3f953e8bd109ed1ee1d8 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 09:50:32 -0500 Subject: [PATCH 16/47] Fixes --- .../GpuShuffledHashJoinExecBase.scala | 5 +- .../rapids/shims/spark31/Spark31Shims.scala | 11 ++- .../com/nvidia/spark/rapids/ShimLoader.scala | 69 ++----------------- 3 files changed, 15 insertions(+), 70 deletions(-) rename {sql-plugin/src/main/scala/com/nvidia/spark/rapids => shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31}/GpuShuffledHashJoinExecBase.scala (98%) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala similarity index 98% rename from sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala rename to shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala index e9fd81d7745..4d1927e845e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecBase.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids +package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} @@ -34,6 +34,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.internal.Logging +/** + * This has to be in the shim layer because GpuHashJoin is in shim. + */ abstract class GpuShuffledHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { def getBuildSide: GpuBuildSide diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 07586305094..5f414d6bc92 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -19,9 +19,8 @@ package com.nvidia.spark.rapids.shims.spark31 import java.time.ZoneId import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.shims.GpuBroadcastNestedLoopJoinExec31 -import org.apache.spark.sql.rapids.GpuTimeSub -import org.apache.spark.sql.rapids.shims.spark31._ +import com.nvidia.spark.rapids.shims.spark31.GpuBroadcastNestedLoopJoinExec + import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging @@ -31,11 +30,12 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNes import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase +import org.apache.spark.sql.rapids.shims.spark31._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.sql.types._ - class Spark31Shims extends SparkShims with Logging { def getGpuBroadcastNestedLoopJoinShims( @@ -47,7 +47,6 @@ class Spark31Shims extends SparkShims with Logging { GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) } - def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuHashJoin => true @@ -173,6 +172,4 @@ class Spark31Shims extends SparkShims with Logging { case _ => throw new Exception("unknown buildSide Type") } } - } - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 3b28fe256bd..549c4bd068d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -36,77 +36,23 @@ object ShimLoader extends Logging { // This is not ideal, but pass the version in here because otherwise loader that match the // same version (3.0.0 Apache and 3.0.0 Databricks) would need to know how to differentiate. - val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimServiceProvider]) + private val sparkShimLoaders = ServiceLoader.load(classOf[SparkShimServiceProvider]) .asScala.filter(_.matchesVersion(sparkVersion)) if (sparkShimLoaders.size > 1) { throw new IllegalArgumentException(s"Multiple Spark Shim Loaders found: $sparkShimLoaders") } logInfo(s"Found shims: $sparkShimLoaders") - val loader = sparkShimLoaders.headOption match { + private val loader = sparkShimLoaders.headOption match { case Some(loader) => loader case None => throw new IllegalArgumentException("Could not find Spark Shim Loader") } - private val sparkShims: SparkShims = loader.buildShim + private var sparkShims: SparkShims = null - val SPARK30DATABRICKSSVERSIONNAME = "3.0.0-databricks" - val SPARK30VERSIONNAME = "3.0.0" - val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" - - private var gpuBroadcastNestedJoinShims: GpuBroadcastNestedLoopJoinExecBase = null - - private val BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES = HashMap( - SPARK30VERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30.GpuBroadcastNestedLoopJoinExec", - SPARK30DATABRICKSSVERSIONNAME -> "com.nvidia.spark.rapids.shims.spark30databricks.GpuBroadcastNestedLoopJoinExec", - SPARK31VERSIONNAME -> "com.nvidia.spark.rapids.shims.spark31.GpuBroadcastNestedLoopJoinExec", - ) - - def getGpuBroadcastNestedLoopJoinShims( - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { - if (gpuBroadcastNestedJoinShims == null) { - gpuBroadcastNestedJoinShims = loadShimsNestedBroadcastJoin(BROADCAST_NESTED_LOOP_JOIN_SHIM_CLASSES, - classOf[GpuBroadcastNestedLoopJoinExecBase], left, right, join, joinType, condition) + def getSparkShims: SparkShims = { + if (sparkShims == null) { + sparkShims = loader.buildShim } - gpuBroadcastNestedJoinShims - } - - private def loadShimsNestedBroadcastJoin[T]( - classMap: Map[String, String], - xface: Class[T], - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression]): T = { - val vers = getVersion - val className = classMap.get(vers) - if (className.isEmpty) { - throw new Exception(s"No shim layer for $vers") - } - createShimNestedBroadcastJoin(className.get, xface, left, right, join, joinType, condition) - } - - private def createShimNestedBroadcastJoin[T]( - className: String, - xface: Class[T], - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression]): T = try { - val clazz = Class.forName(className) - val resultMethod = clazz.getDeclaredMethod("createInstance", - classOf[org.apache.spark.sql.execution.SparkPlan], - classOf[org.apache.spark.sql.execution.SparkPlan], - classOf[org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec], - classOf[org.apache.spark.sql.catalyst.plans.JoinType], - classOf[scala.Option[org.apache.spark.sql.catalyst.expressions.Expression]]) - resultMethod.invoke(clazz, left, right, join, joinType, condition).asInstanceOf[T] - } catch { - case e: Exception => throw new RuntimeException("Could not load shims in class " + className, e) + sparkShims } def getVersion: String = { @@ -118,4 +64,3 @@ object ShimLoader extends Logging { } } } - From e138385e4386fecdc7fd074ecb1de3e26064ce81 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 10:43:16 -0500 Subject: [PATCH 17/47] Cleanup --- .../spark30/GpuBroadcastHashJoinExec.scala | 197 ++++++++++++++-- .../GpuBroadcastHashJoinExecBase.scala | 213 ------------------ .../GpuBroadcastNestedLoopJoinExec.scala | 26 +-- .../spark/rapids/shims/spark30/GpuFirst.scala | 8 +- .../rapids/shims/spark30/GpuHashJoin.scala | 16 +- .../spark/rapids/shims/spark30/GpuLast.scala | 8 +- .../spark30/GpuShuffledHashJoinExec.scala | 199 ++++++++++++++-- .../spark30/GpuShuffledHashJoinExecBase.scala | 199 ---------------- .../shims/spark30/GpuSortMergeJoinExec.scala | 9 +- .../spark30/Spark30ShimServiceProvider.scala | 2 +- .../rapids/shims/spark30/Spark30Shims.scala | 42 +--- .../shims/spark30/GpuFileSourceScanExec.scala | 2 +- .../spark31/GpuBroadcastHashJoinExec.scala | 200 ++++++++++++++-- .../GpuBroadcastHashJoinExecBase.scala | 213 ------------------ .../GpuBroadcastNestedLoopJoinExec.scala | 21 +- .../spark/rapids/shims/spark31/GpuFirst.scala | 9 +- .../rapids/shims/spark31/GpuHashJoin.scala | 18 +- .../spark/rapids/shims/spark31/GpuLast.scala | 9 +- .../spark31/GpuShuffledHashJoinExec.scala | 193 ++++++++++++++-- .../spark31/GpuShuffledHashJoinExecBase.scala | 202 ----------------- .../shims/spark31/GpuSortMergeJoinExec.scala | 9 +- .../spark31/Spark31ShimServiceProvider.scala | 2 +- .../rapids/shims/spark31/Spark31Shims.scala | 42 +--- .../shims/spark31/GpuFileSourceScanExec.scala | 2 +- .../com/nvidia/spark/rapids/SparkShims.scala | 3 +- .../spark/sql/rapids/AggregateFunctions.scala | 1 - .../rapids/GpuFileSourceScanExecBase.scala | 24 +- 27 files changed, 765 insertions(+), 1104 deletions(-) delete mode 100644 shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala delete mode 100644 shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala delete mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala delete mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 07165691742..15feb05ebc7 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -16,14 +16,25 @@ package com.nvidia.spark.rapids.shims.spark30 +import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} -import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.vectorized.ColumnarBatch +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -31,29 +42,173 @@ case class GpuBroadcastHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuBroadcastHashJoinExecBase with Logging { + right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) + } + } - def getBuildSide: GpuBuildSide = { - buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = { + val mode = HashedRelationBroadcastMode(buildKeys) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => + BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil + case GpuBuildRight => + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil + } + } + def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { + case gpu: GpuBroadcastExchangeExec => gpu + case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] + } + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val broadcastRelation = broadcastExchange + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + lazy val builtTable = { + // TODO clean up intermediate results... + val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) + val combined = combine(keys, broadcastRelation.value.batch) + val ret = GpuColumnVector.from(combined) + // Don't warn for a leak, because we cannot control when we are done with this + (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) + ret + } + + val rdd = streamedPlan.executeColumnar() + rdd.mapPartitions(it => + doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, + numOutputBatches, joinTime, filterTime, totalTime)) + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) } } } -object GpuBroadcastHashJoinExec extends Logging { - - def createInstance( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - join: BroadcastHashJoinExec, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan): GpuBroadcastHashJoinExec = { - - GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) +class GpuBroadcastHashJoinMeta( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + ShimLoader.getSparkShims.getBuildSide(join) + } + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => childPlans(0) + case GpuBuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => left + case GpuBuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala deleted file mode 100644 index b1bd9f096f6..00000000000 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExecBase.scala +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.spark30 - -import ai.rapids.cudf.{NvtxColor, Table} - -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.rapids.execution._ - -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - -abstract class GpuBroadcastHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { - - def getBuildSide: GpuBuildSide - - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - getBuildSide match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) - } - } - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) - - override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(buildKeys) - getBuildSide match { - case GpuBuildLeft => - BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil - case GpuBuildRight => - UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil - } - } - def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { - case gpu: GpuBroadcastExchangeExec => gpu - case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] - } - - override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") - - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() - - val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) - - lazy val builtTable = { - // TODO clean up intermediate results... - val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) - val combined = combine(keys, broadcastRelation.value.batch) - val ret = GpuColumnVector.from(combined) - // Don't warn for a leak, because we cannot control when we are done with this - (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) - ret - } - - val rdd = streamedPlan.executeColumnar() - rdd.mapPartitions(it => - doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, - numOutputBatches, joinTime, filterTime, totalTime)) - } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - getBuildSide match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } - -} - - -class GpuBroadcastHashJoinMeta( - join: BroadcastHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) with Logging { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - ShimLoader.getSparkShims.getBuildSide(join) - } - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => childPlans(0) - case GpuBuildRight => childPlans(1) - } - - if (!buildSide.canThisBeReplaced) { - willNotWorkOnGpu("the broadcast for this join must be on the GPU too") - } - - if (!canThisBeReplaced) { - buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") - } - } - - override def convertToGpu(): GpuExec = { - val left = childPlans(0).convertIfNeeded() - val right = childPlans(1).convertIfNeeded() - // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => left - case GpuBuildRight => right - } - if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { - throw new IllegalStateException("the broadcast must be on the GPU too") - } - GpuBroadcastHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, join.buildSide, - condition.map(_.convertToGpu()), - left, right) - } - -} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala index 84e96ecb3d6..3be2f3810fd 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala @@ -16,24 +16,24 @@ package com.nvidia.spark.rapids.shims.spark30 -import com.nvidia.spark.rapids._ -import org.apache.spark.sql.rapids.execution._ +import com.nvidia.spark.rapids.{GpuBuildLeft, GpuBuildRight, GpuBuildSide} -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, BuildLeft, BuildRight} +import org.apache.spark.sql.rapids.execution._ - +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuBroadcastNestedLoopJoinExec( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]) - extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { - + extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) { def getBuildSide: GpuBuildSide = { join.buildSide match { @@ -43,17 +43,3 @@ case class GpuBroadcastNestedLoopJoinExec( } } } - -object GpuBroadcastNestedLoopJoinExec extends Logging { - - def createInstance( - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase= { - - GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) - } - -} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala index e9f5a629227..7de2090ed02 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuFirst.scala @@ -16,14 +16,16 @@ package com.nvidia.spark.rapids.shims.spark30 -import org.apache.spark.sql.rapids.GpuFirstBase -import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuLiteral 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.{Expression, Literal} +import org.apache.spark.sql.rapids.GpuFirstBase - +/** + * Parameters to GpuFirst changed in Spark 3.1 + */ case class GpuFirst(child: Expression, ignoreNullsExpr: Expression) extends GpuFirstBase(child) { override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 7a6451e6029..9b49356bb97 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -19,7 +19,6 @@ import ai.rapids.cudf.Table import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.HashJoin @@ -34,15 +33,7 @@ object GpuHashJoin { rightKeys: Seq[Expression], condition: Option[Expression]): Unit = joinType match { case _: InnerLike => - case FullOuter => - if (leftKeys.exists(_.nullable) || rightKeys.exists(_.nullable)) { - // https://github.com/rapidsai/cudf/issues/5563 - meta.willNotWorkOnGpu("Full outer join does not work on nullable keys") - } - if (condition.isDefined) { - meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") - } - case RightOuter | LeftOuter | LeftSemi | LeftAnti => + case FullOuter | RightOuter | LeftOuter | LeftSemi | LeftAnti => if (condition.isDefined) { meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") } @@ -50,9 +41,7 @@ object GpuHashJoin { } } - -trait GpuHashJoin extends GpuExec with HashJoin with Logging { - +trait GpuHashJoin extends GpuExec with HashJoin { override def output: Seq[Attribute] = { joinType match { @@ -73,7 +62,6 @@ trait GpuHashJoin extends GpuExec with HashJoin with Logging { } } - def doJoinInternal(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala index 50558835bdd..302d50ceaea 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala @@ -16,14 +16,16 @@ package com.nvidia.spark.rapids.shims.spark30 -import org.apache.spark.sql.rapids.GpuLastBase -import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuLiteral 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.{Expression, Literal} +import org.apache.spark.sql.rapids.GpuLastBase - +/** + * Parameters to GpuFirst changed in Spark 3.1 + */ case class GpuLast(child: Expression, ignoreNullsExpr: Expression) extends GpuLastBase(child) { override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index 0e622179bfa..e7a0653c19c 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -16,14 +16,34 @@ package com.nvidia.spark.rapids.shims.spark30 +import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} -import org.apache.spark.internal.Logging +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide, ShuffledHashJoinExec} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +object GpuJoinUtils { + def getBuildSide(buildSide: BuildSide): GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuShuffledHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -31,35 +51,166 @@ case class GpuShuffledHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuShuffledHashJoinExecBase with Logging { + right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - def getBuildSide: GpuBuildSide = { - buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) } } -} -object GpuShuffledHashJoinExec extends Logging { - - def createInstance( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - join: SparkPlan, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan): GpuShuffledHashJoinExec = { - - val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - join.asInstanceOf[ShuffledHashJoinExec].buildSide + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), + "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException( + "GpuShuffledHashJoin does not support the execute() code path.") + } + + override def childrenCoalesceGoal: Seq[CoalesceGoal] = { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => Seq(RequireSingleBatch, null) + case GpuBuildRight => Seq(null, RequireSingleBatch) + } + } + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val buildDataSize = longMetric("buildDataSize") + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val buildTime = longMetric("buildTime") + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { + (streamIter, buildIter) => { + var combinedSize = 0 + val startTime = System.nanoTime() + val buildBatch = + ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) + val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) + val builtTable = try { + // Combine does not inc any reference counting + val combined = combine(keys, buildBatch) + combinedSize = + GpuColumnVector.extractColumns(combined) + .map(_.getBase.getDeviceMemorySize).sum.toInt + GpuColumnVector.from(combined) + } finally { + keys.close() + buildBatch.close() + } + + val delta = System.nanoTime() - startTime + buildTime += delta + totalTime += delta + buildDataSize += combinedSize + val context = TaskContext.get() + context.addTaskCompletionListener[Unit](_ => builtTable.close()) + + doJoin(builtTable, streamIter, boundCondition, + numOutputRows, joinOutputRows, numOutputBatches, + joinTime, filterTime, totalTime) + } + } + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None } else { - BuildRight + Some(tmp) } - GpuShuffledHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + } +} + +class GpuShuffledHashJoinMeta( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } } + diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala deleted file mode 100644 index e61d3d86b63..00000000000 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExecBase.scala +++ /dev/null @@ -1,199 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.spark30 - -import ai.rapids.cudf.{NvtxColor, Table} - -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuHashJoinBaseMeta -import com.nvidia.spark.rapids.GpuMetricNames._ - -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.BinaryExecNode -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - -abstract class GpuShuffledHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { - - def getBuildSide: GpuBuildSide - - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - getBuildSide match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) - } - } - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) - - override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil - - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException( - "GpuShuffledHashJoin does not support the execute() code path.") - } - - override def childrenCoalesceGoal: Seq[CoalesceGoal] = getBuildSide match { - case GpuBuildLeft => Seq(RequireSingleBatch, null) - case GpuBuildRight => Seq(null, RequireSingleBatch) - } - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val buildDataSize = longMetric("buildDataSize") - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val buildTime = longMetric("buildTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") - - val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) - - streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { - (streamIter, buildIter) => { - var combinedSize = 0 - val startTime = System.nanoTime() - val buildBatch = - ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) - val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) - val builtTable = try { - // Combine does not inc any reference counting - val combined = combine(keys, buildBatch) - combinedSize = - GpuColumnVector.extractColumns(combined) - .map(_.getBase.getDeviceMemorySize).sum.toInt - GpuColumnVector.from(combined) - } finally { - keys.close() - buildBatch.close() - } - - val delta = System.nanoTime() - startTime - buildTime += delta - totalTime += delta - buildDataSize += combinedSize - val context = TaskContext.get() - context.addTaskCompletionListener[Unit](_ => builtTable.close()) - - doJoin(builtTable, streamIter, boundCondition, - numOutputRows, joinOutputRows, numOutputBatches, - joinTime, filterTime, totalTime) - } - } - } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - getBuildSide match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } - -} - -class GpuShuffledHashJoinMeta( - join: ShuffledHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - } - - override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, - join.buildSide, - condition.map(_.convertToGpu()), - childPlans(0).convertIfNeeded(), - childPlans(1).convertIfNeeded()) - } -} - diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index eb5f7d4a6b2..c4459471ca1 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -18,16 +18,18 @@ package com.nvidia.spark.rapids.shims.spark30 import com.nvidia.spark.rapids._ -import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.joins._ +/** + * HashJoin changed in Spark 3.1 requiring Shim + */ class GpuSortMergeJoinMeta( join: SortMergeJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { + extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -63,6 +65,7 @@ class GpuSortMergeJoinMeta( } } } + override def convertToGpu(): GpuExec = { GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), @@ -73,6 +76,4 @@ class GpuSortMergeJoinMeta( childPlans(0).convertIfNeeded(), childPlans(1).convertIfNeeded()) } - } - diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala index ca9685aff50..5071d0de123 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.shims.spark30 -import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.{SparkShims, SparkShimServiceProvider} class Spark30ShimServiceProvider extends SparkShimServiceProvider { diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 4dc47828ba8..2e723e90592 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -22,17 +22,16 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.shims.spark30._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.CalendarInterval class Spark30Shims extends SparkShims with Logging { @@ -144,43 +143,12 @@ class Spark30Shims extends SparkShims with Logging { ) } - - def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } + def getBuildSide(join: HashJoin): GpuBuildSide = { + GpuJoinUtils.getBuildSide(join.buildSide) } def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } - } - def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } + GpuJoinUtils.getBuildSide(join.buildSide) } } diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala index bdfe0766b2c..47206481e07 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.rapids.shims.spark30 import java.util.concurrent.TimeUnit.NANOSECONDS import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} -import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} @@ -31,6 +30,7 @@ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 7d6c523d9f4..46df92197b9 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -16,15 +16,26 @@ package com.nvidia.spark.rapids.shims.spark31 +import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} -import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.optimizer.BuildSide import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, SerializeConcatHostBuffersDeserializeBatch} +import org.apache.spark.sql.vectorized.ColumnarBatch +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -32,29 +43,174 @@ case class GpuBroadcastHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuBroadcastHashJoinExecBase with Logging { + right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + + + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) + } + } + + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = { + val mode = HashedRelationBroadcastMode(buildKeys) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => + BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil + case GpuBuildRight => + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil + } + } + def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { + case gpu: GpuBroadcastExchangeExec => gpu + case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] + } + + override def doExecute(): RDD[InternalRow] = + throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val broadcastRelation = broadcastExchange + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + lazy val builtTable = { + // TODO clean up intermediate results... + val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) + val combined = combine(keys, broadcastRelation.value.batch) + val ret = GpuColumnVector.from(combined) + // Don't warn for a leak, because we cannot control when we are done with this + (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) + ret + } + + val rdd = streamedPlan.executeColumnar() + rdd.mapPartitions(it => + doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, + numOutputBatches, joinTime, filterTime, totalTime)) + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } - def getBuildSide: GpuBuildSide = { - buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) } } } -object GpuBroadcastHashJoinExec extends Logging { - - def createInstance( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - join: BroadcastHashJoinExec, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan): GpuBroadcastHashJoinExec = { - - GpuBroadcastHashJoinExec(leftKeys, rightKeys, joinType, join.buildSide, condition, left, right) +class GpuBroadcastHashJoinMeta( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + ShimLoader.getSparkShims.getBuildSide(join) + } + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => childPlans(0) + case GpuBuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => left + case GpuBuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala deleted file mode 100644 index 4d1319f6c12..00000000000 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExecBase.scala +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.spark31 - -import ai.rapids.cudf.{NvtxColor, Table} - -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ -import org.apache.spark.sql.rapids.execution._ - -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - -abstract class GpuBroadcastHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { - - def getBuildSide: GpuBuildSide - - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - getBuildSide match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) - } - } - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) - - override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(buildKeys) - getBuildSide match { - case GpuBuildLeft => - BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil - case GpuBuildRight => - UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil - } - } - def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { - case gpu: GpuBroadcastExchangeExec => gpu - case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] - } - - override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") - - val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() - - val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) - - lazy val builtTable = { - // TODO clean up intermediate results... - val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) - val combined = combine(keys, broadcastRelation.value.batch) - val ret = GpuColumnVector.from(combined) - // Don't warn for a leak, because we cannot control when we are done with this - (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) - ret - } - - val rdd = streamedPlan.executeColumnar() - rdd.mapPartitions(it => - doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, - numOutputBatches, joinTime, filterTime, totalTime)) - } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - getBuildSide match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } - -} - - -class GpuBroadcastHashJoinMeta( - join: BroadcastHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) with Logging { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - ShimLoader.getSparkShims.getBuildSide(join) - } - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => childPlans(0) - case GpuBuildRight => childPlans(1) - } - - if (!buildSide.canThisBeReplaced) { - willNotWorkOnGpu("the broadcast for this join must be on the GPU too") - } - - if (!canThisBeReplaced) { - buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") - } - } - - override def convertToGpu(): GpuExec = { - val left = childPlans(0).convertIfNeeded() - val right = childPlans(1).convertIfNeeded() - // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => left - case GpuBuildRight => right - } - if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { - throw new IllegalStateException("the broadcast must be on the GPU too") - } - GpuBroadcastHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, join.buildSide, - condition.map(_.convertToGpu()), - left, right) - } - -} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala index e8f27dec384..6f666a3bbdc 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala @@ -16,9 +16,8 @@ package com.nvidia.spark.rapids.shims.spark31 -import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.{GpuBuildLeft, GpuBuildRight, GpuBuildSide} -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.JoinType @@ -26,13 +25,16 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.rapids.execution._ +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuBroadcastNestedLoopJoinExec( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]) - extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) with Logging { + extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) { def getBuildSide: GpuBuildSide = { join.buildSide match { @@ -42,16 +44,3 @@ case class GpuBroadcastNestedLoopJoinExec( } } } - -object GpuBroadcastNestedLoopJoinExec extends Logging { - - def createInstance( - left: SparkPlan, - right: SparkPlan, - join: BroadcastNestedLoopJoinExec, - joinType: JoinType, - condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase = { - - GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) - } -} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala index 891b327497e..724d8b1d421 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala @@ -16,14 +16,15 @@ package com.nvidia.spark.rapids.shims.spark31 -import org.apache.spark.sql.rapids.GpuFirstBase - import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.aggregate.FirstLast +import org.apache.spark.sql.rapids.GpuFirstBase - +/** + * Parameters to GpuFirst changed in Spark 3.1 + */ case class GpuFirst(child: Expression, ignoreNulls: Boolean) extends GpuFirstBase(child) { def this(child: Expression) = this(child, false) def this(child: Expression, ignoreNullsExpr: Expression) = { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index 18fea577f56..c5d93bb8f39 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -19,12 +19,11 @@ import ai.rapids.cudf.Table import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { def tagJoin( @@ -34,15 +33,7 @@ object GpuHashJoin { rightKeys: Seq[Expression], condition: Option[Expression]): Unit = joinType match { case _: InnerLike => - case FullOuter => - if (leftKeys.exists(_.nullable) || rightKeys.exists(_.nullable)) { - // https://github.com/rapidsai/cudf/issues/5563 - meta.willNotWorkOnGpu("Full outer join does not work on nullable keys") - } - if (condition.isDefined) { - meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") - } - case RightOuter | LeftOuter | LeftSemi | LeftAnti => + case FullOuter | RightOuter | LeftOuter | LeftSemi | LeftAnti => if (condition.isDefined) { meta.willNotWorkOnGpu(s"$joinType joins currently do not support conditions") } @@ -50,9 +41,7 @@ object GpuHashJoin { } } - -trait GpuHashJoin extends GpuExec with HashJoin with Logging { - +trait GpuHashJoin extends GpuExec with HashJoin { override def output: Seq[Attribute] = { joinType match { @@ -73,7 +62,6 @@ trait GpuHashJoin extends GpuExec with HashJoin with Logging { } } - def doJoinInternal(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala index 3271d5ee8fb..66b730f5441 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala @@ -16,13 +16,14 @@ package com.nvidia.spark.rapids.shims.spark31 -import org.apache.spark.sql.rapids.GpuLastBase - import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.rapids.GpuLastBase - +/** + * Parameters to GpuFirst changed in Spark 3.1 + */ case class GpuLast(child: Expression, ignoreNulls: Boolean) extends GpuLastBase(child) { override def children: Seq[Expression] = child :: Nil diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index 7e345d5508c..7a3bb9d0d63 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -16,15 +16,36 @@ package com.nvidia.spark.rapids.shims.spark31 +import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch +object GpuJoinUtils { + def getBuildSide(buildSide: BuildSide): GpuBuildSide = { + buildSide match { + case BuildRight => GpuBuildRight + case BuildLeft => GpuBuildLeft + case _ => throw new Exception("unknown buildSide Type") + } + } +} + +/** + * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide + */ case class GpuShuffledHashJoinExec ( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -32,34 +53,164 @@ case class GpuShuffledHashJoinExec ( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends GpuShuffledHashJoinExecBase with Logging { + right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - def getBuildSide: GpuBuildSide = { - buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => (lkeys, rkeys) + case GpuBuildRight => (rkeys, lkeys) } } -} -object GpuShuffledHashJoinExec extends Logging { + override lazy val additionalMetrics: Map[String, SQLMetric] = Map( + "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), + "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), + "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), + "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), + "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + + override def requiredChildDistribution: Seq[Distribution] = + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException( + "GpuShuffledHashJoin does not support the execute() code path.") + } + + override def childrenCoalesceGoal: Seq[CoalesceGoal] = { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => Seq(RequireSingleBatch, null) + case GpuBuildRight => Seq(null, RequireSingleBatch) + } + } + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val buildDataSize = longMetric("buildDataSize") + val numOutputRows = longMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val totalTime = longMetric(TOTAL_TIME) + val buildTime = longMetric("buildTime") + val joinTime = longMetric("joinTime") + val filterTime = longMetric("filterTime") + val joinOutputRows = longMetric("joinOutputRows") + + val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) + + streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { + (streamIter, buildIter) => { + var combinedSize = 0 + val startTime = System.nanoTime() + val buildBatch = + ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) + val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) + val builtTable = try { + // Combine does not inc any reference counting + val combined = combine(keys, buildBatch) + combinedSize = + GpuColumnVector.extractColumns(combined) + .map(_.getBase.getDeviceMemorySize).sum.toInt + GpuColumnVector.from(combined) + } finally { + keys.close() + buildBatch.close() + } + + val delta = System.nanoTime() - startTime + buildTime += delta + totalTime += delta + buildDataSize += combinedSize + val context = TaskContext.get() + context.addTaskCompletionListener[Unit](_ => builtTable.close()) + + doJoin(builtTable, streamIter, boundCondition, + numOutputRows, joinOutputRows, numOutputBatches, + joinTime, filterTime, totalTime) + } + } + } + + def doJoinInternal(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + GpuJoinUtils.getBuildSide(buildSide) match { + case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) + case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } - def createInstance( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - join: SparkPlan, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan): GpuShuffledHashJoinExec = { + numJoinOutputRows += joined.numRows() - val buildSide: BuildSide = if (join.isInstanceOf[ShuffledHashJoinExec]) { - join.asInstanceOf[ShuffledHashJoinExec].buildSide + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) } else { - BuildRight + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined } - GpuShuffledHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } +} + +class GpuShuffledHashJoinMeta( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala deleted file mode 100644 index 4d1927e845e..00000000000 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExecBase.scala +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims.spark31 - -import ai.rapids.cudf.{NvtxColor, Table} - -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuHashJoinBaseMeta -import com.nvidia.spark.rapids.GpuMetricNames._ - -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} -import org.apache.spark.sql.execution.BinaryExecNode -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.internal.Logging - - -/** - * This has to be in the shim layer because GpuHashJoin is in shim. - */ -abstract class GpuShuffledHashJoinExecBase extends BinaryExecNode with GpuHashJoin with Logging { - - def getBuildSide: GpuBuildSide - - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - getBuildSide match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) - } - } - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) - - override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil - - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException( - "GpuShuffledHashJoin does not support the execute() code path.") - } - - override def childrenCoalesceGoal: Seq[CoalesceGoal] = getBuildSide match { - case GpuBuildLeft => Seq(RequireSingleBatch, null) - case GpuBuildRight => Seq(null, RequireSingleBatch) - } - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val buildDataSize = longMetric("buildDataSize") - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val buildTime = longMetric("buildTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") - - val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) - - streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { - (streamIter, buildIter) => { - var combinedSize = 0 - val startTime = System.nanoTime() - val buildBatch = - ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) - val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) - val builtTable = try { - // Combine does not inc any reference counting - val combined = combine(keys, buildBatch) - combinedSize = - GpuColumnVector.extractColumns(combined) - .map(_.getBase.getDeviceMemorySize).sum.toInt - GpuColumnVector.from(combined) - } finally { - keys.close() - buildBatch.close() - } - - val delta = System.nanoTime() - startTime - buildTime += delta - totalTime += delta - buildDataSize += combinedSize - val context = TaskContext.get() - context.addTaskCompletionListener[Unit](_ => builtTable.close()) - - doJoin(builtTable, streamIter, boundCondition, - numOutputRows, joinOutputRows, numOutputBatches, - joinTime, filterTime, totalTime) - } - } - } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - getBuildSide match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } - -} - - -class GpuShuffledHashJoinMeta( - join: ShuffledHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - } - - override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, - join.buildSide, - condition.map(_.convertToGpu()), - childPlans(0).convertIfNeeded(), - childPlans(1).convertIfNeeded()) - } -} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala index 075af30aad2..cac07b6a324 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala @@ -18,17 +18,19 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.optimizer.BuildRight import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.joins._ +/** + * HashJoin changed in Spark 3.1 requiring Shim + */ class GpuSortMergeJoinMeta( join: SortMergeJoinExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) with Logging { + extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -38,7 +40,6 @@ class GpuSortMergeJoinMeta( GpuOverrides.wrapExpr(_, conf, Some(this))) override def tagPlanForGpu(): Unit = { - // Use conditions from Hash Join GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) @@ -74,6 +75,4 @@ class GpuSortMergeJoinMeta( childPlans(0).convertIfNeeded(), childPlans(1).convertIfNeeded()) } - } - diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala index 114aee1e69a..b798205ac38 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.shims.spark31 -import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.{SparkShims, SparkShimServiceProvider} class Spark31ShimServiceProvider extends SparkShimServiceProvider { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 5f414d6bc92..498a4171a87 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -19,21 +19,19 @@ package com.nvidia.spark.rapids.shims.spark31 import java.time.ZoneId import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.shims.spark31.GpuBroadcastNestedLoopJoinExec import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.rapids.shims.spark31._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.sql.types._ class Spark31Shims extends SparkShims with Logging { @@ -135,41 +133,11 @@ class Spark31Shims extends SparkShims with Logging { ) } - def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } + def getBuildSide(join: HashJoin): GpuBuildSide = { + GpuJoinUtils.getBuildSide(join.buildSide) } def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } - } - def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - val buildSide = join.buildSide - buildSide match { - case e: buildSide.type if e.toString.contains("BuildRight") => { - GpuBuildRight - } - case l: buildSide.type if l.toString.contains("BuildLeft") => { - GpuBuildLeft - } - case _ => throw new Exception("unknown buildSide Type") - } + GpuJoinUtils.getBuildSide(join.buildSide) } } diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala index 53ba2fff21f..35cef4d7724 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.rapids.shims.spark31 import java.util.concurrent.TimeUnit.NANOSECONDS import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} -import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} @@ -31,6 +30,7 @@ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.rapids.GpuFileSourceScanExecBase import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet 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 c8851ecf7a9..31d7c66a9c4 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 @@ -31,9 +31,8 @@ case object GpuBuildLeft extends GpuBuildSide trait SparkShims { def isGpuHashJoin(plan: SparkPlan): Boolean - def getBuildSide(join: ShuffledHashJoinExec): GpuBuildSide + def getBuildSide(join: HashJoin): GpuBuildSide def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide - def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] def getExecs: Seq[ExecRule[_ <: SparkPlan]] def getGpuBroadcastNestedLoopJoinShims( 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 0676d12f22f..d65256b764d 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 @@ -20,7 +20,6 @@ import ai.rapids.cudf import com.nvidia.spark.rapids._ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, ExprId, ImplicitCastInputTypes, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, Complete, Final, Partial, PartialMerge} import org.apache.spark.sql.catalyst.util.TypeUtils diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala index 26e82195221..429c5b352a6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExecBase.scala @@ -16,23 +16,7 @@ package org.apache.spark.sql.rapids -import java.util.concurrent.TimeUnit.NANOSECONDS - -import com.nvidia.spark.rapids.{GpuExec, GpuReadCSVFileFormat, GpuReadOrcFileFormat, GpuReadParquetFileFormat, SparkPlanMeta} - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{DataSourceScanExec, ExplainUtils, FileSourceScanExec} -import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.collection.BitSet - -trait GpuFileSourceScanExecBase { -} +/** + * Base trait used for GpuFileSourceScanExec to use it in the Shim layer. + */ +trait GpuFileSourceScanExecBase From b77bf340ecdebce3e92180ebd60cdf6ce5e80746 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 12:48:47 -0500 Subject: [PATCH 18/47] pom fixes to generate docs --- dist/pom.xml | 25 +++++++++++++++---------- pom.xml | 1 + sql-plugin/pom.xml | 20 -------------------- 3 files changed, 16 insertions(+), 30 deletions(-) diff --git a/dist/pom.xml b/dist/pom.xml index 6eea5b7ecec..da3886e3c9a 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -46,6 +46,11 @@ rapids-4-spark-shims_${scala.binary.version} ${project.version} + + org.apache.spark + spark-sql_${scala.binary.version} + provided + @@ -102,16 +107,6 @@ - - org.apache.rat - apache-rat-plugin - - - dependency-reduced-pom.xml - *pom.xml.asc - - - net.alchim31.maven scala-maven-plugin @@ -136,6 +131,16 @@ + + org.apache.rat + apache-rat-plugin + + + dependency-reduced-pom.xml + *pom.xml.asc + + + diff --git a/pom.xml b/pom.xml index d2fabe10475..94877426ad6 100644 --- a/pom.xml +++ b/pom.xml @@ -487,6 +487,7 @@ pom.xml.asc jenkins/databricks/*.patch *.jar + **/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index f7465d8efc8..d80721f110a 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -144,26 +144,6 @@ net.alchim31.maven scala-maven-plugin - - - update_config - verify - - run - - - - - - - update_rapids_config - com.nvidia.spark.rapids.RapidsConf - - ${project.basedir}/../docs/configs.md - - - - org.scalastyle From 261bcc785a7f344089b0f309347e6c7712b6cf8f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 13:27:01 -0500 Subject: [PATCH 19/47] Fix Suite for shim classes and cleanup --- pom.xml | 2 +- .../spark30/GpuBroadcastHashJoinExec.scala | 111 +++++++++--------- .../rapids/shims/spark30/Spark30Shims.scala | 14 +++ .../rapids/shims/spark31/Spark31Shims.scala | 14 +++ .../nvidia/spark/rapids/GpuOverrides.scala | 13 -- .../com/nvidia/spark/rapids/SparkShims.scala | 2 + .../sql/rapids/datetimeExpressions.scala | 1 - tests/pom.xml | 8 +- .../spark/rapids/HashSortOptimizeSuite.scala | 7 +- 9 files changed, 96 insertions(+), 76 deletions(-) diff --git a/pom.xml b/pom.xml index 94877426ad6..e3821561a6e 100644 --- a/pom.xml +++ b/pom.xml @@ -130,7 +130,7 @@ - spark31 + spark31tests diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 15feb05ebc7..30f8930c263 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,61 @@ import org.apache.spark.sql.vectorized.ColumnarBatch /** * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide */ +class GpuBroadcastHashJoinMeta( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { + ShimLoader.getSparkShims.getBuildSide(join) + } + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => childPlans(0) + case GpuBuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = getBuildSide(join) match { + case GpuBuildLeft => left + case GpuBuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) + } +} + case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -158,57 +213,3 @@ case class GpuBroadcastHashJoinExec( } } -class GpuBroadcastHashJoinMeta( - join: BroadcastHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - ShimLoader.getSparkShims.getBuildSide(join) - } - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => childPlans(0) - case GpuBuildRight => childPlans(1) - } - - if (!buildSide.canThisBeReplaced) { - willNotWorkOnGpu("the broadcast for this join must be on the GPU too") - } - - if (!canThisBeReplaced) { - buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") - } - } - - override def convertToGpu(): GpuExec = { - val left = childPlans(0).convertIfNeeded() - val right = childPlans(1).convertIfNeeded() - // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => left - case GpuBuildRight => right - } - if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { - throw new IllegalStateException("the broadcast must be on the GPU too") - } - GpuBroadcastHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, join.buildSide, - condition.map(_.convertToGpu()), - left, right) - } -} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 2e723e90592..073631343db 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -51,6 +51,20 @@ class Spark30Shims extends SparkShims with Logging { } } + def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuBroadcastHashJoinExec => true + case p => false + } + } + + def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuShuffledHashJoinExec => true + case p => false + } + } + def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 498a4171a87..70308c771b1 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -52,6 +52,20 @@ class Spark31Shims extends SparkShims with Logging { } } + def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuBroadcastHashJoinExec => true + case p => false + } + } + + def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { + plan match { + case _: GpuShuffledHashJoinExec => true + case p => false + } + } + def getExprs: Seq[ExprRule[_ <: Expression]] = { Seq( 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 8f87aa1a988..a1d9bc02e5f 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 @@ -1672,14 +1672,6 @@ object GpuOverrides { exec[BroadcastExchangeExec]( "The backend for broadcast exchange of data", (exchange, conf, p, r) => new GpuBroadcastMeta(exchange, conf, p, r)), - /* - exec[BroadcastHashJoinExec]( - "Implementation of join using broadcast data", - (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), - exec[ShuffledHashJoinExec]( - "Implementation of join using hashed shuffled data", - (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), - */ exec[BroadcastNestedLoopJoinExec]( "Implementation of join using brute force", (join, conf, p, r) => new GpuBroadcastNestedLoopJoinMeta(join, conf, p, r)) @@ -1700,11 +1692,6 @@ object GpuOverrides { conf.gpuTargetBatchSizeBytes) }) .disabledByDefault("large joins can cause out of memory errors"), - /* - exec[SortMergeJoinExec]( - "Sort merge join, replacing with shuffled hash join", - (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), - */ exec[HashAggregateExec]( "The backend for hash based aggregations", (agg, conf, p, r) => new GpuHashAggregateMeta(agg, conf, p, r)), 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 31d7c66a9c4..54829d3a881 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 @@ -31,6 +31,8 @@ case object GpuBuildLeft extends GpuBuildSide trait SparkShims { def isGpuHashJoin(plan: SparkPlan): Boolean + def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean + def isGpuShuffledHashJoin(plan: SparkPlan): Boolean def getBuildSide(join: HashJoin): GpuBuildSide def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 7b587a600d0..51a17d7715f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -1,5 +1,4 @@ /* - * rg.apache.spark.sql.rapids * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/tests/pom.xml b/tests/pom.xml index c9da73c8f06..71364044755 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -36,7 +36,7 @@ - spark31 + spark31tests 3.1.0-SNAPSHOT @@ -81,6 +81,12 @@ ${project.version} test + + com.nvidia + rapids-4-spark-shims_${scala.binary.version} + ${project.version} + test + org.mockito mockito-core diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala index 1de997eea68..67cd2f2d54b 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala @@ -20,7 +20,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.execution.{SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec -// import org.apache.spark.sql.rapids.execution.GpuBroadcastHashJoinExecBase /** Test plan modifications to add optimizing sorts after hash joins in the plan */ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { @@ -65,11 +64,10 @@ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { assertResult(joinNode) { sortChild.children.head } } - /* test("sort inserted after broadcast hash join") { val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(_.isInstanceOf[GpuBroadcastHashJoinExecBase]) + val joinNode = plan.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) } @@ -78,11 +76,10 @@ class HashSortOptimizeSuite extends FunSuite with BeforeAndAfterAll { spark.conf.set("spark.sql.autoBroadcastJoinThreshold", 0) val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(_.isInstanceOf[GpuShuffledHashJoinExecBase]) + val joinNode = plan.find(ShimLoader.getSparkShims.isGpuShuffledHashJoin(_)) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) } - */ test("config to disable") { spark.conf.set(RapidsConf.ENABLE_HASH_OPTIMIZE_SORT.key, "false") From 5731bb9488c7793256a4760f36c137d2e6ff27c9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 14:14:06 -0500 Subject: [PATCH 20/47] shim layer for Rapids Shuffle Manager --- .../RapidsShuffleInternalManager.scala | 72 +++++++++++++++++++ .../RapidsShuffleInternalManager.scala | 58 +++++++++++++++ .../rapids/RapidsShuffleInternalManager.scala | 17 +---- 3 files changed, 133 insertions(+), 14 deletions(-) create mode 100644 shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala create mode 100644 shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala new file mode 100644 index 00000000000..51b90924372 --- /dev/null +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala @@ -0,0 +1,72 @@ +/* + * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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.shims.spark30 + +import ai.rapids.cudf.{NvtxColor, NvtxRange} +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.shuffle.{RapidsShuffleRequestHandler, RapidsShuffleServer, RapidsShuffleTransport} +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle._ +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage._ + + +/** + * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. + * @note This is an internal class to obtain access to the private + * `ShuffleManager` and `SortShuffleManager` classes. When configuring + * Apache Spark to use the RAPIDS shuffle manager, + * [[com.nvidia.spark.RapidsShuffleManager]] should be used as that is + * the public class. + */ +class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) + extends RapidsShuffleInternalManagerBase(conf, isDriver) with Logging { + + override def getReaderForRange[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + // NOTE: This type of reader is not possible for gpu shuffle, as we'd need + // to use the optimization within our manager, and we don't. + wrapped.getReaderForRange(unwrapHandle(handle), startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) + } + + def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + + getReaderInternal(handle, 0, Int.MaxValue, startPartition, endPartition, context, metrics) + } +} diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala new file mode 100644 index 00000000000..1b6a1216f68 --- /dev/null +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala @@ -0,0 +1,58 @@ +/* + * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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.shims.spark31 + +import ai.rapids.cudf.{NvtxColor, NvtxRange} +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.shuffle.{RapidsShuffleRequestHandler, RapidsShuffleServer, RapidsShuffleTransport} +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle._ +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage._ + + +/** + * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. + * @note This is an internal class to obtain access to the private + * `ShuffleManager` and `SortShuffleManager` classes. When configuring + * Apache Spark to use the RAPIDS shuffle manager, + * [[com.nvidia.spark.RapidsShuffleManager]] should be used as that is + * the public class. + */ +class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) + extends RapidsShuffleInternalManagerBase(conf, isDriver) with Logging { + + def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + + getReaderInternal(handle, 0, Int.MaxValue, startPartition, endPartition, context, metrics) + } +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala index d6e38a0027b..c0175e1806f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala @@ -185,7 +185,7 @@ class RapidsCachingWriter[K, V]( * [[com.nvidia.spark.RapidsShuffleManager]] should be used as that is * the public class. */ -class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) +abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boolean) extends ShuffleManager with Logging { import RapidsShuffleInternalManager._ @@ -295,7 +295,7 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) } } - override def getReaderForRange[K, C]( + def getReaderInternal[K, C]( handle: ShuffleHandle, startMapIndex: Int, endMapIndex: Int, @@ -303,18 +303,7 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - // NOTE: This type of reader is not possible for gpu shuffle, as we'd need - // to use the optimization within our manager, and we don't. - wrapped.getReaderForRange(unwrapHandle(handle), startMapIndex, endMapIndex, - startPartition, endPartition, context, metrics) - } - - override def getReader[K, C]( - handle: ShuffleHandle, - startPartition: Int, - endPartition: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + // startMapIndex and endMapIndex ignored as we don't support those for gpu shuffle. handle match { case gpu: GpuShuffleHandle[_, _] => logInfo(s"Asking map output tracker for dependency ${gpu.dependency}, " + From 46db4497fb90d5b3cf5fa390dce3493cffef4614 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 14:30:15 -0500 Subject: [PATCH 21/47] Shim for shuffle manager --- .../spark30}/RapidsShuffleManager.scala | 4 +-- .../RapidsShuffleInternalManager.scala | 22 +++------------- .../rapids/spark31/RapidsShuffleManager.scala | 26 +++++++++++++++++++ .../RapidsShuffleInternalManager.scala | 18 ++----------- .../rapids/RapidsShuffleInternalManager.scala | 2 +- 5 files changed, 35 insertions(+), 37 deletions(-) rename {sql-plugin/src/main/scala/com/nvidia/spark => shims/spark30/src/main/scala/com/nvidia/spark/rapids/spark30}/RapidsShuffleManager.scala (87%) create mode 100644 shims/spark31/src/main/scala/com/nvidia/spark/rapids/spark31/RapidsShuffleManager.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/RapidsShuffleManager.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/spark30/RapidsShuffleManager.scala similarity index 87% rename from sql-plugin/src/main/scala/com/nvidia/spark/RapidsShuffleManager.scala rename to shims/spark30/src/main/scala/com/nvidia/spark/rapids/spark30/RapidsShuffleManager.scala index db9e415b9e0..42094ea1487 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/RapidsShuffleManager.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/spark30/RapidsShuffleManager.scala @@ -14,10 +14,10 @@ * limitations under the License. */ -package com.nvidia.spark +package com.nvidia.spark.rapids.spark30 import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.RapidsShuffleInternalManager +import org.apache.spark.sql.rapids.shims.spark30.RapidsShuffleInternalManager /** A shuffle manager optimized for the RAPIDS Plugin for Apache Spark. */ sealed class RapidsShuffleManager( diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala index 51b90924372..52589d80ba3 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala @@ -16,22 +16,8 @@ package org.apache.spark.sql.rapids.shims.spark30 -import ai.rapids.cudf.{NvtxColor, NvtxRange} -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.format.TableMeta -import com.nvidia.spark.rapids.shuffle.{RapidsShuffleRequestHandler, RapidsShuffleServer, RapidsShuffleTransport} -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.io.CompressionCodec -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.storage._ - /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -42,7 +28,7 @@ import org.apache.spark.storage._ * the public class. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) with Logging { + extends RapidsShuffleInternalManagerBase(conf, isDriver) { override def getReaderForRange[K, C]( handle: ShuffleHandle, @@ -54,8 +40,8 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { // NOTE: This type of reader is not possible for gpu shuffle, as we'd need // to use the optimization within our manager, and we don't. - wrapped.getReaderForRange(unwrapHandle(handle), startMapIndex, endMapIndex, - startPartition, endPartition, context, metrics) + wrapped.getReaderForRange(RapidsShuffleInternalManager.unwrapHandle(handle), + startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) } def getReader[K, C]( diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/spark31/RapidsShuffleManager.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/spark31/RapidsShuffleManager.scala new file mode 100644 index 00000000000..cf1941f4e07 --- /dev/null +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/spark31/RapidsShuffleManager.scala @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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.spark31 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.rapids.shims.spark31.RapidsShuffleInternalManager + +/** A shuffle manager optimized for the RAPIDS Plugin for Apache Spark. */ +sealed class RapidsShuffleManager( + conf: SparkConf, + isDriver: Boolean) extends RapidsShuffleInternalManager(conf, isDriver) { +} diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala index 1b6a1216f68..dc6e4dbeaeb 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala @@ -16,22 +16,8 @@ package org.apache.spark.sql.rapids.shims.spark31 -import ai.rapids.cudf.{NvtxColor, NvtxRange} -import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.format.TableMeta -import com.nvidia.spark.rapids.shuffle.{RapidsShuffleRequestHandler, RapidsShuffleServer, RapidsShuffleTransport} -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.io.CompressionCodec -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.storage._ - /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -42,7 +28,7 @@ import org.apache.spark.storage._ * the public class. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) with Logging { + extends RapidsShuffleInternalManagerBase(conf, isDriver) { def getReader[K, C]( handle: ShuffleHandle, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala index c0175e1806f..3d07a44b258 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala @@ -192,7 +192,7 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boole private val rapidsConf = new RapidsConf(conf) - private val wrapped = new SortShuffleManager(conf) + protected val wrapped = new SortShuffleManager(conf) GpuShuffleEnv.setRapidsShuffleManagerInitialized(true, this.getClass.getCanonicalName) logWarning("Rapids Shuffle Plugin Enabled") From af1d79df22208e42719b30a27a7be11286809554 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 14:40:59 -0500 Subject: [PATCH 22/47] add in getRapidsShuffleManagerClass --- .../rapids/shims/spark30/Spark30Shims.scala | 20 +++++++++++------- .../rapids/shims/spark31/Spark31Shims.scala | 21 ++++++++++++------- .../com/nvidia/spark/rapids/SparkShims.scala | 1 + .../spark/sql/rapids/GpuShuffleEnv.scala | 3 +-- 4 files changed, 27 insertions(+), 18 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 073631343db..793fac9d37a 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -35,7 +35,7 @@ import org.apache.spark.unsafe.types.CalendarInterval class Spark30Shims extends SparkShims with Logging { - def getGpuBroadcastNestedLoopJoinShims( + override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, @@ -44,28 +44,28 @@ class Spark30Shims extends SparkShims with Logging { GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) } - def isGpuHashJoin(plan: SparkPlan): Boolean = { + override def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuHashJoin => true case p => false } } - def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { + override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuBroadcastHashJoinExec => true case p => false } } - def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { + override def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuShuffledHashJoinExec => true case p => false } } - def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { + override def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( GpuOverrides.exec[FileSourceScanExec]( @@ -106,7 +106,7 @@ class Spark30Shims extends SparkShims with Logging { ) } - def getExprs: Seq[ExprRule[_ <: Expression]] = { + override def getExprs: Seq[ExprRule[_ <: Expression]] = { Seq( GpuOverrides.expr[TimeSub]( "Subtracts interval from timestamp", @@ -157,12 +157,16 @@ class Spark30Shims extends SparkShims with Logging { ) } - def getBuildSide(join: HashJoin): GpuBuildSide = { + override def getBuildSide(join: HashJoin): GpuBuildSide = { GpuJoinUtils.getBuildSide(join.buildSide) } - def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { + override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { GpuJoinUtils.getBuildSide(join.buildSide) } + + override def getRapidsShuffleManagerClass: String = { + classOf[RapidsShuffleManager].getCanonicalName + } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 70308c771b1..2975ca8dc3f 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ class Spark31Shims extends SparkShims with Logging { - def getGpuBroadcastNestedLoopJoinShims( + override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, @@ -45,28 +45,28 @@ class Spark31Shims extends SparkShims with Logging { GpuBroadcastNestedLoopJoinExec(left, right, join, joinType, condition) } - def isGpuHashJoin(plan: SparkPlan): Boolean = { + override def isGpuHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuHashJoin => true case p => false } } - def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { + override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuBroadcastHashJoinExec => true case p => false } } - def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { + override def isGpuShuffledHashJoin(plan: SparkPlan): Boolean = { plan match { case _: GpuShuffledHashJoinExec => true case p => false } } - def getExprs: Seq[ExprRule[_ <: Expression]] = { + override def getExprs: Seq[ExprRule[_ <: Expression]] = { Seq( GpuOverrides.expr[TimeAdd]( @@ -106,7 +106,7 @@ class Spark31Shims extends SparkShims with Logging { } - def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { + override def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( GpuOverrides.exec[FileSourceScanExec]( @@ -147,11 +147,16 @@ class Spark31Shims extends SparkShims with Logging { ) } - def getBuildSide(join: HashJoin): GpuBuildSide = { + override def getBuildSide(join: HashJoin): GpuBuildSide = { GpuJoinUtils.getBuildSide(join.buildSide) } - def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { + override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { GpuJoinUtils.getBuildSide(join.buildSide) } + + override def getRapidsShuffleManagerClass: String = { + classOf[RapidsShuffleManager].getCanonicalName + } + } 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 54829d3a881..bfe15f1319b 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 @@ -33,6 +33,7 @@ trait SparkShims { def isGpuHashJoin(plan: SparkPlan): Boolean def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean def isGpuShuffledHashJoin(plan: SparkPlan): Boolean + def getRapidsShuffleManagerClass: String def getBuildSide(join: HashJoin): GpuBuildSide def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala index f2f578fc2e9..b8f1f0c25ee 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf.{CudaMemInfo, Rmm} -import com.nvidia.spark.RapidsShuffleManager import com.nvidia.spark.rapids._ import org.apache.spark.{SparkConf, SparkEnv} @@ -25,7 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils object GpuShuffleEnv extends Logging { - private val RAPIDS_SHUFFLE_CLASS = classOf[RapidsShuffleManager].getCanonicalName + private val RAPIDS_SHUFFLE_CLASS = ShimLoader.getSparkShims.getRapidsShuffleManagerClass private var isRapidsShuffleManagerInitialized: Boolean = false private val catalog = new RapidsBufferCatalog From 7952d9fcb83637c27d302eed65ed20b5c56ae38a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 14:54:45 -0500 Subject: [PATCH 23/47] Cleanup shuffle manager --- .../nvidia/spark/rapids/shims/spark30/Spark30Shims.scala | 6 ++++-- .../shims/spark30/RapidsShuffleInternalManager.scala | 3 +-- .../nvidia/spark/rapids/shims/spark31/Spark31Shims.scala | 8 ++++---- .../shims/spark31/RapidsShuffleInternalManager.scala | 1 + 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 793fac9d37a..546d148a914 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -19,8 +19,8 @@ package com.nvidia.spark.rapids.shims.spark30 import java.time.ZoneId import com.nvidia.spark.rapids._ -import org.apache.spark.sql.rapids.GpuTimeSub -import org.apache.spark.sql.rapids.shims.spark30._ +import com.nvidia.spark.rapids.spark30.RapidsShuffleManager + import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -29,7 +29,9 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNes import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase +import org.apache.spark.sql.rapids.shims.spark30._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala index 52589d80ba3..c31bda90012 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.rapids.shims.spark30 import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.shuffle._ +import org.apache.spark.sql.rapids.{RapidsShuffleInternalManager, RapidsShuffleInternalManagerBase} /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -46,8 +47,6 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) def getReader[K, C]( handle: ShuffleHandle, - startMapIndex: Int, - endMapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 2975ca8dc3f..a0b340946b6 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -19,22 +19,22 @@ package com.nvidia.spark.rapids.shims.spark31 import java.time.ZoneId import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.spark31.RapidsShuffleManager +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.rapids.shims.spark31._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.sql.types._ -class Spark31Shims extends SparkShims with Logging { +class Spark31Shims extends SparkShims { override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala index dc6e4dbeaeb..02710534abf 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.rapids.shims.spark31 import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.shuffle._ +import org.apache.spark.sql.rapids.RapidsShuffleInternalManagerBase /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. From e495820731fcfa004009604ce8cd3e35b824fe50 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 15:21:30 -0500 Subject: [PATCH 24/47] Changes for shuffle manager --- integration_tests/pom.xml | 13 +++++++++++++ .../rapids/tests/mortgage/MortgageSparkSuite.scala | 4 ++-- .../rapids/tests/tpch/TpchLikeSparkSuite.scala | 4 ++-- .../spark30/RapidsShuffleInternalManager.scala | 4 ++-- .../scala/com/nvidia/spark/rapids/ShimLoader.scala | 12 +++--------- .../sql/rapids/RapidsShuffleInternalManager.scala | 7 +++---- 6 files changed, 25 insertions(+), 19 deletions(-) diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index 564584448e1..a560417f11a 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -28,6 +28,18 @@ rapids-4-spark-integration-tests_2.12 0.2.0-SNAPSHOT + + 3.0.0 + + + + spark31tests + + 3.1.0-SNAPSHOT + + + + org.scala-lang @@ -36,6 +48,7 @@ org.apache.spark spark-sql_${scala.binary.version} + ${spark.test.version} org.scalatest diff --git a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala index d36d65c1295..e2fb9d4530b 100644 --- a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala +++ b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tests.mortgage -import com.nvidia.spark.RapidsShuffleManager +import com.nvidia.spark.rapids.ShimLoader import org.scalatest.FunSuite import org.apache.spark.sql.SparkSession @@ -34,7 +34,7 @@ class MortgageSparkSuite extends FunSuite { .config("spark.rapids.sql.test.enabled", false) .config("spark.rapids.sql.incompatibleOps.enabled", true) .config("spark.rapids.sql.hasNans", false) - val rapidsShuffle = classOf[RapidsShuffleManager].getCanonicalName + val rapidsShuffle = ShimLoader.getSparkShims.getRapidsShuffleManagerClass val prop = System.getProperty("rapids.shuffle.manager.override", "false") if (prop.equalsIgnoreCase("true")) { println("RAPIDS SHUFFLE MANAGER ACTIVE") diff --git a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala index ebbf9fa1067..354047f125f 100644 --- a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala +++ b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tests.tpch -import com.nvidia.spark.RapidsShuffleManager +import com.nvidia.spark.rapids.ShimLoader import com.nvidia.spark.rapids.{ColumnarRdd, ExecutionPlanCaptureCallback} import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -44,7 +44,7 @@ class TpchLikeSparkSuite extends FunSuite with BeforeAndAfterAll { .config("spark.rapids.sql.explain", true) .config("spark.rapids.sql.incompatibleOps.enabled", true) .config("spark.rapids.sql.hasNans", false) - val rapidsShuffle = classOf[RapidsShuffleManager].getCanonicalName + val rapidsShuffle = ShimLoader.getSparkShims.getRapidsShuffleManagerClass val prop = System.getProperty("rapids.shuffle.manager.override", "false") if (prop.equalsIgnoreCase("true")) { println("RAPIDS SHUFFLE MANAGER ACTIVE") diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala index c31bda90012..93692b49028 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.shims.spark30 import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.shuffle._ -import org.apache.spark.sql.rapids.{RapidsShuffleInternalManager, RapidsShuffleInternalManagerBase} +import org.apache.spark.sql.rapids.RapidsShuffleInternalManagerBase /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -41,7 +41,7 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { // NOTE: This type of reader is not possible for gpu shuffle, as we'd need // to use the optimization within our manager, and we don't. - wrapped.getReaderForRange(RapidsShuffleInternalManager.unwrapHandle(handle), + wrapped.getReaderForRange(RapidsShuffleInternalManagerBase.unwrapHandle(handle), startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 549c4bd068d..878acb2a559 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -19,20 +19,14 @@ package com.nvidia.spark.rapids import java.util.ServiceLoader import scala.collection.JavaConverters._ -import scala.collection.immutable.HashMap import org.apache.spark.{SPARK_BUILD_USER, SPARK_VERSION} import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.rapids.execution._ object ShimLoader extends Logging { - private val sparkVersion = getVersion - logInfo(s"Loading shim for version: $sparkVersion") + private val sparkVersion = getSparkVersion + logInfo(s"Loading shim for Spark version: $sparkVersion") // This is not ideal, but pass the version in here because otherwise loader that match the // same version (3.0.0 Apache and 3.0.0 Databricks) would need to know how to differentiate. @@ -55,7 +49,7 @@ object ShimLoader extends Logging { sparkShims } - def getVersion: String = { + def getSparkVersion: String = { // hack for databricks, try to find something more reliable? if (SPARK_BUILD_USER.equals("Databricks")) { SPARK_VERSION + "-databricks" diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala index 3d07a44b258..1cedc0adea1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala @@ -64,7 +64,7 @@ class GpuShuffleBlockResolver(private val wrapped: ShuffleBlockResolver, } -object RapidsShuffleInternalManager extends Logging { +object RapidsShuffleInternalManagerBase extends Logging { def unwrapHandle(handle: ShuffleHandle): ShuffleHandle = handle match { case gh: GpuShuffleHandle[_, _] => gh.wrapped case other => other @@ -188,8 +188,6 @@ class RapidsCachingWriter[K, V]( abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boolean) extends ShuffleManager with Logging { - import RapidsShuffleInternalManager._ - private val rapidsConf = new RapidsConf(conf) protected val wrapped = new SortShuffleManager(conf) @@ -329,7 +327,8 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boole transport, catalog) case other => { - wrapped.getReader(unwrapHandle(other), startPartition, endPartition, context, metrics) + val shuffleHandle = RapidsShuffleInternalManagerBase.unwrapHandle(other) + wrapped.getReader(shuffleHandle, startPartition, endPartition, context, metrics) } } } From 50bad9d643db9c7365798c5ee3f253ebf5bb99a2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 15:45:15 -0500 Subject: [PATCH 25/47] Cleanup --- integration_tests/pom.xml | 12 ++++++++++ .../spark30/GpuBroadcastHashJoinExec.scala | 10 ++++---- .../GpuBroadcastNestedLoopJoinExec.scala | 6 +---- .../rapids/shims/spark30/GpuHashJoin.scala | 4 ++-- .../spark30/GpuShuffledHashJoinExec.scala | 11 ++++----- .../shims/spark30/GpuSortMergeJoinExec.scala | 3 +-- .../rapids/shims/spark30/Spark30Shims.scala | 4 ++-- .../GpuBroadcastNestedLoopJoinExec.scala | 6 +---- .../nvidia/spark/rapids/GpuOverrides.scala | 23 ------------------- 9 files changed, 29 insertions(+), 50 deletions(-) diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index a560417f11a..97c566db86a 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -29,6 +29,7 @@ 0.2.0-SNAPSHOT + 1.7.30 3.0.0 @@ -41,6 +42,17 @@ + + org.slf4j + jul-to-slf4j + ${slf4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + org.scala-lang scala-library diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 30f8930c263..c0f7494884b 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -46,8 +46,8 @@ class GpuBroadcastHashJoinMeta( join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) val rightKeys: Seq[BaseExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = + join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { ShimLoader.getSparkShims.getBuildSide(join) @@ -104,7 +104,7 @@ case class GpuBroadcastHashJoinExec( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => (lkeys, rkeys) case GpuBuildRight => (rkeys, lkeys) } @@ -117,7 +117,7 @@ case class GpuBroadcastHashJoinExec( override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil case GpuBuildRight => @@ -184,7 +184,7 @@ case class GpuBroadcastHashJoinExec( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala index 3be2f3810fd..000c7f437f1 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala @@ -36,10 +36,6 @@ case class GpuBroadcastNestedLoopJoinExec( extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) { def getBuildSide: GpuBuildSide = { - join.buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") - } + GpuJoinUtils.getGpuBuildSide(join.buildSide) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 9b49356bb97..96797ee53bb 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -62,7 +62,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoinInternal(builtTable: Table, + def doJoinInternal(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], numOutputRows: SQLMetric, @@ -164,7 +164,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { + protected def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { val joinedTable = joinType match { case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) .leftJoin(rightTable.onColumns(joinKeyIndices: _*)) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index e7a0653c19c..cf733dd388e 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch object GpuJoinUtils { - def getBuildSide(buildSide: BuildSide): GpuBuildSide = { + def getGpuBuildSide(buildSide: BuildSide): GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight case BuildLeft => GpuBuildLeft @@ -53,13 +53,12 @@ case class GpuShuffledHashJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => (lkeys, rkeys) case GpuBuildRight => (rkeys, lkeys) } @@ -81,7 +80,7 @@ case class GpuShuffledHashJoinExec( } override def childrenCoalesceGoal: Seq[CoalesceGoal] = { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => Seq(RequireSingleBatch, null) case GpuBuildRight => Seq(null, RequireSingleBatch) } @@ -155,7 +154,7 @@ case class GpuShuffledHashJoinExec( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index c4459471ca1..ccaaa61fa97 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,7 +39,6 @@ class GpuSortMergeJoinMeta( GpuOverrides.wrapExpr(_, conf, Some(this))) override def tagPlanForGpu(): Unit = { - // Use conditions from Hash Join GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 546d148a914..798857d6d01 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -160,11 +160,11 @@ class Spark30Shims extends SparkShims with Logging { } override def getBuildSide(join: HashJoin): GpuBuildSide = { - GpuJoinUtils.getBuildSide(join.buildSide) + GpuJoinUtils.getGpuBuildSide(join.buildSide) } override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - GpuJoinUtils.getBuildSide(join.buildSide) + GpuJoinUtils.getGpuBuildSide(join.buildSide) } override def getRapidsShuffleManagerClass: String = { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala index 6f666a3bbdc..8842e128306 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala @@ -37,10 +37,6 @@ case class GpuBroadcastNestedLoopJoinExec( extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition) { def getBuildSide: GpuBuildSide = { - join.buildSide match { - case BuildRight => GpuBuildRight - case BuildLeft => GpuBuildLeft - case _ => throw new Exception("unknown buildSide Type") - } + GpuJoinUtils.getGpuBuildSide(join.buildSide) } } 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 a1d9bc02e5f..625ff478a5a 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 @@ -715,29 +715,6 @@ object GpuOverrides { GpuDateSub(lhs, rhs) } ), - /* - expr[TimeSub]( - "Subtracts interval from timestamp", - (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.interval match { - case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => - if (intvl.months != 0) { - willNotWorkOnGpu("interval months isn't supported") - } - case _ => - willNotWorkOnGpu("only literals are supported for intervals") - } - if (ZoneId.of(a.timeZoneId.get).normalized() != UTC_TIMEZONE_ID) { - willNotWorkOnGpu("Only UTC zone id is supported") - } - } - - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - GpuTimeSub(lhs, rhs) - } - ), - */ expr[NaNvl]( "evaluates to `left` iff left is not NaN, `right` otherwise.", (a, conf, p, r) => new BinaryExprMeta[NaNvl](a, conf, p, r) { From 350c34b6a187450b38c9164529bd65850719e363 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 15:57:50 -0500 Subject: [PATCH 26/47] Change spark3.1 getGpuBuildSide --- .../rapids/shims/spark31/GpuBroadcastHashJoinExec.scala | 6 +++--- .../rapids/shims/spark31/GpuShuffledHashJoinExec.scala | 8 ++++---- .../nvidia/spark/rapids/shims/spark31/Spark31Shims.scala | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 46df92197b9..9652bd14587 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -51,7 +51,7 @@ case class GpuBroadcastHashJoinExec( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => (lkeys, rkeys) case GpuBuildRight => (rkeys, lkeys) } @@ -64,7 +64,7 @@ case class GpuBroadcastHashJoinExec( override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil case GpuBuildRight => @@ -131,7 +131,7 @@ case class GpuBroadcastHashJoinExec( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index 7a3bb9d0d63..f4f27d80a02 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch object GpuJoinUtils { - def getBuildSide(buildSide: BuildSide): GpuBuildSide = { + def getGpuBuildSide(buildSide: BuildSide): GpuBuildSide = { buildSide match { case BuildRight => GpuBuildRight case BuildLeft => GpuBuildLeft @@ -60,7 +60,7 @@ case class GpuShuffledHashJoinExec ( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => (lkeys, rkeys) case GpuBuildRight => (rkeys, lkeys) } @@ -82,7 +82,7 @@ case class GpuShuffledHashJoinExec ( } override def childrenCoalesceGoal: Seq[CoalesceGoal] = { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => Seq(RequireSingleBatch, null) case GpuBuildRight => Seq(null, RequireSingleBatch) } @@ -156,7 +156,7 @@ case class GpuShuffledHashJoinExec ( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getBuildSide(buildSide) match { + GpuJoinUtils.getGpuBuildSide(buildSide) match { case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index a0b340946b6..bafeaf276e0 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -148,11 +148,11 @@ class Spark31Shims extends SparkShims { } override def getBuildSide(join: HashJoin): GpuBuildSide = { - GpuJoinUtils.getBuildSide(join.buildSide) + GpuJoinUtils.getGpuBuildSide(join.buildSide) } override def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide = { - GpuJoinUtils.getBuildSide(join.buildSide) + GpuJoinUtils.getGpuBuildSide(join.buildSide) } override def getRapidsShuffleManagerClass: String = { From 0a9aeedeb15ab81a9d424b80850ae54cd03f7790 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 16:01:25 -0500 Subject: [PATCH 27/47] MapOutputTracker api --- .../nvidia/spark/rapids/shims/spark30/Spark30Shims.scala | 2 ++ .../main/scala/com/nvidia/spark/rapids/SparkShims.scala | 9 ++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 798857d6d01..c6762609eda 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -37,6 +37,8 @@ import org.apache.spark.unsafe.types.CalendarInterval class Spark30Shims extends SparkShims with Logging { + + override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, right: SparkPlan, 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 bfe15f1319b..34b1878976c 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 @@ -29,7 +29,6 @@ case object GpuBuildRight extends GpuBuildSide case object GpuBuildLeft extends GpuBuildSide trait SparkShims { - def isGpuHashJoin(plan: SparkPlan): Boolean def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean def isGpuShuffledHashJoin(plan: SparkPlan): Boolean @@ -38,12 +37,20 @@ trait SparkShims { def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] def getExecs: Seq[ExecRule[_ <: SparkPlan]] + def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, right: SparkPlan, join: BroadcastNestedLoopJoinExec, joinType: JoinType, condition: Option[Expression]): GpuBroadcastNestedLoopJoinExecBase + + def getMapSizesByExecutorId( + shuffleId: Int, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] } From 9b611f407640f494fc2643337796e1b5c8a309a2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 16:08:26 -0500 Subject: [PATCH 28/47] shim for mapoutputTracker api --- .../spark/rapids/shims/spark30/Spark30Shims.scala | 12 +++++++++++- .../spark/rapids/shims/spark31/Spark31Shims.scala | 13 ++++++++++++- .../scala/com/nvidia/spark/rapids/SparkShims.scala | 1 + .../sql/rapids/RapidsShuffleInternalManager.scala | 4 ++-- 4 files changed, 26 insertions(+), 4 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index c6762609eda..222cbb55622 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -21,6 +21,7 @@ import java.time.ZoneId import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.spark30.RapidsShuffleManager +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ @@ -33,11 +34,20 @@ import org.apache.spark.sql.rapids.GpuTimeSub import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.rapids.shims.spark30._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval class Spark30Shims extends SparkShims with Logging { - + override def getMapSizesByExecutorId( + shuffleId: Int, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + // startMapIndex and endMapIndex ignored as we don't support those for gpu shuffle. + SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, startPartition, endPartition) + } override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index bafeaf276e0..21594ae56e3 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -20,7 +20,7 @@ import java.time.ZoneId import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.spark31.RapidsShuffleManager - +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} @@ -33,9 +33,20 @@ import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.rapids.shims.spark31._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.sql.types._ +import org.apache.spark.storage.{BlockId, BlockManagerId} class Spark31Shims extends SparkShims { + override def getMapSizesByExecutorId( + shuffleId: Int, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, + startMapIndex, endMapIndex, startPartition, endPartition) + } + override def getGpuBroadcastNestedLoopJoinShims( left: SparkPlan, right: SparkPlan, 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 34b1878976c..bef2560ddfb 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 @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase +import org.apache.spark.storage.{BlockId, BlockManagerId} sealed abstract class GpuBuildSide diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala index 1cedc0adea1..d55d83f1d50 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala @@ -301,7 +301,6 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boole endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - // startMapIndex and endMapIndex ignored as we don't support those for gpu shuffle. handle match { case gpu: GpuShuffleHandle[_, _] => logInfo(s"Asking map output tracker for dependency ${gpu.dependency}, " + @@ -314,7 +313,8 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boole val nvtxRange = new NvtxRange("getMapSizesByExecId", NvtxColor.CYAN) val blocksByAddress = try { - env.mapOutputTracker.getMapSizesByExecutorId(gpu.shuffleId, startPartition, endPartition) + ShimLoader.getSparkShims.getMapSizesByExecutorId(gpu.shuffleId, + startMapIndex, endMapIndex, startPartition, endPartition) } finally { nvtxRange.close() } From 27d786c35e8e2d1634ebfec149c8d260a0d97748 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 21 Jul 2020 16:15:16 -0500 Subject: [PATCH 29/47] explicitly set version in shims --- shims/spark30/pom.xml | 3 ++- shims/spark31/pom.xml | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index 92138418f15..44fbf1339f6 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -32,7 +32,7 @@ 0.2.0-SNAPSHOT - 3.0.0 + 3.0.0 @@ -58,6 +58,7 @@ org.apache.spark spark-sql_${scala.binary.version} + ${spark30.version} org.apache.orc diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 110ca1ff83c..6b8451ef506 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -32,7 +32,7 @@ 0.2.0-SNAPSHOT - 3.1.0-SNAPSHOT + 3.1.0-SNAPSHOT @@ -58,6 +58,7 @@ org.apache.spark spark-sql_${scala.binary.version} + ${spark31.version} org.apache.orc From df7916df971b90ea9f376fd8615cb9425276cfd6 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 09:34:10 -0500 Subject: [PATCH 30/47] Move ScalaUDF to Shim Signed-off-by: Thomas Graves --- .../rapids/shims/spark30/Spark30Shims.scala | 19 +++++++++++++++---- .../rapids/shims/spark31/Spark31Shims.scala | 13 +++++++++++++ .../com/nvidia/spark/rapids/SparkShims.scala | 11 +++++++++++ .../sql/rapids/GpuFileFormatDataWriter.scala | 2 +- 4 files changed, 40 insertions(+), 5 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 4a3499acffb..89fcefb3c0a 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -22,8 +22,8 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.spark30.RapidsShuffleManager import org.apache.spark.SparkEnv +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} @@ -37,7 +37,20 @@ import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.spark.unsafe.types.CalendarInterval -class Spark30Shims extends SparkShims with Logging { +class Spark30Shims extends SparkShims { + + override def getScalaUDFAsExpression( + function: AnyRef, + dataType: DataType, + children: Seq[Expression], + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, + udfName: Option[String] = None, + nullable: Boolean = true, + udfDeterministic: Boolean = true): Expression = { + // outputEncoder is only used in Spark 3.1+ + ScalaUDF(function, dataType, children, inputEncoders, udfName, nullable, udfDeterministic) + } override def getMapSizesByExecutorId( shuffleId: Int, @@ -127,7 +140,6 @@ class Spark30Shims extends SparkShims with Logging { "Subtracts interval from timestamp", (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { override def tagExprForGpu(): Unit = { - logWarning("in TimeSub") a.interval match { case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => if (intvl.months != 0) { @@ -142,7 +154,6 @@ class Spark30Shims extends SparkShims with Logging { } override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { - logWarning("in TimeSub convert") GpuTimeSub(lhs, rhs) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 9dbefc3601b..34d0f96dd8c 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -21,6 +21,7 @@ import java.time.ZoneId import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.spark31.RapidsShuffleManager import org.apache.spark.SparkEnv +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} @@ -37,6 +38,18 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} class Spark31Shims extends SparkShims { + override def getScalaUDFAsExpression( + function: AnyRef, + dataType: DataType, + children: Seq[Expression], + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, + udfName: Option[String] = None, + nullable: Boolean = true, + udfDeterministic: Boolean = true): Expression = { + ScalaUDF(function, dataType, children, inputEncoders, outputEncoder, udfName, nullable, udfDeterministic) + } + override def getMapSizesByExecutorId( shuffleId: Int, startMapIndex: Int, 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 4a6e84cc5cc..a0f7b08b1ac 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 @@ -16,11 +16,13 @@ package com.nvidia.spark.rapids +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase +import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} sealed abstract class GpuBuildSide @@ -38,6 +40,15 @@ trait SparkShims { def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide def getExprs: Seq[ExprRule[_ <: Expression]] def getExecs: Seq[ExecRule[_ <: SparkPlan]] + def getScalaUDFAsExpression( + function: AnyRef, + dataType: DataType, + children: Seq[Expression], + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, + outputEncoder: Option[ExpressionEncoder[_]] = None, + udfName: Option[String] = None, + nullable: Boolean = true, + udfDeterministic: Boolean = true): Expression def getGpuBroadcastNestedLoopJoinShim( left: SparkPlan, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index 418c2ce2f13..97c4886fa3d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -226,7 +226,7 @@ class GpuDynamicPartitionDataWriter( */ private lazy val partitionPathExpression: Expression = Concat( description.partitionColumns.zipWithIndex.flatMap { case (c, i) => - val partitionName = ScalaUDF( + val partitionName = ShimLoader.getSparkShims.getScalaUDFAsExpression( ExternalCatalogUtils.getPartitionPathString _, StringType, Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) From 46322288cdbbe23bacd8118fe493450b97b71a61 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 09:47:16 -0500 Subject: [PATCH 31/47] Remove unneeded use of GPUBuildSide --- .../spark30/GpuBroadcastHashJoinExec.scala | 34 ++++++++---------- .../spark31/GpuBroadcastHashJoinExec.scala | 36 +++++++++---------- .../spark31/GpuShuffledHashJoinExec.scala | 18 +++++----- .../GpuBroadcastNestedLoopJoinExec.scala | 4 +-- 4 files changed, 42 insertions(+), 50 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index c0f7494884b..8490bc9fbf6 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -49,16 +49,12 @@ class GpuBroadcastHashJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - ShimLoader.getSparkShims.getBuildSide(join) - } - override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => childPlans(0) - case GpuBuildRight => childPlans(1) + val buildSide = join.buildSide match { + case BuildLeft => childPlans(0) + case BuildRight => childPlans(1) } if (!buildSide.canThisBeReplaced) { @@ -74,9 +70,9 @@ class GpuBroadcastHashJoinMeta( val left = childPlans(0).convertIfNeeded() val right = childPlans(1).convertIfNeeded() // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => left - case GpuBuildRight => right + val buildSide = join.buildSide match { + case BuildLeft => left + case BuildRight => right } if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") @@ -104,9 +100,9 @@ case class GpuBroadcastHashJoinExec( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) + buildSide match { + case BuildLeft => (lkeys, rkeys) + case BuildRight => (rkeys, lkeys) } } @@ -117,10 +113,10 @@ case class GpuBroadcastHashJoinExec( override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => + buildSide match { + case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil - case GpuBuildRight => + case BuildRight => UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } @@ -184,9 +180,9 @@ case class GpuBroadcastHashJoinExec( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + buildSide match { + case BuildLeft => doJoinLeftRight(builtTable, streamedTable) + case BuildRight => doJoinLeftRight(streamedTable, builtTable) } } finally { streamedTable.close() diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 9652bd14587..524128219d4 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_RO import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.optimizer.BuildSide +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} @@ -51,9 +51,9 @@ case class GpuBroadcastHashJoinExec( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) + buildSide match { + case BuildLeft => (lkeys, rkeys) + case BuildRight => (rkeys, lkeys) } } @@ -64,10 +64,10 @@ case class GpuBroadcastHashJoinExec( override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => + buildSide match { + case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil - case GpuBuildRight => + case BuildRight => UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } @@ -131,9 +131,9 @@ case class GpuBroadcastHashJoinExec( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + buildSide match { + case BuildLeft => doJoinLeftRight(builtTable, streamedTable) + case BuildRight => doJoinLeftRight(streamedTable, builtTable) } } finally { streamedTable.close() @@ -174,16 +174,12 @@ class GpuBroadcastHashJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map( GpuOverrides.wrapExpr(_, conf, Some(this))) - private def getBuildSide(join: BroadcastHashJoinExec): GpuBuildSide = { - ShimLoader.getSparkShims.getBuildSide(join) - } - override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => childPlans(0) - case GpuBuildRight => childPlans(1) + val buildSide = join.buildSide match { + case BuildLeft => childPlans(0) + case BuildRight => childPlans(1) } if (!buildSide.canThisBeReplaced) { @@ -199,9 +195,9 @@ class GpuBroadcastHashJoinMeta( val left = childPlans(0).convertIfNeeded() val right = childPlans(1).convertIfNeeded() // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = getBuildSide(join) match { - case GpuBuildLeft => left - case GpuBuildRight => right + val buildSide = join.buildSide match { + case BuildLeft => left + case BuildRight => right } if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { throw new IllegalStateException("the broadcast must be on the GPU too") diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index f4f27d80a02..d2bb0d09430 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -60,9 +60,9 @@ case class GpuShuffledHashJoinExec ( "Join keys from two sides should have same types") val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) + buildSide match { + case BuildLeft => (lkeys, rkeys) + case BuildRight => (rkeys, lkeys) } } @@ -82,9 +82,9 @@ case class GpuShuffledHashJoinExec ( } override def childrenCoalesceGoal: Seq[CoalesceGoal] = { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => Seq(RequireSingleBatch, null) - case GpuBuildRight => Seq(null, RequireSingleBatch) + buildSide match { + case BuildLeft => Seq(RequireSingleBatch, null) + case BuildRight => Seq(null, RequireSingleBatch) } } @@ -156,9 +156,9 @@ case class GpuShuffledHashJoinExec ( val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) val joined = try { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) + buildSide match { + case BuildLeft => doJoinLeftRight(builtTable, streamedTable) + case BuildRight => doJoinLeftRight(streamedTable, builtTable) } } finally { streamedTable.close() diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index b56c52ac483..a981dab2231 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -246,8 +246,8 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( case _: InnerLike => GpuBroadcastNestedLoopJoinExecBase.innerLikeJoin(streamedIter, builtTable, getBuildSide, boundCondition, joinTime, joinOutputRows, numOutputRows, numOutputBatches, filterTime, totalTime) - case _ => throw new IllegalArgumentException(s"$joinType + $getBuildSide is not supported" + - s" and should be run on the CPU") + case _ => throw new IllegalArgumentException(s"$joinType + $getBuildSide is not" + + " supported and should be run on the CPU") } } } From b798a05c1c6058b54c488ef92058da6f0595b4d2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 09:58:32 -0500 Subject: [PATCH 32/47] Revert some changes to joins --- .../spark30/GpuBroadcastHashJoinExec.scala | 62 ------------- .../rapids/shims/spark30/GpuHashJoin.scala | 88 +++++++++++++++---- .../spark30/GpuShuffledHashJoinExec.scala | 62 ------------- .../spark31/GpuBroadcastHashJoinExec.scala | 63 ------------- .../rapids/shims/spark31/GpuHashJoin.scala | 87 ++++++++++++++---- .../spark31/GpuShuffledHashJoinExec.scala | 61 ------------- 6 files changed, 138 insertions(+), 285 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 8490bc9fbf6..b5fc2bde13f 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -95,17 +95,6 @@ case class GpuBroadcastHashJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) - } - } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), @@ -156,56 +145,5 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - buildSide match { - case BuildLeft => doJoinLeftRight(builtTable, streamedTable) - case BuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 68bde63aac6..5752252e55d 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -15,15 +15,15 @@ */ package com.nvidia.spark.rapids.shims.spark30 -import ai.rapids.cudf.Table -import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} +import ai.rapids.cudf.{NvtxColor, Table} +import com.nvidia.spark.rapids._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} -import org.apache.spark.sql.execution.joins.HashJoin +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, HashJoin} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} object GpuHashJoin { def tagJoin( @@ -62,17 +62,16 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] - - protected val gpuBuildKeys: Seq[GpuExpression] - protected val gpuStreamedKeys: Seq[GpuExpression] + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + buildSide match { + case BuildLeft => (lkeys, rkeys) + case BuildRight => (rkeys, lkeys) + } + } /** * Place the columns in left and the columns in right into a single ColumnarBatch @@ -137,14 +136,14 @@ trait GpuHashJoin extends GpuExec with HashJoin { if (stream.hasNext) { val cb = stream.next() val startTime = System.nanoTime() - nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } else if (first) { // We have to at least try one in some cases val startTime = System.nanoTime() val cb = GpuColumnVector.emptyBatch(streamedPlan.output.asJava) - nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } @@ -164,7 +163,58 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - protected def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { + def doJoin(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + buildSide match { + case BuildLeft => doJoinLeftRight(builtTable, streamedTable) + case BuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + + private[this] def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { val joinedTable = joinType match { case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) .leftJoin(rightTable.onColumns(joinKeyIndices: _*), false) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index cf733dd388e..1b36930f784 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -53,17 +53,6 @@ case class GpuShuffledHashJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => (lkeys, rkeys) - case GpuBuildRight => (rkeys, lkeys) - } - } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), @@ -130,57 +119,6 @@ case class GpuShuffledHashJoinExec( } } } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => doJoinLeftRight(builtTable, streamedTable) - case GpuBuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } } class GpuShuffledHashJoinMeta( diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 524128219d4..73602b9b7c7 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -45,18 +45,6 @@ case class GpuBroadcastHashJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) - } - } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), @@ -107,57 +95,6 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - buildSide match { - case BuildLeft => doJoinLeftRight(builtTable, streamedTable) - case BuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } } class GpuBroadcastHashJoinMeta( diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index c5d93bb8f39..833e1f64e47 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -15,15 +15,15 @@ */ package com.nvidia.spark.rapids.shims.spark31 -import ai.rapids.cudf.Table -import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec, GpuExpression, RapidsMeta} - +import ai.rapids.cudf.{NvtxColor, Table} +import com.nvidia.spark.rapids._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} object GpuHashJoin { def tagJoin( @@ -62,17 +62,16 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] - - protected val gpuBuildKeys: Seq[GpuExpression] - protected val gpuStreamedKeys: Seq[GpuExpression] + protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { + require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), + "Join keys from two sides should have same types") + val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) + val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + buildSide match { + case BuildLeft => (lkeys, rkeys) + case BuildRight => (rkeys, lkeys) + } + } /** * Place the columns in left and the columns in right into a single ColumnarBatch @@ -137,14 +136,14 @@ trait GpuHashJoin extends GpuExec with HashJoin { if (stream.hasNext) { val cb = stream.next() val startTime = System.nanoTime() - nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } else if (first) { // We have to at least try one in some cases val startTime = System.nanoTime() val cb = GpuColumnVector.emptyBatch(streamedPlan.output.asJava) - nextCb = doJoinInternal(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, + nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) totalTime += (System.nanoTime() - startTime) } @@ -164,7 +163,59 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { + def doJoin(builtTable: Table, + streamedBatch: ColumnarBatch, + boundCondition: Option[Expression], + numOutputRows: SQLMetric, + numJoinOutputRows: SQLMetric, + numOutputBatches: SQLMetric, + joinTime: SQLMetric, + filterTime: SQLMetric): Option[ColumnarBatch] = { + + val streamedTable = try { + val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) + try { + val combined = combine(streamedKeysBatch, streamedBatch) + GpuColumnVector.from(combined) + } finally { + streamedKeysBatch.close() + } + } finally { + streamedBatch.close() + } + + val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) + val joined = try { + buildSide match { + case BuildLeft => doJoinLeftRight(builtTable, streamedTable) + case BuildRight => doJoinLeftRight(streamedTable, builtTable) + } + } finally { + streamedTable.close() + nvtxRange.close() + } + + numJoinOutputRows += joined.numRows() + + val tmp = if (boundCondition.isDefined) { + GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) + } else { + numOutputRows += joined.numRows() + numOutputBatches += 1 + joined + } + if (tmp.numRows() == 0) { + // Not sure if there is a better way to work around this + numOutputBatches.set(numOutputBatches.value - 1) + tmp.close() + None + } else { + Some(tmp) + } + } + + + private[this] def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { val joinedTable = joinType match { case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) .leftJoin(rightTable.onColumns(joinKeyIndices: _*)) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index d2bb0d09430..708fae0a3c9 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -55,16 +55,6 @@ case class GpuShuffledHashJoinExec ( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - protected lazy val (gpuBuildKeys, gpuStreamedKeys) = { - require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), - "Join keys from two sides should have same types") - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) - buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) - } - } override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), @@ -132,57 +122,6 @@ case class GpuShuffledHashJoinExec ( } } } - - def doJoinInternal(builtTable: Table, - streamedBatch: ColumnarBatch, - boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { - - val streamedTable = try { - val streamedKeysBatch = GpuProjectExec.project(streamedBatch, gpuStreamedKeys) - try { - val combined = combine(streamedKeysBatch, streamedBatch) - GpuColumnVector.from(combined) - } finally { - streamedKeysBatch.close() - } - } finally { - streamedBatch.close() - } - - val nvtxRange = new NvtxWithMetrics("hash join", NvtxColor.ORANGE, joinTime) - val joined = try { - buildSide match { - case BuildLeft => doJoinLeftRight(builtTable, streamedTable) - case BuildRight => doJoinLeftRight(streamedTable, builtTable) - } - } finally { - streamedTable.close() - nvtxRange.close() - } - - numJoinOutputRows += joined.numRows() - - val tmp = if (boundCondition.isDefined) { - GpuFilter(joined, boundCondition.get, numOutputRows, numOutputBatches, filterTime) - } else { - numOutputRows += joined.numRows() - numOutputBatches += 1 - joined - } - if (tmp.numRows() == 0) { - // Not sure if there is a better way to work around this - numOutputBatches.set(numOutputBatches.value - 1) - tmp.close() - None - } else { - Some(tmp) - } - } } class GpuShuffledHashJoinMeta( From 456b7842153844642877dfc8f2e0a6b2747ff89a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:12:48 -0500 Subject: [PATCH 33/47] Fix spacing in pom Signed-off-by: Thomas Graves --- pom.xml | 6 +- .../spark30/GpuBroadcastHashJoinExec.scala | 2 +- .../GpuBroadcastNestedLoopJoinExec.scala | 2 +- .../rapids/shims/spark30/GpuHashJoin.scala | 6 +- .../spark30/GpuShuffledHashJoinExec.scala | 60 +++++++++---------- .../shims/spark30/GpuSortMergeJoinExec.scala | 4 +- .../spark31/GpuBroadcastHashJoinExec.scala | 2 +- .../GpuBroadcastNestedLoopJoinExec.scala | 2 +- .../rapids/shims/spark31/GpuHashJoin.scala | 7 ++- .../spark31/GpuShuffledHashJoinExec.scala | 60 +++++++++---------- .../shims/spark31/GpuSortMergeJoinExec.scala | 4 +- .../rapids/shims/spark31/Spark31Shims.scala | 2 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 12 ---- .../GpuBroadcastNestedLoopJoinExec.scala | 12 ++-- 14 files changed, 88 insertions(+), 93 deletions(-) diff --git a/pom.xml b/pom.xml index 39f7d3931c2..41f0c8f10be 100644 --- a/pom.xml +++ b/pom.xml @@ -485,11 +485,11 @@ **/.m2/** .gnupg/** pom.xml.asc - jenkins/databricks/*.patch - *.jar + jenkins/databricks/*.patch + *.jar docs/demo/**/*.ipynb docs/demo/**/*.zpln - **/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider + **/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index b5fc2bde13f..4b6b6f27871 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -40,7 +40,7 @@ class GpuBroadcastHashJoinMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala index 45696c4fa49..dd81a98f8ce 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastNestedLoopJoinExec.scala @@ -37,7 +37,7 @@ case class GpuBroadcastNestedLoopJoinExec( extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition, targetSizeBytes) { - def getBuildSide: GpuBuildSide = { + def getGpuBuildSide: GpuBuildSide = { GpuJoinUtils.getGpuBuildSide(join.buildSide) } } diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala index 5752252e55d..b6325be8b52 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuHashJoin.scala @@ -20,10 +20,10 @@ import com.nvidia.spark.rapids._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, HashJoin} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { def tagJoin( @@ -163,7 +163,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoin(builtTable: Table, + private[this] def doJoin(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], numOutputRows: SQLMetric, diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index 1b36930f784..377c8e54566 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -44,6 +44,36 @@ object GpuJoinUtils { /** * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide */ +class GpuShuffledHashJoinMeta( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + } + + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } +} + case class GpuShuffledHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -121,33 +151,3 @@ case class GpuShuffledHashJoinExec( } } -class GpuShuffledHashJoinMeta( - join: ShuffledHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - } - - override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, - join.buildSide, - condition.map(_.convertToGpu()), - childPlans(0).convertIfNeeded(), - childPlans(1).convertIfNeeded()) - } -} - diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index ed285ecc4d4..515d7249257 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -30,7 +30,7 @@ class GpuSortMergeJoinMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) { + extends SparkPlanMeta[SortMergeJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -39,6 +39,8 @@ class GpuSortMergeJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map( GpuOverrides.wrapExpr(_, conf, Some(this))) + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override def tagPlanForGpu(): Unit = { // Use conditions from Hash Join GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 73602b9b7c7..87f3d176260 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -102,7 +102,7 @@ class GpuBroadcastHashJoinMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala index a485aabf119..7a46926d8f1 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastNestedLoopJoinExec.scala @@ -38,7 +38,7 @@ case class GpuBroadcastNestedLoopJoinExec( extends GpuBroadcastNestedLoopJoinExecBase(left, right, join, joinType, condition, targetSizeBytes) { - def getBuildSide: GpuBuildSide = { + def getGpuBuildSide: GpuBuildSide = { GpuJoinUtils.getGpuBuildSide(join.buildSide) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index 833e1f64e47..532f4c616af 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -17,13 +17,14 @@ package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ + import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} -import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { def tagJoin( @@ -163,7 +164,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - def doJoin(builtTable: Table, + private[this] def doJoin(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], numOutputRows: SQLMetric, diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index 708fae0a3c9..af66abb798f 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -46,6 +46,36 @@ object GpuJoinUtils { /** * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide */ +class GpuShuffledHashJoinMeta( + join: ShuffledHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + } + + override def convertToGpu(): GpuExec = { + GpuShuffledHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, + join.buildSide, + condition.map(_.convertToGpu()), + childPlans(0).convertIfNeeded(), + childPlans(1).convertIfNeeded()) + } +} + case class GpuShuffledHashJoinExec ( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -123,33 +153,3 @@ case class GpuShuffledHashJoinExec ( } } } - -class GpuShuffledHashJoinMeta( - join: ShuffledHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - } - - override def convertToGpu(): GpuExec = { - GpuShuffledHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, - join.buildSide, - condition.map(_.convertToGpu()), - childPlans(0).convertIfNeeded(), - childPlans(1).convertIfNeeded()) - } -} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala index 88504ad1c9c..f19c36faa13 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala @@ -31,7 +31,7 @@ class GpuSortMergeJoinMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends GpuHashJoinBaseMeta[SortMergeJoinExec](join, conf, parent, rule) { + extends SparkPlanMeta[SortMergeJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) @@ -40,6 +40,8 @@ class GpuSortMergeJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map( GpuOverrides.wrapExpr(_, conf, Some(this))) + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override def tagPlanForGpu(): Unit = { // Use conditions from Hash Join GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 34d0f96dd8c..3ae29714a78 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -38,7 +38,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} class Spark31Shims extends SparkShims { - override def getScalaUDFAsExpression( + override def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, children: Seq[Expression], 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 8ef94bc851f..393307a05e3 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 @@ -47,18 +47,6 @@ final class NoRuleConfKeysAndIncompat extends ConfKeysAndIncompat { override def confKey = "NOT_FOUND" } -abstract class GpuHashJoinBaseMeta[INPUT <: SparkPlan]( - plan: INPUT, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends SparkPlanMeta[INPUT](plan, conf, parent, rule) { - - val leftKeys: Seq[BaseExprMeta[_]] - val rightKeys: Seq[BaseExprMeta[_]] - val condition: Option[BaseExprMeta[_]] -} - /** * Holds metadata about a stage in the physical plan that is separate from the plan itself. * This is helpful in deciding when to replace part of the plan with a GPU enabled version. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index a981dab2231..bd10cd62b73 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -139,7 +139,9 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( condition: Option[Expression], targetSizeBytes: Long) extends BinaryExecNode with GpuExec { - def getBuildSide: GpuBuildSide + // Spark BuildSide, BuildRight, BuildLeft changed packages between Spark versions + // so return a GPU version that is agnostic to the Spark version. + def getGpuBuildSide: GpuBuildSide override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException("This should only be called from columnar") @@ -152,7 +154,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) /** BuildRight means the right relation <=> the broadcast relation. */ - private val (streamed, broadcast) = getBuildSide match { + private val (streamed, broadcast) = getGpuBuildSide match { case GpuBuildRight => (left, right) case GpuBuildLeft => (right, left) } @@ -162,7 +164,7 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] } - override def requiredChildDistribution: Seq[Distribution] = getBuildSide match { + override def requiredChildDistribution: Seq[Distribution] = getGpuBuildSide match { case GpuBuildLeft => BroadcastDistribution(IdentityBroadcastMode) :: UnspecifiedDistribution :: Nil case GpuBuildRight => @@ -244,9 +246,9 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( streamed.executeColumnar().mapPartitions { streamedIter => joinType match { case _: InnerLike => GpuBroadcastNestedLoopJoinExecBase.innerLikeJoin(streamedIter, - builtTable, getBuildSide, boundCondition, + builtTable, getGpuBuildSide, boundCondition, joinTime, joinOutputRows, numOutputRows, numOutputBatches, filterTime, totalTime) - case _ => throw new IllegalArgumentException(s"$joinType + $getBuildSide is not" + + case _ => throw new IllegalArgumentException(s"$joinType + $getGpuBuildSide is not" + " supported and should be run on the CPU") } } From fa1b463018744ca290918932398bb73c1cde0e60 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:21:28 -0500 Subject: [PATCH 34/47] More join changes --- .../spark30/GpuBroadcastHashJoinExec.scala | 5 +- .../spark30/GpuShuffledHashJoinExec.scala | 23 ++-- .../shims/spark30/GpuSortMergeJoinExec.scala | 2 +- .../spark31/GpuBroadcastHashJoinExec.scala | 106 +++++++++--------- .../spark31/GpuShuffledHashJoinExec.scala | 21 ++-- .../shims/spark31/GpuSortMergeJoinExec.scala | 4 +- 6 files changed, 79 insertions(+), 82 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 4b6b6f27871..9a1532daf9c 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -49,6 +49,8 @@ class GpuBroadcastHashJoinMeta( val condition: Option[BaseExprMeta[_]] = join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition + override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) @@ -145,5 +147,4 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } -} - +} \ No newline at end of file diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index 377c8e54566..71e1b70e285 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -50,19 +50,20 @@ class GpuShuffledHashJoinMeta( parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) { - val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) val rightKeys: Seq[BaseExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = + join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } - override def convertToGpu(): GpuExec = { + override def convertToGpu(): GpuExec = GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), @@ -71,7 +72,6 @@ class GpuShuffledHashJoinMeta( condition.map(_.convertToGpu()), childPlans(0).convertIfNeeded(), childPlans(1).convertIfNeeded()) - } } case class GpuShuffledHashJoinExec( @@ -81,7 +81,7 @@ case class GpuShuffledHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + right: SparkPlan) extends BinaryExecNode with GpuHashJoin { override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), @@ -98,11 +98,9 @@ case class GpuShuffledHashJoinExec( "GpuShuffledHashJoin does not support the execute() code path.") } - override def childrenCoalesceGoal: Seq[CoalesceGoal] = { - GpuJoinUtils.getGpuBuildSide(buildSide) match { - case GpuBuildLeft => Seq(RequireSingleBatch, null) - case GpuBuildRight => Seq(null, RequireSingleBatch) - } + override def childrenCoalesceGoal: Seq[CoalesceGoal] = buildSide match { + case BuildLeft => Seq(RequireSingleBatch, null) + case BuildRight => Seq(null, RequireSingleBatch) } override def doExecuteColumnar() : RDD[ColumnarBatch] = { @@ -149,5 +147,4 @@ case class GpuShuffledHashJoinExec( } } } -} - +} \ No newline at end of file diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala index 515d7249257..4b47007ee0e 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuSortMergeJoinExec.scala @@ -20,7 +20,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.SortExec -import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, SortMergeJoinExec} /** * HashJoin changed in Spark 3.1 requiring Shim diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 87f3d176260..7e00223cf28 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -36,6 +36,59 @@ import org.apache.spark.sql.vectorized.ColumnarBatch /** * Spark 3.1 changed packages of BuildLeft, BuildRight, BuildSide */ +class GpuBroadcastHashJoinMeta( + join: BroadcastHashJoinExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: ConfKeysAndIncompat) + extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { + + val leftKeys: Seq[BaseExprMeta[_]] = + join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val rightKeys: Seq[BaseExprMeta[_]] = + join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = join.condition.map( + GpuOverrides.wrapExpr(_, conf, Some(this))) + + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition + + override def tagPlanForGpu(): Unit = { + GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) + + val buildSide = join.buildSide match { + case BuildLeft => childPlans(0) + case BuildRight => childPlans(1) + } + + if (!buildSide.canThisBeReplaced) { + willNotWorkOnGpu("the broadcast for this join must be on the GPU too") + } + + if (!canThisBeReplaced) { + buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") + } + } + + override def convertToGpu(): GpuExec = { + val left = childPlans(0).convertIfNeeded() + val right = childPlans(1).convertIfNeeded() + // The broadcast part of this must be a BroadcastExchangeExec + val buildSide = join.buildSide match { + case BuildLeft => left + case BuildRight => right + } + if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { + throw new IllegalStateException("the broadcast must be on the GPU too") + } + GpuBroadcastHashJoinExec( + leftKeys.map(_.convertToGpu()), + rightKeys.map(_.convertToGpu()), + join.joinType, join.buildSide, + condition.map(_.convertToGpu()), + left, right) + } +} + case class GpuBroadcastHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -95,55 +148,4 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } -} - -class GpuBroadcastHashJoinMeta( - join: BroadcastHashJoinExec, - conf: RapidsConf, - parent: Option[RapidsMeta[_, _, _]], - rule: ConfKeysAndIncompat) - extends SparkPlanMeta[BroadcastHashJoinExec](join, conf, parent, rule) { - - val leftKeys: Seq[BaseExprMeta[_]] = - join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val rightKeys: Seq[BaseExprMeta[_]] = - join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) - - val buildSide = join.buildSide match { - case BuildLeft => childPlans(0) - case BuildRight => childPlans(1) - } - - if (!buildSide.canThisBeReplaced) { - willNotWorkOnGpu("the broadcast for this join must be on the GPU too") - } - - if (!canThisBeReplaced) { - buildSide.willNotWorkOnGpu("the BroadcastHashJoin this feeds is not on the GPU") - } - } - - override def convertToGpu(): GpuExec = { - val left = childPlans(0).convertIfNeeded() - val right = childPlans(1).convertIfNeeded() - // The broadcast part of this must be a BroadcastExchangeExec - val buildSide = join.buildSide match { - case BuildLeft => left - case BuildRight => right - } - if (!buildSide.isInstanceOf[GpuBroadcastExchangeExec]) { - throw new IllegalStateException("the broadcast must be on the GPU too") - } - GpuBroadcastHashJoinExec( - leftKeys.map(_.convertToGpu()), - rightKeys.map(_.convertToGpu()), - join.joinType, join.buildSide, - condition.map(_.convertToGpu()), - left, right) - } -} +} \ No newline at end of file diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index af66abb798f..f62a62962c4 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -52,19 +52,20 @@ class GpuShuffledHashJoinMeta( parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { - val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) val rightKeys: Seq[BaseExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = + join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + + override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, join.leftKeys, join.rightKeys, join.condition) } - override def convertToGpu(): GpuExec = { + override def convertToGpu(): GpuExec = GpuShuffledHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), @@ -73,7 +74,6 @@ class GpuShuffledHashJoinMeta( condition.map(_.convertToGpu()), childPlans(0).convertIfNeeded(), childPlans(1).convertIfNeeded()) - } } case class GpuShuffledHashJoinExec ( @@ -85,7 +85,6 @@ case class GpuShuffledHashJoinExec ( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), @@ -101,11 +100,9 @@ case class GpuShuffledHashJoinExec ( "GpuShuffledHashJoin does not support the execute() code path.") } - override def childrenCoalesceGoal: Seq[CoalesceGoal] = { - buildSide match { - case BuildLeft => Seq(RequireSingleBatch, null) - case BuildRight => Seq(null, RequireSingleBatch) - } + override def childrenCoalesceGoal: Seq[CoalesceGoal] = buildSide match { + case BuildLeft => Seq(RequireSingleBatch, null) + case BuildRight => Seq(null, RequireSingleBatch) } override def doExecuteColumnar() : RDD[ColumnarBatch] = { @@ -152,4 +149,4 @@ case class GpuShuffledHashJoinExec ( } } } -} +} \ No newline at end of file diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala index f19c36faa13..237b08e36d2 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuSortMergeJoinExec.scala @@ -18,10 +18,10 @@ package com.nvidia.spark.rapids.shims.spark31 import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.SortExec -import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.joins.SortMergeJoinExec /** * HashJoin changed in Spark 3.1 requiring Shim From 5bb4f999758dda6be29cedcc363a33126abadd32 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:28:11 -0500 Subject: [PATCH 35/47] more cleanup --- .../spark31/GpuBroadcastHashJoinExec.scala | 4 +-- .../rapids/shims/spark31/GpuHashJoin.scala | 28 ++++++++----------- .../spark31/GpuShuffledHashJoinExec.scala | 5 ++-- 3 files changed, 16 insertions(+), 21 deletions(-) diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 7e00223cf28..2a26a5d14f9 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -47,8 +47,8 @@ class GpuBroadcastHashJoinMeta( join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) val rightKeys: Seq[BaseExprMeta[_]] = join.rightKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - val condition: Option[BaseExprMeta[_]] = join.condition.map( - GpuOverrides.wrapExpr(_, conf, Some(this))) + val condition: Option[BaseExprMeta[_]] = + join.condition.map(GpuOverrides.wrapExpr(_, conf, Some(this))) override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ condition diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala index 532f4c616af..59db665f61c 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuHashJoin.scala @@ -128,7 +128,7 @@ trait GpuHashJoin extends GpuExec with HashJoin { TaskContext.get().addTaskCompletionListener[Unit](_ => closeCb()) def closeCb(): Unit = { - nextCb.map(_.close()) + nextCb.foreach(_.close()) nextCb = None } @@ -215,26 +215,22 @@ trait GpuHashJoin extends GpuExec with HashJoin { } } - private[this] def doJoinLeftRight(leftTable: Table, rightTable: Table): ColumnarBatch = { val joinedTable = joinType match { case LeftOuter => leftTable.onColumns(joinKeyIndices: _*) - .leftJoin(rightTable.onColumns(joinKeyIndices: _*)) + .leftJoin(rightTable.onColumns(joinKeyIndices: _*), false) case RightOuter => rightTable.onColumns(joinKeyIndices: _*) - .leftJoin(leftTable.onColumns(joinKeyIndices: _*)) - case _: InnerLike => - leftTable.onColumns(joinKeyIndices: _*).innerJoin(rightTable.onColumns(joinKeyIndices: _*)) - case LeftSemi => - leftTable.onColumns(joinKeyIndices: _*) - .leftSemiJoin(rightTable.onColumns(joinKeyIndices: _*)) - case LeftAnti => - leftTable.onColumns(joinKeyIndices: _*) - .leftAntiJoin(rightTable.onColumns(joinKeyIndices: _*)) - case FullOuter => - leftTable.onColumns(joinKeyIndices: _*) - .fullJoin(rightTable.onColumns(joinKeyIndices: _*)) + .leftJoin(leftTable.onColumns(joinKeyIndices: _*), false) + case _: InnerLike => leftTable.onColumns(joinKeyIndices: _*) + .innerJoin(rightTable.onColumns(joinKeyIndices: _*), false) + case LeftSemi => leftTable.onColumns(joinKeyIndices: _*) + .leftSemiJoin(rightTable.onColumns(joinKeyIndices: _*), false) + case LeftAnti => leftTable.onColumns(joinKeyIndices: _*) + .leftAntiJoin(rightTable.onColumns(joinKeyIndices: _*), false) + case FullOuter => leftTable.onColumns(joinKeyIndices: _*) + .fullJoin(rightTable.onColumns(joinKeyIndices: _*), false) case _ => throw new NotImplementedError(s"Joint Type ${joinType.getClass} is not currently" + - s" supported") + s" supported") } try { val result = joinIndices.map(joinIndex => diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index f62a62962c4..0c0a7665aa8 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -21,7 +21,6 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression @@ -51,7 +50,7 @@ class GpuShuffledHashJoinMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: ConfKeysAndIncompat) - extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) with Logging { + extends SparkPlanMeta[ShuffledHashJoinExec](join, conf, parent, rule) { val leftKeys: Seq[BaseExprMeta[_]] = join.leftKeys.map(GpuOverrides.wrapExpr(_, conf, Some(this))) val rightKeys: Seq[BaseExprMeta[_]] = @@ -76,7 +75,7 @@ class GpuShuffledHashJoinMeta( childPlans(1).convertIfNeeded()) } -case class GpuShuffledHashJoinExec ( +case class GpuShuffledHashJoinExec( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, From f9efe33562b4f97b48d3ad4d4b6046da21b27f22 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:38:00 -0500 Subject: [PATCH 36/47] more cleanup --- .../rapids/shims/spark30/GpuBroadcastHashJoinExec.scala | 6 +++--- .../rapids/shims/spark30/GpuShuffledHashJoinExec.scala | 5 ++--- .../rapids/shims/spark30/Spark30ShimServiceProvider.scala | 2 +- .../nvidia/spark/rapids/shims/spark30/Spark30Shims.scala | 1 - .../sql/rapids/shims/spark30/GpuFileSourceScanExec.scala | 6 +++--- .../rapids/shims/spark31/GpuBroadcastHashJoinExec.scala | 6 +++--- .../rapids/shims/spark31/GpuShuffledHashJoinExec.scala | 4 ++-- .../rapids/shims/spark31/Spark31ShimServiceProvider.scala | 2 +- .../nvidia/spark/rapids/shims/spark31/Spark31Shims.scala | 1 - .../sql/rapids/shims/spark31/GpuFileSourceScanExec.scala | 6 +++--- .../src/main/scala/com/nvidia/spark/rapids/SparkShims.scala | 6 ++++-- 11 files changed, 22 insertions(+), 23 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala index 9a1532daf9c..19e64cd0176 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuBroadcastHashJoinExec.scala @@ -16,9 +16,8 @@ package com.nvidia.spark.rapids.shims.spark30 -import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -111,6 +110,7 @@ case class GpuBroadcastHashJoinExec( UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } + def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { case gpu: GpuBroadcastExchangeExec => gpu case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] @@ -147,4 +147,4 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } -} \ No newline at end of file +} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala index 71e1b70e285..16aadf62933 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuShuffledHashJoinExec.scala @@ -16,9 +16,8 @@ package com.nvidia.spark.rapids.shims.spark30 -import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -147,4 +146,4 @@ case class GpuShuffledHashJoinExec( } } } -} \ No newline at end of file +} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala index 5071d0de123..3d43a547aca 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30ShimServiceProvider.scala @@ -29,4 +29,4 @@ class Spark30ShimServiceProvider extends SparkShimServiceProvider { def buildShim: SparkShims = { new Spark30Shims() } -} \ No newline at end of file +} diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 89fcefb3c0a..4e0f40545ed 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -195,4 +195,3 @@ class Spark30Shims extends SparkShims { classOf[RapidsShuffleManager].getCanonicalName } } - diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala index 47206481e07..90d135226e1 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/GpuFileSourceScanExec.scala @@ -62,9 +62,9 @@ case class GpuFileSourceScanExec( // run into any issues in testing using the one we create here. @transient val logicalRelation = LogicalRelation(relation) try { - constructor.newInstance(relation, output, requiredSchema, partitionFilters, - optionalBucketSet, dataFilters, tableIdentifier, - logicalRelation).asInstanceOf[FileSourceScanExec] + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, dataFilters, tableIdentifier, + logicalRelation).asInstanceOf[FileSourceScanExec] } catch { case il: IllegalArgumentException => // TODO - workaround until https://github.com/NVIDIA/spark-rapids/issues/354 diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala index 2a26a5d14f9..bed75498f1a 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuBroadcastHashJoinExec.scala @@ -16,9 +16,8 @@ package com.nvidia.spark.rapids.shims.spark31 -import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -112,6 +111,7 @@ case class GpuBroadcastHashJoinExec( UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } + def broadcastExchange: GpuBroadcastExchangeExec = buildPlan match { case gpu: GpuBroadcastExchangeExec => gpu case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuBroadcastExchangeExec] @@ -148,4 +148,4 @@ case class GpuBroadcastHashJoinExec( doJoin(builtTable, it, boundCondition, numOutputRows, joinOutputRows, numOutputBatches, joinTime, filterTime, totalTime)) } -} \ No newline at end of file +} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala index 0c0a7665aa8..bff955ebfdf 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuShuffledHashJoinExec.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims.spark31 import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -148,4 +148,4 @@ case class GpuShuffledHashJoinExec( } } } -} \ No newline at end of file +} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala index b798205ac38..ac3bbd223f8 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala @@ -29,4 +29,4 @@ class Spark31ShimServiceProvider extends SparkShimServiceProvider { def buildShim: SparkShims = { new Spark31Shims() } -} \ No newline at end of file +} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 3ae29714a78..009426b68d3 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -183,5 +183,4 @@ class Spark31Shims extends SparkShims { override def getRapidsShuffleManagerClass: String = { classOf[RapidsShuffleManager].getCanonicalName } - } diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala index 35cef4d7724..010dfd7f551 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/GpuFileSourceScanExec.scala @@ -62,9 +62,9 @@ case class GpuFileSourceScanExec( // run into any issues in testing using the one we create here. @transient val logicalRelation = LogicalRelation(relation) try { - constructor.newInstance(relation, output, requiredSchema, partitionFilters, - optionalBucketSet, dataFilters, tableIdentifier, - logicalRelation).asInstanceOf[FileSourceScanExec] + constructor.newInstance(relation, output, requiredSchema, partitionFilters, + optionalBucketSet, dataFilters, tableIdentifier, + logicalRelation).asInstanceOf[FileSourceScanExec] } catch { case il: IllegalArgumentException => // TODO - workaround until https://github.com/NVIDIA/spark-rapids/issues/354 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 a0f7b08b1ac..e6a8aeaeb0b 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 @@ -25,6 +25,10 @@ import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} +/** + * Spark BuildSide, BuildRight, BuildLeft moved packages in Spark 3.1 + * so create GPU versions of these that can be agnostic to Spark version. + */ sealed abstract class GpuBuildSide case object GpuBuildRight extends GpuBuildSide @@ -65,5 +69,3 @@ trait SparkShims { startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] } - - From 6661e4acc9cdf689762cf074535a314ec2270270 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:48:14 -0500 Subject: [PATCH 37/47] more cleanup --- docs/get-started/getting-started.md | 7 +- .../rapids/shims/spark30/Spark30Shims.scala | 155 +++++++++--------- .../RapidsShuffleInternalManager.scala | 1 - .../rapids/shims/spark31/Spark31Shims.scala | 135 ++++++++------- .../RapidsShuffleInternalManager.scala | 1 - 5 files changed, 148 insertions(+), 151 deletions(-) diff --git a/docs/get-started/getting-started.md b/docs/get-started/getting-started.md index 29c533f2416..5a86ce710d3 100644 --- a/docs/get-started/getting-started.md +++ b/docs/get-started/getting-started.md @@ -417,11 +417,14 @@ With `nv_peer_mem`, IB/RoCE-based transfers can perform zero-copy transfers dire 2) Install [UCX 1.8.1](https://github.com/openucx/ucx/releases/tag/v1.8.1). 3) You will need to configure your spark job with extra settings for UCX (we are looking to -simplify these settings in the near future): +simplify these settings in the near future). Choose the version of the shuffle manager +that matches your Spark version. Currently we support +Spark 3.0 (com.nvidia.spark.rapids.spark30.RapidsShuffleManager) and +Spark 3.1 (com.nvidia.spark.rapids.spark31.RapidsShuffleManager): ```shell ... ---conf spark.shuffle.manager=com.nvidia.spark.RapidsShuffleManager \ +--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark30.RapidsShuffleManager \ --conf spark.shuffle.service.enabled=false \ --conf spark.rapids.shuffle.transport.enabled=true \ --conf spark.executorEnv.UCX_TLS=cuda_copy,cuda_ipc,rc,tcp \ diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala index 4e0f40545ed..23c37eb44d9 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/Spark30Shims.scala @@ -39,7 +39,7 @@ import org.apache.spark.unsafe.types.CalendarInterval class Spark30Shims extends SparkShims { - override def getScalaUDFAsExpression( + override def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, children: Seq[Expression], @@ -95,91 +95,90 @@ class Spark30Shims extends SparkShims { override def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( - - GpuOverrides.exec[FileSourceScanExec]( - "Reading data from files, often from Hive tables", - (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { - // partition filters and data filters are not run on the GPU - override val childExprs: Seq[ExprMeta[_]] = Seq.empty - - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) - - override def convertToGpu(): GpuExec = { - val newRelation = HadoopFsRelation( - wrapped.relation.location, - wrapped.relation.partitionSchema, - wrapped.relation.dataSchema, - wrapped.relation.bucketSpec, - GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), - wrapped.relation.options)(wrapped.relation.sparkSession) - GpuFileSourceScanExec( - newRelation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.dataFilters, - wrapped.tableIdentifier) - } - }), - GpuOverrides.exec[SortMergeJoinExec]( - "Sort merge join, replacing with shuffled hash join", - (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), - GpuOverrides.exec[BroadcastHashJoinExec]( - "Implementation of join using broadcast data", - (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), - GpuOverrides.exec[ShuffledHashJoinExec]( - "Implementation of join using hashed shuffled data", - (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { + // partition filters and data filters are not run on the GPU + override val childExprs: Seq[ExprMeta[_]] = Seq.empty + + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) + + override def convertToGpu(): GpuExec = { + val newRelation = HadoopFsRelation( + wrapped.relation.location, + wrapped.relation.partitionSchema, + wrapped.relation.dataSchema, + wrapped.relation.bucketSpec, + GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), + wrapped.relation.options)(wrapped.relation.sparkSession) + GpuFileSourceScanExec( + newRelation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.dataFilters, + wrapped.tableIdentifier) + } + }), + GpuOverrides.exec[SortMergeJoinExec]( + "Sort merge join, replacing with shuffled hash join", + (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), + GpuOverrides.exec[BroadcastHashJoinExec]( + "Implementation of join using broadcast data", + (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[ShuffledHashJoinExec]( + "Implementation of join using hashed shuffled data", + (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), ) } override def getExprs: Seq[ExprRule[_ <: Expression]] = { Seq( - GpuOverrides.expr[TimeSub]( - "Subtracts interval from timestamp", - (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.interval match { - case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => - if (intvl.months != 0) { - willNotWorkOnGpu("interval months isn't supported") - } - case _ => - willNotWorkOnGpu("only literals are supported for intervals") + GpuOverrides.expr[TimeSub]( + "Subtracts interval from timestamp", + (a, conf, p, r) => new BinaryExprMeta[TimeSub](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.interval match { + case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => + if (intvl.months != 0) { + willNotWorkOnGpu("interval months isn't supported") + } + case _ => + willNotWorkOnGpu("only literals are supported for intervals") + } + if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { + willNotWorkOnGpu("Only UTC zone id is supported") + } } - if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { - willNotWorkOnGpu("Only UTC zone id is supported") - } - } - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { - GpuTimeSub(lhs, rhs) + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { + GpuTimeSub(lhs, rhs) + } } - } - ), - GpuOverrides.expr[First]( - "first aggregate operator", - (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { - val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: BaseExprMeta[_] = - GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) - - override def convertToGpu(): GpuExpression = - GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) - }), - GpuOverrides.expr[Last]( - "last aggregate operator", - (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { - val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) - val ignoreNulls: BaseExprMeta[_] = - GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) - override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) - - override def convertToGpu(): GpuExpression = - GpuLast(child.convertToGpu(), ignoreNulls.convertToGpu()) - }), + ), + GpuOverrides.expr[First]( + "first aggregate operator", + (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) + + override def convertToGpu(): GpuExpression = + GpuFirst(child.convertToGpu(), ignoreNulls.convertToGpu()) + }), + GpuOverrides.expr[Last]( + "last aggregate operator", + (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { + val child: BaseExprMeta[_] = GpuOverrides.wrapExpr(a.child, conf, Some(this)) + val ignoreNulls: BaseExprMeta[_] = + GpuOverrides.wrapExpr(a.ignoreNullsExpr, conf, Some(this)) + override val childExprs: Seq[BaseExprMeta[_]] = Seq(child, ignoreNulls) + + override def convertToGpu(): GpuExpression = + GpuLast(child.convertToGpu(), ignoreNulls.convertToGpu()) + }), ) } diff --git a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala index 93692b49028..ca6125954a8 100644 --- a/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala +++ b/shims/spark30/src/main/scala/org/apache/spark/sql/rapids/shims/spark30/RapidsShuffleInternalManager.scala @@ -51,7 +51,6 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - getReaderInternal(handle, 0, Int.MaxValue, startPartition, endPartition, context, metrics) } } diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala index 009426b68d3..b484379bb23 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31Shims.scala @@ -93,82 +93,79 @@ class Spark31Shims extends SparkShims { override def getExprs: Seq[ExprRule[_ <: Expression]] = { Seq( - - GpuOverrides.expr[TimeAdd]( - "Subtracts interval from timestamp", - (a, conf, p, r) => new BinaryExprMeta[TimeAdd](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.interval match { - case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => - if (intvl.months != 0) { - willNotWorkOnGpu("interval months isn't supported") - } - case _ => - willNotWorkOnGpu("only literals are supported for intervals") - } - if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { - willNotWorkOnGpu("Only UTC zone id is supported") + GpuOverrides.expr[TimeAdd]( + "Subtracts interval from timestamp", + (a, conf, p, r) => new BinaryExprMeta[TimeAdd](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.interval match { + case Literal(intvl: CalendarInterval, DataTypes.CalendarIntervalType) => + if (intvl.months != 0) { + willNotWorkOnGpu("interval months isn't supported") + } + case _ => + willNotWorkOnGpu("only literals are supported for intervals") + } + if (ZoneId.of(a.timeZoneId.get).normalized() != GpuOverrides.UTC_TIMEZONE_ID) { + willNotWorkOnGpu("Only UTC zone id is supported") + } } - } - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - GpuTimeSub(lhs, rhs) - } - ), - GpuOverrides.expr[First]( - "first aggregate operator", - (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { - override def convertToGpu(): GpuExpression = - GpuFirst(childExprs(0).convertToGpu(), a.ignoreNulls) - }), - GpuOverrides.expr[Last]( - "last aggregate operator", - (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { - override def convertToGpu(): GpuExpression = - GpuLast(childExprs(0).convertToGpu(), a.ignoreNulls) - }), + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuTimeSub(lhs, rhs) + } + ), + GpuOverrides.expr[First]( + "first aggregate operator", + (a, conf, p, r) => new ExprMeta[First](a, conf, p, r) { + override def convertToGpu(): GpuExpression = + GpuFirst(childExprs(0).convertToGpu(), a.ignoreNulls) + }), + GpuOverrides.expr[Last]( + "last aggregate operator", + (a, conf, p, r) => new ExprMeta[Last](a, conf, p, r) { + override def convertToGpu(): GpuExpression = + GpuLast(childExprs(0).convertToGpu(), a.ignoreNulls) + }), ) } - override def getExecs: Seq[ExecRule[_ <: SparkPlan]] = { Seq( - - GpuOverrides.exec[FileSourceScanExec]( - "Reading data from files, often from Hive tables", - (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { - // partition filters and data filters are not run on the GPU - override val childExprs: Seq[ExprMeta[_]] = Seq.empty - - override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) - - override def convertToGpu(): GpuExec = { - val newRelation = HadoopFsRelation( - wrapped.relation.location, - wrapped.relation.partitionSchema, - wrapped.relation.dataSchema, - wrapped.relation.bucketSpec, - GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), - wrapped.relation.options)(wrapped.relation.sparkSession) - GpuFileSourceScanExec( - newRelation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.dataFilters, - wrapped.tableIdentifier) - } - }), - GpuOverrides.exec[SortMergeJoinExec]( - "Sort merge join, replacing with shuffled hash join", - (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), - GpuOverrides.exec[BroadcastHashJoinExec]( - "Implementation of join using broadcast data", - (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), - GpuOverrides.exec[ShuffledHashJoinExec]( - "Implementation of join using hashed shuffled data", - (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { + // partition filters and data filters are not run on the GPU + override val childExprs: Seq[ExprMeta[_]] = Seq.empty + + override def tagPlanForGpu(): Unit = GpuFileSourceScanExec.tagSupport(this) + + override def convertToGpu(): GpuExec = { + val newRelation = HadoopFsRelation( + wrapped.relation.location, + wrapped.relation.partitionSchema, + wrapped.relation.dataSchema, + wrapped.relation.bucketSpec, + GpuFileSourceScanExec.convertFileFormat(wrapped.relation.fileFormat), + wrapped.relation.options)(wrapped.relation.sparkSession) + GpuFileSourceScanExec( + newRelation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.dataFilters, + wrapped.tableIdentifier) + } + }), + GpuOverrides.exec[SortMergeJoinExec]( + "Sort merge join, replacing with shuffled hash join", + (join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)), + GpuOverrides.exec[BroadcastHashJoinExec]( + "Implementation of join using broadcast data", + (join, conf, p, r) => new GpuBroadcastHashJoinMeta(join, conf, p, r)), + GpuOverrides.exec[ShuffledHashJoinExec]( + "Implementation of join using hashed shuffled data", + (join, conf, p, r) => new GpuShuffledHashJoinMeta(join, conf, p, r)), ) } diff --git a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala index 02710534abf..179be3efc56 100644 --- a/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala +++ b/shims/spark31/src/main/scala/org/apache/spark/sql/rapids/shims/spark31/RapidsShuffleInternalManager.scala @@ -39,7 +39,6 @@ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - getReaderInternal(handle, 0, Int.MaxValue, startPartition, endPartition, context, metrics) } } From acece84f458879ee5a3db2ac76c4c88ba186c20c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 10:54:30 -0500 Subject: [PATCH 38/47] Fix merge issue --- .../com/nvidia/spark/rapids/HashSortOptimizeSuite.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala index a65fa2426ab..287a2a1f8e2 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala @@ -21,11 +21,6 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.execution.{SortExec, SparkPlan} -<<<<<<< HEAD -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec -======= -import org.apache.spark.sql.rapids.execution.GpuBroadcastHashJoinExec ->>>>>>> origin/branch-0.2 /** Test plan modifications to add optimizing sorts after hash joins in the plan */ class HashSortOptimizeSuite extends FunSuite { From ba377d4f1e0644f3df7ff29e291945c70317b885 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 11:04:23 -0500 Subject: [PATCH 39/47] Add newline --- .../sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index bd10cd62b73..1d75563e97c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -255,3 +255,4 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } } } + From 8f26f12e0c002236f5e3abe58a61c8cd8ec0e5d6 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 11:30:38 -0500 Subject: [PATCH 40/47] fix line length --- .../spark/rapids/BroadcastHashJoinSuite.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala index 1355e58959e..f343927d055 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala @@ -36,8 +36,12 @@ class BroadcastHashJoinSuite extends SparkQueryCompareTestSuite { val plan = df5.queryExecution.executedPlan - assert(plan.collect { case p if ShimLoader.getSparkShims.isGpuBroadcastHashJoin(p) => p }.size === 1) - assert(plan.collect { case p if ShimLoader.getSparkShims.isGpuShuffledHashJoin(p) => p }.size === 1) + assert(plan.collect { + case p if ShimLoader.getSparkShims.isGpuBroadcastHashJoin(p) => p + }.size === 1) + assert(plan.collect { + case p if ShimLoader.getSparkShims.isGpuShuffledHashJoin(p) => p + }.size === 1) }, conf) } @@ -55,10 +59,10 @@ class BroadcastHashJoinSuite extends SparkQueryCompareTestSuite { val res1 = plan1.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) val res2 = plan2.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) - assert( - ShimLoader.getSparkShims.getBuildSide(res1.get.asInstanceOf[HashJoin]).toString == "GpuBuildLeft") - assert( - ShimLoader.getSparkShims.getBuildSide(res2.get.asInstanceOf[HashJoin]).toString == "GpuBuildRight") + assert(ShimLoader.getSparkShims.getBuildSide(res1.get.asInstanceOf[HashJoin]).toString == + "GpuBuildLeft") + assert(ShimLoader.getSparkShims.getBuildSide(res2.get.asInstanceOf[HashJoin]).toString == + "GpuBuildRight") } }) } From b1b61554bae36bd0b7998fd6fdac821aa45f892d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 11:36:48 -0500 Subject: [PATCH 41/47] Fix import order --- .../com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala index 354047f125f..5355d7d2549 100644 --- a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala +++ b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/tpch/TpchLikeSparkSuite.scala @@ -16,8 +16,8 @@ package com.nvidia.spark.rapids.tests.tpch -import com.nvidia.spark.rapids.ShimLoader import com.nvidia.spark.rapids.{ColumnarRdd, ExecutionPlanCaptureCallback} +import com.nvidia.spark.rapids.ShimLoader import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.{DataFrame, SparkSession} From e6370126924cfedf656e0aa8bd56a44a640b4125 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 12:14:28 -0500 Subject: [PATCH 42/47] Remove unneeded changes in GpuFirst --- .../scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala | 2 +- .../com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala | 4 ---- .../scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala | 2 +- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala index 302d50ceaea..d5df5f9c424 100644 --- a/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala +++ b/shims/spark30/src/main/scala/com/nvidia/spark/rapids/shims/spark30/GpuLast.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.rapids.GpuLastBase /** - * Parameters to GpuFirst changed in Spark 3.1 + * Parameters to GpuLast changed in Spark 3.1 */ case class GpuLast(child: Expression, ignoreNullsExpr: Expression) extends GpuLastBase(child) { override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala index 724d8b1d421..f1c11e1673f 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuFirst.scala @@ -26,10 +26,6 @@ import org.apache.spark.sql.rapids.GpuFirstBase * Parameters to GpuFirst changed in Spark 3.1 */ case class GpuFirst(child: Expression, ignoreNulls: Boolean) extends GpuFirstBase(child) { - def this(child: Expression) = this(child, false) - def this(child: Expression, ignoreNullsExpr: Expression) = { - this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "last")) - } override def children: Seq[Expression] = child :: Nil override def checkInputDataTypes(): TypeCheckResult = { diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala index 66b730f5441..03d281d0693 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/GpuLast.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.rapids.GpuLastBase /** - * Parameters to GpuFirst changed in Spark 3.1 + * Parameters to GpuLast changed in Spark 3.1 */ case class GpuLast(child: Expression, ignoreNulls: Boolean) extends GpuLastBase(child) { override def children: Seq[Expression] = child :: Nil From 4aed4d0924f8245162dcb517fb1e49ec5b45b9cb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 13:10:03 -0500 Subject: [PATCH 43/47] Cleanup poms and versions check for 3.1 --- integration_tests/pom.xml | 2 +- shims/spark30/pom.xml | 45 ------------------ shims/spark31/pom.xml | 47 +------------------ .../spark31/Spark31ShimServiceProvider.scala | 4 +- 4 files changed, 4 insertions(+), 94 deletions(-) diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index 97c566db86a..54483136802 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -60,7 +60,7 @@ org.apache.spark spark-sql_${scala.binary.version} - ${spark.test.version} + ${spark.test.version} org.scalatest diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index 44fbf1339f6..efbe1e34fd2 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -46,56 +46,11 @@ cudf ${cuda.version} - - com.google.flatbuffers - flatbuffers-java - compile - - - org.scala-lang - scala-library - org.apache.spark spark-sql_${scala.binary.version} ${spark30.version} - - org.apache.orc - orc-core - ${orc.classifier} - - - org.slf4j - slf4j-api - - - - - org.apache.orc - orc-mapreduce - ${orc.classifier} - - - com.google.code.findbugs - jsr305 - - - - - org.apache.hive - hive-storage-api - - - org.slf4j - slf4j-api - - - - - com.google.protobuf - protobuf-java - diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 6b8451ef506..05c407bda6f 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -46,55 +46,10 @@ cudf ${cuda.version} - - com.google.flatbuffers - flatbuffers-java - compile - - - org.scala-lang - scala-library - org.apache.spark spark-sql_${scala.binary.version} - ${spark31.version} - - - org.apache.orc - orc-core - ${orc.classifier} - - - org.slf4j - slf4j-api - - - - - org.apache.orc - orc-mapreduce - ${orc.classifier} - - - com.google.code.findbugs - jsr305 - - - - - org.apache.hive - hive-storage-api - - - org.slf4j - slf4j-api - - - - - com.google.protobuf - protobuf-java + ${spark31.version} diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala index ac3bbd223f8..537e2f60711 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala @@ -20,10 +20,10 @@ import com.nvidia.spark.rapids.{SparkShims, SparkShimServiceProvider} class Spark31ShimServiceProvider extends SparkShimServiceProvider { - val SPARK31VERSIONNAME = "3.1.0-SNAPSHOT" + val SPARK31VERSIONNAMES = Seq("3.1.0-SNAPSHOT", "3.1.0") def matchesVersion(version: String): Boolean = { - version == SPARK31VERSIONNAME + SPARK31VERSIONNAME.contains(version) } def buildShim: SparkShims = { From 9bddb287b3047108257035df91dca28dab1c7c04 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 13:37:12 -0500 Subject: [PATCH 44/47] move slf4j dep up --- integration_tests/pom.xml | 3 --- pom.xml | 12 ++++++++++++ .../shims/spark31/Spark31ShimServiceProvider.scala | 2 +- tests/pom.xml | 3 --- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index 54483136802..e271790525b 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -29,7 +29,6 @@ 0.2.0-SNAPSHOT - 1.7.30 3.0.0 @@ -45,12 +44,10 @@ org.slf4j jul-to-slf4j - ${slf4j.version} org.slf4j jcl-over-slf4j - ${slf4j.version} diff --git a/pom.xml b/pom.xml index 41f0c8f10be..0ffa006c5b9 100644 --- a/pom.xml +++ b/pom.xml @@ -156,6 +156,7 @@ UTF-8 not qarun false + 1.7.30 @@ -172,6 +173,17 @@ ${cuda.version} provided + + org.slf4j + jul-to-slf4j + ${slf4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + org.scala-lang scala-library diff --git a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala index 537e2f60711..2920e9e6dac 100644 --- a/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala +++ b/shims/spark31/src/main/scala/com/nvidia/spark/rapids/shims/spark31/Spark31ShimServiceProvider.scala @@ -23,7 +23,7 @@ class Spark31ShimServiceProvider extends SparkShimServiceProvider { val SPARK31VERSIONNAMES = Seq("3.1.0-SNAPSHOT", "3.1.0") def matchesVersion(version: String): Boolean = { - SPARK31VERSIONNAME.contains(version) + SPARK31VERSIONNAMES.contains(version) } def buildShim: SparkShims = { diff --git a/tests/pom.xml b/tests/pom.xml index 71364044755..729baa43440 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -31,7 +31,6 @@ 0.2.0-SNAPSHOT - 1.7.30 3.0.0 @@ -47,12 +46,10 @@ org.slf4j jul-to-slf4j - ${slf4j.version} org.slf4j jcl-over-slf4j - ${slf4j.version} From c89bdf9b0c1eb5e4d94ba4de437f71f7acad70a8 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 14:11:38 -0500 Subject: [PATCH 45/47] Change parent pom path --- shims/aggregator/pom.xml | 2 +- shims/spark30/pom.xml | 2 +- shims/spark31/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/shims/aggregator/pom.xml b/shims/aggregator/pom.xml index f9576968f74..4345243933a 100644 --- a/shims/aggregator/pom.xml +++ b/shims/aggregator/pom.xml @@ -23,7 +23,7 @@ com.nvidia rapids-4-spark-parent 0.2.0-SNAPSHOT - ../../pom.xml + ../pom.xml com.nvidia rapids-4-spark-shims_2.12 diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index efbe1e34fd2..1b74f76a661 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -23,7 +23,7 @@ com.nvidia rapids-4-spark-parent 0.2.0-SNAPSHOT - ../../pom.xml + ../pom.xml com.nvidia rapids-4-spark-shims-spark30_2.12 diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 05c407bda6f..635b9920f16 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -23,7 +23,7 @@ com.nvidia rapids-4-spark-parent 0.2.0-SNAPSHOT - ../../pom.xml + ../pom.xml com.nvidia rapids-4-spark-shims-spark31_2.12 From 787e12bf3d08451138443bd07cf3428bd3017a5a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 15:02:33 -0500 Subject: [PATCH 46/47] move rat exclude check to shim poms since parent changed --- pom.xml | 1 - shims/spark30/pom.xml | 9 +++++++++ shims/spark31/pom.xml | 9 +++++++++ .../spark/sql/rapids/RapidsShuffleInternalManager.scala | 2 -- 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 0ffa006c5b9..e77777ed077 100644 --- a/pom.xml +++ b/pom.xml @@ -501,7 +501,6 @@ *.jar docs/demo/**/*.ipynb docs/demo/**/*.zpln - **/META-INF/services/com.nvidia.spark.rapids.SparkShimServiceProvider diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index 1b74f76a661..508e7d48111 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -59,6 +59,15 @@ net.alchim31.maven scala-maven-plugin + + org.apache.rat + apache-rat-plugin + + + **/src/main/resources/META-INF/services/* + + + diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 635b9920f16..5eff0463c80 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -59,6 +59,15 @@ net.alchim31.maven scala-maven-plugin + + org.apache.rat + apache-rat-plugin + + + **/src/main/resources/META-INF/services/* + + + diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala index ee26a0cfcfd..dc8a847a098 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManager.scala @@ -182,8 +182,6 @@ class RapidsCachingWriter[K, V]( * @note This is an internal class to obtain access to the private * `ShuffleManager` and `SortShuffleManager` classes. When configuring * Apache Spark to use the RAPIDS shuffle manager, - * [[com.nvidia.spark.RapidsShuffleManager]] should be used as that is - * the public class. */ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, isDriver: Boolean) extends ShuffleManager with Logging { From 954044777c2061e17798447e9284698b8ca5f1b2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Jul 2020 17:11:29 -0500 Subject: [PATCH 47/47] Switch to use parent pom instead of aggregator module --- shims/aggregator/pom.xml | 2 +- shims/pom.xml | 31 ++++++++++++++++++++++++++++++- shims/spark30/pom.xml | 29 +---------------------------- shims/spark31/pom.xml | 30 +----------------------------- 4 files changed, 33 insertions(+), 59 deletions(-) diff --git a/shims/aggregator/pom.xml b/shims/aggregator/pom.xml index 4345243933a..062f4da1845 100644 --- a/shims/aggregator/pom.xml +++ b/shims/aggregator/pom.xml @@ -21,7 +21,7 @@ com.nvidia - rapids-4-spark-parent + rapids-4-spark-shims-aggregator_2.12 0.2.0-SNAPSHOT ../pom.xml diff --git a/shims/pom.xml b/shims/pom.xml index 436d4a31b5f..80333a2bb9b 100644 --- a/shims/pom.xml +++ b/shims/pom.xml @@ -26,7 +26,7 @@ ../pom.xml com.nvidia - rapids-4-spark-shims_aggregator_2.12 + rapids-4-spark-shims-aggregator_2.12 pom RAPIDS Accelerator for Apache Spark SQL Plugin Shims The RAPIDS SQL plugin for Apache Spark Shims @@ -37,4 +37,33 @@ spark31 aggregator + + + com.nvidia + rapids-4-spark-sql_${scala.binary.version} + ${project.version} + + + ai.rapids + cudf + ${cuda.version} + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.rat + apache-rat-plugin + + + **/src/main/resources/META-INF/services/* + + + + + diff --git a/shims/spark30/pom.xml b/shims/spark30/pom.xml index 508e7d48111..665e3b64fa0 100644 --- a/shims/spark30/pom.xml +++ b/shims/spark30/pom.xml @@ -21,7 +21,7 @@ com.nvidia - rapids-4-spark-parent + rapids-4-spark-shims-aggregator_2.12 0.2.0-SNAPSHOT ../pom.xml @@ -36,16 +36,6 @@ - - com.nvidia - rapids-4-spark-sql_${scala.binary.version} - ${project.version} - - - ai.rapids - cudf - ${cuda.version} - org.apache.spark spark-sql_${scala.binary.version} @@ -53,21 +43,4 @@ - - - - net.alchim31.maven - scala-maven-plugin - - - org.apache.rat - apache-rat-plugin - - - **/src/main/resources/META-INF/services/* - - - - - diff --git a/shims/spark31/pom.xml b/shims/spark31/pom.xml index 5eff0463c80..3e4604f9bc3 100644 --- a/shims/spark31/pom.xml +++ b/shims/spark31/pom.xml @@ -21,7 +21,7 @@ com.nvidia - rapids-4-spark-parent + rapids-4-spark-shims-aggregator_2.12 0.2.0-SNAPSHOT ../pom.xml @@ -36,38 +36,10 @@ - - com.nvidia - rapids-4-spark-sql_${scala.binary.version} - ${project.version} - - - ai.rapids - cudf - ${cuda.version} - org.apache.spark spark-sql_${scala.binary.version} ${spark31.version} - - - - - net.alchim31.maven - scala-maven-plugin - - - org.apache.rat - apache-rat-plugin - - - **/src/main/resources/META-INF/services/* - - - - -