Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Added Support For Various Execs for Databricks 13.3 #9637

Merged
merged 16 commits into from
Nov 7, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
{"spark": "333"}
{"spark": "340"}
{"spark": "341"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.hive.rapids.shims

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
{"spark": "333"}
{"spark": "340"}
{"spark": "341"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.hive.rapids.shims

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
{"spark": "333"}
{"spark": "340"}
{"spark": "341"}
{"spark": "341db"}
{"spark": "350"}
spark-rapids-shim-json-lines ***/
package com.nvidia.spark.rapids
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,47 +22,14 @@ package org.apache.spark.rapids.shims

import com.nvidia.spark.rapids.GpuPartitioning

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan}
import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.exchange.ShuffleOrigin

case class GpuShuffleExchangeExec(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin)(
cpuOutputPartitioning: Partitioning)
extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child)
with ShuffleExchangeLike {

override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil

override val outputPartitioning: Partitioning = cpuOutputPartitioning

override def numMappers: Int = shuffleDependencyColumnar.rdd.getNumPartitions

override def numPartitions: Int = shuffleDependencyColumnar.partitioner.numPartitions

override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = {
new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs)
}

// DB SPECIFIC - throw if called since we don't know how its used
override def withNewOutputPartitioning(outputPartitioning: Partitioning) = {
throw new UnsupportedOperationException
}

override def runtimeStatistics: Statistics = {
// note that Spark will only use the sizeInBytes statistic but making the rowCount
// available here means that we can more easily reference it in GpuOverrides when
// planning future query stages when AQE is on
Statistics(
sizeInBytes = metrics("dataSize").value,
rowCount = Some(metrics("numOutputRows").value)
)
}

