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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new d01821e0a99 [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api 
should also use AnalysisException
d01821e0a99 is described below

commit d01821e0a99f527b1c151c19b0f44c26121ad57d
Author: Rui Wang <rui.w...@databricks.com>
AuthorDate: Fri Jul 28 22:47:00 2023 -0400

    [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api should also use 
AnalysisException
    
    ### What changes were proposed in this pull request?
    
    https://github.com/apache/spark/pull/42130 moved AnalysisException to 
sql/api. So we can use AnalysisException for those errors that were moved into 
sql/api but changed to SparkException.
    
    ### Why are the changes needed?
    
    Restore to previous behavior.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No. This PR recovers the behaviors to the past which users should see 
AnalysisException upon many cases.
    
    ### How was this patch tested?
    
    Existing UT.
    
    Closes #42190 from amaliujia/convert_back_errors.
    
    Authored-by: Rui Wang <rui.w...@databricks.com>
    Signed-off-by: Herman van Hovell <her...@databricks.com>
    (cherry picked from commit 3761b7d65bd1e21b5a7c5966c2d03a0fe4a0b982)
    Signed-off-by: Herman van Hovell <her...@databricks.com>
---
 .../apache/spark/sql/errors/DataTypeErrors.scala   | 53 ++++++++++------------
 .../spark/sql/errors/QueryCompilationErrors.scala  |  7 +--
 .../apache/spark/sql/types/StructTypeSuite.scala   | 26 +++++------
 .../apache/spark/sql/CharVarcharTestSuite.scala    | 28 +++++++-----
 .../org/apache/spark/sql/DataFrameSuite.scala      |  2 +-
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  |  6 +--
 .../spark/sql/connector/AlterTableTests.scala      |  2 +-
 .../sql/errors/QueryCompilationErrorsSuite.scala   |  4 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala      |  4 +-
 .../sql/test/DataFrameReaderWriterSuite.scala      | 12 ++---
 10 files changed, 68 insertions(+), 76 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala
index fcc3086b573..7a34a386cd8 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala
@@ -17,8 +17,10 @@
 package org.apache.spark.sql.errors
 
 import org.apache.spark.{SparkArithmeticException, SparkException, 
SparkIllegalArgumentException, SparkNumberFormatException, 
SparkRuntimeException, SparkUnsupportedOperationException}
+import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext}
 import org.apache.spark.sql.catalyst.util.QuotingUtils
+import org.apache.spark.sql.catalyst.util.QuotingUtils.toSQLSchema
 import org.apache.spark.sql.types.{DataType, Decimal, StringType}
 import org.apache.spark.unsafe.types.UTF8String
 
@@ -97,49 +99,45 @@ private[sql] object DataTypeErrors extends 
DataTypeErrorsBase {
   }
 
   def schemaFailToParseError(schema: String, e: Throwable): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "INVALID_SCHEMA.PARSE_ERROR",
       messageParameters = Map(
-        "inputSchema" -> QuotingUtils.toSQLSchema(schema),
+        "inputSchema" -> toSQLSchema(schema),
         "reason" -> e.getMessage
       ),
-      cause = e)
+      cause = Some(e))
   }
 
   def invalidDayTimeIntervalType(startFieldName: String, endFieldName: 
String): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1224",
       messageParameters = Map(
         "startFieldName" -> startFieldName,
-        "endFieldName" -> endFieldName),
-      cause = null)
+        "endFieldName" -> endFieldName))
   }
 
   def invalidDayTimeField(field: Byte, supportedIds: Seq[String]): Throwable = 
{
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1223",
       messageParameters = Map(
         "field" -> field.toString,
-        "supportedIds" -> supportedIds.mkString(", ")),
-      cause = null)
+        "supportedIds" -> supportedIds.mkString(", ")))
   }
 
   def invalidYearMonthField(field: Byte, supportedIds: Seq[String]): Throwable 
= {
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1225",
       messageParameters = Map(
         "field" -> field.toString,
-        "supportedIds" -> supportedIds.mkString(", ")),
-      cause = null)
+        "supportedIds" -> supportedIds.mkString(", ")))
   }
 
   def decimalCannotGreaterThanPrecisionError(scale: Int, precision: Int): 
Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1228",
       messageParameters = Map(
         "scale" -> scale.toString,
-        "precision" -> precision.toString),
-      cause = null)
+        "precision" -> precision.toString))
   }
 
   def negativeScaleNotAllowedError(scale: Int): Throwable = {
@@ -150,10 +148,9 @@ private[sql] object DataTypeErrors extends 
DataTypeErrorsBase {
   }
 
   def attributeNameSyntaxError(name: String): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1049",
-      messageParameters = Map("name" -> name),
-      cause = null)
+      messageParameters = Map("name" -> name))
   }
 
   def cannotMergeIncompatibleDataTypesError(left: DataType, right: DataType): 
