[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 <dongjoon@apache.org> Closes #21141 from dongjoon-hyun/SPARK-23853.
This commit is contained in:
parent
007ae6878f
commit
b857fb549f
|
@ -979,7 +979,7 @@ def _test():
|
|||
globs = pyspark.sql.readwriter.__dict__.copy()
|
||||
sc = SparkContext('local[4]', 'PythonTest')
|
||||
try:
|
||||
spark = SparkSession.builder.enableHiveSupport().getOrCreate()
|
||||
spark = SparkSession.builder.getOrCreate()
|
||||
except py4j.protocol.Py4JError:
|
||||
spark = SparkSession(sc)
|
||||
|
||||
|
|
|
@ -3043,6 +3043,26 @@ class SQLTests(ReusedSQLTestCase):
|
|||
|
||||
class HiveSparkSubmitTests(SparkSubmitTests):
|
||||
|
||||
@classmethod
|
||||
def setUpClass(cls):
|
||||
# get a SparkContext to check for availability of Hive
|
||||
sc = SparkContext('local[4]', cls.__name__)
|
||||
cls.hive_available = True
|
||||
try:
|
||||
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
|
||||
except py4j.protocol.Py4JError:
|
||||
cls.hive_available = False
|
||||
except TypeError:
|
||||
cls.hive_available = False
|
||||
finally:
|
||||
# we don't need this SparkContext for the test
|
||||
sc.stop()
|
||||
|
||||
def setUp(self):
|
||||
super(HiveSparkSubmitTests, self).setUp()
|
||||
if not self.hive_available:
|
||||
self.skipTest("Hive is not available.")
|
||||
|
||||
def test_hivecontext(self):
|
||||
# This test checks that HiveContext is using Hive metastore (SPARK-16224).
|
||||
# It sets a metastore url and checks if there is a derby dir created by
|
||||
|
|
Loading…
Reference in a new issue