Skip to content

Commit

Permalink
[SPARK-13828][SQL] Bring back stack trace of AnalysisException thrown…
Browse files Browse the repository at this point in the history
… from QueryExecution.assertAnalyzed

PR #11443 added an extra `plan: Option[LogicalPlan]` argument to `AnalysisException` and attached partially analyzed plan to thrown `AnalysisException` in `QueryExecution.assertAnalyzed()`.  However, the original stack trace wasn't properly inherited.  This PR fixes this issue by inheriting the stack trace.

A test case is added to verify that the first entry of `AnalysisException` stack trace isn't from `QueryExecution`.

Author: Cheng Lian <lian@databricks.com>

Closes #11677 from liancheng/analysis-exception-stacktrace.
  • Loading branch information
liancheng authored and rxin committed Mar 12, 2016
1 parent ba8c86d commit 4eace4d
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,9 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) {

def assertAnalyzed(): Unit = try sqlContext.analyzer.checkAnalysis(analyzed) catch {
case e: AnalysisException =>
throw new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed))
val ae = new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed))
ae.setStackTrace(e.getStackTrace)
throw ae
}

lazy val analyzed: LogicalPlan = sqlContext.analyzer.execute(logical)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@ import scala.util.Random
import org.scalatest.Matchers._

import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, OneRowRelation, Union}
import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union}
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.execution.aggregate.TungstenAggregate
import org.apache.spark.sql.execution.exchange.{BroadcastExchange, ReusedExchange, ShuffleExchange}
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -1366,4 +1367,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
// another invalid table name test as below
intercept[AnalysisException](df.registerTempTable("table!#"))
}

test("assertAnalyzed shouldn't replace original stack trace") {
val e = intercept[AnalysisException] {
sqlContext.range(1).select('id as 'a, 'id as 'b).groupBy('a).agg('b)
}

assert(e.getStackTrace.head.getClassName != classOf[QueryExecution].getName)
}
}

0 comments on commit 4eace4d

Please sign in to comment.