Skip to content

Commit 52a420f

Browse files
dongjoon-hyunHyukjinKwon
authored andcommitted
[SPARK-23853][PYSPARK][TEST] Run Hive-related PySpark tests only for -Phive
## What changes were proposed in this pull request? When `PyArrow` or `Pandas` are not available, the corresponding PySpark tests are skipped automatically. Currently, PySpark tests fail when we are not using `-Phive`. This PR aims to skip Hive related PySpark tests when `-Phive` is not given. **BEFORE** ```bash $ build/mvn -DskipTests clean package $ python/run-tests.py --python-executables python2.7 --modules pyspark-sql File "/Users/dongjoon/spark/python/pyspark/sql/readwriter.py", line 295, in pyspark.sql.readwriter.DataFrameReader.table ... IllegalArgumentException: u"Error while instantiating 'org.apache.spark.sql.hive.HiveExternalCatalog':" ********************************************************************** 1 of 3 in pyspark.sql.readwriter.DataFrameReader.table ***Test Failed*** 1 failures. ``` **AFTER** ```bash $ build/mvn -DskipTests clean package $ python/run-tests.py --python-executables python2.7 --modules pyspark-sql ... Tests passed in 138 seconds Skipped tests in pyspark.sql.tests with python2.7: ... test_hivecontext (pyspark.sql.tests.HiveSparkSubmitTests) ... skipped 'Hive is not available.' ``` ## How was this patch tested? This is a test-only change. First, this should pass the Jenkins. Then, manually do the following. ```bash build/mvn -DskipTests clean package python/run-tests.py --python-executables python2.7 --modules pyspark-sql ``` Author: Dongjoon Hyun <[email protected]> Closes #21141 from dongjoon-hyun/SPARK-23853. (cherry picked from commit b857fb5) Signed-off-by: hyukjinkwon <[email protected]>
1 parent 235ec9e commit 52a420f

File tree

2 files changed

+21
-1
lines changed

2 files changed

+21
-1
lines changed

python/pyspark/sql/readwriter.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -956,7 +956,7 @@ def _test():
956956
globs = pyspark.sql.readwriter.__dict__.copy()
957957
sc = SparkContext('local[4]', 'PythonTest')
958958
try:
959-
spark = SparkSession.builder.enableHiveSupport().getOrCreate()
959+
spark = SparkSession.builder.getOrCreate()
960960
except py4j.protocol.Py4JError:
961961
spark = SparkSession(sc)
962962

python/pyspark/sql/tests.py

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2935,6 +2935,26 @@ def test_create_dateframe_from_pandas_with_dst(self):
29352935

29362936
class HiveSparkSubmitTests(SparkSubmitTests):
29372937

2938+
@classmethod
2939+
def setUpClass(cls):
2940+
# get a SparkContext to check for availability of Hive
2941+
sc = SparkContext('local[4]', cls.__name__)
2942+
cls.hive_available = True
2943+
try:
2944+
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
2945+
except py4j.protocol.Py4JError:
2946+
cls.hive_available = False
2947+
except TypeError:
2948+
cls.hive_available = False
2949+
finally:
2950+
# we don't need this SparkContext for the test
2951+
sc.stop()
2952+
2953+
def setUp(self):
2954+
super(HiveSparkSubmitTests, self).setUp()
2955+
if not self.hive_available:
2956+
self.skipTest("Hive is not available.")
2957+
29382958
def test_hivecontext(self):
29392959
# This test checks that HiveContext is using Hive metastore (SPARK-16224).
29402960
# It sets a metastore url and checks if there is a derby dir created by

0 commit comments

Comments
 (0)