You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2018/02/08 07:47:17 UTC

spark git commit: [SPARK-23319][TESTS][BRANCH-2.3] Explicitly specify Pandas and PyArrow versions in PySpark tests (to skip or test)

Repository: spark
Updated Branches:
  refs/heads/branch-2.3 db59e5542 -> 053830256


[SPARK-23319][TESTS][BRANCH-2.3] Explicitly specify Pandas and PyArrow versions in PySpark tests (to skip or test)

This PR backports https://github.com/apache/spark/pull/20487 to branch-2.3.

Author: hyukjinkwon <gu...@gmail.com>
Author: Takuya UESHIN <ue...@databricks.com>

Closes #20534 from HyukjinKwon/PR_TOOL_PICK_PR_20487_BRANCH-2.3.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/05383025
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/05383025
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/05383025

Branch: refs/heads/branch-2.3
Commit: 0538302561c4d77b2856b1ce73b3ccbcb6688ac6
Parents: db59e55
Author: hyukjinkwon <gu...@gmail.com>
Authored: Thu Feb 8 16:47:12 2018 +0900
Committer: hyukjinkwon <gu...@gmail.com>
Committed: Thu Feb 8 16:47:12 2018 +0900

----------------------------------------------------------------------
 pom.xml                         |  4 ++
 python/pyspark/sql/dataframe.py |  3 ++
 python/pyspark/sql/session.py   |  3 ++
 python/pyspark/sql/tests.py     | 83 +++++++++++++++++++-----------------
 python/pyspark/sql/utils.py     | 30 +++++++++----
 python/setup.py                 | 10 ++++-
 6 files changed, 86 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a8e448a..9aa531e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -185,6 +185,10 @@
     <paranamer.version>2.8</paranamer.version>
     <maven-antrun.version>1.8</maven-antrun.version>
     <commons-crypto.version>1.0.0</commons-crypto.version>
+    <!--
+    If you are changing Arrow version specification, please check ./python/pyspark/sql/utils.py,
+    ./python/run-tests.py and ./python/setup.py too.
+    -->
     <arrow.version>0.8.0</arrow.version>
 
     <test.java.home>${java.home}</test.java.home>

http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/python/pyspark/sql/dataframe.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 59a4170..8ec24db 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -1913,6 +1913,9 @@ class DataFrame(object):
         0    2  Alice
         1    5    Bob
         """
+        from pyspark.sql.utils import require_minimum_pandas_version
+        require_minimum_pandas_version()
+
         import pandas as pd
 
         if self.sql_ctx.getConf("spark.sql.execution.pandas.respectSessionTimeZone").lower() \

http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/python/pyspark/sql/session.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 6c84023..2ac2ec2 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -640,6 +640,9 @@ class SparkSession(object):
         except Exception:
             has_pandas = False
         if has_pandas and isinstance(data, pandas.DataFrame):
+            from pyspark.sql.utils import require_minimum_pandas_version
+            require_minimum_pandas_version()
+
             if self.conf.get("spark.sql.execution.pandas.respectSessionTimeZone").lower() \
                == "true":
                 timezone = self.conf.get("spark.sql.session.timeZone")

http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/python/pyspark/sql/tests.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 878d402..0e1b2ec 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -48,19 +48,26 @@ if sys.version_info[:2] <= (2, 6):
 else:
     import unittest
 
-_have_pandas = False
-_have_old_pandas = False
+_pandas_requirement_message = None
 try:
-    import pandas
-    try:
-        from pyspark.sql.utils import require_minimum_pandas_version
-        require_minimum_pandas_version()
-        _have_pandas = True
-    except:
-        _have_old_pandas = True
-except:
-    # No Pandas, but that's okay, we'll skip those tests
-    pass
+    from pyspark.sql.utils import require_minimum_pandas_version
+    require_minimum_pandas_version()
+except ImportError as e:
+    from pyspark.util import _exception_message
+    # If Pandas version requirement is not satisfied, skip related tests.
+    _pandas_requirement_message = _exception_message(e)
+
+_pyarrow_requirement_message = None
+try:
+    from pyspark.sql.utils import require_minimum_pyarrow_version
+    require_minimum_pyarrow_version()
+except ImportError as e:
+    from pyspark.util import _exception_message
+    # If Arrow version requirement is not satisfied, skip related tests.
+    _pyarrow_requirement_message = _exception_message(e)
+
+_have_pandas = _pandas_requirement_message is None
+_have_pyarrow = _pyarrow_requirement_message is None
 
 from pyspark import SparkContext
 from pyspark.sql import SparkSession, SQLContext, HiveContext, Column, Row
@@ -75,15 +82,6 @@ from pyspark.sql.window import Window
 from pyspark.sql.utils import AnalysisException, ParseException, IllegalArgumentException
 
 
-_have_arrow = False
-try:
-    import pyarrow
-    _have_arrow = True
-except:
-    # No Arrow, but that's okay, we'll skip those tests
-    pass
-
-
 class UTCOffsetTimezone(datetime.tzinfo):
     """
     Specifies timezone in UTC offset
