Skip to content
Snippets Groups Projects
Commit 54488045 authored by Cheng Lian's avatar Cheng Lian Committed by Reynold Xin
Browse files

[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`
parent 97a0bfe1
No related branches found
No related tags found
No related merge requests found
...@@ -19,9 +19,10 @@ package org.apache.spark.sql.execution ...@@ -19,9 +19,10 @@ package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute} import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.{Row, SQLContext}
trait Command { trait Command {
/** /**
...@@ -86,8 +87,10 @@ case class ExplainCommand( ...@@ -86,8 +87,10 @@ case class ExplainCommand(
extends LeafNode with Command { extends LeafNode with Command {
// Run through the optimizer to generate the physical plan. // 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") "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] = { def execute(): RDD[Row] = {
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment