From 3dfa4ea526c881373eeffe541bc378d1fa598129 Mon Sep 17 00:00:00 2001
From: Liang-Chi Hsieh <viirya@appier.com>
Date: Wed, 28 Oct 2015 21:45:00 -0700
Subject: [PATCH] [SPARK-11322] [PYSPARK] Keep full stack trace in captured
 exception

JIRA: https://issues.apache.org/jira/browse/SPARK-11322

As reported by JoshRosen in [databricks/spark-redshift/issues/89](https://github.com/databricks/spark-redshift/issues/89#issuecomment-149828308), the exception-masking behavior sometimes makes debugging harder. To deal with this issue, we should keep full stack trace in the captured exception.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9283 from viirya/py-exception-stacktrace.
---
 python/pyspark/sql/tests.py |  6 ++++++
 python/pyspark/sql/utils.py | 19 +++++++++++++++----
 2 files changed, 21 insertions(+), 4 deletions(-)

diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 6356d4bd66..4c03a0d4ff 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -1079,6 +1079,12 @@ class SQLTests(ReusedPySparkTestCase):
         df = self.sqlCtx.createDataFrame([(1, 2)], ["a", "b"])
         self.assertRaisesRegexp(IllegalArgumentException, "1024 is not in the permitted values",
                                 lambda: df.select(sha2(df.a, 1024)).collect())
+        try:
+            df.select(sha2(df.a, 1024)).collect()
+        except IllegalArgumentException as e:
+            self.assertRegexpMatches(e.desc, "1024 is not in the permitted values")
+            self.assertRegexpMatches(e.stackTrace,
+                                     "org.apache.spark.sql.functions")
 
     def test_with_column_with_existing_name(self):
         keys = self.df.withColumn("key", self.df.key).select("key").collect()
diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py
index 0f795ca35b..c4fda8bd3b 100644
--- a/python/pyspark/sql/utils.py
+++ b/python/pyspark/sql/utils.py
@@ -18,13 +18,22 @@
 import py4j
 
 
-class AnalysisException(Exception):
+class CapturedException(Exception):
+    def __init__(self, desc, stackTrace):
+        self.desc = desc
+        self.stackTrace = stackTrace
+
+    def __str__(self):
+        return repr(self.desc)
+
+
+class AnalysisException(CapturedException):
     """
     Failed to analyze a SQL query plan.
     """
 
 
-class IllegalArgumentException(Exception):
+class IllegalArgumentException(CapturedException):
     """
     Passed an illegal or inappropriate argument.
     """
@@ -36,10 +45,12 @@ def capture_sql_exception(f):
             return f(*a, **kw)
         except py4j.protocol.Py4JJavaError as e:
             s = e.java_exception.toString()
+            stackTrace = '\n\t at '.join(map(lambda x: x.toString(),
+                                             e.java_exception.getStackTrace()))
             if s.startswith('org.apache.spark.sql.AnalysisException: '):
-                raise AnalysisException(s.split(': ', 1)[1])
+                raise AnalysisException(s.split(': ', 1)[1], stackTrace)
             if s.startswith('java.lang.IllegalArgumentException: '):
-                raise IllegalArgumentException(s.split(': ', 1)[1])
+                raise IllegalArgumentException(s.split(': ', 1)[1], stackTrace)
             raise
     return deco
 
-- 
GitLab