Skip to content

Commit c61270f

Browse files
dilipbiswalcloud-fan
authored andcommitted
[SPARK-27395][SQL] Improve EXPLAIN command
## What changes were proposed in this pull request? This PR aims at improving the way physical plans are explained in spark. Currently, the explain output for physical plan may look very cluttered and each operator's string representation can be very wide and wraps around in the display making it little hard to follow. This especially happens when explaining a query 1) Operating on wide tables 2) Has complex expressions etc. This PR attempts to split the output into two sections. In the header section, we display the basic operator tree with a number associated with each operator. In this section, we strictly control what we output for each operator. In the footer section, each operator is verbosely displayed. Based on the feedback from Maryann, the uncorrelated subqueries (SubqueryExecs) are not included in the main plan. They are printed separately after the main plan and can be correlated by the originating expression id from its parent plan. To illustrate, here is a simple plan displayed in old vs new way. Example query1 : ``` EXPLAIN SELECT key, Max(val) FROM explain_temp1 WHERE key > 0 GROUP BY key HAVING max(val) > 0 ``` Old : ``` *(2) Project [key#2, max(val)#15] +- *(2) Filter (isnotnull(max(val#3)#18) AND (max(val#3)#18 > 0)) +- *(2) HashAggregate(keys=[key#2], functions=[max(val#3)], output=[key#2, max(val)#15, max(val#3)#18]) +- Exchange hashpartitioning(key#2, 200) +- *(1) HashAggregate(keys=[key#2], functions=[partial_max(val#3)], output=[key#2, max#21]) +- *(1) Project [key#2, val#3] +- *(1) Filter (isnotnull(key#2) AND (key#2 > 0)) +- *(1) FileScan parquet default.explain_temp1[key#2,val#3] Batched: true, DataFilters: [isnotnull(key#2), (key#2 > 0)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(key), GreaterThan(key,0)], ReadSchema: struct<key:int,val:int> ``` New : ``` Project (8) +- Filter (7) +- HashAggregate (6) +- Exchange (5) +- HashAggregate (4) +- Project (3) +- Filter (2) +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 [codegen id : 1] Output: [key#2, val#3] (2) Filter [codegen id : 1] Input : [key#2, val#3] Condition : (isnotnull(key#2) AND (key#2 > 0)) (3) Project [codegen id : 1] Output : [key#2, val#3] Input : [key#2, val#3] (4) HashAggregate [codegen id : 1] Input: [key#2, val#3] (5) Exchange Input: [key#2, max#11] (6) HashAggregate [codegen id : 2] Input: [key#2, max#11] (7) Filter [codegen id : 2] Input : [key#2, max(val)#5, max(val#3)#8] Condition : (isnotnull(max(val#3)#8) AND (max(val#3)#8 > 0)) (8) Project [codegen id : 2] Output : [key#2, max(val)#5] Input : [key#2, max(val)#5, max(val#3)#8] ``` Example Query2 (subquery): ``` SELECT * FROM explain_temp1 WHERE KEY = (SELECT Max(KEY) FROM explain_temp2 WHERE KEY = (SELECT Max(KEY) FROM explain_temp3 WHERE val > 0) AND val = 2) AND val > 3 ``` Old: ``` *(1) Project [key#2, val#3] +- *(1) Filter (((isnotnull(KEY#2) AND isnotnull(val#3)) AND (KEY#2 = Subquery scalar-subquery#39)) AND (val#3 > 3)) : +- Subquery scalar-subquery#39 : +- *(2) HashAggregate(keys=[], functions=[max(KEY#26)], output=[max(KEY)#45]) : +- Exchange SinglePartition : +- *(1) HashAggregate(keys=[], functions=[partial_max(KEY#26)], output=[max#47]) : +- *(1) Project [key#26] : +- *(1) Filter (((isnotnull(KEY#26) AND isnotnull(val#27)) AND (KEY#26 = Subquery scalar-subquery#38)) AND (val#27 = 2)) : : +- Subquery scalar-subquery#38 : : +- *(2) HashAggregate(keys=[], functions=[max(KEY#28)], output=[max(KEY)#43]) : : +- Exchange SinglePartition : : +- *(1) HashAggregate(keys=[], functions=[partial_max(KEY#28)], output=[max#49]) : : +- *(1) Project [key#28] : : +- *(1) Filter (isnotnull(val#29) AND (val#29 > 0)) : : +- *(1) FileScan parquet default.explain_temp3[key#28,val#29] Batched: true, DataFilters: [isnotnull(val#29), (val#29 > 0)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp3], PartitionFilters: [], PushedFilters: [IsNotNull(val), GreaterThan(val,0)], ReadSchema: struct<key:int,val:int> : +- *(1) FileScan parquet default.explain_temp2[key#26,val#27] Batched: true, DataFilters: [isnotnull(key#26), isnotnull(val#27), (val#27 = 2)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp2], PartitionFilters: [], PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(val,2)], ReadSchema: struct<key:int,val:int> +- *(1) FileScan parquet default.explain_temp1[key#2,val#3] Batched: true, DataFilters: [isnotnull(key#2), isnotnull(val#3), (val#3 > 3)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(key), IsNotNull(val), GreaterThan(val,3)], ReadSchema: struct<key:int,val:int> ``` New: ``` Project (3) +- Filter (2) +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 [codegen id : 1] Output: [key#2, val#3] (2) Filter [codegen id : 1] Input : [key#2, val#3] Condition : (((isnotnull(KEY#2) AND isnotnull(val#3)) AND (KEY#2 = Subquery scalar-subquery#23)) AND (val#3 > 3)) (3) Project [codegen id : 1] Output : [key#2, val#3] Input : [key#2, val#3] ===== Subqueries ===== Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery scalar-subquery#23 HashAggregate (9) +- Exchange (8) +- HashAggregate (7) +- Project (6) +- Filter (5) +- Scan parquet default.explain_temp2 (4) (4) Scan parquet default.explain_temp2 [codegen id : 1] Output: [key#26, val#27] (5) Filter [codegen id : 1] Input : [key#26, val#27] Condition : (((isnotnull(KEY#26) AND isnotnull(val#27)) AND (KEY#26 = Subquery scalar-subquery#22)) AND (val#27 = 2)) (6) Project [codegen id : 1] Output : [key#26] Input : [key#26, val#27] (7) HashAggregate [codegen id : 1] Input: [key#26] (8) Exchange Input: [max#35] (9) HashAggregate [codegen id : 2] Input: [max#35] Subquery:2 Hosting operator id = 5 Hosting Expression = Subquery scalar-subquery#22 HashAggregate (15) +- Exchange (14) +- HashAggregate (13) +- Project (12) +- Filter (11) +- Scan parquet default.explain_temp3 (10) (10) Scan parquet default.explain_temp3 [codegen id : 1] Output: [key#28, val#29] (11) Filter [codegen id : 1] Input : [key#28, val#29] Condition : (isnotnull(val#29) AND (val#29 > 0)) (12) Project [codegen id : 1] Output : [key#28] Input : [key#28, val#29] (13) HashAggregate [codegen id : 1] Input: [key#28] (14) Exchange Input: [max#37] (15) HashAggregate [codegen id : 2] Input: [max#37] ``` Note: I opened this PR as a WIP to start getting feedback. I will be on vacation starting tomorrow would not be able to immediately incorporate the feedback. I will start to work on them as soon as i can. Also, currently this PR provides a basic infrastructure for explain enhancement. The details about individual operators will be implemented in follow-up prs ## How was this patch tested? Added a new test `explain.sql` that tests basic scenarios. Need to add more tests. Closes apache#24759 from dilipbiswal/explain_feature. Authored-by: Dilip Biswal <dbiswal@us.ibm.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent c353a84 commit c61270f