@@ -2788,7 +2786,6 @@ class SQLTests(ReusedSQLTestCase):
 
     def _to_pandas(self):
         from datetime import datetime, date
-        import numpy as np
         schema = StructType().add("a", IntegerType()).add("b", StringType())\
                              .add("c", BooleanType()).add("d", FloatType())\
                              .add("dt", DateType()).add("ts", TimestampType())
@@ -2801,7 +2798,7 @@ class SQLTests(ReusedSQLTestCase):
         df = self.spark.createDataFrame(data, schema)
         return df.toPandas()
 
-    @unittest.skipIf(not _have_pandas, "Pandas not installed")
+    @unittest.skipIf(not _have_pandas, _pandas_requirement_message)
     def test_to_pandas(self):
         import numpy as np
         pdf = self._to_pandas()
@@ -2813,13 +2810,13 @@ class SQLTests(ReusedSQLTestCase):
         self.assertEquals(types[4], np.object)  # datetime.date
         self.assertEquals(types[5], 'datetime64[ns]')
 
-    @unittest.skipIf(not _have_old_pandas, "Old Pandas not installed")
-    def test_to_pandas_old(self):
+    @unittest.skipIf(_have_pandas, "Required Pandas was found.")
+    def test_to_pandas_required_pandas_not_found(self):
         with QuietTest(self.sc):
             with self.assertRaisesRegexp(ImportError, 'Pandas >= .* must be installed'):
                 self._to_pandas()
 
-    @unittest.skipIf(not _have_pandas, "Pandas not installed")
+    @unittest.skipIf(not _have_pandas, _pandas_requirement_message)
     def test_to_pandas_avoid_astype(self):
         import numpy as np
         schema = StructType().add("a", IntegerType()).add("b", StringType())\
@@ -2837,7 +2834,7 @@ class SQLTests(ReusedSQLTestCase):
         df = self.spark.createDataFrame(data)
         self.assertEqual(df.first(), Row(longarray=[-9223372036854775808, 0, 9223372036854775807]))
 
-    @unittest.skipIf(not _have_pandas, "Pandas not installed")
+    @unittest.skipIf(not _have_pandas, _pandas_requirement_message)
     def test_create_dataframe_from_pandas_with_timestamp(self):
         import pandas as pd
         from datetime import datetime
@@ -2852,14 +2849,16 @@ class SQLTests(ReusedSQLTestCase):
         self.assertTrue(isinstance(df.schema['ts'].dataType, TimestampType))
         self.assertTrue(isinstance(df.schema['d'].dataType, DateType))
 
-    @unittest.skipIf(not _have_old_pandas, "Old Pandas not installed")
-    def test_create_dataframe_from_old_pandas(self):
-        import pandas as pd
-        from datetime import datetime
-        pdf = pd.DataFrame({"ts": [datetime(2017, 10, 31, 1, 1, 1)],
-                            "d": [pd.Timestamp.now().date()]})
+    @unittest.skipIf(_have_pandas, "Required Pandas was found.")
+    def test_create_dataframe_required_pandas_not_found(self):
         with QuietTest(self.sc):
-            with self.assertRaisesRegexp(ImportError, 'Pandas >= .* must be installed'):
+            with self.assertRaisesRegexp(
+                    ImportError,
+                    "(Pandas >= .* must be installed|No module named '?pandas'?)"):
+                import pandas as pd
+                from datetime import datetime
+                pdf = pd.DataFrame({"ts": [datetime(2017, 10, 31, 1, 1, 1)],
+                                    "d": [pd.Timestamp.now().date()]})
                 self.spark.createDataFrame(pdf)
 
 
@@ -3351,7 +3350,9 @@ class DataTypeVerificationTests(unittest.TestCase):
                 _make_type_verifier(data_type, nullable=False)(obj)
 
 
-@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed")
+@unittest.skipIf(
+    not _have_pandas or not _have_pyarrow,
+    _pandas_requirement_message or _pyarrow_requirement_message)
 class ArrowTests(ReusedSQLTestCase):
 
     @classmethod
