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

Unshim many SparkShim interfaces [databricks] #5031

Merged
merged 17 commits into from
Mar 24, 2022
Merged

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.execution.python.shims

import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.python.PythonWorkerSemaphore
import com.nvidia.spark.rapids.shims.{ShimUnaryExecNode, SparkShimImpl}
import com.nvidia.spark.rapids.shims.ShimUnaryExecNode

import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
Expand Down Expand Up @@ -96,7 +96,7 @@ case class GpuFlatMapGroupsInPandasExec(
}

override def requiredChildOrdering: Seq[Seq[SortOrder]] =
Seq(groupingAttributes.map(SparkShimImpl.sortOrder(_, Ascending)))
Seq(groupingAttributes.map(SortOrder(_, Ascending)))

private val pandasFunction = func.asInstanceOf[GpuPythonUDF].func

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,6 @@

package org.apache.spark.rapids.shims

import com.nvidia.spark.rapids.shims.SparkShimImpl

import org.apache.spark.{MapOutputTrackerMaster, Partition, ShuffleDependency, SparkEnv, TaskContext}
import org.apache.spark.shuffle.ShuffleReader
import org.apache.spark.sql.execution.{CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec}
Expand Down Expand Up @@ -57,7 +55,6 @@ object ShuffledBatchRDDUtil {
dependency: ShuffleDependency[Int, ColumnarBatch, ColumnarBatch],
sqlMetricsReporter: SQLShuffleReadMetricsReporter):
(ShuffleReader[Nothing, Nothing], Long) = {
val shim = SparkShimImpl
split.asInstanceOf[ShuffledBatchRDDPartition].spec match {
case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) =>
val reader = SparkEnv.get.shuffleManager.getReader(
Expand All @@ -66,7 +63,7 @@ object ShuffledBatchRDDUtil {
endReducerIndex,
context,
sqlMetricsReporter)
val blocksByAddress = shim.getMapSizesByExecutorId(
val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(
dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex)
val partitionSize = blocksByAddress.flatMap(_._2).map(_._2).sum
(reader, partitionSize)
Expand All @@ -80,7 +77,7 @@ object ShuffledBatchRDDUtil {
reducerIndex + 1,
context,
sqlMetricsReporter)
val blocksByAddress = shim.getMapSizesByExecutorId(
val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(
dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, reducerIndex,
reducerIndex + 1)
val partitionSize = blocksByAddress.flatMap(_._2)
Expand All @@ -96,7 +93,7 @@ object ShuffledBatchRDDUtil {
endReducerIndex,
context,
sqlMetricsReporter)
val blocksByAddress = shim.getMapSizesByExecutorId(
val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(
dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex)
val partitionSize = blocksByAddress.flatMap(_._2)
.filter(_._3 == mapIndex)
Expand Down
Loading