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 @@ -1105,8 +1105,9 @@ class SessionCatalog(
!hiveFunctions.contains(name.funcName.toLowerCase(Locale.ROOT))
}

protected def failFunctionLookup(name: String): Nothing = {
throw new NoSuchFunctionException(db = currentDb, func = name)
protected def failFunctionLookup(name: FunctionIdentifier): Nothing = {
throw new NoSuchFunctionException(
db = name.database.getOrElse(getCurrentDatabase), func = name.funcName)
}

/**
Expand All @@ -1128,7 +1129,7 @@ class SessionCatalog(
qualifiedName.database.orNull,
qualifiedName.identifier)
} else {
failFunctionLookup(name.funcName)
failFunctionLookup(name)
}
}
}
Expand Down Expand Up @@ -1158,8 +1159,8 @@ class SessionCatalog(
}

// If the name itself is not qualified, add the current database to it.
val database = name.database.orElse(Some(currentDb)).map(formatDatabaseName)
val qualifiedName = name.copy(database = database)
val database = formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
val qualifiedName = name.copy(database = Some(database))

if (functionRegistry.functionExists(qualifiedName.unquotedString)) {
// This function has been already loaded into the function registry.
Expand All @@ -1172,10 +1173,10 @@ class SessionCatalog(
// in the metastore). We need to first put the function in the FunctionRegistry.
// TODO: why not just check whether the function exists first?
val catalogFunction = try {
externalCatalog.getFunction(currentDb, name.funcName)
externalCatalog.getFunction(database, name.funcName)
} catch {
case e: AnalysisException => failFunctionLookup(name.funcName)
case e: NoSuchPermanentFunctionException => failFunctionLookup(name.funcName)
case _: AnalysisException => failFunctionLookup(name)
case _: NoSuchPermanentFunctionException => failFunctionLookup(name)
}
loadFunctionResources(catalogFunction.resources)
// Please note that qualifiedName is provided by the user. However,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,20 +140,20 @@ private[sql] class HiveSessionCatalog(
// Hive is case insensitive.
val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT)
if (!hiveFunctions.contains(functionName)) {
failFunctionLookup(funcName.unquotedString)
failFunctionLookup(funcName)
}

// 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.unquotedString))
failFunctionLookup(funcName))
} 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.unquotedString)
case NonFatal(e) => failFunctionLookup(funcName)
}
}
val className = functionInfo.getFunctionClass.getName
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.catalyst.plans.logical.Project
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -590,6 +591,25 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
}
}
}

test("Call the function registered in the not-current database") {
Seq("true", "false").foreach { caseSensitive =>
withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
withDatabase("dAtABaSe1") {
sql("CREATE DATABASE dAtABaSe1")
withUserDefinedFunction("dAtABaSe1.test_avg" -> false) {
sql(s"CREATE FUNCTION dAtABaSe1.test_avg AS '${classOf[GenericUDAFAverage].getName}'")
checkAnswer(sql("SELECT dAtABaSe1.test_avg(1)"), Row(1.0))
}
val message = intercept[AnalysisException] {
sql("SELECT dAtABaSe1.unknownFunc(1)")
}.getMessage
assert(message.contains("Undefined function: 'unknownFunc'") &&
message.contains("nor a permanent function registered in the database 'dAtABaSe1'"))
}
}
}
}
}

class TestPair(x: Int, y: Int) extends Writable with Serializable {
Expand Down