Throwable = {
@@ -182,13 +179,12 @@ private[sql] object DataTypeErrors extends 
DataTypeErrorsBase {
   }
 
   def invalidFieldName(fieldName: Seq[String], path: Seq[String], context: 
Origin): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "INVALID_FIELD_NAME",
       messageParameters = Map(
         "fieldName" -> toSQLId(fieldName),
         "path" -> toSQLId(path)),
-      cause = null,
-      context = context.getQueryContext)
+      origin = context)
   }
 
   def unscaledValueTooLargeForPrecisionError(
@@ -241,13 +237,12 @@ private[sql] object DataTypeErrors extends 
DataTypeErrorsBase {
 
   def ambiguousColumnOrFieldError(
       name: Seq[String], numMatches: Int, context: Origin): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "AMBIGUOUS_COLUMN_OR_FIELD",
       messageParameters = Map(
         "name" -> toSQLId(name),
         "n" -> numMatches.toString),
-      cause = null,
-      context = context.getQueryContext)
+      origin = context)
   }
 
   def castingCauseOverflowError(t: String, from: DataType, to: DataType): 
ArithmeticException = {
@@ -283,16 +278,14 @@ private[sql] object DataTypeErrors extends 
DataTypeErrorsBase {
   }
 
   def charOrVarcharTypeAsStringUnsupportedError(): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING",
-      messageParameters = Map.empty,
-      cause = null)
+      messageParameters = Map.empty)
   }
 
   def userSpecifiedSchemaUnsupportedError(operation: String): Throwable = {
-    new SparkException(
+    new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1189",
-      messageParameters = Map("operation" -> operation),
-      cause = null)
+      messageParameters = Map("operation" -> operation))
   }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 79b88fde483..010b4ec92a9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -1921,12 +1921,7 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
 
   def ambiguousColumnOrFieldError(
       name: Seq[String], numMatches: Int, context: Origin): Throwable = {
-    new AnalysisException(
-      errorClass = "AMBIGUOUS_COLUMN_OR_FIELD",
-      messageParameters = Map(
-        "name" -> toSQLId(name),
-        "n" -> numMatches.toString),
-      origin = context)
+    DataTypeErrors.ambiguousColumnOrFieldError(name, numMatches, context)
   }
 
   def ambiguousReferenceError(name: String, ambiguousReferences: 
Seq[Attribute]): Throwable = {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
index 623c6c69165..fefe9145ee3 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala
@@ -324,7 +324,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
     check(Seq("s1", "S12"), Some(Seq("s1") -> StructField("s12", IntegerType)))
     caseSensitiveCheck(Seq("s1", "S12"), None)
     check(Seq("S1.non_exist"), None)
-    var e = intercept[SparkException] {
+    var e = intercept[AnalysisException] {
       check(Seq("S1", "S12", "S123"), None)
     }
     checkError(
@@ -335,17 +335,17 @@ class StructTypeSuite extends SparkFunSuite with 
SQLHelper {
         "path" -> "`s1`.`s12`"))
 
     // ambiguous name
-    var e2 = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       check(Seq("S2", "x"), None)
     }
     checkError(
-      exception = e2,
+      exception = e,
       errorClass = "AMBIGUOUS_COLUMN_OR_FIELD",
       parameters = Map("name" -> "`S2`.`x`", "n" -> "2"))
     caseSensitiveCheck(Seq("s2", "x"), Some(Seq("s2") -> StructField("x", 
IntegerType)))
 
     // simple map type
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       check(Seq("m1", "key"), None)
     }
     checkError(
@@ -356,7 +356,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
         "path" -> "`m1`"))
     checkCollection(Seq("m1", "key"), Some(Seq("m1") -> StructField("key", 
IntegerType, false)))
     checkCollection(Seq("M1", "value"), Some(Seq("m1") -> StructField("value", 
IntegerType)))
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("M1", "key", "name"), None)
     }
     checkError(
@@ -365,7 +365,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
       parameters = Map(
         "fieldName" -> "`M1`.`key`.`name`",
         "path" -> "`m1`.`key`"))
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("M1", "value", "name"), None)
     }
     checkError(
@@ -382,7 +382,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
     checkCollection(Seq("M2", "value", "b"),
       Some(Seq("m2", "value") -> StructField("b", IntegerType)))
     checkCollection(Seq("M2", "value", "non_exist"), None)
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("m2", "key", "A", "name"), None)
     }
     checkError(
@@ -391,7 +391,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
       parameters = Map(
         "fieldName" -> "`m2`.`key`.`A`.`name`",
         "path" -> "`m2`.`key`.`a`"))
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("M2", "value", "b", "name"), None)
     }
     checkError(
@@ -401,7 +401,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
         "fieldName" -> "`M2`.`value`.`b`.`name`",
         "path" -> "`m2`.`value`.`b`"))
     // simple array type
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       check(Seq("A1", "element"), None)
     }
     checkError(
@@ -411,7 +411,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
         "fieldName" -> "`A1`.`element`",
         "path" -> "`a1`"))
     checkCollection(Seq("A1", "element"), Some(Seq("a1") -> 
StructField("element", IntegerType)))
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("A1", "element", "name"), None)
     }
     checkError(
@@ -425,7 +425,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
     checkCollection(Seq("A2", "element", "C"),
       Some(Seq("a2", "element") -> StructField("c", IntegerType)))
     checkCollection(Seq("A2", "element", "non_exist"), None)
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("a2", "element", "C", "name"), None)
     }
     checkError(
@@ -439,7 +439,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
     checkCollection(Seq("M3", "value", "value", "MA"),
       Some(Seq("m3", "value", "value") -> StructField("ma", IntegerType)))
     checkCollection(Seq("M3", "value", "value", "non_exist"), None)
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("M3", "value", "value", "MA", "name"), None)
     }
     checkError(
@@ -453,7 +453,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper {
     checkCollection(Seq("A3", "element", "element", "D"),
       Some(Seq("a3", "element", "element") -> StructField("d", IntegerType)))
     checkCollection(Seq("A3", "element", "element", "non_exist"), None)
-    e = intercept[SparkException] {
+    e = intercept[AnalysisException] {
       checkCollection(Seq("A3", "element", "element", "D", "name"), None)
     }
     checkError(
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala
index 55009f2416c..4a7632486c0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala
@@ -794,11 +794,15 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
 
   test("invalidate char/varchar in functions") {
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         sql("""SELECT from_json('{"a": "str"}', 'a CHAR(5)')""")
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING",
-      parameters = Map.empty
+      parameters = Map.empty,
+      context = ExpectedContext(
+        fragment = "from_json('{\"a\": \"str\"}', 'a CHAR(5)')",
+        start = 7,
+        stop = 44)
     )
     withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) {
       val df = sql("""SELECT from_json('{"a": "str"}', 'a CHAR(5)')""")
@@ -812,19 +816,19 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
     val df = spark.range(10).map(_.toString).toDF()
     val schema = new StructType().add("id", CharType(5))
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.createDataFrame(df.collectAsList(), schema)
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
     )
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.createDataFrame(df.rdd, schema)
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
     )
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.createDataFrame(df.toJavaRDD, schema)
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
@@ -838,12 +842,12 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
 
   test("invalidate char/varchar in spark.read.schema") {
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.read.schema(new StructType().add("id", CharType(5)))
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING")
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.read.schema("id char(5)")
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
@@ -880,13 +884,13 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
 
   test("invalidate char/varchar in udf's result type") {
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.udf.register("testchar", () => "B", VarcharType(1))
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
     )
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.udf.register("testchar2", (x: String) => x, VarcharType(1))
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
@@ -905,13 +909,13 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
 
   test("invalidate char/varchar in spark.readStream.schema") {
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.readStream.schema(new StructType().add("id", CharType(5)))
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
     )
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         spark.readStream.schema("id char(5)")
       },
       errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING"
@@ -934,7 +938,7 @@ class BasicCharVarcharTestSuite extends QueryTest with 
SharedSparkSession {
       sql("CREATE TABLE t(c char(10), v varchar(255)) USING parquet")
       sql("INSERT INTO t VALUES('spark', 'awesome')")
       val df = sql("SELECT * FROM t")
-      checkError(exception = intercept[SparkException] {
+      checkError(exception = intercept[AnalysisException] {
         df.to(newSchema)
       }, errorClass = "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", parameters = 
Map.empty)
       withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index e1c355dc019..2eba9f18109 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1721,7 +1721,7 @@ class DataFrameSuite extends QueryTest
 
       def checkSyntaxError(name: String): Unit = {
         checkError(
-          exception = intercept[SparkException] {
+          exception = intercept[org.apache.spark.sql.AnalysisException] {
             df(name)
           },
           errorClass = "_LEGACY_ERROR_TEMP_1049",
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..187fab75f63 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
@@ -1150,7 +1150,7 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSparkSession {
     val invalidJsonSchema = """{"fields": [{"a":123}], "type": "struct"}"""
     val invalidJsonSchemaReason = "Failed to convert the JSON string 
'{\"a\":123}' to a field."
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         df.select(from_json($"json", invalidJsonSchema, Map.empty[String, 
String])).collect()
       },
       errorClass = "INVALID_SCHEMA.PARSE_ERROR",
@@ -1165,7 +1165,7 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSparkSession {
       "was expecting (JSON String, Number, Array, Object or token 'null', 
'true' or 'false')\n " +
       "at [Source: (String)\"MAP<INT, cow>\"; line: 1, column: 4]"
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         df.select(from_json($"json", invalidDataType, Map.empty[String, 
String])).collect()
       },
       errorClass = "INVALID_SCHEMA.PARSE_ERROR",
@@ -1180,7 +1180,7 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSparkSession {
       "was expecting (JSON String, Number, Array, Object or token 'null', 
'true' or 'false')\n" +
       " at [Source: (String)\"x INT, a cow\"; line: 1, column: 2]"
     checkError(
-      exception = intercept[SparkException] {
+      exception = intercept[AnalysisException] {
         df.select(from_json($"json", invalidTableSchema, Map.empty[String, 
String])).collect()
       },
       errorClass = "INVALID_SCHEMA.PARSE_ERROR",
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 248783dd6c6..a18c767570f 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
@@ -838,7 +838,7 @@ trait AlterTableTests extends SharedSparkSession with 
QueryErrorsBase {
       assert(e2.getMessage.contains("Missing field point.non_exist"))
 
       // `AlterTable.resolved` checks column existence.
-      intercept[SparkException](
+      intercept[AnalysisException](
         sql(s"ALTER TABLE $t ALTER COLUMN a.y AFTER x"))
     }
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
index 04f9da5312c..7f938deaaa6 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{SPARK_DOC_ROOT, SparkException}
+import org.apache.spark.SPARK_DOC_ROOT
 import org.apache.spark.sql.{AnalysisException, ClassData, 
IntegratedUDFTestUtils, QueryTest, Row}
 import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
 import org.apache.spark.sql.catalyst.parser.ParseException
@@ -532,7 +532,7 @@ class QueryCompilationErrorsSuite
 
       val query = "ALTER TABLE t CHANGE COLUMN c.X COMMENT 'new comment'"
       checkError(
-        exception = intercept[SparkException] {
+        exception = intercept[AnalysisException] {
           sql(query)
         },
         errorClass = "AMBIGUOUS_COLUMN_OR_FIELD",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 94c344fc384..93b6652d516 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -1364,12 +1364,12 @@ class JDBCSuite extends QueryTest with 
SharedSparkSession {
     val schema = StructType(Seq(StructField("name", StringType, false, 
defaultMetadata),
       StructField("theid", IntegerType, false, defaultMetadata)))
     val parts = Array[String]("THEID < 2", "THEID >= 2")
-    val e1 = intercept[SparkException] {
+    val e1 = intercept[AnalysisException] {
       spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, 
new Properties())
     }.getMessage
     assert(e1.contains("User specified schema not supported with `jdbc`"))
 
-    val e2 = intercept[SparkException] {
+    val e2 = intercept[AnalysisException] {
       spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new 
Properties())
     }.getMessage
     assert(e2.contains("User specified schema not supported with `jdbc`"))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
index 967d384d979..17348fe2dcb 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
@@ -32,7 +32,7 @@ import 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
 import org.apache.parquet.schema.Type.Repetition
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.{SparkContext, SparkException, TestUtils}
+import org.apache.spark.{SparkContext, TestUtils}
 import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
 import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol
 import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
@@ -633,12 +633,12 @@ class DataFrameReaderWriterSuite extends QueryTest with 
SharedSparkSession with
     testRead(Option(dir).map(spark.read.text).get, data, textSchema)
 
     // Reader, with user specified schema, should just apply user schema on 
the file data
-    val e = intercept[SparkException] { 
spark.read.schema(userSchema).textFile() }
+    val e = intercept[AnalysisException] { 
spark.read.schema(userSchema).textFile() }
     assert(e.getMessage.toLowerCase(Locale.ROOT).contains(
       "user specified schema not supported"))
-    intercept[SparkException] { spark.read.schema(userSchema).textFile(dir) }
-    intercept[SparkException] { spark.read.schema(userSchema).textFile(dir, 
dir) }
-    intercept[SparkException] { 
spark.read.schema(userSchema).textFile(Seq(dir, dir): _*) }
+    intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir) 
}
+    intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir, 
dir) }
+    intercept[AnalysisException] { 
spark.read.schema(userSchema).textFile(Seq(dir, dir): _*) }
   }
 
   test("csv - API and behavior regarding schema") {
@@ -969,7 +969,7 @@ class DataFrameReaderWriterSuite extends QueryTest with 
SharedSparkSession with
   test("SPARK-16848: table API throws an exception for user specified schema") 
{
     withTable("t") {
       val schema = StructType(StructField("a", StringType) :: Nil)
-      val e = intercept[SparkException] {
+      val e = intercept[AnalysisException] {
         spark.read.schema(schema).table("t")
       }.getMessage
       assert(e.contains("User specified schema not supported with `table`"))


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

Reply via email to