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 f5aae60431c15..8bf6f69f3b17a 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 @@ -40,10 +40,10 @@ class NoSuchPartitionException( class NoSuchPermanentFunctionException(db: String, func: String) extends AnalysisException(s"Function '$func' not found in database '$db'") -class NoSuchFunctionException(db: String, func: String) +class NoSuchFunctionException(db: String, func: String, cause: Option[Throwable] = None) extends AnalysisException( s"Undefined function: '$func'. This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.") + s"a permanent function registered in the database '$db'.", cause = cause) class NoSuchPartitionsException(db: String, table: String, specs: Seq[TablePartitionSpec]) extends AnalysisException( 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 b09b81eabf60d..a54e5e274e1d8 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 @@ -1209,9 +1209,10 @@ class SessionCatalog( databaseExists(db) && externalCatalog.functionExists(db, name.funcName) } - protected def failFunctionLookup(name: FunctionIdentifier): Nothing = { + protected[sql] def failFunctionLookup( + name: FunctionIdentifier, cause: Option[Throwable] = None): Nothing = { throw new NoSuchFunctionException( - db = name.database.getOrElse(getCurrentDatabase), func = name.funcName) + db = name.database.getOrElse(getCurrentDatabase), func = name.funcName, cause) } /** 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 50496a0410528..88cad8436416b 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 @@ -1445,4 +1445,18 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } } + + 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")) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index de41bb418181d..a534fc4b9c14e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, DoubleType} +import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( @@ -141,8 +142,10 @@ private[sql] class HiveSessionCatalog( // built-in function. // Hive is case insensitive. val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT) + logWarning("Encountered a failure during looking up function:" + + s" ${Utils.exceptionString(error)}") if (!hiveFunctions.contains(functionName)) { - failFunctionLookup(funcName) + failFunctionLookup(funcName, Some(error)) } // TODO: Remove this fallback path once we implement the list of fallback functions @@ -150,12 +153,12 @@ private[sql] class HiveSessionCatalog( val functionInfo = { try { Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( - failFunctionLookup(funcName)) + failFunctionLookup(funcName, Some(error))) } catch { // If HiveFunctionRegistry.getFunctionInfo throws an exception, // we are failing to load a Hive builtin function, which means that // the given function is not a Hive builtin function. - case NonFatal(e) => failFunctionLookup(funcName) + case NonFatal(e) => failFunctionLookup(funcName, Some(e)) } } val className = functionInfo.getFunctionClass.getName