[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>
This commit is contained in:
Fokko Driesprong 2020-08-08 08:51:57 -07:00 committed by Dongjoon Hyun
parent eb74d55fb5
commit 9fcf0ea718
92 changed files with 124 additions and 147 deletions

View file

@ -23,7 +23,7 @@ import sys
from subprocess import Popen, PIPE from subprocess import Popen, PIPE
try: try:
from jira.client import JIRA from jira.client import JIRA # noqa: F401
# Old versions have JIRAError in exceptions package, new (0.5+) in utils. # Old versions have JIRAError in exceptions package, new (0.5+) in utils.
try: try:
from jira.exceptions import JIRAError from jira.exceptions import JIRAError
@ -35,7 +35,7 @@ except ImportError:
sys.exit(-1) sys.exit(-1)
try: try:
from github import Github from github import Github # noqa: F401
from github import GithubException from github import GithubException
except ImportError: except ImportError:
print("This tool requires the PyGithub library") print("This tool requires the PyGithub library")

View file

@ -147,8 +147,7 @@ flake8 checks failed."
fi fi
echo "starting $FLAKE8_BUILD test..." echo "starting $FLAKE8_BUILD test..."
FLAKE8_REPORT=$( ($FLAKE8_BUILD . --count --select=E901,E999,F821,F822,F823 \ FLAKE8_REPORT=$( ($FLAKE8_BUILD --append-config dev/tox.ini --count --show-source --statistics .) 2>&1)
--max-line-length=100 --show-source --statistics) 2>&1)
FLAKE8_STATUS=$? FLAKE8_STATUS=$?
if [ "$FLAKE8_STATUS" -ne 0 ]; then if [ "$FLAKE8_STATUS" -ne 0 ]; then

View file

@ -16,7 +16,6 @@
# #
from pyspark.sql import SparkSession from pyspark.sql import SparkSession
from pyspark.mllib.linalg import *
import sys import sys
if __name__ == "__main__": if __name__ == "__main__":

View file

@ -26,7 +26,6 @@ import sys
import subprocess import subprocess
import glob import glob
import shutil import shutil
from collections import namedtuple
from sparktestsupport import SPARK_HOME, USER_HOME, ERROR_CODES from sparktestsupport import SPARK_HOME, USER_HOME, ERROR_CODES
from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r, which from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r, which

View file

@ -17,3 +17,8 @@
ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504
max-line-length=100 max-line-length=100
exclude=python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* exclude=python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/*
[flake8]
select = E901,E999,F821,F822,F823,F401
exclude = python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/*
max-line-length = 100

View file

@ -21,7 +21,7 @@ Run with:
./bin/spark-submit examples/src/main/python/sql/hive.py ./bin/spark-submit examples/src/main/python/sql/hive.py
""" """
# $example on:spark_hive$ # $example on:spark_hive$
from os.path import join, abspath from os.path import abspath
from pyspark.sql import SparkSession from pyspark.sql import SparkSession
from pyspark.sql import Row from pyspark.sql import Row

View file

@ -17,7 +17,6 @@
import time import time
import threading import threading
import sys
import queue as Queue import queue as Queue
from pyspark import SparkConf, SparkContext from pyspark import SparkConf, SparkContext

View file

@ -61,8 +61,8 @@ from pyspark.serializers import MarshalSerializer, PickleSerializer
from pyspark.status import * from pyspark.status import *
from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo
from pyspark.profiler import Profiler, BasicProfiler from pyspark.profiler import Profiler, BasicProfiler
from pyspark.version import __version__ from pyspark.version import __version__ # noqa: F401
from pyspark._globals import _NoValue from pyspark._globals import _NoValue # noqa: F401
def since(version): def since(version):
@ -115,7 +115,7 @@ def keyword_only(func):
# for back compatibility # for back compatibility
from pyspark.sql import SQLContext, HiveContext, Row from pyspark.sql import SQLContext, HiveContext, Row # noqa: F401
__all__ = [ __all__ = [
"SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",

View file

@ -23,7 +23,7 @@ from pyspark.ml.base import Estimator, Model, Predictor, PredictionModel, \
Transformer, UnaryTransformer Transformer, UnaryTransformer
from pyspark.ml.pipeline import Pipeline, PipelineModel from pyspark.ml.pipeline import Pipeline, PipelineModel
from pyspark.ml import classification, clustering, evaluation, feature, fpm, \ from pyspark.ml import classification, clustering, evaluation, feature, fpm, \
image, pipeline, recommendation, regression, stat, tuning, util, linalg, param image, recommendation, regression, stat, tuning, util, linalg, param
__all__ = [ __all__ = [
"Transformer", "UnaryTransformer", "Estimator", "Model", "Transformer", "UnaryTransformer", "Estimator", "Model",

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import array import array
import sys
from abc import ABCMeta from abc import ABCMeta
import copy import copy

View file

@ -15,8 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import sys
from pyspark import keyword_only from pyspark import keyword_only
from pyspark.ml.base import Estimator, Model, Transformer from pyspark.ml.base import Estimator, Model, Transformer
from pyspark.ml.param import Param, Params from pyspark.ml.param import Param, Params

View file

@ -15,18 +15,16 @@
# limitations under the License. # limitations under the License.
# #
import sys
from abc import ABCMeta from abc import ABCMeta
from pyspark import since, keyword_only from pyspark import keyword_only
from pyspark.ml import Predictor, PredictionModel from pyspark.ml import Predictor, PredictionModel
from pyspark.ml.base import _PredictorParams from pyspark.ml.base import _PredictorParams
from pyspark.ml.param.shared import * from pyspark.ml.param.shared import *
from pyspark.ml.tree import _DecisionTreeModel, _DecisionTreeParams, \ from pyspark.ml.tree import _DecisionTreeModel, _DecisionTreeParams, \
_TreeEnsembleModel, _TreeEnsembleParams, _RandomForestParams, _GBTParams, \ _TreeEnsembleModel, _RandomForestParams, _GBTParams, _TreeRegressorParams
_HasVarianceImpurity, _TreeRegressorParams
from pyspark.ml.util import * from pyspark.ml.util import *
from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ from pyspark.ml.wrapper import JavaEstimator, JavaModel, \
JavaPredictor, JavaPredictionModel, JavaWrapper JavaPredictor, JavaPredictionModel, JavaWrapper
from pyspark.ml.common import inherit_doc from pyspark.ml.common import inherit_doc
from pyspark.sql import DataFrame from pyspark.sql import DataFrame

View file

@ -19,7 +19,6 @@ import sys
from pyspark import since, SparkContext from pyspark import since, SparkContext
from pyspark.ml.common import _java2py, _py2java from pyspark.ml.common import _java2py, _py2java
from pyspark.ml.linalg import DenseMatrix, Vectors
from pyspark.ml.wrapper import JavaWrapper, _jvm from pyspark.ml.wrapper import JavaWrapper, _jvm
from pyspark.sql.column import Column, _to_seq from pyspark.sql.column import Column, _to_seq
from pyspark.sql.functions import lit from pyspark.sql.functions import lit
@ -121,7 +120,7 @@ class Correlation(object):
DataFrame contains a single row and a single column of name DataFrame contains a single row and a single column of name
'$METHODNAME($COLUMN)'. '$METHODNAME($COLUMN)'.
>>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.linalg import DenseMatrix, Vectors
>>> from pyspark.ml.stat import Correlation >>> from pyspark.ml.stat import Correlation
>>> dataset = [[Vectors.dense([1, 0, 0, -2])], >>> dataset = [[Vectors.dense([1, 0, 0, -2])],
... [Vectors.dense([4, 5, 0, 3])], ... [Vectors.dense([4, 5, 0, 3])],
@ -412,6 +411,7 @@ class SummaryBuilder(JavaWrapper):
class MultivariateGaussian(object): class MultivariateGaussian(object):
"""Represents a (mean, cov) tuple """Represents a (mean, cov) tuple
>>> from pyspark.ml.linalg import DenseMatrix, Vectors
>>> m = MultivariateGaussian(Vectors.dense([11,12]), DenseMatrix(2, 2, (1.0, 3.0, 5.0, 2.0))) >>> m = MultivariateGaussian(Vectors.dense([11,12]), DenseMatrix(2, 2, (1.0, 3.0, 5.0, 2.0)))
>>> (m.mean, m.cov.toArray()) >>> (m.mean, m.cov.toArray())
(DenseVector([11.0, 12.0]), array([[ 1., 5.], (DenseVector([11.0, 12.0]), array([[ 1., 5.],

View file

@ -330,7 +330,7 @@ class LinearRegressionTest(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_algorithms import * from pyspark.ml.tests.test_algorithms import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -67,7 +67,7 @@ class EstimatorTest(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_base import * from pyspark.ml.tests.test_base import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -53,7 +53,7 @@ class EvaluatorTests(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_evaluation import * from pyspark.ml.tests.test_evaluation import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,7 +16,6 @@
# limitations under the License. # limitations under the License.
# #
import sys
import unittest import unittest
from pyspark.ml.feature import Binarizer, CountVectorizer, CountVectorizerModel, HashingTF, IDF, \ from pyspark.ml.feature import Binarizer, CountVectorizer, CountVectorizerModel, HashingTF, IDF, \
@ -301,7 +300,7 @@ class HashingTFTest(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_feature import * from pyspark.ml.tests.test_feature import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,10 +16,8 @@
# #
import unittest import unittest
import py4j
from pyspark.ml.image import ImageSchema from pyspark.ml.image import ImageSchema
from pyspark.testing.mlutils import PySparkTestCase, SparkSessionTestCase from pyspark.testing.mlutils import SparkSessionTestCase
from pyspark.sql import Row from pyspark.sql import Row
from pyspark.testing.utils import QuietTest from pyspark.testing.utils import QuietTest
@ -68,7 +66,7 @@ class ImageFileFormatTest(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_image import * from pyspark.ml.tests.test_image import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -378,7 +378,7 @@ class MatrixUDTTests(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_linalg import * from pyspark.ml.tests.test_linalg import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -17,7 +17,6 @@
# #
import inspect import inspect
import sys
import array as pyarray import array as pyarray
import unittest import unittest
@ -370,7 +369,7 @@ class DefaultValuesTests(PySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_param import * from pyspark.ml.tests.test_param import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -453,7 +453,7 @@ class PersistenceTest(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_persistence import * from pyspark.ml.tests.test_persistence import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -59,7 +59,7 @@ class PipelineTests(PySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_pipeline import * from pyspark.ml.tests.test_pipeline import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -40,7 +40,7 @@ class ChiSquareTestTests(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_stat import * from pyspark.ml.tests.test_stat import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import sys
import unittest import unittest
from pyspark.ml.classification import BinaryLogisticRegressionSummary, \ from pyspark.ml.classification import BinaryLogisticRegressionSummary, \
@ -443,7 +442,7 @@ class TrainingSummaryTest(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_training_summary import * from pyspark.ml.tests.test_training_summary import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -424,8 +424,6 @@ class CrossValidatorTests(SparkSessionTestCase):
self.assertEqual(loadedCV.getFoldCol(), cv_with_user_folds.getFoldCol()) self.assertEqual(loadedCV.getFoldCol(), cv_with_user_folds.getFoldCol())
def test_invalid_user_specified_folds(self): def test_invalid_user_specified_folds(self):
from pyspark.sql import functions as F
dataset_with_folds = self.spark.createDataFrame( dataset_with_folds = self.spark.createDataFrame(
[(Vectors.dense([0.0]), 0.0, 0), [(Vectors.dense([0.0]), 0.0, 0),
(Vectors.dense([0.4]), 1.0, 1), (Vectors.dense([0.4]), 1.0, 1),
@ -754,7 +752,7 @@ class TrainValidationSplitTests(SparkSessionTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_tuning import * from pyspark.ml.tests.test_tuning import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -117,7 +117,7 @@ class WrapperTests(MLlibTestCase):
self.assertEqual(_java2py(self.sc, java_array), expected_str_list) self.assertEqual(_java2py(self.sc, java_array), expected_str_list)
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.ml.tests.test_wrapper import * from pyspark.ml.tests.test_wrapper import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,7 +16,6 @@
# #
import json import json
import sys
import os import os
import time import time
import uuid import uuid

View file

@ -23,7 +23,7 @@ import numpy
from pyspark import RDD, since from pyspark import RDD, since
from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py
from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.linalg import _convert_to_vector
from pyspark.mllib.regression import ( from pyspark.mllib.regression import (
LabeledPoint, LinearModel, _regression_train_wrapper, LabeledPoint, LinearModel, _regression_train_wrapper,
StreamingLinearAlgorithm) StreamingLinearAlgorithm)
@ -102,6 +102,7 @@ class LogisticRegressionModel(LinearClassificationModel):
in Multinomial Logistic Regression. By default, it is binary in Multinomial Logistic Regression. By default, it is binary
logistic regression so numClasses will be set to 2. logistic regression so numClasses will be set to 2.
>>> from pyspark.mllib.linalg import SparseVector
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(0.0, [0.0, 1.0]),
... LabeledPoint(1.0, [1.0, 0.0]), ... LabeledPoint(1.0, [1.0, 0.0]),
@ -410,6 +411,7 @@ class SVMModel(LinearClassificationModel):
:param intercept: :param intercept:
Intercept computed for this model. Intercept computed for this model.
>>> from pyspark.mllib.linalg import SparseVector
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0]), ... LabeledPoint(0.0, [0.0]),
... LabeledPoint(1.0, [1.0]), ... LabeledPoint(1.0, [1.0]),
@ -569,6 +571,7 @@ class NaiveBayesModel(Saveable, Loader):
Log of class conditional probabilities, whose dimension is C-by-D, Log of class conditional probabilities, whose dimension is C-by-D,
where D is number of features. where D is number of features.
>>> from pyspark.mllib.linalg import SparseVector
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0, 0.0]), ... LabeledPoint(0.0, [0.0, 0.0]),
... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(0.0, [0.0, 1.0]),

View file

@ -25,7 +25,7 @@ from numpy import array, random, tile
from pyspark import SparkContext, since from pyspark import SparkContext, since
from pyspark.rdd import RDD from pyspark.rdd import RDD
from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, callJavaFunc, _py2java, _java2py from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, callJavaFunc, _py2java, _java2py
from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector # noqa: F401
from pyspark.mllib.stat.distribution import MultivariateGaussian from pyspark.mllib.stat.distribution import MultivariateGaussian
from pyspark.mllib.util import Saveable, Loader, inherit_doc, JavaLoader, JavaSaveable from pyspark.mllib.util import Saveable, Loader, inherit_doc, JavaLoader, JavaSaveable
from pyspark.streaming import DStream from pyspark.streaming import DStream

View file

@ -25,9 +25,7 @@ from py4j.protocol import Py4JJavaError
from pyspark import since from pyspark import since
from pyspark.rdd import RDD from pyspark.rdd import RDD
from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper
from pyspark.mllib.linalg import ( from pyspark.mllib.linalg import Vectors, _convert_to_vector
Vectors, DenseVector, SparseVector, _convert_to_vector)
from pyspark.mllib.regression import LabeledPoint
from pyspark.mllib.util import JavaLoader, JavaSaveable from pyspark.mllib.util import JavaLoader, JavaSaveable
__all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler',
@ -60,6 +58,7 @@ class Normalizer(VectorTransformer):
:param p: Normalization in L^p^ space, p = 2 by default. :param p: Normalization in L^p^ space, p = 2 by default.
>>> from pyspark.mllib.linalg import Vectors
>>> v = Vectors.dense(range(3)) >>> v = Vectors.dense(range(3))
>>> nor = Normalizer(1) >>> nor = Normalizer(1)
>>> nor.transform(v) >>> nor.transform(v)
@ -285,6 +284,8 @@ class ChiSqSelector(object):
By default, the selection method is `numTopFeatures`, with the default number of top features By default, the selection method is `numTopFeatures`, with the default number of top features
set to 50. set to 50.
>>> from pyspark.mllib.linalg import SparseVector, DenseVector
>>> from pyspark.mllib.regression import LabeledPoint
>>> data = sc.parallelize([ >>> data = sc.parallelize([
... LabeledPoint(0.0, SparseVector(3, {0: 8.0, 1: 7.0})), ... LabeledPoint(0.0, SparseVector(3, {0: 8.0, 1: 7.0})),
... LabeledPoint(1.0, SparseVector(3, {1: 9.0, 2: 6.0})), ... LabeledPoint(1.0, SparseVector(3, {1: 9.0, 2: 6.0})),

View file

@ -23,7 +23,7 @@ import numpy as np
from pyspark import RDD, since from pyspark import RDD, since
from pyspark.streaming.dstream import DStream from pyspark.streaming.dstream import DStream
from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc
from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.linalg import _convert_to_vector
from pyspark.mllib.util import Saveable, Loader from pyspark.mllib.util import Saveable, Loader
__all__ = ['LabeledPoint', 'LinearModel', __all__ = ['LabeledPoint', 'LinearModel',
@ -102,6 +102,7 @@ class LinearRegressionModelBase(LinearModel):
"""A linear regression model. """A linear regression model.
>>> from pyspark.mllib.linalg import SparseVector
>>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1) >>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1)
>>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6 >>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6
True True
@ -128,6 +129,7 @@ class LinearRegressionModel(LinearRegressionModelBase):
"""A linear regression model derived from a least-squares fit. """A linear regression model derived from a least-squares fit.
>>> from pyspark.mllib.linalg import SparseVector
>>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.regression import LabeledPoint
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0]), ... LabeledPoint(0.0, [0.0]),
@ -297,6 +299,7 @@ class LassoModel(LinearRegressionModelBase):
"""A linear regression model derived from a least-squares fit with """A linear regression model derived from a least-squares fit with
an l_1 penalty term. an l_1 penalty term.
>>> from pyspark.mllib.linalg import SparseVector
>>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.regression import LabeledPoint
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0]), ... LabeledPoint(0.0, [0.0]),
@ -441,6 +444,7 @@ class RidgeRegressionModel(LinearRegressionModelBase):
"""A linear regression model derived from a least-squares fit with """A linear regression model derived from a least-squares fit with
an l_2 penalty term. an l_2 penalty term.
>>> from pyspark.mllib.linalg import SparseVector
>>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.regression import LabeledPoint
>>> data = [ >>> data = [
... LabeledPoint(0.0, [0.0]), ... LabeledPoint(0.0, [0.0]),

View file

@ -292,7 +292,7 @@ class FPGrowthTest(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_algorithms import * from pyspark.mllib.tests.test_algorithms import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -18,9 +18,9 @@
from math import sqrt from math import sqrt
import unittest import unittest
from numpy import array, random, exp, abs, tile from numpy import array, abs, tile
from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, Vectors from pyspark.mllib.linalg import SparseVector, DenseVector, Vectors
from pyspark.mllib.linalg.distributed import RowMatrix from pyspark.mllib.linalg.distributed import RowMatrix
from pyspark.mllib.feature import HashingTF, IDF, StandardScaler, ElementwiseProduct, Word2Vec from pyspark.mllib.feature import HashingTF, IDF, StandardScaler, ElementwiseProduct, Word2Vec
from pyspark.testing.mllibutils import MLlibTestCase from pyspark.testing.mllibutils import MLlibTestCase
@ -182,7 +182,7 @@ class DimensionalityReductionTests(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_feature import * from pyspark.mllib.tests.test_feature import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import sys
import array as pyarray import array as pyarray
import unittest import unittest
@ -639,7 +638,7 @@ class SciPyTests(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_linalg import * from pyspark.mllib.tests.test_linalg import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -20,8 +20,7 @@ import unittest
from numpy import array from numpy import array
from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector, \ from pyspark.mllib.linalg import Vectors, Matrices
DenseMatrix, SparseMatrix, Vectors, Matrices, MatrixUDT
from pyspark.mllib.random import RandomRDDs from pyspark.mllib.random import RandomRDDs
from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.regression import LabeledPoint
from pyspark.mllib.stat import Statistics from pyspark.mllib.stat import Statistics
@ -178,7 +177,7 @@ class KolmogorovSmirnovTest(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_stat import * from pyspark.mllib.tests.test_stat import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
from time import time, sleep
import unittest import unittest
from numpy import array, random, exp, dot, all, mean, abs from numpy import array, random, exp, dot, all, mean, abs
@ -467,7 +466,7 @@ class StreamingLinearRegressionWithTests(MLLibStreamingTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_streaming_algorithms import * from pyspark.mllib.tests.test_streaming_algorithms import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -94,7 +94,7 @@ class SerDeTest(MLlibTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.mllib.tests.test_util import * from pyspark.mllib.tests.test_util import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -18,7 +18,6 @@
import copy import copy
import sys import sys
import os import os
import re
import operator import operator
import shlex import shlex
import warnings import warnings
@ -26,7 +25,6 @@ import heapq
import bisect import bisect
import random import random
from subprocess import Popen, PIPE from subprocess import Popen, PIPE
from tempfile import NamedTemporaryFile
from threading import Thread from threading import Thread
from collections import defaultdict from collections import defaultdict
from itertools import chain from itertools import chain
@ -1566,6 +1564,7 @@ class RDD(object):
used is :class:`pyspark.serializers.PickleSerializer`, default batch size used is :class:`pyspark.serializers.PickleSerializer`, default batch size
is 10. is 10.
>>> from tempfile import NamedTemporaryFile
>>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile = NamedTemporaryFile(delete=True)
>>> tmpFile.close() >>> tmpFile.close()
>>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3)
@ -1586,6 +1585,7 @@ class RDD(object):
:param compressionCodecClass: (None by default) string i.e. :param compressionCodecClass: (None by default) string i.e.
"org.apache.hadoop.io.compress.GzipCodec" "org.apache.hadoop.io.compress.GzipCodec"
>>> from tempfile import NamedTemporaryFile
>>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile = NamedTemporaryFile(delete=True)
>>> tempFile.close() >>> tempFile.close()
>>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name) >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name)
@ -1596,6 +1596,7 @@ class RDD(object):
Empty lines are tolerated when saving to text files. Empty lines are tolerated when saving to text files.
>>> from tempfile import NamedTemporaryFile
>>> tempFile2 = NamedTemporaryFile(delete=True) >>> tempFile2 = NamedTemporaryFile(delete=True)
>>> tempFile2.close() >>> tempFile2.close()
>>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name) >>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name)
@ -1604,6 +1605,7 @@ class RDD(object):
Using compressionCodecClass Using compressionCodecClass
>>> from tempfile import NamedTemporaryFile
>>> tempFile3 = NamedTemporaryFile(delete=True) >>> tempFile3 = NamedTemporaryFile(delete=True)
>>> tempFile3.close() >>> tempFile3.close()
>>> codec = "org.apache.hadoop.io.compress.GzipCodec" >>> codec = "org.apache.hadoop.io.compress.GzipCodec"

View file

@ -14,10 +14,9 @@
# See the License for the specific language governing permissions and # See the License for the specific language governing permissions and
# limitations under the License. # limitations under the License.
# #
import random
import unittest import unittest
from pyspark.resource import ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder,\ from pyspark.resource import ExecutorResourceRequests, ResourceProfileBuilder,\
TaskResourceRequests TaskResourceRequests
@ -73,7 +72,7 @@ class ResourceProfileTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.resource.tests.test_resources import * from pyspark.resource.tests.test_resources import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -122,7 +122,7 @@ def _test():
os.environ["PYSPARK_SUBMIT_ARGS"] = " ".join([jars_args, existing_args]) os.environ["PYSPARK_SUBMIT_ARGS"] = " ".join([jars_args, existing_args])
import doctest import doctest
from pyspark.sql import Row, SparkSession from pyspark.sql import SparkSession
import pyspark.sql.avro.functions import pyspark.sql.avro.functions
globs = pyspark.sql.avro.functions.__dict__.copy() globs = pyspark.sql.avro.functions.__dict__.copy()
spark = SparkSession.builder\ spark = SparkSession.builder\

View file

@ -23,8 +23,7 @@ from pyspark.sql.session import _monkey_patch_RDD, SparkSession
from pyspark.sql.dataframe import DataFrame from pyspark.sql.dataframe import DataFrame
from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.readwriter import DataFrameReader
from pyspark.sql.streaming import DataStreamReader from pyspark.sql.streaming import DataStreamReader
from pyspark.sql.types import IntegerType, Row, StringType from pyspark.sql.udf import UDFRegistration # noqa: F401
from pyspark.sql.udf import UDFRegistration
from pyspark.sql.utils import install_exception_handler from pyspark.sql.utils import install_exception_handler
__all__ = ["SQLContext", "HiveContext"] __all__ = ["SQLContext", "HiveContext"]
@ -53,6 +52,7 @@ class SQLContext(object):
.. note:: Deprecated in 3.0.0. Use :func:`SparkSession.builder.getOrCreate()` instead. .. note:: Deprecated in 3.0.0. Use :func:`SparkSession.builder.getOrCreate()` instead.
>>> from datetime import datetime >>> from datetime import datetime
>>> from pyspark.sql import Row
>>> sqlContext = SQLContext(sc) >>> sqlContext = SQLContext(sc)
>>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1,
... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1),

View file

@ -29,9 +29,10 @@ from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_
from pyspark.sql.dataframe import DataFrame from pyspark.sql.dataframe import DataFrame
from pyspark.sql.types import StringType, DataType from pyspark.sql.types import StringType, DataType
# Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409 # Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409
from pyspark.sql.udf import UserDefinedFunction, _create_udf from pyspark.sql.udf import UserDefinedFunction, _create_udf # noqa: F401
from pyspark.sql.udf import _create_udf
# Keep pandas_udf and PandasUDFType import for backwards compatible import; moved in SPARK-28264 # Keep pandas_udf and PandasUDFType import for backwards compatible import; moved in SPARK-28264
from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType # noqa: F401
from pyspark.sql.utils import to_str from pyspark.sql.utils import to_str
# Note to developers: all of PySpark functions here take string as column names whenever possible. # Note to developers: all of PySpark functions here take string as column names whenever possible.

View file

@ -16,7 +16,6 @@
# #
import functools import functools
import sys
import warnings import warnings
from inspect import getfullargspec from inspect import getfullargspec

View file

@ -1246,7 +1246,7 @@ def _test():
import tempfile import tempfile
import py4j import py4j
from pyspark.context import SparkContext from pyspark.context import SparkContext
from pyspark.sql import SparkSession, Row from pyspark.sql import SparkSession
import pyspark.sql.readwriter import pyspark.sql.readwriter
os.chdir(os.environ["SPARK_HOME"]) os.chdir(os.environ["SPARK_HOME"])

View file

@ -27,7 +27,7 @@ from pyspark.sql.dataframe import DataFrame
from pyspark.sql.pandas.conversion import SparkConversionMixin from pyspark.sql.pandas.conversion import SparkConversionMixin
from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.readwriter import DataFrameReader
from pyspark.sql.streaming import DataStreamReader from pyspark.sql.streaming import DataStreamReader
from pyspark.sql.types import Row, DataType, StringType, StructType, \ from pyspark.sql.types import DataType, StructType, \
_make_type_verifier, _infer_schema, _has_nulltype, _merge_type, _create_converter, \ _make_type_verifier, _infer_schema, _has_nulltype, _merge_type, _create_converter, \
_parse_datatype_string _parse_datatype_string
from pyspark.sql.utils import install_exception_handler from pyspark.sql.utils import install_exception_handler
@ -192,6 +192,7 @@ class SparkSession(SparkConversionMixin):
"""Creates a new SparkSession. """Creates a new SparkSession.
>>> from datetime import datetime >>> from datetime import datetime
>>> from pyspark.sql import Row
>>> spark = SparkSession(sc) >>> spark = SparkSession(sc)
>>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1,
... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1),

View file

@ -1231,7 +1231,7 @@ def _test():
import doctest import doctest
import os import os
import tempfile import tempfile
from pyspark.sql import Row, SparkSession, SQLContext from pyspark.sql import SparkSession, SQLContext
import pyspark.sql.streaming import pyspark.sql.streaming
os.chdir(os.environ["SPARK_HOME"]) os.chdir(os.environ["SPARK_HOME"])

View file

@ -35,7 +35,7 @@ if have_pandas:
from pandas.util.testing import assert_frame_equal from pandas.util.testing import assert_frame_equal
if have_pyarrow: if have_pyarrow:
import pyarrow as pa import pyarrow as pa # noqa: F401
@unittest.skipIf( @unittest.skipIf(

View file

@ -190,7 +190,7 @@ class CatalogTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_catalog import * from pyspark.sql.tests.test_catalog import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -142,7 +142,7 @@ class ColumnTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_column import * from pyspark.sql.tests.test_column import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -46,7 +46,7 @@ class ConfTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_conf import * from pyspark.sql.tests.test_conf import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -273,7 +273,7 @@ class SQLContextTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_context import * from pyspark.sql.tests.test_context import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -379,8 +379,6 @@ class DataFrameTests(ReusedSQLTestCase):
# add tests for SPARK-23647 (test more types for hint) # add tests for SPARK-23647 (test more types for hint)
def test_extended_hint_types(self): def test_extended_hint_types(self):
from pyspark.sql import DataFrame
df = self.spark.range(10e10).toDF("id") df = self.spark.range(10e10).toDF("id")
such_a_nice_list = ["itworks1", "itworks2", "itworks3"] such_a_nice_list = ["itworks1", "itworks2", "itworks3"]
hinted_df = df.hint("my awesome hint", 1.2345, "what", such_a_nice_list) hinted_df = df.hint("my awesome hint", 1.2345, "what", such_a_nice_list)

View file

@ -161,7 +161,7 @@ class DataSourcesTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_datasources import * from pyspark.sql.tests.test_datasources import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,7 +16,6 @@
# #
import datetime import datetime
import sys
from itertools import chain from itertools import chain
import re import re
@ -369,7 +368,7 @@ class FunctionsTests(ReusedSQLTestCase):
self.assertListEqual(actual, expected) self.assertListEqual(actual, expected)
def test_higher_order_function_failures(self): def test_higher_order_function_failures(self):
from pyspark.sql.functions import col, exists, transform from pyspark.sql.functions import col, transform
# Should fail with varargs # Should fail with varargs
with self.assertRaises(ValueError): with self.assertRaises(ValueError):
@ -394,7 +393,7 @@ class FunctionsTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_functions import * from pyspark.sql.tests.test_functions import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -36,7 +36,7 @@ class GroupTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_group import * from pyspark.sql.tests.test_group import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,9 +16,8 @@
# #
import unittest import unittest
import sys
from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType from pyspark.sql.functions import array, explode, col, lit, udf, pandas_udf
from pyspark.sql.types import DoubleType, StructType, StructField, Row from pyspark.sql.types import DoubleType, StructType, StructField, Row
from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \
pandas_requirement_message, pyarrow_requirement_message pandas_requirement_message, pyarrow_requirement_message
@ -26,10 +25,10 @@ from pyspark.testing.utils import QuietTest
if have_pandas: if have_pandas:
import pandas as pd import pandas as pd
from pandas.util.testing import assert_frame_equal, assert_series_equal from pandas.util.testing import assert_frame_equal
if have_pyarrow: if have_pyarrow:
import pyarrow as pa import pyarrow as pa # noqa: F401
@unittest.skipIf( @unittest.skipIf(
@ -245,7 +244,7 @@ class CogroupedMapInPandasTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_cogrouped_map import * from pyspark.sql.tests.test_pandas_cogrouped_map import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -17,7 +17,6 @@
import datetime import datetime
import unittest import unittest
import sys
from collections import OrderedDict from collections import OrderedDict
from decimal import Decimal from decimal import Decimal
@ -35,7 +34,7 @@ if have_pandas:
from pandas.util.testing import assert_frame_equal from pandas.util.testing import assert_frame_equal
if have_pyarrow: if have_pyarrow:
import pyarrow as pa import pyarrow as pa # noqa: F401
@unittest.skipIf( @unittest.skipIf(

View file

@ -15,11 +15,9 @@
# limitations under the License. # limitations under the License.
# #
import os import os
import sys
import time import time
import unittest import unittest
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \
pandas_requirement_message, pyarrow_requirement_message pandas_requirement_message, pyarrow_requirement_message
@ -116,7 +114,7 @@ class MapInPandasTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_map import * from pyspark.sql.tests.test_pandas_map import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -241,7 +241,7 @@ class PandasUDFTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf import * from pyspark.sql.tests.test_pandas_udf import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -510,7 +510,7 @@ class GroupedAggPandasUDFTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf_grouped_agg import * from pyspark.sql.tests.test_pandas_udf_grouped_agg import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -17,7 +17,6 @@
import os import os
import random import random
import shutil import shutil
import sys
import tempfile import tempfile
import time import time
import unittest import unittest
@ -41,7 +40,7 @@ if have_pandas:
import pandas as pd import pandas as pd
if have_pyarrow: if have_pyarrow:
import pyarrow as pa import pyarrow as pa # noqa: F401
@unittest.skipIf( @unittest.skipIf(

View file

@ -243,7 +243,7 @@ class PandasUDFTypeHintsTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf_typehints import * from pyspark.sql.tests.test_pandas_udf_typehints import * # noqa: #401
try: try:
import xmlrunner import xmlrunner

View file

@ -352,7 +352,7 @@ class WindowPandasUDFTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf_window import * from pyspark.sql.tests.test_pandas_udf_window import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -201,7 +201,7 @@ class ReadwriterV2Tests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_readwriter import * from pyspark.sql.tests.test_readwriter import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -139,7 +139,7 @@ class SerdeTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_serde import * from pyspark.sql.tests.test_serde import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -358,7 +358,7 @@ class SparkExtensionsTest(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_session import * from pyspark.sql.tests.test_session import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -567,7 +567,7 @@ class StreamingTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_streaming import * from pyspark.sql.tests.test_streaming import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -969,7 +969,7 @@ class DataTypeVerificationTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.sql.tests.test_types import * from pyspark.sql.tests.test_types import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -21,8 +21,6 @@ import shutil
import tempfile import tempfile
import unittest import unittest
import py4j
from pyspark import SparkContext from pyspark import SparkContext
from pyspark.sql import SparkSession, Column, Row from pyspark.sql import SparkSession, Column, Row
from pyspark.sql.functions import UserDefinedFunction, udf from pyspark.sql.functions import UserDefinedFunction, udf

View file

@ -52,7 +52,7 @@ class UtilsTests(ReusedSQLTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.sql.tests.test_utils import * from pyspark.sql.tests.test_utils import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import sys
import operator import operator
import time import time
from itertools import chain from itertools import chain

View file

@ -175,7 +175,7 @@ class StreamingContextTests(PySparkStreamingTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.streaming.tests.test_context import * from pyspark.streaming.tests.test_context import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -24,7 +24,7 @@ from functools import reduce
from itertools import chain from itertools import chain
import platform import platform
from pyspark import SparkConf, SparkContext, RDD from pyspark import SparkConf, SparkContext
from pyspark.streaming import StreamingContext from pyspark.streaming import StreamingContext
from pyspark.testing.streamingutils import PySparkStreamingTestCase from pyspark.testing.streamingutils import PySparkStreamingTestCase
@ -644,7 +644,7 @@ class CheckpointTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.streaming.tests.test_dstream import * from pyspark.streaming.tests.test_dstream import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -80,7 +80,7 @@ class KinesisStreamTests(PySparkStreamingTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.streaming.tests.test_kinesis import * from pyspark.streaming.tests.test_kinesis import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -149,7 +149,7 @@ class StreamingListenerTests(PySparkStreamingTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.streaming.tests.test_listener import * from pyspark.streaming.tests.test_listener import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -27,13 +27,13 @@ from pyspark import SparkContext, SparkConf
have_scipy = False have_scipy = False
have_numpy = False have_numpy = False
try: try:
import scipy.sparse import scipy.sparse # noqa: F401
have_scipy = True have_scipy = True
except: except:
# No SciPy, but that's okay, we'll skip those tests # No SciPy, but that's okay, we'll skip those tests
pass pass
try: try:
import numpy as np import numpy as np # noqa: F401
have_numpy = True have_numpy = True
except: except:
# No NumPy, but that's okay, we'll skip those tests # No NumPy, but that's okay, we'll skip those tests

View file

@ -238,7 +238,7 @@ class SparkSubmitTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_appsubmit import * from pyspark.tests.test_appsubmit import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -145,7 +145,7 @@ class BroadcastFrameProtocolTest(unittest.TestCase):
if __name__ == '__main__': if __name__ == '__main__':
from pyspark.tests.test_broadcast import * from pyspark.tests.test_broadcast import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -33,7 +33,7 @@ class ConfTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_conf import * from pyspark.tests.test_conf import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -126,7 +126,7 @@ class AddFileTests(PySparkTestCase):
# To ensure that we're actually testing addPyFile's effects, check that # To ensure that we're actually testing addPyFile's effects, check that
# this fails due to `userlibrary` not being on the Python path: # this fails due to `userlibrary` not being on the Python path:
def func(): def func():
from userlibrary import UserClass from userlibrary import UserClass # noqa: F401
self.assertRaises(ImportError, func) self.assertRaises(ImportError, func)
path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
self.sc.addPyFile(path) self.sc.addPyFile(path)
@ -137,7 +137,7 @@ class AddFileTests(PySparkTestCase):
# To ensure that we're actually testing addPyFile's effects, check that # To ensure that we're actually testing addPyFile's effects, check that
# this fails due to `userlibrary` not being on the Python path: # this fails due to `userlibrary` not being on the Python path:
def func(): def func():
from userlib import UserClass from userlib import UserClass # noqa: F401
self.assertRaises(ImportError, func) self.assertRaises(ImportError, func)
path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip") path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip")
self.sc.addPyFile(path) self.sc.addPyFile(path)
@ -318,7 +318,7 @@ class ContextTestsWithResources(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_context import * from pyspark.tests.test_context import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -73,7 +73,7 @@ class DaemonTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_daemon import * from pyspark.tests.test_daemon import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -59,7 +59,7 @@ class JoinTests(ReusedPySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_join import * from pyspark.tests.test_join import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,7 +16,6 @@
# #
import os import os
import time import time
import random
import threading import threading
import unittest import unittest
@ -167,7 +166,7 @@ class PinThreadTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_pin_thread import * from pyspark.tests.test_pin_thread import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -98,7 +98,7 @@ class ProfilerTests2(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_profiler import * from pyspark.tests.test_profiler import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -18,7 +18,6 @@ from datetime import datetime, timedelta
import hashlib import hashlib
import os import os
import random import random
import sys
import tempfile import tempfile
import time import time
from glob import glob from glob import glob
@ -26,7 +25,7 @@ from glob import glob
from py4j.protocol import Py4JJavaError from py4j.protocol import Py4JJavaError
from pyspark import shuffle, RDD from pyspark import shuffle, RDD
from pyspark.resource import ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder,\ from pyspark.resource import ExecutorResourceRequests, ResourceProfileBuilder,\
TaskResourceRequests TaskResourceRequests
from pyspark.serializers import CloudPickleSerializer, BatchedSerializer, PickleSerializer,\ from pyspark.serializers import CloudPickleSerializer, BatchedSerializer, PickleSerializer,\
MarshalSerializer, UTF8Deserializer, NoOpSerializer MarshalSerializer, UTF8Deserializer, NoOpSerializer
@ -882,7 +881,7 @@ class RDDTests(ReusedPySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_rdd import * from pyspark.tests.test_rdd import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -40,7 +40,7 @@ class RDDBarrierTests(ReusedPySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_rddbarrier import * from pyspark.tests.test_rddbarrier import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,10 +16,8 @@
# #
import os import os
import shutil import shutil
import sys
import tempfile import tempfile
import unittest import unittest
from array import array
from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME
@ -306,7 +304,7 @@ class OutputFormatTests(ReusedPySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_readwrite import * from pyspark.tests.test_readwrite import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -87,7 +87,7 @@ class SerializationTestCase(unittest.TestCase):
def test_pickling_file_handles(self): def test_pickling_file_handles(self):
# to be corrected with SPARK-11160 # to be corrected with SPARK-11160
try: try:
import xmlrunner import xmlrunner # noqa: F401
except ImportError: except ImportError:
ser = CloudPickleSerializer() ser = CloudPickleSerializer()
out1 = sys.stderr out1 = sys.stderr

View file

@ -15,7 +15,6 @@
# limitations under the License. # limitations under the License.
# #
import random import random
import sys
import unittest import unittest
from py4j.protocol import Py4JJavaError from py4j.protocol import Py4JJavaError
@ -168,7 +167,7 @@ class SorterTests(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_shuffle import * from pyspark.tests.test_shuffle import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,7 +16,6 @@
# #
import os import os
import random import random
import shutil
import stat import stat
import sys import sys
import tempfile import tempfile
@ -322,7 +321,7 @@ class TaskContextTestsWithResources(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_taskcontext import * from pyspark.tests.test_taskcontext import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -74,7 +74,7 @@ class UtilTests(PySparkTestCase):
if __name__ == "__main__": if __name__ == "__main__":
from pyspark.tests.test_util import * from pyspark.tests.test_util import * # noqa: F401
try: try:
import xmlrunner import xmlrunner

View file

@ -16,14 +16,13 @@
# limitations under the License. # limitations under the License.
# #
import os import os
import sys
import tempfile import tempfile
import threading import threading
import time import time
import unittest import unittest
has_resource_module = True has_resource_module = True
try: try:
import resource import resource # noqa: F401
except ImportError: except ImportError:
has_resource_module = False has_resource_module = False
@ -200,7 +199,7 @@ class WorkerMemoryTest(unittest.TestCase):
if __name__ == "__main__": if __name__ == "__main__":
import unittest import unittest
from pyspark.tests.test_worker import * from pyspark.tests.test_worker import * # noqa: F401
try: try:
import xmlrunner import xmlrunner