Repository: spark
Updated Branches:
refs/heads/branch-1.1 292f28d4f -> f8ac8ed7f
[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 156eb3966176de02ec3ec90ae10e50a7ebfbbf4f)
Signed-off-by: Michael Armbrust <[email protected]>
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f8ac8ed7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f8ac8ed7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f8ac8ed7
Branch: refs/heads/branch-1.1
Commit: f8ac8ed7f88d2ee976b38d4a156f64efb3740650
Parents: 292f28d
Author: Cheng Hao <[email protected]>
Authored: Mon Aug 25 17:43:56 2014 -0700
Committer: Michael Armbrust <[email protected]>
Committed: Mon Aug 25 17:44:27 2014 -0700
----------------------------------------------------------------------
.../sql/catalyst/plans/logical/commands.scala | 2 +-
.../scala/org/apache/spark/sql/SQLContext.scala | 13 +++--
.../spark/sql/execution/SparkStrategies.scala | 4 +-
.../apache/spark/sql/execution/commands.scala | 10 ++--
.../org/apache/spark/sql/hive/HiveContext.scala | 2 +-
.../org/apache/spark/sql/hive/HiveQl.scala | 5 +-
.../sql/hive/execution/HiveExplainSuite.scala | 54 ++++++++++++++++++++
.../sql/hive/execution/HiveQuerySuite.scala | 2 +-
8 files changed, 78 insertions(+), 14 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
----------------------------------------------------------------------
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
index 481a5a4..a01809c 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
@@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value:
Option[String]) extends Comman
* Returned by a parser when the users only wants to see what query plan would
be executed, without
* actually performing the execution.
*/
-case class ExplainCommand(plan: LogicalPlan) extends Command {
+case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false)
extends Command {
override def output =
Seq(AttributeReference("plan", StringType, nullable = false)())
}
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
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 af9f7c6..8a9f4de 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
}
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
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 517b778..8dacb84 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
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
----------------------------------------------------------------------
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 38f3756..031b695 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] = {
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 29baefe..d9b2bc7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -424,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
logical match {
case _: NativeCommand => "<Native command: executed by Hive>"
case _: SetCommand => "<SET command: executed by Hive, and noted by
SQLContext>"
- case _ => executedPlan.toString
+ case _ => super.simpleString
}
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 1d9ba1b..5da6e8d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -409,10 +409,9 @@ private[hive] object HiveQl {
ExplainCommand(NoRelation)
case Token("TOK_EXPLAIN", explainArgs) =>
// Ignore FORMATTED if present.
- val Some(query) :: _ :: _ :: Nil =
+ val Some(query) :: _ :: extended :: Nil =
getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs)
- // TODO: support EXTENDED?
- ExplainCommand(nodeToPlan(query))
+ ExplainCommand(nodeToPlan(query), extended != None)
case Token("TOK_DESCTABLE", describeArgs) =>
// Reference:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
----------------------------------------------------------------------
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
new file mode 100644
index 0000000..4ed58f4
--- /dev/null
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.Row
+
+/**
+ * A set of tests that validates support for Hive Explain command.
+ */
+class HiveExplainSuite extends QueryTest {
+ private def check(sqlCmd: String, exists: Boolean, keywords: String*) {
+ val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString
+ for (key <- keywords) {
+ if (exists) {
+ assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist
in result)")
+ } else {
+ assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in
the result)")
+ }
+ }
+ }
+
+ test("explain extended command") {
+ check(" explain select * from src where key=123 ", true,
+ "== Physical Plan ==")
+ check(" explain select * from src where key=123 ", false,
+ "== Parsed Logical Plan ==",
+ "== Analyzed Logical Plan ==",
+ "== Optimized Logical Plan ==")
+ check(" explain extended select * from src where key=123 ", true,
+ "== Parsed Logical Plan ==",
+ "== Analyzed Logical Plan ==",
+ "== Optimized Logical Plan ==",
+ "== Physical Plan ==",
+ "Code Generation", "== RDD ==")
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/f8ac8ed7/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index 26e4ec6..6d925e5 100644
---
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -327,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest {
def isExplanation(result: SchemaRDD) = {
val explanation = result.select('plan).collect().map { case Row(plan:
String) => plan }
- explanation.size > 1 && explanation.head.startsWith("Physical execution
plan")
+ explanation.exists(_ == "== Physical Plan ==")
}
test("SPARK-1704: Explain commands as a SchemaRDD") {
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]