This is an automated email from the ASF dual-hosted git repository.

hvanhovell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new fdeb8d8551e [SPARK-44321][CONNECT] Decouple ParseException from 
AnalysisException
fdeb8d8551e is described below

commit fdeb8d8551e3389e734a8416c63f942c6e20b063
Author: Herman van Hovell <her...@databricks.com>
AuthorDate: Sat Jul 8 15:10:44 2023 -0400

    [SPARK-44321][CONNECT] Decouple ParseException from AnalysisException
    
    ### What changes were proposed in this pull request?
    This PR decouples ParseException from AnalysisException.
    
    ### Why are the changes needed?
    We are moving (parts of) parsing to sql/api to share (datatype) parsing 
between connect and sql.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes and no. This is a breaking change in the sense that the parent class of 
the ParseException changes from AnalysisException to SparkException.
    
    ### How was this patch tested?
    Existing tests.
    
    Closes #41879 from hvanhovell/SPARK-44321.
    
    Authored-by: Herman van Hovell <her...@databricks.com>
    Signed-off-by: Herman van Hovell <her...@databricks.com>
---
 docs/sql-migration-guide.md                        |  1 +
 .../apache/spark/sql/catalyst/trees/origin.scala   |  7 +++
 .../org/apache/spark/sql/AnalysisException.scala   |  6 ++-
 .../apache/spark/sql/catalyst/parser/parsers.scala | 51 ++++++++++++++--------
 .../apache/spark/sql/catalyst/trees/TreeNode.scala |  7 ++-
 .../analyzer-results/csv-functions.sql.out         |  2 +-
 .../analyzer-results/json-functions.sql.out        |  2 +-
 .../sql-tests/results/csv-functions.sql.out        |  2 +-
 .../sql-tests/results/json-functions.sql.out       |  2 +-
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  |  3 +-
 .../apache/spark/sql/LateralColumnAliasSuite.scala |  3 +-
 .../org/apache/spark/sql/ParametersSuite.scala     | 13 +++---
 .../org/apache/spark/sql/SQLInsertTestSuite.scala  |  3 +-
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  5 ++-
 .../spark/sql/connector/AlterTableTests.scala      |  5 ++-
 .../apache/spark/sql/execution/SQLViewSuite.scala  |  6 +--
 .../spark/sql/execution/command/DDLSuite.scala     | 23 +++++-----
 .../execution/command/PlanResolutionSuite.scala    |  6 +--
 .../command/v2/AlterTableReplaceColumnsSuite.scala |  4 +-
 .../sql/sources/CreateTableAsSelectSuite.scala     |  5 +--
 .../org/apache/spark/sql/sources/InsertSuite.scala |  4 +-
 .../org/apache/spark/sql/hive/InsertSuite.scala    |  4 +-
 .../scala/org/apache/spark/sql/hive/UDFSuite.scala |  5 ++-
 .../spark/sql/hive/execution/HiveDDLSuite.scala    | 12 ++---
 .../spark/sql/hive/execution/SQLQuerySuite.scala   |  6 +--
 25 files changed, 110 insertions(+), 77 deletions(-)

diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index b1452d41328..42c36ba6818 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -28,6 +28,7 @@ license: |
 - Since Spark 3.5, Spark thrift server will interrupt task when canceling a 
running statement. To restore the previous behavior, set 
`spark.sql.thriftServer.interruptOnCancel` to `false`.
 - Since Spark 3.5, the Avro will throw `AnalysisException` when reading 
Interval types as Date or Timestamp types, or reading Decimal types with lower 
precision. To restore the legacy behavior, set 
`spark.sql.legacy.avro.allowIncompatibleSchema` to `true`
 - Since Spark 3.5, Row's json and prettyJson methods are moved to `ToJsonUtil`.
+- Since Spark 3.5, ParseException is a subclass of SparkException instead of 
AnalysisException.
 
 ## Upgrading from Spark SQL 3.3 to 3.4
 
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala
index ccf131546de..ec3e627ac95 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala
@@ -42,6 +42,13 @@ case class Origin(
   }
 }
 
