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

[SPARK-26065][SQL] Change query hint from a LogicalPlan to a field #23036

Closed
wants to merge 8 commits into from
Closed
Show file tree
Hide file tree
Changes from 6 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 @@ -943,7 +943,7 @@ class Analyzer(
failAnalysis("Invalid usage of '*' in explode/json_tuple/UDTF")

// To resolve duplicate expression IDs for Join and Intersect
case j @ Join(left, right, _, _) if !j.duplicateResolved =>
case j @ Join(left, right, _, _, _) if !j.duplicateResolved =>
j.copy(right = dedupRight(left, right))
case i @ Intersect(left, right, _) if !i.duplicateResolved =>
i.copy(right = dedupRight(left, right))
Expand Down Expand Up @@ -2249,13 +2249,14 @@ class Analyzer(
*/
object ResolveNaturalAndUsingJoin extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
case j @ Join(left, right, UsingJoin(joinType, usingCols), _)
case j @ Join(left, right, UsingJoin(joinType, usingCols), _, hint)
if left.resolved && right.resolved && j.duplicateResolved =>
commonNaturalJoinProcessing(left, right, joinType, usingCols, None)
case j @ Join(left, right, NaturalJoin(joinType), condition) if j.resolvedExceptNatural =>
commonNaturalJoinProcessing(left, right, joinType, usingCols, None, hint)
case j @ Join(left, right, NaturalJoin(joinType), condition, hint)
if j.resolvedExceptNatural =>
// find common column names from both sides
val joinNames = left.output.map(_.name).intersect(right.output.map(_.name))
commonNaturalJoinProcessing(left, right, joinType, joinNames, condition)
commonNaturalJoinProcessing(left, right, joinType, joinNames, condition, hint)
}
}