26 files changed

+1318
-50
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -282,6 +282,10 @@ abstract class Expression extends TreeNode[Expression] {
282282
val childrenSQL = children.map(_.sql).mkString(", ")
283283
s"$prettyName($childrenSQL)"
284284
}
285+
286+
override def simpleStringWithNodeId(): String = {
287+
throw new UnsupportedOperationException(s"$nodeName does not implement simpleStringWithNodeId")
288+
}
285289
}
286290

287291

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,9 @@ trait Block extends TreeNode[Block] with JavaCode {
198198
}
199199

200200
override def verboseString(maxFields: Int): String = toString
201+
override def simpleStringWithNodeId(): String = {
202+
throw new UnsupportedOperationException(s"$nodeName does not implement simpleStringWithNodeId")
203+
}
201204
}
202205

203206
object Block {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala

Lines changed: 23 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@ package org.apache.spark.sql.catalyst.plans
1919

2020
import org.apache.spark.sql.AnalysisException
2121
import org.apache.spark.sql.catalyst.expressions._
22-
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
22+
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode, TreeNodeTag}
23+
import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat
2324
import org.apache.spark.sql.internal.SQLConf
2425
import org.apache.spark.sql.types.{DataType, StructType}
2526

@@ -179,6 +180,20 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT
179180

