From 4318911f878987eae062e19f97f63a6fd52fa35f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 3 Jul 2014 13:42:54 -0700 Subject: [PATCH] Don't throw TreeNodeException in `execution.ExplainCommand` This is a fix for the problem revealed by [PR #1265](https://github.com/apache/spark/pull/1265) --- .../scala/org/apache/spark/sql/execution/commands.scala | 9 ++++++--- 1 file 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 acb1b0f4dc229..98d2f89c8ae71 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] = {