Expand Down Expand Up @@ -2360,7 +2361,8 @@ class Analyzer(
right: LogicalPlan,
joinType: JoinType,
joinNames: Seq[String],
condition: Option[Expression]) = {
condition: Option[Expression],
hint: JoinHint) = {
val leftKeys = joinNames.map { keyName =>
left.output.find(attr => resolver(attr.name, keyName)).getOrElse {
throw new AnalysisException(s"USING column `$keyName` cannot be resolved on the left " +
Expand Down Expand Up @@ -2401,7 +2403,7 @@ class Analyzer(
sys.error("Unsupported natural join type " + joinType)
}
// use Project to trim unnecessary fields
Project(projectList, Join(left, right, joinType, newCondition))
Project(projectList, Join(left, right, joinType, newCondition, hint))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ trait CheckAnalysis extends PredicateHelper {
failAnalysis("Null-aware predicate sub-queries cannot be used in nested " +
s"conditions: $condition")

case j @ Join(_, _, _, Some(condition)) if condition.dataType != BooleanType =>
case j @ Join(_, _, _, Some(condition), _) if condition.dataType != BooleanType =>
failAnalysis(
s"join condition '${condition.sql}' " +
s"of type ${condition.dataType.catalogString} is not a boolean.")
Expand Down Expand Up @@ -609,7 +609,7 @@ trait CheckAnalysis extends PredicateHelper {
failOnNonEqualCorrelatedPredicate(foundNonEqualCorrelatedPred, a)

// Join can host correlated expressions.
case j @ Join(left, right, joinType, _) =>
case j @ Join(left, right, joinType, _, _) =>
joinType match {
// Inner join, like Filter, can be anywhere.
case _: InnerLike =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
*/
def isWatermarkInJoinKeys(plan: LogicalPlan): Boolean = {
plan match {
case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _) =>
case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _, _) =>
(leftKeys ++ rightKeys).exists {
case a: AttributeReference => a.metadata.contains(EventTimeWatermark.delayKey)
case _ => false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,7 +229,7 @@ object UnsupportedOperationChecker {
throwError("dropDuplicates is not supported after aggregation on a " +
"streaming DataFrame/Dataset")

case Join(left, right, joinType, condition) =>
case Join(left, right, joinType, condition, _) =>

joinType match {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,7 @@ package object dsl {
otherPlan: LogicalPlan,
joinType: JoinType = Inner,
condition: Option[Expression] = None): LogicalPlan =
Join(logicalPlan, otherPlan, joinType, condition)
Join(logicalPlan, otherPlan, joinType, condition, JoinHint.NONE)

def cogroup[Key: Encoder, Left: Encoder, Right: Encoder, Result: Encoder](
otherPlan: LogicalPlan,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import scala.collection.mutable
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper}
import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, JoinType}
import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.internal.SQLConf

Expand All @@ -31,6 +31,40 @@ import org.apache.spark.sql.internal.SQLConf
* Cost-based join reorder.
* We may have several join reorder algorithms in the future. This class is the entry of these
* algorithms, and chooses which one to use.
*
* Note that join strategy hints, e.g. the broadcast hint, do not interfere with the reordering.
* Such hints will be applied on the equivalent counterparts (i.e., join between the same relations
* regardless of the join order) of the original nodes after reordering.
* For example, the plan before reordering is like:
*
* Join
* / \
* Hint1 t4
* /
* Join
* / \
* Join t3
* / \
* Hint2 t2
* /
* t1
*
* The original join order as illustrated above is "((t1 JOIN t2) JOIN t3) JOIN t4", and after
* reordering, the new join order is "((t1 JOIN t3) JOIN t2) JOIN t4", so the new plan will be like:
*
* Join
* / \
* Hint1 t4
* /
* Join
* / \
* Join t2
* / \
* t1 t3
*
* "Hint1" is applied on "(t1 JOIN t3) JOIN t2" as it is equivalent to the original hinted node,
* "(t1 JOIN t2) JOIN t3"; while "Hint2" has disappeared from the new plan since there is no
* equivalent node to "t1 JOIN t2".
*/
object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper {

Expand All @@ -40,24 +74,30 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper {
if (!conf.cboEnabled || !conf.joinReorderEnabled) {
plan
} else {
// Use a map to track the hints on the join items.
val hintMap = new mutable.HashMap[AttributeSet, HintInfo]
val result = plan transformDown {
// Start reordering with a joinable item, which is an InnerLike join with conditions.
case j @ Join(_, _, _: InnerLike, Some(cond)) =>
reorder(j, j.output)
case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond)))
case j @ Join(_, _, _: InnerLike, Some(cond), _) =>
reorder(j, j.output, hintMap)
case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond), _))
if projectList.forall(_.isInstanceOf[Attribute]) =>
reorder(p, p.output)
reorder(p, p.output, hintMap)
}

// After reordering is finished, convert OrderedJoin back to Join
result transformDown {
case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond)
// After reordering is finished, convert OrderedJoin back to Join.
result transform {
case OrderedJoin(left, right, jt, cond) =>
Join(left, right, jt, cond,
JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet)))
maryannxue marked this conversation as resolved.
Show resolved Hide resolved
}
}
}

private def reorder(plan: LogicalPlan, output: Seq[Attribute]): LogicalPlan = {
val (items, conditions) = extractInnerJoins(plan)
private def reorder(
plan: LogicalPlan,
output: Seq[Attribute],
hintMap: mutable.HashMap[AttributeSet, HintInfo]): LogicalPlan = {
val (items, conditions) = extractInnerJoins(plan, hintMap)
val result =
// Do reordering if the number of items is appropriate and join conditions exist.
// We also need to check if costs of all items can be evaluated.
Expand All @@ -75,27 +115,31 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper {
* Extracts items of consecutive inner joins and join conditions.
* This method works for bushy trees and left/right deep trees.
*/
private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = {
private def extractInnerJoins(
plan: LogicalPlan,
hintMap: mutable.HashMap[AttributeSet, HintInfo]): (Seq[LogicalPlan], Set[Expression]) = {
plan match {
case Join(left, right, _: InnerLike, Some(cond)) =>
val (leftPlans, leftConditions) = extractInnerJoins(left)
val (rightPlans, rightConditions) = extractInnerJoins(right)
case Join(left, right, _: InnerLike, Some(cond), hint) =>
hint.leftHint.foreach(hintMap.put(left.outputSet, _))
hint.rightHint.foreach(hintMap.put(right.outputSet, _))
val (leftPlans, leftConditions) = extractInnerJoins(left, hintMap)
val (rightPlans, rightConditions) = extractInnerJoins(right, hintMap)
(leftPlans ++ rightPlans, splitConjunctivePredicates(cond).toSet ++
leftConditions ++ rightConditions)
case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond)))
case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), _))
if projectList.forall(_.isInstanceOf[Attribute]) =>
extractInnerJoins(j)
extractInnerJoins(j, hintMap)
case _ =>
(Seq(plan), Set())
}
}

