zml1206 commented on code in PR #6093:
URL: https://github.com/apache/incubator-gluten/pull/6093#discussion_r1645291872
##########
gluten-core/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala:
##########
@@ -124,32 +124,10 @@ case class OffloadJoin() extends OffloadSingleNode with
LogLevelUtil {
}
plan match {
case plan: ShuffledHashJoinExec =>
- val left = plan.left
- val right = plan.right
logDebug(s"Columnar Processing for ${plan.getClass} is currently
supported.")
- BackendsApiManager.getSparkPlanExecApiInstance
- .genShuffledHashJoinExecTransformer(
- plan.leftKeys,
- plan.rightKeys,
- plan.joinType,
- TransformHints.getShuffleHashJoinBuildSide(plan),
- plan.condition,
- left,
- right,
- plan.isSkewJoin)
+ ShuffledHashJoinExecTransformerBase.from(plan)
case plan: SortMergeJoinExec =>
Review Comment:
Do validate is in `AddTransformHintRule`
```
val transformer = OffloadJoin.transformSortMergeJoinExec(plan)
transformer.doValidate().tagOnFallback(plan)
```
##########
gluten-core/src/main/scala/org/apache/gluten/extension/StrategyOverrides.scala:
##########
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You 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.gluten.extension
-
-import org.apache.gluten.{GlutenConfig, GlutenSparkExtensionsInjector}
-import org.apache.gluten.backendsapi.BackendsApiManager
-import org.apache.gluten.extension.columnar.TRANSFORM_UNSUPPORTED
-import org.apache.gluten.extension.columnar.TransformHints.TAG
-import org.apache.gluten.utils.LogicalPlanSelector
-
-import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy}
-import org.apache.spark.sql.catalyst.SQLConfHelper
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight,
JoinSelectionHelper}
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.{joins, JoinSelectionShim, SparkPlan}
-import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec,
LogicalQueryStage}
-import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
-
-object StrategyOverrides extends GlutenSparkExtensionsInjector {
- override def inject(extensions: SparkSessionExtensions): Unit = {
- extensions.injectPlannerStrategy(JoinSelectionOverrides)
- }
-}
-
-case class JoinSelectionOverrides(session: SparkSession)
- extends Strategy
- with JoinSelectionHelper
- with SQLConfHelper {
-
- private def isBroadcastStage(plan: LogicalPlan): Boolean = plan match {
- case LogicalQueryStage(_, _: BroadcastQueryStageExec) => true
- case _ => false
- }
-
- def extractEqualJoinKeyCondition(
- joinType: JoinType,
- leftKeys: Seq[Expression],
- rightKeys: Seq[Expression],
- condition: Option[Expression],
- left: LogicalPlan,
- right: LogicalPlan,
- hint: JoinHint,
- forceShuffledHashJoin: Boolean): Seq[SparkPlan] = {
- if (isBroadcastStage(left) || isBroadcastStage(right)) {
- val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight
- Seq(
- BroadcastHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide,
- condition,
- planLater(left),
- planLater(right)))
- } else {
- // Generate BHJ here, avoid to do match in `JoinSelection` again.
- val isHintEmpty = hint.leftHint.isEmpty && hint.rightHint.isEmpty
- val buildSide = getBroadcastBuildSide(left, right, joinType, hint,
!isHintEmpty, conf)
- if (buildSide.isDefined) {
- return Seq(
- joins.BroadcastHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide.get,
- condition,
- planLater(left),
- planLater(right)))
- }
-
- if (
- forceShuffledHashJoin &&
- !BackendsApiManager.getSparkPlanExecApiInstance.joinFallback(
- joinType,
- left.outputSet,
- right.outputSet,
- condition) &&
- !left.getTagValue(TAG).isDefined &&
- !right.getTagValue(TAG).isDefined
- ) {
- // Force use of ShuffledHashJoin in preference to SortMergeJoin. With
no respect to
- // conf setting "spark.sql.join.preferSortMergeJoin".
- val (leftBuildable, rightBuildable) =
- if (BackendsApiManager.getSettings.utilizeShuffledHashJoinHint()) {
- // Currently, ClickHouse backend can not support AQE, so it needs
to use join hint
- // to decide the build side, after supporting AQE, will remove
this.
- val leftHintEnabled = hintToShuffleHashJoinLeft(hint)
- val rightHintEnabled = hintToShuffleHashJoinRight(hint)
- val leftHintMergeEnabled =
hint.leftHint.exists(_.strategy.contains(SHUFFLE_MERGE))
- val rightHintMergeEnabled =
hint.rightHint.exists(_.strategy.contains(SHUFFLE_MERGE))
- if (leftHintEnabled || rightHintEnabled) {
- (leftHintEnabled, rightHintEnabled)
- } else if (leftHintMergeEnabled || rightHintMergeEnabled) {
- // hack: when set SHUFFLE_MERGE hint, it means that
- // it don't use this side as the build side
- (!leftHintMergeEnabled, !rightHintMergeEnabled)
- } else {
- (
-
BackendsApiManager.getSettings.supportHashBuildJoinTypeOnLeft(joinType),
-
BackendsApiManager.getSettings.supportHashBuildJoinTypeOnRight(joinType))
- }
- } else {
- (canBuildShuffledHashJoinLeft(joinType),
canBuildShuffledHashJoinRight(joinType))
- }
-
- if (!leftBuildable && !rightBuildable) {
- return Nil
- }
- val buildSide = if (!leftBuildable) {
- BuildRight
- } else if (!rightBuildable) {
- BuildLeft
- } else {
- getSmallerSide(left, right)
- }
-
- return Option(buildSide)
- .map {
- buildSide =>
- Seq(
- joins.ShuffledHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide,
- condition,
- planLater(left),
- planLater(right)))
- }
- .getOrElse(Nil)
- }
- Nil
- }
- }
-
- def existsMultiJoins(plan: LogicalPlan, count: Int = 0): Boolean = {
Review Comment:
My mistake, I will add it to the column rule later.
##########
gluten-core/src/main/scala/org/apache/gluten/extension/StrategyOverrides.scala:
##########
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You 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.gluten.extension
-
-import org.apache.gluten.{GlutenConfig, GlutenSparkExtensionsInjector}
-import org.apache.gluten.backendsapi.BackendsApiManager
-import org.apache.gluten.extension.columnar.TRANSFORM_UNSUPPORTED
-import org.apache.gluten.extension.columnar.TransformHints.TAG
-import org.apache.gluten.utils.LogicalPlanSelector
-
-import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy}
-import org.apache.spark.sql.catalyst.SQLConfHelper
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight,
JoinSelectionHelper}
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.{joins, JoinSelectionShim, SparkPlan}
-import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec,
LogicalQueryStage}
-import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
-
-object StrategyOverrides extends GlutenSparkExtensionsInjector {
- override def inject(extensions: SparkSessionExtensions): Unit = {
- extensions.injectPlannerStrategy(JoinSelectionOverrides)
- }
-}
-
-case class JoinSelectionOverrides(session: SparkSession)
- extends Strategy
- with JoinSelectionHelper
- with SQLConfHelper {
-
- private def isBroadcastStage(plan: LogicalPlan): Boolean = plan match {
- case LogicalQueryStage(_, _: BroadcastQueryStageExec) => true
- case _ => false
- }
-
- def extractEqualJoinKeyCondition(
- joinType: JoinType,
- leftKeys: Seq[Expression],
- rightKeys: Seq[Expression],
- condition: Option[Expression],
- left: LogicalPlan,
- right: LogicalPlan,
- hint: JoinHint,
- forceShuffledHashJoin: Boolean): Seq[SparkPlan] = {
- if (isBroadcastStage(left) || isBroadcastStage(right)) {
- val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight
- Seq(
- BroadcastHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide,
- condition,
- planLater(left),
- planLater(right)))
- } else {
- // Generate BHJ here, avoid to do match in `JoinSelection` again.
- val isHintEmpty = hint.leftHint.isEmpty && hint.rightHint.isEmpty
- val buildSide = getBroadcastBuildSide(left, right, joinType, hint,
!isHintEmpty, conf)
- if (buildSide.isDefined) {
- return Seq(
- joins.BroadcastHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide.get,
- condition,
- planLater(left),
- planLater(right)))
- }
-
- if (
- forceShuffledHashJoin &&
- !BackendsApiManager.getSparkPlanExecApiInstance.joinFallback(
- joinType,
- left.outputSet,
- right.outputSet,
- condition) &&
- !left.getTagValue(TAG).isDefined &&
- !right.getTagValue(TAG).isDefined
- ) {
- // Force use of ShuffledHashJoin in preference to SortMergeJoin. With
no respect to
- // conf setting "spark.sql.join.preferSortMergeJoin".
- val (leftBuildable, rightBuildable) =
- if (BackendsApiManager.getSettings.utilizeShuffledHashJoinHint()) {
- // Currently, ClickHouse backend can not support AQE, so it needs
to use join hint
- // to decide the build side, after supporting AQE, will remove
this.
- val leftHintEnabled = hintToShuffleHashJoinLeft(hint)
- val rightHintEnabled = hintToShuffleHashJoinRight(hint)
- val leftHintMergeEnabled =
hint.leftHint.exists(_.strategy.contains(SHUFFLE_MERGE))
- val rightHintMergeEnabled =
hint.rightHint.exists(_.strategy.contains(SHUFFLE_MERGE))
- if (leftHintEnabled || rightHintEnabled) {
- (leftHintEnabled, rightHintEnabled)
- } else if (leftHintMergeEnabled || rightHintMergeEnabled) {
- // hack: when set SHUFFLE_MERGE hint, it means that
- // it don't use this side as the build side
- (!leftHintMergeEnabled, !rightHintMergeEnabled)
- } else {
- (
-
BackendsApiManager.getSettings.supportHashBuildJoinTypeOnLeft(joinType),
-
BackendsApiManager.getSettings.supportHashBuildJoinTypeOnRight(joinType))
- }
- } else {
- (canBuildShuffledHashJoinLeft(joinType),
canBuildShuffledHashJoinRight(joinType))
- }
-
- if (!leftBuildable && !rightBuildable) {
- return Nil
- }
- val buildSide = if (!leftBuildable) {
- BuildRight
- } else if (!rightBuildable) {
- BuildLeft
- } else {
- getSmallerSide(left, right)
- }
-
- return Option(buildSide)
- .map {
- buildSide =>
- Seq(
- joins.ShuffledHashJoinExec(
- leftKeys,
- rightKeys,
- joinType,
- buildSide,
- condition,
- planLater(left),
- planLater(right)))
- }
- .getOrElse(Nil)
- }
- Nil
- }
- }
-
- def existsMultiJoins(plan: LogicalPlan, count: Int = 0): Boolean = {
Review Comment:
My mistake, I will add it to columnar later.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]