Skip to content

Commit

Permalink
[SPARK-48806] Pass actual exception when url_decode fails
Browse files Browse the repository at this point in the history
  • Loading branch information
wForget committed Jul 4, 2024
1 parent 257a788 commit 5a1c090
Show file tree
Hide file tree
Showing 3 changed files with 14 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ object UrlCodec {
UTF8String.fromString(URLDecoder.decode(src.toString, enc.toString))
} catch {
case e: IllegalArgumentException =>
throw QueryExecutionErrors.illegalUrlError(src)
throw QueryExecutionErrors.illegalUrlError(src, e)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -365,10 +365,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
cause = e)
}

def illegalUrlError(url: UTF8String): Throwable = {
def illegalUrlError(url: UTF8String, e: IllegalArgumentException): Throwable = {
new SparkIllegalArgumentException(
errorClass = "CANNOT_DECODE_URL",
messageParameters = Map("url" -> url.toString)
messageParameters = Map("url" -> url.toString),
cause = e
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql

import org.apache.spark.{SPARK_DOC_ROOT, SparkRuntimeException}
import org.apache.spark.{SPARK_DOC_ROOT, SparkIllegalArgumentException, SparkRuntimeException}
import org.apache.spark.sql.catalyst.expressions.Cast._
import org.apache.spark.sql.execution.FormattedMode
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -1273,4 +1273,13 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession {
)
)
}

test("url_decode exception") {
val e = intercept[SparkIllegalArgumentException] {
sql("select url_decode('https%3A%2F%2spark.apache.org')").collect()
}
assert(e.getCause.isInstanceOf[IllegalArgumentException] &&
e.getCause.getMessage
.startsWith("URLDecoder: Illegal hex characters in escape (%) pattern - "))
}
}

0 comments on commit 5a1c090

Please sign in to comment.