@@ -3615,7 +3616,9 @@ class ArrowTests(ReusedSQLTestCase):
         self.assertEqual(pdf_col_names, df_arrow.columns)
 
 
-@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed")
+@unittest.skipIf(
+    not _have_pandas or not _have_pyarrow,
+    _pandas_requirement_message or _pyarrow_requirement_message)
 class PandasUDFTests(ReusedSQLTestCase):
     def test_pandas_udf_basic(self):
         from pyspark.rdd import PythonEvalType
@@ -3739,7 +3742,9 @@ class PandasUDFTests(ReusedSQLTestCase):
                     return k
 
 
-@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed")
+@unittest.skipIf(
+    not _have_pandas or not _have_pyarrow,
+    _pandas_requirement_message or _pyarrow_requirement_message)
 class ScalarPandasUDFTests(ReusedSQLTestCase):
 
     @classmethod
@@ -4252,7 +4257,9 @@ class ScalarPandasUDFTests(ReusedSQLTestCase):
         self.assertEquals(expected.collect(), res2.collect())
 
 
-@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed")
+@unittest.skipIf(
+    not _have_pandas or not _have_pyarrow,
+    _pandas_requirement_message or _pyarrow_requirement_message)
 class GroupedMapPandasUDFTests(ReusedSQLTestCase):
 
     def assertFramesEqual(self, expected, result):

http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/python/pyspark/sql/utils.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py
index 08c34c6..5782986 100644
--- a/python/pyspark/sql/utils.py
+++ b/python/pyspark/sql/utils.py
@@ -115,18 +115,32 @@ def toJArray(gateway, jtype, arr):
 def require_minimum_pandas_version():
     """ Raise ImportError if minimum version of Pandas is not installed
     """
+    # TODO(HyukjinKwon): Relocate and deduplicate the version specification.
+    minimum_pandas_version = "0.19.2"
+
     from distutils.version import LooseVersion
-    import pandas
-    if LooseVersion(pandas.__version__) < LooseVersion('0.19.2'):
-        raise ImportError("Pandas >= 0.19.2 must be installed on calling Python process; "
-                          "however, your version was %s." % pandas.__version__)
+    try:
+        import pandas
+    except ImportError:
+        raise ImportError("Pandas >= %s must be installed; however, "
+                          "it was not found." % minimum_pandas_version)
+    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__))
 
 
 def require_minimum_pyarrow_version():
     """ Raise ImportError if minimum version of pyarrow is not installed
     """
+    # TODO(HyukjinKwon): Relocate and deduplicate the version specification.
+    minimum_pyarrow_version = "0.8.0"
+
     from distutils.version import LooseVersion
-    import pyarrow
-    if LooseVersion(pyarrow.__version__) < LooseVersion('0.8.0'):
-        raise ImportError("pyarrow >= 0.8.0 must be installed on calling Python process; "
-                          "however, your version was %s." % pyarrow.__version__)
+    try:
+        import pyarrow
+    except ImportError:
+        raise ImportError("PyArrow >= %s must be installed; however, "
+                          "it was not found." % minimum_pyarrow_version)
+    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__))

http://git-wip-us.apache.org/repos/asf/spark/blob/05383025/python/setup.py
----------------------------------------------------------------------
diff --git a/python/setup.py b/python/setup.py
index 251d452..6a98401 100644
--- a/python/setup.py
+++ b/python/setup.py
@@ -100,6 +100,11 @@ if (in_spark):
               file=sys.stderr)
         exit(-1)
 
+# If you are changing the versions here, please also change ./python/pyspark/sql/utils.py and
+# ./python/run-tests.py. In case of Arrow, you should also check ./pom.xml.
+_minimum_pandas_version = "0.19.2"
+_minimum_pyarrow_version = "0.8.0"
+
 try:
     # We copy the shell script to be under pyspark/python/pyspark so that the launcher scripts
     # find it where expected. The rest of the files aren't copied because they are accessed
@@ -201,7 +206,10 @@ try:
         extras_require={
             'ml': ['numpy>=1.7'],
             'mllib': ['numpy>=1.7'],
-            'sql': ['pandas>=0.19.2', 'pyarrow>=0.8.0']
+            'sql': [
+                'pandas>=%s' % _minimum_pandas_version,
+                'pyarrow>=%s' % _minimum_pyarrow_version,
+            ]
         },
         classifiers=[
             'Development Status :: 5 - Production/Stable',


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