Skip to content

Commit

Permalink
[SPARK-34168][SQL] Support DPP in AQE when the join is Broadcast hash…
Browse files Browse the repository at this point in the history
… join at the beginning

This PR is to enable AQE and DPP when the join is broadcast hash join at the beginning, which can benefit the performance improvement from DPP and AQE at the same time. This PR will make use of the result of build side and then insert the DPP filter into the probe side.

Improve performance

No

adding new ut

Closes apache#31258 from JkSelf/supportDPP1.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
JkSelf authored and shipenglei committed Oct 15, 2021
1 parent ad0963d commit 925ee15
Show file tree
Hide file tree
Showing 11 changed files with 198 additions and 38 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._

/**
* Similar to [[SubqueryBroadcastExec]], this node is used to store the
* initial physical plan of DPP subquery filters when enabling both AQE and DPP.
* It is intermediate physical plan and not executable.
* After the build side is executed, this node will be replaced with the
* [[SubqueryBroadcastExec]] and the child will be optimized with the ReusedExchange
* from the build side.
*/
case class SubqueryAdaptiveBroadcastExec(
name: String,
index: Int,
buildKeys: Seq[Expression],
child: SparkPlan) extends BaseSubqueryExec with UnaryExecNode {

protected override def doExecute(): RDD[InternalRow] = {
throw new UnsupportedOperationException(
"SubqueryAdaptiveBroadcastExec does not support the execute() code path.")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,7 @@ case class AdaptiveSparkPlanExec(
// A list of physical optimizer rules to be applied to a new stage before its execution. These
// optimizations should be stage-independent.
@transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq(
PlanAdaptiveDynamicPruningFilters(context.stageCache),
ReuseAdaptiveSubquery(context.subqueryCache),
CoalesceShufflePartitions(context.session),
// The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs'
Expand Down Expand Up @@ -463,7 +464,7 @@ case class AdaptiveSparkPlanExec(
throw new IllegalStateException(
"Custom columnar rules cannot transform shuffle node to something else.")
}
ShuffleQueryStageExec(currentStageId, newShuffle)
ShuffleQueryStageExec(currentStageId, newShuffle, s.canonicalized)
case b: BroadcastExchangeLike =>
val newBroadcast = applyPhysicalRules(
b.withNewChildren(Seq(optimizedPlan)),
Expand All @@ -473,7 +474,7 @@ case class AdaptiveSparkPlanExec(
throw new IllegalStateException(
"Custom columnar rules cannot transform broadcast node to something else.")
}
BroadcastQueryStageExec(currentStageId, newBroadcast)
BroadcastQueryStageExec(currentStageId, newBroadcast, b.canonicalized)
}
currentStageId += 1
setLogicalLinkForNewQueryStage(queryStage, e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,9 @@ case class CustomShuffleReaderExec private(
partitionSpecs.map(_.asInstanceOf[PartialMapperPartitionSpec].mapIndex).toSet.size ==
partitionSpecs.length) {
child match {
case ShuffleQueryStageExec(_, s: ShuffleExchangeLike) =>
case ShuffleQueryStageExec(_, s: ShuffleExchangeLike, _) =>
s.child.outputPartitioning
case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeLike)) =>
case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeLike), _) =>
s.child.outputPartitioning match {
case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning]
case other => other
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.adaptive
import scala.collection.mutable

import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ListQuery, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.{ListQuery, SubqueryExpression}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution
import org.apache.spark.sql.catalyst.rules.Rule
Expand Down Expand Up @@ -98,10 +98,8 @@ case class InsertAdaptiveSparkPlan(
}

private def supportAdaptive(plan: SparkPlan): Boolean = {
// TODO migrate dynamic-partition-pruning onto adaptive execution.
sanityCheck(plan) &&
!plan.logicalLink.exists(_.isStreaming) &&
!plan.expressions.exists(_.find(_.isInstanceOf[DynamicPruningSubquery]).isDefined) &&
plan.children.forall(supportAdaptive)
}

Expand All @@ -113,8 +111,8 @@ case class InsertAdaptiveSparkPlan(
* For each sub-query, generate the adaptive execution plan for each sub-query by applying this
* rule, or reuse the execution plan from another sub-query of the same semantics if possible.
*/
private def buildSubqueryMap(plan: SparkPlan): Map[Long, SubqueryExec] = {
val subqueryMap = mutable.HashMap.empty[Long, SubqueryExec]
private def buildSubqueryMap(plan: SparkPlan): Map[Long, BaseSubqueryExec] = {
val subqueryMap = mutable.HashMap.empty[Long, BaseSubqueryExec]
plan.foreach(_.expressions.foreach(_.foreach {
case expressions.ScalarSubquery(p, _, exprId)
if !subqueryMap.contains(exprId.id) =>
Expand All @@ -129,6 +127,16 @@ case class InsertAdaptiveSparkPlan(
verifyAdaptivePlan(executedPlan, query)
val subquery = SubqueryExec(s"subquery#${exprId.id}", executedPlan)
subqueryMap.put(exprId.id, subquery)
case expressions.DynamicPruningSubquery(value, buildPlan,
buildKeys, broadcastKeyIndex, onlyInBroadcast, exprId)
if !subqueryMap.contains(exprId.id) =>
val executedPlan = compileSubquery(buildPlan)
verifyAdaptivePlan(executedPlan, buildPlan)

val name = s"dynamicpruning#${exprId.id}"
val subquery = SubqueryAdaptiveBroadcastExec(
name, broadcastKeyIndex, buildKeys, executedPlan)
subqueryMap.put(exprId.id, subquery)
case _ =>
}))

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.execution.adaptive

import scala.collection.concurrent.TrieMap

import org.apache.spark.sql.catalyst.expressions.{BindReferences, DynamicPruningExpression, Literal}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec
import org.apache.spark.sql.execution.joins.{HashedRelationBroadcastMode, HashJoin}

/**
* A rule to insert dynamic pruning predicates in order to reuse the results of broadcast.
*/
case class PlanAdaptiveDynamicPruningFilters(
stageCache: TrieMap[SparkPlan, QueryStageExec]) extends Rule[SparkPlan] {
def apply(plan: SparkPlan): SparkPlan = {
if (!conf.dynamicPartitionPruningEnabled) {
return plan
}

plan transformAllExpressions {
case DynamicPruningExpression(InSubqueryExec(
value, SubqueryAdaptiveBroadcastExec(name, index, buildKeys,
adaptivePlan: AdaptiveSparkPlanExec), exprId, _)) =>
val packedKeys = BindReferences.bindReferences(
HashJoin.rewriteKeyExpr(buildKeys), adaptivePlan.executedPlan.output)
val mode = HashedRelationBroadcastMode(packedKeys)
// plan a broadcast exchange of the build side of the join
val exchange = BroadcastExchangeExec(mode, adaptivePlan.executedPlan)
val existingStage = stageCache.get(exchange.canonicalized)
if (existingStage.nonEmpty && conf.exchangeReuseEnabled) {
val name = s"dynamicpruning#${exprId.id}"
val reuseQueryStage = existingStage.get.newReuseInstance(0, exchange.output)
val broadcastValues =
SubqueryBroadcastExec(name, index, buildKeys, reuseQueryStage)
DynamicPruningExpression(InSubqueryExec(value, broadcastValues, exprId))
} else {
DynamicPruningExpression(Literal.TrueLiteral)
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,13 @@
package org.apache.spark.sql.execution.adaptive

import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, ListQuery, Literal}
import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, DynamicPruningExpression, ListQuery, Literal}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution
import org.apache.spark.sql.execution.{InSubqueryExec, SparkPlan, SubqueryExec}
import org.apache.spark.sql.execution.{BaseSubqueryExec, InSubqueryExec, SparkPlan}

case class PlanAdaptiveSubqueries(subqueryMap: Map[Long, SubqueryExec]) extends Rule[SparkPlan] {
case class PlanAdaptiveSubqueries(
subqueryMap: Map[Long, BaseSubqueryExec]) extends Rule[SparkPlan] {

def apply(plan: SparkPlan): SparkPlan = {
plan.transformAllExpressions {
Expand All @@ -40,6 +41,8 @@ case class PlanAdaptiveSubqueries(subqueryMap: Map[Long, SubqueryExec]) extends
)
}
InSubqueryExec(expr, subqueryMap(exprId.id), exprId)
case expressions.DynamicPruningSubquery(value, _, _, _, _, exprId) =>
DynamicPruningExpression(InSubqueryExec(value, subqueryMap(exprId.id), exprId))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,11 @@ abstract class QueryStageExec extends LeafExecNode {
*/
val plan: SparkPlan

/**
* The canonicalized plan before applying query stage optimizer rules.
*/
val _canonicalized: SparkPlan

/**
* Materialize this query stage, to prepare for the execution, like submitting map stages,
* broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this
Expand Down Expand Up @@ -116,7 +121,7 @@ abstract class QueryStageExec extends LeafExecNode {
override def supportsColumnar: Boolean = plan.supportsColumnar
protected override def doExecuteColumnar(): RDD[ColumnarBatch] = plan.executeColumnar()
override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast()
override def doCanonicalize(): SparkPlan = plan.canonicalized
override def doCanonicalize(): SparkPlan = _canonicalized

protected override def stringArgs: Iterator[Any] = Iterator.single(id)

Expand Down Expand Up @@ -146,10 +151,15 @@ abstract class QueryStageExec extends LeafExecNode {

/**
* A shuffle query stage whose child is a [[ShuffleExchangeLike]] or [[ReusedExchangeExec]].
*
* @param id the query stage id.
* @param plan the underlying plan.
* @param _canonicalized the canonicalized plan before applying query stage optimizer rules.
*/
case class ShuffleQueryStageExec(
override val id: Int,
override val plan: SparkPlan) extends QueryStageExec {
override val plan: SparkPlan,
override val _canonicalized: SparkPlan) extends QueryStageExec {

@transient val shuffle = plan match {
case s: ShuffleExchangeLike => s
Expand All @@ -165,7 +175,8 @@ case class ShuffleQueryStageExec(
override def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec = {
val reuse = ShuffleQueryStageExec(
newStageId,
ReusedExchangeExec(newOutput, shuffle))
ReusedExchangeExec(newOutput, shuffle),
_canonicalized)
reuse._resultOption = this._resultOption
reuse
}
Expand Down Expand Up @@ -194,10 +205,15 @@ case class ShuffleQueryStageExec(

/**
* A broadcast query stage whose child is a [[BroadcastExchangeLike]] or [[ReusedExchangeExec]].
*
* @param id the query stage id.
* @param plan the underlying plan.
* @param _canonicalized the canonicalized plan before applying query stage optimizer rules.
*/
case class BroadcastQueryStageExec(
override val id: Int,
override val plan: SparkPlan) extends QueryStageExec {
override val plan: SparkPlan,
override val _canonicalized: SparkPlan) extends QueryStageExec {

@transient val broadcast = plan match {
case b: BroadcastExchangeLike => b
Expand Down Expand Up @@ -229,7 +245,8 @@ case class BroadcastQueryStageExec(
override def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec = {
val reuse = BroadcastQueryStageExec(
newStageId,
ReusedExchangeExec(newOutput, broadcast))
ReusedExchangeExec(newOutput, broadcast),
_canonicalized)
reuse._resultOption = this._resultOption
reuse
}
Expand Down
Loading

0 comments on commit 925ee15

Please sign in to comment.