aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-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] = {