Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -1211,6 +1211,14 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val REMOVE_REDUNDANT_PROJECTS_ENABLED = buildConf("spark.sql.execution.removeRedundantProjects")
.internal()
.doc("Whether to remove redundant project exec node based on children's output and " +
"ordering requirement.")
.version("3.1.0")
.booleanConf
.createWithDefault(true)

val STATE_STORE_PROVIDER_CLASS =
buildConf("spark.sql.streaming.stateStore.providerClass")
.internal()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,7 @@ object QueryExecution {
CoalesceBucketsInJoin(sparkSession.sessionState.conf),
PlanDynamicPruningFilters(sparkSession),
PlanSubqueries(sparkSession),
RemoveRedundantProjects(sparkSession.sessionState.conf),
EnsureRequirements(sparkSession.sessionState.conf),
ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf,
sparkSession.sessionState.columnarRules),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
/*
* 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.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, PartialMerge}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase
import org.apache.spark.sql.execution.window.WindowExec
import org.apache.spark.sql.internal.SQLConf

/**
* Remove redundant ProjectExec node from the spark plan. A ProjectExec node is redundant when
* - It has the same output attributes and orders as its child's output and the ordering of
* the attributes is required.
* - It has the same output attributes as its child's output when attribute output ordering
* is not required.
* This rule needs to be a physical rule because project nodes are useful during logical
* optimization to prune data. During physical planning, redundant project nodes can be removed
* to simplify the query plan.
*/
case class RemoveRedundantProjects(conf: SQLConf) extends Rule[SparkPlan] {
def apply(plan: SparkPlan): SparkPlan = {
if (!conf.getConf(SQLConf.REMOVE_REDUNDANT_PROJECTS_ENABLED)) {
plan
} else {
removeProject(plan, true)
}
}

private def removeProject(plan: SparkPlan, requireOrdering: Boolean): SparkPlan = {
plan match {
case p @ ProjectExec(_, child) =>
if (isRedundant(p, child, requireOrdering)) {
val newPlan = removeProject(child, requireOrdering)
newPlan.setLogicalLink(child.logicalLink.get)
newPlan
} else {
p.mapChildren(removeProject(_, false))
}
case op: TakeOrderedAndProjectExec =>
op.mapChildren(removeProject(_, false))
case a: BaseAggregateExec =>
// BaseAggregateExec require specific column ordering when mode is Final or PartialMerge.
// See comments in BaseAggregateExec inputAttributes method.
val keepOrdering = a.aggregateExpressions
.exists(ae => ae.mode.equals(Final) || ae.mode.equals(PartialMerge))
a.mapChildren(removeProject(_, keepOrdering))
case g: GenerateExec => g.mapChildren(removeProject(_, false))
// JoinExec ordering requirement will inherit from its parent. If there is no ProjectExec in
// its ancestors, JoinExec should require output columns to be ordered.
case o => o.mapChildren(removeProject(_, requireOrdering))
}
}

/**
* Check if the nullability change is positive. It catches the case when the project output
* attribute is not nullable, but the child output attribute is nullable.
*/
private def checkNullability(output: Seq[Attribute], childOutput: Seq[Attribute]): Boolean =
output.zip(childOutput).forall { case (attr1, attr2) => attr1.nullable || !attr2.nullable }

private def isRedundant(
project: ProjectExec,
child: SparkPlan,
requireOrdering: Boolean): Boolean = {
child match {
// If a DataSourceV2ScanExec node does not support columnar, a ProjectExec node is required
// to convert the rows to UnsafeRow. See DataSourceV2Strategy for more details.
case d: DataSourceV2ScanExecBase if !d.supportsColumnar => false
case _ =>
if (requireOrdering) {
project.output.map(_.exprId.id) == child.output.map(_.exprId.id) &&
checkNullability(project.output, child.output)
} else {
val orderedProjectOutput = project.output.sortBy(_.exprId.id)
val orderedChildOutput = child.output.sortBy(_.exprId.id)
orderedProjectOutput.map(_.exprId.id) == orderedChildOutput.map(_.exprId.id) &&
checkNullability(orderedProjectOutput, orderedChildOutput)
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,12 +83,14 @@ case class AdaptiveSparkPlanExec(
)
}

@transient private val removeRedundantProjects = RemoveRedundantProjects(conf)
@transient private val ensureRequirements = EnsureRequirements(conf)

// A list of physical plan rules to be applied before creation of query stages. The physical
// plan should reach a final status of query stages (i.e., no more addition or removal of
// Exchange nodes) after running these rules.
private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq(
removeRedundantProjects,
ensureRequirements
) ++ context.session.sessionState.queryStagePrepRules

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.dynamicpruning

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions.{Alias, BindReferences, DynamicPruningExpression, DynamicPruningSubquery, Expression, ListQuery, Literal, PredicateHelper}
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeSeq, BindReferences, DynamicPruningExpression, DynamicPruningSubquery, Expression, ListQuery, Literal, PredicateHelper}
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight}
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode
Expand All @@ -40,8 +40,8 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession)
/**
* Identify the shape in which keys of a given plan are broadcasted.
*/
private def broadcastMode(keys: Seq[Expression], plan: LogicalPlan): BroadcastMode = {
val packedKeys = BindReferences.bindReferences(HashJoin.rewriteKeyExpr(keys), plan.output)
private def broadcastMode(keys: Seq[Expression], output: AttributeSeq): BroadcastMode = {
val packedKeys = BindReferences.bindReferences(HashJoin.rewriteKeyExpr(keys), output)
HashedRelationBroadcastMode(packedKeys)
}

Expand All @@ -67,8 +67,8 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession)
}.isDefined

if (canReuseExchange) {
val mode = broadcastMode(buildKeys, buildPlan)
val executedPlan = QueryExecution.prepareExecutedPlan(sparkSession, sparkPlan)
val mode = broadcastMode(buildKeys, executedPlan.output)
// plan a broadcast exchange of the build side of the join
val exchange = BroadcastExchangeExec(mode, executedPlan)
val name = s"dynamicpruning#${exprId.id}"
Expand Down
Loading