Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -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}")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -1588,10 +1587,9 @@ class SessionCatalog(
TableFunctionRegistry.builtin.functionExists(name)
}

protected[sql] def failFunctionLookup(
Copy link
Member

Choose a reason for hiding this comment

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

Hmm, where do we set cause? As I see, we always call failFunctionLookup without cause (except for the test case).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We used to pass the cause parameter before, when looking up builtin functions from Hive. The code path is gone now.

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)
}

/**
Expand Down Expand Up @@ -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)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

not quite related but it's obviously redundant to try-catch and re-throw the error. externalCatalog.getFunction can throw NoSuchFunctionExcetion and we should just propagate it.

loadFunctionResources(catalogFunction.resources)
// Please note that qualifiedName is provided by the user. However,
// catalogFunction.identifier.unquotedString is returned by the underlying
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down