180181
override def verboseString(maxFields: Int): String = simpleString(maxFields)
181182

183+
override def simpleStringWithNodeId(): String = {
184+
val operatorId = getTagValue(QueryPlan.OP_ID_TAG).map(id => s"$id").getOrElse("unknown")
185+
s"$nodeName ($operatorId)".trim
186+
}
187+
188+
def verboseStringWithOperatorId(): String = {
189+
val codegenIdStr =
190+
getTagValue(QueryPlan.CODEGEN_ID_TAG).map(id => s"[codegen id : $id]").getOrElse("")
191+
val operatorId = getTagValue(QueryPlan.OP_ID_TAG).map(id => s"$id").getOrElse("unknown")
192+
s"""
193+
|($operatorId) $nodeName $codegenIdStr
194+
""".stripMargin
195+
}
196+
182197
/**
183198
* All the subqueries of current plan.
184199
*/
@@ -204,7 +219,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT
204219
subqueries ++ subqueries.flatMap(_.subqueriesAll)
205220
}
206221

207-
override protected def innerChildren: Seq[QueryPlan[_]] = subqueries
222+
override def innerChildren: Seq[QueryPlan[_]] = subqueries
208223

209224
/**
210225
* A private mutable variable to indicate whether this plan is the result of canonicalization.
@@ -289,6 +304,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT
289304
}
290305

291306
object QueryPlan extends PredicateHelper {
307+
val OP_ID_TAG = TreeNodeTag[Int]("operatorId")
308+
val CODEGEN_ID_TAG = new TreeNodeTag[Int]("wholeStageCodegenId")
309+
292310
/**
293311
* Normalize the exprIds in the given expression, by updating the exprId in `AttributeReference`
294312
* with its referenced ordinal from input attributes. It's similar to `BindReferences` but we
@@ -335,9 +353,10 @@ object QueryPlan extends PredicateHelper {
335353
append: String => Unit,
336354
verbose: Boolean,
337355
addSuffix: Boolean,
338-
maxFields: Int = SQLConf.get.maxToStringFields): Unit = {
356+
maxFields: Int = SQLConf.get.maxToStringFields,
357+
printOperatorId: Boolean = false): Unit = {
339358
try {
340-
plan.treeString(append, verbose, addSuffix, maxFields)
359+
plan.treeString(append, verbose, addSuffix, maxFields, printOperatorId)
341360
} catch {
342361
case e: AnalysisException => append(e.toString)
343362
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala

Lines changed: 29 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat,
3434
import org.apache.spark.sql.catalyst.errors._
3535
import org.apache.spark.sql.catalyst.expressions._
3636
import org.apache.spark.sql.catalyst.plans.JoinType
37+
import org.apache.spark.sql.catalyst.plans.QueryPlan
3738
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning}
3839
import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat
3940
import org.apache.spark.sql.catalyst.util.truncatedString
@@ -530,9 +531,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
530531
* @param maxFields Maximum number of fields that will be converted to strings.
531532
* Any elements beyond the limit will be dropped.
532533
*/
533-
def simpleString(maxFields: Int): String = {
534-
s"$nodeName ${argString(maxFields)}".trim
535-
}
534+
def simpleString(maxFields: Int): String = s"$nodeName ${argString(maxFields)}".trim
535+
536+
/**
537+
* ONE line description of this node containing the node identifier.
538+
* @return
539+
*/
540+
def simpleStringWithNodeId(): String
536541

