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

Refactor to stop inheriting from HashJoin #1321

Merged
merged 2 commits into from
Dec 9, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec
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.{GpuBroadcastExchangeExecBase, SerializeConcatHostBuffersDeserializeBatch}
import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch}
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.vectorized.ColumnarBatch

Expand Down Expand Up @@ -85,7 +85,7 @@ class GpuBroadcastHashJoinMeta(
GpuBroadcastHashJoinExec(
leftKeys.map(_.convertToGpu()),
rightKeys.map(_.convertToGpu()),
join.joinType, join.buildSide,
join.joinType, GpuJoinUtils.getGpuBuildSide(join.buildSide),
condition.map(_.convertToGpu()),
left, right)
}
Expand All @@ -95,7 +95,7 @@ case class GpuBroadcastHashJoinExec(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
buildSide: BuildSide,
buildSide: GpuBuildSide,
condition: Option[Expression],
left: SparkPlan,
right: SparkPlan) extends BinaryExecNode with GpuHashJoin {
Expand All @@ -109,9 +109,9 @@ case class GpuBroadcastHashJoinExec(
override def requiredChildDistribution: Seq[Distribution] = {
val mode = HashedRelationBroadcastMode(buildKeys)
buildSide match {
case BuildLeft =>
case GpuBuildLeft =>
BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil
case BuildRight =>
case GpuBuildRight =>
UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClustered
import org.apache.spark.sql.execution.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.rapids.execution.GpuShuffledHashJoinBase
import org.apache.spark.sql.rapids.execution.{GpuHashJoin, GpuShuffledHashJoinBase}
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.vectorized.ColumnarBatch

Expand Down Expand Up @@ -73,7 +73,7 @@ class GpuShuffledHashJoinMeta(
leftKeys.map(_.convertToGpu()),
rightKeys.map(_.convertToGpu()),
join.joinType,
join.buildSide,
GpuJoinUtils.getGpuBuildSide(join.buildSide),
condition.map(_.convertToGpu()),
childPlans(0).convertIfNeeded(),
childPlans(1).convertIfNeeded(),
Expand All @@ -84,7 +84,7 @@ case class GpuShuffledHashJoinExec(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
buildSide: BuildSide,
buildSide: GpuBuildSide,
condition: Option[Expression],
left: SparkPlan,
right: SparkPlan,
Expand Down Expand Up @@ -116,8 +116,8 @@ case class GpuShuffledHashJoinExec(
}

override def childrenCoalesceGoal: Seq[CoalesceGoal] = buildSide match {
case BuildLeft => Seq(RequireSingleBatch, null)
case BuildRight => Seq(null, RequireSingleBatch)
case GpuBuildLeft => Seq(RequireSingleBatch, null)
case GpuBuildRight => Seq(null, RequireSingleBatch)
}

override def doExecuteColumnar() : RDD[ColumnarBatch] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,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.{BuildLeft, BuildRight, SortMergeJoinExec}
import org.apache.spark.sql.rapids.execution.GpuHashJoin
import org.apache.spark.sql.types.DataType

/**
Expand Down Expand Up @@ -85,7 +86,7 @@ class GpuSortMergeJoinMeta(
leftKeys.map(_.convertToGpu()),
rightKeys.map(_.convertToGpu()),
join.joinType,
buildSide,
GpuJoinUtils.getGpuBuildSide(buildSide),
condition.map(_.convertToGpu()),
childPlans(0).convertIfNeeded(),
childPlans(1).convertIfNeeded(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,13 +105,6 @@ class Spark300Shims extends SparkShims {
queryStage.shuffle.asInstanceOf[GpuShuffleExchangeExecBase]
}

override def isGpuHashJoin(plan: SparkPlan): Boolean = {
plan match {
case _: GpuHashJoin => true
case _ => false
}
}

override def isGpuBroadcastHashJoin(plan: SparkPlan): Boolean = {
plan match {
case _: GpuBroadcastHashJoinExec => true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@
*/
package com.nvidia.spark.rapids.shims.spark301

import com.nvidia.spark.rapids.{BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBroadcastJoinMeta, GpuColumnVector, GpuExec, GpuOverrides, GpuProjectExec, RapidsConf, RapidsMeta, SparkPlanMeta}
import com.nvidia.spark.rapids.{BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBroadcastJoinMeta, GpuBuildLeft, GpuBuildRight, GpuBuildSide, GpuColumnVector, GpuExec, GpuOverrides, GpuProjectExec, RapidsConf, RapidsMeta, SparkPlanMeta}
import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME}
import com.nvidia.spark.rapids.shims.spark300.GpuHashJoin
import com.nvidia.spark.rapids.shims.spark300.GpuJoinUtils

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
Expand All @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec
import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide, HashedRelationBroadcastMode}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.rapids.execution.SerializeConcatHostBuffersDeserializeBatch
import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch}
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.vectorized.ColumnarBatch

Expand Down Expand Up @@ -82,7 +82,8 @@ class GpuBroadcastHashJoinMeta(
GpuBroadcastHashJoinExec(
leftKeys.map(_.convertToGpu()),
rightKeys.map(_.convertToGpu()),
join.joinType, join.buildSide,
join.joinType,
GpuJoinUtils.getGpuBuildSide(join.buildSide),
condition.map(_.convertToGpu()),
left, right)
}
Expand All @@ -92,7 +93,7 @@ case class GpuBroadcastHashJoinExec(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
buildSide: BuildSide,
buildSide: GpuBuildSide,
condition: Option[Expression],
left: SparkPlan,
right: SparkPlan) extends BinaryExecNode with GpuHashJoin {
Expand All @@ -106,9 +107,9 @@ case class GpuBroadcastHashJoinExec(
override def requiredChildDistribution: Seq[Distribution] = {
val mode = HashedRelationBroadcastMode(buildKeys)
buildSide match {
case BuildLeft =>
case GpuBuildLeft =>
BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil
case BuildRight =>
case GpuBuildRight =>
UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
*/
package com.nvidia.spark.rapids.shims.spark301db

import com.nvidia.spark.rapids.{BaseExprMeta, ConfKeysAndIncompat, GpuBindReferences, GpuBroadcastJoinMeta, GpuColumnVector, GpuExec, GpuOverrides, GpuProjectExec, RapidsConf, RapidsMeta, SparkPlanMeta}
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME}

import org.apache.spark.rdd.RDD
Expand All @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec
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.SerializeConcatHostBuffersDeserializeBatch
import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch}
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.vectorized.ColumnarBatch

Expand Down Expand Up @@ -82,7 +82,8 @@ class GpuBroadcastHashJoinMeta(
GpuBroadcastHashJoinExec(
leftKeys.map(_.convertToGpu()),
rightKeys.map(_.convertToGpu()),
join.joinType, join.buildSide,
join.joinType,
GpuJoinUtils.getGpuBuildSide(join.buildSide),
condition.map(_.convertToGpu()),
left, right)
}
Expand All @@ -92,7 +93,7 @@ case class GpuBroadcastHashJoinExec(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
buildSide: BuildSide,
buildSide: GpuBuildSide,
condition: Option[Expression],
left: SparkPlan,
right: SparkPlan) extends BinaryExecNode with GpuHashJoin {
Expand All @@ -106,9 +107,9 @@ case class GpuBroadcastHashJoinExec(
override def requiredChildDistribution: Seq[Distribution] = {
val mode = HashedRelationBroadcastMode(buildKeys)
buildSide match {
case BuildLeft =>
case GpuBuildLeft =>
BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil
case BuildRight =>
case GpuBuildRight =>
UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil
}
}
Expand Down
Loading