Skip to content

Commit a3feffd

Browse files
panbingkunLuciferYang
authored andcommitted
[SPARK-48585][SQL] Make built-in JdbcDialect's method classifyException throw out the original exception
### What changes were proposed in this pull request? The pr aims to make `built-in` JdbcDialect's method classifyException throw out the `original` exception. ### Why are the changes needed? As discussed in #46912 (comment), the following code: https://github.com/apache/spark/blob/df4156aa3217cf0f58b4c6cbf33c967bb43f7155/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala#L746-L751 have lost the original cause of the error, let's correct it. ### Does this PR introduce _any_ user-facing change? Yes, more accurate error conditions for end users. ### How was this patch tested? - Manually test. - Update existed UT & Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46937 from panbingkun/improve_JDBCTableCatalog. Authored-by: panbingkun <[email protected]> Signed-off-by: yangjie01 <[email protected]>
1 parent c5809b6 commit a3feffd

File tree

14 files changed

+52
-31
lines changed

14 files changed

+52
-31
lines changed

connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -83,14 +83,16 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
8383

8484
def testCreateTableWithProperty(tbl: String): Unit = {}
8585

86-
def checkErrorFailedLoadTable(e: AnalysisException, tbl: String): Unit = {
87-
checkError(
86+
private def checkErrorFailedJDBC(
87+
e: AnalysisException,
88+
errorClass: String,
89+
tbl: String): Unit = {
90+
checkErrorMatchPVals(
8891
exception = e,
89-
errorClass = "FAILED_JDBC.UNCLASSIFIED",
92+
errorClass = errorClass,
9093
parameters = Map(
91-
"url" -> "jdbc:",
92-
"message" -> s"Failed to load table: $tbl"
93-
)
94+
"url" -> "jdbc:.*",
95+
"tableName" -> s"`$tbl`")
9496
)
9597
}
9698

@@ -132,7 +134,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
132134
val e = intercept[AnalysisException] {
133135
sql(s"ALTER TABLE $catalogName.not_existing_table ADD COLUMNS (C4 STRING)")
134136
}
135-
checkErrorFailedLoadTable(e, "not_existing_table")
137+
checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table")
136138
}
137139

138140
test("SPARK-33034: ALTER TABLE ... drop column") {
@@ -154,7 +156,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
154156
val e = intercept[AnalysisException] {
155157
sql(s"ALTER TABLE $catalogName.not_existing_table DROP COLUMN C1")
156158
}
157-
checkErrorFailedLoadTable(e, "not_existing_table")
159+
checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table")
158160
}
159161

160162
test("SPARK-33034: ALTER TABLE ... update column type") {
@@ -170,7 +172,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
170172
val e = intercept[AnalysisException] {
171173
sql(s"ALTER TABLE $catalogName.not_existing_table ALTER COLUMN id TYPE DOUBLE")
172174
}
173-
checkErrorFailedLoadTable(e, "not_existing_table")
175+
checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table")
174176
}
175177

176178
test("SPARK-33034: ALTER TABLE ... rename column") {
@@ -198,7 +200,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
198200
val e = intercept[AnalysisException] {
199201
sql(s"ALTER TABLE $catalogName.not_existing_table RENAME COLUMN ID TO C")
200202
}
201-
checkErrorFailedLoadTable(e, "not_existing_table")
203+
checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table")
202204
}
203205

204206
test("SPARK-33034: ALTER TABLE ... update column nullability") {
@@ -209,7 +211,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
209211
val e = intercept[AnalysisException] {
210212
sql(s"ALTER TABLE $catalogName.not_existing_table ALTER COLUMN ID DROP NOT NULL")
211213
}
212-
checkErrorFailedLoadTable(e, "not_existing_table")
214+
checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table")
213215
}
214216

215217
test("CREATE TABLE with table comment") {
@@ -231,7 +233,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
231233
val e = intercept[AnalysisException] {
232234
sql(s"CREATE TABLE $catalogName.new_table (i INT) TBLPROPERTIES('a'='1')")
233235
}
234-
assert(e.getErrorClass == "FAILED_JDBC.UNCLASSIFIED")
236+
checkErrorFailedJDBC(e, "FAILED_JDBC.CREATE_TABLE", "new_table")
235237
testCreateTableWithProperty(s"$catalogName.new_table")
236238
}
237239
}

sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,8 @@ import org.apache.spark.sql.types.{DataType, MetadataBuilder}
2626
*
2727
* @param dialects List of dialects.
2828
*/
29-
private class AggregatedDialect(dialects: List[JdbcDialect]) extends JdbcDialect {
29+
private class AggregatedDialect(dialects: List[JdbcDialect])
30+
extends JdbcDialect with NoLegacyJDBCError {
3031

3132
require(dialects.nonEmpty)
3233

sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import org.apache.spark.sql.connector.expressions.Expression
3131
import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
3232
import org.apache.spark.sql.types._
3333

34-
private case class DB2Dialect() extends JdbcDialect with SQLConfHelper {
34+
private case class DB2Dialect() extends JdbcDialect with SQLConfHelper with NoLegacyJDBCError {
3535

3636
override def canHandle(url: String): Boolean =
3737
url.toLowerCase(Locale.ROOT).startsWith("jdbc:db2")

sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
2525
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
2626
import org.apache.spark.sql.types._
2727

28-
private case class DatabricksDialect() extends JdbcDialect {
28+
private case class DatabricksDialect() extends JdbcDialect with NoLegacyJDBCError {
2929

3030
override def canHandle(url: String): Boolean = {
3131
url.startsWith("jdbc:databricks")

sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors
2525
import org.apache.spark.sql.types._
2626

2727

28-
private case class DerbyDialect() extends JdbcDialect {
28+
private case class DerbyDialect() extends JdbcDialect with NoLegacyJDBCError {
2929

3030
override def canHandle(url: String): Boolean =
3131
url.toLowerCase(Locale.ROOT).startsWith("jdbc:derby")

sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, N
3737
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
3838
import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DecimalType, MetadataBuilder, ShortType, StringType, TimestampType}
3939

40-
private[sql] case class H2Dialect() extends JdbcDialect {
40+
private[sql] case class H2Dialect() extends JdbcDialect with NoLegacyJDBCError {
4141
override def canHandle(url: String): Boolean =
4242
url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2")
4343

sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -841,6 +841,23 @@ abstract class JdbcDialect extends Serializable with Logging {
841841
metadata: MetadataBuilder): Unit = {}
842842
}
843843

844+
/**
845+
* Make the `classifyException` method throw out the original exception
846+
*/
847+
trait NoLegacyJDBCError extends JdbcDialect {
848+
849+
override def classifyException(
850+
e: Throwable,
851+
errorClass: String,
852+
messageParameters: Map[String, String],
853+
description: String): AnalysisException = {
854+
new AnalysisException(
855+
errorClass = errorClass,
856+
messageParameters = messageParameters,
857+
cause = Some(e))
858+
}
859+
}
860+
844861
/**
845862
* :: DeveloperApi ::
846863
* Registry of dialects that apply to every new jdbc `org.apache.spark.sql.DataFrame`.

sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ import org.apache.spark.sql.jdbc.MsSqlServerDialect.{GEOGRAPHY, GEOMETRY}
3434
import org.apache.spark.sql.types._
3535

3636

37-
private case class MsSqlServerDialect() extends JdbcDialect {
37+
private case class MsSqlServerDialect() extends JdbcDialect with NoLegacyJDBCError {
3838
override def canHandle(url: String): Boolean =
3939
url.toLowerCase(Locale.ROOT).startsWith("jdbc:sqlserver")
4040

sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors
3535
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
3636
import org.apache.spark.sql.types._
3737

38-
private case class MySQLDialect() extends JdbcDialect with SQLConfHelper {
38+
private case class MySQLDialect() extends JdbcDialect with SQLConfHelper with NoLegacyJDBCError {
3939

4040
override def canHandle(url : String): Boolean =
4141
url.toLowerCase(Locale.ROOT).startsWith("jdbc:mysql")

sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.sql.jdbc.OracleDialect._
3030
import org.apache.spark.sql.types._
3131

3232

33-
private case class OracleDialect() extends JdbcDialect with SQLConfHelper {
33+
private case class OracleDialect() extends JdbcDialect with SQLConfHelper with NoLegacyJDBCError {
3434
override def canHandle(url: String): Boolean =
3535
url.toLowerCase(Locale.ROOT).startsWith("jdbc:oracle")
3636

0 commit comments

Comments
 (0)