Skip to content

Commit 7251be9

Browse files
committed
fix.
1 parent bd5ae26 commit 7251be9

File tree

3 files changed

+37
-76
lines changed

3 files changed

+37
-76
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 19 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface}
4141
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View}
4242
import org.apache.spark.sql.catalyst.util.StringUtils
4343
import org.apache.spark.sql.internal.SQLConf
44+
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
4445
import org.apache.spark.sql.types.StructType
4546
import org.apache.spark.util.Utils
4647

@@ -129,6 +130,13 @@ class SessionCatalog(
129130
if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
130131
}
131132

133+
/**
134+
* Checks whether the Hive metastore is being used
135+
*/
136+
private def isUsingHiveMetastore: Boolean = {
137+
conf.getConf(CATALOG_IMPLEMENTATION).toLowerCase(Locale.ROOT) == "hive"
138+
}
139+
132140
private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = {
133141
val cacheSize = conf.tableRelationCacheSize
134142
CacheBuilder.newBuilder().maximumSize(cacheSize).build[QualifiedTableName, LogicalPlan]()
@@ -1094,27 +1102,24 @@ class SessionCatalog(
10941102
// ----------------------------------------------------------------
10951103

10961104
/**
1097-
* Construct a [[FunctionBuilder]] based on the provided class that represents a function.
1098-
*
1099-
* This performs reflection to decide what type of [[Expression]] to return in the builder.
1105+
* Constructs a [[FunctionBuilder]] based on the provided class that represents a function.
11001106
*/
11011107
protected def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
11021108
val clazz = Utils.classForName(functionClassName)
11031109
(children: Seq[Expression]) => {
11041110
try {
11051111
makeFunctionExpression(name, Utils.classForName(functionClassName), children).getOrElse {
1106-
throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
1112+
val extraMsg =
1113+
if (!isUsingHiveMetastore) "Use sparkSession.udf.register(...) instead." else ""
1114+
throw new AnalysisException(
1115+
s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'. $extraMsg")
11071116
}
11081117
} catch {
1109-
case NonFatal(exception) =>
1110-
val e = exception match {
1111-
// Since we are using shim, the exceptions thrown by the underlying method of
1112-
// Method.invoke() are wrapped by InvocationTargetException
1113-
case i: InvocationTargetException => i.getCause
1114-
case o => o
1115-
}
1118+
case ae: AnalysisException =>
1119+
throw ae
1120+
case NonFatal(e) =>
11161121
val analysisException =
1117-
new AnalysisException(s"No handler for UDAF '${clazz.getCanonicalName}': $e")
1122+
new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e")
11181123
analysisException.setStackTrace(e.getStackTrace)
11191124
throw analysisException
11201125
}
@@ -1123,6 +1128,8 @@ class SessionCatalog(
11231128

11241129
/**
11251130
* Construct a [[FunctionBuilder]] based on the provided class that represents a function.
1131+
*
1132+
* This performs reflection to decide what type of [[Expression]] to return in the builder.
11261133
*/
11271134
protected def makeFunctionExpression(
11281135
name: String,

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala

Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF}
2727
import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry}
2828
import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF}
2929

30-
import org.apache.spark.sql.AnalysisException
3130
import org.apache.spark.sql.catalyst.FunctionIdentifier
3231
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
3332
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
@@ -37,7 +36,6 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface
3736
import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
3837
import org.apache.spark.sql.internal.SQLConf
3938
import org.apache.spark.sql.types.{DecimalType, DoubleType}
40-
import org.apache.spark.util.Utils
4139

4240

4341
private[sql] class HiveSessionCatalog(
@@ -58,25 +56,6 @@ private[sql] class HiveSessionCatalog(
5856
parser,
5957
functionResourceLoader) {
6058

61-
override def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
62-
val clazz = Utils.classForName(functionClassName)
63-
(children: Seq[Expression]) => {
64-
try {
65-
makeFunctionExpression(name, Utils.classForName(functionClassName), children).getOrElse {
66-
throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
67-
}
68-
} catch {
69-
case ae: AnalysisException =>
70-
throw ae
71-
case NonFatal(e) =>
72-
val analysisException =
73-
new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
74-
analysisException.setStackTrace(e.getStackTrace)
75-
throw analysisException
76-
}
77-
}
78-
}
79-
8059
/**
8160
* Construct a [[FunctionBuilder]] based on the provided class that represents a function.
8261
*/

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala

Lines changed: 18 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -404,59 +404,34 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
404404
}
405405

406406
test("Hive UDFs with insufficient number of input arguments should trigger an analysis error") {
407-
Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF")
407+
withTempView("testUDF") {
408+
Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF")
409+
410+
def testErrorMsgForFunc(funcName: String, className: String): Unit = {
411+
withUserDefinedFunction(funcName -> true) {
412+
sql(s"CREATE TEMPORARY FUNCTION $funcName AS '$className'")
413+
val message = intercept[AnalysisException] {
414+
sql(s"SELECT $funcName() FROM testUDF")
415+
}.getMessage
416+
assert(message.contains(s"No handler for UDF/UDAF/UDTF '$className'"))
417+
}
418+
}
408419

409-
{
410420
// HiveSimpleUDF
411-
sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'")
412-
val message = intercept[AnalysisException] {
413-
sql("SELECT testUDFTwoListList() FROM testUDF")
414-
}.getMessage
415-
assert(message.contains("No handler for Hive UDF"))
416-
sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList")
417-
}
421+
testErrorMsgForFunc("testUDFTwoListList", classOf[UDFTwoListList].getName)
418422

419-
{
420423
// HiveGenericUDF
421-
sql(s"CREATE TEMPORARY FUNCTION testUDFAnd AS '${classOf[GenericUDFOPAnd].getName}'")
422-
val message = intercept[AnalysisException] {
423-
sql("SELECT testUDFAnd() FROM testUDF")
424-
}.getMessage
425-
assert(message.contains("No handler for Hive UDF"))
426-
sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFAnd")
427-
}
424+
testErrorMsgForFunc("testUDFAnd", classOf[GenericUDFOPAnd].getName)
428425

429-
{
430426
// Hive UDAF
431-
sql(s"CREATE TEMPORARY FUNCTION testUDAFPercentile AS '${classOf[UDAFPercentile].getName}'")
432-
val message = intercept[AnalysisException] {
433-
sql("SELECT testUDAFPercentile(a) FROM testUDF GROUP BY b")
434-
}.getMessage
435-
assert(message.contains("No handler for Hive UDF"))
436-
sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFPercentile")
437-
}
427+
testErrorMsgForFunc("testUDAFPercentile", classOf[UDAFPercentile].getName)
438428

439-
{
440429
// AbstractGenericUDAFResolver
441-
sql(s"CREATE TEMPORARY FUNCTION testUDAFAverage AS '${classOf[GenericUDAFAverage].getName}'")
442-
val message = intercept[AnalysisException] {
443-
sql("SELECT testUDAFAverage() FROM testUDF GROUP BY b")
444-
}.getMessage
445-
assert(message.contains("No handler for Hive UDF"))
446-
sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFAverage")
447-
}
430+
testErrorMsgForFunc("testUDAFAverage", classOf[GenericUDAFAverage].getName)
448431

449-
{
450-
// Hive UDTF
451-
sql(s"CREATE TEMPORARY FUNCTION testUDTFExplode AS '${classOf[GenericUDTFExplode].getName}'")
452-
val message = intercept[AnalysisException] {
453-
sql("SELECT testUDTFExplode() FROM testUDF")
454-
}.getMessage
455-
assert(message.contains("No handler for Hive UDF"))
456-
sql("DROP TEMPORARY FUNCTION IF EXISTS testUDTFExplode")
432+
// AbstractGenericUDAFResolver
433+
testErrorMsgForFunc("testUDTFExplode", classOf[GenericUDTFExplode].getName)
457434
}
458-
459-
spark.catalog.dropTempView("testUDF")
460435
}
461436

462437
test("Hive UDF in group by") {

0 commit comments

Comments
 (0)