Skip to content

Commit e236569

Browse files
lianchengrxin
authored andcommitted
[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 <[email protected]> Closes #1294 from liancheng/safe-explain and squashes the following commits: 4318911 [Cheng Lian] Don't throw TreeNodeException in `execution.ExplainCommand` (cherry picked from commit 5448804) Signed-off-by: Reynold Xin <[email protected]>
1 parent 9f7cf5b commit e236569

File tree

1 file changed

+6
-3
lines changed

1 file changed

+6
-3
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,10 @@ package org.apache.spark.sql.execution
1919

2020
import org.apache.spark.annotation.DeveloperApi
2121
import org.apache.spark.rdd.RDD
22-
import org.apache.spark.sql.{SQLContext, Row}
23-
import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute}
22+
import org.apache.spark.sql.catalyst.errors.TreeNodeException
23+
import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow}
2424
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
25+
import org.apache.spark.sql.{Row, SQLContext}
2526

2627
trait Command {
2728
/**
@@ -104,8 +105,10 @@ case class ExplainCommand(
104105
extends LeafNode with Command {
105106

106107
// Run through the optimizer to generate the physical plan.
107-
override protected[sql] lazy val sideEffectResult: Seq[String] = {
108+
override protected[sql] lazy val sideEffectResult: Seq[String] = try {
108109
"Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
110+
} catch { case cause: TreeNodeException[_] =>
111+
"Error occurred during query planning: " +: cause.getMessage.split("\n")
109112
}
110113

111114
def execute(): RDD[Row] = {

0 commit comments

Comments
 (0)