+/**
+ * Helper trait for objects that can be traced back to an [[Origin]].
+ */
+trait WithOrigin {
+  def origin: Origin
+}
+
 /**
  * Provides a location for TreeNodes to ask about the context of their origin. 
 For example, which
  * line of code is currently being parsed.
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
index de9b2fa0087..ccc3b5f85f7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
@@ -22,7 +22,7 @@ import scala.collection.JavaConverters._
 import org.apache.spark.{QueryContext, SparkThrowable, SparkThrowableHelper}
 import org.apache.spark.annotation.Stable
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.trees.Origin
+import org.apache.spark.sql.catalyst.trees.{Origin, WithOrigin}
 
 /**
  * Thrown when a query fails to analyze, usually because the query itself is 
invalid.
@@ -40,7 +40,7 @@ class AnalysisException protected[sql] (
     val errorClass: Option[String] = None,
     val messageParameters: Map[String, String] = Map.empty,
     val context: Array[QueryContext] = Array.empty)
-  extends Exception(message, cause.orNull) with SparkThrowable with 
Serializable {
+  extends Exception(message, cause.orNull) with SparkThrowable with 
Serializable with WithOrigin {
 
   def this(
       errorClass: String,
@@ -139,4 +139,6 @@ class AnalysisException protected[sql] (
   override def getErrorClass: String = errorClass.orNull
 
   override def getQueryContext: Array[QueryContext] = context
+
+  override lazy val origin: Origin = Origin(line, startPosition)
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
index fc7a7cfe1f8..d4c1c1d9db6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
@@ -16,15 +16,17 @@
  */
 package org.apache.spark.sql.catalyst.parser
 
+import scala.collection.JavaConverters._
+
 import org.antlr.v4.runtime._
 import org.antlr.v4.runtime.atn.PredictionMode
 import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException}
 import org.antlr.v4.runtime.tree.TerminalNodeImpl
 
-import org.apache.spark.{QueryContext, SparkThrowableHelper}
+import org.apache.spark.{QueryContext, SparkException, SparkThrowable, 
SparkThrowableHelper}
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{AnalysisException, SqlApiConf}
-import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
+import org.apache.spark.sql.SqlApiConf
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, WithOrigin}
 import org.apache.spark.sql.errors.QueryParsingErrors
 import org.apache.spark.sql.types.{DataType, StructType}
 
@@ -92,10 +94,15 @@ abstract class AbstractParser extends 
DataTypeParserInterface with Logging {
         throw e
       case e: ParseException =>
         throw e.withCommand(command)
-      case e: AnalysisException =>
-        val position = Origin(e.line, e.startPosition)
-        throw new ParseException(Option(command), e.message, position, 
position,
-          e.errorClass, e.messageParameters)
+      case e: SparkThrowable with WithOrigin =>
+        throw new ParseException(
+          command = Option(command),
+          message = e.getMessage,
+          start = e.origin,
+          stop = e.origin,
+          errorClass = Option(e.getErrorClass),
+          messageParameters = e.getMessageParameters.asScala.toMap,
+          queryContext = e.getQueryContext)
     }
   }
 
