Skip to content

Commit 1aadbc4

Browse files
garlandz-dbhvanhovell
authored andcommitted
[SPARK-54887] Add previously removed legacy error class back in
### What changes were proposed in this pull request? This legacy class was removed in a recent commit apache@8acdc7a#diff-cda1fdf98b32b5dcf475ca37f06ca8f621ac2234ad19f473f5b9121ce714f2b9 ### Why are the changes needed? to maintain parity ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Added regression (and previously missing) tests on all these legacy error clases ### Was this patch authored or co-authored using generative AI tooling? Closes apache#54008 from garlandz-db/readd_old_class. Authored-by: Garland Zhang <garland.zhang@databricks.com> Signed-off-by: Herman van Hövell <herman@databricks.com>
1 parent fe9e5c0 commit 1aadbc4

File tree

3 files changed

+94
-45
lines changed

3 files changed

+94
-45
lines changed

sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -138,19 +138,16 @@ class AnalysisException protected (
138138
context = origin.getQueryContext,
139139
cause = cause)
140140

141-
def this(
142-
message: String,
143-
cause: Option[Throwable],
144-
errorClass: Option[String],
141+
private[sql] def this(
142+
errorClass: String,
145143
messageParameters: Map[String, String],
146144
context: Array[QueryContext],
145+
cause: Option[Throwable],
147146
sqlState: Option[String]) =
148147
this(
149-
message = message,
150-
line = None,
151-
startPosition = None,
148+
message = SparkThrowableHelper.getMessage(errorClass, messageParameters),
152149
cause = cause,
153-
errorClass = errorClass,
150+
errorClass = Some(errorClass),
154151
messageParameters = messageParameters,
155152
context = context,
156153
sqlState = sqlState)

sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -257,6 +257,33 @@ class SparkConnectClientSuite extends ConnectFunSuite {
257257
}
258258
}
259259

