Skip to content

Commit 682e7f2

Browse files
peter-tothcloud-fan
authored andcommitted
[SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse
### What changes were proposed in this pull request? This PR: 1. Fixes an issue in `ReuseExchange` rule that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate traversals in `ReuseExchange` when the 2nd traversal modifies an exchange that has already been referenced (reused) in the 1st traversal. Consider the following query: ``` WITH t AS ( SELECT df1.id, df2.k FROM df1 JOIN df2 ON df1.k = df2.k WHERE df2.id < 2 ) SELECT * FROM t AS a JOIN t AS b ON a.id = b.id ``` Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker): ``` == Physical Plan == *(7) SortMergeJoin [id#14L], [id#18L], Inner :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#14L, 5), true, [id=#298] : +- *(2) Project [id#14L, k#17L] : +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight : :- *(2) Project [id#14L, k#15L] : : +- *(2) Filter isnotnull(id#14L) : : +- *(2) ColumnarToRow : : +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#289] : : +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179] : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179] : +- *(1) Project [k#17L] : +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L)) : +- *(1) ColumnarToRow : +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint> +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0 +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#184] <== this reuse node points to a non-existing node ``` After this PR: ``` == Physical Plan == *(7) SortMergeJoin [id#14L], [id#18L], Inner :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#14L, 5), true, [id=#231] : +- *(2) Project [id#14L, k#17L] : +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight : :- *(2) Project [id#14L, k#15L] : : +- *(2) Filter isnotnull(id#14L) : : +- *(2) ColumnarToRow : : +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#103] : : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102] : : +- *(1) Project [k#17L] : : +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L)) : : +- *(1) ColumnarToRow : : +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint> : +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102] +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0 +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#231] ``` 2. Fixes an issue with separate consecutive `ReuseExchange` and `ReuseSubquery` rules that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate rules when `ReuseSubquery` rule modifies an exchange that has already been referenced (reused) in `ReuseExchange` rule. Consider the following query: ``` WITH t AS ( SELECT df1.id, df2.k FROM df1 JOIN df2 ON df1.k = df2.k WHERE df2.id < 2 ), t2 AS ( SELECT * FROM t UNION SELECT * FROM t ) SELECT * FROM t2 AS a JOIN t2 AS b ON a.id = b.id ``` Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker): ``` == Physical Plan == *(15) SortMergeJoin [id#46L], [id#58L], Inner :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#46L, 5), true, [id=#979] : +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[]) : +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#975] : +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[]) : +- Union : :- *(2) Project [id#46L, k#49L] : : +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight : : :- *(2) Project [id#46L, k#47L] : : : +- *(2) Filter isnotnull(id#46L) : : : +- *(2) ColumnarToRow : : : +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : : +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926] : : : +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656] : : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656] : : +- *(1) Project [k#49L] : : +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L)) : : +- *(1) ColumnarToRow : : +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint> : +- *(4) Project [id#46L, k#49L] : +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight : :- *(4) Project [id#46L, k#47L] : : +- *(4) Filter isnotnull(id#46L) : : +- *(4) ColumnarToRow : : +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926] : +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656] +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0 +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#761] <== this reuse node points to a non-existing node ``` After this PR: ``` == Physical Plan == *(15) SortMergeJoin [id#46L], [id#58L], Inner :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#46L, 5), true, [id=#793] : +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[]) : +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#789] : +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[]) : +- Union : :- *(2) Project [id#46L, k#49L] : : +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight : : :- *(2) Project [id#46L, k#47L] : : : +- *(2) Filter isnotnull(id#46L) : : : +- *(2) ColumnarToRow : : : +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : : +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485] : : : +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484] : : : +- *(1) Project [k#49L] : : : +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L)) : : : +- *(1) ColumnarToRow : : : +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint> : : +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484] : +- *(4) Project [id#46L, k#49L] : +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight : :- *(4) Project [id#46L, k#47L] : : +- *(4) Filter isnotnull(id#46L) : : +- *(4) ColumnarToRow : : +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint> : : +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485] : +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484] +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0 +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#793] ``` (This example contains issue 1 as well.) 3. Improves the reuse of exchanges and subqueries by enabling reuse across the whole plan. This means that the new combined rule utilizes the reuse opportunities between parent and subqueries by traversing the whole plan. The traversal is started on the top level query only. 4. Due to the order of traversal this PR does while adding reuse nodes, the reuse nodes appear in parent queries if reuse is possible between different levels of queries (typical for DPP). This is not an issue from execution perspective, but this also means "forward references" in explain formatted output where parent queries come first. The changes I made to `ExplainUtils` are to handle these references properly. This PR fixes the above 3 issues by unifying the separate rules into a `ReuseExchangeAndSubquery` rule that does a 1 pass, whole-plan, bottom-up traversal. ### Why are the changes needed? Performance improvement. ### How was this patch tested? - New UTs in `ReuseExchangeAndSubquerySuite` to cover 1. and 2. - New UTs in `DynamicPartitionPruningSuite`, `SubquerySuite` and `ExchangeSuite` to cover 3. - New `ReuseMapSuite` to test `ReuseMap`. - Checked new golden files of `PlanStabilitySuite`s for invalid reuse references. - TPCDS benchmarks. Closes #28885 from peter-toth/SPARK-29375-SPARK-28940-whole-plan-reuse. Authored-by: Peter Toth <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 248fda3 commit 682e7f2

File tree

490 files changed

+51841
-52460
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

490 files changed

+51841
-52460
lines changed
Lines changed: 73 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,73 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.util
19+
20+
import scala.collection.mutable.{ArrayBuffer, Map}
21+
22+
import org.apache.spark.sql.catalyst.plans.QueryPlan
23+
import org.apache.spark.sql.types.StructType
24+
25+
/**
26+
* Map of canonicalized plans that can be used to find reuse possibilities.
27+
*
28+
* To avoid costly canonicalization of a plan:
29+
* - we use its schema first to check if it can be replaced to a reused one at all
30+
* - we insert it into the map of canonicalized plans only when at least 2 have the same schema
31+
*
32+
* @tparam T the type of the node we want to reuse
33+
* @tparam T2 the type of the canonicalized node
34+
*/
35+
class ReuseMap[T <: T2, T2 <: QueryPlan[T2]] {
36+
private val map = Map[StructType, ArrayBuffer[T]]()
37+
38+
/**
39+
* Find a matching plan with the same canonicalized form in the map or add the new plan to the
40+
* map otherwise.
41+
*
42+
* @param plan the input plan
43+
* @return the matching plan or the input plan
44+
*/
45+
private def lookupOrElseAdd(plan: T): T = {
46+
val sameSchema = map.getOrElseUpdate(plan.schema, ArrayBuffer())
47+
val samePlan = sameSchema.find(plan.sameResult)
48+
if (samePlan.isDefined) {
49+
samePlan.get
50+
} else {
51+
sameSchema += plan
52+
plan
53+
}
54+
}
55+
56+
/**
57+
* Find a matching plan with the same canonicalized form in the map and apply `f` on it or add
58+
* the new plan to the map otherwise.
59+
*
60+
* @param plan the input plan
61+
* @param f the function to apply
62+
* @tparam T2 the type of the reuse node
63+
* @return the matching plan with `f` applied or the input plan
64+
*/
65+
def reuseOrElseAdd[T2 >: T](plan: T, f: T => T2): T2 = {
66+
val found = lookupOrElseAdd(plan)
67+
if (found eq plan) {
68+
plan
69+
} else {
70+
f(found)
71+
}
72+
}
73+
}
Lines changed: 73 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,73 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.util
19+
20+
import org.apache.spark.SparkFunSuite
21+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
22+
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
23+
import org.apache.spark.sql.types.IntegerType
24+
25+
case class TestNode(children: Seq[TestNode], output: Seq[Attribute]) extends LogicalPlan {
26+
override protected def withNewChildrenInternal(
27+
newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = copy(children = children)
28+
}
29+
case class TestReuseNode(child: LogicalPlan) extends UnaryNode {
30+
override def output: Seq[Attribute] = child.output
31+
32+
override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
33+
copy(child = newChild)
34+
}
35+
36+
class ReuseMapSuite extends SparkFunSuite {
37+
private val leafNode1 = TestNode(Nil, Seq(AttributeReference("a", IntegerType)()))
38+
private val leafNode2 = TestNode(Nil, Seq(AttributeReference("b", IntegerType)()))
39+
private val parentNode1 = TestNode(Seq(leafNode1), Seq(AttributeReference("a", IntegerType)()))
40+
private val parentNode2 = TestNode(Seq(leafNode2), Seq(AttributeReference("b", IntegerType)()))
41+
42+
private def reuse(testNode: TestNode) = TestReuseNode(testNode)
43+
44+
test("no reuse if same instance") {
45+
val reuseMap = new ReuseMap[TestNode, LogicalPlan]()
46+
47+
reuseMap.reuseOrElseAdd(leafNode1, reuse)
48+
reuseMap.reuseOrElseAdd(parentNode1, reuse)
49+
50+
assert(reuseMap.reuseOrElseAdd(leafNode1, reuse) == leafNode1)
51+
assert(reuseMap.reuseOrElseAdd(parentNode1, reuse) == parentNode1)
52+
}
53+
54+
test("reuse if different instance with same canonicalized plan") {
55+
val reuseMap = new ReuseMap[TestNode, LogicalPlan]()
56+
reuseMap.reuseOrElseAdd(leafNode1, reuse)
57+
reuseMap.reuseOrElseAdd(parentNode1, reuse)
58+
59+
assert(reuseMap.reuseOrElseAdd(leafNode1.clone.asInstanceOf[TestNode], reuse) ==
60+
reuse(leafNode1))
61+
assert(reuseMap.reuseOrElseAdd(parentNode1.clone.asInstanceOf[TestNode], reuse) ==
62+
reuse(parentNode1))
63+
}
64+
65+
test("no reuse if different canonicalized plan") {
66+
val reuseMap = new ReuseMap[TestNode, LogicalPlan]()
67+
reuseMap.reuseOrElseAdd(leafNode1, reuse)
68+
reuseMap.reuseOrElseAdd(parentNode1, reuse)
69+
70+
assert(reuseMap.reuseOrElseAdd(leafNode2, reuse) == leafNode2)
71+
assert(reuseMap.reuseOrElseAdd(parentNode2, reuse) == parentNode2)
72+
}
73+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala

Lines changed: 85 additions & 71 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark.sql.execution
1919

20-
import scala.collection.mutable
2120
import scala.collection.mutable.ArrayBuffer
2221

2322
import org.apache.spark.sql.AnalysisException
@@ -28,11 +27,9 @@ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveS
2827
object ExplainUtils extends AdaptiveSparkPlanHelper {
2928
/**
3029
* Given a input physical plan, performs the following tasks.
31-
* 1. Computes the operator id for current operator and records it in the operator
32-
* by setting a tag.
33-
* 2. Computes the whole stage codegen id for current operator and records it in the
30+
* 1. Computes the whole stage codegen id for current operator and records it in the
3431
* operator by setting a tag.
35-
* 3. Generate the two part explain output for this plan.
32+
* 2. Generate the two part explain output for this plan.
3633
* 1. First part explains the operator tree with each operator tagged with an unique
3734
* identifier.
3835
* 2. Second part explains each operator in a verbose manner.
@@ -41,22 +38,11 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
4138
*
4239
* @param plan Input query plan to process
4340
* @param append function used to append the explain output
44-
* @param startOperatorID The start value of operation id. The subsequent operations will
45-
* be assigned higher value.
46-
*
47-
* @return The last generated operation id for this input plan. This is to ensure we
48-
* always assign incrementing unique id to each operator.
49-
*
5041
*/
5142
private def processPlanSkippingSubqueries[T <: QueryPlan[T]](
52-
plan: => QueryPlan[T],
53-
append: String => Unit,
54-
startOperatorID: Int): Int = {
55-
56-
val operationIDs = new mutable.ArrayBuffer[(Int, QueryPlan[_])]()
57-
var currentOperatorID = startOperatorID
43+
plan: T,
44+
append: String => Unit): Unit = {
5845
try {
59-
currentOperatorID = generateOperatorIDs(plan, currentOperatorID, operationIDs)
6046
generateWholeStageCodegenIds(plan)
6147

6248
QueryPlan.append(
@@ -67,31 +53,36 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
6753
printOperatorId = true)
6854

6955
append("\n")
70-
var i: Integer = 0
71-
for ((opId, curPlan) <- operationIDs) {
72-
append(curPlan.verboseStringWithOperatorId())
73-
}
56+
57+
val operationsWithID = ArrayBuffer.empty[QueryPlan[_]]
58+
collectOperatorsWithID(plan, operationsWithID)
59+
operationsWithID.foreach(p => append(p.verboseStringWithOperatorId()))
60+
7461
} catch {
7562
case e: AnalysisException => append(e.toString)
7663
}
77-
currentOperatorID
7864
}
7965

8066
/**
8167
* Given a input physical plan, performs the following tasks.
8268
* 1. Generates the explain output for the input plan excluding the subquery plans.
8369
* 2. Generates the explain output for each subquery referenced in the plan.
8470
*/
85-
def processPlan[T <: QueryPlan[T]](
86-
plan: => QueryPlan[T],
87-
append: String => Unit): Unit = {
71+
def processPlan[T <: QueryPlan[T]](plan: T, append: String => Unit): Unit = {
8872
try {
89-
val subqueries = ArrayBuffer.empty[(SparkPlan, Expression, BaseSubqueryExec)]
9073
var currentOperatorID = 0
91-
currentOperatorID = processPlanSkippingSubqueries(plan, append, currentOperatorID)
74+
currentOperatorID = generateOperatorIDs(plan, currentOperatorID)
75+
76+
val subqueries = ArrayBuffer.empty[(SparkPlan, Expression, BaseSubqueryExec)]
9277
getSubqueries(plan, subqueries)
93-
var i = 0
9478

79+
subqueries.foldLeft(currentOperatorID) {
80+
(curId, plan) => generateOperatorIDs(plan._3.child, curId)
81+
}
82+
83+
processPlanSkippingSubqueries(plan, append)
84+
85+
var i = 0
9586
for (sub <- subqueries) {
9687
if (i == 0) {
9788
append("\n===== Subqueries =====\n\n")
@@ -104,10 +95,7 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
10495
// the explain output. In case of subquery reuse, we don't print subquery plan more
10596
// than once. So we skip [[ReusedSubqueryExec]] here.
10697
if (!sub._3.isInstanceOf[ReusedSubqueryExec]) {
107-
currentOperatorID = processPlanSkippingSubqueries(
108-
sub._3.child,
109-
append,
110-
currentOperatorID)
98+
processPlanSkippingSubqueries(sub._3.child, append)
11199
}
112100
append("\n")
113101
}
@@ -117,59 +105,85 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
117105
}
118106

119107
/**
120-
* Traverses the supplied input plan in a bottom-up fashion does the following :
121-
* 1. produces a map : operator identifier -> operator
122-
* 2. Records the operator id via setting a tag in the operator.
108+
* Traverses the supplied input plan in a bottom-up fashion and records the operator id via
109+
* setting a tag in the operator.
123110
* Note :
124-
* 1. Operator such as WholeStageCodegenExec and InputAdapter are skipped as they don't
125-
* appear in the explain output.
126-
* 2. operator identifier starts at startOperatorID + 1
111+
* - Operator such as WholeStageCodegenExec and InputAdapter are skipped as they don't
112+
* appear in the explain output.
113+
* - Operator identifier starts at startOperatorID + 1
114+
*
127115
* @param plan Input query plan to process
128-
* @param startOperatorID The start value of operation id. The subsequent operations will
129-
* be assigned higher value.
130-
* @param operatorIDs A output parameter that contains a map of operator id and query plan. This
131-
* is used by caller to print the detail portion of the plan.
132-
* @return The last generated operation id for this input plan. This is to ensure we
133-
* always assign incrementing unique id to each operator.
116+
* @param startOperatorID The start value of operation id. The subsequent operations will be
117+
* assigned higher value.
118+
* @return The last generated operation id for this input plan. This is to ensure we always
119+
* assign incrementing unique id to each operator.
134120
*/
135-
private def generateOperatorIDs(
136-
plan: QueryPlan[_],
137-
startOperatorID: Int,
138-
operatorIDs: mutable.ArrayBuffer[(Int, QueryPlan[_])]): Int = {
121+
private def generateOperatorIDs(plan: QueryPlan[_], startOperatorID: Int): Int = {
139122
var currentOperationID = startOperatorID
140123
// Skip the subqueries as they are not printed as part of main query block.
141124
if (plan.isInstanceOf[BaseSubqueryExec]) {
142125
return currentOperationID
143126
}
144-
plan.foreachUp {
145-
case p: WholeStageCodegenExec =>
146-
case p: InputAdapter =>
147-
case other: QueryPlan[_] =>
148127

149-
def setOpId(): Unit = if (other.getTagValue(QueryPlan.OP_ID_TAG).isEmpty) {
150-
currentOperationID += 1
151-
other.setTagValue(QueryPlan.OP_ID_TAG, currentOperationID)
152-
operatorIDs += ((currentOperationID, other))
153-
}
128+
def setOpId(plan: QueryPlan[_]): Unit = if (plan.getTagValue(QueryPlan.OP_ID_TAG).isEmpty) {
129+
currentOperationID += 1
130+
plan.setTagValue(QueryPlan.OP_ID_TAG, currentOperationID)
131+
}
154132

155-
other match {
156-
case p: AdaptiveSparkPlanExec =>
157-
currentOperationID =
158-
generateOperatorIDs(p.executedPlan, currentOperationID, operatorIDs)
159-
setOpId()
160-
case p: QueryStageExec =>
161-
currentOperationID = generateOperatorIDs(p.plan, currentOperationID, operatorIDs)
162-
setOpId()
163-
case _ =>
164-
setOpId()
165-
other.innerChildren.foldLeft(currentOperationID) {
166-
(curId, plan) => generateOperatorIDs(plan, curId, operatorIDs)
167-
}
133+
plan.foreachUp {
134+
case _: WholeStageCodegenExec =>
135+
case _: InputAdapter =>
136+
case p: AdaptiveSparkPlanExec =>
137+
currentOperationID = generateOperatorIDs(p.executedPlan, currentOperationID)
138+
setOpId(p)
139+
case p: QueryStageExec =>
140+
currentOperationID = generateOperatorIDs(p.plan, currentOperationID)
141+
setOpId(p)
142+
case other: QueryPlan[_] =>
143+
setOpId(other)
144+
other.innerChildren.foldLeft(currentOperationID) {
145+
(curId, plan) => generateOperatorIDs(plan, curId)
168146
}
169147
}
170148
currentOperationID
171149
}
172150

151+
/**
152+
* Traverses the supplied input plan in a bottom-up fashion and collects operators with assigned
153+
* ids.
154+
*
155+
* @param plan Input query plan to process
156+
* @param operators An output parameter that contains the operators.
157+
*/
158+
private def collectOperatorsWithID(
159+
plan: QueryPlan[_],
160+
operators: ArrayBuffer[QueryPlan[_]]): Unit = {
161+
// Skip the subqueries as they are not printed as part of main query block.
162+
if (plan.isInstanceOf[BaseSubqueryExec]) {
163+
return
164+
}
165+
166+
def collectOperatorWithID(plan: QueryPlan[_]): Unit = {
167+
if (plan.getTagValue(QueryPlan.OP_ID_TAG).isDefined) {
168+
operators += plan
169+
}
170+
}
171+
172+
plan.foreachUp {
173+
case _: WholeStageCodegenExec =>
174+
case _: InputAdapter =>
175+
case p: AdaptiveSparkPlanExec =>
176+
collectOperatorsWithID(p.executedPlan, operators)
177+
collectOperatorWithID(p)
178+
case p: QueryStageExec =>
179+
collectOperatorsWithID(p.plan, operators)
180+
collectOperatorWithID(p)
181+
case other: QueryPlan[_] =>
182+
collectOperatorWithID(other)
183+
other.innerChildren.foreach(collectOperatorsWithID(_, operators))
184+
}
185+
}
186+
173187
/**
174188
* Traverses the supplied input plan in a top-down fashion and records the
175189
* whole stage code gen id in the plan via setting a tag.

0 commit comments

Comments
 (0)