Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.util.Utils


/**
Expand All @@ -40,10 +41,12 @@ 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"Undefined function: '$func'. This function is neither a registered temporary function nor " +
s"a permanent function registered in the database '$db'." +
s"${cause.map(th => s"Exception thrown during look up:" +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is too complex for string interpolation. But why not just set the exception's cause?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srowen Thanks.IIUC.Since some times the cause may be undefined.For that case, we do not need to print Exception thrown during look up.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean, why not set the cause on AnalysisException? the cause may be null if the Option is None, that's fine.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Update @srowen Thanks!

s" ${Utils.exceptionString(th)}").getOrElse("")}")

class NoSuchPartitionsException(db: String, table: String, specs: Seq[TablePartitionSpec])
extends AnalysisException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1445,4 +1445,12 @@ abstract class SessionCatalogSuite extends AnalysisTest {
}
}
}

test("SPARK-24544: test print actual failure cause when look up function failed") {
withBasicCatalog { catalog =>
intercept[NoSuchFunctionException] {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this test anything about the cause of the exception? maybe get the cause and assert about its message?

catalog.failFunctionLookup(FunctionIdentifier("failureFunc"), Some(new Exception("Test")))
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -141,21 +142,23 @@ private[sql] class HiveSessionCatalog(
// built-in function.
// Hive is case insensitive.
val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT)
logWarning(s"Encounter a failure during looking up function:" +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Encounter -> Encountered, and the first string doesn't need interpolation

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @srowen Updated!

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
// defined below in hiveFunctions.
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
Expand Down