diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 805f3080c8472..5813c2fa4dad3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -74,19 +74,17 @@ case class NoSuchPartitionException( case class NoSuchPermanentFunctionException(db: String, func: String) extends AnalysisException(s"Function '$func' not found in database '$db'") -case class NoSuchFunctionException( - override val message: String, - override val cause: Option[Throwable]) - extends AnalysisException(message, cause = cause) { +case class NoSuchFunctionException(override val message: String) + extends AnalysisException(message) { - def this(db: String, func: String, cause: Option[Throwable] = None) = { + def this(db: String, func: String) = { this(s"Undefined function: '$func'. " + - s"This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.", cause = cause) + "This function is neither a registered temporary function nor " + + s"a permanent function registered in the database '$db'.") } def this(identifier: Identifier) = { - this(s"Undefined function: ${identifier.quoted}", cause = None) + this(s"Undefined function: ${identifier.quoted}") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 5d5d8b202c533..1ada2ffa4fc15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -1588,10 +1587,9 @@ class SessionCatalog( TableFunctionRegistry.builtin.functionExists(name) } - protected[sql] def failFunctionLookup( - name: FunctionIdentifier, cause: Option[Throwable] = None): Nothing = { + protected[sql] def failFunctionLookup(name: FunctionIdentifier): Nothing = { throw new NoSuchFunctionException( - db = name.database.getOrElse(getCurrentDatabase), func = name.funcName, cause) + db = name.database.getOrElse(getCurrentDatabase), func = name.funcName) } /** @@ -1732,11 +1730,7 @@ class SessionCatalog( // The function has not been loaded to the function registry, which means // that the function is a persistent function (if it actually has been registered // in the metastore). We need to first put the function in the function registry. - val catalogFunction = try { - externalCatalog.getFunction(db, funcName) - } catch { - case _: AnalysisException => failFunctionLookup(qualifiedIdent) - } + val catalogFunction = externalCatalog.getFunction(db, funcName) loadFunctionResources(catalogFunction.resources) // Please note that qualifiedName is provided by the user. However, // catalogFunction.identifier.unquotedString is returned by the underlying diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 62491e04831cc..6ed14bd641653 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -1728,20 +1728,6 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { } } - test("SPARK-24544: test print actual failure cause when look up function failed") { - withBasicCatalog { catalog => - val cause = intercept[NoSuchFunctionException] { - catalog.failFunctionLookup(FunctionIdentifier("failureFunc"), - Some(new Exception("Actual error"))) - } - - // fullStackTrace will be printed, but `cause.getMessage` has been - // override in `AnalysisException`,so here we get the root cause - // exception message for check. - assert(cause.cause.get.getMessage.contains("Actual error")) - } - } - test("expire table relation cache if TTL is configured") { case class TestCommand() extends LeafCommand