aboutsummaryrefslogtreecommitdiff
path: root/sql/core
diff options
context:
space:
mode:
authorCheng Hao <hao.cheng@intel.com>2014-08-25 17:43:56 -0700
committerMichael Armbrust <michael@databricks.com>2014-08-25 17:44:27 -0700
commitf8ac8ed7f88d2ee976b38d4a156f64efb3740650 (patch)
tree5bd17ec41ee123ba8ad2843396a6c36a2dee1d6d /sql/core
parent292f28d4f7cbfdb8b90809926a6d69df7ed817e7 (diff)
downloadspark-f8ac8ed7f88d2ee976b38d4a156f64efb3740650.tar.gz
spark-f8ac8ed7f88d2ee976b38d4a156f64efb3740650.tar.bz2
spark-f8ac8ed7f88d2ee976b38d4a156f64efb3740650.zip
[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 <hao.cheng@intel.com> 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 156eb3966176de02ec3ec90ae10e50a7ebfbbf4f) Signed-off-by: Michael Armbrust <michael@databricks.com>
Diffstat (limited to 'sql/core')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala13
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala10
3 files changed, 19 insertions, 8 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index af9f7c62a1..8a9f4deb6a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -408,10 +408,18 @@ class SQLContext(@transient val sparkContext: SparkContext)
protected def stringOrError[A](f: => A): String =
try f.toString catch { case e: Throwable => e.toString }
- def simpleString: String = stringOrError(executedPlan)
+ def simpleString: String =
+ s"""== Physical Plan ==
+ |${stringOrError(executedPlan)}
+ """
override def toString: String =
- s"""== Logical Plan ==
+ // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)})
+ // however, the `toRdd` will cause the real execution, which is not what we want.
+ // We need to think about how to avoid the side effect.
+ s"""== Parsed Logical Plan ==
+ |${stringOrError(logical)}
+ |== Analyzed Logical Plan ==
|${stringOrError(analyzed)}
|== Optimized Logical Plan ==
|${stringOrError(optimizedPlan)}
@@ -419,7 +427,6 @@ class SQLContext(@transient val sparkContext: SparkContext)
|${stringOrError(executedPlan)}
|Code Generation: ${executedPlan.codegenEnabled}
|== RDD ==
- |${stringOrError(toRdd.toDebugString)}
""".stripMargin.trim
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 517b77804a..8dacb84c8a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -301,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.SetCommand(key, value) =>
Seq(execution.SetCommand(key, value, plan.output)(context))
- case logical.ExplainCommand(logicalPlan) =>
- Seq(execution.ExplainCommand(logicalPlan, plan.output)(context))
+ case logical.ExplainCommand(logicalPlan, extended) =>
+ Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context))
case logical.CacheCommand(tableName, cache) =>
Seq(execution.CacheCommand(tableName, cache)(context))
case _ => Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 38f37564f1..031b695169 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -108,15 +108,19 @@ case class SetCommand(
*/
@DeveloperApi
case class ExplainCommand(
- logicalPlan: LogicalPlan, output: Seq[Attribute])(
+ logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)(
@transient context: SQLContext)
extends LeafNode with Command {
// Run through the optimizer to generate the physical plan.
override protected[sql] lazy val sideEffectResult: Seq[String] = try {
- "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
+ // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties.
+ val queryExecution = context.executePlan(logicalPlan)
+ val outputString = if (extended) queryExecution.toString else queryExecution.simpleString
+
+ outputString.split("\n")
} catch { case cause: TreeNodeException[_] =>
- "Error occurred during query planning: " +: cause.getMessage.split("\n")
+ ("Error occurred during query planning: \n" + cause.getMessage).split("\n")
}
def execute(): RDD[Row] = {