Skip to content

Commit 4eace4d

Browse files
lianchengrxin
authored andcommitted
[SPARK-13828][SQL] Bring back stack trace of AnalysisException thrown 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 <[email protected]> Closes #11677 from liancheng/analysis-exception-stacktrace.
1 parent ba8c86d commit 4eace4d

File tree

2 files changed

+13
-2
lines changed

2 files changed

+13
-2
lines changed

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,9 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) {
3333

3434
def assertAnalyzed(): Unit = try sqlContext.analyzer.checkAnalysis(analyzed) catch {
3535
case e: AnalysisException =>
36-
throw new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed))
36+
val ae = new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed))
37+
ae.setStackTrace(e.getStackTrace)
38+
throw ae
3739
}
3840

3941
lazy val analyzed: LogicalPlan = sqlContext.analyzer.execute(logical)

sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,8 @@ import scala.util.Random
2525
import org.scalatest.Matchers._
2626

2727
import org.apache.spark.SparkException
28-
import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, OneRowRelation, Union}
28+
import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union}
29+
import org.apache.spark.sql.execution.QueryExecution
2930
import org.apache.spark.sql.execution.aggregate.TungstenAggregate
3031
import org.apache.spark.sql.execution.exchange.{BroadcastExchange, ReusedExchange, ShuffleExchange}
3132
import org.apache.spark.sql.functions._
@@ -1366,4 +1367,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
13661367
// another invalid table name test as below
13671368
intercept[AnalysisException](df.registerTempTable("table!#"))
13681369
}
1370+
1371+
test("assertAnalyzed shouldn't replace original stack trace") {
1372+
val e = intercept[AnalysisException] {
1373+
sqlContext.range(1).select('id as 'a, 'id as 'b).groupBy('a).agg('b)
1374+
}
1375+
1376+
assert(e.getStackTrace.head.getClassName != classOf[QueryExecution].getName)
1377+
}
13691378
}

0 commit comments

Comments
 (0)