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 7 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
@@ -0,0 +1,59 @@
/*
* 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.catalyst.optimizer

import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule

/**
* Replaces [[ResolvedHint]] operators from the plan. Move the [[HintInfo]] to associated [[Join]]
* operators, otherwise remove it if no [[Join]] operator is matched.
*/
object EliminateResolvedHint extends Rule[LogicalPlan] {
Copy link
Contributor

Choose a reason for hiding this comment

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

do we have to run it at the beginning of optimizer? Can we run it at the end of analyzer?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's because of the Dataset interface. The ResolvedHint of the join children nodes would have been gone by the time we construct a join node.

// This is also called in the beginning of the optimization phase, and as a result
// is using transformUp rather than resolveOperators.
def apply(plan: LogicalPlan): LogicalPlan = {
val pulledUp = plan transformUp {
case j: Join =>
val leftHint = mergeHints(collectHints(j.left))
val rightHint = mergeHints(collectHints(j.right))
j.copy(hint = JoinHint(leftHint, rightHint))
}
pulledUp.transform {
case h: ResolvedHint => h.child
}
}

private def mergeHints(hints: Seq[HintInfo]): Option[HintInfo] = {
hints.reduceOption((h1, h2) => HintInfo(
broadcast = h1.broadcast || h2.broadcast))
}

private def collectHints(plan: LogicalPlan): Seq[HintInfo] = {
plan match {
case h: ResolvedHint => collectHints(h.child) :+ h.hints
case u: UnaryNode => collectHints(u.child)
// TODO revisit this logic:
// except and intersect are semi/anti-joins which won't return more data then
// their left argument, so the broadcast hint should be propagated here
case i: Intersect => collectHints(i.left)
case e: Except => collectHints(e.left)
case _ => Seq.empty
}
}
}
Loading