[SPARK-7899] [PYSPARK] Fix Python 3 pyspark/sql/types module conflict
This PR makes the types module in `pyspark/sql/types` work with pylint static analysis by removing the dynamic naming of the `pyspark/sql/_types` module to `pyspark/sql/types`.
Tests are now loaded using `$PYSPARK_DRIVER_PYTHON -m module` rather than `$PYSPARK_DRIVER_PYTHON module.py`. The old method adds the location of `module.py` to `sys.path`, so this change prevents accidental use of relative paths in Python.
Author: Michael Nazario <mnazario@palantir.com>
Closes #6439 from mnazario/feature/SPARK-7899 and squashes the following commits:
366ef30 [Michael Nazario] Remove hack on random.py
bb8b04d
[Michael Nazario] Make doctests consistent with other tests
6ee4f75 [Michael Nazario] Change test scripts to use "-m"
673528f [Michael Nazario] Move _types back to types
This commit is contained in:
parent
5f48e5c33b
commit
1c5b19827a
|
@ -90,11 +90,7 @@ if [[ -n "$SPARK_TESTING" ]]; then
|
|||
unset YARN_CONF_DIR
|
||||
unset HADOOP_CONF_DIR
|
||||
export PYTHONHASHSEED=0
|
||||
if [[ -n "$PYSPARK_DOC_TEST" ]]; then
|
||||
exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1
|
||||
else
|
||||
exec "$PYSPARK_DRIVER_PYTHON" $1
|
||||
fi
|
||||
exec "$PYSPARK_DRIVER_PYTHON" -m $1
|
||||
exit
|
||||
fi
|
||||
|
||||
|
|
|
@ -261,3 +261,7 @@ def _start_update_server():
|
|||
thread.daemon = True
|
||||
thread.start()
|
||||
return server
|
||||
|
||||
if __name__ == "__main__":
|
||||
import doctest
|
||||
doctest.testmod()
|
||||
|
|
|
@ -28,11 +28,3 @@ if numpy.version.version < '1.4':
|
|||
|
||||
__all__ = ['classification', 'clustering', 'feature', 'fpm', 'linalg', 'random',
|
||||
'recommendation', 'regression', 'stat', 'tree', 'util']
|
||||
|
||||
import sys
|
||||
from . import rand as random
|
||||
modname = __name__ + '.random'
|
||||
random.__name__ = modname
|
||||
random.RandomRDDs.__module__ = modname
|
||||
sys.modules[modname] = random
|
||||
del modname, sys
|
||||
|
|
|
@ -50,18 +50,6 @@ def since(version):
|
|||
return f
|
||||
return deco
|
||||
|
||||
# fix the module name conflict for Python 3+
|
||||
import sys
|
||||
from . import _types as types
|
||||
modname = __name__ + '.types'
|
||||
types.__name__ = modname
|
||||
# update the __module__ for all objects, make them picklable
|
||||
for v in types.__dict__.values():
|
||||
if hasattr(v, "__module__") and v.__module__.endswith('._types'):
|
||||
v.__module__ = modname
|
||||
sys.modules[modname] = types
|
||||
del modname, sys
|
||||
|
||||
from pyspark.sql.types import Row
|
||||
from pyspark.sql.context import SQLContext, HiveContext
|
||||
from pyspark.sql.column import Column
|
||||
|
|
|
@ -57,54 +57,54 @@ function run_test() {
|
|||
|
||||
function run_core_tests() {
|
||||
echo "Run core tests ..."
|
||||
run_test "pyspark/rdd.py"
|
||||
run_test "pyspark/context.py"
|
||||
run_test "pyspark/conf.py"
|
||||
PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py"
|
||||
PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py"
|
||||
run_test "pyspark/serializers.py"
|
||||
run_test "pyspark/profiler.py"
|
||||
run_test "pyspark/shuffle.py"
|
||||
run_test "pyspark/tests.py"
|
||||
run_test "pyspark.rdd"
|
||||
run_test "pyspark.context"
|
||||
run_test "pyspark.conf"
|
||||
run_test "pyspark.broadcast"
|
||||
run_test "pyspark.accumulators"
|
||||
run_test "pyspark.serializers"
|
||||
run_test "pyspark.profiler"
|
||||
run_test "pyspark.shuffle"
|
||||
run_test "pyspark.tests"
|
||||
}
|
||||
|
||||
function run_sql_tests() {
|
||||
echo "Run sql tests ..."
|
||||
run_test "pyspark/sql/_types.py"
|
||||
run_test "pyspark/sql/context.py"
|
||||
run_test "pyspark/sql/column.py"
|
||||
run_test "pyspark/sql/dataframe.py"
|
||||
run_test "pyspark/sql/group.py"
|
||||
run_test "pyspark/sql/functions.py"
|
||||
run_test "pyspark/sql/tests.py"
|
||||
run_test "pyspark.sql.types"
|
||||
run_test "pyspark.sql.context"
|
||||
run_test "pyspark.sql.column"
|
||||
run_test "pyspark.sql.dataframe"
|
||||
run_test "pyspark.sql.group"
|
||||
run_test "pyspark.sql.functions"
|
||||
run_test "pyspark.sql.tests"
|
||||
}
|
||||
|
||||
function run_mllib_tests() {
|
||||
echo "Run mllib tests ..."
|
||||
run_test "pyspark/mllib/classification.py"
|
||||
run_test "pyspark/mllib/clustering.py"
|
||||
run_test "pyspark/mllib/evaluation.py"
|
||||
run_test "pyspark/mllib/feature.py"
|
||||
run_test "pyspark/mllib/fpm.py"
|
||||
run_test "pyspark/mllib/linalg.py"
|
||||
run_test "pyspark/mllib/rand.py"
|
||||
run_test "pyspark/mllib/recommendation.py"
|
||||
run_test "pyspark/mllib/regression.py"
|
||||
run_test "pyspark/mllib/stat/_statistics.py"
|
||||
run_test "pyspark/mllib/tree.py"
|
||||
run_test "pyspark/mllib/util.py"
|
||||
run_test "pyspark/mllib/tests.py"
|
||||
run_test "pyspark.mllib.classification"
|
||||
run_test "pyspark.mllib.clustering"
|
||||
run_test "pyspark.mllib.evaluation"
|
||||
run_test "pyspark.mllib.feature"
|
||||
run_test "pyspark.mllib.fpm"
|
||||
run_test "pyspark.mllib.linalg"
|
||||
run_test "pyspark.mllib.random"
|
||||
run_test "pyspark.mllib.recommendation"
|
||||
run_test "pyspark.mllib.regression"
|
||||
run_test "pyspark.mllib.stat._statistics"
|
||||
run_test "pyspark.mllib.tree"
|
||||
run_test "pyspark.mllib.util"
|
||||
run_test "pyspark.mllib.tests"
|
||||
}
|
||||
|
||||
function run_ml_tests() {
|
||||
echo "Run ml tests ..."
|
||||
run_test "pyspark/ml/feature.py"
|
||||
run_test "pyspark/ml/classification.py"
|
||||
run_test "pyspark/ml/recommendation.py"
|
||||
run_test "pyspark/ml/regression.py"
|
||||
run_test "pyspark/ml/tuning.py"
|
||||
run_test "pyspark/ml/tests.py"
|
||||
run_test "pyspark/ml/evaluation.py"
|
||||
run_test "pyspark.ml.feature"
|
||||
run_test "pyspark.ml.classification"
|
||||
run_test "pyspark.ml.recommendation"
|
||||
run_test "pyspark.ml.regression"
|
||||
run_test "pyspark.ml.tuning"
|
||||
run_test "pyspark.ml.tests"
|
||||
run_test "pyspark.ml.evaluation"
|
||||
}
|
||||
|
||||
function run_streaming_tests() {
|
||||
|
@ -124,8 +124,8 @@ function run_streaming_tests() {
|
|||
done
|
||||
|
||||
export PYSPARK_SUBMIT_ARGS="--jars ${KAFKA_ASSEMBLY_JAR} pyspark-shell"
|
||||
run_test "pyspark/streaming/util.py"
|
||||
run_test "pyspark/streaming/tests.py"
|
||||
run_test "pyspark.streaming.util"
|
||||
run_test "pyspark.streaming.tests"
|
||||
}
|
||||
|
||||
echo "Running PySpark tests. Output is in python/$LOG_FILE."
|
||||
|
|
Loading…
Reference in a new issue