spark-instrumented-optimizer/python/pyspark/tests/test_context.py
Fokko Driesprong 9fcf0ea718 [SPARK-32319][PYSPARK] Disallow the use of unused imports
Disallow the use of unused imports:

- Unnecessary increases the memory footprint of the application
- Removes the imports that are required for the examples in the docstring from the file-scope to the example itself. This keeps the files itself clean, and gives a more complete example as it also includes the imports :)

```
fokkodriesprongFan spark % flake8 python | grep -i "imported but unused"
python/pyspark/cloudpickle.py:46:1: F401 'functools.partial' imported but unused
python/pyspark/cloudpickle.py:55:1: F401 'traceback' imported but unused
python/pyspark/heapq3.py:868:5: F401 '_heapq.*' imported but unused
python/pyspark/__init__.py:61:1: F401 'pyspark.version.__version__' imported but unused
python/pyspark/__init__.py:62:1: F401 'pyspark._globals._NoValue' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.SQLContext' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.HiveContext' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.Row' imported but unused
python/pyspark/rdd.py:21:1: F401 're' imported but unused
python/pyspark/rdd.py:29:1: F401 'tempfile.NamedTemporaryFile' imported but unused
python/pyspark/mllib/regression.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/classification.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/feature.py:30:1: F401 'pyspark.mllib.regression.LabeledPoint' imported but unused
python/pyspark/mllib/tests/test_linalg.py:18:1: F401 'sys' imported but unused
python/pyspark/mllib/tests/test_linalg.py:642:5: F401 'pyspark.mllib.tests.test_linalg.*' imported but unused
python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.random' imported but unused
python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.exp' imported but unused
python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused
python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused
python/pyspark/mllib/tests/test_feature.py:185:5: F401 'pyspark.mllib.tests.test_feature.*' imported but unused
python/pyspark/mllib/tests/test_util.py:97:5: F401 'pyspark.mllib.tests.test_util.*' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg._convert_to_vector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseMatrix' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseMatrix' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.MatrixUDT' imported but unused
python/pyspark/mllib/tests/test_stat.py:181:5: F401 'pyspark.mllib.tests.test_stat.*' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.time' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.sleep' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:470:5: F401 'pyspark.mllib.tests.test_streaming_algorithms.*' imported but unused
python/pyspark/mllib/tests/test_algorithms.py:295:5: F401 'pyspark.mllib.tests.test_algorithms.*' imported but unused
python/pyspark/tests/test_serializers.py:90:13: F401 'xmlrunner' imported but unused
python/pyspark/tests/test_rdd.py:21:1: F401 'sys' imported but unused
python/pyspark/tests/test_rdd.py:29:1: F401 'pyspark.resource.ResourceProfile' imported but unused
python/pyspark/tests/test_rdd.py:885:5: F401 'pyspark.tests.test_rdd.*' imported but unused
python/pyspark/tests/test_readwrite.py:19:1: F401 'sys' imported but unused
python/pyspark/tests/test_readwrite.py:22:1: F401 'array.array' imported but unused
python/pyspark/tests/test_readwrite.py:309:5: F401 'pyspark.tests.test_readwrite.*' imported but unused
python/pyspark/tests/test_join.py:62:5: F401 'pyspark.tests.test_join.*' imported but unused
python/pyspark/tests/test_taskcontext.py:19:1: F401 'shutil' imported but unused
python/pyspark/tests/test_taskcontext.py:325:5: F401 'pyspark.tests.test_taskcontext.*' imported but unused
python/pyspark/tests/test_conf.py:36:5: F401 'pyspark.tests.test_conf.*' imported but unused
python/pyspark/tests/test_broadcast.py:148:5: F401 'pyspark.tests.test_broadcast.*' imported but unused
python/pyspark/tests/test_daemon.py:76:5: F401 'pyspark.tests.test_daemon.*' imported but unused
python/pyspark/tests/test_util.py:77:5: F401 'pyspark.tests.test_util.*' imported but unused
python/pyspark/tests/test_pin_thread.py:19:1: F401 'random' imported but unused
python/pyspark/tests/test_pin_thread.py:149:5: F401 'pyspark.tests.test_pin_thread.*' imported but unused
python/pyspark/tests/test_worker.py:19:1: F401 'sys' imported but unused
python/pyspark/tests/test_worker.py:26:5: F401 'resource' imported but unused
python/pyspark/tests/test_worker.py:203:5: F401 'pyspark.tests.test_worker.*' imported but unused
python/pyspark/tests/test_profiler.py:101:5: F401 'pyspark.tests.test_profiler.*' imported but unused
python/pyspark/tests/test_shuffle.py:18:1: F401 'sys' imported but unused
python/pyspark/tests/test_shuffle.py:171:5: F401 'pyspark.tests.test_shuffle.*' imported but unused
python/pyspark/tests/test_rddbarrier.py:43:5: F401 'pyspark.tests.test_rddbarrier.*' imported but unused
python/pyspark/tests/test_context.py:129:13: F401 'userlibrary.UserClass' imported but unused
python/pyspark/tests/test_context.py:140:13: F401 'userlib.UserClass' imported but unused
python/pyspark/tests/test_context.py:310:5: F401 'pyspark.tests.test_context.*' imported but unused
python/pyspark/tests/test_appsubmit.py:241:5: F401 'pyspark.tests.test_appsubmit.*' imported but unused
python/pyspark/streaming/dstream.py:18:1: F401 'sys' imported but unused
python/pyspark/streaming/tests/test_dstream.py:27:1: F401 'pyspark.RDD' imported but unused
python/pyspark/streaming/tests/test_dstream.py:647:5: F401 'pyspark.streaming.tests.test_dstream.*' imported but unused
python/pyspark/streaming/tests/test_kinesis.py:83:5: F401 'pyspark.streaming.tests.test_kinesis.*' imported but unused
python/pyspark/streaming/tests/test_listener.py:152:5: F401 'pyspark.streaming.tests.test_listener.*' imported but unused
python/pyspark/streaming/tests/test_context.py:178:5: F401 'pyspark.streaming.tests.test_context.*' imported but unused
python/pyspark/testing/utils.py:30:5: F401 'scipy.sparse' imported but unused
python/pyspark/testing/utils.py:36:5: F401 'numpy as np' imported but unused
python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._TreeEnsembleParams' imported but unused
python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._HasVarianceImpurity' imported but unused
python/pyspark/ml/regression.py:29:1: F401 'pyspark.ml.wrapper.JavaParams' imported but unused
python/pyspark/ml/util.py:19:1: F401 'sys' imported but unused
python/pyspark/ml/__init__.py:25:1: F401 'pyspark.ml.pipeline' imported but unused
python/pyspark/ml/pipeline.py:18:1: F401 'sys' imported but unused
python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.DenseMatrix' imported but unused
python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.Vectors' imported but unused
python/pyspark/ml/tests/test_training_summary.py:18:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_training_summary.py:364:5: F401 'pyspark.ml.tests.test_training_summary.*' imported but unused
python/pyspark/ml/tests/test_linalg.py:381:5: F401 'pyspark.ml.tests.test_linalg.*' imported but unused
python/pyspark/ml/tests/test_tuning.py:427:9: F401 'pyspark.sql.functions as F' imported but unused
python/pyspark/ml/tests/test_tuning.py:757:5: F401 'pyspark.ml.tests.test_tuning.*' imported but unused
python/pyspark/ml/tests/test_wrapper.py:120:5: F401 'pyspark.ml.tests.test_wrapper.*' imported but unused
python/pyspark/ml/tests/test_feature.py:19:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_feature.py:304:5: F401 'pyspark.ml.tests.test_feature.*' imported but unused
python/pyspark/ml/tests/test_image.py:19:1: F401 'py4j' imported but unused
python/pyspark/ml/tests/test_image.py:22:1: F401 'pyspark.testing.mlutils.PySparkTestCase' imported but unused
python/pyspark/ml/tests/test_image.py:71:5: F401 'pyspark.ml.tests.test_image.*' imported but unused
python/pyspark/ml/tests/test_persistence.py:456:5: F401 'pyspark.ml.tests.test_persistence.*' imported but unused
python/pyspark/ml/tests/test_evaluation.py:56:5: F401 'pyspark.ml.tests.test_evaluation.*' imported but unused
python/pyspark/ml/tests/test_stat.py:43:5: F401 'pyspark.ml.tests.test_stat.*' imported but unused
python/pyspark/ml/tests/test_base.py:70:5: F401 'pyspark.ml.tests.test_base.*' imported but unused
python/pyspark/ml/tests/test_param.py:20:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_param.py:375:5: F401 'pyspark.ml.tests.test_param.*' imported but unused
python/pyspark/ml/tests/test_pipeline.py:62:5: F401 'pyspark.ml.tests.test_pipeline.*' imported but unused
python/pyspark/ml/tests/test_algorithms.py:333:5: F401 'pyspark.ml.tests.test_algorithms.*' imported but unused
python/pyspark/ml/param/__init__.py:18:1: F401 'sys' imported but unused
python/pyspark/resource/tests/test_resources.py:17:1: F401 'random' imported but unused
python/pyspark/resource/tests/test_resources.py:20:1: F401 'pyspark.resource.ResourceProfile' imported but unused
python/pyspark/resource/tests/test_resources.py:75:5: F401 'pyspark.resource.tests.test_resources.*' imported but unused
python/pyspark/sql/functions.py:32:1: F401 'pyspark.sql.udf.UserDefinedFunction' imported but unused
python/pyspark/sql/functions.py:34:1: F401 'pyspark.sql.pandas.functions.pandas_udf' imported but unused
python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.Row' imported but unused
python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.StringType' imported but unused
python/pyspark/sql/readwriter.py:1084:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.IntegerType' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.Row' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.StringType' imported but unused
python/pyspark/sql/context.py:27:1: F401 'pyspark.sql.udf.UDFRegistration' imported but unused
python/pyspark/sql/streaming.py:1212:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/tests/test_utils.py:55:5: F401 'pyspark.sql.tests.test_utils.*' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:18:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.pandas_udf' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:119:5: F401 'pyspark.sql.tests.test_pandas_map.*' imported but unused
python/pyspark/sql/tests/test_catalog.py:193:5: F401 'pyspark.sql.tests.test_catalog.*' imported but unused
python/pyspark/sql/tests/test_group.py:39:5: F401 'pyspark.sql.tests.test_group.*' imported but unused
python/pyspark/sql/tests/test_session.py:361:5: F401 'pyspark.sql.tests.test_session.*' imported but unused
python/pyspark/sql/tests/test_conf.py:49:5: F401 'pyspark.sql.tests.test_conf.*' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:19:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.sum' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:29:5: F401 'pandas.util.testing.assert_series_equal' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:32:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:248:5: F401 'pyspark.sql.tests.test_pandas_cogrouped_map.*' imported but unused
python/pyspark/sql/tests/test_udf.py:24:1: F401 'py4j' imported but unused
python/pyspark/sql/tests/test_pandas_udf_typehints.py:246:5: F401 'pyspark.sql.tests.test_pandas_udf_typehints.*' imported but unused
python/pyspark/sql/tests/test_functions.py:19:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_functions.py:362:9: F401 'pyspark.sql.functions.exists' imported but unused
python/pyspark/sql/tests/test_functions.py:387:5: F401 'pyspark.sql.tests.test_functions.*' imported but unused
python/pyspark/sql/tests/test_pandas_udf_scalar.py:21:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_udf_scalar.py:45:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_udf_window.py:355:5: F401 'pyspark.sql.tests.test_pandas_udf_window.*' imported but unused
python/pyspark/sql/tests/test_arrow.py:38:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_grouped_map.py:20:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_grouped_map.py:38:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_dataframe.py:382:9: F401 'pyspark.sql.DataFrame' imported but unused
python/pyspark/sql/avro/functions.py:125:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/pandas/functions.py:19:1: F401 'sys' imported but unused
```

