Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8941#discussion_r40877302
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala 
---
    @@ -60,20 +60,36 @@ private[hive] class HiveFunctionRegistry(underlying: 
analysis.FunctionRegistry)
     
           val functionClassName = functionInfo.getFunctionClass.getName
     
    -      if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
    -        HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children)
    -      } else if 
(classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
    -        HiveGenericUDF(new HiveFunctionWrapper(functionClassName), 
children)
    -      } else if (
    -        
classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass))
 {
    -        HiveUDAFFunction(new HiveFunctionWrapper(functionClassName), 
children)
    -      } else if 
(classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) {
    -        HiveUDAFFunction(
    -          new HiveFunctionWrapper(functionClassName), children, 
isUDAFBridgeRequired = true)
    -      } else if 
(classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) {
    -        HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), 
children)
    -      } else {
    -        sys.error(s"No handler for udf ${functionInfo.getFunctionClass}")
    +      // When we instantiate hive UDF wrapper class, we may throw 
exception if the input expressions
    +      // don't satisfy the hive UDF, such as type mismatch, input number 
mismatch, etc. Here we
    +      // catch the exception and throw AnalysisException instead.
    +      try {
    +        if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
    +          HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), 
children)
    +        } else if 
(classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
    +          HiveGenericUDF(new HiveFunctionWrapper(functionClassName), 
children)
    +        } else if (
    +          
classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass))
 {
    +          HiveUDAFFunction(new HiveFunctionWrapper(functionClassName), 
children)
    +        } else if 
(classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) {
    +          HiveUDAFFunction(
    +            new HiveFunctionWrapper(functionClassName), children, 
isUDAFBridgeRequired = true)
    +        } else if 
(classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) {
    +          val udtf = HiveGenericUDTF(new 
HiveFunctionWrapper(functionClassName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for udf 
${functionInfo.getFunctionClass}")
    +        }
    +      } catch {
    +        case analysisException: AnalysisException =>
    +          // If the exception is an AnalysisException, just throw it.
    +          throw analysisException
    +        case throwable: Throwable =>
    +          // If there is any error, we throw an AnalysisException.
    --- End diff --
    
    If there is any error => If there is any other error


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to