-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-16849][SQL] Improve subquery execution by deduplicating the subqueries with the same results #14452
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
Closed
Closed
[SPARK-16849][SQL] Improve subquery execution by deduplicating the subqueries with the same results #14452
Changes from all commits
Commits
Show all changes
37 commits
Select commit
Hold shift + click to select a range
6bfdb32
Dedup common subqueries.
viirya 229ae31
Add logical node CommonSubqueryAlias to represent subquery in CTE.
viirya 734b050
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya 70af8d6
Update with the change of access privileges for metrics.
viirya ba11d34
Use optimized plan of common subquery.
viirya 4680994
Add test case for self-join.
viirya 11815da
Synchonized on computed output of common subquery.
viirya 5282de7
Fix some bugs.
viirya bdb6e84
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya 9fe1fbe
Reuse all subqueries, instead of CTE subqueries.
viirya a14459c
Can not use optimizedPlan when replacing common subquery because we n…
viirya 0d5eea7
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya 9d7a15d
Optimization of common subqueries.
viirya 5ef961d
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya f3aa0aa
Fix a bug and add test.
viirya e094c14
Pushdowned projection list should use attributes.
viirya 50f0e19
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya f0954cd
Fix a bug in filter pushdown.
viirya 6a8011b
Fix filter pushdown.
viirya 6cb40f1
Skip duplicating table scan nodes.
viirya 6a08486
Don't do nested common subquery deduplication.
viirya 3337303
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya c6d987f
Change to common subquery query plans to make the query tree more rea…
viirya 38c57e8
Add executed plan into otherCopyArgs.
viirya 49eefb0
Fix the issue.
viirya 79c45ac
Continue fixing query plan string.
viirya 5729bb2
Improve filter pushdown.
viirya e9b0952
Fix filter pushdown again.
viirya e1d5b05
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya e1d050f
Fix subquery execution.
viirya 6d79beb
Fix the cnf.
viirya bc70354
Fix predicate explosion.
viirya 23e2dc8
Deal with no pushdown predicate.
viirya cebfbf5
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya 9faf90a
Merge remote-tracking branch 'upstream/master' into single-exec-subquery
viirya f153c12
Cache the rdd of common subquery.
viirya aeba1c3
Cleaning some codes.
viirya File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import scala.annotation.tailrec | |
| import scala.collection.immutable.HashSet | ||
| import scala.collection.mutable | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.collection.mutable.HashMap | ||
|
|
||
| import org.apache.spark.api.java.function.FilterFunction | ||
| import org.apache.spark.sql.AnalysisException | ||
|
|
@@ -51,7 +52,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) | |
| // 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, | ||
| EliminateSubqueryAliases, | ||
| EliminateNonDuplicatedSubqueryAliases, | ||
| ReplaceExpressions, | ||
| ComputeCurrentTime, | ||
| GetCurrentDatabase(sessionCatalog), | ||
|
|
@@ -92,6 +93,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) | |
| CombineFilters, | ||
| CombineLimits, | ||
| CombineUnions, | ||
| // Pushdown Filters again after combination | ||
| PushDownPredicate, | ||
| // Constant folding and strength reduction | ||
| NullPropagation, | ||
| FoldablePropagation, | ||
|
|
@@ -421,6 +424,9 @@ object ColumnPruning extends Rule[LogicalPlan] { | |
| // Can't prune the columns on LeafNode | ||
| case p @ Project(_, _: LeafNode) => p | ||
|
|
||
| // Don't prune the columns on common subquery. | ||
| case p @ Project(_, SubqueryAlias(_, _, _, true)) => p | ||
|
|
||
| // for all other logical plans that inherits the output from it's children | ||
| case p @ Project(_, child) => | ||
| val required = child.references ++ p.references | ||
|
|
@@ -1167,3 +1173,211 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { | |
| a.copy(groupingExpressions = newGrouping) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Optimizes the logical plans wrapped in SubqueryAlias and operators on them. | ||
| * The SubqueryAlias which are remaining in optimization phase are common subqueries, | ||
| * i.e., they are duplicate in the whole query plan. The logical plans wrapped in | ||
| * SubqueryAlias will be executed individually later. However, some operators such as | ||
| * Project and Filter can be optimized with the wrapped logical plans. Thus, this rule | ||
| * considers the optimization of the wrapped logical plans and operators on SubqueryAlias. | ||
| */ | ||
| case class OptimizeCommonSubqueries(optimizer: Optimizer) | ||
| extends Rule[LogicalPlan] with PredicateHelper { | ||
| // Optimized the subqueries which all have a Project parent node and the same results. | ||
| private def optimizeProjectWithSubqueries( | ||
| plan: LogicalPlan, | ||
| keyPlan: LogicalPlan, | ||
| subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { | ||
| plan transform { | ||
| case p @ Project(pList, s @ SubqueryAlias(alias, subquery, v, true)) | ||
| if s.sameResult(keyPlan) => | ||
| val pListForAll: Seq[NamedExpression] = subqueries.flatMap { case Project(pList, child) => | ||
| val rewrites = buildRewrites(child, subquery) | ||
| pList.map(pushToOtherPlan(_, rewrites)) | ||
| } | ||
|
|
||
| val newSubquery = Project(pListForAll, subquery) | ||
| val optimized = optimizer.execute(newSubquery) | ||
| // Check if any optimization is performed. | ||
| if (optimized.sameResult(newSubquery)) { | ||
| // No optimization happens. Let's keep original subquery. | ||
| p | ||
| } else { | ||
| Project(pList.map(_.toAttribute), SubqueryAlias(alias, newSubquery, v, true)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Maps Attributes from the source side to the corresponding Attribute on the target side. | ||
| */ | ||
| private def buildRewrites(source: LogicalPlan, target: LogicalPlan): AttributeMap[Attribute] = { | ||
| assert(source.output.size == target.output.size) | ||
| AttributeMap(source.output.zip(target.output)) | ||
| } | ||
|
|
||
| /** | ||
| * Rewrites an expression so that it can be pushed to another LogicalPlan. | ||
| */ | ||
| private def pushToOtherPlan[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = { | ||
| val result = e transformUp { | ||
| case a: Attribute => rewrites.get(a).getOrElse(a) | ||
| } | ||
|
|
||
| // We must promise the compiler that we did not discard the names in the case of project | ||
| // expressions. This is safe since the only transformation is from Attribute => Attribute. | ||
| result.asInstanceOf[A] | ||
| } | ||
|
|
||
| private def optimizeFilterWithSubqueries( | ||
| plan: LogicalPlan, | ||
| keyPlan: LogicalPlan, | ||
| subqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = { | ||
| var pushdownConds = splitConjunctivePredicates(subqueries(0).asInstanceOf[Filter].condition) | ||
| subqueries.tail.foreach { | ||
| case Filter(otherCond, child) => | ||
| val rewrites = buildRewrites(child, subqueries(0).asInstanceOf[Filter].child) | ||
| // We can't simply push down all conditions from other Filter by concatenating them with | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This part has been extracted out as #15558 and can be removed if that PR is merged. |
||
| // [[Or]]. Because if any conditions contains a sub-condition which can't be pushed down | ||
| // through intermediate operators, it makes all concatenated conditions not pushed doen. | ||
| // E.g., first condition is [a && b] and second condition is [c]. If b can't be pushed | ||
| // down, the final condition [[a && b] || c] can't be pushed down too. | ||
| val pushdowns = new ArrayBuffer[Expression]() | ||
| splitConjunctivePredicates(otherCond).foreach { cond => | ||
| val rewritten = pushToOtherPlan(cond, rewrites) | ||
| pushdownConds.flatMap { pushdown => | ||
| val subConds = splitDisjunctivePredicates(pushdown) | ||
| val orCond = Or(pushdown, rewritten) | ||
| // To avoid exponential explosion of predicates, we skip [[IsNotNull]] and predicates | ||
| // which semantically equal to existing predicates. | ||
| if (rewritten.isInstanceOf[IsNotNull] | ||
| || pushdown.isInstanceOf[IsNotNull] | ||
| || subConds.exists(rewritten.semanticEquals(_)) | ||
| || pushdowns.exists(orCond.semanticEquals(_)) | ||
| || pushdownConds.exists(orCond.semanticEquals(_))) { | ||
| None | ||
| } else { | ||
| Some(orCond) | ||
| } | ||
| }.map { cond => | ||
| if (!pushdowns.exists(cond.semanticEquals(_))) { | ||
| pushdowns += cond | ||
| } | ||
| } | ||
| } | ||
| pushdownConds = pushdowns.toSeq | ||
| } | ||
| // No pushdown for common subqueries. | ||
| if (pushdownConds.isEmpty) { | ||
| plan | ||
| } else { | ||
| val finalPushdownCondition: Expression = pushdownConds.reduce(And) | ||
| plan transformDown { | ||
| case f @ Filter(cond, s @ SubqueryAlias(a, subquery, v, true)) if s.sameResult(keyPlan) => | ||
| val pushdownCond: Expression = subqueries.foldLeft(finalPushdownCondition) { | ||
| case (currentCond, sub) => | ||
| val rewrites = buildRewrites(sub.asInstanceOf[Filter].child, subquery) | ||
| pushToOtherPlan(currentCond, rewrites) | ||
| } | ||
|
|
||
| val newSubquery = Filter(pushdownCond, subquery) | ||
| val optimized = optimizer.execute(newSubquery) | ||
|
|
||
| // Check if any optimization is performed. | ||
| if (optimized.sameResult(newSubquery)) { | ||
| // No optimization happens. Let's keep original subquery. | ||
| f | ||
| } else { | ||
| Filter(cond, SubqueryAlias(a, newSubquery, v, true)) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def apply(plan: LogicalPlan): LogicalPlan = { | ||
| val subqueryMap = HashMap.empty[LogicalPlan, ArrayBuffer[LogicalPlan]] | ||
|
|
||
| // Constructs the groups of the subqueries with the same results. | ||
| plan.foreach { | ||
| case u: UnaryNode | ||
| if u.child.isInstanceOf[SubqueryAlias] && | ||
| u.child.asInstanceOf[SubqueryAlias].commonSubquery => | ||
|
|
||
| val child = u.child.asInstanceOf[SubqueryAlias].child | ||
| // Looking for the existing group with the same results. | ||
| subqueryMap.find { case (key, _) => | ||
| if (key.sameResult(child)) { | ||
| true | ||
| } else { | ||
| false | ||
| } | ||
| }.map { case (_, subqueries) => | ||
| // If found, add current logical plan into this group. | ||
| subqueries += u | ||
| }.getOrElse { | ||
| // If not, create a new group. | ||
| subqueryMap += ((child, ArrayBuffer[LogicalPlan](u))) | ||
| } | ||
| case _ => | ||
| } | ||
|
|
||
| // Begins to optimize common SubqueryAlias with outside operators. | ||
| // We only need to take care two cases: | ||
| // 1. All subqueries have a Project on them. | ||
| // 2. All subqueries have a Filter on them. | ||
| var currentPlan = plan | ||
| subqueryMap.foreach { case (key, subqueries) => | ||
| if (subqueries.length > 1) { | ||
| val allProject = subqueries.forall(_.isInstanceOf[Project]) | ||
| if (allProject) { | ||
| currentPlan = optimizeProjectWithSubqueries(currentPlan, key, subqueries) | ||
| } else { | ||
| val allFilter = subqueries.forall(_.isInstanceOf[Filter]) | ||
| if (allFilter) { | ||
| currentPlan = optimizeFilterWithSubqueries(currentPlan, key, subqueries) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| currentPlan | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Removes the [[SubqueryAlias]] operators which are not duplicated in the query plan. | ||
| */ | ||
| object EliminateNonDuplicatedSubqueryAliases extends Rule[LogicalPlan] { | ||
| def apply(plan: LogicalPlan): LogicalPlan = { | ||
| val subqueries = ArrayBuffer[LogicalPlan]() | ||
| val duplicateSubqueries = ArrayBuffer[LogicalPlan]() | ||
|
|
||
| // Eliminates the recursive subqueries which have the same output. | ||
| val cleanedPlan = plan.transformDown { | ||
| case s @ SubqueryAlias(_, child, _, _) | ||
| if child.find(p => p.isInstanceOf[SubqueryAlias] && p.sameResult(s)).isDefined => | ||
| child | ||
| } | ||
|
|
||
| // Collects duplicated subqueries but ignores the SubqueryAlias of table scan. | ||
| cleanedPlan.foreach { | ||
| case SubqueryAlias(_, child, _, _) if !child.isInstanceOf[MultiInstanceRelation] => | ||
| if (subqueries.indexWhere(s => s.sameResult(child)) >= 0) { | ||
| duplicateSubqueries += child | ||
| } else { | ||
| subqueries += child | ||
| } | ||
| case _ => | ||
| } | ||
|
|
||
| // Eliminates non-duplicated subqueries. | ||
| cleanedPlan.transformDown { | ||
| case SubqueryAlias(alias, child, v, _) => | ||
| if (duplicateSubqueries.indexWhere(s => s.sameResult(child)) < 0) { | ||
| child | ||
| } else { | ||
| SubqueryAlias(alias, child, v, commonSubquery = true) | ||
| } | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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 hidden or 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.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This change is submitted at #14912. Because it prevents this PR pushdown some predicates, for the easy of test, I also include that change here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#14912 is changed. But the changed solution is more complicated so I don't want to include it here. So I keep this change and wait for #14912 to be merged first.