This is an automated email from the ASF dual-hosted git repository.

srowen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 2bf4d97  [SPARK-24544][SQL] Print actual failure cause when look up 
function failed
2bf4d97 is described below

commit 2bf4d97118c20812e26a7ea59826ee470ab42f7c
Author: zhoukang <zhoukang199...@gmail.com>
AuthorDate: Tue Jan 1 09:13:13 2019 -0600

    [SPARK-24544][SQL] Print actual failure cause when look up function failed
    
    ## What changes were proposed in this pull request?
    
    When we operate as below:
    `
    0: jdbc:hive2://xxx/> create  function funnel_analysis as 
'com.xxx.hive.extend.udf.UapFunnelAnalysis';
    `
    
    `
    0: jdbc:hive2://xxx/> select funnel_analysis(1,",",1,'');
    Error: org.apache.spark.sql.AnalysisException: Undefined function: 
'funnel_analysis'. This function is neither a registered temporary function nor 
a permanent function registered in the database 'xxx'.; line 1 pos 7 
(state=,code=0)
    `
    
    `
    0: jdbc:hive2://xxx/> describe function funnel_analysis;
    +-----------------------------------------------------------+--+
    |                       function_desc                       |
    +-----------------------------------------------------------+--+
    | Function: xxx.funnel_analysis                            |
    | Class: com.xxx.hive.extend.udf.UapFunnelAnalysis  |
    | Usage: N/A.                                               |
    +-----------------------------------------------------------+--+
    `
    We can see describe funtion will get right information,but when we actually 
use this funtion,we will get an undefined exception.
    Which is really misleading,the real cause is below:
     `
    No handler for Hive UDF 'com.xxx.xxx.hive.extend.udf.UapFunnelAnalysis': 
java.lang.IllegalStateException: Should not be called directly;
        at 
org.apache.hadoop.hive.ql.udf.generic.GenericUDTF.initialize(GenericUDTF.java:72)
        at 
org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector$lzycompute(hiveUDFs.scala:204)
        at 
org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector(hiveUDFs.scala:204)
        at 
org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema$lzycompute(hiveUDFs.scala:212)
        at 
org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema(hiveUDFs.scala:212)
    `
    This patch print the actual failure for quick debugging.
    ## How was this patch tested?
    UT
    
    Closes #21790 from caneGuy/zhoukang/print-warning1.
    
    Authored-by: zhoukang <zhoukang199...@gmail.com>
    Signed-off-by: Sean Owen <sean.o...@databricks.com>
---
 .../spark/sql/catalyst/analysis/NoSuchItemException.scala  |  4 ++--
 .../apache/spark/sql/catalyst/catalog/SessionCatalog.scala |  5 +++--
 .../spark/sql/catalyst/catalog/SessionCatalogSuite.scala   | 14 ++++++++++++++
 .../org/apache/spark/sql/hive/HiveSessionCatalog.scala     |  9 ++++++---
 4 files changed, 25 insertions(+), 7 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
index f5aae60..8bf6f69 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
@@ -40,10 +40,10 @@ 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"a permanent function registered in the database '$db'.", cause = cause)
 
 class NoSuchPartitionsException(db: String, table: String, specs: 
Seq[TablePartitionSpec])
   extends AnalysisException(
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index b6771ec..1dbe946 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -1222,9 +1222,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)
   }
 
   /**
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index 19e8c03..92f87ea 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -1448,4 +1448,18 @@ abstract class SessionCatalogSuite extends AnalysisTest {
       }
     }
   }
+
+  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"))
+    }
+  }
 }
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
index 405c0c8..7560805 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
@@ -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(
@@ -141,8 +142,10 @@ private[sql] class HiveSessionCatalog(
           // let's try to load it as a Hive's built-in function.
           // Hive is case insensitive.
           val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT)
+          logWarning("Encountered a failure during looking up function:" +
+            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
@@ -150,12 +153,12 @@ private[sql] class HiveSessionCatalog(
           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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to