[SPARK-34803][PYSPARK] Pass the raised ImportError if pandas or pyarrow fail to import
### What changes were proposed in this pull request? Pass the raised `ImportError` on failing to import pandas/pyarrow. This will help the user identify whether pandas/pyarrow are indeed not in the environment or if they threw a different `ImportError`. ### Why are the changes needed? This can already happen in Pandas for example where it could throw an `ImportError` on its initialisation path if `dateutil` doesn't satisfy a certain version requirement https://github.com/pandas-dev/pandas/blob/0.24.x/pandas/compat/__init__.py#L438 ### Does this PR introduce _any_ user-facing change? Yes, it will now show the root cause of the exception when pandas or arrow is missing during import. ### How was this patch tested? Manually tested. ```python from pyspark.sql.functions import pandas_udf spark.range(1).select(pandas_udf(lambda x: x)) ``` Before: ``` Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/...//spark/python/pyspark/sql/pandas/functions.py", line 332, in pandas_udf require_minimum_pyarrow_version() File "/.../spark/python/pyspark/sql/pandas/utils.py", line 53, in require_minimum_pyarrow_version raise ImportError("PyArrow >= %s must be installed; however, " ImportError: PyArrow >= 1.0.0 must be installed; however, it was not found. ``` After: ``` Traceback (most recent call last): File "/.../spark/python/pyspark/sql/pandas/utils.py", line 49, in require_minimum_pyarrow_version import pyarrow ModuleNotFoundError: No module named 'pyarrow' The above exception was the direct cause of the following exception: Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/pandas/functions.py", line 332, in pandas_udf require_minimum_pyarrow_version() File "/.../spark/python/pyspark/sql/pandas/utils.py", line 55, in require_minimum_pyarrow_version raise ImportError("PyArrow >= %s must be installed; however, " ImportError: PyArrow >= 1.0.0 must be installed; however, it was not found. ``` Closes #31902 from johnhany97/jayad/spark-34803. Lead-authored-by: John Ayad <johnhany97@gmail.com> Co-authored-by: John H. Ayad <johnhany97@gmail.com> Co-authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
This commit is contained in:
parent
8a552bfc76
commit
ddfc75ec64
|
@ -26,11 +26,12 @@ def require_minimum_pandas_version():
|
|||
try:
|
||||
import pandas
|
||||
have_pandas = True
|
||||
except ImportError:
|
||||
except ImportError as error:
|
||||
have_pandas = False
|
||||
raised_error = error
|
||||
if not have_pandas:
|
||||
raise ImportError("Pandas >= %s must be installed; however, "
|
||||
"it was not found." % minimum_pandas_version)
|
||||
"it was not found." % minimum_pandas_version) from raised_error
|
||||
if LooseVersion(pandas.__version__) < LooseVersion(minimum_pandas_version):
|
||||
raise ImportError("Pandas >= %s must be installed; however, "
|
||||
"your version was %s." % (minimum_pandas_version, pandas.__version__))
|
||||
|
@ -47,11 +48,12 @@ def require_minimum_pyarrow_version():
|
|||
try:
|
||||
import pyarrow
|
||||
have_arrow = True
|
||||
except ImportError:
|
||||
except ImportError as error:
|
||||
have_arrow = False
|
||||
raised_error = error
|
||||
if not have_arrow:
|
||||
raise ImportError("PyArrow >= %s must be installed; however, "
|
||||
"it was not found." % minimum_pyarrow_version)
|
||||
"it was not found." % minimum_pyarrow_version) from raised_error
|
||||
if LooseVersion(pyarrow.__version__) < LooseVersion(minimum_pyarrow_version):
|
||||
raise ImportError("PyArrow >= %s must be installed; however, "
|
||||
"your version was %s." % (minimum_pyarrow_version, pyarrow.__version__))
|
||||
|
|
Loading…
Reference in a new issue