Skip to content

[SPARK-23776][python][test] Check for needed components/files before running pyspark-sql tests #20909

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 15 additions & 0 deletions python/pyspark/sql/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -964,6 +964,21 @@ def _test():
except py4j.protocol.Py4JError:
spark = SparkSession(sc)

hive_enabled = True
try:
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
except py4j.protocol.Py4JError:
hive_enabled = False
except TypeError:
hive_enabled = False

if not hive_enabled:
# if hive is not enabled, then skip doctests that need hive
# TODO: Need to communicate with outside world that this test
# has been skipped.
m = pyspark.sql.readwriter
m.__dict__["DataFrameReader"].__dict__["table"].__doc__ = ""

Copy link
Member

@dongjoon-hyun dongjoon-hyun Apr 24, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for pointer, @HyukjinKwon .

For readwriter.py, we had better test without Hive. How do you think , @HyukjinKwon and @bersprockets ?

- spark = SparkSession.builder.enableHiveSupport().getOrCreate()
+ spark = SparkSession.builder.getOrCreate()

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yup, it looks better.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dongjoon-hyun Sounds good. That change will be done in PR #21141, correct?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bersprockets I was thinking like that but wanted to ask your thought per this PR. I am okay with either way.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@HyukjinKwon @dongjoon-hyun I agree, it should go in PR #21141.

globs['tempfile'] = tempfile
globs['os'] = os
globs['sc'] = sc
Expand Down
14 changes: 14 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -2977,6 +2977,20 @@ def test_create_dateframe_from_pandas_with_dst(self):

class HiveSparkSubmitTests(SparkSubmitTests):

@classmethod
def setUpClass(cls):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this way is more correct, as @felixcheung pointed out.

# get a SparkContext to check for availability of Hive
sc = SparkContext('local[4]', cls.__name__)
try:
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
except py4j.protocol.Py4JError:
raise unittest.SkipTest("Hive is not available")
except TypeError:
raise unittest.SkipTest("Hive is not available")
finally:
# we don't need SparkContext for the test
sc.stop()

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
Expand Down
15 changes: 15 additions & 0 deletions python/pyspark/sql/udf.py
Original file line number Diff line number Diff line change
Expand Up @@ -385,8 +385,23 @@ def registerJavaUDAF(self, name, javaClassName):

def _test():
import doctest
import os
import os.path
import glob
from pyspark.sql import SparkSession
import pyspark.sql.udf

SPARK_HOME = os.environ["SPARK_HOME"]
filename_pattern = "sql/core/target/scala-*/test-classes/" + \
"test/org/apache/spark/sql/JavaStringLength.class"
if not glob.glob(os.path.join(SPARK_HOME, filename_pattern)):
# if test udf files are not compiled, then skip the below doctests
# TODO: Need to communicate with outside world that these tests
# have been skipped.
m = pyspark.sql.udf
m.__dict__["UDFRegistration"].__dict__["registerJavaFunction"].__doc__ = ""
m.__dict__["UDFRegistration"].__dict__["registerJavaUDAF"].__doc__ = ""
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah.. hmm... yea this one was the last resort I was thinking ... let me investigate other possible ways for some more days.


globs = pyspark.sql.udf.__dict__.copy()
spark = SparkSession.builder\
.master("local[4]")\
Expand Down