-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24044][PYTHON] Explicitly print out skipped tests from unittest module #21107
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
Changes from all commits
b075bb8
8c1f16e
56b9001
3dd74a0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -3088,23 +3088,28 @@ def setUpClass(cls): | |
| filename_pattern = ( | ||
| "sql/core/target/scala-*/test-classes/org/apache/spark/sql/" | ||
| "TestQueryExecutionListener.class") | ||
| if not glob.glob(os.path.join(SPARK_HOME, filename_pattern)): | ||
| raise unittest.SkipTest( | ||
| cls.has_listener = bool(glob.glob(os.path.join(SPARK_HOME, filename_pattern))) | ||
|
|
||
| if cls.has_listener: | ||
| # Note that 'spark.sql.queryExecutionListeners' is a static immutable configuration. | ||
| cls.spark = SparkSession.builder \ | ||
| .master("local[4]") \ | ||
| .appName(cls.__name__) \ | ||
| .config( | ||
| "spark.sql.queryExecutionListeners", | ||
| "org.apache.spark.sql.TestQueryExecutionListener") \ | ||
| .getOrCreate() | ||
|
|
||
| def setUp(self): | ||
| if not self.has_listener: | ||
| raise self.skipTest( | ||
| "'org.apache.spark.sql.TestQueryExecutionListener' is not " | ||
| "available. Will skip the related tests.") | ||
|
|
||
| # Note that 'spark.sql.queryExecutionListeners' is a static immutable configuration. | ||
| cls.spark = SparkSession.builder \ | ||
| .master("local[4]") \ | ||
| .appName(cls.__name__) \ | ||
| .config( | ||
| "spark.sql.queryExecutionListeners", | ||
| "org.apache.spark.sql.TestQueryExecutionListener") \ | ||
| .getOrCreate() | ||
|
|
||
| @classmethod | ||
| def tearDownClass(cls): | ||
| cls.spark.stop() | ||
| if hasattr(cls, "spark"): | ||
| cls.spark.stop() | ||
|
|
||
| def tearDown(self): | ||
| self.spark._jvm.OnSuccessCall.clear() | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not called if the test is skipped during
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yup. |
||
|
|
@@ -3188,18 +3193,22 @@ class HiveContextSQLTests(ReusedPySparkTestCase): | |
| def setUpClass(cls): | ||
| ReusedPySparkTestCase.setUpClass() | ||
| cls.tempdir = tempfile.NamedTemporaryFile(delete=False) | ||
| cls.hive_available = True | ||
| try: | ||
| cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf() | ||
| except py4j.protocol.Py4JError: | ||
| cls.tearDownClass() | ||
| raise unittest.SkipTest("Hive is not available") | ||
| cls.hive_available = False | ||
| except TypeError: | ||
| cls.tearDownClass() | ||
| raise unittest.SkipTest("Hive is not available") | ||
| cls.hive_available = False | ||
| os.unlink(cls.tempdir.name) | ||
| cls.spark = HiveContext._createForTesting(cls.sc) | ||
| cls.testData = [Row(key=i, value=str(i)) for i in range(100)] | ||
| cls.df = cls.sc.parallelize(cls.testData).toDF() | ||
| if cls.hive_available: | ||
| cls.spark = HiveContext._createForTesting(cls.sc) | ||
| cls.testData = [Row(key=i, value=str(i)) for i in range(100)] | ||
| cls.df = cls.sc.parallelize(cls.testData).toDF() | ||
|
|
||
| def setUp(self): | ||
| if not self.hive_available: | ||
| self.skipTest("Hive is not available.") | ||
|
|
||
| @classmethod | ||
| def tearDownClass(cls): | ||
|
|
@@ -5308,6 +5317,6 @@ def test_invalid_args(self): | |
| if __name__ == "__main__": | ||
| from pyspark.sql.tests import * | ||
| if xmlrunner: | ||
| unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports')) | ||
| unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports'), verbosity=2) | ||
| else: | ||
| unittest.main() | ||
| unittest.main(verbosity=2) | ||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -32,6 +32,7 @@ | |||||||||||||||||||||
| else: | ||||||||||||||||||||||
| import queue as Queue | ||||||||||||||||||||||
| from distutils.version import LooseVersion | ||||||||||||||||||||||
| from multiprocessing import Manager | ||||||||||||||||||||||
|
|
||||||||||||||||||||||
|
|
||||||||||||||||||||||
| # Append `SPARK_HOME/dev` to the Python path so that we can import the sparktestsupport module | ||||||||||||||||||||||
|
|
@@ -50,6 +51,7 @@ def print_red(text): | |||||||||||||||||||||
| print('\033[31m' + text + '\033[0m') | ||||||||||||||||||||||
|
|
||||||||||||||||||||||
|
|
||||||||||||||||||||||
| SKIPPED_TESTS = Manager().dict() | ||||||||||||||||||||||
| LOG_FILE = os.path.join(SPARK_HOME, "python/unit-tests.log") | ||||||||||||||||||||||
| FAILURE_REPORTING_LOCK = Lock() | ||||||||||||||||||||||
| LOGGER = logging.getLogger() | ||||||||||||||||||||||
|
|
@@ -109,8 +111,34 @@ def run_individual_python_test(test_name, pyspark_python): | |||||||||||||||||||||
| # this code is invoked from a thread other than the main thread. | ||||||||||||||||||||||
| os._exit(-1) | ||||||||||||||||||||||
| else: | ||||||||||||||||||||||
| per_test_output.close() | ||||||||||||||||||||||
| LOGGER.info("Finished test(%s): %s (%is)", pyspark_python, test_name, duration) | ||||||||||||||||||||||
| skipped_counts = 0 | ||||||||||||||||||||||
| try: | ||||||||||||||||||||||
| per_test_output.seek(0) | ||||||||||||||||||||||
| # Here expects skipped test output from unittest when verbosity level is | ||||||||||||||||||||||
| # 2 (or --verbose option is enabled). | ||||||||||||||||||||||
| decoded_lines = map(lambda line: line.decode(), iter(per_test_output)) | ||||||||||||||||||||||
| skipped_tests = list(filter( | ||||||||||||||||||||||
| lambda line: re.search('test_.* \(pyspark\..*\) ... skipped ', line), | ||||||||||||||||||||||
| decoded_lines)) | ||||||||||||||||||||||
| skipped_counts = len(skipped_tests) | ||||||||||||||||||||||
| if skipped_counts > 0: | ||||||||||||||||||||||
| key = (pyspark_python, test_name) | ||||||||||||||||||||||
| SKIPPED_TESTS[key] = skipped_tests | ||||||||||||||||||||||
| per_test_output.close() | ||||||||||||||||||||||
| except: | ||||||||||||||||||||||
| import traceback | ||||||||||||||||||||||
| print_red("\nGot an exception while trying to store " | ||||||||||||||||||||||
| "skipped test output:\n%s" % traceback.format_exc()) | ||||||||||||||||||||||
|
||||||||||||||||||||||
| # Here, we use os._exit() instead of sys.exit() in order to force Python to exit even if | ||||||||||||||||||||||
| # this code is invoked from a thread other than the main thread. | ||||||||||||||||||||||
| os._exit(-1) | ||||||||||||||||||||||
| if skipped_counts != 0: | ||||||||||||||||||||||
| LOGGER.info( | ||||||||||||||||||||||
| "Finished test(%s): %s (%is) ... %s tests were skipped", pyspark_python, test_name, | ||||||||||||||||||||||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure if there's a better format. let me know. |
||||||||||||||||||||||
| duration, skipped_counts) | ||||||||||||||||||||||
| else: | ||||||||||||||||||||||
| LOGGER.info( | ||||||||||||||||||||||
| "Finished test(%s): %s (%is)", pyspark_python, test_name, duration) | ||||||||||||||||||||||
|
|
||||||||||||||||||||||
|
|
||||||||||||||||||||||
| def get_default_python_executables(): | ||||||||||||||||||||||
|
|
@@ -152,65 +180,17 @@ def parse_opts(): | |||||||||||||||||||||
| return opts | ||||||||||||||||||||||
|
|
||||||||||||||||||||||
|
|
||||||||||||||||||||||
| def _check_dependencies(python_exec, modules_to_test): | ||||||||||||||||||||||
| if "COVERAGE_PROCESS_START" in os.environ: | ||||||||||||||||||||||
| # Make sure if coverage is installed. | ||||||||||||||||||||||
| try: | ||||||||||||||||||||||
| subprocess_check_output( | ||||||||||||||||||||||
| [python_exec, "-c", "import coverage"], | ||||||||||||||||||||||
| stderr=open(os.devnull, 'w')) | ||||||||||||||||||||||
| except: | ||||||||||||||||||||||
| print_red("Coverage is not installed in Python executable '%s' " | ||||||||||||||||||||||
| "but 'COVERAGE_PROCESS_START' environment variable is set, " | ||||||||||||||||||||||
| "exiting." % python_exec) | ||||||||||||||||||||||
| sys.exit(-1) | ||||||||||||||||||||||
|
|
||||||||||||||||||||||
| # If we should test 'pyspark-sql', it checks if PyArrow and Pandas are installed and | ||||||||||||||||||||||
| # explicitly prints out. See SPARK-23300. | ||||||||||||||||||||||
| if pyspark_sql in modules_to_test: | ||||||||||||||||||||||
| # TODO(HyukjinKwon): Relocate and deduplicate these version specifications. | ||||||||||||||||||||||
| minimum_pyarrow_version = '0.8.0' | ||||||||||||||||||||||
|
||||||||||||||||||||||
| _pyarrow_requirement_message = None | |
| try: | |
| from pyspark.sql.utils import require_minimum_pyarrow_version | |
| require_minimum_pyarrow_version() | |
| except ImportError as e: | |
| # If Arrow version requirement is not satisfied, skip related tests. | |
| _pyarrow_requirement_message = _exception_message(e) |
spark/python/pyspark/sql/tests.py
Lines 3121 to 3123 in ab7b961
| @unittest.skipIf( | |
| not _have_pandas or not _have_pyarrow, | |
| _pandas_requirement_message or _pyarrow_requirement_message) |
which prints out a skip message like:
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped
'PyArrow >= 0.8.0 must be installed; however, it was not found.'
which I am capturing here with a regex pattern.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Gotcha. Thanks for the explanation!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will it be too verbose to print all skipped tests? An option is to record them into LOG_FILE only. No strong preference here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yea, I get it but I feel sure people felt it should be printed out in the console from previous discussions. I actually don't feel strongly too.
Anyway we could have only few of them eventually because most of them are by missing Pandas and Arrow so probably this could be fine.
No strong preference here too. let me know if other guys have a preference.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
i think print is fine for now. we could also change later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@viirya, @bersprockets and @BryanCutler, these were the output from my partial testing in my local.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A little worry that it can be too verbose when skipped tests are too many. See #21107 (comment).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I assume there is no way to only print out the skipped test class name?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No clean way as far as I can tell. I should do another regex thing but .. I would like to avoid this way as possible as I can ..
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Gotcha. Yeah the current implementation looks good to me.