@@ -141,7 +148,7 @@ private[parser] class UpperCaseCharStream(wrapped: 
CodePointCharStream) extends
 }
 
 /**
- * The ParseErrorListener converts parse errors into AnalysisExceptions.
+ * The ParseErrorListener converts parse errors into ParseExceptions.
  */
 case object ParseErrorListener extends BaseErrorListener {
   override def syntaxError(
@@ -171,25 +178,23 @@ case object ParseErrorListener extends BaseErrorListener {
 }
 
 /**
- * A [[ParseException]] is an [[AnalysisException]] that is thrown during the 
parse process. It
+ * A [[ParseException]] is an [[SparkException]] that is thrown during the 
parse process. It
  * contains fields and an extended error message that make reporting and 
diagnosing errors easier.
  */
 class ParseException(
     val command: Option[String],
-    message: String,
+    val message: String,
     val start: Origin,
     val stop: Origin,
-    errorClass: Option[String] = None,
-    messageParameters: Map[String, String] = Map.empty,
-    queryContext: Array[QueryContext] = ParseException.getQueryContext())
-  extends AnalysisException(
+    val errorClass: Option[String] = None,
+    val messageParameters: Map[String, String] = Map.empty,
+    val queryContext: Array[QueryContext] = ParseException.getQueryContext())
+  extends SparkException(
     message,
-    start.line,
-    start.startPosition,
-    None,
-    None,
+    cause = null,
     errorClass,
-    messageParameters) {
+    messageParameters,
+    queryContext) {
 
   def this(errorClass: String, messageParameters: Map[String, String], ctx: 
ParserRuleContext) =
     this(Option(ParserUtils.command(ctx)),
@@ -216,6 +221,14 @@ class ParseException(
       Some(errorClass),
       messageParameters)
 
+  // Methods added to retain compatibility with AnalysisException.
+  @deprecated("Use start.line instead.")
+  def line: Option[Int] = start.line
+  @deprecated("Use start.startPosition instead.")
+  def startPosition: Option[Int] = start.startPosition
+  @deprecated("ParseException is never caused by another exception.")
+  def cause: Option[Throwable] = None
+
   override def getMessage: String = {
     val builder = new StringBuilder
     builder ++= "\n" ++= message
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 150eff5796c..8d3f81666f8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -61,11 +61,14 @@ object AlwaysProcess {
 }
 
 // scalastyle:off
-abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product with 
TreePatternBits {
+abstract class TreeNode[BaseType <: TreeNode[BaseType]]
+  extends Product
+  with TreePatternBits
+  with WithOrigin {
 // scalastyle:on
   self: BaseType =>
 
-  val origin: Origin = CurrentOrigin.get
+  override val origin: Origin = CurrentOrigin.get
 
   /**
    * A mutable map for holding auxiliary information of this tree node. It 
will be carried over
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out
index e59f247d944..cfdbc0ce2e9 100644
--- 
a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out
@@ -36,7 +36,7 @@ org.apache.spark.sql.AnalysisException
 -- !query
 select from_csv('1', 'a InvalidType')
 -- !query analysis
-org.apache.spark.sql.AnalysisException
+org.apache.spark.sql.catalyst.parser.ParseException
 {
   "errorClass" : "PARSE_SYNTAX_ERROR",
   "sqlState" : "42601",
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out
index a3872b4ebc5..79a6bc7de7f 100644
--- 
a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out
@@ -152,7 +152,7 @@ org.apache.spark.sql.AnalysisException
 -- !query
 select from_json('{"a":1}', 'a InvalidType')
 -- !query analysis
-org.apache.spark.sql.AnalysisException
+org.apache.spark.sql.catalyst.parser.ParseException
 {
   "errorClass" : "PARSE_SYNTAX_ERROR",
   "sqlState" : "42601",
diff --git 
a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out 
b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
index 38fcc982b98..50d7f406b02 100644
--- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
@@ -42,7 +42,7 @@ select from_csv('1', 'a InvalidType')
 -- !query schema
 struct<>
 -- !query output
-org.apache.spark.sql.AnalysisException
+org.apache.spark.sql.catalyst.parser.ParseException
 {
   "errorClass" : "PARSE_SYNTAX_ERROR",
   "sqlState" : "42601",
diff --git 
a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out 
b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
index 5bf4c4e44ed..253a255fc16 100644
--- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
@@ -172,7 +172,7 @@ select from_json('{"a":1}', 'a InvalidType')
 -- !query schema
 struct<>
 -- !query output
-org.apache.spark.sql.AnalysisException
+org.apache.spark.sql.catalyst.parser.ParseException
 {
   "errorClass" : "PARSE_SYNTAX_ERROR",
   "sqlState" : "42601",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index b21a8344fd0..2e248a27888 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -29,6 +29,7 @@ import org.apache.spark.{SparkException, 
SparkRuntimeException}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Literal, StructsToJson}
 import org.apache.spark.sql.catalyst.expressions.Cast._
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
@@ -540,7 +541,7 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSparkSession {
     )
 
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         df3.selectExpr("""from_json(value, 'time InvalidType')""")
       },
       errorClass = "PARSE_SYNTAX_ERROR",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
index fac83eab2d2..1e3a0d70c7f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala
@@ -21,6 +21,7 @@ import org.scalactic.source.Position
 import org.scalatest.Tag
 
 import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
ExpressionSet}
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.catalyst.plans.logical.Aggregate
 import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
 import org.apache.spark.sql.internal.SQLConf
@@ -939,7 +940,7 @@ class LateralColumnAliasSuite extends 
LateralColumnAliasSuiteBase {
       lca = "`jy`", windowExprRegex = "\"sum.*\"")
     // this is initially not supported
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql("select name, dept, 1 as n, rank() over " +
           "(partition by dept order by salary rows between n preceding and 
current row) as rank " +
           s"from $testTable where dept in (1, 6)")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala
index 725956e259b..1ab9dce1c94 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import java.time.{Instant, LocalDate, LocalDateTime, ZoneId}
 
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.functions.lit
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
@@ -239,7 +240,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
     val sqlText = "CREATE VIEW v AS SELECT :p AS p"
     val args = Map("p" -> 1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
@@ -254,7 +255,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
     val sqlText = "CREATE VIEW v AS SELECT ? AS p"
     val args = Array(1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
@@ -269,7 +270,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
     val sqlText = "CREATE VIEW v AS WITH cte(a) AS (SELECT (SELECT :p) AS a)  
SELECT a FROM cte"
     val args = Map("p" -> 1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
@@ -284,7 +285,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
     val sqlText = "CREATE VIEW v AS WITH cte(a) AS (SELECT (SELECT ?) AS a)  
SELECT a FROM cte"
     val args = Array(1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
@@ -303,7 +304,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
         |SELECT a FROM cte)""".stripMargin
     val args = Map("p" -> 1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
@@ -322,7 +323,7 @@ class ParametersSuite extends QueryTest with 
SharedSparkSession {
         |SELECT a FROM cte)""".stripMargin
     val args = Array(1)
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         spark.sql(sqlText, args)
       },
       errorClass = 
"UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
index 98e71362fda..09ea35737f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.{SparkConf, SparkNumberFormatException, SparkThrowable}
 import org.apache.spark.sql.catalyst.expressions.Hex
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
@@ -391,7 +392,7 @@ trait SQLInsertTestSuite extends QueryTest with 
SQLTestUtils {
     withTable("t") {
       sql(s"CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY 
(c)")
       checkError(
-        exception = intercept[AnalysisException] {
+        exception = intercept[ParseException] {
           sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)")
         },
         sqlState = None,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 089464dd569..7ad27f05a58 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{GenericRow, 
Hex}
 import org.apache.spark.sql.catalyst.expressions.Cast._
 import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial}
 import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, 
NestedColumnAliasingSuite}
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, Project, 
RepartitionByExpression, Sort}
 import 
org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
 import org.apache.spark.sql.execution.{CommandResultExec, UnionExec}
@@ -593,7 +594,7 @@ class SQLQuerySuite extends QueryTest with 
SharedSparkSession with AdaptiveSpark
   }
 
   test("Allow only a single WITH clause per query") {
-    intercept[AnalysisException] {
+    intercept[ParseException] {
       sql(
         "with q1 as (select * from testData) with q2 as (select * from q1) 
select * from q2")
     }
@@ -1535,7 +1536,7 @@ class SQLQuerySuite extends QueryTest with 
SharedSparkSession with AdaptiveSpark
           .save(path)
 
         // We don't support creating a temporary table while specifying a 
database
-        intercept[AnalysisException] {
+        intercept[ParseException] {
           spark.sql(
             s"""
               |CREATE TEMPORARY VIEW db.t
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
index 7e68add40db..6d430a41f0f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
@@ -22,6 +22,7 @@ import scala.collection.JavaConverters._
 import org.apache.spark.SparkException
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.catalyst.util.quoteIdentifier
 import 
org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership
 import org.apache.spark.sql.connector.catalog.Table
@@ -129,10 +130,10 @@ trait AlterTableTests extends SharedSparkSession with 
QueryErrorsBase {
     withTable(t) {
       sql(s"CREATE TABLE $t (id int, point struct<x: double, y: double>) USING 
$v2Format")
       val e1 =
-        intercept[AnalysisException](sql(s"ALTER TABLE $t ADD COLUMN data 
interval"))
+        intercept[ParseException](sql(s"ALTER TABLE $t ADD COLUMN data 
interval"))
       assert(e1.getMessage.contains("Cannot use interval type in the table 
schema."))
       val e2 =
-        intercept[AnalysisException](sql(s"ALTER TABLE $t ADD COLUMN point.z 
interval"))
+        intercept[ParseException](sql(s"ALTER TABLE $t ADD COLUMN point.z 
interval"))
       assert(e2.getMessage.contains("Cannot use interval type in the table 
schema."))
     }
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
index 35c6ecb1dcd..4dbbed0dd7b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
@@ -320,7 +320,7 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
 
   test("error handling: fail if the temp view name contains the database 
prefix") {
     // Fully qualified table name like "database.table" is not allowed for 
temporary view
-    val e = intercept[AnalysisException] {
+    val e = intercept[ParseException] {
       sql("CREATE OR REPLACE TEMPORARY VIEW default.myabcdview AS SELECT * 
FROM jt")
     }
     assert(e.message.contains(
@@ -329,7 +329,7 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
 
   test("error handling: disallow IF NOT EXISTS for CREATE TEMPORARY VIEW") {
     withTempView("myabcdview") {
-      val e = intercept[AnalysisException] {
+      val e = intercept[ParseException] {
         sql("CREATE TEMPORARY VIEW IF NOT EXISTS myabcdview AS SELECT * FROM 
jt")
       }
       assert(e.message.contains("It is not allowed to define a TEMPORARY view 
with IF NOT EXISTS"))
@@ -483,7 +483,7 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
 
       sql("DROP VIEW testView")
 
-      val e = intercept[AnalysisException] {
+      val e = intercept[ParseException] {
         sql("CREATE OR REPLACE VIEW IF NOT EXISTS testView AS SELECT id FROM 
jt")
       }
       assert(e.message.contains(
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 4e498e6a754..9853d7fa6c4 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, 
QualifiedTableName, Ta
 import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.parser.ParseException
 import 
org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
 import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER
 import org.apache.spark.sql.internal.SQLConf
@@ -960,7 +961,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase 
{
     createTable(catalog, tableIdent)
     val sql1 = "ALTER TABLE dbx.tab1 CLUSTERED BY (blood, lemon, grape) INTO 
11 BUCKETS"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -968,7 +969,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase 
{
       context = ExpectedContext(fragment = sql1, start = 0, stop = 70))
     val sql2 = "ALTER TABLE dbx.tab1 CLUSTERED BY (fuji) SORTED BY (grape) 
INTO 5 BUCKETS"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql2)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -976,7 +977,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase 
{
       context = ExpectedContext(fragment = sql2, start = 0, stop = 72))
     val sql3 = "ALTER TABLE dbx.tab1 NOT CLUSTERED"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql3)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -984,7 +985,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase 
{
       context = ExpectedContext(fragment = sql3, start = 0, stop = 33))
     val sql4 = "ALTER TABLE dbx.tab1 NOT SORTED"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql4)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1000,7 +1001,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
     val sql1 = "ALTER TABLE dbx.tab1 SKEWED BY (dt, country) ON " +
       "(('2008-08-08', 'us'), ('2009-09-09', 'uk'), ('2010-10-10', 'cn'))"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1010,7 +1011,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
     val sql2 = "ALTER TABLE dbx.tab1 SKEWED BY (dt, country) ON " +
       "(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql2)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1019,7 +1020,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
     )
     val sql3 = "ALTER TABLE dbx.tab1 NOT SKEWED"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql3)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1028,7 +1029,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
     )
     val sql4 = "ALTER TABLE dbx.tab1 NOT STORED AS DIRECTORIES"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql4)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1040,7 +1041,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
   test("alter table: add partition is not supported for views") {
     val sql1 = "ALTER VIEW dbx.tab1 ADD IF NOT EXISTS PARTITION (b='2')"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1052,7 +1053,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
   test("alter table: drop partition is not supported for views") {
     val sql1 = "ALTER VIEW dbx.tab1 DROP IF EXISTS PARTITION (b='2')"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -1159,7 +1160,7 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
     // table to alter does not exist
     val sql1 = "ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
index 1f44344cb1a..8eb0d5456c1 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
@@ -407,7 +407,7 @@ class PlanResolutionSuite extends AnalysisTest {
         |OPTIONS (path '/tmp/file')
         |LOCATION '/tmp/file'""".stripMargin
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         parseAndResolve(v2)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0032",
@@ -1395,7 +1395,7 @@ class PlanResolutionSuite extends AnalysisTest {
   test("alter table: alter column action is not specified") {
     val sql = "ALTER TABLE v1Table ALTER COLUMN i"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         parseAndResolve(sql)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -2746,7 +2746,7 @@ class PlanResolutionSuite extends AnalysisTest {
       """CREATE TABLE page_view
         |STORED BY 'storage.handler.class.name' AS SELECT * FROM 
src""".stripMargin
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         extractTableDesc(s4)
       },
       errorClass = "_LEGACY_ERROR_TEMP_0035",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala
index e305ad2de29..599820d7622 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableReplaceColumnsSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.execution.command.v2
 
-import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.command
 
 /**
@@ -32,7 +32,7 @@ class AlterTableReplaceColumnsSuite
     "Support for DEFAULT column values is not implemented yet") {
     val sql1 = "ALTER TABLE t1 REPLACE COLUMNS (ym INT default 1)"
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[ParseException] {
         sql(sql1)
       },
       errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITHOUT_SUGGESTION",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
index b5f6d2f9f68..6f897a9c0b7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.sources
 import java.io.File
 
 import org.apache.spark.SparkException
-import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTableType}
 import org.apache.spark.sql.catalyst.parser.ParseException
@@ -224,7 +223,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with 
SharedSparkSession {
   test("create table using as select - with invalid number of buckets") {
     withTable("t") {
       Seq(0, 100001).foreach(numBuckets => {
-        val e = intercept[AnalysisException] {
+        val e = intercept[ParseException] {
           sql(
             s"""
                |CREATE TABLE t USING PARQUET
@@ -265,7 +264,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with 
SharedSparkSession {
 
       // Over the new limit
       withTable("t") {
-        val e = intercept[AnalysisException](
+        val e = intercept[ParseException](
           sql(createTableSql(path.toURI.toString, maxNrBuckets + 1)))
         assert(
           e.getMessage.contains("Number of buckets should be greater than 0 
but less than "))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
index 71c3301a0eb..48bdd799017 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
@@ -1251,7 +1251,7 @@ class InsertSuite extends DataSourceTest with 
SharedSparkSession {
     withTable("t") {
       withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") {
         checkError(
-          exception = intercept[AnalysisException] {
+          exception = intercept[ParseException] {
             sql("create table t(i boolean, s bigint default 42L) using 
parquet")
           },
           errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION",
@@ -1628,7 +1628,7 @@ class InsertSuite extends DataSourceTest with 
SharedSparkSession {
       withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") {
         sql("create table t(i boolean) using parquet")
         checkError(
-          exception = intercept[AnalysisException] {
+          exception = intercept[ParseException] {
             sql("alter table t add column s bigint default 42L")
           },
           errorClass = "UNSUPPORTED_DEFAULT_VALUE.WITH_SUGGESTION",
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala
index adf0db957e4..420b4fc83ec 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala
@@ -194,7 +194,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton 
with BeforeAndAfter
         """.stripMargin)
     checkAnswer(sql(selQuery), Row(5, 2, 3, 6))
 
-    val e = intercept[AnalysisException] {
+    val e = intercept[ParseException] {
       sql(
         s"""
            |INSERT OVERWRITE TABLE $tableName
@@ -394,7 +394,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton 
with BeforeAndAfter
         sql(s"INSERT INTO TABLE $tableName PARTITION (c=15, b=16) SELECT 13")
 
         // c is defined twice. Analyzer will complain.
-        intercept[AnalysisException] {
+        intercept[ParseException] {
           sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, c=16) 
SELECT 13")
         }
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
index 36450a2c1a2..5dce214a896 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
@@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterEach
 
 import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
 import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.parser.ParseException
 import 
org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 import org.apache.spark.sql.test.SQLTestUtils
@@ -76,7 +77,7 @@ class UDFSuite
 
   test("temporary function: create and drop") {
     withUserDefinedFunction(functionName -> true) {
-      intercept[AnalysisException] {
+      intercept[ParseException] {
         sql(s"CREATE TEMPORARY FUNCTION default.$functionName AS 
'$functionClass'")
       }
       sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'")
@@ -84,7 +85,7 @@ class UDFSuite
         sql(s"SELECT $functionNameLower(value) from $testTableName"),
         expectedDF
       )
-      intercept[AnalysisException] {
+      intercept[ParseException] {
         sql(s"DROP TEMPORARY FUNCTION default.$functionName")
       }
     }
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 86d54ac967b..2555c3f80d0 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -3025,7 +3025,7 @@ class HiveDDLSuite
         """CREATE TABLE targetDsTable LIKE sourceHiveTable USING PARQUET
           |ROW FORMAT SERDE 
'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'""".stripMargin
       checkError(
-        exception = intercept[AnalysisException] {
+        exception = intercept[ParseException] {
           sql(sql1)
         },
         errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -3041,7 +3041,7 @@ class HiveDDLSuite
           |ROW FORMAT SERDE 
'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
           |WITH SERDEPROPERTIES ('test' = 'test')""".stripMargin
       checkError(
-        exception = intercept[AnalysisException] {
+        exception = intercept[ParseException] {
           sql(sql2)
         },
         errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -3057,7 +3057,7 @@ class HiveDDLSuite
           |ROW FORMAT SERDE 
'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
           |WITH SERDEPROPERTIES ('test' = 'test')""".stripMargin
       checkError(
-        exception = intercept[AnalysisException] {
+        exception = intercept[ParseException] {
           sql(sql3)
         },
         errorClass = "_LEGACY_ERROR_TEMP_0047",
@@ -3071,7 +3071,7 @@ class HiveDDLSuite
           |STORED AS INPUTFORMAT 'inFormat' OUTPUTFORMAT 'outFormat'
           |ROW FORMAT SERDE 
'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'""".stripMargin
       checkError(
-        exception = intercept[AnalysisException] {
+        exception = intercept[ParseException] {
           sql(sql4)
         },
         errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -3145,7 +3145,7 @@ class HiveDDLSuite
                  |ROW FORMAT SERDE 
'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
                  |STORED AS $format""".stripMargin
             checkError(
-              exception = intercept[AnalysisException] {
+              exception = intercept[ParseException] {
                 sql(sql1)
               },
               errorClass = "_LEGACY_ERROR_TEMP_0035",
@@ -3184,7 +3184,7 @@ class HiveDDLSuite
                |ROW FORMAT DELIMITED
                |STORED AS PARQUET""".stripMargin
           checkError(
-            exception = intercept[AnalysisException] {
+            exception = intercept[ParseException] {
               sql(sql1)
             },
             errorClass = "_LEGACY_ERROR_TEMP_0035",
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index c59a0da177c..e93576761c8 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -694,14 +694,14 @@ abstract class SQLQuerySuiteBase extends QueryTest with 
SQLTestUtils with TestHi
       }
 
       withTable("gen__tmp") {
-        val e = intercept[AnalysisException] {
+        val e = intercept[ParseException] {
           sql("create table gen__tmp(a int, b string) as select key, value 
from mytable1")
         }.getMessage
         assert(e.contains("Schema may not be specified in a Create Table As 
Select (CTAS)"))
       }
 
       withTable("gen__tmp") {
-        val e = intercept[AnalysisException] {
+        val e = intercept[ParseException] {
           sql(
             """
               |CREATE TABLE gen__tmp
@@ -1231,7 +1231,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with 
SQLTestUtils with TestHi
           .save(path)
 
         // We don't support creating a temporary table while specifying a 
database
-        intercept[AnalysisException] {
+        intercept[ParseException] {
           spark.sql(
             s"""
               |CREATE TEMPORARY VIEW db.t


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org


Reply via email to