260+
test("Legacy error class is set as default") {
261+
Seq(
262+
("org.apache.spark.sql.AnalysisException", "_LEGACY_ERROR_TEMP_3100"),
263+
("java.lang.NumberFormatException", "_LEGACY_ERROR_TEMP_3104"),
264+
("java.lang.IllegalArgumentException", "_LEGACY_ERROR_TEMP_3105"),
265+
("java.lang.ArithmeticException", "_LEGACY_ERROR_TEMP_3106"),
266+
("java.lang.UnsupportedOperationException", "_LEGACY_ERROR_TEMP_3107"),
267+
("java.lang.ArrayIndexOutOfBoundsException", "_LEGACY_ERROR_TEMP_3108"),
268+
("java.time.DateTimeException", "_LEGACY_ERROR_TEMP_3109")).foreach {
269+
case (className, legacyErrorClass) =>
270+
val baseParams = GrpcExceptionConverter.ErrorParams(
271+
message = "Test error message",
272+
cause = None,
273+
errorClass = None,
274+
messageParameters = Map.empty,
275+
queryContext = Array.empty,
276+
sqlState = None)
277+
278+
val error = GrpcExceptionConverter
279+
.errorFactory(className)(baseParams)
280+
.asInstanceOf[SparkThrowable]
281+
assert(error.asInstanceOf[Exception].getMessage.contains("Test error message"))
282+
assert(error.getCondition == legacyErrorClass)
283+
assert(error.getSqlState == "XXKCM")
284+
}
285+
}
286+
260287
private case class TestPackURI(
261288
connectionString: String,
262289
isCorrect: Boolean,

sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala

Lines changed: 62 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -268,14 +268,15 @@ private[client] object GrpcExceptionConverter {
268268
errorClass = getErrorClassOrFallback(params),
269269
messageParameters = errorParamsToMessageParameters(params),
270270
queryContext = params.queryContext)),
271-
errorConstructor(params =>
271+
errorConstructor(params => {
272+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3100")
272273
new AnalysisException(
273-
message = params.message,
274-
cause = params.cause,
275-
errorClass = resolveParams(params).errorClass,
276-
messageParameters = errorParamsToMessageParameters(params),
277-
context = params.queryContext,
278-
sqlState = getSqlStateOrFallback(params))),
274+
errorClass = updatedParams.errorClass.get,
275+
messageParameters = updatedParams.messageParameters,
276+
cause = updatedParams.cause,
277+
context = updatedParams.queryContext,
278+
sqlState = getSqlStateOrFallback(updatedParams))
279+
}),
279280
errorConstructor(params =>
280281
new NamespaceAlreadyExistsException(
281282
getErrorClassOrFallback(params),
@@ -304,45 +305,57 @@ private[client] object GrpcExceptionConverter {
304305
getErrorClassOrFallback(params),
305306
errorParamsToMessageParameters(params),
306307
params.cause)),
307-
errorConstructor[NumberFormatException](params =>
308+
errorConstructor[NumberFormatException](params => {
309+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3104")
308310
new SparkNumberFormatException(
309-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3104"),
310-
messageParameters = errorParamsToMessageParameters(params),
311-
params.queryContext,
312-
getSqlStateOrFallback(params))),
313-
errorConstructor[IllegalArgumentException](params =>
311+
errorClass = updatedParams.errorClass.get,
312+
messageParameters = updatedParams.messageParameters,
313+
updatedParams.queryContext,
314+
getSqlStateOrFallback(updatedParams))
315+
}),
316+
errorConstructor[IllegalArgumentException](params => {
317+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3105")
314318
new SparkIllegalArgumentException(
315-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3105"),
316-
messageParameters = errorParamsToMessageParameters(params),
317-
params.queryContext,
319+
errorClass = updatedParams.errorClass.get,
320+
messageParameters = updatedParams.messageParameters,
321+
updatedParams.queryContext,
318322
summary = "",
319-
cause = params.cause.orNull,
320-
getSqlStateOrFallback(params))),
321-
errorConstructor[ArithmeticException](params =>
323+
cause = updatedParams.cause.orNull,
324+
getSqlStateOrFallback(updatedParams))
325+
}),
326+
errorConstructor[ArithmeticException](params => {
327+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3106")
322328
new SparkArithmeticException(
323-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3106"),
324-
messageParameters = errorParamsToMessageParameters(params),
325-
params.queryContext,
326-
getSqlStateOrFallback(params))),
327-
errorConstructor[UnsupportedOperationException](params =>
329+
errorClass = updatedParams.errorClass.get,
330+
messageParameters = updatedParams.messageParameters,
331+
updatedParams.queryContext,
332+
getSqlStateOrFallback(updatedParams))
333+
}),
334+
errorConstructor[UnsupportedOperationException](params => {
335+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3107")
328336
new SparkUnsupportedOperationException(
329-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3107"),
330-
messageParameters = errorParamsToMessageParameters(params),
331-
getSqlStateOrFallback(params))),
332-
errorConstructor[ArrayIndexOutOfBoundsException](params =>
337+
errorClass = updatedParams.errorClass.get,
338+
messageParameters = updatedParams.messageParameters,
339+
getSqlStateOrFallback(updatedParams))
340+
}),
341+
errorConstructor[ArrayIndexOutOfBoundsException](params => {
342+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3108")
333343
new SparkArrayIndexOutOfBoundsException(
334-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3108"),
335-
messageParameters = errorParamsToMessageParameters(params),
336-
params.queryContext,
337-
getSqlStateOrFallback(params))),
338-
errorConstructor[DateTimeException](params =>
344+
errorClass = updatedParams.errorClass.get,
345+
messageParameters = updatedParams.messageParameters,
346+
updatedParams.queryContext,
347+
getSqlStateOrFallback(updatedParams))
348+
}),
349+
errorConstructor[DateTimeException](params => {
350+
val updatedParams = getParamsWithLegacyErrorClass(params, "_LEGACY_ERROR_TEMP_3109")
339351
new SparkDateTimeException(
340-
errorClass = params.errorClass.getOrElse("_LEGACY_ERROR_TEMP_3109"),
341-
messageParameters = errorParamsToMessageParameters(params),
342-
params.queryContext,
352+
errorClass = updatedParams.errorClass.get,
353+
messageParameters = updatedParams.messageParameters,
354+
updatedParams.queryContext,
343355
summary = "",
344356
cause = None,
345-
getSqlStateOrFallback(params))),
357+
getSqlStateOrFallback(updatedParams))
358+
}),
346359
errorConstructor(params =>
347360
new SparkRuntimeException(
348361
getErrorClassOrFallback(params),
@@ -365,6 +378,18 @@ private[client] object GrpcExceptionConverter {
365378
context = params.queryContext,
366379
sqlState = getSqlStateOrFallback(params))))
367380

381+
// Explicitly deal with cases where there are fallback legacy error classes
382+
private def getParamsWithLegacyErrorClass(
383+
params: ErrorParams,
384+
fallbackErrorClass: String): ErrorParams = {
385+
if (params.errorClass.isDefined) {
386+
return params
387+
}
388+
params.copy(
389+
errorClass = Some(fallbackErrorClass),
390+
messageParameters = Map("message" -> params.message))
391+
}
392+
368393
/**
369394
* errorsToThrowable reconstructs the exception based on a list of protobuf messages
370395
* FetchErrorDetailsResponse.Error with un-truncated error messages and server-side stacktrace

0 commit comments

Comments
 (0)