537542
/** ONE line description of this node with more information */
538543
def verboseString(maxFields: Int): String
@@ -548,19 +553,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
548553
final def treeString(
549554
verbose: Boolean,
550555
addSuffix: Boolean = false,
551-
maxFields: Int = SQLConf.get.maxToStringFields): String = {
556+
maxFields: Int = SQLConf.get.maxToStringFields,
557+
printOperatorId: Boolean = false): String = {
552558
val concat = new PlanStringConcat()
553-
554-
treeString(concat.append, verbose, addSuffix, maxFields)
559+
treeString(concat.append, verbose, addSuffix, maxFields, printOperatorId)
555560
concat.toString
556561
}
557562

558563
def treeString(
559564
append: String => Unit,
560565
verbose: Boolean,
561566
addSuffix: Boolean,
562-
maxFields: Int): Unit = {
563-
generateTreeString(0, Nil, append, verbose, "", addSuffix, maxFields)
567+
maxFields: Int,
568+
printOperatorId: Boolean): Unit = {
569+
generateTreeString(0, Nil, append, verbose, "", addSuffix, maxFields, printOperatorId)
564570
}
565571

566572
/**
@@ -609,7 +615,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
609615
* All the nodes that should be shown as a inner nested tree of this node.
610616
* For example, this can be used to show sub-queries.
611617
*/
612-
protected def innerChildren: Seq[TreeNode[_]] = Seq.empty
618+
def innerChildren: Seq[TreeNode[_]] = Seq.empty
613619

614620
/**
615621
* Appends the string representation of this node and its children to the given Writer.
@@ -627,7 +633,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
627633
verbose: Boolean,
628634
prefix: String = "",
629635
addSuffix: Boolean = false,
630-
maxFields: Int): Unit = {
636+
maxFields: Int,
637+
printNodeId: Boolean): Unit = {
631638

632639
if (depth > 0) {
633640
lastChildren.init.foreach { isLast =>
@@ -639,7 +646,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
639646
val str = if (verbose) {
640647
if (addSuffix) verboseStringWithSuffix(maxFields) else verboseString(maxFields)
641648
} else {
642-
simpleString(maxFields)
649+
if (printNodeId) {
650+
simpleStringWithNodeId()
651+
} else {
652+
simpleString(maxFields)
653+
}
643654
}
644655
append(prefix)
645656
append(str)
@@ -648,17 +659,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
648659
if (innerChildren.nonEmpty) {
649660
innerChildren.init.foreach(_.generateTreeString(
650661
depth + 2, lastChildren :+ children.isEmpty :+ false, append, verbose,
651-
addSuffix = addSuffix, maxFields = maxFields))
662+
addSuffix = addSuffix, maxFields = maxFields, printNodeId = printNodeId))
652663
innerChildren.last.generateTreeString(
653664
depth + 2, lastChildren :+ children.isEmpty :+ true, append, verbose,
654-
addSuffix = addSuffix, maxFields = maxFields)
665+
addSuffix = addSuffix, maxFields = maxFields, printNodeId = printNodeId)
655666
}
656667

657668
if (children.nonEmpty) {
658669
children.init.foreach(_.generateTreeString(
659-
depth + 1, lastChildren :+ false, append, verbose, prefix, addSuffix, maxFields))
670+
depth + 1, lastChildren :+ false, append, verbose, prefix, addSuffix,
671+
maxFields, printNodeId = printNodeId)
672+
)
660673
children.last.generateTreeString(
661-
depth + 1, lastChildren :+ true, append, verbose, prefix, addSuffix, maxFields)
674+
depth + 1, lastChildren :+ true, append, verbose, prefix,
675+
addSuffix, maxFields, printNodeId = printNodeId)
662676
}
663677
}
664678

0 commit comments

Comments
 (0)