After:
```
fokkodriesprongFan spark % flake8 python | grep -i "imported but unused"
fokkodriesprongFan spark %
```

### What changes were proposed in this pull request?

Removing unused imports from the Python files to keep everything nice and tidy.

### Why are the changes needed?

Cleaning up of the imports that aren't used, and suppressing the imports that are used as references to other modules, preserving backward compatibility.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Adding the rule to the existing Flake8 checks.

Closes #29121 from Fokko/SPARK-32319.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 08:51:57 -07:00

329 lines
13 KiB
Python

#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import os
import shutil
import stat
import tempfile
import threading
import time
import unittest
from collections import namedtuple
from pyspark import SparkConf, SparkFiles, SparkContext
from pyspark.testing.utils import ReusedPySparkTestCase, PySparkTestCase, QuietTest, SPARK_HOME
class CheckpointTests(ReusedPySparkTestCase):
def setUp(self):
self.checkpointDir = tempfile.NamedTemporaryFile(delete=False)
os.unlink(self.checkpointDir.name)
self.sc.setCheckpointDir(self.checkpointDir.name)
def tearDown(self):
shutil.rmtree(self.checkpointDir.name)
def test_basic_checkpointing(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
self.assertFalse(flatMappedRDD.isCheckpointed())
self.assertTrue(flatMappedRDD.getCheckpointFile() is None)
flatMappedRDD.checkpoint()
result = flatMappedRDD.collect()
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.isCheckpointed())
self.assertEqual(flatMappedRDD.collect(), result)
self.assertEqual("file:" + self.checkpointDir.name,
os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile())))
def test_checkpoint_and_restore(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
flatMappedRDD = parCollection.flatMap(lambda x: [x])
self.assertFalse(flatMappedRDD.isCheckpointed())
self.assertTrue(flatMappedRDD.getCheckpointFile() is None)
flatMappedRDD.checkpoint()
flatMappedRDD.count() # forces a checkpoint to be computed
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.getCheckpointFile() is not None)
recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(),
flatMappedRDD._jrdd_deserializer)
self.assertEqual([1, 2, 3, 4], recovered.collect())
class LocalCheckpointTests(ReusedPySparkTestCase):
def test_basic_localcheckpointing(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
self.assertFalse(flatMappedRDD.isCheckpointed())
self.assertFalse(flatMappedRDD.isLocallyCheckpointed())
flatMappedRDD.localCheckpoint()
result = flatMappedRDD.collect()
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.isCheckpointed())
self.assertTrue(flatMappedRDD.isLocallyCheckpointed())
self.assertEqual(flatMappedRDD.collect(), result)
class AddFileTests(PySparkTestCase):
def test_add_py_file(self):
# To ensure that we're actually testing addPyFile's effects, check that
# this job fails due to `userlibrary` not being on the Python path:
# disable logging in log4j temporarily
def func(x):
from userlibrary import UserClass
return UserClass().hello()
with QuietTest(self.sc):
self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first)
# Add the file, so the job should now succeed:
path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
self.sc.addPyFile(path)
res = self.sc.parallelize(range(2)).map(func).first()
self.assertEqual("Hello World!", res)
def test_add_file_locally(self):
path = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
self.sc.addFile(path)
download_path = SparkFiles.get("hello.txt")
self.assertNotEqual(path, download_path)
with open(download_path) as test_file:
self.assertEqual("Hello World!\n", test_file.readline())
def test_add_file_recursively_locally(self):
path = os.path.join(SPARK_HOME, "python/test_support/hello")
self.sc.addFile(path, True)
download_path = SparkFiles.get("hello")
self.assertNotEqual(path, download_path)
with open(download_path + "/hello.txt") as test_file:
self.assertEqual("Hello World!\n", test_file.readline())
with open(download_path + "/sub_hello/sub_hello.txt") as test_file:
self.assertEqual("Sub Hello World!\n", test_file.readline())
def test_add_py_file_locally(self):
# To ensure that we're actually testing addPyFile's effects, check that
# this fails due to `userlibrary` not being on the Python path:
def func():
from userlibrary import UserClass # noqa: F401
self.assertRaises(ImportError, func)
path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
self.sc.addPyFile(path)
from userlibrary import UserClass
self.assertEqual("Hello World!", UserClass().hello())
def test_add_egg_file_locally(self):
# To ensure that we're actually testing addPyFile's effects, check that
# this fails due to `userlibrary` not being on the Python path:
def func():
from userlib import UserClass # noqa: F401
self.assertRaises(ImportError, func)
path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip")
self.sc.addPyFile(path)
from userlib import UserClass
self.assertEqual("Hello World from inside a package!", UserClass().hello())
def test_overwrite_system_module(self):
self.sc.addPyFile(os.path.join(SPARK_HOME, "python/test_support/SimpleHTTPServer.py"))
import SimpleHTTPServer
self.assertEqual("My Server", SimpleHTTPServer.__name__)
def func(x):
import SimpleHTTPServer
return SimpleHTTPServer.__name__
self.assertEqual(["My Server"], self.sc.parallelize(range(1)).map(func).collect())
class ContextTests(unittest.TestCase):
def test_failed_sparkcontext_creation(self):
# Regression test for SPARK-1550
self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name"))
def test_get_or_create(self):
with SparkContext.getOrCreate() as sc:
self.assertTrue(SparkContext.getOrCreate() is sc)
def test_parallelize_eager_cleanup(self):
with SparkContext() as sc:
temp_files = os.listdir(sc._temp_dir)
rdd = sc.parallelize([0, 1, 2])
post_parallalize_temp_files = os.listdir(sc._temp_dir)
self.assertEqual(temp_files, post_parallalize_temp_files)
def test_set_conf(self):
# This is for an internal use case. When there is an existing SparkContext,
# SparkSession's builder needs to set configs into SparkContext's conf.
sc = SparkContext()
sc._conf.set("spark.test.SPARK16224", "SPARK16224")
self.assertEqual(sc._jsc.sc().conf().get("spark.test.SPARK16224"), "SPARK16224")
sc.stop()
def test_stop(self):
sc = SparkContext()
self.assertNotEqual(SparkContext._active_spark_context, None)
sc.stop()
self.assertEqual(SparkContext._active_spark_context, None)
def test_with(self):
with SparkContext() as sc:
self.assertNotEqual(SparkContext._active_spark_context, None)
self.assertEqual(SparkContext._active_spark_context, None)
def test_with_exception(self):
try:
with SparkContext() as sc:
self.assertNotEqual(SparkContext._active_spark_context, None)
raise Exception()
except:
pass
self.assertEqual(SparkContext._active_spark_context, None)
def test_with_stop(self):
with SparkContext() as sc:
self.assertNotEqual(SparkContext._active_spark_context, None)
sc.stop()
self.assertEqual(SparkContext._active_spark_context, None)
def test_progress_api(self):
with SparkContext() as sc:
sc.setJobGroup('test_progress_api', '', True)
rdd = sc.parallelize(range(10)).map(lambda x: time.sleep(100))
def run():
# When thread is pinned, job group should be set for each thread for now.
# Local properties seem not being inherited like Scala side does.
if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true":
sc.setJobGroup('test_progress_api', '', True)
try:
rdd.count()
except Exception:
pass
t = threading.Thread(target=run)
t.daemon = True
t.start()
# wait for scheduler to start
time.sleep(1)
tracker = sc.statusTracker()
jobIds = tracker.getJobIdsForGroup('test_progress_api')
self.assertEqual(1, len(jobIds))
job = tracker.getJobInfo(jobIds[0])
self.assertEqual(1, len(job.stageIds))
stage = tracker.getStageInfo(job.stageIds[0])
self.assertEqual(rdd.getNumPartitions(), stage.numTasks)
sc.cancelAllJobs()
t.join()
# wait for event listener to update the status
time.sleep(1)
job = tracker.getJobInfo(jobIds[0])
self.assertEqual('FAILED', job.status)
self.assertEqual([], tracker.getActiveJobsIds())
self.assertEqual([], tracker.getActiveStageIds())
sc.stop()
def test_startTime(self):
with SparkContext() as sc:
self.assertGreater(sc.startTime, 0)
def test_forbid_insecure_gateway(self):
# Fail immediately if you try to create a SparkContext
# with an insecure gateway
parameters = namedtuple('MockGatewayParameters', 'auth_token')(None)
mock_insecure_gateway = namedtuple('MockJavaGateway', 'gateway_parameters')(parameters)
with self.assertRaises(ValueError) as context:
SparkContext(gateway=mock_insecure_gateway)
self.assertIn("insecure Py4j gateway", str(context.exception))
def test_resources(self):
"""Test the resources are empty by default."""
with SparkContext() as sc:
resources = sc.resources
self.assertEqual(len(resources), 0)
def test_disallow_to_create_spark_context_in_executors(self):
# SPARK-32160: SparkContext should not be created in executors.
with SparkContext("local-cluster[3, 1, 1024]") as sc:
with self.assertRaises(Exception) as context:
sc.range(2).foreach(lambda _: SparkContext())
self.assertIn("SparkContext should only be created and accessed on the driver.",
str(context.exception))
def test_allow_to_create_spark_context_in_executors(self):
# SPARK-32160: SparkContext can be created in executors if the config is set.
def create_spark_context():
conf = SparkConf().set("spark.executor.allowSparkContext", "true")
with SparkContext(conf=conf):
pass
with SparkContext("local-cluster[3, 1, 1024]") as sc:
sc.range(2).foreach(lambda _: create_spark_context())
class ContextTestsWithResources(unittest.TestCase):
def setUp(self):
class_name = self.__class__.__name__
self.tempFile = tempfile.NamedTemporaryFile(delete=False)
self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}')
self.tempFile.close()
# create temporary directory for Worker resources coordination
self.tempdir = tempfile.NamedTemporaryFile(delete=False)
os.unlink(self.tempdir.name)
os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP |
stat.S_IROTH | stat.S_IXOTH)
conf = SparkConf().set("spark.test.home", SPARK_HOME)
conf = conf.set("spark.driver.resource.gpu.amount", "1")
conf = conf.set("spark.driver.resource.gpu.discoveryScript", self.tempFile.name)
self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf)
def test_resources(self):
"""Test the resources are available."""
resources = self.sc.resources
self.assertEqual(len(resources), 1)
self.assertTrue('gpu' in resources)
self.assertEqual(resources['gpu'].name, 'gpu')
self.assertEqual(resources['gpu'].addresses, ['0'])
def tearDown(self):
os.unlink(self.tempFile.name)
self.sc.stop()
if __name__ == "__main__":
from pyspark.tests.test_context import * # noqa: F401
try:
import xmlrunner
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
except ImportError:
testRunner = None
unittest.main(testRunner=testRunner, verbosity=2)