aboutsummaryrefslogtreecommitdiff
path: root/sql/core
diff options
context:
space:
mode:
authorCheng Lian <lian.cs.zju@gmail.com>2014-07-03 23:41:54 -0700
committerReynold Xin <rxin@apache.org>2014-07-03 23:41:54 -0700
commit544880457de556d1ad52e8cb7e1eca19da95f517 (patch)
treec8be1b1595362ca6ee15dc71eddf9926c6d0cce8 /sql/core
parent97a0bfe1c0261384f09d53f9350de52fb6446d59 (diff)
downloadspark-544880457de556d1ad52e8cb7e1eca19da95f517.tar.gz
spark-544880457de556d1ad52e8cb7e1eca19da95f517.tar.bz2
spark-544880457de556d1ad52e8cb7e1eca19da95f517.zip
[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.zju@gmail.com> Closes #1294 from liancheng/safe-explain and squashes the following commits: 4318911 [Cheng Lian] Don't throw TreeNodeException in `execution.ExplainCommand`
Diffstat (limited to 'sql/core')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala9
1 files changed, 6 insertions, 3 deletions
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 acb1b0f4dc..98d2f89c8a 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] = {