Skip to content

Commit f8ac8ed

Browse files
chenghao-intelmarmbrus
authored andcommitted
[SPARK-3058] [SQL] Support EXTENDED for EXPLAIN
Provide `extended` keyword support for `explain` command in SQL. e.g. ``` explain extended select key as a1, value as a2 from src where key=1; == Parsed Logical Plan == Project ['key AS a1#3,'value AS a2#4] Filter ('key = 1) UnresolvedRelation None, src, None == Analyzed Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = CAST(1, DoubleType)) MetastoreRelation default, src, None == Optimized Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) MetastoreRelation default, src, None == Physical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) HiveTableScan [key#8,value#9], (MetastoreRelation default, src, None), None Code Generation: false == RDD == (2) MappedRDD[14] at map at HiveContext.scala:350 MapPartitionsRDD[13] at mapPartitions at basicOperators.scala:42 MapPartitionsRDD[12] at mapPartitions at basicOperators.scala:57 MapPartitionsRDD[11] at mapPartitions at TableReader.scala:112 MappedRDD[10] at map at TableReader.scala:240 HadoopRDD[9] at HadoopRDD at TableReader.scala:230 ``` It's the sub task of #1847. But can go without any dependency. Author: Cheng Hao <[email protected]> Closes #1962 from chenghao-intel/explain_extended and squashes the following commits: 295db74 [Cheng Hao] Fix bug in printing the simple execution plan 48bc989 [Cheng Hao] Support EXTENDED for EXPLAIN (cherry picked from commit 156eb39) Signed-off-by: Michael Armbrust <[email protected]>
1 parent 292f28d commit f8ac8ed

File tree

8 files changed

+78
-14
lines changed

8 files changed

+78
-14
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman
5050
* Returned by a parser when the users only wants to see what query plan would be executed, without
5151
* actually performing the execution.
5252
*/
53-
case class ExplainCommand(plan: LogicalPlan) extends Command {
53+
case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends Command {
5454
override def output =
5555
Seq(AttributeReference("plan", StringType, nullable = false)())
5656
}

sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -408,18 +408,25 @@ class SQLContext(@transient val sparkContext: SparkContext)
408408
protected def stringOrError[A](f: => A): String =
409409
try f.toString catch { case e: Throwable => e.toString }
410410

411-
def simpleString: String = stringOrError(executedPlan)
411+
def simpleString: String =
412+
s"""== Physical Plan ==
413+
|${stringOrError(executedPlan)}
414+
"""
412415

413416
override def toString: String =
414-
s"""== Logical Plan ==
417+
// TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)})
418+
// however, the `toRdd` will cause the real execution, which is not what we want.
419+
// We need to think about how to avoid the side effect.
420+
s"""== Parsed Logical Plan ==
421+
|${stringOrError(logical)}
422+
|== Analyzed Logical Plan ==
415423
|${stringOrError(analyzed)}
416424
|== Optimized Logical Plan ==
417425
|${stringOrError(optimizedPlan)}
418426
|== Physical Plan ==
419427
|${stringOrError(executedPlan)}
420428
|Code Generation: ${executedPlan.codegenEnabled}
421429
|== RDD ==
422-
|${stringOrError(toRdd.toDebugString)}
423430
""".stripMargin.trim
424431
}
425432

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -301,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
301301
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
302302
case logical.SetCommand(key, value) =>
303303
Seq(execution.SetCommand(key, value, plan.output)(context))
304-
case logical.ExplainCommand(logicalPlan) =>
305-
Seq(execution.ExplainCommand(logicalPlan, plan.output)(context))
304+
case logical.ExplainCommand(logicalPlan, extended) =>
305+
Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context))
306306
case logical.CacheCommand(tableName, cache) =>
307307
Seq(execution.CacheCommand(tableName, cache)(context))
308308
case _ => Nil

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

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -108,15 +108,19 @@ case class SetCommand(
108108
*/
109109
@DeveloperApi
110110
case class ExplainCommand(
111-
logicalPlan: LogicalPlan, output: Seq[Attribute])(
111+
logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)(
112112
@transient context: SQLContext)
113113
extends LeafNode with Command {
114114

115115
// Run through the optimizer to generate the physical plan.
116116
override protected[sql] lazy val sideEffectResult: Seq[String] = try {
117-
"Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
117+
// TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties.
118+
val queryExecution = context.executePlan(logicalPlan)
119+
val outputString = if (extended) queryExecution.toString else queryExecution.simpleString
120+
121+
outputString.split("\n")
118122
} catch { case cause: TreeNodeException[_] =>
119-
"Error occurred during query planning: " +: cause.getMessage.split("\n")
123+
("Error occurred during query planning: \n" + cause.getMessage).split("\n")
120124
}
121125

122126
def execute(): RDD[Row] = {

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -424,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
424424
logical match {
425425
case _: NativeCommand => "<Native command: executed by Hive>"
426426
case _: SetCommand => "<SET command: executed by Hive, and noted by SQLContext>"
427-
case _ => executedPlan.toString
427+
case _ => super.simpleString
428428
}
429429
}
430430
}

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -409,10 +409,9 @@ private[hive] object HiveQl {
409409
ExplainCommand(NoRelation)
410410
case Token("TOK_EXPLAIN", explainArgs) =>
411411
// Ignore FORMATTED if present.
412-
val Some(query) :: _ :: _ :: Nil =
412+
val Some(query) :: _ :: extended :: Nil =
413413
getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs)
414-
// TODO: support EXTENDED?
415-
ExplainCommand(nodeToPlan(query))
414+
ExplainCommand(nodeToPlan(query), extended != None)
416415

417416
case Token("TOK_DESCTABLE", describeArgs) =>
418417
// Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
Lines changed: 54 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,54 @@
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.hive.execution
19+
20+
import org.apache.spark.sql.QueryTest
21+
import org.apache.spark.sql.hive.test.TestHive
22+
import org.apache.spark.sql.hive.test.TestHive._
23+
import org.apache.spark.sql.Row
24+
25+
/**
26+
* A set of tests that validates support for Hive Explain command.
27+
*/
28+
class HiveExplainSuite extends QueryTest {
29+
private def check(sqlCmd: String, exists: Boolean, keywords: String*) {
30+
val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString
31+
for (key <- keywords) {
32+
if (exists) {
33+
assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)")
34+
} else {
35+
assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)")
36+
}
37+
}
38+
}
39+
40+
test("explain extended command") {
41+
check(" explain select * from src where key=123 ", true,
42+
"== Physical Plan ==")
43+
check(" explain select * from src where key=123 ", false,
44+
"== Parsed Logical Plan ==",
45+
"== Analyzed Logical Plan ==",
46+
"== Optimized Logical Plan ==")
47+
check(" explain extended select * from src where key=123 ", true,
48+
"== Parsed Logical Plan ==",
49+
"== Analyzed Logical Plan ==",
50+
"== Optimized Logical Plan ==",
51+
"== Physical Plan ==",
52+
"Code Generation", "== RDD ==")
53+
}
54+
}

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -327,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest {
327327

328328
def isExplanation(result: SchemaRDD) = {
329329
val explanation = result.select('plan).collect().map { case Row(plan: String) => plan }
330-
explanation.size > 1 && explanation.head.startsWith("Physical execution plan")
330+
explanation.exists(_ == "== Physical Plan ==")
331331
}
332332

333333
test("SPARK-1704: Explain commands as a SchemaRDD") {

0 commit comments

Comments
 (0)