override def shuffleId: Int = shuffleDependencyColumnar.shuffleId
}
extends GpuDatabricksShuffleExchangeExecBase(gpuOutputPartitioning,
child, shuffleOrigin)(cpuOutputPartitioning)
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Copyright (c) 2022-2023, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/*** spark-rapids-shim-json-lines
{"spark": "321db"}
{"spark": "330db"}
{"spark": "332db"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.rapids.shims

import com.nvidia.spark.rapids.GpuPartitioning

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan}
import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD}

abstract class GpuDatabricksShuffleExchangeExecBase(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin)(
cpuOutputPartitioning: Partitioning)
extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child)
with ShuffleExchangeLike {

override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil

override val outputPartitioning: Partitioning = cpuOutputPartitioning

override def numMappers: Int = shuffleDependencyColumnar.rdd.getNumPartitions

override def numPartitions: Int = shuffleDependencyColumnar.partitioner.numPartitions

override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = {
new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs)
}

// DB SPECIFIC - throw if called since we don't know how its used
override def withNewOutputPartitioning(outputPartitioning: Partitioning) = {
throw new UnsupportedOperationException
}

override def runtimeStatistics: Statistics = {
// note that Spark will only use the sizeInBytes statistic but making the rowCount
// available here means that we can more easily reference it in GpuOverrides when
// planning future query stages when AQE is on
Statistics(
sizeInBytes = metrics("dataSize").value,
rowCount = Some(metrics("numOutputRows").value)
)
}

override def shuffleId: Int = shuffleDependencyColumnar.shuffleId
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import com.nvidia.spark.rapids._
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuShuffleExchangeExecBase}
import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuCustomShuffleReaderExec, GpuShuffleExchangeExecBase}

abstract class GpuBroadcastJoinMeta[INPUT <: SparkPlan](plan: INPUT,
conf: RapidsConf,
Expand All @@ -51,18 +51,21 @@ abstract class GpuBroadcastJoinMeta[INPUT <: SparkPlan](plan: INPUT,
}

def verifyBuildSideWasReplaced(buildSide: SparkPlan): Unit = {
def isOnGpu(sqse: ShuffleQueryStageExec): Boolean = sqse.plan match {
case _: GpuShuffleExchangeExecBase => true
case ReusedExchangeExec(_, _: GpuShuffleExchangeExecBase) => true
case _ => false
}
val buildSideOnGpu = buildSide match {
case bqse: BroadcastQueryStageExec => bqse.plan.isInstanceOf[GpuBroadcastExchangeExec] ||
bqse.plan.isInstanceOf[ReusedExchangeExec] &&
bqse.plan.asInstanceOf[ReusedExchangeExec]
.child.isInstanceOf[GpuBroadcastExchangeExec]
case sqse: ShuffleQueryStageExec => sqse.plan.isInstanceOf[GpuShuffleExchangeExecBase] ||
sqse.plan.isInstanceOf[ReusedExchangeExec] &&
sqse.plan.asInstanceOf[ReusedExchangeExec]
.child.isInstanceOf[GpuShuffleExchangeExecBase]
case sqse: ShuffleQueryStageExec => isOnGpu(sqse)
case reused: ReusedExchangeExec => reused.child.isInstanceOf[GpuBroadcastExchangeExec] ||
reused.child.isInstanceOf[GpuShuffleExchangeExecBase]
case _: GpuBroadcastExchangeExec | _: GpuShuffleExchangeExecBase => true
case GpuCustomShuffleReaderExec(sqse: ShuffleQueryStageExec, _) => isOnGpu(sqse)
case _ => false
}
if (!buildSideOnGpu) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,12 @@
{"spark": "332db"}
{"spark": "340"}
{"spark": "341"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.execution.datasources.parquet

import org.apache.spark.memory.MemoryMode
import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues
import org.apache.spark.sql.execution.vectorized.WritableColumnVector
import org.apache.spark.sql.types.StructType

Expand All @@ -38,7 +40,7 @@ object ParquetCVShims {
missingColumns: java.util.Set[ParquetColumn],
isTopLevel: Boolean): ParquetColumnVector = {
val defaultValue = if (sparkSchema != null) {
sparkSchema.existenceDefaultValues(idx)
getExistenceDefaultValues(sparkSchema)
} else null
new ParquetColumnVector(column, vector, capacity, memoryMode, missingColumns, isTopLevel,
defaultValue)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
/*** spark-rapids-shim-json-lines
{"spark": "330db"}
{"spark": "332db"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution

Expand Down Expand Up @@ -112,16 +113,22 @@ case class GpuBroadcastHashJoinExec(
executorBroadcast
}

def shuffleExchange: GpuShuffleExchangeExec = buildPlan match {
case bqse: ShuffleQueryStageExec if bqse.plan.isInstanceOf[GpuShuffleExchangeExec] =>
bqse.plan.asInstanceOf[GpuShuffleExchangeExec]
case bqse: ShuffleQueryStageExec if bqse.plan.isInstanceOf[ReusedExchangeExec] =>
bqse.plan.asInstanceOf[ReusedExchangeExec].child.asInstanceOf[GpuShuffleExchangeExec]
case gpu: GpuShuffleExchangeExec => gpu
case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuShuffleExchangeExec]
def shuffleExchange: GpuShuffleExchangeExec = {
def from(p: ShuffleQueryStageExec): GpuShuffleExchangeExec = p.plan match {
case g: GpuShuffleExchangeExec => g
case ReusedExchangeExec(_, g: GpuShuffleExchangeExec) => g
case _ => throw new IllegalStateException(s"cannot locate GPU shuffle in $p")
}
buildPlan match {
case gpu: GpuShuffleExchangeExec => gpu
case sqse: ShuffleQueryStageExec => from(sqse)
case reused: ReusedExchangeExec => reused.child.asInstanceOf[GpuShuffleExchangeExec]
case GpuShuffleCoalesceExec(GpuCustomShuffleReaderExec(sqse: ShuffleQueryStageExec, _), _) =>
from(sqse)
case GpuCustomShuffleReaderExec(sqse: ShuffleQueryStageExec, _) => from(sqse)
}
}


private def getExecutorBuiltBatchAndStreamIter(
buildRelation: RDD[ColumnarBatch],
buildSchema: StructType,
Expand Down Expand Up @@ -158,7 +165,7 @@ case class GpuBroadcastHashJoinExec(

// Get all the broadcast data from the shuffle coalesced into a single partition
val partitionSpecs = Seq(CoalescedPartitionSpec(0, shuffleExchange.numPartitions))
val buildRelation = shuffleExchange.getShuffleRDD(partitionSpecs.toArray)
val buildRelation = ShuffleExchangeShim.getShuffleRDD(shuffleExchange, partitionSpecs)
.asInstanceOf[RDD[ColumnarBatch]]

val rdd = streamedPlan.executeColumnar()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
/*** spark-rapids-shim-json-lines
{"spark": "330db"}
{"spark": "332db"}
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution

Expand Down Expand Up @@ -130,7 +131,7 @@ case class GpuBroadcastNestedLoopJoinExec(
if (executorBroadcast) {
// Get all the broadcast data from the shuffle coalesced into a single partition
val partitionSpecs = Seq(CoalescedPartitionSpec(0, shuffleExchange.numPartitions))
shuffleExchange.getShuffleRDD(partitionSpecs.toArray).asInstanceOf[RDD[ColumnarBatch]]
ShuffleExchangeShim.getShuffleRDD(shuffleExchange, partitionSpecs)
} else {
broadcastExchange.executeColumnarBroadcast[Any]()
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

/*** spark-rapids-shim-json-lines
{"spark": "330db"}
{"spark": "332db"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution

import org.apache.spark.rapids.shims.GpuShuffleExchangeExec
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.execution.CoalescedPartitionSpec
import org.apache.spark.sql.vectorized.ColumnarBatch

object ShuffleExchangeShim {
def getShuffleRDD(
shuffleExchange: GpuShuffleExchangeExec,
partitionSpecs: Seq[CoalescedPartitionSpec]): RDD[ColumnarBatch] = {
shuffleExchange.getShuffleRDD(partitionSpecs.toArray)
.asInstanceOf[RDD[ColumnarBatch]]
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
*/

/*** spark-rapids-shim-json-lines
{"spark": "341db"}
{"spark": "350"}
spark-rapids-shim-json-lines ***/
package com.nvidia.spark.rapids.shims
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

/*** spark-rapids-shim-json-lines
{"spark": "341db"}
spark-rapids-shim-json-lines ***/
package com.nvidia.spark.rapids.shims

import org.apache.spark.paths.SparkPath
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.PartitionedFile

object PartitionedFileUtilsShim {
// Wrapper for case class constructor so Java code can access
// the default values across Spark versions.
def newPartitionedFile(
partitionValues: InternalRow,
filePath: String,
start: Long,
length: Long): PartitionedFile = {
PartitionedFile(partitionValues, SparkPath.fromPathString(filePath), start, length)
}

def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = {
pf.copy(locations = locations)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ spark-rapids-shim-json-lines ***/
package com.nvidia.spark.rapids.spark341db

import org.apache.spark.SparkConf
import org.apache.spark.sql.shims.ProxyRapidsShuffleInternalManagerBase
import org.apache.spark.sql.rapids.ProxyRapidsShuffleInternalManagerBase

/** A shuffle manager optimized for the RAPIDS Plugin for Apache Spark. */
sealed class RapidsShuffleManager(
Expand Down
Loading