private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match {
case j @ Join(left, right, jt: InnerLike, Some(cond)) =>
case j @ Join(left, right, jt: InnerLike, Some(cond), _) =>
val replacedLeft = replaceWithOrderedJoin(left)
val replacedRight = replaceWithOrderedJoin(right)
OrderedJoin(replacedLeft, replacedRight, jt, Some(cond))
case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond))) =>
case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), _)) =>
p.copy(child = replaceWithOrderedJoin(j))
case _ =>
plan
Expand Down Expand Up @@ -295,7 +339,7 @@ object JoinReorderDP extends PredicateHelper with Logging {
} else {
(otherPlan, onePlan)
}
val newJoin = Join(left, right, Inner, joinConds.reduceOption(And))
val newJoin = Join(left, right, Inner, joinConds.reduceOption(And), JoinHint.NONE)
val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds
val remainingConds = conditions -- collectedJoinConds
val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
// However, because we also use the analyzer to canonicalized queries (for view definition),
// we do not eliminate subqueries or compute current time in the analyzer.
Batch("Finish Analysis", Once,
EliminateResolvedHint,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also add it to nonExcludableRules

EliminateSubqueryAliases,
EliminateView,
ReplaceExpressions,
Expand Down Expand Up @@ -356,7 +357,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
// not allowed to use the same attributes. We use a blacklist to prevent us from creating a
// situation in which this happens; the rule will only remove an alias if its child
// attribute is not on the black list.
case Join(left, right, joinType, condition) =>
case Join(left, right, joinType, condition, hint) =>
val newLeft = removeRedundantAliases(left, blacklist ++ right.outputSet)
val newRight = removeRedundantAliases(right, blacklist ++ newLeft.outputSet)
val mapping = AttributeMap(
Expand All @@ -365,7 +366,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
val newCondition = condition.map(_.transform {
case a: Attribute => mapping.getOrElse(a, a)
})
Join(newLeft, newRight, joinType, newCondition)
Join(newLeft, newRight, joinType, newCondition, hint)

case _ =>
// Remove redundant aliases in the subtree(s).
Expand Down Expand Up @@ -460,7 +461,7 @@ object LimitPushDown extends Rule[LogicalPlan] {
// on both sides if it is applied multiple times. Therefore:
// - If one side is already limited, stack another limit on top if the new limit is smaller.
// The redundant limit will be collapsed by the CombineLimits rule.
case LocalLimit(exp, join @ Join(left, right, joinType, _)) =>
case LocalLimit(exp, join @ Join(left, right, joinType, _, _)) =>
val newJoin = joinType match {
case RightOuter => join.copy(right = maybePushLocalLimit(exp, right))
case LeftOuter => join.copy(left = maybePushLocalLimit(exp, left))
Expand Down Expand Up @@ -578,7 +579,7 @@ object ColumnPruning extends Rule[LogicalPlan] {
p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices))

// Eliminate unneeded attributes from right side of a Left Existence Join.
case j @ Join(_, right, LeftExistence(_), _) =>
case j @ Join(_, right, LeftExistence(_), _, _) =>
j.copy(right = prunedChild(right, j.references))

// all the columns will be used to compare, so we can't prune them
Expand Down Expand Up @@ -792,7 +793,7 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan]
filter
}

case join @ Join(left, right, joinType, conditionOpt) =>
case join @ Join(left, right, joinType, conditionOpt, _) =>
joinType match {
// For inner join, we can infer additional filters for both sides. LeftSemi is kind of an
// inner join, it just drops the right side in the final output.
Expand Down Expand Up @@ -919,7 +920,6 @@ object RemoveRedundantSorts extends Rule[LogicalPlan] {
def canEliminateSort(plan: LogicalPlan): Boolean = plan match {
case p: Project => p.projectList.forall(_.deterministic)
case f: Filter => f.condition.deterministic
case _: ResolvedHint => true
case _ => false
}
}
Expand Down Expand Up @@ -1094,7 +1094,6 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper {
// Note that some operators (e.g. project, aggregate, union) are being handled separately
// (earlier in this rule).
case _: AppendColumns => true
case _: ResolvedHint => true
case _: Distinct => true
case _: Generate => true
case _: Pivot => true
Expand Down Expand Up @@ -1179,7 +1178,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {

def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// push the where condition down into join filter
case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) =>
case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) =>
val (leftFilterConditions, rightFilterConditions, commonFilterCondition) =
split(splitConjunctivePredicates(filterCondition), left, right)
joinType match {
Expand All @@ -1193,7 +1192,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {
commonFilterCondition.partition(canEvaluateWithinJoin)
val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And)

val join = Join(newLeft, newRight, joinType, newJoinCond)
val join = Join(newLeft, newRight, joinType, newJoinCond, hint)
if (others.nonEmpty) {
Filter(others.reduceLeft(And), join)
} else {
Expand All @@ -1205,7 +1204,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {
val newRight = rightFilterConditions.
reduceLeftOption(And).map(Filter(_, right)).getOrElse(right)
val newJoinCond = joinCondition
val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond)
val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond, hint)

(leftFilterConditions ++ commonFilterCondition).
reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin)
Expand All @@ -1215,7 +1214,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {
reduceLeftOption(And).map(Filter(_, left)).getOrElse(left)
val newRight = right
val newJoinCond = joinCondition
val newJoin = Join(newLeft, newRight, joinType, newJoinCond)
val newJoin = Join(newLeft, newRight, joinType, newJoinCond, hint)

(rightFilterConditions ++ commonFilterCondition).
reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin)
Expand All @@ -1225,7 +1224,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {
}

// push down the join filter into sub query scanning if applicable
case j @ Join(left, right, joinType, joinCondition) =>
case j @ Join(left, right, joinType, joinCondition, hint) =>
val (leftJoinConditions, rightJoinConditions, commonJoinCondition) =
split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right)

Expand All @@ -1238,23 +1237,23 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper {
reduceLeftOption(And).map(Filter(_, right)).getOrElse(right)
val newJoinCond = commonJoinCondition.reduceLeftOption(And)

Join(newLeft, newRight, joinType, newJoinCond)
Join(newLeft, newRight, joinType, newJoinCond, hint)
case RightOuter =>
// push down the left side only join filter for left side sub query
val newLeft = leftJoinConditions.
reduceLeftOption(And).map(Filter(_, left)).getOrElse(left)
val newRight = right
val newJoinCond = (rightJoinConditions ++ commonJoinCondition).reduceLeftOption(And)

Join(newLeft, newRight, RightOuter, newJoinCond)
Join(newLeft, newRight, RightOuter, newJoinCond, hint)
case LeftOuter | LeftAnti | ExistenceJoin(_) =>
// push down the right side only join filter for right sub query
val newLeft = left
val newRight = rightJoinConditions.
reduceLeftOption(And).map(Filter(_, right)).getOrElse(right)
val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And)

Join(newLeft, newRight, joinType, newJoinCond)
Join(newLeft, newRight, joinType, newJoinCond, hint)
case FullOuter => j
case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node")
case UsingJoin(_, _) => sys.error("Untransformed Using join node")
Expand Down Expand Up @@ -1310,7 +1309,7 @@ object CheckCartesianProducts extends Rule[LogicalPlan] with PredicateHelper {
if (SQLConf.get.crossJoinEnabled) {
plan
} else plan transform {
case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _)
case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _, _)
if isCartesianProduct(j) =>
throw new AnalysisException(
s"""Detected implicit cartesian product for ${j.joinType.sql} join between logical plans
Expand Down Expand Up @@ -1449,7 +1448,7 @@ object ReplaceIntersectWithSemiJoin extends Rule[LogicalPlan] {
case Intersect(left, right, false) =>
assert(left.output.size == right.output.size)
val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) }
Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And)))
Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And), JoinHint.NONE))
}
}

Expand All @@ -1470,7 +1469,7 @@ object ReplaceExceptWithAntiJoin extends Rule[LogicalPlan] {
case Except(left, right, false) =>
assert(left.output.size == right.output.size)
val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) }
Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And)))
Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And), JoinHint.NONE))
}
}

Expand Down
Loading