Repository: spark
Updated Branches:
  refs/heads/master 97a0bfe1c -> 544880457


[SPARK-2059][SQL] Don't throw TreeNodeException in `execution.ExplainCommand`

This is a fix for the problem revealed by PR #1265.

Currently `HiveComparisonSuite` ignores output of `ExplainCommand` since 
Catalyst query plan is quite different from Hive query plan. But exceptions 
throw from `CheckResolution` still breaks test cases. This PR catches any 
`TreeNodeException` and reports it as part of the query explanation.

After merging this PR, PR #1265 can also be merged safely.

For a normal query:

```
scala> hql("explain select key from src").foreach(println)
...
[Physical execution plan:]
[HiveTableScan [key#9], (MetastoreRelation default, src, None), None]
```

For a wrong query with unresolved attribute(s):

```
scala> hql("explain select kay from src").foreach(println)
...
[Error occurred during query planning: ]
[Unresolved attributes: 'kay, tree:]
[Project ['kay]]
[ LowerCaseSchema ]
[  MetastoreRelation default, src, None]
```

Author: Cheng Lian <lian.cs....@gmail.com>

Closes #1294 from liancheng/safe-explain and squashes the following commits:

4318911 [Cheng Lian] Don't throw TreeNodeException in `execution.ExplainCommand`


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/54488045
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/54488045
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/54488045

Branch: refs/heads/master
Commit: 544880457de556d1ad52e8cb7e1eca19da95f517
Parents: 97a0bfe
Author: Cheng Lian <lian.cs....@gmail.com>
Authored: Thu Jul 3 23:41:54 2014 -0700
Committer: Reynold Xin <r...@apache.org>
Committed: Thu Jul 3 23:41:54 2014 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/execution/commands.scala     | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/54488045/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 acb1b0f..98d2f89 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
@@ -19,9 +19,10 @@ package org.apache.spark.sql.execution
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{SQLContext, Row}
-import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute}
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.{Row, SQLContext}
 
 trait Command {
   /**
@@ -86,8 +87,10 @@ case class ExplainCommand(
   extends LeafNode with Command {
 
   // Run through the optimizer to generate the physical plan.
-  override protected[sql] lazy val sideEffectResult: Seq[String] = {
+  override protected[sql] lazy val sideEffectResult: Seq[String] = try {
     "Physical execution plan:" +: 
context.executePlan(logicalPlan).executedPlan.toString.split("\n")
+  } catch { case cause: TreeNodeException[_] =>
+    "Error occurred during query planning: " +: cause.getMessage.split("\n")
   }
 
   def execute(): RDD[Row] = {

Reply via email to