-
Notifications
You must be signed in to change notification settings - Fork 28.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[CARMEL-6113] Support BHJ fallback to SMJ in AQE (#1042)
* [CARMEL-6113] Support BHJ fallback to SMJ in AQE * fix code style * Add log if FallbackBroadcastStage takes effect
- Loading branch information
1 parent
20fb439
commit e12e8d3
Showing
9 changed files
with
371 additions
and
22 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
26 changes: 26 additions & 0 deletions
26
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlException.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
/* | ||
* 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.spark.sql.execution | ||
|
||
import org.apache.spark.SparkException | ||
|
||
/** | ||
* Exception thrown when the table cannot be broadcast. | ||
*/ | ||
private[spark] case class CannotBroadcastTableException(message: String) | ||
extends SparkException(message) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/FallbackBroadcastStage.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
/* | ||
* 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.spark.sql.execution.adaptive | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys | ||
import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, Join, LogicalPlan, NO_BROADCAST_HASH} | ||
|
||
/** | ||
* This fallback handler detects stage that has been proved cannot be broadcast and | ||
* adds a no-broadcast-hash-join hint to avoid it being broadcast. | ||
*/ | ||
|
||
object FallbackBroadcastStage extends FallbackStageHandler with Logging { | ||
override def apply(logicalPlan: LogicalPlan, stage: LogicalQueryStage): (LogicalPlan, Int) = { | ||
var effectiveCount = 0 | ||
val originLogicalQueryStage = stage match { | ||
case LogicalQueryStage(l@LogicalQueryStage(_, _), _) => l | ||
case _ => stage | ||
} | ||
val newLogicalPlan = logicalPlan.transformDown { | ||
// case j@Join(left, _, _, _, hint) | ||
case j @ ExtractEquiJoinKeys(_, _, _, _, left, _, hint) if left.sameResult(stage) || | ||
left.sameResult(originLogicalQueryStage) => | ||
var newHint = hint | ||
if (!hint.leftHint.exists(_.strategy.isDefined)) { | ||
newHint = newHint.copy(leftHint = | ||
Some(hint.leftHint.getOrElse(HintInfo()).copy(strategy = Some(NO_BROADCAST_HASH)))) | ||
} | ||
|
||
if (newHint.ne(hint)) { | ||
effectiveCount = effectiveCount + 1 | ||
|
||
logInfo(s"FallbackBroadcastStage takes effect for " + | ||
s"logicalPlan: $logicalPlan " + | ||
s"stage: $stage") | ||
j.copy(hint = newHint, left = if (left.sameResult(stage)) { | ||
stage.logicalPlan | ||
} else originLogicalQueryStage.logicalPlan) | ||
} else { | ||
j | ||
} | ||
|
||
// case j@Join(_, right, _, _, hint) | ||
case j @ ExtractEquiJoinKeys(_, _, _, _, _, right, hint) if right.sameResult(stage) || | ||
right.sameResult(originLogicalQueryStage) => | ||
var newHint = hint | ||
if (!hint.rightHint.exists(_.strategy.isDefined)) { | ||
newHint = newHint.copy(rightHint = | ||
Some(hint.rightHint.getOrElse(HintInfo()).copy(strategy = Some(NO_BROADCAST_HASH)))) | ||
} | ||
|
||
if (newHint.ne(hint)) { | ||
effectiveCount = effectiveCount + 1 | ||
|
||
logInfo(s"FallbackBroadcastStage takes effect for " + | ||
s"logicalPlan: $logicalPlan " + | ||
s"stage: $stage") | ||
j.copy(hint = newHint, right = if (right.sameResult(stage)) { | ||
stage.logicalPlan | ||
} else originLogicalQueryStage.logicalPlan) | ||
} else { | ||
j | ||
} | ||
} | ||
(newLogicalPlan, effectiveCount) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.